diff --git a/430/_images/functions_color_bar.png b/430/_images/functions_color_bar.png new file mode 100644 index 000000000..b01f4e0dc Binary files /dev/null and b/430/_images/functions_color_bar.png differ diff --git a/430/_sources/admin.md.txt b/430/_sources/admin.md.txt new file mode 100644 index 000000000..ba37ea2f6 --- /dev/null +++ b/430/_sources/admin.md.txt @@ -0,0 +1,26 @@ +# Administration + +```{toctree} +:maxdepth: 1 + +admin/web-interface +admin/tuning +admin/jmx +admin/properties +admin/spill +admin/resource-groups +admin/session-property-managers +admin/dist-sort +admin/dynamic-filtering +admin/graceful-shutdown +admin/fault-tolerant-execution +``` + +## Event listeners + +```{toctree} +:titlesonly: true + +admin/event-listeners-http +admin/event-listeners-mysql +``` diff --git a/430/_sources/admin/dist-sort.md.txt b/430/_sources/admin/dist-sort.md.txt new file mode 100644 index 000000000..44adf32b1 --- /dev/null +++ b/430/_sources/admin/dist-sort.md.txt @@ -0,0 +1,15 @@ +# Distributed sort + +Distributed sort allows to sort data, which exceeds `query.max-memory-per-node`. +Distributed sort is enabled via the `distributed_sort` session property, or +`distributed-sort` configuration property set in +`etc/config.properties` of the coordinator. Distributed sort is enabled by +default. + +When distributed sort is enabled, the sort operator executes in parallel on multiple +nodes in the cluster. Partially sorted data from each Trino worker node is then streamed +to a single worker node for a final merge. This technique allows to utilize memory of multiple +Trino worker nodes for sorting. The primary purpose of distributed sort is to allow for sorting +of data sets which don't normally fit into single node memory. Performance improvement +can be expected, but it won't scale linearly with the number of nodes, since the +data needs to be merged by a single node. diff --git a/430/_sources/admin/dynamic-filtering.md.txt b/430/_sources/admin/dynamic-filtering.md.txt new file mode 100644 index 000000000..f6ae597e0 --- /dev/null +++ b/430/_sources/admin/dynamic-filtering.md.txt @@ -0,0 +1,264 @@ +# Dynamic filtering + +Dynamic filtering optimizations significantly improve the performance of queries +with selective joins by avoiding reading of data that would be filtered by join condition. + +Consider the following query which captures a common pattern of a fact table `store_sales` +joined with a filtered dimension table `date_dim`: + +> SELECT count(\*) +> FROM store_sales +> JOIN date_dim ON store_sales.ss_sold_date_sk = date_dim.d_date_sk +> WHERE d_following_holiday='Y' AND d_year = 2000; + +Without dynamic filtering, Trino pushes predicates for the dimension table to the +table scan on `date_dim`, and it scans all the data in the fact table since there +are no filters on `store_sales` in the query. The join operator ends up throwing away +most of the probe-side rows as the join criteria is highly selective. + +When dynamic filtering is enabled, Trino collects candidate values for join condition +from the processed dimension table on the right side of join. In the case of broadcast joins, +the runtime predicates generated from this collection are pushed into the local table scan +on the left side of the join running on the same worker. + +Additionally, these runtime predicates are communicated to the coordinator over the network +so that dynamic filtering can also be performed on the coordinator during enumeration of +table scan splits. + +For example, in the case of the Hive connector, dynamic filters are used +to skip loading of partitions which don't match the join criteria. +This is known as **dynamic partition pruning**. + +After completing the collection of dynamic filters, the coordinator also distributes them +to worker nodes over the network for partitioned joins. This allows push down of dynamic +filters from partitioned joins into the table scans on the left side of that join. +Distribution of dynamic filters from the coordinator to workers is enabled by default. +It can be disabled by setting either the `enable-coordinator-dynamic-filters-distribution` +configuration property, or the session property +`enable_coordinator_dynamic_filters_distribution` to `false`. + +The results of dynamic filtering optimization can include the following benefits: + +- improved overall query performance +- reduced network traffic between Trino and the data source +- reduced load on the remote data source + +Dynamic filtering is enabled by default. It can be disabled by setting either the +`enable-dynamic-filtering` configuration property, or the session property +`enable_dynamic_filtering` to `false`. + +Support for push down of dynamic filters is specific to each connector, +and the relevant underlying database or storage system. The documentation for +specific connectors with support for dynamic filtering includes further details, +for example the {ref}`Hive connector ` +or the {ref}`Memory connector `. + +## Analysis and confirmation + +Dynamic filtering depends on a number of factors: + +- Planner support for dynamic filtering for a given join operation in Trino. + Currently inner and right joins with `=`, `<`, `<=`, `>`, `>=` or + `IS NOT DISTINCT FROM` join conditions, and + semi-joins with `IN` conditions are supported. +- Connector support for utilizing dynamic filters pushed into the table scan at runtime. + For example, the Hive connector can push dynamic filters into ORC and Parquet readers + to perform stripe or row-group pruning. +- Connector support for utilizing dynamic filters at the splits enumeration stage. +- Size of right (build) side of the join. + +You can take a closer look at the {doc}`EXPLAIN plan ` of the query +to analyze if the planner is adding dynamic filters to a specific query's plan. +For example, the explain plan for the above query can be obtained by running +the following statement: + +``` +EXPLAIN +SELECT count(*) +FROM store_sales +JOIN date_dim ON store_sales.ss_sold_date_sk = date_dim.d_date_sk +WHERE d_following_holiday='Y' AND d_year = 2000; +``` + +The explain plan for this query shows `dynamicFilterAssignments` in the +`InnerJoin` node with dynamic filter `df_370` collected from build symbol `d_date_sk`. +You can also see the `dynamicFilter` predicate as part of the Hive `ScanFilterProject` +operator where `df_370` is associated with probe symbol `ss_sold_date_sk`. +This shows you that the planner is successful in pushing dynamic filters +down to the connector in the query plan. + +```text +... + +Fragment 1 [SOURCE] + Output layout: [count_3] + Output partitioning: SINGLE [] + Aggregate(PARTIAL) + │ Layout: [count_3:bigint] + │ count_3 := count(*) + └─ InnerJoin[(""ss_sold_date_sk"" = ""d_date_sk"")][$hashvalue, $hashvalue_4] + │ Layout: [] + │ Estimates: {rows: 0 (0B), cpu: 0, memory: 0B, network: 0B} + │ Distribution: REPLICATED + │ dynamicFilterAssignments = {d_date_sk -> #df_370} + ├─ ScanFilterProject[table = hive:default:store_sales, grouped = false, filterPredicate = true, dynamicFilters = {""ss_sold_date_sk"" = #df_370}] + │ Layout: [ss_sold_date_sk:bigint, $hashvalue:bigint] + │ Estimates: {rows: 0 (0B), cpu: 0, memory: 0B, network: 0B}/{rows: 0 (0B), cpu: 0, memory: 0B, network: 0B}/{rows: 0 (0B), cpu: 0, memory: 0B, network: 0B} + │ $hashvalue := combine_hash(bigint '0', COALESCE(""$operator$hash_code""(""ss_sold_date_sk""), 0)) + │ ss_sold_date_sk := ss_sold_date_sk:bigint:REGULAR + └─ LocalExchange[HASH][$hashvalue_4] (""d_date_sk"") + │ Layout: [d_date_sk:bigint, $hashvalue_4:bigint] + │ Estimates: {rows: 0 (0B), cpu: 0, memory: 0B, network: 0B} + └─ RemoteSource[2] + Layout: [d_date_sk:bigint, $hashvalue_5:bigint] + +Fragment 2 [SOURCE] + Output layout: [d_date_sk, $hashvalue_6] + Output partitioning: BROADCAST [] + ScanFilterProject[table = hive:default:date_dim, grouped = false, filterPredicate = ((""d_following_holiday"" = CAST('Y' AS char(1))) AND (""d_year"" = 2000))] + Layout: [d_date_sk:bigint, $hashvalue_6:bigint] + Estimates: {rows: 0 (0B), cpu: 0, memory: 0B, network: 0B}/{rows: 0 (0B), cpu: 0, memory: 0B, network: 0B}/{rows: 0 (0B), cpu: 0, memory: 0B, network: 0B} + $hashvalue_6 := combine_hash(bigint '0', COALESCE(""$operator$hash_code""(""d_date_sk""), 0)) + d_following_holiday := d_following_holiday:char(1):REGULAR + d_date_sk := d_date_sk:bigint:REGULAR + d_year := d_year:int:REGULAR +``` + +During execution of a query with dynamic filters, Trino populates statistics +about dynamic filters in the QueryInfo JSON available through the +{doc}`/admin/web-interface`. +In the `queryStats` section, statistics about dynamic filters collected +by the coordinator can be found in the `dynamicFiltersStats` structure. + +```text +"dynamicFiltersStats" : { + "dynamicFilterDomainStats" : [ { + "dynamicFilterId" : "df_370", + "simplifiedDomain" : "[ SortedRangeSet[type=bigint, ranges=3, {[2451546], ..., [2451905]}] ]", + "collectionDuration" : "2.34s" + } ], + "lazyDynamicFilters" : 1, + "replicatedDynamicFilters" : 1, + "totalDynamicFilters" : 1, + "dynamicFiltersCompleted" : 1 +} +``` + +Push down of dynamic filters into a table scan on the worker nodes can be +verified by looking at the operator statistics for that table scan. +`dynamicFilterSplitsProcessed` records the number of splits +processed after a dynamic filter is pushed down to the table scan. + +```text +"operatorType" : "ScanFilterAndProjectOperator", +"totalDrivers" : 1, +"addInputCalls" : 762, +"addInputWall" : "0.00ns", +"addInputCpu" : "0.00ns", +"physicalInputDataSize" : "0B", +"physicalInputPositions" : 28800991, +"inputPositions" : 28800991, +"dynamicFilterSplitsProcessed" : 1, +``` + +Dynamic filters are reported as a part of the +{doc}`EXPLAIN ANALYZE plan ` in the statistics for +`ScanFilterProject` nodes. + +```text +... + + └─ InnerJoin[("ss_sold_date_sk" = "d_date_sk")][$hashvalue, $hashvalue_4] + │ Layout: [] + │ Estimates: {rows: 11859 (0B), cpu: 8.84M, memory: 3.19kB, network: 3.19kB} + │ CPU: 78.00ms (30.00%), Scheduled: 295.00ms (47.05%), Output: 296 rows (0B) + │ Left (probe) Input avg.: 120527.00 rows, Input std.dev.: 0.00% + │ Right (build) Input avg.: 0.19 rows, Input std.dev.: 208.17% + │ Distribution: REPLICATED + │ dynamicFilterAssignments = {d_date_sk -> #df_370} + ├─ ScanFilterProject[table = hive:default:store_sales, grouped = false, filterPredicate = true, dynamicFilters = {"ss_sold_date_sk" = #df_370}] + │ Layout: [ss_sold_date_sk:bigint, $hashvalue:bigint] + │ Estimates: {rows: 120527 (2.03MB), cpu: 1017.64k, memory: 0B, network: 0B}/{rows: 120527 (2.03MB), cpu: 1.99M, memory: 0B, network: 0B}/{rows: 120527 (2.03MB), cpu: 4.02M, memory: 0B, network: 0B} + │ CPU: 49.00ms (18.85%), Scheduled: 123.00ms (19.62%), Output: 120527 rows (2.07MB) + │ Input avg.: 120527.00 rows, Input std.dev.: 0.00% + │ $hashvalue := combine_hash(bigint '0', COALESCE("$operator$hash_code"("ss_sold_date_sk"), 0)) + │ ss_sold_date_sk := ss_sold_date_sk:bigint:REGULAR + │ Input: 120527 rows (1.03MB), Filtered: 0.00% + │ Dynamic filters: + │ - df_370, [ SortedRangeSet[type=bigint, ranges=3, {[2451546], ..., [2451905]}] ], collection time=2.34s + | +... +``` + +## Dynamic filter collection thresholds + +In order for dynamic filtering to work, the smaller dimension table +needs to be chosen as a join’s build side. The cost-based optimizer can automatically +do this using table statistics provided by connectors. Therefore, it is recommended +to keep {doc}`table statistics ` up to date and rely on the +CBO to correctly choose the smaller table on the build side of join. + +Collection of values of the join key columns from the build side for +dynamic filtering may incur additional CPU overhead during query execution. +Therefore, to limit the overhead of collecting dynamic filters +to the cases where the join operator is likely to be selective, +Trino defines thresholds on the size of dynamic filters collected from build side tasks. +Collection of dynamic filters for joins with large build sides can be enabled +using the `enable-large-dynamic-filters` configuration property or the +`enable_large_dynamic_filters` session property. + +When large dynamic filters are enabled, limits on the size of dynamic filters can +be configured using the configuration properties +`dynamic-filtering.large.max-distinct-values-per-driver`, +`dynamic-filtering.large.max-size-per-driver` , +`dynamic-filtering.large.range-row-limit-per-driver`, +`dynamic-filtering.large-partitioned.max-distinct-values-per-driver`, +`dynamic-filtering.large-partitioned.max-size-per-driver` and +`dynamic-filtering.large-partitioned.range-row-limit-per-driver`. + +Similarly, limits for dynamic filters when `enable-large-dynamic-filters` +is not enabled can be configured using configuration properties like +`dynamic-filtering.small.max-distinct-values-per-driver`, +`dynamic-filtering.small.max-size-per-driver` , +`dynamic-filtering.small.range-row-limit-per-driver`, +`dynamic-filtering.small-partitioned.max-distinct-values-per-driver`, +`dynamic-filtering.small-partitioned.max-size-per-driver` and +`dynamic-filtering.small-partitioned.range-row-limit-per-driver`. + +The `dynamic-filtering.large.*` and `dynamic-filtering.small.*` limits are applied +when dynamic filters are collected before build side is partitioned on join +keys (when broadcast join is chosen or when fault tolerant execution is enabled). The +`dynamic-filtering.large-partitioned.*` and `dynamic-filtering.small-partitioned.*` +limits are applied when dynamic filters are collected after build side is partitioned +on join keys (when partitioned join is chosen and fault tolerant execution is disabled). + +The properties based on `max-distinct-values-per-driver` and `max-size-per-driver` +define thresholds for the size up to which dynamic filters are collected in a +distinct values data structure. When the build side exceeds these thresholds, +Trino switches to collecting min and max values per column to reduce overhead. +This min-max filter has much lower granularity than the distinct values filter. +However, it may still be beneficial in filtering some data from the probe side, +especially when a range of values is selected from the build side of the join. +The limits for min-max filters collection are defined by the properties +based on `range-row-limit-per-driver`. + +## Dimension tables layout + +Dynamic filtering works best for dimension tables where +table keys are correlated with columns. + +For example, a date dimension key column should be correlated with a date column, +so the table keys monotonically increase with date values. +An address dimension key can be composed of other columns such as +`COUNTRY-STATE-ZIP-ADDRESS_ID` with an example value of `US-NY-10001-1234`. +This usage allows dynamic filtering to succeed even with a large number +of selected rows from the dimension table. + +## Limitations + +- Min-max dynamic filter collection is not supported for `DOUBLE`, `REAL` and unorderable data types. +- Dynamic filtering is not supported for `DOUBLE` and `REAL` data types when using `IS NOT DISTINCT FROM` predicate. +- Dynamic filtering is supported when the join key contains a cast from the build key type to the + probe key type. Dynamic filtering is also supported in limited scenarios when there is an implicit + cast from the probe key type to the build key type. For example, dynamic filtering is supported when + the build side key is of `DOUBLE` type and the probe side key is of `REAL` or `INTEGER` type. diff --git a/430/_sources/admin/event-listeners-http.md.txt b/430/_sources/admin/event-listeners-http.md.txt new file mode 100644 index 000000000..6eeea6d51 --- /dev/null +++ b/430/_sources/admin/event-listeners-http.md.txt @@ -0,0 +1,123 @@ +# HTTP event listener + +The HTTP event listener plugin allows streaming of query events, encoded in +JSON format, to an external service for further processing, by POSTing them +to a specified URI. + +## Rationale + +This event listener is a simple first step into better understanding the usage +of a datalake using query events provided by Trino. These can provide CPU and memory +usage metrics, what data is being accessed with resolution down to specific columns, +and metadata about the query processing. + +Running the capture system separate from Trino reduces the performance impact and +avoids downtime for non-client-facing changes. + +(http-event-listener-requirements)= +## Requirements + +You need to perform the following steps: + +- Provide an HTTP/S service that accepts POST events with a JSON body. +- Configure `http-event-listener.connect-ingest-uri` in the event listener properties file + with the URI of the service. +- Detail the events to send in the {ref}`http-event-listener-configuration` section. + +(http-event-listener-configuration)= + +## Configuration + +To configure the HTTP event listener plugin, create an event listener properties +file in `etc` named `http-event-listener.properties` with the following contents +as an example: + +```properties +event-listener.name=http +http-event-listener.log-created=true +http-event-listener.connect-ingest-uri= +``` + +And set add `etc/http-event-listener.properties` to `event-listener.config-files` +in {ref}`config-properties`: + +```properties +event-listener.config-files=etc/http-event-listener.properties,... +``` + +### Configuration properties + +:::{list-table} +:widths: 40, 40, 20 +:header-rows: 1 + +* - Property name + - Description + - Default + +* - http-event-listener.log-created + - Enable the plugin to log `QueryCreatedEvent` events + - `false` + +* - http-event-listener.log-completed + - Enable the plugin to log `QueryCompletedEvent` events + - `false` + +* - http-event-listener.log-split + - Enable the plugin to log `SplitCompletedEvent` events + - `false` + +* - http-event-listener.connect-ingest-uri + - The URI that the plugin will POST events to + - None. See the [requirements](http-event-listener-requirements) section. + +* - http-event-listener.connect-http-headers + - List of custom HTTP headers to be sent along with the events. See + [](http-event-listener-custom-headers) for more details + - Empty + +* - http-event-listener.connect-retry-count + - The number of retries on server error. A server is considered to be + in an error state when the response code is 500 or higher + - `0` + +* - http-event-listener.connect-retry-delay + - Duration for which to delay between attempts to send a request + - `1s` + +* - http-event-listener.connect-backoff-base + - The base used for exponential backoff when retrying on server error. + The formula used to calculate the delay is + `attemptDelay = retryDelay * backoffBase^{attemptCount}`. + Attempt count starts from 0. Leave this empty or set to 1 to disable + exponential backoff and keep constant delays + - `2` + +* - http-event-listener.connect-max-delay + - The upper bound of a delay between 2 retries. This should be + used with exponential backoff. + - `1m` + +* - http-event-listener.* + - Pass configuration onto the HTTP client + - +::: + +(http-event-listener-custom-headers)= + +### Custom HTTP headers + +Providing custom HTTP headers is a useful mechanism for sending metadata along with +event messages. + +Providing headers follows the pattern of `key:value` pairs separated by commas: + +```text +http-event-listener.connect-http-headers="Header-Name-1:header value 1,Header-Value-2:header value 2,..." +``` + +If you need to use a comma(`,`) or colon(`:`) in a header name or value, +escape it using a backslash (`\`). + +Keep in mind that these are static, so they can not carry information +taken from the event itself. diff --git a/430/_sources/admin/event-listeners-mysql.md.txt b/430/_sources/admin/event-listeners-mysql.md.txt new file mode 100644 index 000000000..25a99d732 --- /dev/null +++ b/430/_sources/admin/event-listeners-mysql.md.txt @@ -0,0 +1,71 @@ +# MySQL event listener + +The MySQL event listener plugin allows streaming of query events to an external +MySQL database. The query history in the database can then be accessed directly +in MySQL or via Trino in a catalog using the [MySQL connector](/connector/mysql). + +## Rationale + +This event listener is a first step to store the query history of your Trino +cluster. The query events can provide CPU and memory usage metrics, what data is +being accessed with resolution down to specific columns, and metadata about the +query processing. + +Running the capture system separate from Trino reduces the performance impact +and avoids downtime for non-client-facing changes. + +## Requirements + +You need to perform the following steps: + +- Create a MySQL database. +- Determine the JDBC connection URL for the database. +- Ensure network access from the Trino coordinator to MySQL is available. + Port 3306 is the default port. + +(mysql-event-listener-configuration)= + +## Configuration + +To configure the MySQL event listener plugin, create an event listener properties +file in `etc` named `mysql-event-listener.properties` with the following contents +as an example: + +```properties +event-listener.name=mysql +mysql-event-listener.db.url=jdbc:mysql://example.net:3306 +``` + +The `mysql-event-listener.db.url` defines the connection to a MySQL database +available at the domain `example.net` on port 3306. You can pass further +parameters to the MySQL JDBC driver. The supported parameters for the URL are +documented in the [MySQL Developer +Guide](https://dev.mysql.com/doc/connector-j/8.0/en/connector-j-reference-configuration-properties.html). + +And set `event-listener.config-files` to `etc/mysql-event-listener.properties` +in {ref}`config-properties`: + +```properties +event-listener.config-files=etc/mysql-event-listener.properties +``` + +If another event listener is already configured, add the new value +`etc/mysql-event-listener.properties` with a separating comma. + +After this configuration and successful start of the Trino cluster, the table +`trino_queries` is created in the MySQL database. From then on, any query +processing event is captured by the event listener and a new row is inserted +into the table. The table includes many columns, such as query identifier, query +string, user, catalog, and others with information about the query processing. + +### Configuration properties + +:::{list-table} +:widths: 40, 60 +:header-rows: 1 + +* - Property name + - Description +* - `mysql-event-listener.db.url` + - JDBC connection URL to the database including credentials +::: \ No newline at end of file diff --git a/430/_sources/admin/fault-tolerant-execution.md.txt b/430/_sources/admin/fault-tolerant-execution.md.txt new file mode 100644 index 000000000..54185a71f --- /dev/null +++ b/430/_sources/admin/fault-tolerant-execution.md.txt @@ -0,0 +1,611 @@ +# Fault-tolerant execution + +By default, if a Trino node lacks the resources to execute a task or +otherwise fails during query execution, the query fails and must be run again +manually. The longer the runtime of a query, the more likely it is to be +susceptible to such failures. + +Fault-tolerant execution is a mechanism in Trino that enables a cluster to +mitigate query failures by retrying queries or their component tasks in +the event of failure. With fault-tolerant execution enabled, intermediate +exchange data is spooled and can be re-used by another worker in the event of a +worker outage or other fault during query execution. + +:::{note} +Fault tolerance does not apply to broken queries or other user error. For +example, Trino does not spend resources retrying a query that fails because +its SQL cannot be parsed. + +For a step-by-step guide explaining how to configure a Trino cluster with +fault-tolerant execution to improve query processing resilience, read +{doc}`/installation/query-resiliency`. +::: + +## Configuration + +Fault-tolerant execution is disabled by default. To enable the feature, set the +`retry-policy` configuration property to either `QUERY` or `TASK` +depending on the desired {ref}`retry policy `. + +```properties +retry-policy=QUERY +``` + +:::{warning} +Setting `retry-policy` may cause queries to fail with connectors that do not +explicitly support fault-tolerant execution, resulting in a "This connector +does not support query retries" error message. + +Support for fault-tolerant execution of SQL statements varies on a +per-connector basis, with more details in the documentation for each +connector. The following connectors support fault-tolerant execution: + +- {ref}`BigQuery connector ` +- {ref}`Delta Lake connector ` +- {ref}`Hive connector ` +- {ref}`Iceberg connector ` +- {ref}`MongoDB connector ` +- {ref}`MySQL connector ` +- {ref}`Oracle connector ` +- {ref}`PostgreSQL connector ` +- {ref}`Redshift connector ` +- {ref}`SQL Server connector ` +::: + +The following configuration properties control the behavior of fault-tolerant +execution on a Trino cluster: + + +:::{list-table} Fault-tolerant execution configuration properties +:widths: 30, 50, 20 +:header-rows: 1 + +* - Property name + - Description + - Default value +* - `retry-policy` + - Configures what is retried in the event of failure, either `QUERY` to retry + the whole query, or `TASK` to retry tasks individually if they fail. See + [retry policy](fte-retry-policy) for more information. + - `NONE` +* - `exchange.deduplication-buffer-size` + - [Data size](prop-type-data-size) of the coordinator's in-memory buffer used + by fault-tolerant execution to store output of query + [stages](trino-concept-stage). If this buffer is filled during query + execution, the query fails with a "Task descriptor storage capacity has been + exceeded" error message unless an [exchange manager](fte-exchange-manager) + is configured. + - `32MB` +* - `exchange.compression-enabled` + - Enable compression of spooling data. Setting to `true` is recommended + when using an [exchange manager](fte-exchange-manager). + - ``false`` +::: + +(fte-retry-policy)= + +## Retry policy + +The `retry-policy` configuration property designates whether Trino retries +entire queries or a query's individual tasks in the event of failure. + +### QUERY + +A `QUERY` retry policy instructs Trino to automatically retry a query in the +event of an error occuring on a worker node. A `QUERY` retry policy is +recommended when the majority of the Trino cluster's workload consists of many +small queries. + +By default Trino does not implement fault tolerance for queries whose result set +exceeds 32MB in size, such as {doc}`/sql/select` statements that return a very +large data set to the user. This limit can be increased by modifying the +`exchange.deduplication-buffer-size` configuration property to be greater than +the default value of `32MB`, but this results in higher memory usage on the +coordinator. + +To enable fault-tolerant execution on queries with a larger result set, it is +strongly recommended to configure an {ref}`exchange manager +` that utilizes external storage for spooled data and +therefore allows for storage of spilled data beyond the in-memory buffer size. + +### TASK + +A `TASK` retry policy instructs Trino to retry individual query {ref}`tasks +` in the event of failure. You must configure an +{ref}`exchange manager ` to use the task retry policy. +This policy is recommended when executing large batch queries, as the cluster +can more efficiently retry smaller tasks within the query rather than retry the +whole query. + +When a cluster is configured with a `TASK` retry policy, some relevant +configuration properties have their default values changed to follow best +practices for a fault-tolerant cluster. However, this automatic change does not +affect clusters that have these properties manually configured. If you have +any of the following properties configured in the `config.properties` file on +a cluster with a `TASK` retry policy, it is strongly recommended to make the +following changes: + +- Set the `task.low-memory-killer.policy` + {doc}`query management property ` to + `total-reservation-on-blocked-nodes`, or queries may + need to be manually killed if the cluster runs out of memory. +- Set the `query.low-memory-killer.delay` + {doc}`query management property ` to + `0s` so the cluster immediately unblocks nodes that run out of memory. +- Modify the `query.remote-task.max-error-duration` + {doc}`query management property ` + to adjust how long Trino allows a remote task to try reconnecting before + considering it lost and rescheduling. + +:::{note} +A `TASK` retry policy is best suited for large batch queries, but this +policy can result in higher latency for short-running queries executed in high +volume. As a best practice, it is recommended to run a dedicated cluster +with a `TASK` retry policy for large batch queries, separate from another +cluster that handles short queries. +::: + +## Advanced configuration + +You can further configure fault-tolerant execution with the following +configuration properties. The default values for these properties should work +for most deployments, but you can change these values for testing or +troubleshooting purposes. + +### Retry limits + +The following configuration properties control the thresholds at which +queries/tasks are no longer retried in the event of repeated failures: + +:::{list-table} Fault tolerance retry limit configuration properties +:widths: 30, 50, 20, 30 +:header-rows: 1 + +* - Property name + - Description + - Default value + - Retry policy +* - `query-retry-attempts` + - Maximum number of times Trino may attempt to retry a query before declaring + the query as failed. + - `4` + - Only `QUERY` +* - `task-retry-attempts-per-task` + - Maximum number of times Trino may attempt to retry a single task before + declaring the query as failed. + - `4` + - Only `TASK` +* - `retry-initial-delay` + - Minimum [time](prop-type-duration) that a failed query or task must wait + before it is retried. May be overridden with the `retry_initial_delay` + [session property](session-properties-definition). + - `10s` + - `QUERY` and `TASK` +* - `retry-max-delay` + - Maximum :ref:`time ` that a failed query or task must + wait before it is retried. Wait time is increased on each subsequent + failure. May be overridden with the ``retry_max_delay`` [session + property](session-properties-definition). + - `1m` + - `QUERY` and `TASK` +* - `retry-delay-scale-factor` + - Factor by which retry delay is increased on each query or task failure. May + be overridden with the `retry_delay_scale_factor` [session + property](session-properties-definition). + - `2.0` + - `QUERY` and `TASK` +::: + +### Task sizing + +With a `TASK` retry policy, it is important to manage the amount of data +processed in each task. If tasks are too small, the management of task +coordination can take more processing time and resources than executing the task +itself. If tasks are too large, then a single task may require more resources +than are available on any one node and therefore prevent the query from +completing. + +Trino supports limited automatic task sizing. If issues are occurring +during fault-tolerant task execution, you can configure the following +configuration properties to manually control task sizing. These configuration +properties only apply to a `TASK` retry policy. + +:::{list-table} Task sizing configuration properties +:widths: 30, 50, 20 +:header-rows: 1 + +* - Property name + - Description + - Default value +* - `fault-tolerant-execution-standard-split-size` + - Standard [split](trino-concept-splits) [data size]( prop-type-data-size) + processed by tasks that read data from source tables. Value is interpreted + with split weight taken into account. If the weight of splits produced by a + catalog denotes that they are lighter or heavier than "standard" split, then + the number of splits processed by a single task is adjusted accordingly. + + May be overridden for the current session with the + `fault_tolerant_execution_standard_split_size` [session + property](session-properties-definition). + - `64MB` +* - `fault-tolerant-execution-max-task-split-count` + - Maximum number of [splits](trino-concept-splits) processed by a single task. + This value is not split weight-adjusted and serves as protection against + situations where catalogs report an incorrect split weight. + + May be overridden for the current session with the + `fault_tolerant_execution_max_task_split_count` [session + property](session-properties-definition). + - `256` +* - `fault-tolerant-execution-arbitrary-distribution-compute-task-target-size-growth-period` + - The number of tasks created for any given non-writer stage of arbitrary + distribution before task size is increased. + - `64` +* - `fault-tolerant-execution-arbitrary-distribution-compute-task-target-size-growth-factor` + - Growth factor for adaptive sizing of non-writer tasks of arbitrary + distribution for fault-tolerant execution. Lower bound is 1.0. For every + task size increase, new task target size is old task target size multiplied + by this growth factor. + - `1.26` +* - `fault-tolerant-execution-arbitrary-distribution-compute-task-target-size-min` + - Initial/minimum target input [data size](prop-type-data-size) for non-writer + tasks of arbitrary distribution of fault-tolerant execution. + - `512MB` +* - `fault-tolerant-execution-arbitrary-distribution-compute-task-target-size-max` + - Maximum target input [data size](prop-type-data-size) for each non-writer + task of arbitrary distribution of fault-tolerant execution. + - `50GB` +* - `fault-tolerant-execution-arbitrary-distribution-write-task-target-size-growth-period` + - The number of tasks created for any given writer stage of arbitrary + distribution before task size is increased. + - `64` +* - `fault-tolerant-execution-arbitrary-distribution-write-task-target-size-growth-factor` + - Growth factor for adaptive sizing of writer tasks of arbitrary distribution + for fault-tolerant execution. Lower bound is 1.0. For every task size + increase, new task target size is old task target size multiplied by this + growth factor. + - `1.26` +* - `fault-tolerant-execution-arbitrary-distribution-write-task-target-size-min` + - Initial/minimum target input [data size](prop-type-data-size) for writer + tasks of arbitrary distribution of fault-tolerant execution. + - `4GB` +* - `fault-tolerant-execution-arbitrary-distribution-write-task-target-size-max` + - Maximum target input [data size](prop-type-data-size) for writer tasks of + arbitrary distribution of fault-tolerant execution. + - `50GB` +* - `fault-tolerant-execution-hash-distribution-compute-task-target-size` + - Target input [data size](prop-type-data-size) for non-writer tasks of hash + distribution of fault-tolerant execution. + - `512MB` +* - `fault-tolerant-execution-hash-distribution-write-task-target-size` + - Target input [data size](prop-type-data-size) of writer tasks of hash + distribution of fault-tolerant execution. + - ``4GB`` +* - `fault-tolerant-execution-hash-distribution-write-task-target-max-count` + - Soft upper bound on number of writer tasks in a stage of hash distribution + of fault-tolerant execution. + - `2000` +::: + +### Node allocation + +With a `TASK` retry policy, nodes are allocated to tasks based on available +memory and estimated memory usage. If task failure occurs due to exceeding +available memory on a node, the task is restarted with a request to allocate the +full node for its execution. + +The initial task memory-requirements estimation is static and configured with +the `fault-tolerant-task-memory` configuration property. This property only +applies to a `TASK` retry policy. + +:::{list-table} Node allocation configuration properties +:widths: 30, 50, 20 +:header-rows: 1 + +* - Property name + - Description + - Default value +* - `fault-tolerant-execution-task-memory` + - Initial task memory [data size](prop-type-data-size) estimation + used for bin-packing when allocating nodes for tasks. May be overridden + for the current session with the + `fault_tolerant_execution_task_memory` + [session property](session-properties-definition). + - `5GB` +::: + +### Other tuning + +The following additional configuration property can be used to manage +fault-tolerant execution: + +:::{list-table} Other fault-tolerant execution configuration properties +:widths: 30, 50, 20, 30 +:header-rows: 1 + +* - Property name + - Description + - Default value + - Retry policy +* - `fault-tolerant-execution-task-descriptor-storage-max-memory` + - Maximum [data size](prop-type-data-size) of memory to be used to + store task descriptors for fault tolerant queries on coordinator. Extra + memory is needed to be able to reschedule tasks in case of a failure. + - (JVM heap size * 0.15) + - Only `TASK` +* - `fault-tolerant-execution-max-partition-count` + - Maximum number of partitions to use for distributed joins and aggregations, + similar in function to the ``query.max-hash-partition-count`` [query + management property](/admin/properties-query-management). It is not + recommended to increase this property value above the default of `50`, which + may result in instability and poor performance. May be overridden for the + current session with the `fault_tolerant_execution_max_partition_count` + [session property](session-properties-definition). + - `50` + - Only `TASK` +* - `fault-tolerant-execution-min-partition-count` + - Minimum number of partitions to use for distributed joins and aggregations, + similar in function to the `query.min-hash-partition-count` [query + management property](/admin/properties-query-management). May be overridden + for the current session with the + `fault_tolerant_execution_min_partition_count` [session + property](session-properties-definition). + - `4` + - Only `TASK` +* - `fault-tolerant-execution-min-partition-count-for-write` + - Minimum number of partitions to use for distributed joins and aggregations + in write queries, similar in function to the + `query.min-hash-partition-count-for-write` [query management + property](/admin/properties-query-management). May be overridden for the + current session with the + `fault_tolerant_execution_min_partition_count_for_write` [session + property](session-properties-definition). + - `50` + - Only `TASK` +* - `max-tasks-waiting-for-node-per-stage` + - Allow for up to configured number of tasks to wait for node allocation + per stage, before pausing scheduling for other tasks from this stage. + - 5 + - Only `TASK` +::: + +(fte-exchange-manager)= + +## Exchange manager + +Exchange spooling is responsible for storing and managing spooled data for +fault-tolerant execution. You can configure a filesystem-based exchange manager +that stores spooled data in a specified location, such as {ref}`AWS S3 +` and S3-compatible systems, {ref}`Azure Blob Storage +`, {ref}`Google Cloud Storage `, +or {ref}`HDFS `. + +### Configuration + +To configure an exchange manager, create a new +`etc/exchange-manager.properties` configuration file on the coordinator and +all worker nodes. In this file, set the `exchange-manager.name` configuration +property to `filesystem` or `hdfs`, and set additional configuration properties as needed +for your storage solution. + +The following table lists the available configuration properties for +`exchange-manager.properties`, their default values, and which filesystem(s) +the property may be configured for: + +:::{list-table} Exchange manager configuration properties +:widths: 30, 50, 20, 30 +:header-rows: 1 + +* - Property name + - Description + - Default value + - Supported filesystem +* - `exchange.base-directories` + - Comma-separated list of URI locations that the exchange manager uses to + store spooling data. + - + - Any +* - `exchange.sink-buffer-pool-min-size` + - The minimum buffer pool size for an exchange sink. The larger the buffer + pool size, the larger the write parallelism and memory usage. + - `10` + - Any +* - `exchange.sink-buffers-per-partition` + - The number of buffers per partition in the buffer pool. The larger the + buffer pool size, the larger the write parallelism and memory usage. + - `2` + - Any +* - `exchange.sink-max-file-size` + - Max [data size](prop-type-data-size) of files written by exchange sinks. + - ``1GB`` + - Any +* - `exchange.source-concurrent-readers` + - Number of concurrent readers to read from spooling storage. The larger the + number of concurrent readers, the larger the read parallelism and memory + usage. + - `4` + - Any +* - `exchange.s3.aws-access-key` + - AWS access key to use. Required for a connection to AWS S3 and GCS, can be + ignored for other S3 storage systems. + - + - AWS S3, GCS +* - `exchange.s3.aws-secret-key` + - AWS secret key to use. Required for a connection to AWS S3 and GCS, can be + ignored for other S3 storage systems. + - + - AWS S3, GCS +* - `exchange.s3.iam-role` + - IAM role to assume. + - + - AWS S3, GCS +* - `exchange.s3.external-id` + - External ID for the IAM role trust policy. + - + - AWS S3, GCS +* - `exchange.s3.region` + - Region of the S3 bucket. + - + - AWS S3, GCS +* - `exchange.s3.endpoint` + - S3 storage endpoint server if using an S3-compatible storage system that + is not AWS. If using AWS S3, this can be ignored. If using GCS, set it + to `https://storage.googleapis.com`. + - + - Any S3-compatible storage +* - `exchange.s3.max-error-retries` + - Maximum number of times the exchange manager's S3 client should retry + a request. + - `10` + - Any S3-compatible storage +* - `exchange.s3.path-style-access` + - Enables using [path-style access](https://docs.aws.amazon.com/AmazonS3/latest/userguide/VirtualHosting.html#path-style-access) + for all requests to S3. + - `false` + - Any S3-compatible storage +* - `exchange.s3.upload.part-size` + - Part [data size](prop-type-data-size) for S3 multi-part upload. + - `5MB` + - Any S3-compatible storage +* - `exchange.gcs.json-key-file-path` + - Path to the JSON file that contains your Google Cloud Platform service + account key. Not to be set together with `exchange.gcs.json-key` + - + - GCS +* - `exchange.gcs.json-key` + - Your Google Cloud Platform service account key in JSON format. Not to be set + together with `exchange.gcs.json-key-file-path` + - + - GCS +* - `exchange.azure.connection-string` + - Connection string used to access the spooling container. + - + - Azure Blob Storage +* - `exchange.azure.block-size` + - Block [data size](prop-type-data-size) for Azure block blob parallel upload. + - `4MB` + - Azure Blob Storage +* - `exchange.azure.max-error-retries` + - Maximum number of times the exchange manager's Azure client should + retry a request. + - `10` + - Azure Blob Storage +* - `exchange.hdfs.block-size` + - Block [data size](prop-type-data-size) for HDFS storage. + - `4MB` + - HDFS +* - `hdfs.config.resources` + - Comma-separated list of paths to HDFS configuration files, for example + `/etc/hdfs-site.xml`. The files must exist on all nodes in the Trino + cluster. + - + - HDFS +::: + +It is recommended to set the `exchange.compression-enabled` property to +`true` in the cluster's `config.properties` file, to reduce the exchange +manager's overall I/O load. It is also recommended to configure a bucket +lifecycle rule to automatically expire abandoned objects in the event of a node +crash. + +(fte-exchange-aws-s3)= + +#### AWS S3 + +The following example `exchange-manager.properties` configuration specifies an +AWS S3 bucket as the spooling storage destination. Note that the destination +does not have to be in AWS, but can be any S3-compatible storage system. + +```properties +exchange-manager.name=filesystem +exchange.base-directories=s3://exchange-spooling-bucket +exchange.s3.region=us-west-1 +exchange.s3.aws-access-key=example-access-key +exchange.s3.aws-secret-key=example-secret-key +``` + +You can configure multiple S3 buckets for the exchange manager to distribute +spooled data across buckets, reducing the I/O load on any one bucket. If a query +fails with the error message +"software.amazon.awssdk.services.s3.model.S3Exception: Please reduce your +request rate", this indicates that the workload is I/O intensive, and you should +specify multiple S3 buckets in `exchange.base-directories` to balance the +load: + +```properties +exchange.base-directories=s3://exchange-spooling-bucket-1,s3://exchange-spooling-bucket-2 +``` + +(fte-exchange-azure-blob)= + +#### Azure Blob Storage + +The following example `exchange-manager.properties` configuration specifies an +Azure Blob Storage container as the spooling storage destination. You must use +Azure Blob Storage, not Azure Data Lake Storage or any other hierarchical +storage option in Azure. + +```properties +exchange-manager.name=filesystem +exchange.base-directories=abfs://container_name@account_name.dfs.core.windows.net +exchange.azure.connection-string=connection-string +``` + +(fte-exchange-gcs)= + +#### Google Cloud Storage + +To enable exchange spooling on GCS in Trino, change the request endpoint to the +`https://storage.googleapis.com` Google storage URI, and configure your AWS +access/secret keys to use the GCS HMAC keys. If you deploy Trino on GCP, you +must either create a service account with access to your spooling bucket or +configure the key path to your GCS credential file. + +For more information on GCS's S3 compatibility, refer to the [Google Cloud +documentation on S3 migration](https://cloud.google.com/storage/docs/aws-simple-migration). + +The following example `exchange-manager.properties` configuration specifies a +GCS bucket as the spooling storage destination. + +```properties +exchange-manager.name=filesystem +exchange.base-directories=gs://exchange-spooling-bucket +exchange.s3.region=us-west-1 +exchange.s3.aws-access-key=example-access-key +exchange.s3.aws-secret-key=example-secret-key +exchange.s3.endpoint=https://storage.googleapis.com +exchange.gcs.json-key-file-path=/path/to/gcs_keyfile.json +``` + +(fte-exchange-hdfs)= + +#### HDFS + +The following `exchange-manager.properties` configuration example specifies HDFS +as the spooling storage destination. + +```properties +exchange-manager.name=hdfs +exchange.base-directories=hadoop-master:9000/exchange-spooling-directory +hdfs.config.resources=/usr/lib/hadoop/etc/hadoop/core-site.xml +``` + +(fte-exchange-local-filesystem)= + +#### Local filesystem storage + +The following example `exchange-manager.properties` configuration specifies a +local directory, `/tmp/trino-exchange-manager`, as the spooling storage +destination. + +:::{note} +It is only recommended to use a local filesystem for exchange in standalone, +non-production clusters. A local directory can only be used for exchange in +a distributed cluster if the exchange directory is shared and accessible +from all worker nodes. +::: + +```properties +exchange-manager.name=filesystem +exchange.base-directories=/tmp/trino-exchange-manager +``` diff --git a/430/_sources/admin/graceful-shutdown.md.txt b/430/_sources/admin/graceful-shutdown.md.txt new file mode 100644 index 000000000..9deb35b35 --- /dev/null +++ b/430/_sources/admin/graceful-shutdown.md.txt @@ -0,0 +1,41 @@ +# Graceful shutdown + +Trino has a graceful shutdown API that can be used exclusively on workers in +order to ensure that they terminate without affecting running queries, given a +sufficient grace period. + +You can invoke the API with a HTTP PUT request: + +```bash +curl -v -X PUT -d '"SHUTTING_DOWN"' -H "Content-type: application/json" \ + http://worker:8081/v1/info/state +``` + +A successful invocation is logged with a `Shutdown requested` message at +`INFO` level in the worker server log. + +Keep the following aspects in mind: + +- If your cluster is secure, you need to provide a basic-authorization header, + or satisfy whatever other security you have enabled. +- If you have TLS/HTTPS enabled, you have to ensure the worker certificate is + CA signed, or trusted by the server calling the shut down endpoint. + Otherwise, you can make the call `--insecure`, but that isn't recommended. +- The `default` {doc}`/security/built-in-system-access-control` does not allow + graceful shutdowns. You can use the `allow-all` system access control, or + configure {ref}`system information rules + ` with the `file` system access + control. These configuration must be present on all workers. + +## Shutdown behavior + +Once the API is called, the worker performs the following steps: + +- Go into `SHUTTING_DOWN` state. +- Sleep for `shutdown.grace-period`, which defaults to 2 minutes. + : - After this, the coordinator is aware of the shutdown and stops sending + tasks to the worker. +- Block until all active tasks are complete. +- Sleep for the grace period again in order to ensure the coordinator sees + all tasks are complete. +- Shutdown the application. diff --git a/430/_sources/admin/jmx.md.txt b/430/_sources/admin/jmx.md.txt new file mode 100644 index 000000000..b6d6dad46 --- /dev/null +++ b/430/_sources/admin/jmx.md.txt @@ -0,0 +1,70 @@ +# Monitoring with JMX + +Trino exposes a large number of different metrics via the Java Management Extensions (JMX). + +You have to enable JMX by setting the ports used by the RMI registry and server +in the {ref}`config.properties file `: + +```text +jmx.rmiregistry.port=9080 +jmx.rmiserver.port=9081 +``` + +- `jmx.rmiregistry.port`: + Specifies the port for the JMX RMI registry. JMX clients should connect to this port. +- `jmx.rmiserver.port`: + Specifies the port for the JMX RMI server. Trino exports many metrics, + that are useful for monitoring via JMX. + +Additionally configure a Java system property in the +[jvm.config](jvm-config) with the RMI server port: + +```properties +-Dcom.sun.management.jmxremote.rmi.port=9081 +``` + +JConsole (supplied with the JDK), [VisualVM](https://visualvm.github.io/), and +many other tools can be used to access the metrics in a client application. +Many monitoring solutions support JMX. You can also use the +{doc}`/connector/jmx` and query the metrics using SQL. + +Many of these JMX metrics are a complex metric object such as a `CounterStat` +that has a collection of related metrics. For example, `InputPositions` has +`InputPositions.TotalCount`, `InputPositions.OneMinute.Count`, and so on. + +A small subset of the available metrics are described below. + +## JVM + +- Heap size: `java.lang:type=Memory:HeapMemoryUsage.used` +- Thread count: `java.lang:type=Threading:ThreadCount` + +## Trino cluster and nodes + +- Active nodes: + `trino.failuredetector:name=HeartbeatFailureDetector:ActiveCount` +- Free memory (general pool): + `trino.memory:type=ClusterMemoryPool:name=general:FreeDistributedBytes` +- Cumulative count (since Trino started) of queries that ran out of memory and were killed: + `trino.memory:name=ClusterMemoryManager:QueriesKilledDueToOutOfMemory` + +## Trino queries + +- Active queries currently executing or queued: `trino.execution:name=QueryManager:RunningQueries` +- Queries started: `trino.execution:name=QueryManager:StartedQueries.FiveMinute.Count` +- Failed queries from last 5 min (all): `trino.execution:name=QueryManager:FailedQueries.FiveMinute.Count` +- Failed queries from last 5 min (internal): `trino.execution:name=QueryManager:InternalFailures.FiveMinute.Count` +- Failed queries from last 5 min (external): `trino.execution:name=QueryManager:ExternalFailures.FiveMinute.Count` +- Failed queries (user): `trino.execution:name=QueryManager:UserErrorFailures.FiveMinute.Count` +- Execution latency (P50): `trino.execution:name=QueryManager:ExecutionTime.FiveMinutes.P50` +- Input data rate (P90): `trino.execution:name=QueryManager:WallInputBytesRate.FiveMinutes.P90` + +## Trino tasks + +- Input data bytes: `trino.execution:name=SqlTaskManager:InputDataSize.FiveMinute.Count` +- Input rows: `trino.execution:name=SqlTaskManager:InputPositions.FiveMinute.Count` + +## Connectors + +Many connectors provide their own metrics. The metric names typically start with +`trino.plugin`. diff --git a/430/_sources/admin/properties-exchange.md.txt b/430/_sources/admin/properties-exchange.md.txt new file mode 100644 index 000000000..3abaf957f --- /dev/null +++ b/430/_sources/admin/properties-exchange.md.txt @@ -0,0 +1,91 @@ +# Exchange properties + +Exchanges transfer data between Trino nodes for different stages of +a query. Adjusting these properties may help to resolve inter-node +communication issues or improve network utilization. + +## `exchange.client-threads` + +- **Type:** {ref}`prop-type-integer` +- **Minimum value:** `1` +- **Default value:** `25` + +Number of threads used by exchange clients to fetch data from other Trino +nodes. A higher value can improve performance for large clusters or clusters +with very high concurrency, but excessively high values may cause a drop +in performance due to context switches and additional memory usage. + +## `exchange.concurrent-request-multiplier` + +- **Type:** {ref}`prop-type-integer` +- **Minimum value:** `1` +- **Default value:** `3` + +Multiplier determining the number of concurrent requests relative to +available buffer memory. The maximum number of requests is determined +using a heuristic of the number of clients that can fit into available +buffer space, based on average buffer usage per request times this +multiplier. For example, with an `exchange.max-buffer-size` of `32 MB` +and `20 MB` already used and average size per request being `2MB`, +the maximum number of clients is +`multiplier * ((32MB - 20MB) / 2MB) = multiplier * 6`. Tuning this +value adjusts the heuristic, which may increase concurrency and improve +network utilization. + +## `exchange.data-integrity-verification` + +- **Type:** {ref}`prop-type-string` +- **Allowed values:** `NONE`, `ABORT`, `RETRY` +- **Default value:** `ABORT` + +Configure the resulting behavior of data integrity issues. By default, +`ABORT` causes queries to be aborted when data integrity issues are +detected as part of the built-in verification. Setting the property to +`NONE` disables the verification. `RETRY` causes the data exchange to be +repeated when integrity issues are detected. + +## `exchange.max-buffer-size` + +- **Type:** {ref}`prop-type-data-size` +- **Default value:** `32MB` + +Size of buffer in the exchange client that holds data fetched from other +nodes before it is processed. A larger buffer can increase network +throughput for larger clusters, and thus decrease query processing time, +but reduces the amount of memory available for other usages. + +## `exchange.max-response-size` + +- **Type:** {ref}`prop-type-data-size` +- **Minimum value:** `1MB` +- **Default value:** `16MB` + +Maximum size of a response returned from an exchange request. The response +is placed in the exchange client buffer, which is shared across all +concurrent requests for the exchange. + +Increasing the value may improve network throughput, if there is high +latency. Decreasing the value may improve query performance for large +clusters as it reduces skew, due to the exchange client buffer holding +responses for more tasks, rather than hold more data from fewer tasks. + +## `sink.max-buffer-size` + +- **Type:** {ref}`prop-type-data-size` +- **Default value:** `32MB` + +Output buffer size for task data that is waiting to be pulled by upstream +tasks. If the task output is hash partitioned, then the buffer is +shared across all of the partitioned consumers. Increasing this value may +improve network throughput for data transferred between stages, if the +network has high latency, or if there are many nodes in the cluster. + +## `sink.max-broadcast-buffer-size` + +- **Type** `data size` +- **Default value:** `200MB` + +Broadcast output buffer size for task data that is waiting to be pulled by +upstream tasks. The broadcast buffer is used to store and transfer build side +data for replicated joins. If the buffer is too small, it prevents scaling of +join probe side tasks, when new nodes are added to the cluster. diff --git a/430/_sources/admin/properties-general.md.txt b/430/_sources/admin/properties-general.md.txt new file mode 100644 index 000000000..860bb44b4 --- /dev/null +++ b/430/_sources/admin/properties-general.md.txt @@ -0,0 +1,69 @@ +# General properties + +## `join-distribution-type` + +- **Type:** {ref}`prop-type-string` +- **Allowed values:** `AUTOMATIC`, `PARTITIONED`, `BROADCAST` +- **Default value:** `AUTOMATIC` +- **Session property:** `join_distribution_type` + +The type of distributed join to use. When set to `PARTITIONED`, Trino +uses hash distributed joins. When set to `BROADCAST`, it broadcasts the +right table to all nodes in the cluster that have data from the left table. +Partitioned joins require redistributing both tables using a hash of the join key. +This can be slower, sometimes substantially, than broadcast joins, but allows much +larger joins. In particular broadcast joins are faster, if the right table is +much smaller than the left. However, broadcast joins require that the tables on the right +side of the join after filtering fit in memory on each node, whereas distributed joins +only need to fit in distributed memory across all nodes. When set to `AUTOMATIC`, +Trino makes a cost based decision as to which distribution type is optimal. +It considers switching the left and right inputs to the join. In `AUTOMATIC` +mode, Trino defaults to hash distributed joins if no cost could be computed, such as if +the tables do not have statistics. + +## `redistribute-writes` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `true` +- **Session property:** `redistribute_writes` + +This property enables redistribution of data before writing. This can +eliminate the performance impact of data skew when writing by hashing it +across nodes in the cluster. It can be disabled, when it is known that the +output data set is not skewed, in order to avoid the overhead of hashing and +redistributing all the data across the network. + +## `protocol.v1.alternate-header-name` + +**Type:** `string` + +The 351 release of Trino changes the HTTP client protocol headers to start with +`X-Trino-`. Clients for versions 350 and lower expect the HTTP headers to +start with `X-Presto-`, while newer clients expect `X-Trino-`. You can support these +older clients by setting this property to `Presto`. + +The preferred approach to migrating from versions earlier than 351 is to update +all clients together with the release, or immediately afterwards, and then +remove usage of this property. + +Ensure to use this only as a temporary measure to assist in your migration +efforts. + +## `protocol.v1.prepared-statement-compression.length-threshold` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** `2048` + +Prepared statements that are submitted to Trino for processing, and are longer +than the value of this property, are compressed for transport via the HTTP +header to improve handling, and to avoid failures due to hitting HTTP header +size limits. + +## `protocol.v1.prepared-statement-compression.min-gain` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** `512` + +Prepared statement compression is not applied if the size gain is less than the +configured value. Smaller statements do not benefit from compression, and are +left uncompressed. diff --git a/430/_sources/admin/properties-http-client.md.txt b/430/_sources/admin/properties-http-client.md.txt new file mode 100644 index 000000000..5c40b022e --- /dev/null +++ b/430/_sources/admin/properties-http-client.md.txt @@ -0,0 +1,177 @@ +# HTTP client properties + +HTTP client properties allow you to configure the connection from Trino to +external services using HTTP. + +The following properties can be used after adding the specific prefix to the +property. For example, for {doc}`/security/oauth2`, you can enable HTTP for +interactions with the external OAuth 2.0 provider by adding the prefix +`oauth2-jwk` to the `http-client.connect-timeout` property, and increasing +the connection timeout to ten seconds by setting the value to `10`: + +``` +oauth2-jwk.http-client.connect-timeout=10s +``` + +The following prefixes are supported: + +- `oauth2-jwk` for {doc}`/security/oauth2` +- `jwk` for {doc}`/security/jwt` + +## General properties + +### `http-client.connect-timeout` + +- **Type:** {ref}`prop-type-duration` +- **Default value:** `5s` +- **Minimum value:** `0ms` + +Timeout value for establishing the connection to the external service. + +### `http-client.max-connections` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** `200` + +Maximum connections allowed to the service. + +### `http-client.request-timeout` + +- **Type:** {ref}`prop-type-duration` +- **Default value:** `5m` +- **Minimum value:** `0ms` + +Timeout value for the overall request. + +## TLS and security properties + +### `http-client.https.excluded-cipher` + +- **Type:** {ref}`prop-type-string` + +A comma-separated list of regexes for the names of cipher algorithms to exclude. + +### `http-client.https.included-cipher` + +- **Type:** {ref}`prop-type-string` + +A comma-separated list of regexes for the names of the cipher algorithms to use. + +### `http-client.https.hostname-verification` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `true` + +Verify that the server hostname matches the server DNS name in the +SubjectAlternativeName (SAN) field of the certificate. + +### `http-client.key-store-password` + +- **Type:** {ref}`prop-type-string` + +Password for the keystore. + +### `http-client.key-store-path` + +- **Type:** {ref}`prop-type-string` + +File path on the server to the keystore file. + +### `http-client.secure-random-algorithm` + +- **Type:** {ref}`prop-type-string` + +Set the secure random algorithm for the connection. The default varies by +operating system. Algorithms are specified according to standard algorithm name +documentation. + +Possible types include `NativePRNG`, `NativePRNGBlocking`, +`NativePRNGNonBlocking`, `PKCS11`, and `SHA1PRNG`. + +### `http-client.trust-store-password` + +- **Type:** {ref}`prop-type-string` + +Password for the truststore. + +### `http-client.trust-store-path` + +- **Type:** {ref}`prop-type-string` + +File path on the server to the truststore file. + +## Proxy properties + +### `http-client.http-proxy` + +- **Type:** {ref}`prop-type-string` + +Host and port for an HTTP proxy with the format `example.net:8080`. + +### `http-client.http-proxy.secure` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `false` + +Enable HTTPS for the proxy. + +### `http-client.socks-proxy` + +- **Type:** {ref}`prop-type-string` + +Host and port for a SOCKS proxy. + +## Request logging + +### `http-client.log.compression-enabled` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `true` + +Enable log file compression. The client uses the `.gz` format for log files. + +### `http-client.log.enabled` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `false` + +Enable logging of HTTP requests. + +### `http-client.log.flush-interval` + +- **Type:** {ref}`prop-type-duration` +- **Default value:** `10s` + +Frequency of flushing the log data to disk. + +### `http-client.log.max-history` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** `15` + +Retention limit of log files in days. Files older than the `max-history` are +deleted when the HTTP client creates files for new logging periods. + +### `http-client.log.max-size` + +- **Type:** {ref}`prop-type-data-size` +- **Default value:** `1GB` + +Maximum total size of all log files on disk. + +### `http-client.log.path` + +- **Type:** {ref}`prop-type-string` +- **Default value:** `var/log/` + +Sets the path of the log files. All log files are named `http-client.log`, and +have the prefix of the specific HTTP client added. For example, +`jwk-http-client.log`. + +### `http-client.log.queue-size` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** `10000` +- **Minimum value:** `1` + +Size of the HTTP client logging queue. diff --git a/430/_sources/admin/properties-logging.md.txt b/430/_sources/admin/properties-logging.md.txt new file mode 100644 index 000000000..43f6595af --- /dev/null +++ b/430/_sources/admin/properties-logging.md.txt @@ -0,0 +1,98 @@ +# Logging properties + +## `log.annotation-file` + +- **Type:** {ref}`prop-type-string` + +An optional properties file that contains annotations to be included with +each log message. This can be used to include machine-specific or +environment-specific information into logs which are centrally aggregated. +The annotation values can contain references to environment variables. + +```properties +environment=production +host=${ENV:HOSTNAME} +``` + +## `log.format` + +- **Type:** {ref}`prop-type-string` +- **Default value:** `TEXT` + +The file format for log records. Can be set to either `TEXT` or `JSON`. When +set to `JSON`, the log record is formatted as a JSON object, one record per +line. Any newlines in the field values, such as exception stack traces, are +escaped as normal in the JSON object. This allows for capturing and indexing +exceptions as singular fields in a logging search system. + +## `log.path` + +- **Type:** {ref}`prop-type-string` + +The path to the log file used by Trino. The path is relative to the data +directory, configured to `var/log/server.log` by the launcher script as +detailed in {ref}`running-trino`. Alternatively, you can write logs to separate +the process (typically running next to Trino as a sidecar process) via the TCP +protocol by using a log path of the format `tcp://host:port`. + +## `log.max-size` + +- **Type:** {ref}`prop-type-data-size` +- **Default value:** `100MB` + +The maximum file size for the general application log file. + +## `log.max-total-size` + +- **Type:** {ref}`prop-type-data-size` +- **Default value:** `1GB` + +The maximum file size for all general application log files combined. + +## `log.compression` + +- **Type:** {ref}`prop-type-string` +- **Default value:** `GZIP` + +The compression format for rotated log files. Can be set to either `GZIP` or `NONE`. When +set to `NONE`, compression is disabled. + +## `http-server.log.enabled` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `true` + +Flag to enable or disable logging for the HTTP server. + +## `http-server.log.compression.enabled` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `true` + +Flag to enable or disable compression of the log files of the HTTP server. + +## `http-server.log.path` + +- **Type:** {ref}`prop-type-string` +- **Default value:** `var/log/http-request.log` + +The path to the log file used by the HTTP server. The path is relative to +the data directory, configured by the launcher script as detailed in +{ref}`running-trino`. + +## `http-server.log.max-history` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** `15` + +The maximum number of log files for the HTTP server to use, before +log rotation replaces old content. + +## `http-server.log.max-size` + +- **Type:** {ref}`prop-type-data-size` +- **Default value:** `unlimited` + +The maximum file size for the log file of the HTTP server. Defaults to +`unlimited`, setting a {ref}`prop-type-data-size` value limits the file size +to that value. diff --git a/430/_sources/admin/properties-node-scheduler.md.txt b/430/_sources/admin/properties-node-scheduler.md.txt new file mode 100644 index 000000000..8213a90f7 --- /dev/null +++ b/430/_sources/admin/properties-node-scheduler.md.txt @@ -0,0 +1,185 @@ +# Node scheduler properties + +## `node-scheduler.include-coordinator` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `true` + +Allows scheduling work on the coordinator so that a single machine can function +as both coordinator and worker. For large clusters, processing work on the +coordinator can negatively impact query performance because the machine's +resources are not available for the critical coordinator tasks of scheduling, +managing, and monitoring query execution. + +### Splits + +## `node-scheduler.max-splits-per-node` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** `100` + +The target value for the total number of splits that can be running for +each worker node, assuming all splits have the standard split weight. + +Using a higher value is recommended, if queries are submitted in large batches +(e.g., running a large group of reports periodically), or for connectors that +produce many splits that complete quickly but do not support assigning split +weight values to express that to the split scheduler. Increasing this value may +improve query latency, by ensuring that the workers have enough splits to keep +them fully utilized. + +When connectors do support weight based split scheduling, the number of splits +assigned will depend on the weight of the individual splits. If splits are +small, more of them are allowed to be assigned to each worker to compensate. + +Setting this too high wastes memory and may result in lower performance +due to splits not being balanced across workers. Ideally, it should be set +such that there is always at least one split waiting to be processed, but +not higher. + +## `node-scheduler.min-pending-splits-per-task` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** `10` + +The minimum number of outstanding splits with the standard split weight guaranteed to be scheduled on a node (even when the node +is already at the limit for total number of splits) for a single task given the task has remaining splits to process. +Allowing a minimum number of splits per stage is required to prevent starvation and deadlocks. + +This value must be smaller or equal than `max-adjusted-pending-splits-per-task` and +`node-scheduler.max-splits-per-node`, is usually increased for the same reasons, +and has similar drawbacks if set too high. + +## `node-scheduler.max-adjusted-pending-splits-per-task` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** `2000` + +The maximum number of outstanding splits with the standard split weight guaranteed to be scheduled on a node (even when the node +is already at the limit for total number of splits) for a single task given the task has remaining splits to process. +Split queue size is adjusted dynamically during split scheduling and cannot exceed `node-scheduler.max-adjusted-pending-splits-per-task`. +Split queue size per task will be adjusted upward if node processes splits faster than it receives them. + +Usually increased for the same reasons as `node-scheduler.max-splits-per-node`, with smaller drawbacks +if set too high. + +:::{note} +Only applies for `uniform` {ref}`scheduler policy `. +::: + +## `node-scheduler.max-unacknowledged-splits-per-task` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** `2000` + +Maximum number of splits that are either queued on the coordinator, but not yet sent or confirmed to have been received by +the worker. This limit enforcement takes precedence over other existing split limit configurations +like `node-scheduler.max-splits-per-node` or `node-scheduler.max-adjusted-pending-splits-per-task` +and is designed to prevent large task update requests that might cause a query to fail. + +## `node-scheduler.min-candidates` + +- **Type:** {ref}`prop-type-integer` +- **Minimum value:** `1` +- **Default value:** `10` + +The minimum number of candidate nodes that are evaluated by the +node scheduler when choosing the target node for a split. Setting +this value too low may prevent splits from being properly balanced +across all worker nodes. Setting it too high may increase query +latency and increase CPU usage on the coordinator. + +(node-scheduler-policy)= + +## `node-scheduler.policy` + +- **Type:** {ref}`prop-type-string` +- **Allowed values:** `uniform`, `topology` +- **Default value:** `uniform` + +Sets the node scheduler policy to use when scheduling splits. `uniform` attempts +to schedule splits on the host where the data is located, while maintaining a uniform +distribution across all hosts. `topology` tries to schedule splits according to +the topology distance between nodes and splits. It is recommended to use `uniform` +for clusters where distributed storage runs on the same nodes as Trino workers. + +### Network topology + +## `node-scheduler.network-topology.segments` + +- **Type:** {ref}`prop-type-string` +- **Default value:** `machine` + +A comma-separated string describing the meaning of each segment of a network location. +For example, setting `region,rack,machine` means a network location contains three segments. + +## `node-scheduler.network-topology.type` + +- **Type:** {ref}`prop-type-string` +- **Allowed values:** `flat`, `file`, `subnet` +- **Default value:** `flat` + +Sets the network topology type. To use this option, `node-scheduler.policy` +must be set to `topology`. + +- `flat`: the topology has only one segment, with one value for each machine. +- `file`: the topology is loaded from a file using the properties + `node-scheduler.network-topology.file` and + `node-scheduler.network-topology.refresh-period` described in the + following sections. +- `subnet`: the topology is derived based on subnet configuration provided + through properties `node-scheduler.network-topology.subnet.cidr-prefix-lengths` + and `node-scheduler.network-topology.subnet.ip-address-protocol` described + in the following sections. + +### File based network topology + +## `node-scheduler.network-topology.file` + +- **Type:** {ref}`prop-type-string` + +Load the network topology from a file. To use this option, `node-scheduler.network-topology.type` +must be set to `file`. Each line contains a mapping between a host name and a +network location, separated by whitespace. Network location must begin with a leading +`/` and segments are separated by a `/`. + +```text +192.168.0.1 /region1/rack1/machine1 +192.168.0.2 /region1/rack1/machine2 +hdfs01.example.com /region2/rack2/machine3 +``` + +## `node-scheduler.network-topology.refresh-period` + +- **Type:** {ref}`prop-type-duration` +- **Minimum value:** `1ms` +- **Default value:** `5m` + +Controls how often the network topology file is reloaded. To use this option, +`node-scheduler.network-topology.type` must be set to `file`. + +### Subnet based network topology + +## `node-scheduler.network-topology.subnet.ip-address-protocol` + +- **Type:** {ref}`prop-type-string` +- **Allowed values:** `IPv4`, `IPv6` +- **Default value:** `IPv4` + +Sets the IP address protocol to be used for computing subnet based +topology. To use this option, `node-scheduler.network-topology.type` must +be set to `subnet`. + +## `node-scheduler.network-topology.subnet.cidr-prefix-lengths` + +A comma-separated list of {ref}`prop-type-integer` values defining CIDR prefix +lengths for subnet masks. The prefix lengths must be in increasing order. The +maximum prefix length values for IPv4 and IPv6 protocols are 32 and 128 +respectively. To use this option, `node-scheduler.network-topology.type` must +be set to `subnet`. + +For example, the value `24,25,27` for this property with IPv4 protocol means +that masks applied on the IP address to compute location segments are +`255.255.255.0`, `255.255.255.128` and `255.255.255.224`. So the segments +created for an address `192.168.0.172` are `[192.168.0.0, 192.168.0.128, +192.168.0.160, 192.168.0.172]`. diff --git a/430/_sources/admin/properties-optimizer.md.txt b/430/_sources/admin/properties-optimizer.md.txt new file mode 100644 index 000000000..9fc94c6ae --- /dev/null +++ b/430/_sources/admin/properties-optimizer.md.txt @@ -0,0 +1,258 @@ +# Optimizer properties + +## `optimizer.dictionary-aggregation` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `false` +- **Session property:** `dictionary_aggregation` + +Enables optimization for aggregations on dictionaries. + +## `optimizer.optimize-hash-generation` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `true` +- **Session property:** `optimize_hash_generation + +Compute hash codes for distribution, joins, and aggregations early during execution, +allowing result to be shared between operations later in the query. This can reduce +CPU usage by avoiding computing the same hash multiple times, but at the cost of +additional network transfer for the hashes. In most cases it decreases overall +query processing time. + +It is often helpful to disable this property, when using {doc}`/sql/explain` in order +to make the query plan easier to read. + +## `optimizer.optimize-metadata-queries` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `false` + +Enable optimization of some aggregations by using values that are stored as metadata. +This allows Trino to execute some simple queries in constant time. Currently, this +optimization applies to `max`, `min` and `approx_distinct` of partition +keys and other aggregation insensitive to the cardinality of the input,including +`DISTINCT` aggregates. Using this may speed up some queries significantly. + +The main drawback is that it can produce incorrect results, if the connector returns +partition keys for partitions that have no rows. In particular, the Hive connector +can return empty partitions, if they were created by other systems. Trino cannot +create them. + +## `optimizer.mark-distinct-strategy` + +- **Type:** {ref}`prop-type-string` +- **Allowed values:** `AUTOMATIC`, `ALWAYS`, `NONE` +- **Default value:** `AUTOMATIC` +- **Session property:** `mark_distinct_strategy` + +The mark distinct strategy to use for distinct aggregations. `NONE` does not use +`MarkDistinct` operator. `ALWAYS` uses `MarkDistinct` for multiple distinct +aggregations or for mix of distinct and non-distinct aggregations. +`AUTOMATIC` limits the use of `MarkDistinct` only for cases with limited +concurrency (global or small cardinality aggregations), where direct distinct +aggregation implementation cannot utilize CPU efficiently. + +## `optimizer.push-aggregation-through-outer-join` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `true` +- **Session property:** `push_aggregation_through_join` + +When an aggregation is above an outer join and all columns from the outer side of the join +are in the grouping clause, the aggregation is pushed below the outer join. This optimization +is particularly useful for correlated scalar subqueries, which get rewritten to an aggregation +over an outer join. For example: + +``` +SELECT * FROM item i + WHERE i.i_current_price > ( + SELECT AVG(j.i_current_price) FROM item j + WHERE i.i_category = j.i_category); +``` + +Enabling this optimization can substantially speed up queries by reducing the +amount of data that needs to be processed by the join. However, it may slow down +some queries that have very selective joins. + +## `optimizer.push-table-write-through-union` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `true` +- **Session property:** `push_table_write_through_union` + +Parallelize writes when using `UNION ALL` in queries that write data. This improves the +speed of writing output tables in `UNION ALL` queries, because these writes do not require +additional synchronization when collecting results. Enabling this optimization can improve +`UNION ALL` speed, when write speed is not yet saturated. However, it may slow down queries +in an already heavily loaded system. + +## `optimizer.join-reordering-strategy` + +- **Type:** {ref}`prop-type-string` +- **Allowed values:** `AUTOMATIC`, `ELIMINATE_CROSS_JOINS`, `NONE` +- **Default value:** `AUTOMATIC` +- **Session property:** `join_reordering_strategy` + +The join reordering strategy to use. `NONE` maintains the order the tables are listed in the +query. `ELIMINATE_CROSS_JOINS` reorders joins to eliminate cross joins, where possible, and +otherwise maintains the original query order. When reordering joins, it also strives to maintain the +original table order as much as possible. `AUTOMATIC` enumerates possible orders, and uses +statistics-based cost estimation to determine the least cost order. If stats are not available, or if +for any reason a cost could not be computed, the `ELIMINATE_CROSS_JOINS` strategy is used. + +## `optimizer.max-reordered-joins` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** `9` + +When optimizer.join-reordering-strategy is set to cost-based, this property determines +the maximum number of joins that can be reordered at once. + +:::{warning} +The number of possible join orders scales factorially with the number of +relations, so increasing this value can cause serious performance issues. +::: + +## `optimizer.optimize-duplicate-insensitive-joins` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `true` + +Reduces number of rows produced by joins when optimizer detects that duplicated +join output rows can be skipped. + +## `optimizer.use-exact-partitioning` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `false` + +Re-partition data unless the partitioning of the upstream +{ref}`stage ` exactly matches what the downstream stage +expects. This can also be specified using the `use_exact_partitioning` session +property. + +## `optimizer.use-table-scan-node-partitioning` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `true` + +Use connector provided table node partitioning when reading tables. +For example, table node partitioning corresponds to Hive table buckets. +When set to `true` and minimal partition to task ratio is matched or exceeded, +each table partition is read by a separate worker. The minimal ratio is defined in +`optimizer.table-scan-node-partitioning-min-bucket-to-task-ratio`. + +Partition reader assignments are distributed across workers for +parallel processing. Use of table scan node partitioning can improve +query performance by reducing query complexity. For example, +cluster wide data reshuffling might not be needed when processing an aggregation query. +However, query parallelism might be reduced when partition count is +low compared to number of workers. + +## `optimizer.table-scan-node-partitioning-min-bucket-to-task-ratio` + +- **Type:** {ref}`prop-type-double` +- **Default value:** `0.5` + +Specifies minimal bucket to task ratio that has to be matched or exceeded in order +to use table scan node partitioning. When the table bucket count is small +compared to the number of workers, then the table scan is distributed across +all workers for improved parallelism. + +## `optimizer.colocated-joins-enabled` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `true` +- **Session property:** `colocated_join` + +Use co-located joins when both sides of a join have the same table partitioning on the join keys +and the conditions for `optimizer.use-table-scan-node-partitioning` are met. +For example, a join on bucketed Hive tables with matching bucketing schemes can +avoid exchanging data between workers using a co-located join to improve query performance. + +## `optimizer.filter-conjunction-independence-factor` + +- **Type:** {ref}`prop-type-double` +- **Default value:** `0.75` +- **Min allowed value:** `0` +- **Max allowed value:** `1` + +Scales the strength of independence assumption for estimating the selectivity of +the conjunction of multiple predicates. Lower values for this property will produce +more conservative estimates by assuming a greater degree of correlation between the +columns of the predicates in a conjunction. A value of `0` results in the +optimizer assuming that the columns of the predicates are fully correlated and only +the most selective predicate drives the selectivity of a conjunction of predicates. + +## `optimizer.join-multi-clause-independence-factor` + +- **Type:** {ref}`prop-type-double` +- **Default value:** `0.25` +- **Min allowed value:** `0` +- **Max allowed value:** `1` + +Scales the strength of independence assumption for estimating the output of a +multi-clause join. Lower values for this property will produce more +conservative estimates by assuming a greater degree of correlation between the +columns of the clauses in a join. A value of `0` results in the optimizer +assuming that the columns of the join clauses are fully correlated and only +the most selective clause drives the selectivity of the join. + +## `optimizer.non-estimatable-predicate-approximation.enabled` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `true` + +Enables approximation of the output row count of filters whose costs cannot be +accurately estimated even with complete statistics. This allows the optimizer to +produce more efficient plans in the presence of filters which were previously +not estimated. + +## `optimizer.join-partitioned-build-min-row-count` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** `1000000` +- **Min allowed value:** `0` + +The minimum number of join build side rows required to use partitioned join lookup. +If the build side of a join is estimated to be smaller than the configured threshold, +single threaded join lookup is used to improve join performance. +A value of `0` disables this optimization. + +## `optimizer.min-input-size-per-task` + +- **Type:** {ref}`prop-type-data-size` +- **Default value:** `5GB` +- **Min allowed value:** `0MB` +- **Session property:** `min_input_size_per_task` + +The minimum input size required per task. This will help optimizer to determine hash +partition count for joins and aggregations. Limiting hash partition count for small queries +increases concurrency on large clusters where multiple small queries are running concurrently. +The estimated value will always be between `min_hash_partition_count` and +`max_hash_partition_count` session property. +A value of `0MB` disables this optimization. + +## `optimizer.min-input-rows-per-task` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** `10000000` +- **Min allowed value:** `0` +- **Session property:** `min_input_rows_per_task` + +The minimum number of input rows required per task. This will help optimizer to determine hash +partition count for joins and aggregations. Limiting hash partition count for small queries +increases concurrency on large clusters where multiple small queries are running concurrently. +The estimated value will always be between `min_hash_partition_count` and +`max_hash_partition_count` session property. +A value of `0` disables this optimization. + +## `optimizer.use-cost-based-partitioning` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `true` +- **Session property:** `use_cost_based_partitioning` + +When enabled the cost based optimizer is used to determine if repartitioning the output of an +already partitioned stage is necessary. diff --git a/430/_sources/admin/properties-query-management.md.txt b/430/_sources/admin/properties-query-management.md.txt new file mode 100644 index 000000000..6f1e95ad9 --- /dev/null +++ b/430/_sources/admin/properties-query-management.md.txt @@ -0,0 +1,278 @@ +# Query management properties + +## `query.client.timeout` + +- **Type:** {ref}`prop-type-duration` +- **Default value:** `5m` + +Configures how long the cluster runs without contact from the client +application, such as the CLI, before it abandons and cancels its work. + +## `query.execution-policy` + +- **Type:** {ref}`prop-type-string` +- **Default value:** `phased` +- **Session property:** `execution_policy` + +Configures the algorithm to organize the processing of all of the +stages of a query. You can use the following execution policies: + +- `phased` schedules stages in a sequence to avoid blockages because of + inter-stage dependencies. This policy maximizes cluster resource utilization + and provides the lowest query wall time. +- `all-at-once` schedules all of the stages of a query at one time. As a + result, cluster resource utilization is initially high, but inter-stage + dependencies typically prevent full processing and cause longer queue times + which increases the query wall time overall. + +## `query.determine-partition-count-for-write-enabled` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `false` +- **Session property:** `determine_partition_count_for_write_enabled` + +Enables determining the number of partitions based on amount of data read and processed by the +query for write queries. + +## `query.max-hash-partition-count` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** `100` +- **Session property:** `max_hash_partition_count` + +The maximum number of partitions to use for processing distributed operations, such as +joins, aggregations, partitioned window functions and others. + +## `query.min-hash-partition-count` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** `4` +- **Session property:** `min_hash_partition_count` + +The minimum number of partitions to use for processing distributed operations, such as +joins, aggregations, partitioned window functions and others. + +## `query.min-hash-partition-count-for-write` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** `50` +- **Session property:** `min_hash_partition_count_for_writre` + +The minimum number of partitions to use for processing distributed operations in write queries, +such as joins, aggregations, partitioned window functions and others. + +## `query.max-writer-tasks-count` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** `100` +- **Session property:** `max_writer_tasks_count` + +The maximum number of tasks that will take part in writing data during +`INSERT`, `CREATE TABLE AS SELECT` and `EXECUTE` queries. +The limit is only applicable when `redistribute-writes` or `scale-writers` is be enabled. + +## `query.low-memory-killer.policy` + +- **Type:** {ref}`prop-type-string` +- **Default value:** `total-reservation-on-blocked-nodes` + +Configures the behavior to handle killing running queries in the event of low +memory availability. Supports the following values: + +- `none` - Do not kill any queries in the event of low memory. +- `total-reservation` - Kill the query currently using the most total memory. +- `total-reservation-on-blocked-nodes` - Kill the query currently using the + most memory specifically on nodes that are now out of memory. + +:::{note} +Only applies for queries with task level retries disabled (`retry-policy` set to `NONE` or `QUERY`) +::: + +## `task.low-memory-killer.policy` + +- **Type:** {ref}`prop-type-string` +- **Default value:** `total-reservation-on-blocked-nodes` + +Configures the behavior to handle killing running tasks in the event of low +memory availability. Supports the following values: + +- `none` - Do not kill any tasks in the event of low memory. +- `total-reservation-on-blocked-nodes` - Kill the tasks which are part of the queries + which has task retries enabled and are currently using the most memory specifically + on nodes that are now out of memory. +- `least-waste` - Kill the tasks which are part of the queries + which has task retries enabled and use significant amount of memory on nodes + which are now out of memory. This policy avoids killing tasks which are already + executing for a long time, so significant amount of work is not wasted. + +:::{note} +Only applies for queries with task level retries enabled (`retry-policy=TASK`) +::: + +## `query.low-memory-killer.delay` + +- **Type:** {ref}`prop-type-duration` +- **Default value:** `5m` + +The amount of time a query is allowed to recover between running out of memory +and being killed, if `query.low-memory-killer.policy` or +`task.low-memory-killer.policy` is set to value differnt than `none`. + +## `query.max-execution-time` + +- **Type:** {ref}`prop-type-duration` +- **Default value:** `100d` +- **Session property:** `query_max_execution_time` + +The maximum allowed time for a query to be actively executing on the +cluster, before it is terminated. Compared to the run time below, execution +time does not include analysis, query planning or wait times in a queue. + +## `query.max-length` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** `1,000,000` +- **Maximum value:** `1,000,000,000` + +The maximum number of characters allowed for the SQL query text. Longer queries +are not processed, and terminated with error `QUERY_TEXT_TOO_LARGE`. + +## `query.max-planning-time` + +- **Type:** {ref}`prop-type-duration` +- **Default value:** `10m` +- **Session property:** `query_max_planning_time` + +The maximum allowed time for a query to be actively planning the execution. +After this period the coordinator will make its best effort to stop the +query. Note that some operations in planning phase are not easily cancellable +and may not terminate immediately. + +## `query.max-run-time` + +- **Type:** {ref}`prop-type-duration` +- **Default value:** `100d` +- **Session property:** `query_max_run_time` + +The maximum allowed time for a query to be processed on the cluster, before +it is terminated. The time includes time for analysis and planning, but also +time spend in a queue waiting, so essentially this is the time allowed for a +query to exist since creation. + +## `query.max-scan-physical-bytes` + +- **Type:** {ref}`prop-type-data-size` +- **Session property:** `query_max_scan_physical_bytes` + +The maximum number of bytes that can be scanned by a query during its execution. +When this limit is reached, query processing is terminated to prevent excessive +resource usage. + +## `query.max-stage-count` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** `150` +- **Minimum value:** `1` + +The maximum number of stages allowed to be generated per query. If a query +generates more stages than this it will get killed with error +`QUERY_HAS_TOO_MANY_STAGES`. + +:::{warning} +Setting this to a high value can cause queries with large number of +stages to introduce instability in the cluster causing unrelated queries +to get killed with `REMOTE_TASK_ERROR` and the message +`Max requests queued per destination exceeded for HttpDestination ...` +::: + +## `query.max-history` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** `100` + +The maximum number of queries to keep in the query history to provide +statistics and other information. If this amount is reached, queries are +removed based on age. + +## `query.min-expire-age` + +- **Type:** {ref}`prop-type-duration` +- **Default value:** `15m` + +The minimal age of a query in the history before it is expired. An expired +query is removed from the query history buffer and no longer available in +the {doc}`/admin/web-interface`. + +## `query.remote-task.enable-adaptive-request-size` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `true` +- **Session property:** `query_remote_task_enable_adaptive_request_size` + +Enables dynamically splitting up server requests sent by tasks, which can +prevent out-of-memory errors for large schemas. The default settings are +optimized for typical usage and should only be modified by advanced users +working with extremely large tables. + +## `query.remote-task.guaranteed-splits-per-task` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** `3` +- **Session property:** `query_remote_task_guaranteed_splits_per_task` + +The minimum number of splits that should be assigned to each remote task to +ensure that each task has a minimum amount of work to perform. Requires +`query.remote-task.enable-adaptive-request-size` to be enabled. + +## `query.remote-task.max-error-duration` + +- **Type:** {ref}`prop-type-duration` +- **Default value:** `5m` + +Timeout value for remote tasks that fail to communicate with the coordinator. If +the coordinator is unable to receive updates from a remote task before this +value is reached, the coordinator treats the task as failed. + +## `query.remote-task.max-request-size` + +- **Type:** {ref}`prop-type-data-size` +- **Default value:** `8MB` +- **Session property:** `query_remote_task_max_request_size` + +The maximum size of a single request made by a remote task. Requires +`query.remote-task.enable-adaptive-request-size` to be enabled. + +## `query.remote-task.request-size-headroom` + +- **Type:** {ref}`prop-type-data-size` +- **Default value:** `2MB` +- **Session property:** `query_remote_task_request_size_headroom` + +Determines the amount of headroom that should be allocated beyond the size of +the request data. Requires `query.remote-task.enable-adaptive-request-size` to +be enabled. + +## `query.info-url-template` + +- **Type:** {ref}`prop-type-string` +- **Default value:** `(URL of the query info page on the coordinator)` + +Configure redirection of clients to an alternative location for query +information. The URL must contain a query id placeholder `${QUERY_ID}`. + +For example `https://example.com/query/${QUERY_ID}`. + +The `${QUERY_ID}` gets replaced with the actual query's id. + +## `retry-policy` + +- **Type:** {ref}`prop-type-string` +- **Default value:** `NONE` + +The {ref}`retry policy ` to use for +{doc}`/admin/fault-tolerant-execution`. Supports the following values: + +- `NONE` - Disable fault-tolerant execution. +- `TASK` - Retry individual tasks within a query in the event of failure. + Requires configuration of an {ref}`exchange manager `. +- `QUERY` - Retry the whole query in the event of failure. diff --git a/430/_sources/admin/properties-regexp-function.md.txt b/430/_sources/admin/properties-regexp-function.md.txt new file mode 100644 index 000000000..caaa02897 --- /dev/null +++ b/430/_sources/admin/properties-regexp-function.md.txt @@ -0,0 +1,42 @@ +# Regular expression function properties + +These properties allow tuning the {doc}`/functions/regexp`. + +## `regex-library` + +- **Type:** {ref}`prop-type-string` +- **Allowed values:** `JONI`, `RE2J` +- **Default value:** `JONI` + +Which library to use for regular expression functions. +`JONI` is generally faster for common usage, but can require exponential +time for certain expression patterns. `RE2J` uses a different algorithm, +which guarantees linear time, but is often slower. + +## `re2j.dfa-states-limit` + +- **Type:** {ref}`prop-type-integer` +- **Minimum value:** `2` +- **Default value:** `2147483647` + +The maximum number of states to use when RE2J builds the fast, +but potentially memory intensive, deterministic finite automaton (DFA) +for regular expression matching. If the limit is reached, RE2J falls +back to the algorithm that uses the slower, but less memory intensive +non-deterministic finite automaton (NFA). Decreasing this value decreases the +maximum memory footprint of a regular expression search at the cost of speed. + +## `re2j.dfa-retries` + +- **Type:** {ref}`prop-type-integer` +- **Minimum value:** `0` +- **Default value:** `5` + +The number of times that RE2J retries the DFA algorithm, when +it reaches a states limit before using the slower, but less memory +intensive NFA algorithm, for all future inputs for that search. If hitting the +limit for a given input row is likely to be an outlier, you want to be able +to process subsequent rows using the faster DFA algorithm. If you are likely +to hit the limit on matches for subsequent rows as well, you want to use the +correct algorithm from the beginning so as not to waste time and resources. +The more rows you are processing, the larger this value should be. diff --git a/430/_sources/admin/properties-resource-management.md.txt b/430/_sources/admin/properties-resource-management.md.txt new file mode 100644 index 000000000..9c6bb21ba --- /dev/null +++ b/430/_sources/admin/properties-resource-management.md.txt @@ -0,0 +1,105 @@ +# Resource management properties + +(prop-resource-query-max-cpu-time)= + +## `query.max-cpu-time` + +- **Type:** {ref}`prop-type-duration` +- **Default value:** `1_000_000_000d` + +This is the max amount of CPU time that a query can use across the entire +cluster. Queries that exceed this limit are killed. + +(prop-resource-query-max-memory-per-node)= + +## `query.max-memory-per-node` + +- **Type:** {ref}`prop-type-data-size` +- **Default value:** (JVM max memory * 0.3) + +This is the max amount of user memory a query can use on a worker. +User memory is allocated during execution for things that are directly +attributable to, or controllable by, a user query. For example, memory used +by the hash tables built during execution, memory used during sorting, etc. +When the user memory allocation of a query on any worker hits this limit, +it is killed. + +:::{warning} +The sum of {ref}`prop-resource-query-max-memory-per-node` and +{ref}`prop-resource-memory-heap-headroom-per-node` must be less than the +maximum heap size in the JVM on the node. See {ref}`jvm-config`. +::: + +:::{note} +Does not apply for queries with task level retries enabled (`retry-policy=TASK`) +::: + +(prop-resource-query-max-memory)= + +## `query.max-memory` + +- **Type:** {ref}`prop-type-data-size` +- **Default value:** `20GB` + +This is the max amount of user memory a query can use across the entire cluster. +User memory is allocated during execution for things that are directly +attributable to, or controllable by, a user query. For example, memory used +by the hash tables built during execution, memory used during sorting, etc. +When the user memory allocation of a query across all workers hits this limit +it is killed. + +:::{warning} +{ref}`prop-resource-query-max-total-memory` must be greater than +{ref}`prop-resource-query-max-memory`. +::: + +:::{note} +Does not apply for queries with task level retries enabled (`retry-policy=TASK`) +::: + +(prop-resource-query-max-total-memory)= + +## `query.max-total-memory` + +- **Type:** {ref}`prop-type-data-size` +- **Default value:** (`query.max-memory` * 2) + +This is the max amount of memory a query can use across the entire cluster, +including revocable memory. When the memory allocated by a query across all +workers hits this limit it is killed. The value of `query.max-total-memory` +must be greater than `query.max-memory`. + +:::{warning} +{ref}`prop-resource-query-max-total-memory` must be greater than +{ref}`prop-resource-query-max-memory`. +::: + +:::{note} +Does not apply for queries with task level retries enabled (`retry-policy=TASK`) +::: + +(prop-resource-memory-heap-headroom-per-node)= + +## `memory.heap-headroom-per-node` + +- **Type:** {ref}`prop-type-data-size` +- **Default value:** (JVM max memory * 0.3) + +This is the amount of memory set aside as headroom/buffer in the JVM heap +for allocations that are not tracked by Trino. + +:::{warning} +The sum of {ref}`prop-resource-query-max-memory-per-node` and +{ref}`prop-resource-memory-heap-headroom-per-node` must be less than the +maximum heap size in the JVM on the node. See {ref}`jvm-config`. +::: + +(prop-resource-exchange-deduplication-buffer-size)= + +## `exchange.deduplication-buffer-size` + +- **Type:** {ref}`prop-type-data-size` +- **Default value:** `32MB` + +Size of the buffer used for spooled data during +{doc}`/admin/fault-tolerant-execution`. diff --git a/430/_sources/admin/properties-spilling.md.txt b/430/_sources/admin/properties-spilling.md.txt new file mode 100644 index 000000000..62b4ad71d --- /dev/null +++ b/430/_sources/admin/properties-spilling.md.txt @@ -0,0 +1,81 @@ +# Spilling properties + +These properties control {doc}`spill`. + +## `spill-enabled` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `false` +- **Session property:** `spill_enabled` + +Try spilling memory to disk to avoid exceeding memory limits for the query. + +Spilling works by offloading memory to disk. This process can allow a query with a large memory +footprint to pass at the cost of slower execution times. Spilling is supported for +aggregations, joins (inner and outer), sorting, and window functions. This property does not +reduce memory usage required for other join types. + +## `spiller-spill-path` + +- **Type:** {ref}`prop-type-string` +- **No default value.** Must be set when spilling is enabled + +Directory where spilled content is written. It can be a comma separated +list to spill simultaneously to multiple directories, which helps to utilize +multiple drives installed in the system. + +It is not recommended to spill to system drives. Most importantly, do not spill +to the drive on which the JVM logs are written, as disk overutilization might +cause JVM to pause for lengthy periods, causing queries to fail. + +## `spiller-max-used-space-threshold` + +- **Type:** {ref}`prop-type-double` +- **Default value:** `0.9` + +If disk space usage ratio of a given spill path is above this threshold, +this spill path is not eligible for spilling. + +## `spiller-threads` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** `4` + +Number of spiller threads. Increase this value if the default is not able +to saturate the underlying spilling device (for example, when using RAID). + +## `max-spill-per-node` + +- **Type:** {ref}`prop-type-data-size` +- **Default value:** `100GB` + +Max spill space to be used by all queries on a single node. + +## `query-max-spill-per-node` + +- **Type:** {ref}`prop-type-data-size` +- **Default value:** `100GB` + +Max spill space to be used by a single query on a single node. + +## `aggregation-operator-unspill-memory-limit` + +- **Type:** {ref}`prop-type-data-size` +- **Default value:** `4MB` + +Limit for memory used for unspilling a single aggregation operator instance. + +## `spill-compression-enabled` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `false` + +Enables data compression for pages spilled to disk. + +## `spill-encryption-enabled` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `false` + +Enables using a randomly generated secret key (per spill file) to encrypt and decrypt +data spilled to disk. diff --git a/430/_sources/admin/properties-task.md.txt b/430/_sources/admin/properties-task.md.txt new file mode 100644 index 000000000..02f22788c --- /dev/null +++ b/430/_sources/admin/properties-task.md.txt @@ -0,0 +1,180 @@ +# Task properties + +## `task.concurrency` + +- **Type:** {ref}`prop-type-integer` +- **Restrictions:** Must be a power of two +- **Default value:** The number of physical CPUs of the node, with a minimum value of 2 and a maximum of 32 +- **Session property:** `task_concurrency` + +Default local concurrency for parallel operators, such as joins and aggregations. +This value should be adjusted up or down based on the query concurrency and worker +resource utilization. Lower values are better for clusters that run many queries +concurrently, because the cluster is already utilized by all the running +queries, so adding more concurrency results in slow downs due to context +switching and other overhead. Higher values are better for clusters that only run +one or a few queries at a time. + +## `task.http-response-threads` + +- **Type:** {ref}`prop-type-integer` +- **Minimum value:** `1` +- **Default value:** `100` + +Maximum number of threads that may be created to handle HTTP responses. Threads are +created on demand and are cleaned up when idle, thus there is no overhead to a large +value, if the number of requests to be handled is small. More threads may be helpful +on clusters with a high number of concurrent queries, or on clusters with hundreds +or thousands of workers. + +## `task.http-timeout-threads` + +- **Type:** {ref}`prop-type-integer` +- **Minimum value:** `1` +- **Default value:** `3` + +Number of threads used to handle timeouts when generating HTTP responses. This value +should be increased if all the threads are frequently in use. This can be monitored +via the `trino.server:name=AsyncHttpExecutionMBean:TimeoutExecutor` +JMX object. If `ActiveCount` is always the same as `PoolSize`, increase the +number of threads. + +## `task.info-update-interval` + +- **Type:** {ref}`prop-type-duration` +- **Minimum value:** `1ms` +- **Maximum value:** `10s` +- **Default value:** `3s` + +Controls staleness of task information, which is used in scheduling. Larger values +can reduce coordinator CPU load, but may result in suboptimal split scheduling. + +## `task.max-drivers-per-task` + +- **Type:** {ref}`prop-type-integer` +- **Minimum value:** `1` +- **Default Value:** `2147483647` + +Controls the maximum number of drivers a task runs concurrently. Setting this value +reduces the likelihood that a task uses too many drivers and can improve concurrent query +performance. This can lead to resource waste if it runs too few concurrent queries. + +## `task.max-partial-aggregation-memory` + +- **Type:** {ref}`prop-type-data-size` +- **Default value:** `16MB` + +Maximum size of partial aggregation results for distributed aggregations. Increasing this +value can result in less network transfer and lower CPU utilization, by allowing more +groups to be kept locally before being flushed, at the cost of additional memory usage. + +## `task.max-worker-threads` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** (Node CPUs * 2) + +Sets the number of threads used by workers to process splits. Increasing this number +can improve throughput, if worker CPU utilization is low and all the threads are in use, +but it causes increased heap space usage. Setting the value too high may cause a drop +in performance due to a context switching. The number of active threads is available +via the `RunningSplits` property of the +`trino.execution.executor:name=TaskExecutor.RunningSplits` JMX object. + +## `task.min-drivers` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** (`task.max-worker-threads` * 2) + +The target number of running leaf splits on a worker. This is a minimum value because +each leaf task is guaranteed at least `3` running splits. Non-leaf tasks are also +guaranteed to run in order to prevent deadlocks. A lower value may improve responsiveness +for new tasks, but can result in underutilized resources. A higher value can increase +resource utilization, but uses additional memory. + +## `task.min-drivers-per-task` + +- **Type:** {ref}`prop-type-integer` +- **Minimum value:** `1` +- **Default Value:** `3` + +The minimum number of drivers guaranteed to run concurrently for a single task given +the task has remaining splits to process. + +## `task.scale-writers.enabled` + +- **Description:** see details at {ref}`prop-task-scale-writers` + +(prop-task-min-writer-count)= +## `task.min-writer-count` + +- **Type:** {ref}`prop-type-integer` +- **Default value:** `1` +- **Session property:** `task_min_writer_count` + +The number of concurrent writer threads per worker per query when +{ref}`preferred partitioning ` and +{ref}`task writer scaling ` are not used. Increasing this value may +increase write speed, especially when a query is not I/O bound and can take advantage of +additional CPU for parallel writes. + +Some connectors can be bottlenecked on the CPU when writing due to compression or other factors. +Setting this too high may cause the cluster to become overloaded due to excessive resource +utilization. Especially when the engine is inserting into a partitioned table without using +{ref}`preferred partitioning `. In such case, each writer thread +could write to all partitions. This can lead to out of memory error since writing to a partition +allocates a certain amount of memory for buffering. + +(prop-task-max-writer-count)= +## `task.max-writer-count` + +- **Type:** {ref}`prop-type-integer` +- **Restrictions:** Must be a power of two +- **Default value:** The number of physical CPUs of the node, with a minimum value of 2 and a maximum of 64 +- **Session property:** `task_max_writer_count` + +The number of concurrent writer threads per worker per query when either +{ref}`task writer scaling ` or +{ref}`preferred partitioning ` is used. Increasing this value may +increase write speed, especially when a query is not I/O bound and can take advantage of additional +CPU for parallel writes. Some connectors can be bottlenecked on CPU when writing due to compression +or other factors. Setting this too high may cause the cluster to become overloaded due to excessive +resource utilization. + +## `task.interrupt-stuck-split-tasks-enabled` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `true` + +Enables Trino detecting and failing tasks containing splits that have been stuck. Can be +specified by `task.interrupt-stuck-split-tasks-timeout` and +`task.interrupt-stuck-split-tasks-detection-interval`. Only applies to threads that +are blocked by the third-party Joni regular expression library. + +## `task.interrupt-stuck-split-tasks-warning-threshold` + +- **Type:** {ref}`prop-type-duration` +- **Minimum value:** `1m` +- **Default value:** `10m` + +Print out call stacks at `/v1/maxActiveSplits` endpoint and generate JMX metrics +for splits running longer than the threshold. + +## `task.interrupt-stuck-split-tasks-timeout` + +- **Type:** {ref}`prop-type-duration` +- **Minimum value:** `3m` +- **Default value:** `10m` + +The length of time Trino waits for a blocked split processing thread before failing the +task. Only applies to threads that are blocked by the third-party Joni regular +expression library. + +## `task.interrupt-stuck-split-tasks-detection-interval` + +- **Type:** {ref}`prop-type-duration` +- **Minimum value:** `1m` +- **Default value:** `2m` + +The interval of Trino checks for splits that have processing time exceeding +`task.interrupt-stuck-split-tasks-timeout`. Only applies to threads that are blocked +by the third-party Joni regular expression library. diff --git a/430/_sources/admin/properties-web-interface.md.txt b/430/_sources/admin/properties-web-interface.md.txt new file mode 100644 index 000000000..95c608756 --- /dev/null +++ b/430/_sources/admin/properties-web-interface.md.txt @@ -0,0 +1,45 @@ +# Web UI properties + +The following properties can be used to configure the {doc}`web-interface`. + +## `web-ui.authentication.type` + +- **Type:** {ref}`prop-type-string` +- **Allowed values:** `FORM`, `FIXED`, `CERTIFICATE`, `KERBEROS`, `JWT`, `OAUTH2` +- **Default value:** `FORM` + +The authentication mechanism to allow user access to the Web UI. See +{ref}`Web UI Authentication `. + +## `web-ui.enabled` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `true` + +This property controls whether or not the Web UI is available. + +## `web-ui.shared-secret` + +- **Type:** {ref}`prop-type-string` +- **Default value:** randomly generated unless set + +The shared secret is used to generate authentication cookies for users of +the Web UI. If not set to a static value, any coordinator restart generates +a new random value, which in turn invalidates the session of any currently +logged in Web UI user. + +## `web-ui.session-timeout` + +- **Type:** {ref}`prop-type-duration` +- **Default value:** `1d` + +The duration how long a user can be logged into the Web UI, before the +session times out, which forces an automatic log-out. + +## `web-ui.user` + +- **Type:** {ref}`prop-type-string` +- **Default value:** None + +The username automatically used for authentication to the Web UI with the `fixed` +authentication type. See {ref}`Web UI Authentication `. diff --git a/430/_sources/admin/properties-write-partitioning.md.txt b/430/_sources/admin/properties-write-partitioning.md.txt new file mode 100644 index 000000000..cdd4487c3 --- /dev/null +++ b/430/_sources/admin/properties-write-partitioning.md.txt @@ -0,0 +1,15 @@ +# Write partitioning properties + +(preferred-write-partitioning)= + +## `use-preferred-write-partitioning` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `true` +- **Session property:** `use_preferred_write_partitioning` + +Enable preferred write partitioning. When set to `true`, each partition is +written by a separate writer. For some connectors such as the Hive connector, +only a single new file is written per partition, instead of multiple files. +Partition writer assignments are distributed across worker nodes for parallel +processing. diff --git a/430/_sources/admin/properties-writer-scaling.md.txt b/430/_sources/admin/properties-writer-scaling.md.txt new file mode 100644 index 000000000..2e8bda93c --- /dev/null +++ b/430/_sources/admin/properties-writer-scaling.md.txt @@ -0,0 +1,45 @@ +# Writer scaling properties + +Writer scaling allows Trino to dynamically scale out the number of writer tasks +rather than allocating a fixed number of tasks. Additional tasks are added when +the average amount of physical data per writer is above a minimum threshold, but +only if the query is bottlenecked on writing. + +Writer scaling is useful with connectors like Hive that produce one or more +files per writer -- reducing the number of writers results in a larger average +file size. However, writer scaling can have a small impact on query wall time +due to the decreased writer parallelism while the writer count ramps up to match +the needs of the query. + +## `scale-writers` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `true` +- **Session property:** `scale_writers` + +Enable writer scaling by dynamically increasing the number of writer tasks on +the cluster. + +(prop-task-scale-writers)= + +## `task.scale-writers.enabled` + +- **Type:** {ref}`prop-type-boolean` +- **Default value:** `true` +- **Session property:** `task_scale_writers_enabled` + +Enable scaling the number of concurrent writers within a task. The maximum +writer count per task for scaling is [](prop-task-max-writer-count). Additional +writers are added only when the average amount of uncompressed data processed +per writer is above the minimum threshold of `writer-scaling-min-data-processed` +and query is bottlenecked on writing. + +(writer-scaling-min-data-processed)= +## `writer-scaling-min-data-processed` + +- **Type:** {ref}`prop-type-data-size` +- **Default value:** `100MB` +- **Session property:** `writer_scaling_min_data_processed` + +The minimum amount of uncompressed data that must be processed by a writer +before another writer can be added. diff --git a/430/_sources/admin/properties.md.txt b/430/_sources/admin/properties.md.txt new file mode 100644 index 000000000..88c2ad833 --- /dev/null +++ b/430/_sources/admin/properties.md.txt @@ -0,0 +1,114 @@ +# Properties reference + +This section describes the most important configuration properties and (where +applicable) their corresponding {ref}`session properties +`, that may be used to tune Trino or alter its +behavior when required. Unless specified otherwise, configuration properties +must be set on the coordinator and all worker nodes. + +The following pages are not a complete list of all configuration and +session properties available in Trino, and do not include any connector-specific +catalog configuration properties. For more information on catalog configuration +properties, refer to the {doc}`connector documentation `. + +```{toctree} +:titlesonly: true + +General +Resource management +Query management +Spilling +Exchange +Task +Write partitioning +Writer scaling +Node scheduler +Optimizer +Logging +Web UI +Regular expression function +HTTP client +``` + +## Property value types + +Trino configuration properties support different value types with their own +allowed values and syntax. Additional limitations apply on a per-property basis, +and disallowed values result in a validation error. + +(prop-type-boolean)= + +### `boolean` + +The properties of type `boolean` support two values, `true` or `false`. + +(prop-type-data-size)= + +### `data size` + +The properties of type `data size` support values that describe an amount of +data, measured in byte-based units. These units are incremented in multiples of +1024, so one megabyte is 1024 kilobytes, one kilobyte is 1024 bytes, and so on. +For example, the value `6GB` describes six gigabytes, which is +(6 * 1024 * 1024 * 1024) = 6442450944 bytes. + +The `data size` type supports the following units: + +- `B`: Bytes +- `kB`: Kilobytes +- `MB`: Megabytes +- `GB`: Gigabytes +- `TB`: Terabytes +- `PB`: Petabytes + +(prop-type-double)= + +### `double` + +The properties of type `double` support numerical values including decimals, +such as `1.6`. `double` type values can be negative, if supported by the +specific property. + +(prop-type-duration)= + +### `duration` + +The properties of type `duration` support values describing an +amount of time, using the syntax of a non-negative number followed by a time +unit. For example, the value `7m` describes seven minutes. + +The `duration` type supports the following units: + +- `ns`: Nanoseconds +- `us`: Microseconds +- `ms`: Milliseconds +- `s`: Seconds +- `m`: Minutes +- `h`: Hours +- `d`: Days + +A duration of `0` is treated as zero regardless of the unit that follows. +For example, `0s` and `0m` both mean the same thing. + +Properties of type `duration` also support decimal values, such as `2.25d`. +These are handled as a fractional value of the specified unit. For example, the +value `1.5m` equals one and a half minutes, or 90 seconds. + +(prop-type-integer)= + +### `integer` + +The properties of type `integer` support whole numeric values, such as `5` +and `1000`. Negative values are supported as well, for example `-7`. +`integer` type values must be whole numbers, decimal values such as `2.5` +are not supported. + +Some `integer` type properties enforce their own minimum and maximum values. + +(prop-type-string)= + +### `string` + +The properties of type `string` support a set of values that consist of a +sequence of characters. Allowed values are defined on a property-by-property +basis, refer to the specific property for its supported and default values. diff --git a/430/_sources/admin/resource-groups.md.txt b/430/_sources/admin/resource-groups.md.txt new file mode 100644 index 000000000..241b5b552 --- /dev/null +++ b/430/_sources/admin/resource-groups.md.txt @@ -0,0 +1,343 @@ +# Resource groups + +Resource groups place limits on resource usage, and can enforce queueing policies on +queries that run within them, or divide their resources among sub-groups. A query +belongs to a single resource group, and consumes resources from that group (and its ancestors). +Except for the limit on queued queries, when a resource group runs out of a resource +it does not cause running queries to fail; instead new queries become queued. +A resource group may have sub-groups or may accept queries, but may not do both. + +The resource groups and associated selection rules are configured by a manager, which is pluggable. + +You can use a file-based or a database-based resource group manager: + +- Add a file `etc/resource-groups.properties` +- Set the `resource-groups.configuration-manager` property to `file` or `db` +- Add further configuration properties for the desired manager. + +## File resource group manager + +The file resource group manager reads a JSON configuration file, specified with +`resource-groups.config-file`: + +```text +resource-groups.configuration-manager=file +resource-groups.config-file=etc/resource-groups.json +``` + +The path to the JSON file can be an absolute path, or a path relative to the Trino +data directory. The JSON file only needs to be present on the coordinator. + +## Database resource group manager + +The database resource group manager loads the configuration from a relational database. The +supported databases are MySQL, PostgreSQL, and Oracle. + +```text +resource-groups.configuration-manager=db +resource-groups.config-db-url=jdbc:mysql://localhost:3306/resource_groups +resource-groups.config-db-user=username +resource-groups.config-db-password=password +``` + +The resource group configuration must be populated through tables +`resource_groups_global_properties`, `resource_groups`, and +`selectors`. If any of the tables do not exist when Trino starts, they +will be created automatically. + +The rules in the `selectors` table are processed in descending order of the +values in the `priority` field. + +The `resource_groups` table also contains an `environment` field which is +matched with the value contained in the `node.environment` property in +{ref}`node-properties`. This allows the resource group configuration for different +Trino clusters to be stored in the same database if required. + +The configuration is reloaded from the database every second, and the changes +are reflected automatically for incoming queries. + +:::{list-table} Database resource group manager properties +:widths: 40, 50, 10 +:header-rows: 1 + +* - Property name + - Description + - Default value +* - `resource-groups.config-db-url` + - Database URL to load configuration from. + - `none` +* - `resource-groups.config-db-user` + - Database user to connect with. + - `none` +* - `resource-groups.config-db-password` + - Password for database user to connect with. + - `none` +* - `resource-groups.max-refresh-interval` + - The maximum time period for which the cluster will continue to accept + queries after refresh failures, causing configuration to become stale. + - `1h` +* - `resource-groups.refresh-interval` + - How often the cluster reloads from the database + - `1s` +* - `resource-groups.exact-match-selector-enabled` + - Setting this flag enables usage of an additional + `exact_match_source_selectors` table to configure resource group selection + rules defined exact name based matches for source, environment and query + type. By default, the rules are only loaded from the `selectors` table, with + a regex-based filter for `source`, among other filters. + - `false` +::: + +## Resource group properties + +- `name` (required): name of the group. May be a template (see below). + +- `maxQueued` (required): maximum number of queued queries. Once this limit is reached + new queries are rejected. + +- `softConcurrencyLimit` (optional): number of concurrently running queries after which + new queries will only run if all peer resource groups below their soft limits are ineligible + or if all eligible peers are above soft limits. + +- `hardConcurrencyLimit` (required): maximum number of running queries. + +- `softMemoryLimit` (required): maximum amount of distributed memory this + group may use, before new queries become queued. May be specified as + an absolute value (i.e. `1GB`) or as a percentage (i.e. `10%`) of the cluster's memory. + +- `softCpuLimit` (optional): maximum amount of CPU time this + group may use in a period (see `cpuQuotaPeriod`), before a penalty is applied to + the maximum number of running queries. `hardCpuLimit` must also be specified. + +- `hardCpuLimit` (optional): maximum amount of CPU time this + group may use in a period. + +- `schedulingPolicy` (optional): specifies how queued queries are selected to run, + and how sub-groups become eligible to start their queries. May be one of three values: + + - `fair` (default): queued queries are processed first-in-first-out, and sub-groups + must take turns starting new queries, if they have any queued. + - `weighted_fair`: sub-groups are selected based on their `schedulingWeight` and the number of + queries they are already running concurrently. The expected share of running queries for a + sub-group is computed based on the weights for all currently eligible sub-groups. The sub-group + with the least concurrency relative to its share is selected to start the next query. + - `weighted`: queued queries are selected stochastically in proportion to their priority, + specified via the `query_priority` {doc}`session property `. Sub groups are selected + to start new queries in proportion to their `schedulingWeight`. + - `query_priority`: all sub-groups must also be configured with `query_priority`. + Queued queries are selected strictly according to their priority. + +- `schedulingWeight` (optional): weight of this sub-group used in `weighted` + and the `weighted_fair` scheduling policy. Defaults to `1`. See + {ref}`scheduleweight-example`. + +- `jmxExport` (optional): If true, group statistics are exported to JMX for monitoring. + Defaults to `false`. + +- `subGroups` (optional): list of sub-groups. + +(scheduleweight-example)= + +### Scheduling weight example + +Schedule weighting is a method of assigning a priority to a resource. Sub-groups +with a higher scheduling weight are given higher priority. For example, to +ensure timely execution of scheduled pipelines queries, weight them higher than +adhoc queries. + +In the following example, pipeline queries are weighted with a value of `350`, +which is higher than the adhoc queries that have a scheduling weight of `150`. +This means that approximately 70% (350 out of 500 queries) of your queries come +from the pipeline sub-group, and 30% (150 out of 500 queries) come from the adhoc +sub-group in a given timeframe. Alternatively, if you set each sub-group value to +`1`, the weight of the queries for the pipeline and adhoc sub-groups are split +evenly and each receive 50% of the queries in a given timeframe. + +```{literalinclude} schedule-weight-example.json +:language: text +``` + +## Selector rules + +- `user` (optional): regex to match against user name. + +- `userGroup` (optional): regex to match against every user group the user belongs to. + +- `source` (optional): regex to match against source string. + +- `queryType` (optional): string to match against the type of the query submitted: + + - `SELECT`: `SELECT` queries. + - `EXPLAIN`: `EXPLAIN` queries (but not `EXPLAIN ANALYZE`). + - `DESCRIBE`: `DESCRIBE`, `DESCRIBE INPUT`, `DESCRIBE OUTPUT`, and `SHOW` queries. + - `INSERT`: `INSERT`, `CREATE TABLE AS`, and `REFRESH MATERIALIZED VIEW` queries. + - `UPDATE`: `UPDATE` queries. + - `DELETE`: `DELETE` queries. + - `ANALYZE`: `ANALYZE` queries. + - `DATA_DEFINITION`: Queries that alter/create/drop the metadata of schemas/tables/views, + and that manage prepared statements, privileges, sessions, and transactions. + +- `clientTags` (optional): list of tags. To match, every tag in this list must be in the list of + client-provided tags associated with the query. + +- `group` (required): the group these queries will run in. + +Selectors are processed sequentially and the first one that matches will be used. + +## Global properties + +- `cpuQuotaPeriod` (optional): the period in which cpu quotas are enforced. + +## Providing selector properties + +The source name can be set as follows: + +- CLI: use the `--source` option. +- JDBC driver when used in client apps: add the `source` property to the + connection configuration and set the value when using a Java application that + uses the JDBC Driver. +- JDBC driver used with Java programs: add a property with the key `source` + and the value on the `Connection` instance as shown in {ref}`the example + `. + +Client tags can be set as follows: + +- CLI: use the `--client-tags` option. +- JDBC driver when used in client apps: add the `clientTags` property to the + connection configuration and set the value when using a Java application that + uses the JDBC Driver. +- JDBC driver used with Java programs: add a property with the key + `clientTags` and the value on the `Connection` instance as shown in + {ref}`the example `. + +## Example + +In the example configuration below, there are several resource groups, some of which are templates. +Templates allow administrators to construct resource group trees dynamically. For example, in +the `pipeline_${USER}` group, `${USER}` is expanded to the name of the user that submitted +the query. `${SOURCE}` is also supported, which is expanded to the source that submitted the +query. You may also use custom named variables in the `source` and `user` regular expressions. + +There are four selectors, that define which queries run in which resource group: + +- The first selector matches queries from `bob` and places them in the admin group. +- The second selector matches queries from `admin` user group and places them in the admin group. +- The third selector matches all data definition (DDL) queries from a source name that includes `pipeline` + and places them in the `global.data_definition` group. This could help reduce queue times for this + class of queries, since they are expected to be fast. +- The fourth selector matches queries from a source name that includes `pipeline`, and places them in a + dynamically-created per-user pipeline group under the `global.pipeline` group. +- The fifth selector matches queries that come from BI tools which have a source matching the regular + expression `jdbc#(?.*)` and have client provided tags that are a superset of `hipri`. + These are placed in a dynamically-created sub-group under the `global.adhoc` group. + The dynamic sub-groups are created based on the values of named variables `toolname` and `user`. + The values are derived from the source regular expression and the query user respectively. + Consider a query with a source `jdbc#powerfulbi`, user `kayla`, and client tags `hipri` and `fast`. + This query is routed to the `global.adhoc.bi-powerfulbi.kayla` resource group. +- The last selector is a catch-all, which places all queries that have not yet been matched into a per-user + adhoc group. + +Together, these selectors implement the following policy: + +- The user `bob` and any user belonging to user group `admin` + is an admin and can run up to 50 concurrent queries. + Queries will be run based on user-provided priority. + +For the remaining users: + +- No more than 100 total queries may run concurrently. +- Up to 5 concurrent DDL queries with a source `pipeline` can run. Queries are run in FIFO order. +- Non-DDL queries will run under the `global.pipeline` group, with a total concurrency of 45, and a per-user + concurrency of 5. Queries are run in FIFO order. +- For BI tools, each tool can run up to 10 concurrent queries, and each user can run up to 3. If the total demand + exceeds the limit of 10, the user with the fewest running queries gets the next concurrency slot. This policy + results in fairness when under contention. +- All remaining queries are placed into a per-user group under `global.adhoc.other` that behaves similarly. + +### File resource group manager + +```{literalinclude} resource-groups-example.json +:language: json +``` + +### Database resource group manager + +This example is for a MySQL database. + +```sql +-- global properties +INSERT INTO resource_groups_global_properties (name, value) VALUES ('cpu_quota_period', '1h'); + +-- Every row in resource_groups table indicates a resource group. +-- The enviroment name is 'test_environment', make sure it matches `node.environment` in your cluster. +-- The parent-child relationship is indicated by the ID in 'parent' column. + +-- create a root group 'global' with NULL parent +INSERT INTO resource_groups (name, soft_memory_limit, hard_concurrency_limit, max_queued, scheduling_policy, jmx_export, environment) VALUES ('global', '80%', 100, 1000, 'weighted', true, 'test_environment'); + +-- get ID of 'global' group +SELECT resource_group_id FROM resource_groups WHERE name = 'global'; -- 1 +-- create two new groups with 'global' as parent +INSERT INTO resource_groups (name, soft_memory_limit, hard_concurrency_limit, max_queued, scheduling_weight, environment, parent) VALUES ('data_definition', '10%', 5, 100, 1, 'test_environment', 1); +INSERT INTO resource_groups (name, soft_memory_limit, hard_concurrency_limit, max_queued, scheduling_weight, environment, parent) VALUES ('adhoc', '10%', 50, 1, 10, 'test_environment', 1); + +-- get ID of 'adhoc' group +SELECT resource_group_id FROM resource_groups WHERE name = 'adhoc'; -- 3 +-- create 'other' group with 'adhoc' as parent +INSERT INTO resource_groups (name, soft_memory_limit, hard_concurrency_limit, max_queued, scheduling_weight, scheduling_policy, environment, parent) VALUES ('other', '10%', 2, 1, 10, 'weighted_fair', 'test_environment', 3); + +-- get ID of 'other' group +SELECT resource_group_id FROM resource_groups WHERE name = 'other'; -- 4 +-- create '${USER}' group with 'other' as parent. +INSERT INTO resource_groups (name, soft_memory_limit, hard_concurrency_limit, max_queued, environment, parent) VALUES ('${USER}', '10%', 1, 100, 'test_environment', 4); + +-- create 'bi-${toolname}' group with 'adhoc' as parent +INSERT INTO resource_groups (name, soft_memory_limit, hard_concurrency_limit, max_queued, scheduling_weight, scheduling_policy, environment, parent) VALUES ('bi-${toolname}', '10%', 10, 100, 10, 'weighted_fair', 'test_environment', 3); + +-- get ID of 'bi-${toolname}' group +SELECT resource_group_id FROM resource_groups WHERE name = 'bi-${toolname}'; -- 6 +-- create '${USER}' group with 'bi-${toolname}' as parent. This indicates +-- nested group 'global.adhoc.bi-${toolname}.${USER}', and will have a +-- different ID than 'global.adhoc.other.${USER}' created above. +INSERT INTO resource_groups (name, soft_memory_limit, hard_concurrency_limit, max_queued, environment, parent) VALUES ('${USER}', '10%', 3, 10, 'test_environment', 6); + +-- create 'pipeline' group with 'global' as parent +INSERT INTO resource_groups (name, soft_memory_limit, hard_concurrency_limit, max_queued, scheduling_weight, jmx_export, environment, parent) VALUES ('pipeline', '80%', 45, 100, 1, true, 'test_environment', 1); + +-- get ID of 'pipeline' group +SELECT resource_group_id FROM resource_groups WHERE name = 'pipeline'; -- 8 +-- create 'pipeline_${USER}' group with 'pipeline' as parent +INSERT INTO resource_groups (name, soft_memory_limit, hard_concurrency_limit, max_queued, environment, parent) VALUES ('pipeline_${USER}', '50%', 5, 100, 'test_environment', 8); + +-- create a root group 'admin' with NULL parent +INSERT INTO resource_groups (name, soft_memory_limit, hard_concurrency_limit, max_queued, scheduling_policy, environment, jmx_export) VALUES ('admin', '100%', 50, 100, 'query_priority', 'test_environment', true); + + +-- Selectors + +-- use ID of 'admin' resource group for selector +INSERT INTO selectors (resource_group_id, user_regex, priority) VALUES ((SELECT resource_group_id FROM resource_groups WHERE name = 'admin'), 'bob', 6); + +-- use ID of 'admin' resource group for selector +INSERT INTO selectors (resource_group_id, user_group_regex, priority) VALUES ((SELECT resource_group_id FROM resource_groups WHERE name = 'admin'), 'admin', 5); + +-- use ID of 'global.data_definition' resource group for selector +INSERT INTO selectors (resource_group_id, source_regex, query_type, priority) VALUES ((SELECT resource_group_id FROM resource_groups WHERE name = 'data_definition'), '.*pipeline.*', 'DATA_DEFINITION', 4); + +-- use ID of 'global.pipeline.pipeline_${USER}' resource group for selector +INSERT INTO selectors (resource_group_id, source_regex, priority) VALUES ((SELECT resource_group_id FROM resource_groups WHERE name = 'pipeline_${USER}'), '.*pipeline.*', 3); + +-- get ID of 'global.adhoc.bi-${toolname}.${USER}' resource group by disambiguating group name using parent ID +SELECT A.resource_group_id self_id, B.resource_group_id parent_id, concat(B.name, '.', A.name) name_with_parent +FROM resource_groups A JOIN resource_groups B ON A.parent = B.resource_group_id +WHERE A.name = '${USER}' AND B.name = 'bi-${toolname}'; +-- 7 | 6 | bi-${toolname}.${USER} +INSERT INTO selectors (resource_group_id, source_regex, client_tags, priority) VALUES (7, 'jdbc#(?.*)', '["hipri"]', 2); + +-- get ID of 'global.adhoc.other.${USER}' resource group for by disambiguating group name using parent ID +SELECT A.resource_group_id self_id, B.resource_group_id parent_id, concat(B.name, '.', A.name) name_with_parent +FROM resource_groups A JOIN resource_groups B ON A.parent = B.resource_group_id +WHERE A.name = '${USER}' AND B.name = 'other'; +-- | 5 | 4 | other.${USER} | +INSERT INTO selectors (resource_group_id, priority) VALUES (5, 1); +``` diff --git a/430/_sources/admin/session-property-managers.md.txt b/430/_sources/admin/session-property-managers.md.txt new file mode 100644 index 000000000..456c16ca4 --- /dev/null +++ b/430/_sources/admin/session-property-managers.md.txt @@ -0,0 +1,80 @@ +# Session property managers + +Administrators can add session properties to control the behavior for subsets of their workload. +These properties are defaults, and can be overridden by users, if authorized to do so. Session +properties can be used to control resource usage, enable or disable features, and change query +characteristics. Session property managers are pluggable. + +Add an `etc/session-property-config.properties` file with the following contents to enable +the built-in manager, that reads a JSON config file: + +```text +session-property-config.configuration-manager=file +session-property-manager.config-file=etc/session-property-config.json +``` + +Change the value of `session-property-manager.config-file` to point to a JSON config file, +which can be an absolute path, or a path relative to the Trino data directory. + +This configuration file consists of a list of match rules, each of which specify a list of +conditions that the query must meet, and a list of session properties that should be applied +by default. All matching rules contribute to constructing a list of session properties. Rules +are applied in the order they are specified. Rules specified later in the file override values +for properties that have been previously encountered. + +## Match rules + +- `user` (optional): regex to match against user name. +- `source` (optional): regex to match against source string. +- `queryType` (optional): string to match against the type of the query submitted: + : - `DATA_DEFINITION`: Queries that alter/create/drop the metadata of schemas/tables/views, and that manage + prepared statements, privileges, sessions, and transactions. + - `DELETE`: `DELETE` queries. + - `DESCRIBE`: `DESCRIBE`, `DESCRIBE INPUT`, `DESCRIBE OUTPUT`, and `SHOW` queries. + - `EXPLAIN`: `EXPLAIN` queries. + - `INSERT`: `INSERT` and `CREATE TABLE AS` queries. + - `SELECT`: `SELECT` queries. +- `clientTags` (optional): list of tags. To match, every tag in this list must be in the list of + client-provided tags associated with the query. +- `group` (optional): regex to match against the fully qualified name of the resource group the query is + routed to. +- `sessionProperties`: map with string keys and values. Each entry is a system or catalog property name and + corresponding value. Values must be specified as strings, no matter the actual data type. + +## Example + +Consider the following set of requirements: + +- All queries running under the `global` resource group must have an execution time limit of 8 hours. +- All interactive queries are routed to sub-groups under the `global.interactive` group, and have an execution time + limit of 1 hour (tighter than the constraint on `global`). +- All ETL queries (tagged with 'etl') are routed to sub-groups under the `global.pipeline` group, and must be + configured with certain properties to control writer behavior and a hive catalog property. + +These requirements can be expressed with the following rules: + +```json +[ + { + "group": "global.*", + "sessionProperties": { + "query_max_execution_time": "8h" + } + }, + { + "group": "global.interactive.*", + "sessionProperties": { + "query_max_execution_time": "1h" + } + }, + { + "group": "global.pipeline.*", + "clientTags": ["etl"], + "sessionProperties": { + "scale_writers": "true", + "writer_min_size": "1GB", + "hive.insert_existing_partitions_behavior": "overwrite" + } + } +] +``` diff --git a/430/_sources/admin/spill.md.txt b/430/_sources/admin/spill.md.txt new file mode 100644 index 000000000..5ee1a9552 --- /dev/null +++ b/430/_sources/admin/spill.md.txt @@ -0,0 +1,125 @@ +# Spill to disk + +## Overview + +In the case of memory intensive operations, Trino allows offloading +intermediate operation results to disk. The goal of this mechanism is to +enable execution of queries that require amounts of memory exceeding per query +or per node limits. + +The mechanism is similar to OS level page swapping. However, it is +implemented on the application level to address specific needs of Trino. + +Properties related to spilling are described in {doc}`properties-spilling`. + +## Memory management and spill + +By default, Trino kills queries, if the memory requested by the query execution +exceeds session properties `query_max_memory` or +`query_max_memory_per_node`. This mechanism ensures fairness in allocation +of memory to queries, and prevents deadlock caused by memory allocation. +It is efficient when there is a lot of small queries in the cluster, but +leads to killing large queries that don't stay within the limits. + +To overcome this inefficiency, the concept of revocable memory was introduced. A +query can request memory that does not count toward the limits, but this memory +can be revoked by the memory manager at any time. When memory is revoked, the +query runner spills intermediate data from memory to disk and continues to +process it later. + +In practice, when the cluster is idle, and all memory is available, a memory +intensive query may use all of the memory in the cluster. On the other hand, +when the cluster does not have much free memory, the same query may be forced to +use disk as storage for intermediate data. A query, that is forced to spill to +disk, may have a longer execution time by orders of magnitude than a query that +runs completely in memory. + +Please note that enabling spill-to-disk does not guarantee execution of all +memory intensive queries. It is still possible that the query runner fails +to divide intermediate data into chunks small enough so that every chunk fits into +memory, leading to `Out of memory` errors while loading the data from disk. + +## Spill disk space + +Spilling intermediate results to disk, and retrieving them back, is expensive +in terms of IO operations. Thus, queries that use spill likely become +throttled by disk. To increase query performance, it is recommended to +provide multiple paths on separate local devices for spill (property +`spiller-spill-path` in {doc}`properties-spilling`). + +The system drive should not be used for spilling, especially not to the drive where the JVM +is running and writing logs. Doing so may lead to cluster instability. Additionally, +it is recommended to monitor the disk saturation of the configured spill paths. + +Trino treats spill paths as independent disks (see [JBOD](https://wikipedia.org/wiki/Non-RAID_drive_architectures#JBOD)), so +there is no need to use RAID for spill. + +## Spill compression + +When spill compression is enabled (`spill-compression-enabled` property in +{doc}`properties-spilling`), spilled pages are compressed, before being +written to disk. Enabling this feature can reduce disk IO at the cost +of extra CPU load to compress and decompress spilled pages. + +## Spill encryption + +When spill encryption is enabled (`spill-encryption-enabled` property in +{doc}`properties-spilling`), spill contents are encrypted with a randomly generated +(per spill file) secret key. Enabling this increases CPU load and reduces throughput +of spilling to disk, but can protect spilled data from being recovered from spill files. +Consider reducing the value of `memory-revoking-threshold` when spill +encryption is enabled, to account for the increase in latency of spilling. + +## Supported operations + +Not all operations support spilling to disk, and each handles spilling +differently. Currently, the mechanism is implemented for the following +operations. + +### Joins + +During the join operation, one of the tables being joined is stored in memory. +This table is called the build table. The rows from the other table stream +through and are passed onto the next operation, if they match rows in the build +table. The most memory-intensive part of the join is this build table. + +When the task concurrency is greater than one, the build table is partitioned. +The number of partitions is equal to the value of the `task.concurrency` +configuration parameter (see {doc}`properties-task`). + +When the build table is partitioned, the spill-to-disk mechanism can decrease +the peak memory usage needed by the join operation. When a query approaches the +memory limit, a subset of the partitions of the build table gets spilled to disk, +along with rows from the other table that fall into those same partitions. The +number of partitions, that get spilled, influences the amount of disk space needed. + +Afterward, the spilled partitions are read back one-by-one to finish the join +operation. + +With this mechanism, the peak memory used by the join operator can be decreased +to the size of the largest build table partition. Assuming no data skew, this +is `1 / task.concurrency` times the size of the whole build table. + +### Aggregations + +Aggregation functions perform an operation on a group of values and return one +value. If the number of groups you're aggregating over is large, a significant +amount of memory may be needed. When spill-to-disk is enabled, if there is not +enough memory, intermediate cumulated aggregation results are written to disk. +They are loaded back and merged with a lower memory footprint. + +### Order by + +If your trying to sort a larger amount of data, a significant amount of memory +may be needed. When spill to disk for `order by` is enabled, if there is not enough +memory, intermediate sorted results are written to disk. They are loaded back and +merged with a lower memory footprint. + +### Window functions + +Window functions perform an operator over a window of rows, and return one value +for each row. If this window of rows is large, a significant amount of memory may +be needed. When spill to disk for window functions is enabled, if there is not enough +memory, intermediate results are written to disk. They are loaded back and merged +when memory is available. There is a current limitation that spill does not work +in all cases, such as when a single window is very large. diff --git a/430/_sources/admin/tuning.md.txt b/430/_sources/admin/tuning.md.txt new file mode 100644 index 000000000..2a3f40643 --- /dev/null +++ b/430/_sources/admin/tuning.md.txt @@ -0,0 +1,16 @@ +# Tuning Trino + +The default Trino settings should work well for most workloads. The following +information may help you, if your cluster is facing a specific performance problem. + +## Config properties + +See {doc}`/admin/properties`. + +## JVM settings + +The following can be helpful for diagnosing garbage collection (GC) issues: + +```text +-Xlog:gc*,safepoint::time,level,tags,tid +``` diff --git a/430/_sources/admin/web-interface.md.txt b/430/_sources/admin/web-interface.md.txt new file mode 100644 index 000000000..68e453e49 --- /dev/null +++ b/430/_sources/admin/web-interface.md.txt @@ -0,0 +1,93 @@ +# Web UI + +Trino provides a web-based user interface (UI) for monitoring a Trino cluster +and managing queries. The Web UI is accessible on the coordinator via +HTTP or HTTPS, using the corresponding port number specified in the coordinator +{ref}`config-properties`. It can be configured with {doc}`/admin/properties-web-interface`. + +The Web UI can be disabled entirely with the `web-ui.enabled` property. + +(web-ui-authentication)= + +## Authentication + +The Web UI requires users to authenticate. If Trino is not configured to require +authentication, then any username can be used, and no password is required or +allowed. Typically, users login with the same username that they use for +running queries. + +If no system access control is installed, then all users are able to view and kill +any query. This can be restricted by using {ref}`query rules ` with the +{doc}`/security/built-in-system-access-control`. Users always have permission to view +or kill their own queries. + +### Password authentication + +Typically, a password-based authentication method +such as {doc}`LDAP ` or {doc}`password file ` +is used to secure both the Trino server and the Web UI. When the Trino server +is configured to use a password authenticator, the Web UI authentication type +is automatically set to `FORM`. In this case, the Web UI displays a login form +that accepts a username and password. + +### Fixed user authentication + +If you require the Web UI to be accessible without authentication, you can set a fixed +username that will be used for all Web UI access by setting the authentication type to +`FIXED` and setting the username with the `web-ui.user` configuration property. +If there is a system access control installed, this user must have permission to view +(and possibly to kill) queries. + +### Other authentication types + +The following Web UI authentication types are also supported: + +- `CERTIFICATE`, see details in {doc}`/security/certificate` +- `KERBEROS`, see details in {doc}`/security/kerberos` +- `JWT`, see details in {doc}`/security/jwt` +- `OAUTH2`, see details in {doc}`/security/oauth2` + +For these authentication types, the username is defined by {doc}`/security/user-mapping`. + +(web-ui-overview)= + +## User interface overview + +The main page has a list of queries along with information like unique query ID, query text, +query state, percentage completed, username and source from which this query originated. +The currently running queries are at the top of the page, followed by the most recently +completed or failed queries. + +The possible query states are as follows: + +- `QUEUED` -- Query has been accepted and is awaiting execution. +- `PLANNING` -- Query is being planned. +- `STARTING` -- Query execution is being started. +- `RUNNING` -- Query has at least one running task. +- `BLOCKED` -- Query is blocked and is waiting for resources (buffer space, memory, splits, etc.). +- `FINISHING` -- Query is finishing (e.g. commit for autocommit queries). +- `FINISHED` -- Query has finished executing and all output has been consumed. +- `FAILED` -- Query execution failed. + +The `BLOCKED` state is normal, but if it is persistent, it should be investigated. +It has many potential causes: insufficient memory or splits, disk or network I/O bottlenecks, data skew +(all the data goes to a few workers), a lack of parallelism (only a few workers available), or computationally +expensive stages of the query following a given stage. Additionally, a query can be in +the `BLOCKED` state if a client is not processing the data fast enough (common with "SELECT \*" queries). + +For more detailed information about a query, simply click the query ID link. +The query detail page has a summary section, graphical representation of various stages of the +query and a list of tasks. Each task ID can be clicked to get more information about that task. + +The summary section has a button to kill the currently running query. There are two visualizations +available in the summary section: task execution and timeline. The full JSON document containing +information and statistics about the query is available by clicking the *JSON* link. These visualizations +and other statistics can be used to analyze where time is being spent for a query. + +## Configuring query history + +The following configuration properties affect {doc}`how query history +is collected ` for display in the Web UI: + +- `query.min-expire-age` +- `query.max-history` diff --git a/430/_sources/appendix.md.txt b/430/_sources/appendix.md.txt new file mode 100644 index 000000000..72fe770f3 --- /dev/null +++ b/430/_sources/appendix.md.txt @@ -0,0 +1,8 @@ +# Appendix + +```{toctree} +:maxdepth: 1 + +appendix/from-hive +appendix/legal-notices +``` diff --git a/430/_sources/appendix/from-hive.md.txt b/430/_sources/appendix/from-hive.md.txt new file mode 100644 index 000000000..03475abf0 --- /dev/null +++ b/430/_sources/appendix/from-hive.md.txt @@ -0,0 +1,188 @@ +# Migrating from Hive + +Trino uses ANSI SQL syntax and semantics, whereas Hive uses a language similar +to SQL called HiveQL which is loosely modeled after MySQL (which itself has many +differences from ANSI SQL). + +## Use subscript for accessing a dynamic index of an array instead of a udf + +The subscript operator in SQL supports full expressions, unlike Hive (which only supports constants). Therefore you can write queries like: + +``` +SELECT my_array[CARDINALITY(my_array)] as last_element +FROM ... +``` + +## Avoid out of bounds access of arrays + +Accessing out of bounds elements of an array will result in an exception. You can avoid this with an `if` as follows: + +``` +SELECT IF(CARDINALITY(my_array) >= 3, my_array[3], NULL) +FROM ... +``` + +## Use ANSI SQL syntax for arrays + +Arrays are indexed starting from 1, not from 0: + +``` +SELECT my_array[1] AS first_element +FROM ... +``` + +Construct arrays with ANSI syntax: + +``` +SELECT ARRAY[1, 2, 3] AS my_array +``` + +## Use ANSI SQL syntax for identifiers and strings + +Strings are delimited with single quotes and identifiers are quoted with double quotes, not backquotes: + +``` +SELECT name AS "User Name" +FROM "7day_active" +WHERE name = 'foo' +``` + +## Quote identifiers that start with numbers + +Identifiers that start with numbers are not legal in ANSI SQL and must be quoted using double quotes: + +``` +SELECT * +FROM "7day_active" +``` + +## Use the standard string concatenation operator + +Use the ANSI SQL string concatenation operator: + +``` +SELECT a || b || c +FROM ... +``` + +## Use standard types for CAST targets + +The following standard types are supported for `CAST` targets: + +``` +SELECT + CAST(x AS varchar) +, CAST(x AS bigint) +, CAST(x AS double) +, CAST(x AS boolean) +FROM ... +``` + +In particular, use `VARCHAR` instead of `STRING`. + +## Use CAST when dividing integers + +Trino follows the standard behavior of performing integer division when dividing two integers. For example, dividing `7` by `2` will result in `3`, not `3.5`. +To perform floating point division on two integers, cast one of them to a double: + +``` +SELECT CAST(5 AS DOUBLE) / 2 +``` + +## Use WITH for complex expressions or queries + +When you want to re-use a complex output expression as a filter, use either an inline subquery or factor it out using the `WITH` clause: + +``` +WITH a AS ( + SELECT substr(name, 1, 3) x + FROM ... +) +SELECT * +FROM a +WHERE x = 'foo' +``` + +## Use UNNEST to expand arrays and maps + +Trino supports {ref}`unnest` for expanding arrays and maps. +Use `UNNEST` instead of `LATERAL VIEW explode()`. + +Hive query: + +``` +SELECT student, score +FROM tests +LATERAL VIEW explode(scores) t AS score; +``` + +Trino query: + +``` +SELECT student, score +FROM tests +CROSS JOIN UNNEST(scores) AS t (score); +``` + +## Use ANSI SQL syntax for date and time INTERVAL expressions + +Trino supports the ANSI SQL style `INTERVAL` expressions that differs from the implementation used in Hive. + +- The `INTERVAL` keyword is required and is not optional. +- Date and time units must be singular. For example `day` and not `days`. +- Values must be quoted. + +Hive query: + +``` +SELECT cast('2000-08-19' as date) + 14 days; +``` + +Equivalent Trino query: + +``` +SELECT cast('2000-08-19' as date) + INTERVAL '14' day; +``` + +## Caution with datediff + +The Hive `datediff` function returns the difference between the two dates in +days and is declared as: + +```text +datediff(string enddate, string startdate) -> integer +``` + +The equivalent Trino function {ref}`date_diff` +uses a reverse order for the two date parameters and requires a unit. This has +to be taken into account when migrating: + +Hive query: + +``` +datediff(enddate, startdate) +``` + +Trino query: + +``` +date_diff('day', startdate, enddate) +``` + +## Overwriting data on insert + +By default, `INSERT` queries are not allowed to overwrite existing data. You +can use the catalog session property `insert_existing_partitions_behavior` to +allow overwrites. Prepend the name of the catalog using the Hive connector, for +example `hdfs`, and set the property in the session before you run the insert +query: + +``` +SET SESSION hdfs.insert_existing_partitions_behavior = 'OVERWRITE'; +INSERT INTO hdfs.schema.table ... +``` + +The resulting behavior is equivalent to using [INSERT OVERWRITE](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DML) in Hive. + +Insert overwrite operation is not supported by Trino when the table is stored on +encrypted HDFS, when the table is unpartitioned or table is transactional. diff --git a/430/_sources/appendix/legal-notices.md.txt b/430/_sources/appendix/legal-notices.md.txt new file mode 100644 index 000000000..2d462dc22 --- /dev/null +++ b/430/_sources/appendix/legal-notices.md.txt @@ -0,0 +1,57 @@ +# Legal notices + +## License + +Trino is open source software licensed under the +[Apache License 2.0](https://github.com/trinodb/trino/blob/master/LICENSE). + +## Code + +Source code is available at [https://github.com/trinodb](https://github.com/trinodb). + +## Governance + +The project is run by volunteer contributions and supported by the [Trino +Software Foundation](https://trino.io/foundation.html). + +## Trademarks + +Product names, other names, logos and other material used on this site are +registered trademarks of various entities including, but not limited to, the +following trademark owners and names: + +[American National Standards Institute](https://www.ansi.org/) + +- ANSI, and other names + +[Apache Software Foundation](https://apache.org/) + +- Apache Hadoop, Apache Hive, Apache Iceberg, Apache Kafka, and other names + +[Amazon](https://trademarks.amazon.com/) + +- AWS, S3, Glue, EMR, and other names + +[Docker Inc.](https://www.docker.com/) + +- Docker + +[Google](https://www.google.com/permissions/trademark/trademark-list/) + +- GCP, YouTube and other names + +[Linux Mark Institute](http://www.linuxmark.org/) + +- Linux + +[Microsoft](https://www.microsoft.com/en-us/legal/intellectualproperty/Trademarks/EN-US.aspx) + +- Azure, AKS, and others + +[Oracle](https://www.oracle.com/) + +- Java, JVM, OpenJDK, and other names + +[The Linux Foundation](https://www.linuxfoundation.org/trademark-list/) + +- Kubernetes, Presto, and other names diff --git a/430/_sources/client.md.txt b/430/_sources/client.md.txt new file mode 100644 index 000000000..f45885978 --- /dev/null +++ b/430/_sources/client.md.txt @@ -0,0 +1,20 @@ +# Clients + +A client is used to send queries to Trino and receive results, or otherwise +interact with Trino and the connected data sources. + +Some clients, such as the {doc}`command line interface `, can +provide a user interface directly. Clients like the {doc}`JDBC driver +`, provide a mechanism for other tools to connect to Trino. + +The following clients are available: + +```{toctree} +:maxdepth: 1 + +client/cli +client/jdbc +``` + +In addition, the community provides [numerous other clients](https://trino.io/resources.html) for platforms such as Python, and these +can in turn be used to connect applications using these platforms. diff --git a/430/_sources/client/cli.md.txt b/430/_sources/client/cli.md.txt new file mode 100644 index 000000000..2d09a0eab --- /dev/null +++ b/430/_sources/client/cli.md.txt @@ -0,0 +1,673 @@ +# Command line interface + +The Trino CLI provides a terminal-based, interactive shell for running +queries. The CLI is a +[self-executing](http://skife.org/java/unix/2011/06/20/really_executable_jars.html) +JAR file, which means it acts like a normal UNIX executable. + +## Requirements + +The CLI requires a Java virtual machine available on the path. +It can be used with Java version 8 and higher. + +The CLI uses the {doc}`Trino client REST API ` over +HTTP/HTTPS to communicate with the coordinator on the cluster. + +The CLI version should be identical to the version of the Trino cluster, or +newer. Older versions typically work, but only a subset is regularly tested. +Versions before 350 are not supported. + +(cli-installation)= + +## Installation + +Download {maven_download}`cli`, rename it to `trino`, make it executable with +`chmod +x`, and run it to show the version of the CLI: + +```text +./trino --version +``` + +Run the CLI with `--help` or `-h` to see all available options. + +Windows users, and users unable to execute the preceeding steps, can use the +equivalent `java` command with the `-jar` option to run the CLI, and show +the version: + +```text +java -jar trino-cli-*-executable.jar --version +``` + +The syntax can be used for the examples in the following sections. In addition, +using the `java` command allows you to add configuration options for the Java +runtime with the `-D` syntax. You can use this for debugging and +troubleshooting, such as when {ref}`specifying additional Kerberos debug options +`. + +## Running the CLI + +The minimal command to start the CLI in interactive mode specifies the URL of +the coordinator in the Trino cluster: + +```text +./trino http://trino.example.com:8080 +``` + +If successful, you will get a prompt to execute commands. Use the `help` +command to see a list of supported commands. Use the `clear` command to clear +the terminal. To stop and exit the CLI, run `exit` or `quit`.: + +```text +trino> help + +Supported commands: +QUIT +EXIT +CLEAR +EXPLAIN [ ( option [, ...] ) ] + options: FORMAT { TEXT | GRAPHVIZ | JSON } + TYPE { LOGICAL | DISTRIBUTED | VALIDATE | IO } +DESCRIBE +SHOW COLUMNS FROM
+SHOW FUNCTIONS +SHOW CATALOGS [LIKE ] +SHOW SCHEMAS [FROM ] [LIKE ] +SHOW TABLES [FROM ] [LIKE ] +USE [.] +``` + +You can now run SQL statements. After processing, the CLI will show results and +statistics. + +```text +trino> SELECT count(*) FROM tpch.tiny.nation; + +_col0 +------- + 25 +(1 row) + +Query 20220324_213359_00007_w6hbk, FINISHED, 1 node +Splits: 13 total, 13 done (100.00%) +2.92 [25 rows, 0B] [8 rows/s, 0B/s] +``` + +As part of starting the CLI, you can set the default catalog and schema. This +allows you to query tables directly without specifying catalog and schema. + +```text +./trino http://trino.example.com:8080/tpch/tiny + +trino:tiny> SHOW TABLES; + + Table +---------- +customer +lineitem +nation +orders +part +partsupp +region +supplier +(8 rows) +``` + +You can also set the default catalog and schema with the {doc}`/sql/use` +statement. + +```text +trino> USE tpch.tiny; +USE +trino:tiny> +``` + +Many other options are available to further configure the CLI in interactive +mode: + +:::{list-table} +:widths: 40, 60 +:header-rows: 1 + +* - Option + - Description +* - `--catalog` + - Sets the default catalog. You can change the default catalog and schema with + [](/sql/use). +* - `--client-info` + - Adds arbitrary text as extra information about the client. +* - `--client-request-timeout` + - Sets the duration for query processing, after which, the client request is + terminated. Defaults to `2m`. +* - `--client-tags` + - Adds extra tags information about the client and the CLI user. Separate + multiple tags with commas. The tags can be used as input for + [](/admin/resource-groups). +* - `--debug` + - Enables display of debug information during CLI usage for + [](cli-troubleshooting). Displays more information about query + processing statistics. +* - `--disable-auto-suggestion` + - Disables autocomplete suggestions. +* - `--disable-compression` + - Disables compression of query results. +* - `--editing-mode` + - Sets key bindings in the CLI to be compatible with VI or + EMACS editors. Defaults to `EMACS`. +* - `--http-proxy` + - Configures the URL of the HTTP proxy to connect to Trino. +* - `--history-file` + - Path to the [history file](cli-history). Defaults to `~/.trino_history`. +* - `--network-logging` + - Configures the level of detail provided for network logging of the CLI. + Defaults to `NONE`, other options are `BASIC`, `HEADERS`, or `BODY`. +* - `--output-format-interactive=` + - Specify the [format](cli-output-format) to use for printing query results. + Defaults to `ALIGNED`. +* - `--pager=` + - Path to the pager program used to display the query results. Set to an empty + value to completely disable pagination. Defaults to `less` with a carefully + selected set of options. +* - `--no-progress` + - Do not show query processing progress. +* - `--password` + - Prompts for a password. Use if your Trino server requires password + authentication. You can set the `TRINO_PASSWORD` environment variable with + the password value to avoid the prompt. For more information, see + [](cli-username-password-auth). +* - `--schema` + - Sets the default schema. You can change the default catalog and schema + with [](/sql/use). +* - `--server` + - The HTTP/HTTPS address and port of the Trino coordinator. The port must be + set to the port the Trino coordinator is listening for connections on. Trino + server location defaults to `http://localhost:8080`. Can only be set if URL + is not specified. +* - `--session` + - Sets one or more [session properties](session-properties-definition). + Property can be used multiple times with the format + `session_property_name=value`. +* - `--socks-proxy` + - Configures the URL of the SOCKS proxy to connect to Trino. +* - `--source` + - Specifies the name of the application or source connecting to Trino. + Defaults to `trino-cli`. The value can be used as input for + [](/admin/resource-groups). +* - `--timezone` + - Sets the time zone for the session using the [time zone name]( + ). Defaults to + the timezone set on your workstation. +* - `--user` + - Sets the username for [](cli-username-password-auth). Defaults to your + operating system username. You can override the default username, if your + cluster uses a different username or authentication mechanism. +::: + +Most of the options can also be set as parameters in the URL. This means +a JDBC URL can be used in the CLI after removing the `jdbc:` prefix. +However, the same parameter may not be specified using both methods. +See {doc}`the JDBC driver parameter reference ` +to find out URL parameter names. For example: + +```text +./trino 'https://trino.example.com?SSL=true&SSLVerification=FULL&clientInfo=extra' +``` + +(cli-tls)= + +## TLS/HTTPS + +Trino is typically available with an HTTPS URL. This means that all network +traffic between the CLI and Trino uses TLS. {doc}`TLS configuration +` is common, since it is a requirement for {ref}`any +authentication `. + +Use the HTTPS URL to connect to the server: + +```text +./trino https://trino.example.com +``` + +The recommended TLS implementation is to use a globally trusted certificate. In +this case, no other options are necessary, since the JVM running the CLI +recognizes these certificates. + +Use the options from the following table to further configure TLS and +certificate usage: + +:::{list-table} +:widths: 40, 60 +:header-rows: 1 + +* - Option + - Description +* - `--insecure` + - Skip certificate validation when connecting with TLS/HTTPS (should only be + used for debugging). +* - `--keystore-path` + - The location of the Java Keystore file that contains the certificate of the + server to connect with TLS. +* - `--keystore-password` + - The password for the keystore. This must match the password you specified + when creating the keystore. +* - `--keystore-type` + - Determined by the keystore file format. The default keystore type is JKS. + This advanced option is only necessary if you use a custom Java Cryptography + Architecture (JCA) provider implementation. +* - `--truststore-password` + - The password for the truststore. This must match the password you specified + when creating the truststore. +* - `--truststore-path` + - The location of the Java truststore file that will be used to secure TLS. +* - `--truststore-type` + - Determined by the truststore file format. The default keystore type is JKS. + This advanced option is only necessary if you use a custom Java Cryptography + Architecture (JCA) provider implementation. +* - `--use-system-truststore` + - Verify the server certificate using the system truststore of the operating + system. Windows and macOS are supported. For other operating systems, the + default Java truststore is used. The truststore type can be overridden using + `--truststore-type`. +::: + +(cli-authentication)= + +## Authentication + +The Trino CLI supports many {doc}`/security/authentication-types` detailed in +the following sections: + +(cli-username-password-auth)= + +### Username and password authentication + +Username and password authentication is typically configured in a cluster using +the `PASSWORD` {doc}`authentication type `, +for example with {doc}`/security/ldap` or {doc}`/security/password-file`. + +The following code example connects to the server, establishes your user name, +and prompts the CLI for your password: + +```text +./trino https://trino.example.com --user=exampleusername --password +``` + +Alternatively, set the password as the value of the `TRINO_PASSWORD` +environment variable. Typically use single quotes to avoid problems with +special characters such as `$`: + +```text +export TRINO_PASSWORD='LongSecurePassword123!@#' +``` + +If the `TRINO_PASSWORD` environment variable is set, you are not prompted +to provide a password to connect with the CLI. + +```text +./trino https://trino.example.com --user=exampleusername --password +``` + +(cli-external-sso-auth)= + +### External authentication - SSO + +Use the `--external-authentication` option for browser-based SSO +authentication, as detailed in {doc}`/security/oauth2`. With this configuration, +the CLI displays a URL that you must open in a web browser for authentication. + +The detailed behavior is as follows: + +- Start the CLI with the `--external-authentication` option and execute a + query. +- The CLI starts and connects to Trino. +- A message appears in the CLI directing you to open a browser with a specified + URL when the first query is submitted. +- Open the URL in a browser and follow through the authentication process. +- The CLI automatically receives a token. +- When successfully authenticated in the browser, the CLI proceeds to execute + the query. +- Further queries in the CLI session do not require additional logins while the + authentication token remains valid. Token expiration depends on the external + authentication type configuration. +- Expired tokens force you to log in again. + +(cli-certificate-auth)= + +### Certificate authentication + +Use the following CLI arguments to connect to a cluster that uses +{doc}`certificate authentication `. + +:::{list-table} CLI options for certificate authentication +:widths: 35 65 +:header-rows: 1 + +* - Option + - Description +* - `--keystore-path=` + - Absolute or relative path to a [PEM](/security/inspect-pem) or + [JKS](/security/inspect-jks) file, which must contain a certificate + that is trusted by the Trino cluster you are connecting to. +* - `--keystore-password=` + - Only required if the keystore has a password. +::: + +The truststore related options are independent of client certificate +authentication with the CLI; instead, they control the client's trust of the +server's certificate. + +(cli-jwt-auth)= + +### JWT authentication + +To access a Trino cluster configured to use {doc}`/security/jwt`, use the +`--access-token=` option to pass a JWT to the server. + +(cli-kerberos-auth)= + +### Kerberos authentication + +The Trino CLI can connect to a Trino cluster that has {doc}`/security/kerberos` +enabled. + +Invoking the CLI with Kerberos support enabled requires a number of additional +command line options. You also need the {ref}`Kerberos configuration files +` for your user on the machine running the CLI. The +simplest way to invoke the CLI is with a wrapper script: + +```text +#!/bin/bash + +./trino \ + --server https://trino.example.com \ + --krb5-config-path /etc/krb5.conf \ + --krb5-principal someuser@EXAMPLE.COM \ + --krb5-keytab-path /home/someuser/someuser.keytab \ + --krb5-remote-service-name trino +``` + +When using Kerberos authentication, access to the Trino coordinator must be +through {doc}`TLS and HTTPS `. + +The following table lists the available options for Kerberos authentication: + +:::{list-table} CLI options for Kerberos authentication +:widths: 40, 60 +:header-rows: 1 + +* - Option + - Description +* - `--krb5-config-path` + - Path to Kerberos configuration files. +* - `--krb5-credential-cache-path` + - Kerberos credential cache path. +* - `--krb5-disable-remote-service-hostname-canonicalization` + - Disable service hostname canonicalization using the DNS reverse lookup. +* - `--krb5-keytab-path` + - The location of the keytab that can be used to authenticate the principal + specified by `--krb5-principal`. +* - `--krb5-principal` + - The principal to use when authenticating to the coordinator. +* - `--krb5-remote-service-name` + - Trino coordinator Kerberos service name. +* - `--krb5-service-principal-pattern` + - Remote kerberos service principal pattern. Defaults to `${SERVICE}@${HOST}`. +::: + +(cli-kerberos-debug)= + +#### Additional Kerberos debugging information + +You can enable additional Kerberos debugging information for the Trino CLI +process by passing `-Dsun.security.krb5.debug=true`, +`-Dtrino.client.debugKerberos=true`, and +`-Djava.security.debug=gssloginconfig,configfile,configparser,logincontext` +as a JVM argument when {ref}`starting the CLI process `: + +```text +java \ + -Dsun.security.krb5.debug=true \ + -Djava.security.debug=gssloginconfig,configfile,configparser,logincontext \ + -Dtrino.client.debugKerberos=true \ + -jar trino-cli-*-executable.jar \ + --server https://trino.example.com \ + --krb5-config-path /etc/krb5.conf \ + --krb5-principal someuser@EXAMPLE.COM \ + --krb5-keytab-path /home/someuser/someuser.keytab \ + --krb5-remote-service-name trino +``` + +For help with interpreting Kerberos debugging messages, see {ref}`additional +resources `. + +## Pagination + +By default, the results of queries are paginated using the `less` program +which is configured with a carefully selected set of options. This behavior +can be overridden by setting the `--pager` option or +the `TRINO_PAGER` environment variable to the name of a different program +such as `more` or [pspg](https://github.com/okbob/pspg), +or it can be set to an empty value to completely disable pagination. + +(cli-history)= + +## History + +The CLI keeps a history of your previously used commands. You can access your +history by scrolling or searching. Use the up and down arrows to scroll and +{kbd}`Control+S` and {kbd}`Control+R` to search. To execute a query again, +press {kbd}`Enter`. + +By default, you can locate the Trino history file in `~/.trino_history`. +Use the `--history-file` option or the `` `TRINO_HISTORY_FILE `` environment variable +to change the default. + +### Auto suggestion + +The CLI generates autocomplete suggestions based on command history. + +Press {kbd}`→` to accept the suggestion and replace the current command line +buffer. Press {kbd}`Ctrl+→` ({kbd}`Option+→` on Mac) to accept only the next +keyword. Continue typing to reject the suggestion. + +## Configuration file + +The CLI can read default values for all options from a file. It uses the first +file found from the ordered list of locations: + +- File path set as value of the `TRINO_CONFIG` environment variable. +- `.trino_config` in the current users home directory. +- `$XDG_CONFIG_HOME/trino/config`. + +For example, you could create separate configuration files with different +authentication options, like `kerberos-cli.properties` and `ldap-cli.properties`. +Assuming they're located in the current directory, you can set the +`TRINO_CONFIG` environment variable for a single invocation of the CLI by +adding it before the `trino` command: + +```text +TRINO_CONFIG=kerberos-cli.properties trino https://first-cluster.example.com:8443 +TRINO_CONFIG=ldap-cli.properties trino https://second-cluster.example.com:8443 +``` + +In the preceding example, the default configuration files are not used. + +You can use all supported options without the `--` prefix in the configuration +properties file. Options that normally don't take an argument are boolean, so +set them to either `true` or `false`. For example: + +```properties +output-format-interactive=AUTO +timezone=Europe/Warsaw +user=trino-client +network-logging=BASIC +krb5-disable-remote-service-hostname-canonicalization=true +``` + +## Batch mode + +Running the Trino CLI with the `--execute`, `--file`, or passing queries to +the standard input uses the batch (non-interactive) mode. In this mode +the CLI does not report progress, and exits after processing the supplied +queries. Results are printed in `CSV` format by default. You can configure +other formats and redirect the output to a file. + +The following options are available to further configure the CLI in batch +mode: + +:::{list-table} +:widths: 40, 60 +:header-rows: 1 + +* - Option + - Description +* - `--execute=` + - Execute specified statements and exit. +* - `-f`, `--file=` + - Execute statements from file and exit. +* - `--ignore-errors` + - Continue processing in batch mode when an error occurs. Default is to exit + immediately. +* - `--output-format=` + - Specify the [format](cli-output-format) to use for printing query results. + Defaults to `CSV`. +* - `--progress` + - Show query progress in batch mode. It does not affect the output, which, for + example can be safely redirected to a file. +::: + +### Examples + +Consider the following command run as shown, or with the +`--output-format=CSV` option, which is the default for non-interactive usage: + +```text +trino --execute 'SELECT nationkey, name, regionkey FROM tpch.sf1.nation LIMIT 3' +``` + +The output is as follows: + +```text +"0","ALGERIA","0" +"1","ARGENTINA","1" +"2","BRAZIL","1" +``` + +The output with the `--output-format=JSON` option: + +```json +{"nationkey":0,"name":"ALGERIA","regionkey":0} +{"nationkey":1,"name":"ARGENTINA","regionkey":1} +{"nationkey":2,"name":"BRAZIL","regionkey":1} +``` + +The output with the `--output-format=ALIGNED` option, which is the default +for interactive usage: + +```text +nationkey | name | regionkey +----------+-----------+---------- + 0 | ALGERIA | 0 + 1 | ARGENTINA | 1 + 2 | BRAZIL | 1 +``` + +The output with the `--output-format=VERTICAL` option: + +```text +-[ RECORD 1 ]-------- +nationkey | 0 +name | ALGERIA +regionkey | 0 +-[ RECORD 2 ]-------- +nationkey | 1 +name | ARGENTINA +regionkey | 1 +-[ RECORD 3 ]-------- +nationkey | 2 +name | BRAZIL +regionkey | 1 +``` + +The preceding command with `--output-format=NULL` produces no output. +However, if you have an error in the query, such as incorrectly using +`region` instead of `regionkey`, the command has an exit status of 1 +and displays an error message (which is unaffected by the output format): + +```text +Query 20200707_170726_00030_2iup9 failed: line 1:25: Column 'region' cannot be resolved +SELECT nationkey, name, region FROM tpch.sf1.nation LIMIT 3 +``` + +(cli-output-format)= + +## Output formats + +The Trino CLI provides the options `--output-format` +and `--output-format-interactive` to control how the output is displayed. +The available options shown in the following table must be entered +in uppercase. The default value is `ALIGNED` in interactive mode, +and `CSV` in non-interactive mode. + +:::{list-table} Output format options +:widths: 25, 75 +:header-rows: 1 + +* - Option + - Description +* - `CSV` + - Comma-separated values, each value quoted. No header row. +* - `CSV_HEADER` + - Comma-separated values, quoted with header row. +* - `CSV_UNQUOTED` + - Comma-separated values without quotes. +* - `CSV_HEADER_UNQUOTED` + - Comma-separated values with header row but no quotes. +* - `TSV` + - Tab-separated values. +* - `TSV_HEADER` + - Tab-separated values with header row. +* - `JSON` + - Output rows emitted as JSON objects with name-value pairs. +* - `ALIGNED` + - Output emitted as an ASCII character table with values. +* - `VERTICAL` + - Output emitted as record-oriented top-down lines, one per value. +* - `AUTO` + - Same as `ALIGNED` if output would fit the current terminal width, + and `VERTICAL` otherwise. +* - `MARKDOWN` + - Output emitted as a Markdown table. +* - `NULL` + - Suppresses normal query results. This can be useful during development to + test a query's shell return code or to see whether it results in error + messages. +::: + +(cli-troubleshooting)= + +## Troubleshooting + +If something goes wrong, you see an error message: + +```text +$ trino +trino> select count(*) from tpch.tiny.nations; +Query 20200804_201646_00003_f5f6c failed: line 1:22: Table 'tpch.tiny.nations' does not exist +select count(*) from tpch.tiny.nations +``` + +To view debug information, including the stack trace for failures, use the +`--debug` option: + +```text +$ trino --debug +trino> select count(*) from tpch.tiny.nations; +Query 20200804_201629_00002_f5f6c failed: line 1:22: Table 'tpch.tiny.nations' does not exist +io.trino.spi.TrinoException: line 1:22: Table 'tpch.tiny.nations' does not exist +at io.trino.sql.analyzer.SemanticExceptions.semanticException(SemanticExceptions.java:48) +at io.trino.sql.analyzer.SemanticExceptions.semanticException(SemanticExceptions.java:43) +... +at java.base/java.lang.Thread.run(Thread.java:834) +select count(*) from tpch.tiny.nations +``` diff --git a/430/_sources/client/jdbc.md.txt b/430/_sources/client/jdbc.md.txt new file mode 100644 index 000000000..ab0535ad7 --- /dev/null +++ b/430/_sources/client/jdbc.md.txt @@ -0,0 +1,248 @@ +# JDBC driver + +The Trino [JDBC driver](https://wikipedia.org/wiki/JDBC_driver) allows +users to access Trino using Java-based applications, and other non-Java +applications running in a JVM. Both desktop and server-side applications, such +as those used for reporting and database development, use the JDBC driver. + +## Requirements + +The Trino JDBC driver has the following requirements: + +- Java version 8 or higher. +- All users that connect to Trino with the JDBC driver must be granted access to + query tables in the `system.jdbc` schema. + +The JDBC driver version should be identical to the version of the Trino cluster, +or newer. Older versions typically work, but only a subset is regularly tested. +Versions before 350 are not supported. + +## Installing + +Download {maven_download}`jdbc` and add it to the classpath of your Java application. + +The driver is also available from Maven Central: + +```{eval-rst} +.. parsed-literal:: + + + io.trino + trino-jdbc + \ |version|\ + +``` + +We recommend using the latest version of the JDBC driver. A list of all +available versions can be found in the [Maven Central Repository](https://repo1.maven.org/maven2/io/trino/trino-jdbc/). Navigate to the +directory for the desired version, and select the `trino-jdbc-xxx.jar` file +to download, where `xxx` is the version number. + +Once downloaded, you must add the JAR file to a directory in the classpath +of users on systems where they will access Trino. + +After you have downloaded the JDBC driver and added it to your +classpath, you'll typically need to restart your application in order to +recognize the new driver. Then, depending on your application, you +may need to manually register and configure the driver. + +The CLI uses the HTTP protocol and the +{doc}`Trino client REST API ` to communicate +with Trino. + +## Registering and configuring the driver + +Drivers are commonly loaded automatically by applications once they are added to +its classpath. If your application does not, such as is the case for some +GUI-based SQL editors, read this section. The steps to register the JDBC driver +in a UI or on the command line depend upon the specific application you are +using. Please check your application's documentation. + +Once registered, you must also configure the connection information as described +in the following section. + +## Connecting + +When your driver is loaded, registered and configured, you are ready to connect +to Trino from your application. The following JDBC URL formats are supported: + +```text +jdbc:trino://host:port +jdbc:trino://host:port/catalog +jdbc:trino://host:port/catalog/schema +``` + +The following is an example of a JDBC URL used to create a connection: + +```text +jdbc:trino://example.net:8080/hive/sales +``` + +This example JDBC URL locates a Trino instance running on port `8080` on +`example.net`, with the catalog `hive` and the schema `sales` defined. + +:::{note} +Typically, the JDBC driver classname is configured automatically by your +client. If it is not, use `io.trino.jdbc.TrinoDriver` wherever a driver +classname is required. +::: + +(jdbc-java-connection)= + +## Connection parameters + +The driver supports various parameters that may be set as URL parameters, +or as properties passed to `DriverManager`. Both of the following +examples are equivalent: + +```java +// properties +String url = "jdbc:trino://example.net:8080/hive/sales"; +Properties properties = new Properties(); +properties.setProperty("user", "test"); +properties.setProperty("password", "secret"); +properties.setProperty("SSL", "true"); +Connection connection = DriverManager.getConnection(url, properties); + +// URL parameters +String url = "jdbc:trino://example.net:8443/hive/sales?user=test&password=secret&SSL=true"; +Connection connection = DriverManager.getConnection(url); +``` + +These methods may be mixed; some parameters may be specified in the URL, +while others are specified using properties. However, the same parameter +may not be specified using both methods. + +(jdbc-parameter-reference)= + +## Parameter reference + +:::{list-table} +:widths: 35, 65 +:header-rows: 1 + +* - Name + - Description +* - `user` + - Username to use for authentication and authorization. +* - `password` + - Password to use for LDAP authentication. +* - `sessionUser` + - Session username override, used for impersonation. +* - `socksProxy` + - SOCKS proxy host and port. Example: `localhost:1080` +* - `httpProxy` + - HTTP proxy host and port. Example: `localhost:8888` +* - `clientInfo` + - Extra information about the client. +* - `clientTags` + - Client tags for selecting resource groups. Example: `abc,xyz` +* - `traceToken` + - Trace token for correlating requests across systems. +* - `source` + - Source name for the Trino query. This parameter should be used in preference + to `ApplicationName`. Thus, it takes precedence over `ApplicationName` + and/or `applicationNamePrefix`. +* - `applicationNamePrefix` + - Prefix to append to any specified `ApplicationName` client info property, + which is used to set the source name for the Trino query if the `source` + parameter has not been set. If neither this property nor `ApplicationName` + or `source` are set, the source name for the query is `trino-jdbc`. +* - `accessToken` + - [JWT](/security/jwt) access token for token based authentication. +* - `SSL` + - Set `true` to specify using TLS/HTTPS for connections. +* - `SSLVerification` + - The method of TLS verification. There are three modes: `FULL` + (default), `CA` and `NONE`. For `FULL`, the normal TLS verification + is performed. For `CA`, only the CA is verified but hostname mismatch + is allowed. For `NONE`, there is no verification. +* - `SSLKeyStorePath` + - Use only when connecting to a Trino cluster that has [certificate + authentication](/security/certificate) enabled. Specifies the path to a + [PEM](/security/inspect-pem) or [JKS](/security/inspect-jks) file, which must + contain a certificate that is trusted by the Trino cluster you connect to. +* - `SSLKeyStorePassword` + - The password for the KeyStore, if any. +* - `SSLKeyStoreType` + - The type of the KeyStore. The default type is provided by the Java + `keystore.type` security property or `jks` if none exists. +* - `SSLTrustStorePath` + - The location of the Java TrustStore file to use to validate HTTPS server + certificates. +* - `SSLTrustStorePassword` + - The password for the TrustStore. +* - `SSLTrustStoreType` + - The type of the TrustStore. The default type is provided by the Java + `keystore.type` security property or `jks` if none exists. +* - `SSLUseSystemTrustStore` + - Set `true` to automatically use the system TrustStore based on the operating + system. The supported OSes are Windows and macOS. For Windows, the + `Windows-ROOT` TrustStore is selected. For macOS, the `KeychainStore` + TrustStore is selected. For other OSes, the default Java TrustStore is + loaded. The TrustStore specification can be overridden using + `SSLTrustStoreType`. +* - `hostnameInCertificate` + - Expected hostname in the certificate presented by the Trino server. Only + applicable with full SSL verification enabled. +* - `KerberosRemoteServiceName` + - Trino coordinator Kerberos service name. This parameter is required for + Kerberos authentication. +* - `KerberosPrincipal` + - The principal to use when authenticating to the Trino coordinator. +* - `KerberosUseCanonicalHostname` + - Use the canonical hostname of the Trino coordinator for the Kerberos service + principal by first resolving the hostname to an IP address and then doing a + reverse DNS lookup for that IP address. This is enabled by default. +* - `KerberosServicePrincipalPattern` + - Trino coordinator Kerberos service principal pattern. The default is + `${SERVICE}@${HOST}`. `${SERVICE}` is replaced with the value of + `KerberosRemoteServiceName` and `${HOST}` is replaced with the hostname of + the coordinator (after canonicalization if enabled). +* - `KerberosConfigPath` + - Kerberos configuration file. +* - `KerberosKeytabPath` + - Kerberos keytab file. +* - `KerberosCredentialCachePath` + - Kerberos credential cache. +* - `KerberosDelegation` + - Set to `true` to use the token from an existing Kerberos context. This + allows client to use Kerberos authentication without passing the Keytab or + credential cache. Defaults to `false`. +* - `extraCredentials` + - Extra credentials for connecting to external services, specified as a list + of key-value pairs. For example, `foo:bar;abc:xyz` creates the credential + named `abc` with value `xyz` and the credential named `foo` with value + `bar`. +* - `roles` + - Authorization roles to use for catalogs, specified as a list of key-value + pairs for the catalog and role. For example, `catalog1:roleA;catalog2:roleB` + sets `roleA` for `catalog1` and `roleB` for `catalog2`. +* - `sessionProperties` + - Session properties to set for the system and for catalogs, specified as a + list of key-value pairs. For example, `abc:xyz;example.foo:bar` sets the + system property `abc` to the value `xyz` and the `foo` property for catalog + `example` to the value `bar`. +* - `externalAuthentication` + - Set to true if you want to use external authentication via + [](/security/oauth2). Use a local web browser to authenticate with an + identity provider (IdP) that has been configured for the Trino coordinator. +* - `externalAuthenticationTokenCache` + - Allows the sharing of external authentication tokens between different + connections for the same authenticated user until the cache is invalidated, + such as when a client is restarted or when the classloader reloads the JDBC + driver. This is disabled by default, with a value of `NONE`. To enable, set + the value to `MEMORY`. If the JDBC driver is used in a shared mode by + different users, the first registered token is stored and authenticates all + users. +* - `disableCompression` + - Whether compression should be enabled. +* - `assumeLiteralUnderscoreInMetadataCallsForNonConformingClients` + - When enabled, the name patterns passed to `DatabaseMetaData` methods are + treated as underscores. You can use this as a workaround for applications + that do not escape schema or table names when passing them to + `DatabaseMetaData` methods as schema or table name patterns. ::: +* - `timezone` + - Sets the time zone for the session using the [time zone + passed](https://docs.oracle.com/en/java/javase/17/docs/api/java.base/java/time/ZoneId.html#of(java.lang.String)). + Defaults to the timezone of the JVM running the JDBC driver. diff --git a/430/_sources/connector.md.txt b/430/_sources/connector.md.txt new file mode 100644 index 000000000..3b86e28f3 --- /dev/null +++ b/430/_sources/connector.md.txt @@ -0,0 +1,45 @@ +# Connectors + +This section describes the connectors available in Trino to access data +from different data sources. + +```{toctree} +:maxdepth: 1 + +Accumulo +Atop +BigQuery +Black Hole +Cassandra +ClickHouse +Delta Lake +Druid +Elasticsearch +Google Sheets +Hive +Hudi +Iceberg +Ignite +JMX +Kafka +Kinesis +Kudu +Local File +MariaDB +Memory +MongoDB +MySQL +Oracle +Phoenix +Pinot +PostgreSQL +Prometheus +Redis +Redshift +SingleStore +SQL Server +System +Thrift +TPCDS +TPCH +``` diff --git a/430/_sources/connector/accumulo.md.txt b/430/_sources/connector/accumulo.md.txt new file mode 100644 index 000000000..55813bee3 --- /dev/null +++ b/430/_sources/connector/accumulo.md.txt @@ -0,0 +1,792 @@ +# Accumulo connector + +```{raw} html + +``` + +The Accumulo connector supports reading and writing data from +[Apache Accumulo](https://accumulo.apache.org/). +Please read this page thoroughly to understand the capabilities and features of the connector. + +## Installing the iterator dependency + +The Accumulo connector uses custom Accumulo iterators in +order to push various information in SQL predicate clauses to Accumulo for +server-side filtering, known as *predicate pushdown*. In order +for the server-side iterators to work, you need to add the `trino-accumulo-iterators` +JAR file to Accumulo's `lib/ext` directory on each TabletServer node. + +```bash +# For each TabletServer node: +scp $TRINO_HOME/plugins/accumulo/trino-accumulo-iterators-*.jar [tabletserver_address]:$ACCUMULO_HOME/lib/ext + +# TabletServer should pick up new JAR files in ext directory, but may require restart +``` + +## Requirements + +To connect to Accumulo, you need: + +- Accumulo versions 1.x starting with 1.7.4. Versions 2.x are not supported. +- Network access from the Trino coordinator and workers to the Accumulo + Zookeeper server. Port 2181 is the default port. + +## Connector configuration + +Create `etc/catalog/example.properties` to mount the `accumulo` connector as +the `example` catalog, with the following connector properties as appropriate +for your setup: + +```text +connector.name=accumulo +accumulo.instance=xxx +accumulo.zookeepers=xxx +accumulo.username=username +accumulo.password=password +``` + +Replace the `accumulo.xxx` properties as required. + +## Configuration variables + +| Property name | Default value | Required | Description | +| -------------------------------------------- | ----------------- | -------- | -------------------------------------------------------------------------------- | +| `accumulo.instance` | (none) | Yes | Name of the Accumulo instance | +| `accumulo.zookeepers` | (none) | Yes | ZooKeeper connect string | +| `accumulo.username` | (none) | Yes | Accumulo user for Trino | +| `accumulo.password` | (none) | Yes | Accumulo password for user | +| `accumulo.zookeeper.metadata.root` | `/trino-accumulo` | No | Root znode for storing metadata. Only relevant if using default Metadata Manager | +| `accumulo.cardinality.cache.size` | `100000` | No | Sets the size of the index cardinality cache | +| `accumulo.cardinality.cache.expire.duration` | `5m` | No | Sets the expiration duration of the cardinality cache. | + +## Usage + +Simply begin using SQL to create a new table in Accumulo to begin +working with data. By default, the first column of the table definition +is set to the Accumulo row ID. This should be the primary key of your +table, and keep in mind that any `INSERT` statements containing the same +row ID is effectively an UPDATE as far as Accumulo is concerned, as any +previous data in the cell is overwritten. The row ID can be +any valid Trino datatype. If the first column is not your primary key, you +can set the row ID column using the `row_id` table property within the `WITH` +clause of your table definition. + +Simply issue a `CREATE TABLE` statement to create a new Trino/Accumulo table: + +``` +CREATE TABLE example_schema.scientists ( + recordkey VARCHAR, + name VARCHAR, + age BIGINT, + birthday DATE +); +``` + +```sql +DESCRIBE example_schema.scientists; +``` + +```text + Column | Type | Extra | Comment +-----------+---------+-------+--------------------------------------------------- + recordkey | varchar | | Accumulo row ID + name | varchar | | Accumulo column name:name. Indexed: false + age | bigint | | Accumulo column age:age. Indexed: false + birthday | date | | Accumulo column birthday:birthday. Indexed: false +``` + +This command creates a new Accumulo table with the `recordkey` column +as the Accumulo row ID. The name, age, and birthday columns are mapped to +auto-generated column family and qualifier values (which, in practice, +are both identical to the Trino column name). + +When creating a table using SQL, you can optionally specify a +`column_mapping` table property. The value of this property is a +comma-delimited list of triples, Trino column **:** Accumulo column +family **:** accumulo column qualifier, with one triple for every +non-row ID column. This sets the mapping of the Trino column name to +the corresponding Accumulo column family and column qualifier. + +If you don't specify the `column_mapping` table property, then the +connector auto-generates column names (respecting any configured locality groups). +Auto-generation of column names is only available for internal tables, so if your +table is external you must specify the column_mapping property. + +For a full list of table properties, see [Table Properties](accumulo-table-properties). + +For example: + +```sql +CREATE TABLE example_schema.scientists ( + recordkey VARCHAR, + name VARCHAR, + age BIGINT, + birthday DATE +) +WITH ( + column_mapping = 'name:metadata:name,age:metadata:age,birthday:metadata:date' +); +``` + +```sql +DESCRIBE example_schema.scientists; +``` + +```text + Column | Type | Extra | Comment +-----------+---------+-------+----------------------------------------------- + recordkey | varchar | | Accumulo row ID + name | varchar | | Accumulo column metadata:name. Indexed: false + age | bigint | | Accumulo column metadata:age. Indexed: false + birthday | date | | Accumulo column metadata:date. Indexed: false +``` + +You can then issue `INSERT` statements to put data into Accumulo. + +:::{note} +While issuing `INSERT` statements is convenient, +this method of loading data into Accumulo is low-throughput. You want +to use the Accumulo APIs to write `Mutations` directly to the tables. +See the section on [Loading Data](accumulo-loading-data) for more details. +::: + +```sql +INSERT INTO example_schema.scientists VALUES +('row1', 'Grace Hopper', 109, DATE '1906-12-09' ), +('row2', 'Alan Turing', 103, DATE '1912-06-23' ); +``` + +```sql +SELECT * FROM example_schema.scientists; +``` + +```text + recordkey | name | age | birthday +-----------+--------------+-----+------------ + row1 | Grace Hopper | 109 | 1906-12-09 + row2 | Alan Turing | 103 | 1912-06-23 +(2 rows) +``` + +As you'd expect, rows inserted into Accumulo via the shell or +programmatically will also show up when queried. (The Accumulo shell +thinks "-5321" is an option and not a number... so we'll just make TBL a +little younger.) + +```bash +$ accumulo shell -u root -p secret +root@default> table example_schema.scientists +root@default example_schema.scientists> insert row3 metadata name "Tim Berners-Lee" +root@default example_schema.scientists> insert row3 metadata age 60 +root@default example_schema.scientists> insert row3 metadata date 5321 +``` + +```sql +SELECT * FROM example_schema.scientists; +``` + +```text + recordkey | name | age | birthday +-----------+-----------------+-----+------------ + row1 | Grace Hopper | 109 | 1906-12-09 + row2 | Alan Turing | 103 | 1912-06-23 + row3 | Tim Berners-Lee | 60 | 1984-07-27 +(3 rows) +``` + +You can also drop tables using `DROP TABLE`. This command drops both +metadata and the tables. See the below section on [External +Tables](accumulo-external-tables) for more details on internal and external +tables. + +```sql +DROP TABLE example_schema.scientists; +``` + +## Indexing columns + +Internally, the connector creates an Accumulo `Range` and packs it in +a split. This split gets passed to a Trino Worker to read the data from +the `Range` via a `BatchScanner`. When issuing a query that results +in a full table scan, each Trino Worker gets a single `Range` that +maps to a single tablet of the table. When issuing a query with a +predicate (i.e. `WHERE x = 10` clause), Trino passes the values +within the predicate (`10`) to the connector so it can use this +information to scan less data. When the Accumulo row ID is used as part +of the predicate clause, this narrows down the `Range` lookup to quickly +retrieve a subset of data from Accumulo. + +But what about the other columns? If you're frequently querying on +non-row ID columns, you should consider using the **indexing** +feature built into the Accumulo connector. This feature can drastically +reduce query runtime when selecting a handful of values from the table, +and the heavy lifting is done for you when loading data via Trino +`INSERT` statements. Keep in mind writing data to Accumulo via +`INSERT` does not have high throughput. + +To enable indexing, add the `index_columns` table property and specify +a comma-delimited list of Trino column names you wish to index (we use the +`string` serializer here to help with this example -- you +should be using the default `lexicoder` serializer). + +```sql +CREATE TABLE example_schema.scientists ( + recordkey VARCHAR, + name VARCHAR, + age BIGINT, + birthday DATE +) +WITH ( + serializer = 'string', + index_columns='name,age,birthday' +); +``` + +After creating the table, we see there are an additional two Accumulo +tables to store the index and metrics. + +```text +root@default> tables +accumulo.metadata +accumulo.root +example_schema.scientists +example_schema.scientists_idx +example_schema.scientists_idx_metrics +trace +``` + +After inserting data, we can look at the index table and see there are +indexed values for the name, age, and birthday columns. The connector +queries this index table + +```sql +INSERT INTO example_schema.scientists VALUES +('row1', 'Grace Hopper', 109, DATE '1906-12-09'), +('row2', 'Alan Turing', 103, DATE '1912-06-23'); +``` + +```text +root@default> scan -t example_schema.scientists_idx +-21011 metadata_date:row2 [] +-23034 metadata_date:row1 [] +103 metadata_age:row2 [] +109 metadata_age:row1 [] +Alan Turing metadata_name:row2 [] +Grace Hopper metadata_name:row1 [] +``` + +When issuing a query with a `WHERE` clause against indexed columns, +the connector searches the index table for all row IDs that contain the +value within the predicate. These row IDs are bundled into a Trino +split as single-value `Range` objects, the number of row IDs per split +is controlled by the value of `accumulo.index_rows_per_split`, and +passed to a Trino worker to be configured in the `BatchScanner` which +scans the data table. + +```sql +SELECT * FROM example_schema.scientists WHERE age = 109; +``` + +```text + recordkey | name | age | birthday +-----------+--------------+-----+------------ + row1 | Grace Hopper | 109 | 1906-12-09 +(1 row) +``` + +(accumulo-loading-data)= +## Loading data + +The Accumulo connector supports loading data via INSERT statements, however +this method tends to be low-throughput and should not be relied on when +throughput is a concern. + +(accumulo-external-tables)= +## External tables + +By default, the tables created using SQL statements via Trino are +*internal* tables, that is both the Trino table metadata and the +Accumulo tables are managed by Trino. When you create an internal +table, the Accumulo table is created as well. You receive an error +if the Accumulo table already exists. When an internal table is dropped +via Trino, the Accumulo table, and any index tables, are dropped as +well. + +To change this behavior, set the `external` property to `true` when +issuing the `CREATE` statement. This makes the table an *external* +table, and a `DROP TABLE` command **only** deletes the metadata +associated with the table. If the Accumulo tables do not already exist, +they are created by the connector. + +Creating an external table *will* set any configured locality groups as well +as the iterators on the index and metrics tables, if the table is indexed. +In short, the only difference between an external table and an internal table, +is that the connector deletes the Accumulo tables when a `DROP TABLE` command +is issued. + +External tables can be a bit more difficult to work with, as the data is stored +in an expected format. If the data is not stored correctly, then you're +gonna have a bad time. Users must provide a `column_mapping` property +when creating the table. This creates the mapping of Trino column name +to the column family/qualifier for the cell of the table. The value of the +cell is stored in the `Value` of the Accumulo key/value pair. By default, +this value is expected to be serialized using Accumulo's *lexicoder* API. +If you are storing values as strings, you can specify a different serializer +using the `serializer` property of the table. See the section on +[Table Properties](accumulo-table-properties) for more information. + +Next, we create the Trino external table. + +```sql +CREATE TABLE external_table ( + a VARCHAR, + b BIGINT, + c DATE +) +WITH ( + column_mapping = 'a:md:a,b:md:b,c:md:c', + external = true, + index_columns = 'b,c', + locality_groups = 'foo:b,c' +); +``` + +After creating the table, usage of the table continues as usual: + +```sql +INSERT INTO external_table VALUES +('1', 1, DATE '2015-03-06'), +('2', 2, DATE '2015-03-07'); +``` + +```sql +SELECT * FROM external_table; +``` + +```text + a | b | c +---+---+------------ + 1 | 1 | 2015-03-06 + 2 | 2 | 2015-03-06 +(2 rows) +``` + +```sql +DROP TABLE external_table; +``` + +After dropping the table, the table still exists in Accumulo because it is *external*. + +```text +root@default> tables +accumulo.metadata +accumulo.root +external_table +external_table_idx +external_table_idx_metrics +trace +``` + +If we wanted to add a new column to the table, we can create the table again and specify a new column. +Any existing rows in the table have a value of NULL. This command re-configures the Accumulo +tables, setting the locality groups and iterator configuration. + +```sql +CREATE TABLE external_table ( + a VARCHAR, + b BIGINT, + c DATE, + d INTEGER +) +WITH ( + column_mapping = 'a:md:a,b:md:b,c:md:c,d:md:d', + external = true, + index_columns = 'b,c,d', + locality_groups = 'foo:b,c,d' +); + +SELECT * FROM external_table; +``` + +```sql + a | b | c | d +---+---+------------+------ + 1 | 1 | 2015-03-06 | NULL + 2 | 2 | 2015-03-07 | NULL +(2 rows) +``` + +(accumulo-table-properties)= +## Table properties + +Table property usage example: + +```sql +CREATE TABLE example_schema.scientists ( + recordkey VARCHAR, + name VARCHAR, + age BIGINT, + birthday DATE +) +WITH ( + column_mapping = 'name:metadata:name,age:metadata:age,birthday:metadata:date', + index_columns = 'name,age' +); +``` + +| Property name | Default value | Description | +| ----------------- | -------------- | -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| `column_mapping` | (generated) | Comma-delimited list of column metadata: `col_name:col_family:col_qualifier,[...]`. Required for external tables. Not setting this property results in auto-generated column names. | +| `index_columns` | (none) | A comma-delimited list of Trino columns that are indexed in this table's corresponding index table | +| `external` | `false` | If true, Trino will only do metadata operations for the table. Otherwise, Trino will create and drop Accumulo tables where appropriate. | +| `locality_groups` | (none) | List of locality groups to set on the Accumulo table. Only valid on internal tables. String format is locality group name, colon, comma delimited list of column families in the group. Groups are delimited by pipes. Example: `group1:famA,famB,famC\|group2:famD,famE,famF\|etc...` | +| `row_id` | (first column) | Trino column name that maps to the Accumulo row ID. | +| `serializer` | `default` | Serializer for Accumulo data encodings. Can either be `default`, `string`, `lexicoder` or a Java class name. Default is `default`, i.e. the value from `AccumuloRowSerializer.getDefault()`, i.e. `lexicoder`. | +| `scan_auths` | (user auths) | Scan-time authorizations set on the batch scanner. | + +## Session properties + +You can change the default value of a session property by using {doc}`/sql/set-session`. +Note that session properties are prefixed with the catalog name: + +``` +SET SESSION example.column_filter_optimizations_enabled = false; +``` + +| Property name | Default value | Description | +| ------------------------------------------ | ------------- | -------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| `optimize_locality_enabled` | `true` | Set to true to enable data locality for non-indexed scans | +| `optimize_split_ranges_enabled` | `true` | Set to true to split non-indexed queries by tablet splits. Should generally be true. | +| `optimize_index_enabled` | `true` | Set to true to enable usage of the secondary index on query | +| `index_rows_per_split` | `10000` | The number of Accumulo row IDs that are packed into a single Trino split | +| `index_threshold` | `0.2` | The ratio between number of rows to be scanned based on the index over the total number of rows. If the ratio is below this threshold, the index will be used. | +| `index_lowest_cardinality_threshold` | `0.01` | The threshold where the column with the lowest cardinality will be used instead of computing an intersection of ranges in the index. Secondary index must be enabled | +| `index_metrics_enabled` | `true` | Set to true to enable usage of the metrics table to optimize usage of the index | +| `scan_username` | (config) | User to impersonate when scanning the tables. This property trumps the `scan_auths` table property | +| `index_short_circuit_cardinality_fetch` | `true` | Short circuit the retrieval of index metrics once any column is less than the lowest cardinality threshold | +| `index_cardinality_cache_polling_duration` | `10ms` | Sets the cardinality cache polling duration for short circuit retrieval of index metrics | + +## Adding columns + +Adding a new column to an existing table cannot be done today via +`ALTER TABLE [table] ADD COLUMN [name] [type]` because of the additional +metadata required for the columns to work; the column family, qualifier, +and if the column is indexed. + +## Serializers + +The Trino connector for Accumulo has a pluggable serializer framework +for handling I/O between Trino and Accumulo. This enables end-users the +ability to programmatically serialized and deserialize their special data +formats within Accumulo, while abstracting away the complexity of the +connector itself. + +There are two types of serializers currently available; a `string` +serializer that treats values as Java `String`, and a `lexicoder` +serializer that leverages Accumulo's Lexicoder API to store values. The +default serializer is the `lexicoder` serializer, as this serializer +does not require expensive conversion operations back and forth between +`String` objects and the Trino types -- the cell's value is encoded as a +byte array. + +Additionally, the `lexicoder` serializer does proper lexigraphical ordering of +numerical types like `BIGINT` or `TIMESTAMP`. This is essential for the connector +to properly leverage the secondary index when querying for data. + +You can change the default the serializer by specifying the +`serializer` table property, using either `default` (which is +`lexicoder`), `string` or `lexicoder` for the built-in types, or +you could provide your own implementation by extending +`AccumuloRowSerializer`, adding it to the Trino `CLASSPATH`, and +specifying the fully-qualified Java class name in the connector configuration. + +```sql +CREATE TABLE example_schema.scientists ( + recordkey VARCHAR, + name VARCHAR, + age BIGINT, + birthday DATE +) +WITH ( + column_mapping = 'name:metadata:name,age:metadata:age,birthday:metadata:date', + serializer = 'default' +); +``` + +```sql +INSERT INTO example_schema.scientists VALUES +('row1', 'Grace Hopper', 109, DATE '1906-12-09' ), +('row2', 'Alan Turing', 103, DATE '1912-06-23' ); +``` + +```text +root@default> scan -t example_schema.scientists +row1 metadata:age [] \x08\x80\x00\x00\x00\x00\x00\x00m +row1 metadata:date [] \x08\x7F\xFF\xFF\xFF\xFF\xFF\xA6\x06 +row1 metadata:name [] Grace Hopper +row2 metadata:age [] \x08\x80\x00\x00\x00\x00\x00\x00g +row2 metadata:date [] \x08\x7F\xFF\xFF\xFF\xFF\xFF\xAD\xED +row2 metadata:name [] Alan Turing +``` + +```sql +CREATE TABLE example_schema.stringy_scientists ( + recordkey VARCHAR, + name VARCHAR, + age BIGINT, + birthday DATE +) +WITH ( + column_mapping = 'name:metadata:name,age:metadata:age,birthday:metadata:date', + serializer = 'string' +); +``` + +```sql +INSERT INTO example_schema.stringy_scientists VALUES +('row1', 'Grace Hopper', 109, DATE '1906-12-09' ), +('row2', 'Alan Turing', 103, DATE '1912-06-23' ); +``` + +```text +root@default> scan -t example_schema.stringy_scientists +row1 metadata:age [] 109 +row1 metadata:date [] -23034 +row1 metadata:name [] Grace Hopper +row2 metadata:age [] 103 +row2 metadata:date [] -21011 +row2 metadata:name [] Alan Turing +``` + +```sql +CREATE TABLE example_schema.custom_scientists ( + recordkey VARCHAR, + name VARCHAR, + age BIGINT, + birthday DATE +) +WITH ( + column_mapping = 'name:metadata:name,age:metadata:age,birthday:metadata:date', + serializer = 'my.serializer.package.MySerializer' +); +``` + +## Metadata management + +Metadata for the Trino/Accumulo tables is stored in ZooKeeper. You can, +and should, issue SQL statements in Trino to create and drop tables. +This is the easiest method of creating the metadata required to make the +connector work. It is best to not mess with the metadata, but here are +the details of how it is stored. + +A root node in ZooKeeper holds all the mappings, and the format is as +follows: + +```text +/metadata-root/schema/table +``` + +Where `metadata-root` is the value of `zookeeper.metadata.root` in +the config file (default is `/trino-accumulo`), `schema` is the +Trino schema (which is identical to the Accumulo namespace name), and +`table` is the Trino table name (again, identical to Accumulo name). +The data of the `table` ZooKeeper node is a serialized +`AccumuloTable` Java object (which resides in the connector code). +This table contains the schema (namespace) name, table name, column +definitions, the serializer to use for the table, and any additional +table properties. + +If you have a need to programmatically manipulate the ZooKeeper metadata +for Accumulo, take a look at +`io.trino.plugin.accumulo.metadata.ZooKeeperMetadataManager` for some +Java code to simplify the process. + +## Converting table from internal to external + +If your table is *internal*, you can convert it to an external table by deleting +the corresponding znode in ZooKeeper, effectively making the table no longer exist as +far as Trino is concerned. Then, create the table again using the same DDL, but adding the +`external = true` table property. + +For example: + +1\. We're starting with an internal table `foo.bar` that was created with the below DDL. +If you have not previously defined a table property for `column_mapping` (like this example), +be sure to describe the table **before** deleting the metadata. We need the column mappings +when creating the external table. + +```sql +CREATE TABLE foo.bar (a VARCHAR, b BIGINT, c DATE) +WITH ( + index_columns = 'b,c' +); +``` + +```sql +DESCRIBE foo.bar; +``` + +```text + Column | Type | Extra | Comment +--------+---------+-------+------------------------------------- + a | varchar | | Accumulo row ID + b | bigint | | Accumulo column b:b. Indexed: true + c | date | | Accumulo column c:c. Indexed: true +``` + +2\. Using the ZooKeeper CLI, delete the corresponding znode. Note this uses the default ZooKeeper +metadata root of `/trino-accumulo` + +```text +$ zkCli.sh +[zk: localhost:2181(CONNECTED) 1] delete /trino-accumulo/foo/bar +``` + +3\. Re-create the table using the same DDL as before, but adding the `external=true` property. +Note that if you had not previously defined the column_mapping, you need to add the property +to the new DDL (external tables require this property to be set). The column mappings are in +the output of the `DESCRIBE` statement. + +```sql +CREATE TABLE foo.bar ( + a VARCHAR, + b BIGINT, + c DATE +) +WITH ( + column_mapping = 'a:a:a,b:b:b,c:c:c', + index_columns = 'b,c', + external = true +); +``` + +(accumulo-type-mapping)= + +## Type mapping + +Because Trino and Accumulo each support types that the other does not, this +connector modifies some types when reading or writing data. Data types may not +map the same way in both directions between Trino and the data source. Refer to +the following sections for type mapping in each direction. + +### Accumulo type to Trino type mapping + +The connector maps Accumulo types to the corresponding Trino types following +this table: + +```{eval-rst} +.. list-table:: Accumulo type to Trino type mapping + :widths: 30, 20, 50 + :header-rows: 1 + + * - Accumulo type + - Trino type + - Notes + * - ``BOOLEAN`` + - ``BOOLEAN`` + - + * - ``TINYINT`` + - ``TINYINT`` + - + * - ``SMALLINT`` + - ``SMALLINT`` + - + * - ``INTEGER`` + - ``INTEGER`` + - + * - ``BIGINT`` + - ``BIGINT`` + - + * - ``REAL`` + - ``REAL`` + - + * - ``DOUBLE`` + - ``DOUBLE`` + - + * - ``VARCHAR(n)`` + - ``VARCHAR(n)`` + - + * - ``VARBINARY`` + - ``VARBINARY`` + - + * - ``DATE`` + - ``DATE`` + - + * - ``TIME(n)`` + - ``TIME(n)`` + - + * - ``TIMESTAMP(n)`` + - ``TIMESTAMP(n)`` + - +``` + +No other types are supported + +### Trino type to Accumulo type mapping + +The connector maps Trino types to the corresponding Trino type to Accumulo type +mapping types following this table: + +```{eval-rst} +.. list-table:: Trino type to Accumulo type mapping + :widths: 30, 20, 50 + :header-rows: 1 + + * - Trino type + - Accumulo type + - Notes + * - ``BOOLEAN`` + - ``BOOLEAN`` + - + * - ``TINYINT`` + - ``TINYINT`` + - Trino only supports writing values belonging to ``[0, 127]`` + * - ``SMALLINT`` + - ``SMALLINT`` + - + * - ``INTEGER`` + - ``INTEGER`` + - + * - ``BIGINT`` + - ``BIGINT`` + - + * - ``REAL`` + - ``REAL`` + - + * - ``DOUBLE`` + - ``DOUBLE`` + - + * - ``VARCHAR(n)`` + - ``VARCHAR(n)`` + - + * - ``VARBINARY`` + - ``VARBINARY`` + - + * - ``DATE`` + - ``DATE`` + - + * - ``TIME(n)`` + - ``TIME(n)`` + - + * - ``TIMESTAMP(n)`` + - ``TIMESTAMP(n)`` + - +``` + +No other types are supported + +(accumulo-sql-support)= + +## SQL support + +The connector provides read and write access to data and metadata in +the Accumulo database. In addition to the {ref}`globally available +` and {ref}`read operation ` +statements, the connector supports the following features: + +- {doc}`/sql/insert` +- {doc}`/sql/create-table` +- {doc}`/sql/create-table-as` +- {doc}`/sql/drop-table` +- {doc}`/sql/create-schema` +- {doc}`/sql/drop-schema` diff --git a/430/_sources/connector/atop.md.txt b/430/_sources/connector/atop.md.txt new file mode 100644 index 000000000..64eca21db --- /dev/null +++ b/430/_sources/connector/atop.md.txt @@ -0,0 +1,146 @@ +# Atop connector + +The Atop connector supports reading disk utilization statistics from the [Atop](https://www.atoptool.nl/) +(Advanced System and Process Monitor) Linux server performance analysis tool. + +## Requirements + +In order to use this connector, the host on which the Trino worker is running +needs to have the `atop` tool installed locally. + +## Connector configuration + +The connector can read disk utilization statistics on the Trino cluster. +Create a catalog properties file that specifies the Atop connector by +setting the `connector.name` to `atop`. + +For example, create the file `etc/catalog/example.properties` with the +following connector properties as appropriate for your setup: + +```text +connector.name=atop +atop.executable-path=/usr/bin/atop +``` + +## Configuration properties + +```{eval-rst} +.. list-table:: + :widths: 42, 18, 5, 35 + :header-rows: 1 + + * - Property name + - Default value + - Required + - Description + * - ``atop.concurrent-readers-per-node`` + - ``1`` + - Yes + - The number of concurrent read operations allowed per node. + * - ``atop.executable-path`` + - (none) + - Yes + - The file path on the local file system for the ``atop`` utility. + * - ``atop.executable-read-timeout`` + - ``1ms`` + - Yes + - The timeout when reading from the atop process. + * - ``atop.max-history-days`` + - ``30`` + - Yes + - The maximum number of days in the past to take into account for statistics. + * - ``atop.security`` + - ``ALLOW_ALL`` + - Yes + - The :doc:`access control ` for the connector. + * - ``atop.time-zone`` + - System default + - Yes + - The time zone identifier in which the atop data is collected. Generally the timezone of the host. + Sample time zone identifiers: ``Europe/Vienna``, ``+0100``, ``UTC``. +``` + +## Usage + +The Atop connector provides a `default` schema. + +The tables exposed by this connector can be retrieved by running `SHOW TABLES`: + +``` +SHOW TABLES FROM example.default; +``` + +```text + Table +--------- + disks + reboots +(2 rows) +``` + +The `disks` table offers disk utilization statistics recorded on the Trino node. + +```{eval-rst} +.. list-table:: Disks columns + :widths: 30, 30, 40 + :header-rows: 1 + + * - Name + - Type + - Description + * - ``host_ip`` + - ``VARCHAR`` + - Trino worker IP + * - ``start_time`` + - ``TIMESTAMP(3) WITH TIME ZONE`` + - Interval start time for the statistics + * - ``end_time`` + - ``TIMESTAMP(3) WITH TIME ZONE`` + - Interval end time for the statistics + * - ``device_name`` + - ``VARCHAR`` + - Logical volume/hard disk name + * - ``utilization_percent`` + - ``DOUBLE`` + - The percentage of time the unit was busy handling requests + * - ``io_time`` + - ``INTERVAL DAY TO SECOND`` + - Time spent for I/O + * - ``read_requests`` + - ``BIGINT`` + - Number of reads issued + * - ``sectors_read`` + - ``BIGINT`` + - Number of sectors transferred for reads + * - ``write_requests`` + - ``BIGINT`` + - Number of writes issued + * - ``sectors_written`` + - ``BIGINT`` + - Number of sectors transferred for write +``` + +The `reboots` table offers information about the system reboots performed on the Trino node. + +```{eval-rst} +.. list-table:: Reboots columns + :widths: 30, 30, 40 + :header-rows: 1 + + * - Name + - Type + - Description + * - ``host_ip`` + - ``VARCHAR`` + - Trino worker IP + * - ``power_on_time`` + - ``TIMESTAMP(3) WITH TIME ZONE`` + - The boot/reboot timestamp + +``` + +## SQL support + +The connector provides {ref}`globally available ` and +{ref}`read operation ` statements to access system and process monitor +information on your Trino nodes. diff --git a/430/_sources/connector/bigquery.md.txt b/430/_sources/connector/bigquery.md.txt new file mode 100644 index 000000000..6cdfef2c3 --- /dev/null +++ b/430/_sources/connector/bigquery.md.txt @@ -0,0 +1,374 @@ +# BigQuery connector + +```{raw} html + +``` + +The BigQuery connector allows querying the data stored in [BigQuery](https://cloud.google.com/bigquery/). This can be used to join data between +different systems like BigQuery and Hive. The connector uses the [BigQuery +Storage API](https://cloud.google.com/bigquery/docs/reference/storage/) to +read the data from the tables. + +## BigQuery Storage API + +The Storage API streams data in parallel directly from BigQuery via gRPC without +using Google Cloud Storage as an intermediary. +It has a number of advantages over using the previous export-based read flow +that should generally lead to better read performance: + +**Direct Streaming** + +: It does not leave any temporary files in Google Cloud Storage. Rows are read + directly from BigQuery servers using an Avro wire format. + +**Column Filtering** + +: The new API allows column filtering to only read the data you are interested in. + [Backed by a columnar datastore](https://cloud.google.com/blog/products/bigquery/inside-capacitor-bigquerys-next-generation-columnar-storage-format), + it can efficiently stream data without reading all columns. + +**Dynamic Sharding** + +: The API rebalances records between readers until they all complete. This means + that all Map phases will finish nearly concurrently. See this blog article on + [how dynamic sharding is similarly used in Google Cloud Dataflow](https://cloud.google.com/blog/products/gcp/no-shard-left-behind-dynamic-work-rebalancing-in-google-cloud-dataflow). + +(bigquery-requirements)= +## Requirements + +To connect to BigQuery, you need: + +- To enable the [BigQuery Storage Read API](https://cloud.google.com/bigquery/docs/reference/storage/#enabling_the_api). + +- Network access from your Trino coordinator and workers to the + Google Cloud API service endpoint. This endpoint uses HTTPS, or port 443. + +- To configure BigQuery so that the Trino coordinator and workers have [permissions + in BigQuery](https://cloud.google.com/bigquery/docs/reference/storage#permissions). + +- To set up authentication. Your authentiation options differ depending on whether + you are using Dataproc/Google Compute Engine (GCE) or not. + + **On Dataproc/GCE** the authentication is done from the machine's role. + + **Outside Dataproc/GCE** you have 3 options: + + - Use a service account JSON key and `GOOGLE_APPLICATION_CREDENTIALS` as + described in the Google Cloud authentication [getting started guide](https://cloud.google.com/docs/authentication/getting-started). + - Set `bigquery.credentials-key` in the catalog properties file. It should + contain the contents of the JSON file, encoded using base64. + - Set `bigquery.credentials-file` in the catalog properties file. It should + point to the location of the JSON file. + +## Configuration + +To configure the BigQuery connector, create a catalog properties file in +`etc/catalog` named `example.properties`, to mount the BigQuery connector as +the `example` catalog. Create the file with the following contents, replacing +the connection properties as appropriate for your setup: + +```text +connector.name=bigquery +bigquery.project-id= +``` + +### Multiple GCP projects + +The BigQuery connector can only access a single GCP project.Thus, if you have +data in multiple GCP projects, You need to create several catalogs, each +pointing to a different GCP project. For example, if you have two GCP projects, +one for the sales and one for analytics, you can create two properties files in +`etc/catalog` named `sales.properties` and `analytics.properties`, both +having `connector.name=bigquery` but with different `project-id`. This will +create the two catalogs, `sales` and `analytics` respectively. + +### Configuring partitioning + +By default the connector creates one partition per 400MB in the table being +read (before filtering). This should roughly correspond to the maximum number +of readers supported by the BigQuery Storage API. This can be configured +explicitly with the `bigquery.parallelism` property. BigQuery may limit the +number of partitions based on server constraints. + +(bigquery-arrow-serialization-support)= +### Arrow serialization support + +This is an experimental feature which introduces support for using Apache Arrow +as the serialization format when reading from BigQuery. Please note there are +a few caveats: + +- Using Apache Arrow serialization is disabled by default. In order to enable + it, set the `bigquery.experimental.arrow-serialization.enabled` + configuration property to `true` and add + `--add-opens=java.base/java.nio=ALL-UNNAMED` to the Trino + {ref}`jvm-config`. + +(bigquery-reading-from-views)= +### Reading from views + +The connector has a preliminary support for reading from [BigQuery views](https://cloud.google.com/bigquery/docs/views-intro). Please note there are +a few caveats: + +- Reading from views is disabled by default. In order to enable it, set the + `bigquery.views-enabled` configuration property to `true`. +- BigQuery views are not materialized by default, which means that the + connector needs to materialize them before it can read them. This process + affects the read performance. +- The materialization process can also incur additional costs to your BigQuery bill. +- By default, the materialized views are created in the same project and + dataset. Those can be configured by the optional `bigquery.view-materialization-project` + and `bigquery.view-materialization-dataset` properties, respectively. The + service account must have write permission to the project and the dataset in + order to materialize the view. + +### Configuration properties + +| Property | Description | Default | +| --------------------------------------------------- | --------------------------------------------------------------------------------------------------------------------------------------------------------------- | ---------------------------------------------------- | +| `bigquery.project-id` | The Google Cloud Project ID where the data reside | Taken from the service account | +| `bigquery.parent-project-id` | The project ID Google Cloud Project to bill for the export | Taken from the service account | +| `bigquery.parallelism` | The number of partitions to split the data into | The number of executors | +| `bigquery.views-enabled` | Enables the connector to read from views and not only tables. Please read [this section](bigquery-reading-from-views) before enabling this feature. | `false` | +| `bigquery.view-expire-duration` | Expire duration for the materialized view. | `24h` | +| `bigquery.view-materialization-project` | The project where the materialized view is going to be created | The view's project | +| `bigquery.view-materialization-dataset` | The dataset where the materialized view is going to be created | The view's dataset | +| `bigquery.skip-view-materialization` | Use REST API to access views instead of Storage API. BigQuery `BIGNUMERIC` and `TIMESTAMP` types are unsupported. | `false` | +| `bigquery.views-cache-ttl` | Duration for which the materialization of a view will be cached and reused. Set to `0ms` to disable the cache. | `15m` | +| `bigquery.metadata.cache-ttl` | Duration for which metadata retrieved from BigQuery is cached and reused. Set to `0ms` to disable the cache. | `0ms` | +| `bigquery.max-read-rows-retries` | The number of retries in case of retryable server issues | `3` | +| `bigquery.credentials-key` | The base64 encoded credentials key | None. See the [requirements](bigquery-requirements) section. | +| `bigquery.credentials-file` | The path to the JSON credentials file | None. See the [requirements](bigquery-requirements) section. | +| `bigquery.case-insensitive-name-matching` | Match dataset and table names case-insensitively | `false` | +| `bigquery.query-results-cache.enabled` | Enable [query results cache](https://cloud.google.com/bigquery/docs/cached-results) | `false` | +| `bigquery.experimental.arrow-serialization.enabled` | Enable using Apache Arrow serialization when reading data from BigQuery. Please read this [section](bigquery-arrow-serialization-support) before enabling this feature. | `false` | +| `bigquery.rpc-proxy.enabled` | Use a proxy for communication with BigQuery. | `false` | +| `bigquery.rpc-proxy.uri` | Proxy URI to use if connecting through a proxy. | | +| `bigquery.rpc-proxy.username` | Proxy user name to use if connecting through a proxy. | | +| `bigquery.rpc-proxy.password` | Proxy password to use if connecting through a proxy. | | +| `bigquery.rpc-proxy.keystore-path` | Keystore containing client certificates to present to proxy if connecting through a proxy. Only required if proxy uses mutual TLS. | | +| `bigquery.rpc-proxy.keystore-password` | Password of the keystore specified by `bigquery.rpc-proxy.keystore-path`. | | +| `bigquery.rpc-proxy.truststore-path` | Truststore containing certificates of the proxy server if connecting through a proxy. | | +| `bigquery.rpc-proxy.truststore-password` | Password of the truststore specified by `bigquery.rpc-proxy.truststore-path`. | | + +(bigquery-type-mapping)= + +## Type mapping + +Because Trino and BigQuery each support types that the other does not, this +connector {ref}`modifies some types ` when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction. + +### BigQuery type to Trino type mapping + +The connector maps BigQuery types to the corresponding Trino types according +to the following table: + +```{eval-rst} +.. list-table:: BigQuery type to Trino type mapping + :widths: 30, 30, 50 + :header-rows: 1 + + * - BigQuery type + - Trino type + - Notes + * - ``BOOLEAN`` + - ``BOOLEAN`` + - + * - ``INT64`` + - ``BIGINT`` + - ``INT``, ``SMALLINT``, ``INTEGER``, ``BIGINT``, ``TINYINT``, and + ``BYTEINT`` are aliases for ``INT64`` in BigQuery. + * - ``FLOAT64`` + - ``DOUBLE`` + - + * - ``NUMERIC`` + - ``DECIMAL(P,S)`` + - The default precision and scale of ``NUMERIC`` is ``(38, 9)``. + * - ``BIGNUMERIC`` + - ``DECIMAL(P,S)`` + - Precision > 38 is not supported. The default precision and scale of + ``BIGNUMERIC`` is ``(77, 38)``. + * - ``DATE`` + - ``DATE`` + - + * - ``DATETIME`` + - ``TIMESTAMP(6)`` + - + * - ``STRING`` + - ``VARCHAR`` + - + * - ``BYTES`` + - ``VARBINARY`` + - + * - ``TIME`` + - ``TIME(6)`` + - + * - ``TIMESTAMP`` + - ``TIMESTAMP(6) WITH TIME ZONE`` + - Time zone is UTC + * - ``GEOGRAPHY`` + - ``VARCHAR`` + - In `Well-known text (WKT) `_ format + * - ``ARRAY`` + - ``ARRAY`` + - + * - ``RECORD`` + - ``ROW`` + - +``` + +No other types are supported. + +### Trino type to BigQuery type mapping + +The connector maps Trino types to the corresponding BigQuery types according +to the following table: + +```{eval-rst} +.. list-table:: Trino type to BigQuery type mapping + :widths: 30, 30, 50 + :header-rows: 1 + + * - Trino type + - BigQuery type + - Notes + * - ``BOOLEAN`` + - ``BOOLEAN`` + - + * - ``VARBINARY`` + - ``BYTES`` + - + * - ``DATE`` + - ``DATE`` + - + * - ``DOUBLE`` + - ``FLOAT`` + - + * - ``BIGINT`` + - ``INT64`` + - ``INT``, ``SMALLINT``, ``INTEGER``, ``BIGINT``, ``TINYINT``, and + ``BYTEINT`` are aliases for ``INT64`` in BigQuery. + * - ``DECIMAL(P,S)`` + - ``NUMERIC`` + - The default precision and scale of ``NUMERIC`` is ``(38, 9)``. + * - ``VARCHAR`` + - ``STRING`` + - + * - ``TIMESTAMP(6)`` + - ``DATETIME`` + - +``` + +No other types are supported. + +## System tables + +For each Trino table which maps to BigQuery view there exists a system table +which exposes BigQuery view definition. Given a BigQuery view `example_view` +you can send query `SELECT * example_view$view_definition` to see the SQL +which defines view in BigQuery. + +(bigquery-special-columns)= + +## Special columns + +In addition to the defined columns, the BigQuery connector exposes +partition information in a number of hidden columns: + +- `$partition_date`: Equivalent to `_PARTITIONDATE` pseudo-column in BigQuery +- `$partition_time`: Equivalent to `_PARTITIONTIME` pseudo-column in BigQuery + +You can use these columns in your SQL statements like any other column. They +can be selected directly, or used in conditional statements. For example, you +can inspect the partition date and time for each record: + +``` +SELECT *, "$partition_date", "$partition_time" +FROM example.web.page_views; +``` + +Retrieve all records stored in the partition `_PARTITIONDATE = '2022-04-07'`: + +``` +SELECT * +FROM example.web.page_views +WHERE "$partition_date" = date '2022-04-07'; +``` + +:::{note} +Two special partitions `__NULL__` and `__UNPARTITIONED__` are not supported. +::: + +(bigquery-sql-support)= + +## SQL support + +The connector provides read and write access to data and metadata in the +BigQuery database. In addition to the +{ref}`globally available ` and +{ref}`read operation ` statements, the connector supports +the following features: + +- {doc}`/sql/insert` +- {doc}`/sql/truncate` +- {doc}`/sql/create-table` +- {doc}`/sql/create-table-as` +- {doc}`/sql/drop-table` +- {doc}`/sql/create-schema` +- {doc}`/sql/drop-schema` +- {doc}`/sql/comment` + +(bigquery-fte-support)= + +## Fault-tolerant execution support + +The connector supports {doc}`/admin/fault-tolerant-execution` of query +processing. Read and write operations are both supported with any retry policy. + +## Table functions + +The connector provides specific {doc}`table functions ` to +access BigQuery. + +(bigquery-query-function)= + +### `query(varchar) -> table` + +The `query` function allows you to query the underlying BigQuery directly. It +requires syntax native to BigQuery, because the full query is pushed down and +processed by BigQuery. This can be useful for accessing native features which are +not available in Trino or for improving query performance in situations where +running a query natively may be faster. + +```{include} query-passthrough-warning.fragment +``` + +For example, query the `example` catalog and group and concatenate all +employee IDs by manager ID: + +``` +SELECT + * +FROM + TABLE( + example.system.query( + query => 'SELECT + manager_id, STRING_AGG(employee_id) + FROM + company.employees + GROUP BY + manager_id' + ) + ); +``` + +```{include} query-table-function-ordering.fragment +``` + +## FAQ + +### What is the Pricing for the Storage API? + +See the [BigQuery pricing documentation](https://cloud.google.com/bigquery/pricing#storage-api). diff --git a/430/_sources/connector/blackhole.md.txt b/430/_sources/connector/blackhole.md.txt new file mode 100644 index 000000000..707c20af3 --- /dev/null +++ b/430/_sources/connector/blackhole.md.txt @@ -0,0 +1,132 @@ +# Black Hole connector + +Primarily Black Hole connector is designed for high performance testing of +other components. It works like the `/dev/null` device on Unix-like +operating systems for data writing and like `/dev/null` or `/dev/zero` +for data reading. However, it also has some other features that allow testing Trino +in a more controlled manner. Metadata for any tables created via this connector +is kept in memory on the coordinator and discarded when Trino restarts. +Created tables are by default always empty, and any data written to them +is ignored and data reads return no rows. + +During table creation, a desired rows number can be specified. +In such cases, writes behave in the same way, but reads +always return the specified number of some constant rows. +You shouldn't rely on the content of such rows. + +## Configuration + +Create `etc/catalog/example.properties` to mount the `blackhole` connector +as the `example` catalog, with the following contents: + +```text +connector.name=blackhole +``` + +## Examples + +Create a table using the blackhole connector: + +``` +CREATE TABLE example.test.nation AS +SELECT * from tpch.tiny.nation; +``` + +Insert data into a table in the blackhole connector: + +``` +INSERT INTO example.test.nation +SELECT * FROM tpch.tiny.nation; +``` + +Select from the blackhole connector: + +``` +SELECT count(*) FROM example.test.nation; +``` + +The above query always returns zero. + +Create a table with a constant number of rows (500 * 1000 * 2000): + +``` +CREATE TABLE example.test.nation ( + nationkey BIGINT, + name VARCHAR +) +WITH ( + split_count = 500, + pages_per_split = 1000, + rows_per_page = 2000 +); +``` + +Now query it: + +``` +SELECT count(*) FROM example.test.nation; +``` + +The above query returns 1,000,000,000. + +Length of variable length columns can be controlled using the `field_length` +table property (default value is equal to 16): + +``` +CREATE TABLE example.test.nation ( + nationkey BIGINT, + name VARCHAR +) +WITH ( + split_count = 500, + pages_per_split = 1000, + rows_per_page = 2000, + field_length = 100 +); +``` + +The consuming and producing rate can be slowed down +using the `page_processing_delay` table property. +Setting this property to `5s` leads to a 5 second +delay before consuming or producing a new page: + +``` +CREATE TABLE example.test.delay ( + dummy BIGINT +) +WITH ( + split_count = 1, + pages_per_split = 1, + rows_per_page = 1, + page_processing_delay = '5s' +); +``` + +(blackhole-sql-support)= + +## SQL support + +The connector provides {ref}`globally available `, +{ref}`read operation `, and supports the following +additional features: + +- {doc}`/sql/insert` +- {doc}`/sql/update` +- {doc}`/sql/delete` +- {doc}`/sql/merge` +- {doc}`/sql/create-table` +- {doc}`/sql/create-table-as` +- {doc}`/sql/show-create-table` +- {doc}`/sql/drop-table` +- {doc}`/sql/comment` +- {doc}`/sql/create-schema` +- {doc}`/sql/drop-schema` +- {doc}`/sql/create-view` +- {doc}`/sql/show-create-view` +- {doc}`/sql/drop-view` + +:::{note} +The connector discards all written data. While read operations are supported, +they return rows with all NULL values, with the number of rows controlled +via table properties. +::: diff --git a/430/_sources/connector/cassandra.md.txt b/430/_sources/connector/cassandra.md.txt new file mode 100644 index 000000000..9fd930521 --- /dev/null +++ b/430/_sources/connector/cassandra.md.txt @@ -0,0 +1,371 @@ +# Cassandra connector + +```{raw} html + +``` + +The Cassandra connector allows querying data stored in +[Apache Cassandra](https://cassandra.apache.org/). + +## Requirements + +To connect to Cassandra, you need: + +- Cassandra version 3.0 or higher. +- Network access from the Trino coordinator and workers to Cassandra. + Port 9042 is the default port. + +## Configuration + +To configure the Cassandra connector, create a catalog properties file +`etc/catalog/example.properties` with the following contents, replacing +`host1,host2` with a comma-separated list of the Cassandra nodes, used to +discovery the cluster topology: + +```text +connector.name=cassandra +cassandra.contact-points=host1,host2 +cassandra.load-policy.dc-aware.local-dc=datacenter1 +``` + +You also need to set `cassandra.native-protocol-port`, if your +Cassandra nodes are not using the default port 9042. + +### Multiple Cassandra clusters + +You can have as many catalogs as you need, so if you have additional +Cassandra clusters, simply add another properties file to `etc/catalog` +with a different name, making sure it ends in `.properties`. For +example, if you name the property file `sales.properties`, Trino +creates a catalog named `sales` using the configured connector. + +## Configuration properties + +The following configuration properties are available: + +| Property name | Description | +| -------------------------------- | --------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| `cassandra.contact-points` | Comma-separated list of hosts in a Cassandra cluster. The Cassandra driver uses these contact points to discover cluster topology. At least one Cassandra host is required. | +| `cassandra.native-protocol-port` | The Cassandra server port running the native client protocol, defaults to `9042`. | +| `cassandra.consistency-level` | Consistency levels in Cassandra refer to the level of consistency to be used for both read and write operations. More information about consistency levels can be found in the [Cassandra consistency] documentation. This property defaults to a consistency level of `ONE`. Possible values include `ALL`, `EACH_QUORUM`, `QUORUM`, `LOCAL_QUORUM`, `ONE`, `TWO`, `THREE`, `LOCAL_ONE`, `ANY`, `SERIAL`, `LOCAL_SERIAL`. | +| `cassandra.allow-drop-table` | Enables {doc}`/sql/drop-table` operations. Defaults to `false`. | +| `cassandra.username` | Username used for authentication to the Cassandra cluster. This is a global setting used for all connections, regardless of the user connected to Trino. | +| `cassandra.password` | Password used for authentication to the Cassandra cluster. This is a global setting used for all connections, regardless of the user connected to Trino. | +| `cassandra.protocol-version` | It is possible to override the protocol version for older Cassandra clusters. By default, the value corresponds to the default protocol version used in the underlying Cassandra java driver. Possible values include `V3`, `V4`, `V5`, `V6`. | + +:::{note} +If authorization is enabled, `cassandra.username` must have enough permissions to perform `SELECT` queries on +the `system.size_estimates` table. +::: + +The following advanced configuration properties are available: + +| Property name | Description | +| ---------------------------------------------------------- | ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| `cassandra.fetch-size` | Number of rows fetched at a time in a Cassandra query. | +| `cassandra.partition-size-for-batch-select` | Number of partitions batched together into a single select for a single partion key column table. | +| `cassandra.split-size` | Number of keys per split when querying Cassandra. | +| `cassandra.splits-per-node` | Number of splits per node. By default, the values from the `system.size_estimates` table are used. Only override when connecting to Cassandra versions \< 2.1.5, which lacks the `system.size_estimates` table. | +| `cassandra.batch-size` | Maximum number of statements to execute in one batch. | +| `cassandra.client.read-timeout` | Maximum time the Cassandra driver waits for an answer to a query from one Cassandra node. Note that the underlying Cassandra driver may retry a query against more than one node in the event of a read timeout. Increasing this may help with queries that use an index. | +| `cassandra.client.connect-timeout` | Maximum time the Cassandra driver waits to establish a connection to a Cassandra node. Increasing this may help with heavily loaded Cassandra clusters. | +| `cassandra.client.so-linger` | Number of seconds to linger on close if unsent data is queued. If set to zero, the socket will be closed immediately. When this option is non-zero, a socket lingers that many seconds for an acknowledgement that all data was written to a peer. This option can be used to avoid consuming sockets on a Cassandra server by immediately closing connections when they are no longer needed. | +| `cassandra.retry-policy` | Policy used to retry failed requests to Cassandra. This property defaults to `DEFAULT`. Using `BACKOFF` may help when queries fail with *"not enough replicas"*. The other possible values are `DOWNGRADING_CONSISTENCY` and `FALLTHROUGH`. | +| `cassandra.load-policy.use-dc-aware` | Set to `true` if the load balancing policy requires a local datacenter, defaults to `true`. | +| `cassandra.load-policy.dc-aware.local-dc` | The name of the datacenter considered "local". | +| `cassandra.load-policy.dc-aware.used-hosts-per-remote-dc` | Uses the provided number of host per remote datacenter as failover for the local hosts for `DefaultLoadBalancingPolicy`. | +| `cassandra.load-policy.dc-aware.allow-remote-dc-for-local` | Set to `true` to allow to use hosts of remote datacenter for local consistency level. | +| `cassandra.no-host-available-retry-timeout` | Retry timeout for `AllNodesFailedException`, defaults to `1m`. | +| `cassandra.speculative-execution.limit` | The number of speculative executions. This is disabled by default. | +| `cassandra.speculative-execution.delay` | The delay between each speculative execution, defaults to `500ms`. | +| `cassandra.tls.enabled` | Whether TLS security is enabled, defaults to `false`. | +| `cassandra.tls.keystore-path` | Path to the {doc}`PEM ` or {doc}`JKS ` key store file. | +| `cassandra.tls.truststore-path` | Path to the {doc}`PEM ` or {doc}`JKS ` trust store file. | +| `cassandra.tls.keystore-password` | Password for the key store. | +| `cassandra.tls.truststore-password` | Password for the trust store. | + +## Querying Cassandra tables + +The `users` table is an example Cassandra table from the Cassandra +[Getting Started] guide. It can be created along with the `example_keyspace` +keyspace using Cassandra's cqlsh (CQL interactive terminal): + +```text +cqlsh> CREATE KEYSPACE example_keyspace + ... WITH REPLICATION = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }; +cqlsh> USE example_keyspace; +cqlsh:example_keyspace> CREATE TABLE users ( + ... user_id int PRIMARY KEY, + ... fname text, + ... lname text + ... ); +``` + +This table can be described in Trino: + +``` +DESCRIBE example.example_keyspace.users; +``` + +```text + Column | Type | Extra | Comment +---------+---------+-------+--------- + user_id | bigint | | + fname | varchar | | + lname | varchar | | +(3 rows) +``` + +This table can then be queried in Trino: + +``` +SELECT * FROM example.example_keyspace.users; +``` + +(cassandra-type-mapping)= + +## Type mapping + +Because Trino and Cassandra each support types that the other does not, this +connector {ref}`modifies some types ` when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction. + +### Cassandra type to Trino type mapping + +The connector maps Cassandra types to the corresponding Trino types according to +the following table: + +```{eval-rst} +.. list-table:: Cassandra type to Trino type mapping + :widths: 30, 25, 50 + :header-rows: 1 + + * - Cassandra type + - Trino type + - Notes + * - ``BOOLEAN`` + - ``BOOLEAN`` + - + * - ``TINYINT`` + - ``TINYINT`` + - + * - ``SMALLINT`` + - ``SMALLINT`` + - + * - ``INT`` + - ``INTEGER`` + - + * - ``BIGINT`` + - ``BIGINT`` + - + * - ``FLOAT`` + - ``REAL`` + - + * - ``DOUBLE`` + - ``DOUBLE`` + - + * - ``DECIMAL`` + - ``DOUBLE`` + - + * - ``ASCII`` + - ``VARCHAR`` + - US-ASCII character string + * - ``TEXT`` + - ``VARCHAR`` + - UTF-8 encoded string + * - ``VARCHAR`` + - ``VARCHAR`` + - UTF-8 encoded string + * - ``VARINT`` + - ``VARCHAR`` + - Arbitrary-precision integer + * - ``BLOB`` + - ``VARBINARY`` + - + * - ``DATE`` + - ``DATE`` + - + * - ``TIME`` + - ``TIME(9)`` + - + * - ``TIMESTAMP`` + - ``TIMESTAMP(3) WITH TIME ZONE`` + - + * - ``LIST`` + - ``VARCHAR`` + - + * - ``MAP`` + - ``VARCHAR`` + - + * - ``SET`` + - ``VARCHAR`` + - + * - ``TUPLE`` + - ``ROW`` with anonymous fields + - + * - ``UDT`` + - ``ROW`` with field names + - + * - ``INET`` + - ``IPADDRESS`` + - + * - ``UUID`` + - ``UUID`` + - + * - ``TIMEUUID`` + - ``UUID`` + - +``` + +No other types are supported. + +### Trino type to Cassandra type mapping + +The connector maps Trino types to the corresponding Cassandra types according to +the following table: + +```{eval-rst} +.. list-table:: Trino type to Cassandra type mapping + :widths: 30, 25, 50 + :header-rows: 1 + + * - Trino type + - Cassandra type + - Notes + + * - ``BOOLEAN`` + - ``BOOLEAN`` + - + * - ``TINYINT`` + - ``TINYINT`` + - + * - ``SMALLINT`` + - ``SMALLINT`` + - + * - ``INTEGER`` + - ``INT`` + - + * - ``BIGINT`` + - ``BIGINT`` + - + * - ``REAL`` + - ``FLOAT`` + - + * - ``DOUBLE`` + - ``DOUBLE`` + - + * - ``VARCHAR`` + - ``TEXT`` + - + * - ``DATE`` + - ``DATE`` + - + * - ``TIMESTAMP(3) WITH TIME ZONE`` + - ``TIMESTAMP`` + - + * - ``IPADDRESS`` + - ``INET`` + - + * - ``UUID`` + - ``UUID`` + - + +``` + +No other types are supported. + +## Partition key types + +Partition keys can only be of the following types: + +- ASCII +- TEXT +- VARCHAR +- BIGINT +- BOOLEAN +- DOUBLE +- INET +- INT +- FLOAT +- DECIMAL +- TIMESTAMP +- UUID +- TIMEUUID + +## Limitations + +- Queries without filters containing the partition key result in fetching all partitions. + This causes a full scan of the entire data set, and is therefore much slower compared to a similar + query with a partition key as a filter. +- `IN` list filters are only allowed on index (that is, partition key or clustering key) columns. +- Range (`<` or `>` and `BETWEEN`) filters can be applied only to the partition keys. + +(cassandra-sql-support)= + +## SQL support + +The connector provides read and write access to data and metadata in +the Cassandra database. In addition to the {ref}`globally available +` and {ref}`read operation ` +statements, the connector supports the following features: + +- {doc}`/sql/insert` +- {doc}`/sql/delete` see {ref}`sql-delete-limitation` +- {doc}`/sql/truncate` +- {doc}`/sql/create-table` +- {doc}`/sql/create-table-as` +- {doc}`/sql/drop-table` + +## Table functions + +The connector provides specific {doc}`table functions ` to +access Cassandra. +.. \_cassandra-query-function: + +### `query(varchar) -> table` + +The `query` function allows you to query the underlying Cassandra directly. It +requires syntax native to Cassandra, because the full query is pushed down and +processed by Cassandra. This can be useful for accessing native features which are +not available in Trino or for improving query performance in situations where +running a query natively may be faster. + +```{include} query-table-function-ordering.fragment +``` + +As a simple example, to select an entire table: + +``` +SELECT + * +FROM + TABLE( + example.system.query( + query => 'SELECT + * + FROM + tpch.nation' + ) + ); +``` + +### DROP TABLE + +By default, `DROP TABLE` operations are disabled on Cassandra catalogs. To +enable `DROP TABLE`, set the `cassandra.allow-drop-table` catalog +configuration property to `true`: + +```properties +cassandra.allow-drop-table=true +``` + +(sql-delete-limitation)= + +### SQL delete limitation + +`DELETE` is only supported if the `WHERE` clause matches entire partitions. + +[cassandra consistency]: https://docs.datastax.com/en/cassandra-oss/2.2/cassandra/dml/dmlConfigConsistency.html +[getting started]: https://cassandra.apache.org/doc/latest/cassandra/getting_started/index.html diff --git a/430/_sources/connector/clickhouse.md.txt b/430/_sources/connector/clickhouse.md.txt new file mode 100644 index 000000000..e01b664aa --- /dev/null +++ b/430/_sources/connector/clickhouse.md.txt @@ -0,0 +1,366 @@ +--- +myst: + substitutions: + default_domain_compaction_threshold: "`1000`" +--- + +# ClickHouse connector + +```{raw} html + +``` + +The ClickHouse connector allows querying tables in an external +[ClickHouse](https://clickhouse.com/) server. This can be used to +query data in the databases on that server, or combine it with other data +from different catalogs accessing ClickHouse or any other supported data source. + +## Requirements + +To connect to a ClickHouse server, you need: + +- ClickHouse (version 21.8 or higher) or Altinity (version 20.8 or higher). +- Network access from the Trino coordinator and workers to the ClickHouse + server. Port 8123 is the default port. + +## Configuration + +The connector can query a ClickHouse server. Create a catalog properties file +that specifies the ClickHouse connector by setting the `connector.name` to +`clickhouse`. + +For example, create the file `etc/catalog/example.properties`. Replace the +connection properties as appropriate for your setup: + +```none +connector.name=clickhouse +connection-url=jdbc:clickhouse://host1:8123/ +connection-user=exampleuser +connection-password=examplepassword +``` + +The `connection-url` defines the connection information and parameters to pass +to the ClickHouse JDBC driver. The supported parameters for the URL are +available in the [ClickHouse JDBC driver configuration](https://clickhouse.com/docs/en/integrations/java#configuration). + +The `connection-user` and `connection-password` are typically required and +determine the user credentials for the connection, often a service user. You can +use {doc}`secrets ` to avoid actual values in the catalog +properties files. + +(clickhouse-tls)= + +### Connection security + +If you have TLS configured with a globally-trusted certificate installed on your +data source, you can enable TLS between your cluster and the data +source by appending a parameter to the JDBC connection string set in the +`connection-url` catalog configuration property. + +For example, with version 2.6.4 of the ClickHouse JDBC driver, enable TLS by +appending the `ssl=true` parameter to the `connection-url` configuration +property: + +```properties +connection-url=jdbc:clickhouse://host1:8443/?ssl=true +``` + +For more information on TLS configuration options, see the [Clickhouse JDBC +driver documentation](https://clickhouse.com/docs/en/interfaces/jdbc/) + +```{include} jdbc-authentication.fragment +``` + +### Multiple ClickHouse servers + +If you have multiple ClickHouse servers you need to configure one +catalog for each server. To add another catalog: + +- Add another properties file to `etc/catalog` +- Save it with a different name that ends in `.properties` + +For example, if you name the property file `sales.properties`, Trino uses the +configured connector to create a catalog named `sales`. + +```{include} jdbc-common-configurations.fragment +``` + +```{include} query-comment-format.fragment +``` + +```{include} jdbc-domain-compaction-threshold.fragment +``` + +```{include} jdbc-procedures.fragment +``` + +```{include} jdbc-case-insensitive-matching.fragment +``` + +```{include} non-transactional-insert.fragment +``` + +## Querying ClickHouse + +The ClickHouse connector provides a schema for every ClickHouse *database*. +Run `SHOW SCHEMAS` to see the available ClickHouse databases: + +``` +SHOW SCHEMAS FROM example; +``` + +If you have a ClickHouse database named `web`, run `SHOW TABLES` to view the +tables in this database: + +``` +SHOW TABLES FROM example.web; +``` + +Run `DESCRIBE` or `SHOW COLUMNS` to list the columns in the `clicks` table +in the `web` databases: + +``` +DESCRIBE example.web.clicks; +SHOW COLUMNS FROM example.web.clicks; +``` + +Run `SELECT` to access the `clicks` table in the `web` database: + +``` +SELECT * FROM example.web.clicks; +``` + +:::{note} +If you used a different name for your catalog properties file, use +that catalog name instead of `example` in the above examples. +::: + +## Table properties + +Table property usage example: + +``` +CREATE TABLE default.trino_ck ( + id int NOT NULL, + birthday DATE NOT NULL, + name VARCHAR, + age BIGINT, + logdate DATE NOT NULL +) +WITH ( + engine = 'MergeTree', + order_by = ARRAY['id', 'birthday'], + partition_by = ARRAY['toYYYYMM(logdate)'], + primary_key = ARRAY['id'], + sample_by = 'id' +); +``` + +The following are supported ClickHouse table properties from [https://clickhouse.tech/docs/en/engines/table-engines/mergetree-family/mergetree/](https://clickhouse.tech/docs/en/engines/table-engines/mergetree-family/mergetree/) + +| Property name | Default value | Description | +| -------------- | ------------- | ----------------------------------------------------------------------------------------------------------------------- | +| `engine` | `Log` | Name and parameters of the engine. | +| `order_by` | (none) | Array of columns or expressions to concatenate to create the sorting key. Required if `engine` is `MergeTree`. | +| `partition_by` | (none) | Array of columns or expressions to use as nested partition keys. Optional. | +| `primary_key` | (none) | Array of columns or expressions to concatenate to create the primary key. Optional. | +| `sample_by` | (none) | An expression to use for [sampling](https://clickhouse.tech/docs/en/sql-reference/statements/select/sample/). Optional. | + +Currently the connector only supports `Log` and `MergeTree` table engines +in create table statement. `ReplicatedMergeTree` engine is not yet supported. + +(clickhouse-type-mapping)= + +## Type mapping + +Because Trino and ClickHouse each support types that the other does not, this +connector {ref}`modifies some types ` when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction. + +### ClickHouse type to Trino type mapping + +The connector maps ClickHouse types to the corresponding Trino types according +to the following table: + +```{eval-rst} +.. list-table:: ClickHouse type to Trino type mapping + :widths: 30, 25, 50 + :header-rows: 1 + + * - ClickHouse type + - Trino type + - Notes + * - ``Int8`` + - ``TINYINT`` + - ``TINYINT``, ``BOOL``, ``BOOLEAN``, and ``INT1`` are aliases of ``Int8`` + * - ``Int16`` + - ``SMALLINT`` + - ``SMALLINT`` and ``INT2`` are aliases of ``Int16`` + * - ``Int32`` + - ``INTEGER`` + - ``INT``, ``INT4``, and ``INTEGER`` are aliases of ``Int32`` + * - ``Int64`` + - ``BIGINT`` + - ``BIGINT`` is an alias of ``Int64`` + * - ``UInt8`` + - ``SMALLINT`` + - + * - ``UInt16`` + - ``INTEGER`` + - + * - ``UInt32`` + - ``BIGINT`` + - + * - ``UInt64`` + - ``DECIMAL(20,0)`` + - + * - ``Float32`` + - ``REAL`` + - ``FLOAT`` is an alias of ``Float32`` + * - ``Float64`` + - ``DOUBLE`` + - ``DOUBLE`` is an alias of ``Float64`` + * - ``Decimal`` + - ``DECIMAL`` + - + * - ``FixedString`` + - ``VARBINARY`` + - Enabling ``clickhouse.map-string-as-varchar`` config property changes the + mapping to ``VARCHAR`` + * - ``String`` + - ``VARBINARY`` + - Enabling ``clickhouse.map-string-as-varchar`` config property changes the + mapping to ``VARCHAR`` + * - ``Date`` + - ``DATE`` + - + * - ``DateTime[(timezone)]`` + - ``TIMESTAMP(0) [WITH TIME ZONE]`` + - + * - ``IPv4`` + - ``IPADDRESS`` + - + * - ``IPv6`` + - ``IPADDRESS`` + - + * - ``Enum8`` + - ``VARCHAR`` + - + * - ``Enum16`` + - ``VARCHAR`` + - + * - ``UUID`` + - ``UUID`` + - +``` + +No other types are supported. + +### Trino type to ClickHouse type mapping + +The connector maps Trino types to the corresponding ClickHouse types according +to the following table: + +```{eval-rst} +.. list-table:: Trino type to ClickHouse type mapping + :widths: 30, 25, 50 + :header-rows: 1 + + * - Trino type + - ClickHouse type + - Notes + * - ``BOOLEAN`` + - ``UInt8`` + - + * - ``TINYINT`` + - ``Int8`` + - ``TINYINT``, ``BOOL``, ``BOOLEAN``, and ``INT1`` are aliases of ``Int8`` + * - ``SMALLINT`` + - ``Int16`` + - ``SMALLINT`` and ``INT2`` are aliases of ``Int16`` + * - ``INTEGER`` + - ``Int32`` + - ``INT``, ``INT4``, and ``INTEGER`` are aliases of ``Int32`` + * - ``BIGINT`` + - ``Int64`` + - ``BIGINT`` is an alias of ``Int64`` + * - ``REAL`` + - ``Float32`` + - ``FLOAT`` is an alias of ``Float32`` + * - ``DOUBLE`` + - ``Float64`` + - ``DOUBLE`` is an alias of ``Float64`` + * - ``DECIMAL(p,s)`` + - ``Decimal(p,s)`` + - + * - ``VARCHAR`` + - ``String`` + - + * - ``CHAR`` + - ``String`` + - + * - ``VARBINARY`` + - ``String`` + - Enabling ``clickhouse.map-string-as-varchar`` config property changes the + mapping to ``VARCHAR`` + * - ``DATE`` + - ``Date`` + - + * - ``TIMESTAMP(0)`` + - ``DateTime`` + - + * - ``UUID`` + - ``UUID`` + - +``` + +No other types are supported. + +```{include} jdbc-type-mapping.fragment +``` + +(clickhouse-sql-support)= + +## SQL support + +The connector provides read and write access to data and metadata in +a ClickHouse catalog. In addition to the {ref}`globally available +` and {ref}`read operation ` +statements, the connector supports the following features: + +- {doc}`/sql/insert` +- {doc}`/sql/truncate` +- {ref}`sql-schema-table-management` + +```{include} alter-schema-limitation.fragment +``` + +## Performance + +The connector includes a number of performance improvements, detailed in the +following sections. + +(clickhouse-pushdown)= + +### Pushdown + +The connector supports pushdown for a number of operations: + +- {ref}`limit-pushdown` + +{ref}`Aggregate pushdown ` for the following functions: + +- {func}`avg` +- {func}`count` +- {func}`max` +- {func}`min` +- {func}`sum` + + +```{include} pushdown-correctness-behavior.fragment +``` + +```{include} no-pushdown-text-type.fragment +``` diff --git a/430/_sources/connector/delta-lake.md.txt b/430/_sources/connector/delta-lake.md.txt new file mode 100644 index 000000000..0bcbbc91f --- /dev/null +++ b/430/_sources/connector/delta-lake.md.txt @@ -0,0 +1,1096 @@ +# Delta Lake connector + +```{raw} html + +``` + +The Delta Lake connector allows querying data stored in the [Delta Lake](https://delta.io) format, including [Databricks Delta Lake](https://docs.databricks.com/delta/index.html). The connector can natively +read the Delta Lake transaction log and thus detect when external systems change +data. + +## Requirements + +To connect to Databricks Delta Lake, you need: + +- Tables written by Databricks Runtime 7.3 LTS, 9.1 LTS, 10.4 LTS, 11.3 LTS, + 12.2 LTS and 13.3 LTS are supported. +- Deployments using AWS, HDFS, Azure Storage, and Google Cloud Storage (GCS) are + fully supported. +- Network access from the coordinator and workers to the Delta Lake storage. +- Access to the Hive metastore service (HMS) of Delta Lake or a separate HMS, + or a Glue metastore. +- Network access to the HMS from the coordinator and workers. Port 9083 is the + default port for the Thrift protocol used by the HMS. +- Data files stored in the Parquet file format. These can be configured using + {ref}`file format configuration properties ` per + catalog. + +## General configuration + +To configure the Delta Lake connector, create a catalog properties file +`etc/catalog/example.properties` that references the `delta_lake` +connector and defines a metastore. You must configure a metastore for table +metadata. If you are using a {ref}`Hive metastore `, +`hive.metastore.uri` must be configured: + +```properties +connector.name=delta_lake +hive.metastore.uri=thrift://example.net:9083 +``` + +If you are using {ref}`AWS Glue ` as your metastore, you +must instead set `hive.metastore` to `glue`: + +```properties +connector.name=delta_lake +hive.metastore=glue +``` + +Each metastore type has specific configuration properties along with +{ref}`general metastore configuration properties `. + +The connector recognizes Delta Lake tables created in the metastore by the Databricks +runtime. If non-Delta Lake tables are present in the metastore as well, they are not +visible to the connector. + +To configure access to S3 and S3-compatible storage, Azure storage, and others, +consult the appropriate section of the Hive documentation: + +- {doc}`Amazon S3 ` +- {doc}`Azure storage documentation ` +- {ref}`GCS ` + +### Delta Lake general configuration properties + +The following configuration properties are all using reasonable, tested default +values. Typical usage does not require you to configure them. + +```{eval-rst} +.. list-table:: Delta Lake configuration properties + :widths: 30, 55, 15 + :header-rows: 1 + + * - Property name + - Description + - Default + * - ``delta.metadata.cache-ttl`` + - Frequency of checks for metadata updates equivalent to transactions to + update the metadata cache specified in :ref:`prop-type-duration`. + - ``5m`` + * - ``delta.metadata.cache-size`` + - The maximum number of Delta table metadata entries to cache. + - ``1000`` + * - ``delta.metadata.live-files.cache-size`` + - Amount of memory allocated for caching information about files. Must + be specified in :ref:`prop-type-data-size` values such as ``64MB``. + Default is calculated to 10% of the maximum memory allocated to the JVM. + - + * - ``delta.metadata.live-files.cache-ttl`` + - Caching duration for active files that correspond to the Delta Lake + tables. + - ``30m`` + * - ``delta.compression-codec`` + - The compression codec to be used when writing new data files. + Possible values are: + + * ``NONE`` + * ``SNAPPY`` + * ``ZSTD`` + * ``GZIP`` + + The equivalent catalog session property is ``compression_codec``. + - ``SNAPPY`` + * - ``delta.max-partitions-per-writer`` + - Maximum number of partitions per writer. + - ``100`` + * - ``delta.hide-non-delta-lake-tables`` + - Hide information about tables that are not managed by Delta Lake. Hiding + only applies to tables with the metadata managed in a Glue catalog, and + does not apply to usage with a Hive metastore service. + - ``false`` + * - ``delta.enable-non-concurrent-writes`` + - Enable :ref:`write support ` for all + supported file systems. Specifically, take note of the warning about + concurrency and checkpoints. + - ``false`` + * - ``delta.default-checkpoint-writing-interval`` + - Default integer count to write transaction log checkpoint entries. If + the value is set to N, then checkpoints are written after every Nth + statement performing table writes. The value can be overridden for a + specific table with the ``checkpoint_interval`` table property. + - ``10`` + * - ``delta.hive-catalog-name`` + - Name of the catalog to which ``SELECT`` queries are redirected when a + Hive table is detected. + - + * - ``delta.checkpoint-row-statistics-writing.enabled`` + - Enable writing row statistics to checkpoint files. + - ``true`` + * - ``delta.dynamic-filtering.wait-timeout`` + - Duration to wait for completion of :doc:`dynamic filtering + ` during split generation. + The equivalent catalog session property is + ``dynamic_filtering_wait_timeout``. + - + * - ``delta.table-statistics-enabled`` + - Enables :ref:`Table statistics ` for + performance improvements. The equivalent catalog session property + is ``statistics_enabled``. + - ``true`` + * - ``delta.extended-statistics.enabled`` + - Enable statistics collection with :doc:`/sql/analyze` and + use of extended statistics. The equivalent catalog session property + is ``extended_statistics_enabled``. + - ``true`` + * - ``delta.extended-statistics.collect-on-write`` + - Enable collection of extended statistics for write operations. + The equivalent catalog session property is + ``extended_statistics_collect_on_write``. + - ``true`` + * - ``delta.per-transaction-metastore-cache-maximum-size`` + - Maximum number of metastore data objects per transaction in + the Hive metastore cache. + - ``1000`` + * - ``delta.delete-schema-locations-fallback`` + - Whether schema locations are deleted when Trino can't + determine whether they contain external files. + - ``false`` + * - ``delta.parquet.time-zone`` + - Time zone for Parquet read and write. + - JVM default + * - ``delta.target-max-file-size`` + - Target maximum size of written files; the actual size could be larger. + The equivalent catalog session property is ``target_max_file_size``. + - ``1GB`` + * - ``delta.unique-table-location`` + - Use randomized, unique table locations. + - ``true`` + * - ``delta.register-table-procedure.enabled`` + - Enable to allow users to call the ``register_table`` procedure. + - ``false`` + * - ``delta.vacuum.min-retention`` + - Minimum retention threshold for the files taken into account + for removal by the :ref:`VACUUM` procedure. + The equivalent catalog session property is + ``vacuum_min_retention``. + - ``7 DAYS`` +``` + +### Catalog session properties + +The following table describes {ref}`catalog session properties +` supported by the Delta Lake connector: + +```{eval-rst} +.. list-table:: Catalog session properties + :widths: 40, 60, 20 + :header-rows: 1 + + * - Property name + - Description + - Default + * - ``parquet_max_read_block_size`` + - The maximum block size used when reading Parquet files. + - ``16MB`` + * - ``parquet_writer_block_size`` + - The maximum block size created by the Parquet writer. + - ``128MB`` + * - ``parquet_writer_page_size`` + - The maximum page size created by the Parquet writer. + - ``1MB`` + * - ``parquet_writer_batch_size`` + - Maximum number of rows processed by the Parquet writer in a batch. + - ``10000`` + * - ``projection_pushdown_enabled`` + - Read only projected fields from row columns while performing ``SELECT`` queries + - ``true`` +``` + +(delta-lake-type-mapping)= + +## Type mapping + +Because Trino and Delta Lake each support types that the other does not, this +connector {ref}`modifies some types ` when reading or +writing data. Data types might not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction. + +See the [Delta Transaction Log specification](https://github.com/delta-io/delta/blob/master/PROTOCOL.md#primitive-types) +for more information about supported data types in the Delta Lake table format +specification. + +### Delta Lake to Trino type mapping + +The connector maps Delta Lake types to the corresponding Trino types following +this table: + +```{eval-rst} +.. list-table:: Delta Lake to Trino type mapping + :widths: 40, 60 + :header-rows: 1 + + * - Delta Lake type + - Trino type + * - ``BOOLEAN`` + - ``BOOLEAN`` + * - ``INTEGER`` + - ``INTEGER`` + * - ``BYTE`` + - ``TINYINT`` + * - ``SHORT`` + - ``SMALLINT`` + * - ``LONG`` + - ``BIGINT`` + * - ``FLOAT`` + - ``REAL`` + * - ``DOUBLE`` + - ``DOUBLE`` + * - ``DECIMAL(p,s)`` + - ``DECIMAL(p,s)`` + * - ``STRING`` + - ``VARCHAR`` + * - ``BINARY`` + - ``VARBINARY`` + * - ``DATE`` + - ``DATE`` + * - ``TIMESTAMPNTZ`` (``TIMESTAMP_NTZ``) + - ``TIMESTAMP(6)`` + * - ``TIMESTAMP`` + - ``TIMESTAMP(3) WITH TIME ZONE`` + * - ``ARRAY`` + - ``ARRAY`` + * - ``MAP`` + - ``MAP`` + * - ``STRUCT(...)`` + - ``ROW(...)`` +``` + +No other types are supported. + +### Trino to Delta Lake type mapping + +The connector maps Trino types to the corresponding Delta Lake types following +this table: + +```{eval-rst} +.. list-table:: Trino to Delta Lake type mapping + :widths: 60, 40 + :header-rows: 1 + + * - Trino type + - Delta Lake type + * - ``BOOLEAN`` + - ``BOOLEAN`` + * - ``INTEGER`` + - ``INTEGER`` + * - ``TINYINT`` + - ``BYTE`` + * - ``SMALLINT`` + - ``SHORT`` + * - ``BIGINT`` + - ``LONG`` + * - ``REAL`` + - ``FLOAT`` + * - ``DOUBLE`` + - ``DOUBLE`` + * - ``DECIMAL(p,s)`` + - ``DECIMAL(p,s)`` + * - ``VARCHAR`` + - ``STRING`` + * - ``VARBINARY`` + - ``BINARY`` + * - ``DATE`` + - ``DATE`` + * - ``TIMESTAMP`` + - ``TIMESTAMPNTZ`` (``TIMESTAMP_NTZ``) + * - ``TIMESTAMP(3) WITH TIME ZONE`` + - ``TIMESTAMP`` + * - ``ARRAY`` + - ``ARRAY`` + * - ``MAP`` + - ``MAP`` + * - ``ROW(...)`` + - ``STRUCT(...)`` +``` + +No other types are supported. + +## Security + +The Delta Lake connector allows you to choose one of several means of providing +authorization at the catalog level. You can select a different type of +authorization check in different Delta Lake catalog files. + +(delta-lake-authorization)= + +### Authorization checks + +Enable authorization checks for the connector by setting the `delta.security` +property in the catalog properties file. This property must be one of the +security values in the following table: + +```{eval-rst} +.. list-table:: Delta Lake security values + :widths: 30, 60 + :header-rows: 1 + + * - Property value + - Description + * - ``ALLOW_ALL`` (default value) + - No authorization checks are enforced. + * - ``SYSTEM`` + - The connector relies on system-level access control. + * - ``READ_ONLY`` + - Operations that read data or metadata, such as :doc:`/sql/select` are + permitted. No operations that write data or metadata, such as + :doc:`/sql/create-table`, :doc:`/sql/insert`, or :doc:`/sql/delete` are + allowed. + * - ``FILE`` + - Authorization checks are enforced using a catalog-level access control + configuration file whose path is specified in the ``security.config-file`` + catalog configuration property. See + :ref:`catalog-file-based-access-control` for information on the + authorization configuration file. +``` + +(delta-lake-sql-support)= + +## SQL support + +The connector provides read and write access to data and metadata in +Delta Lake. In addition to the {ref}`globally available +` and {ref}`read operation ` +statements, the connector supports the following features: + +- {ref}`sql-write-operations`: + + - {ref}`sql-data-management`, see details for {ref}`Delta Lake data + management ` + - {ref}`sql-schema-table-management`, see details for {ref}`Delta Lake schema + and table management ` + - {ref}`sql-view-management` + +### Procedures + +Use the {doc}`/sql/call` statement to perform data manipulation or +administrative tasks. Procedures are available in the system schema of each +catalog. The following code snippet displays how to call the +`example_procedure` in the `examplecatalog` catalog: + +``` +CALL examplecatalog.system.example_procedure() +``` + +(delta-lake-register-table)= + +#### Register table + +The connector can register table into the metastore with existing transaction +logs and data files. + +The `system.register_table` procedure allows the caller to register an +existing Delta Lake table in the metastore, using its existing transaction logs +and data files: + +``` +CALL example.system.register_table(schema_name => 'testdb', table_name => 'customer_orders', table_location => 's3://my-bucket/a/path') +``` + +To prevent unauthorized users from accessing data, this procedure is disabled by +default. The procedure is enabled only when +`delta.register-table-procedure.enabled` is set to `true`. + +(delta-lake-unregister-table)= + +#### Unregister table + +The connector can unregister existing Delta Lake tables from the metastore. + +The procedure `system.unregister_table` allows the caller to unregister an +existing Delta Lake table from the metastores without deleting the data: + +``` +CALL example.system.unregister_table(schema_name => 'testdb', table_name => 'customer_orders') +``` + +(delta-lake-flush-metadata-cache)= + +#### Flush metadata cache + +- `system.flush_metadata_cache()` + + Flushes all metadata caches. + +- `system.flush_metadata_cache(schema_name => ..., table_name => ...)` + + Flushes metadata cache entries of a specific table. + Procedure requires passing named parameters. + +(delta-lake-vacuum)= + +#### `VACUUM` + +The `VACUUM` procedure removes all old files that are not in the transaction +log, as well as files that are not needed to read table snapshots newer than the +current time minus the retention period defined by the `retention period` +parameter. + +Users with `INSERT` and `DELETE` permissions on a table can run `VACUUM` +as follows: + +```shell +CALL example.system.vacuum('exampleschemaname', 'exampletablename', '7d'); +``` + +All parameters are required and must be presented in the following order: + +- Schema name +- Table name +- Retention period + +The `delta.vacuum.min-retention` configuration property provides a safety +measure to ensure that files are retained as expected. The minimum value for +this property is `0s`. There is a minimum retention session property as well, +`vacuum_min_retention`. + +(delta-lake-data-management)= + +### Data management + +You can use the connector to {doc}`/sql/insert`, {doc}`/sql/delete`, +{doc}`/sql/update`, and {doc}`/sql/merge` data in Delta Lake tables. + +Write operations are supported for tables stored on the following systems: + +- Azure ADLS Gen2, Google Cloud Storage + + Writes to the Azure ADLS Gen2 and Google Cloud Storage are + enabled by default. Trino detects write collisions on these storage systems + when writing from multiple Trino clusters, or from other query engines. + +- S3 and S3-compatible storage + + Writes to {doc}`Amazon S3 ` and S3-compatible storage must be enabled + with the `delta.enable-non-concurrent-writes` property. Writes to S3 can + safely be made from multiple Trino clusters; however, write collisions are not + detected when writing concurrently from other Delta Lake engines. You must + make sure that no concurrent data modifications are run to avoid data + corruption. + +(delta-lake-schema-table-management)= + +### Schema and table management + +The {ref}`sql-schema-table-management` functionality includes support for: + +- {doc}`/sql/create-table` +- {doc}`/sql/create-table-as` +- {doc}`/sql/drop-table` +- {doc}`/sql/alter-table`, see details for {ref}`Delta Lake ALTER TABLE + ` +- {doc}`/sql/create-schema` +- {doc}`/sql/drop-schema` +- {doc}`/sql/alter-schema` +- {doc}`/sql/comment` + +The connector supports creating schemas. You can create a schema with or without +a specified location. + +You can create a schema with the {doc}`/sql/create-schema` statement and the +`location` schema property. Tables in this schema are located in a +subdirectory under the schema location. Data files for tables in this schema +using the default location are cleaned up if the table is dropped: + +``` +CREATE SCHEMA example.example_schema +WITH (location = 's3://my-bucket/a/path'); +``` + +Optionally, the location can be omitted. Tables in this schema must have a +location included when you create them. The data files for these tables are not +removed if the table is dropped: + +``` +CREATE SCHEMA example.example_schema; +``` + +When Delta Lake tables exist in storage but not in the metastore, Trino can be +used to register the tables: + +``` +CREATE TABLE example.default.example_table ( + dummy BIGINT +) +WITH ( + location = '...' +) +``` + +Columns listed in the DDL, such as `dummy` in the preceding example, are +ignored. The table schema is read from the transaction log instead. If the +schema is changed by an external system, Trino automatically uses the new +schema. + +:::{warning} +Using `CREATE TABLE` with an existing table content is deprecated, instead +use the `system.register_table` procedure. The `CREATE TABLE ... WITH +(location=...)` syntax can be temporarily re-enabled using the +`delta.legacy-create-table-with-existing-location.enabled` catalog +configuration property or +`legacy_create_table_with_existing_location_enabled` catalog session +property. +::: + +If the specified location does not already contain a Delta table, the connector +automatically writes the initial transaction log entries and registers the table +in the metastore. As a result, any Databricks engine can write to the table: + +``` +CREATE TABLE example.default.new_table (id BIGINT, address VARCHAR); +``` + +The Delta Lake connector also supports creating tables using the {doc}`CREATE +TABLE AS ` syntax. + +(delta-lake-alter-table)= + +The connector supports the following [](/sql/alter-table) statements. + +#### ALTER TABLE EXECUTE + +The connector supports the following commands for use with {ref}`ALTER TABLE +EXECUTE `. + +```{include} optimize.fragment +``` + +(delta-lake-alter-table-rename-to)= + +#### ALTER TABLE RENAME TO + +The connector only supports the `ALTER TABLE RENAME TO` statement when met with +one of the following conditions: + +* The table type is external. +* The table is backed by a metastore that does not perform object storage + operations, for example, AWS Glue or Thrift. + +#### Table properties + +The following table properties are available for use: + +```{eval-rst} +.. list-table:: Delta Lake table properties + :widths: 40, 60 + :header-rows: 1 + + * - Property name + - Description + * - ``location`` + - File system location URI for the table. + * - ``partitioned_by`` + - Set partition columns. + * - ``checkpoint_interval`` + - Set the checkpoint interval in number of table writes. + * - ``change_data_feed_enabled`` + - Enables storing change data feed entries. + * - ``column_mapping_mode`` + - Column mapping mode. Possible values are: + + * ``ID`` + * ``NAME`` + * ``NONE`` + + Defaults to ``NONE``. +``` + +The following example uses all available table properties: + +``` +CREATE TABLE example.default.example_partitioned_table +WITH ( + location = 's3://my-bucket/a/path', + partitioned_by = ARRAY['regionkey'], + checkpoint_interval = 5, + change_data_feed_enabled = false, + column_mapping_mode = 'name' +) +AS SELECT name, comment, regionkey FROM tpch.tiny.nation; +``` + +#### Metadata tables + +The connector exposes several metadata tables for each Delta Lake table. +These metadata tables contain information about the internal structure +of the Delta Lake table. You can query each metadata table by appending the +metadata table name to the table name: + +``` +SELECT * FROM "test_table$history" +``` + +##### `$history` table + +The `$history` table provides a log of the metadata changes performed on +the Delta Lake table. + +You can retrieve the changelog of the Delta Lake table `test_table` +by using the following query: + +``` +SELECT * FROM "test_table$history" +``` + +```text + version | timestamp | user_id | user_name | operation | operation_parameters | cluster_id | read_version | isolation_level | is_blind_append +---------+---------------------------------------+---------+-----------+--------------+---------------------------------------+---------------------------------+--------------+-------------------+---------------- + 2 | 2023-01-19 07:40:54.684 Europe/Vienna | trino | trino | WRITE | {queryId=20230119_064054_00008_4vq5t} | trino-406-trino-coordinator | 2 | WriteSerializable | true + 1 | 2023-01-19 07:40:41.373 Europe/Vienna | trino | trino | ADD COLUMNS | {queryId=20230119_064041_00007_4vq5t} | trino-406-trino-coordinator | 0 | WriteSerializable | true + 0 | 2023-01-19 07:40:10.497 Europe/Vienna | trino | trino | CREATE TABLE | {queryId=20230119_064010_00005_4vq5t} | trino-406-trino-coordinator | 0 | WriteSerializable | true +``` + +The output of the query has the following history columns: + +```{eval-rst} +.. list-table:: History columns + :widths: 30, 30, 40 + :header-rows: 1 + + * - Name + - Type + - Description + * - ``version`` + - ``BIGINT`` + - The version of the table corresponding to the operation + * - ``timestamp`` + - ``TIMESTAMP(3) WITH TIME ZONE`` + - The time when the table version became active + * - ``user_id`` + - ``VARCHAR`` + - The identifier for the user which performed the operation + * - ``user_name`` + - ``VARCHAR`` + - The username for the user which performed the operation + * - ``operation`` + - ``VARCHAR`` + - The name of the operation performed on the table + * - ``operation_parameters`` + - ``map(VARCHAR, VARCHAR)`` + - Parameters of the operation + * - ``cluster_id`` + - ``VARCHAR`` + - The ID of the cluster which ran the operation + * - ``read_version`` + - ``BIGINT`` + - The version of the table which was read in order to perform the operation + * - ``isolation_level`` + - ``VARCHAR`` + - The level of isolation used to perform the operation + * - ``is_blind_append`` + - ``BOOLEAN`` + - Whether or not the operation appended data +``` + +##### `$properties` table + +The `$properties` table provides access to Delta Lake table configuration, +table features and table properties. The table rows are key/value pairs. + +You can retrieve the properties of the Delta +table `test_table` by using the following query: + +``` +SELECT * FROM "test_table$properties" +``` + +```text + key | value | +----------------------------+-----------------+ +delta.minReaderVersion | 1 | +delta.minWriterVersion | 4 | +delta.columnMapping.mode | name | +delta.feature.columnMapping | supported | +``` + +(delta-lake-special-columns)= + +#### Metadata columns + +In addition to the defined columns, the Delta Lake connector automatically +exposes metadata in a number of hidden columns in each table. You can use these +columns in your SQL statements like any other column, e.g., they can be selected +directly or used in conditional statements. + +- `$path` + : Full file system path name of the file for this row. +- `$file_modified_time` + : Date and time of the last modification of the file for this row. +- `$file_size` + : Size of the file for this row. + +(delta-lake-fte-support)= + +## Fault-tolerant execution support + +The connector supports {doc}`/admin/fault-tolerant-execution` of query +processing. Read and write operations are both supported with any retry policy. + +## Table functions + +The connector provides the following table functions: + +### table_changes + +Allows reading Change Data Feed (CDF) entries to expose row-level changes +between two versions of a Delta Lake table. When the `change_data_feed_enabled` +table property is set to `true` on a specific Delta Lake table, +the connector records change events for all data changes on the table. +This is how these changes can be read: + +```sql +SELECT + * +FROM + TABLE( + system.table_changes( + schema_name => 'test_schema', + table_name => 'tableName', + since_version => 0 + ) + ); +``` + +`schema_name` - type `VARCHAR`, required, name of the schema for which the function is called + +`table_name` - type `VARCHAR`, required, name of the table for which the function is called + +`since_version` - type `BIGINT`, optional, version from which changes are shown, exclusive + +In addition to returning the columns present in the table, the function +returns the following values for each change event: + +- `_change_type` + : Gives the type of change that occurred. Possible values are `insert`, + `delete`, `update_preimage` and `update_postimage`. +- `_commit_version` + : Shows the table version for which the change occurred. +- `_commit_timestamp` + : Represents the timestamp for the commit in which the specified change happened. + +This is how it would be normally used: + +Create table: + +```sql +CREATE TABLE test_schema.pages (page_url VARCHAR, domain VARCHAR, views INTEGER) + WITH (change_data_feed_enabled = true); +``` + +Insert data: + +```sql +INSERT INTO test_schema.pages + VALUES + ('url1', 'domain1', 1), + ('url2', 'domain2', 2), + ('url3', 'domain1', 3); +INSERT INTO test_schema.pages + VALUES + ('url4', 'domain1', 400), + ('url5', 'domain2', 500), + ('url6', 'domain3', 2); +``` + +Update data: + +```sql +UPDATE test_schema.pages + SET domain = 'domain4' + WHERE views = 2; +``` + +Select changes: + +```sql +SELECT + * +FROM + TABLE( + system.table_changes( + schema_name => 'test_schema', + table_name => 'pages', + since_version => 1 + ) + ) +ORDER BY _commit_version ASC; +``` + +The preceding sequence of SQL statements returns the following result: + +```text +page_url | domain | views | _change_type | _commit_version | _commit_timestamp +url4 | domain1 | 400 | insert | 2 | 2023-03-10T21:22:23.000+0000 +url5 | domain2 | 500 | insert | 2 | 2023-03-10T21:22:23.000+0000 +url6 | domain3 | 2 | insert | 2 | 2023-03-10T21:22:23.000+0000 +url2 | domain2 | 2 | update_preimage | 3 | 2023-03-10T22:23:24.000+0000 +url2 | domain4 | 2 | update_postimage | 3 | 2023-03-10T22:23:24.000+0000 +url6 | domain3 | 2 | update_preimage | 3 | 2023-03-10T22:23:24.000+0000 +url6 | domain4 | 2 | update_postimage | 3 | 2023-03-10T22:23:24.000+0000 +``` + +The output shows what changes happen in which version. +For example in version 3 two rows were modified, first one changed from +`('url2', 'domain2', 2)` into `('url2', 'domain4', 2)` and the second from +`('url6', 'domain2', 2)` into `('url6', 'domain4', 2)`. + +If `since_version` is not provided the function produces change events +starting from when the table was created. + +```sql +SELECT + * +FROM + TABLE( + system.table_changes( + schema_name => 'test_schema', + table_name => 'pages' + ) + ) +ORDER BY _commit_version ASC; +``` + +The preceding SQL statement returns the following result: + +```text +page_url | domain | views | _change_type | _commit_version | _commit_timestamp +url1 | domain1 | 1 | insert | 1 | 2023-03-10T20:21:22.000+0000 +url2 | domain2 | 2 | insert | 1 | 2023-03-10T20:21:22.000+0000 +url3 | domain1 | 3 | insert | 1 | 2023-03-10T20:21:22.000+0000 +url4 | domain1 | 400 | insert | 2 | 2023-03-10T21:22:23.000+0000 +url5 | domain2 | 500 | insert | 2 | 2023-03-10T21:22:23.000+0000 +url6 | domain3 | 2 | insert | 2 | 2023-03-10T21:22:23.000+0000 +url2 | domain2 | 2 | update_preimage | 3 | 2023-03-10T22:23:24.000+0000 +url2 | domain4 | 2 | update_postimage | 3 | 2023-03-10T22:23:24.000+0000 +url6 | domain3 | 2 | update_preimage | 3 | 2023-03-10T22:23:24.000+0000 +url6 | domain4 | 2 | update_postimage | 3 | 2023-03-10T22:23:24.000+0000 +``` + +You can see changes that occurred at version 1 as three inserts. They are +not visible in the previous statement when `since_version` value was set to 1. + +## Performance + +The connector includes a number of performance improvements detailed in the +following sections: + +- Support for {doc}`write partitioning `. + +(delta-lake-table-statistics)= + +### Table statistics + +Use {doc}`/sql/analyze` statements in Trino to populate data size and +number of distinct values (NDV) extended table statistics in Delta Lake. +The minimum value, maximum value, value count, and null value count +statistics are computed on the fly out of the transaction log of the +Delta Lake table. The {doc}`cost-based optimizer +` then uses these statistics to improve +query performance. + +Extended statistics enable a broader set of optimizations, including join +reordering. The controlling catalog property `delta.table-statistics-enabled` +is enabled by default. The equivalent {ref}`catalog session property +` is `statistics_enabled`. + +Each `ANALYZE` statement updates the table statistics incrementally, so only +the data changed since the last `ANALYZE` is counted. The table statistics are +not automatically updated by write operations such as `INSERT`, `UPDATE`, +and `DELETE`. You must manually run `ANALYZE` again to update the table +statistics. + +To collect statistics for a table, execute the following statement: + +``` +ANALYZE table_schema.table_name; +``` + +To recalculate from scratch the statistics for the table use additional parameter `mode`: + +> ANALYZE table_schema.table_name WITH(mode = 'full_refresh'); + +There are two modes available `full_refresh` and `incremental`. +The procedure use `incremental` by default. + +To gain the most benefit from cost-based optimizations, run periodic `ANALYZE` +statements on every large table that is frequently queried. + +#### Fine-tuning + +The `files_modified_after` property is useful if you want to run the +`ANALYZE` statement on a table that was previously analyzed. You can use it to +limit the amount of data used to generate the table statistics: + +```SQL +ANALYZE example_table WITH(files_modified_after = TIMESTAMP '2021-08-23 +16:43:01.321 Z') +``` + +As a result, only files newer than the specified time stamp are used in the +analysis. + +You can also specify a set or subset of columns to analyze using the `columns` +property: + +```SQL +ANALYZE example_table WITH(columns = ARRAY['nationkey', 'regionkey']) +``` + +To run `ANALYZE` with `columns` more than once, the next `ANALYZE` must +run on the same set or a subset of the original columns used. + +To broaden the set of `columns`, drop the statistics and reanalyze the table. + +#### Disable and drop extended statistics + +You can disable extended statistics with the catalog configuration property +`delta.extended-statistics.enabled` set to `false`. Alternatively, you can +disable it for a session, with the {doc}`catalog session property +` `extended_statistics_enabled` set to `false`. + +If a table is changed with many delete and update operation, calling `ANALYZE` +does not result in accurate statistics. To correct the statistics, you have to +drop the extended statistics and analyze the table again. + +Use the `system.drop_extended_stats` procedure in the catalog to drop the +extended statistics for a specified table in a specified schema: + +``` +CALL example.system.drop_extended_stats('example_schema', 'example_table') +``` + +### Memory usage + +The Delta Lake connector is memory intensive and the amount of required memory +grows with the size of Delta Lake transaction logs of any accessed tables. It is +important to take that into account when provisioning the coordinator. + +You must decrease memory usage by keeping the number of active data files in +the table low by regularly running `OPTIMIZE` and `VACUUM` in Delta Lake. + +#### Memory monitoring + +When using the Delta Lake connector, you must monitor memory usage on the +coordinator. Specifically, monitor JVM heap utilization using standard tools as +part of routine operation of the cluster. + +A good proxy for memory usage is the cache utilization of Delta Lake caches. It +is exposed by the connector with the +`plugin.deltalake.transactionlog:name=,type=transactionlogaccess` +JMX bean. + +You can access it with any standard monitoring software with JMX support, or use +the {doc}`/connector/jmx` with the following query: + +``` +SELECT * FROM jmx.current."*.plugin.deltalake.transactionlog:name=,type=transactionlogaccess" +``` + +Following is an example result: + +```text +datafilemetadatacachestats.hitrate | 0.97 +datafilemetadatacachestats.missrate | 0.03 +datafilemetadatacachestats.requestcount | 3232 +metadatacachestats.hitrate | 0.98 +metadatacachestats.missrate | 0.02 +metadatacachestats.requestcount | 6783 +node | trino-master +object_name | io.trino.plugin.deltalake.transactionlog:type=TransactionLogAccess,name=delta +``` + +In a healthy system, both `datafilemetadatacachestats.hitrate` and +`metadatacachestats.hitrate` are close to `1.0`. + +(delta-lake-table-redirection)= + +### Table redirection + +```{include} table-redirection.fragment +``` + +The connector supports redirection from Delta Lake tables to Hive tables +with the `delta.hive-catalog-name` catalog configuration property. + +### Performance tuning configuration properties + +The following table describes performance tuning catalog properties specific to +the Delta Lake connector. + +:::{warning} +Performance tuning configuration properties are considered expert-level +features. Altering these properties from their default values is likely to +cause instability and performance degradation. It is strongly suggested that +you use them only to address non-trivial performance issues, and that you +keep a backup of the original values if you change them. +::: + +```{eval-rst} +.. list-table:: Delta Lake performance tuning configuration properties + :widths: 30, 50, 20 + :header-rows: 1 + + * - Property name + - Description + - Default + * - ``delta.domain-compaction-threshold`` + - Minimum size of query predicates above which Trino compacts the + predicates. Pushing a large list of predicates down to the data source + can compromise performance. For optimization in that situation, Trino + can compact the large predicates. If necessary, adjust the threshold to + ensure a balance between performance and predicate pushdown. + - ``1000`` + * - ``delta.max-outstanding-splits`` + - The target number of buffered splits for each table scan in a query, + before the scheduler tries to pause. + - ``1000`` + * - ``delta.max-splits-per-second`` + - Sets the maximum number of splits used per second to access underlying + storage. Reduce this number if your limit is routinely exceeded, based + on your filesystem limits. This is set to the absolute maximum value, + which results in Trino maximizing the parallelization of data access + by default. Attempting to set it higher results in Trino not being + able to start. + - ``Integer.MAX_VALUE`` + * - ``delta.max-initial-splits`` + - For each query, the coordinator assigns file sections to read first + at the ``initial-split-size`` until the ``max-initial-splits`` is + reached. Then it starts issuing reads of the ``max-split-size`` size. + - ``200`` + * - ``delta.max-initial-split-size`` + - Sets the initial :ref:`prop-type-data-size` for a single read section + assigned to a worker until ``max-initial-splits`` have been processed. + You can also use the corresponding catalog session property + ``.max_initial_split_size``. + - ``32MB`` + * - ``delta.max-split-size`` + - Sets the largest :ref:`prop-type-data-size` for a single read section + assigned to a worker after ``max-initial-splits`` have been processed. + You can also use the corresponding catalog session property + ``.max_split_size``. + - ``64MB`` + * - ``delta.minimum-assigned-split-weight`` + - A decimal value in the range (0, 1] used as a minimum for weights + assigned to each split. A low value might improve performance on tables + with small files. A higher value might improve performance for queries + with highly skewed aggregations or joins. + - ``0.05`` + * - ``delta.projection-pushdown-enabled`` + - Read only projected fields from row columns while performing ``SELECT`` queries + - ``true`` + * - ``delta.query-partition-filter-required`` + - Set to ``true`` to force a query to use a partition filter. You can use + the ``query_partition_filter_required`` catalog session property for + temporary, catalog specific use. + - ``false`` +``` diff --git a/430/_sources/connector/druid.md.txt b/430/_sources/connector/druid.md.txt new file mode 100644 index 000000000..a01f3b612 --- /dev/null +++ b/430/_sources/connector/druid.md.txt @@ -0,0 +1,167 @@ +--- +myst: + substitutions: + default_domain_compaction_threshold: '`32`' +--- + +# Druid connector + +```{raw} html + +``` + +The Druid connector allows querying an [Apache Druid](https://druid.apache.org/) +database from Trino. + +## Requirements + +To connect to Druid, you need: + +- Druid version 0.18.0 or higher. +- Network access from the Trino coordinator and workers to your Druid broker. + Port 8082 is the default port. + +## Configuration + +Create a catalog properties file that specifies the Druid connector by setting +the `connector.name` to `druid` and configuring the `connection-url` with +the JDBC string to connect to Druid. + +For example, to access a database as `example`, create the file +`etc/catalog/example.properties`. Replace `BROKER:8082` with the correct +host and port of your Druid broker. + +```properties +connector.name=druid +connection-url=jdbc:avatica:remote:url=http://BROKER:8082/druid/v2/sql/avatica/ +``` + +You can add authentication details to connect to a Druid deployment that is +secured by basic authentication by updating the URL and adding credentials: + +```properties +connection-url=jdbc:avatica:remote:url=http://BROKER:port/druid/v2/sql/avatica/;authentication=BASIC +connection-user=root +connection-password=secret +``` + +Now you can access your Druid database in Trino with the `example` catalog +name from the properties file. + +The `connection-user` and `connection-password` are typically required and +determine the user credentials for the connection, often a service user. You can +use {doc}`secrets ` to avoid actual values in the catalog +properties files. + +```{include} jdbc-authentication.fragment +``` + +```{include} jdbc-common-configurations.fragment +``` + +```{include} query-comment-format.fragment +``` + +```{include} jdbc-domain-compaction-threshold.fragment +``` + +```{include} jdbc-procedures.fragment +``` + +```{include} jdbc-case-insensitive-matching.fragment +``` + +(druid-type-mapping)= + +## Type mapping + +Because Trino and Druid each support types that the other does not, this +connector {ref}`modifies some types ` when reading data. + +### Druid type to Trino type mapping + +The connector maps Druid types to the corresponding Trino types according to the +following table: + +```{eval-rst} +.. list-table:: Druid type to Trino type mapping + :widths: 30, 30, 50 + :header-rows: 1 + + * - Druid type + - Trino type + - Notes + * - ``STRING`` + - ``VARCHAR`` + - + * - ``FLOAT`` + - ``REAL`` + - + * - ``DOUBLE`` + - ``DOUBLE`` + - + * - ``LONG`` + - ``BIGINT`` + - Except for the special ``_time`` column, which is mapped to ``TIMESTAMP``. + * - ``TIMESTAMP`` + - ``TIMESTAMP`` + - Only applicable to the special ``_time`` column. +``` + +No other data types are supported. + +Druid does not have a real `NULL` value for any data type. By +default, Druid treats `NULL` as the default value for a data type. For +example, `LONG` would be `0`, `DOUBLE` would be `0.0`, `STRING` would +be an empty string `''`, and so forth. + +```{include} jdbc-type-mapping.fragment +``` + +(druid-sql-support)= + +## SQL support + +The connector provides {ref}`globally available ` and +{ref}`read operation ` statements to access data and +metadata in the Druid database. + +## Table functions + +The connector provides specific {doc}`table functions ` to +access Druid. + +(druid-query-function)= + +### `query(varchar) -> table` + +The `query` function allows you to query the underlying database directly. It +requires syntax native to Druid, because the full query is pushed down and +processed in Druid. This can be useful for accessing native features which are +not available in Trino or for improving query performance in situations where +running a query natively may be faster. + +```{include} query-passthrough-warning.fragment +``` + +As an example, query the `example` catalog and use `STRING_TO_MV` and +`MV_LENGTH` from [Druid SQL's multi-value string functions](https://druid.apache.org/docs/latest/querying/sql-multivalue-string-functions.html) +to split and then count the number of comma-separated values in a column: + +``` +SELECT + num_reports +FROM + TABLE( + example.system.query( + query => 'SELECT + MV_LENGTH( + STRING_TO_MV(direct_reports, ",") + ) AS num_reports + FROM company.managers' + ) + ); +``` + +```{include} query-table-function-ordering.fragment +``` diff --git a/430/_sources/connector/elasticsearch.md.txt b/430/_sources/connector/elasticsearch.md.txt new file mode 100644 index 000000000..5ec58563f --- /dev/null +++ b/430/_sources/connector/elasticsearch.md.txt @@ -0,0 +1,447 @@ +# Elasticsearch connector + +```{raw} html + +``` + +The Elasticsearch Connector allows access to [Elasticsearch](https://www.elastic.co/products/elasticsearch) data from Trino. +This document describes how to setup the Elasticsearch Connector to run SQL queries against Elasticsearch. + +:::{note} +Elasticsearch (6.6.0 or later) or OpenSearch (1.1.0 or later) is required. +::: + +## Configuration + +To configure the Elasticsearch connector, create a catalog properties file +`etc/catalog/example.properties` with the following contents, replacing the +properties as appropriate for your setup: + +```text +connector.name=elasticsearch +elasticsearch.host=localhost +elasticsearch.port=9200 +elasticsearch.default-schema-name=default +``` + +### Configuration properties + +```{eval-rst} +.. list-table:: Elasticsearch configuration properties + :widths: 35, 55, 10 + :header-rows: 1 + + * - Property name + - Description + - Default + * - ``elasticsearch.host`` + - The comma-separated list of host names for the Elasticsearch node to + connect to. This property is required. + - + * - ``elasticsearch.port`` + - Port of the Elasticsearch node to connect to. + - ``9200`` + * - ``elasticsearch.default-schema-name`` + - The schema that contains all tables defined without a qualifying schema + name. + - ``default`` + * - ``elasticsearch.scroll-size`` + - Sets the maximum number of hits that can be returned with each + Elasticsearch scroll request. + - ``1000`` + * - ``elasticsearch.scroll-timeout`` + - Amount of time Elasticsearch keeps the + `search context `_ + alive for scroll requests. + - ``1m`` + * - ``elasticsearch.request-timeout`` + - Timeout value for all Elasticsearch requests. + - ``10s`` + * - ``elasticsearch.connect-timeout`` + - Timeout value for all Elasticsearch connection attempts. + - ``1s`` + * - ``elasticsearch.backoff-init-delay`` + - The minimum duration between backpressure retry attempts for a single + request to Elasticsearch. Setting it too low might overwhelm an already + struggling ES cluster. + - ``500ms`` + * - ``elasticsearch.backoff-max-delay`` + - The maximum duration between backpressure retry attempts for a single + request to Elasticsearch. + - ``20s`` + * - ``elasticsearch.max-retry-time`` + - The maximum duration across all retry attempts for a single request to + Elasticsearch. + - ``20s`` + * - ``elasticsearch.node-refresh-interval`` + - How often the list of available Elasticsearch nodes is refreshed. + - ``1m`` + * - ``elasticsearch.ignore-publish-address`` + - Disables using the address published by Elasticsearch to connect for + queries. + - +``` + +## TLS security + +The Elasticsearch connector provides additional security options to support +Elasticsearch clusters that have been configured to use TLS. + +If your cluster has globally-trusted certificates, you should only need to +enable TLS. If you require custom configuration for certificates, the connector +supports key stores and trust stores in PEM or Java Key Store (JKS) format. + +The allowed configuration values are: + +```{eval-rst} +.. list-table:: TLS Security Properties + :widths: 40, 60 + :header-rows: 1 + + * - Property name + - Description + * - ``elasticsearch.tls.enabled`` + - Enables TLS security. + * - ``elasticsearch.tls.keystore-path`` + - The path to the :doc:`PEM ` or + :doc:`JKS ` key store. + * - ``elasticsearch.tls.truststore-path`` + - The path to :doc:`PEM ` or + :doc:`JKS ` trust store. + * - ``elasticsearch.tls.keystore-password`` + - The key password for the key store specified by + ``elasticsearch.tls.keystore-path``. + * - ``elasticsearch.tls.truststore-password`` + - The key password for the trust store specified by + ``elasticsearch.tls.truststore-path``. +``` + +(elasticesearch-type-mapping)= + +## Type mapping + +Because Trino and Elasticsearch each support types that the other does not, this +connector {ref}`maps some types ` when reading data. + +### Elasticsearch type to Trino type mapping + +The connector maps Elasticsearch types to the corresponding Trino types +according to the following table: + +```{eval-rst} +.. list-table:: Elasticsearch type to Trino type mapping + :widths: 30, 30, 50 + :header-rows: 1 + + * - Elasticsearch type + - Trino type + - Notes + * - ``BOOLEAN`` + - ``BOOLEAN`` + - + * - ``DOUBLE`` + - ``DOUBLE`` + - + * - ``FLOAT`` + - ``REAL`` + - + * - ``BYTE`` + - ``TINYINT`` + - + * - ``SHORT`` + - ``SMALLINT`` + - + * - ``INTEGER`` + - ``INTEGER`` + - + * - ``LONG`` + - ``BIGINT`` + - + * - ``KEYWORD`` + - ``VARCHAR`` + - + * - ``TEXT`` + - ``VARCHAR`` + - + * - ``DATE`` + - ``TIMESTAMP`` + - For more information, see :ref:`elasticsearch-date-types`. + * - ``IPADDRESS`` + - ``IP`` + - +``` + +No other types are supported. + +(elasticsearch-array-types)= + +### Array types + +Fields in Elasticsearch can contain [zero or more values](https://www.elastic.co/guide/en/elasticsearch/reference/current/array.html) +, but there is no dedicated array type. To indicate a field contains an array, it can be annotated in a Trino-specific structure in +the [\_meta](https://www.elastic.co/guide/en/elasticsearch/reference/current/mapping-meta-field.html) section of the index mapping. + +For example, you can have an Elasticsearch index that contains documents with the following structure: + +```json +{ + "array_string_field": ["trino","the","lean","machine-ohs"], + "long_field": 314159265359, + "id_field": "564e6982-88ee-4498-aa98-df9e3f6b6109", + "timestamp_field": "1987-09-17T06:22:48.000Z", + "object_field": { + "array_int_field": [86,75,309], + "int_field": 2 + } +} +``` + +The array fields of this structure can be defined by using the following command to add the field +property definition to the `_meta.trino` property of the target index mapping. + +```shell +curl --request PUT \ + --url localhost:9200/doc/_mapping \ + --header 'content-type: application/json' \ + --data ' +{ + "_meta": { + "trino":{ + "array_string_field":{ + "isArray":true + }, + "object_field":{ + "array_int_field":{ + "isArray":true + } + }, + } + } +}' +``` + +:::{note} +It is not allowed to use `asRawJson` and `isArray` flags simultaneously for the same column. +::: + +(elasticsearch-date-types)= + +### Date types + +Elasticsearch supports a wide array of [date] formats including +[built-in date formats] and also [custom date formats]. +The Elasticsearch connector supports only the default `date` type. All other +date formats including [built-in date formats] and [custom date formats] are +not supported. Dates with the [format] property are ignored. + +### Raw JSON transform + +There are many occurrences where documents in Elasticsearch have more complex +structures that are not represented in the mapping. For example, a single +`keyword` field can have widely different content including a single +`keyword` value, an array, or a multidimensional `keyword` array with any +level of nesting. + +```shell +curl --request PUT \ + --url localhost:9200/doc/_mapping \ + --header 'content-type: application/json' \ + --data ' +{ + "properties": { + "array_string_field":{ + "type": "keyword" + } + } +}' +``` + +Notice for the `array_string_field` that all the following documents are legal +for Elasticsearch. See the [Elasticsearch array documentation](https://www.elastic.co/guide/en/elasticsearch/reference/current/array.html) +for more details. + +```json +[ + { + "array_string_field": "trino" + }, + { + "array_string_field": ["trino","is","the","besto"] + }, + { + "array_string_field": ["trino",["is","the","besto"]] + }, + { + "array_string_field": ["trino",["is",["the","besto"]]] + } +] +``` + +Further, Elasticsearch supports types, such as +[dense_vector](https://www.elastic.co/guide/en/elasticsearch/reference/current/dense-vector.html), +that are not supported in Trino. New types are constantly emerging which can +cause parsing exceptions for users that use of these types in Elasticsearch. To +manage all of these scenarios, you can transform fields to raw JSON by +annotating it in a Trino-specific structure in the [\_meta](https://www.elastic.co/guide/en/elasticsearch/reference/current/mapping-meta-field.html) +section of the index mapping. This indicates to Trino that the field, and all +nested fields beneath, need to be cast to a `VARCHAR` field that contains +the raw JSON content. These fields can be defined by using the following command +to add the field property definition to the `_meta.presto` property of the +target index mapping. + +```shell +curl --request PUT \ + --url localhost:9200/doc/_mapping \ + --header 'content-type: application/json' \ + --data ' +{ + "_meta": { + "presto":{ + "array_string_field":{ + "asRawJson":true + } + } + } +}' +``` + +This preceding configurations causes Trino to return the `array_string_field` +field as a `VARCHAR` containing raw JSON. You can parse these fields with the +{doc}`built-in JSON functions `. + +:::{note} +It is not allowed to use `asRawJson` and `isArray` flags simultaneously for the same column. +::: + +## Special columns + +The following hidden columns are available: + +| Column | Description | +| -------- | ------------------------------------------------------ | +| \_id | The Elasticsearch document ID | +| \_score | The document score returned by the Elasticsearch query | +| \_source | The source of the original document | + +(elasticsearch-full-text-queries)= + +## Full text queries + +Trino SQL queries can be combined with Elasticsearch queries by providing the [full text query] +as part of the table name, separated by a colon. For example: + +```sql +SELECT * FROM "tweets: +trino SQL^2" +``` + +## Predicate push down + +The connector supports predicate push down of below data types: + +| Elasticsearch | Trino | Supports | +| ------------- | ------------- | ------------- | +| `binary` | `VARBINARY` | `NO` | +| `boolean` | `BOOLEAN` | `YES` | +| `double` | `DOUBLE` | `YES` | +| `float` | `REAL` | `YES` | +| `byte` | `TINYINT` | `YES` | +| `short` | `SMALLINT` | `YES` | +| `integer` | `INTEGER` | `YES` | +| `long` | `BIGINT` | `YES` | +| `keyword` | `VARCHAR` | `YES` | +| `text` | `VARCHAR` | `NO` | +| `date` | `TIMESTAMP` | `YES` | +| `ip` | `IPADDRESS` | `NO` | +| (all others) | (unsupported) | (unsupported) | + +## AWS authorization + +To enable AWS authorization using IAM policies, the `elasticsearch.security` option needs to be set to `AWS`. +Additionally, the following options need to be configured appropriately: + +| Property name | Description | +| ------------------------------- | ----------------------------------------------------------------------------------------------------------------------------------------------- | +| `elasticsearch.aws.region` | AWS region or the Elasticsearch endpoint. This option is required. | +| `elasticsearch.aws.access-key` | AWS access key to use to connect to the Elasticsearch domain. If not set, the Default AWS Credentials Provider chain will be used. | +| `elasticsearch.aws.secret-key` | AWS secret key to use to connect to the Elasticsearch domain. If not set, the Default AWS Credentials Provider chain will be used. | +| `elasticsearch.aws.iam-role` | Optional ARN of an IAM Role to assume to connect to the Elasticsearch domain. Note: the configured IAM user has to be able to assume this role. | +| `elasticsearch.aws.external-id` | Optional external ID to pass while assuming an AWS IAM Role. | + +## Password authentication + +To enable password authentication, the `elasticsearch.security` option needs to be set to `PASSWORD`. +Additionally the following options need to be configured appropriately: + +| Property name | Description | +| ----------------------------- | --------------------------------------------- | +| `elasticsearch.auth.user` | User name to use to connect to Elasticsearch. | +| `elasticsearch.auth.password` | Password to use to connect to Elasticsearch. | + +(elasticsearch-sql-support)= + +## SQL support + +The connector provides {ref}`globally available ` and +{ref}`read operation ` statements to access data and +metadata in the Elasticsearch catalog. + +## Table functions + +The connector provides specific {doc}`table functions ` to +access Elasticsearch. + +(elasticsearch-raw-query-function)= + +### `raw_query(varchar) -> table` + +The `raw_query` function allows you to query the underlying database directly. +This function requires [Elastic Query DSL](https://www.elastic.co/guide/en/elasticsearch/reference/current/query-dsl.html) +syntax, because the full query is pushed down and processed in Elasticsearch. +This can be useful for accessing native features which are not available in +Trino or for improving query performance in situations where running a query +natively may be faster. + +```{eval-rst} +.. include:: query-passthrough-warning.fragment +``` + +The `raw_query` function requires three parameters: + +- `schema`: The schema in the catalog that the query is to be executed on. +- `index`: The index in Elasticsearch to be searched. +- `query`: The query to be executed, written in Elastic Query DSL. + +Once executed, the query returns a single row containing the resulting JSON +payload returned by Elasticsearch. + +For example, query the `example` catalog and use the `raw_query` table +function to search for documents in the `orders` index where the country name +is `ALGERIA`: + +``` +SELECT + * +FROM + TABLE( + example.system.raw_query( + schema => 'sales', + index => 'orders', + query => '{ + "query": { + "match": { + "name": "ALGERIA" + } + } + }' + ) + ); +``` + +```{include} query-table-function-ordering.fragment +``` + +[built-in date formats]: https://www.elastic.co/guide/en/elasticsearch/reference/current/mapping-date-format.html#built-in-date-formats +[custom date formats]: https://www.elastic.co/guide/en/elasticsearch/reference/current/mapping-date-format.html#custom-date-formats +[date]: https://www.elastic.co/guide/en/elasticsearch/reference/current/date.html +[format]: https://www.elastic.co/guide/en/elasticsearch/reference/current/mapping-date-format.html#mapping-date-format +[full text query]: https://www.elastic.co/guide/en/elasticsearch/reference/current/query-dsl-query-string-query.html#query-string-syntax diff --git a/430/_sources/connector/googlesheets.md.txt b/430/_sources/connector/googlesheets.md.txt new file mode 100644 index 000000000..8615d6d58 --- /dev/null +++ b/430/_sources/connector/googlesheets.md.txt @@ -0,0 +1,175 @@ +# Google Sheets connector + +```{raw} html + +``` + +The Google Sheets connector allows reading and writing [Google Sheets](https://www.google.com/sheets/about/) spreadsheets as tables in Trino. + +## Configuration + +Create `etc/catalog/example.properties` to mount the Google Sheets connector +as the `example` catalog, with the following contents: + +```text +connector.name=gsheets +gsheets.credentials-path=/path/to/google-sheets-credentials.json +gsheets.metadata-sheet-id=exampleId +``` + +## Configuration properties + +The following configuration properties are available: + +| Property name | Description | +| ----------------------------- | ---------------------------------------------------------------- | +| `gsheets.credentials-path` | Path to the Google API JSON key file | +| `gsheets.credentials-key` | The base64 encoded credentials key | +| `gsheets.metadata-sheet-id` | Sheet ID of the spreadsheet, that contains the table mapping | +| `gsheets.max-data-cache-size` | Maximum number of spreadsheets to cache, defaults to `1000` | +| `gsheets.data-cache-ttl` | How long to cache spreadsheet data or metadata, defaults to `5m` | +| `gsheets.connection-timeout` | Timeout when connection to Google Sheets API, defaults to `20s` | +| `gsheets.read-timeout` | Timeout when reading from Google Sheets API, defaults to `20s` | +| `gsheets.write-timeout` | Timeout when writing to Google Sheets API, defaults to `20s` | + +## Credentials + +The connector requires credentials in order to access the Google Sheets API. + +1. Open the [Google Sheets API](https://console.developers.google.com/apis/library/sheets.googleapis.com) + page and click the *Enable* button. This takes you to the API manager page. +2. Select a project using the drop down menu at the top of the page. + Create a new project, if you do not already have one. +3. Choose *Credentials* in the left panel. +4. Click *Manage service accounts*, then create a service account for the connector. + On the *Create key* step, create and download a key in JSON format. + +The key file needs to be available on the Trino coordinator and workers. +Set the `gsheets.credentials-path` configuration property to point to this file. +The exact name of the file does not matter -- it can be named anything. + +Alternatively, set the `gsheets.credentials-key` configuration property. +It should contain the contents of the JSON file, encoded using base64. + +## Metadata sheet + +The metadata sheet is used to map table names to sheet IDs. +Create a new metadata sheet. The first row must be a header row +containing the following columns in this order: + +- Table Name +- Sheet ID +- Owner (optional) +- Notes (optional) + +See this [example sheet](https://docs.google.com/spreadsheets/d/1Es4HhWALUQjoa-bQh4a8B5HROz7dpGMfq_HbfoaW5LM) +as a reference. + +The metadata sheet must be shared with the service account user, +the one for which the key credentials file was created. Click the *Share* +button to share the sheet with the email address of the service account. + +Set the `gsheets.metadata-sheet-id` configuration property to the ID of this sheet. + +## Querying sheets + +The service account user must have access to the sheet in order for Trino +to query it. Click the *Share* button to share the sheet with the email +address of the service account. + +The sheet needs to be mapped to a Trino table name. Specify a table name +(column A) and the sheet ID (column B) in the metadata sheet. To refer +to a specific range in the sheet, add the range after the sheet ID, separated +with `#`. If a range is not provided, the connector loads only 10,000 rows by default from +the first tab in the sheet. + +The first row of the provided sheet range is used as the header and will determine the column +names of the Trino table. +For more details on sheet range syntax see the [google sheets docs](https://developers.google.com/sheets/api/guides/concepts). + +## Writing to sheets + +The same way sheets can be queried, they can also be written by appending data to existing sheets. +In this case the service account user must also have **Editor** permissions on the sheet. + +After data is written to a table, the table contents are removed from the cache +described in [API usage limits](gsheets-api-usage). If the table is accessed +immediately after the write, querying the Google Sheets API may not reflect the +change yet. In that case the old version of the table is read and cached for the +configured amount of time, and it might take some time for the written changes +to propagate properly. + +Keep in mind that the Google Sheets API has [usage limits](https://developers.google.com/sheets/api/limits), that limit the speed of inserting data. +If you run into timeouts you can increase timeout times to avoid `503: The service is currently unavailable` errors. + +(gsheets-api-usage)= +## API usage limits + +The Google Sheets API has [usage limits](https://developers.google.com/sheets/api/limits), +that may impact the usage of this connector. Increasing the cache duration and/or size +may prevent the limit from being reached. Running queries on the `information_schema.columns` +table without a schema and table name filter may lead to hitting the limit, as this requires +fetching the sheet data for every table, unless it is already cached. + +## Type mapping + +Because Trino and Google Sheets each support types that the other does not, this +connector {ref}`modifies some types ` when reading data. + +### Google Sheets type to Trino type mapping + +The connector maps Google Sheets types to the corresponding Trino types +following this table: + +```{eval-rst} +.. list-table:: Google Sheets type to Trino type mapping + :widths: 30, 20 + :header-rows: 1 + + * - Google Sheets type + - Trino type + * - ``TEXT`` + - ``VARCHAR`` +``` + +No other types are supported. + +(google-sheets-sql-support)= + +## SQL support + +In addition to the {ref}`globally available ` and {ref}`read operation ` statements, +this connector supports the following features: + +- {doc}`/sql/insert` + +## Table functions + +The connector provides specific {doc}`/functions/table` to access Google Sheets. + +(google-sheets-sheet-function)= + +### `sheet(id, range) -> table` + +The `sheet` function allows you to query a Google Sheet directly without +specifying it as a named table in the metadata sheet. + +For example, for a catalog named 'example': + +``` +SELECT * +FROM + TABLE(example.system.sheet( + id => 'googleSheetIdHere')); +``` + +A sheet range or named range can be provided as an optional `range` argument. +The default sheet range is `$1:$10000` if one is not provided: + +``` +SELECT * +FROM + TABLE(example.system.sheet( + id => 'googleSheetIdHere', + range => 'TabName!A1:B4')); +``` diff --git a/430/_sources/connector/hive-alluxio.md.txt b/430/_sources/connector/hive-alluxio.md.txt new file mode 100644 index 000000000..594295178 --- /dev/null +++ b/430/_sources/connector/hive-alluxio.md.txt @@ -0,0 +1,16 @@ +# Hive connector with Alluxio + +The {doc}`hive` can read and write tables stored in the [Alluxio Data Orchestration +System](https://www.alluxio.io/), +leveraging Alluxio's distributed block-level read/write caching functionality. +The tables must be created in the Hive metastore with the `alluxio://` +location prefix (see [Running Apache Hive with Alluxio](https://docs.alluxio.io/os/user/stable/en/compute/Hive.html) +for details and examples). + +Trino queries will then transparently retrieve and cache files or objects from +a variety of disparate storage systems including HDFS and S3. + +## Setting up Alluxio with Trino + +For information on how to setup, configure, and use Alluxio, refer to [Alluxio's +documentation on using their platform with Trino](https://docs.alluxio.io/ee/user/stable/en/compute/Trino.html). diff --git a/430/_sources/connector/hive-azure.md.txt b/430/_sources/connector/hive-azure.md.txt new file mode 100644 index 000000000..048d90fcf --- /dev/null +++ b/430/_sources/connector/hive-azure.md.txt @@ -0,0 +1,250 @@ +# Hive connector with Azure Storage + +The {doc}`hive` can be configured to use [Azure Data Lake Storage (Gen2)](https://azure.microsoft.com/products/storage/data-lake-storage/). Trino +supports Azure Blob File System (ABFS) to access data in ADLS Gen2. + +Trino also supports [ADLS Gen1](https://learn.microsoft.com/azure/data-lake-store/data-lake-store-overview) +and Windows Azure Storage Blob driver (WASB), but we recommend [migrating to +ADLS Gen2](https://learn.microsoft.com/azure/storage/blobs/data-lake-storage-migrate-gen1-to-gen2-azure-portal), +as ADLS Gen1 and WASB are legacy options that will be removed in the future. +Learn more from [the official documentation](https://docs.microsoft.com/azure/data-lake-store/data-lake-store-overview). + +## Hive connector configuration for Azure Storage credentials + +To configure Trino to use the Azure Storage credentials, set the following +configuration properties in the catalog properties file. It is best to use this +type of configuration if the primary storage account is linked to the cluster. + +The specific configuration depends on the type of storage and uses the +properties from the following sections in the catalog properties file. + +For more complex use cases, such as configuring multiple secondary storage +accounts using Hadoop's `core-site.xml`, see the +{ref}`hive-azure-advanced-config` options. + +### ADLS Gen2 / ABFS storage + +To connect to ABFS storage, you may either use the storage account's access +key, or a service principal. Do not use both sets of properties at the +same time. + +```{eval-rst} +.. list-table:: ABFS Access Key + :widths: 30, 70 + :header-rows: 1 + + * - Property name + - Description + * - ``hive.azure.abfs-storage-account`` + - The name of the ADLS Gen2 storage account + * - ``hive.azure.abfs-access-key`` + - The decrypted access key for the ADLS Gen2 storage account +``` + +```{eval-rst} +.. list-table:: ABFS Service Principal OAuth + :widths: 30, 70 + :header-rows: 1 + + * - Property name + - Description + * - ``hive.azure.abfs.oauth.endpoint`` + - The service principal / application's OAuth 2.0 token endpoint (v1). + * - ``hive.azure.abfs.oauth.client-id`` + - The service principal's client/application ID. + * - ``hive.azure.abfs.oauth.secret`` + - A client secret for the service principal. +``` + +When using a service principal, it must have the Storage Blob Data Owner, +Contributor, or Reader role on the storage account you are using, depending on +which operations you would like to use. + +### ADLS Gen1 (legacy) + +While it is advised to migrate to ADLS Gen2 whenever possible, if you still +choose to use ADLS Gen1 you need to include the following properties in your +catalog configuration. + +:::{note} +Credentials for the filesystem can be configured using `ClientCredential` +type. To authenticate with ADLS Gen1 you must create a new application +secret for your ADLS Gen1 account's App Registration, and save this value +because you won't able to retrieve the key later. Refer to the Azure +[documentation](https://docs.microsoft.com/azure/data-lake-store/data-lake-store-service-to-service-authenticate-using-active-directory) +for details. +::: + +```{eval-rst} +.. list-table:: ADLS properties + :widths: 30, 70 + :header-rows: 1 + + * - Property name + - Description + * - ``hive.azure.adl-client-id`` + - Client (Application) ID from the App Registrations for your storage + account + * - ``hive.azure.adl-credential`` + - Value of the new client (application) secret created + * - ``hive.azure.adl-refresh-url`` + - OAuth 2.0 token endpoint url + * - ``hive.azure.adl-proxy-host`` + - Proxy host and port in ``host:port`` format. Use this property to connect + to an ADLS endpoint via a SOCKS proxy. +``` + +### WASB storage (legacy) + +```{eval-rst} +.. list-table:: WASB properties + :widths: 30, 70 + :header-rows: 1 + + * - Property name + - Description + * - ``hive.azure.wasb-storage-account`` + - Storage account name of Azure Blob Storage + * - ``hive.azure.wasb-access-key`` + - The decrypted access key for the Azure Blob Storage +``` + +(hive-azure-advanced-config)= + +### Advanced configuration + +All of the configuration properties for the Azure storage driver are stored in +the Hadoop `core-site.xml` configuration file. When there are secondary +storage accounts involved, we recommend configuring Trino using a +`core-site.xml` containing the appropriate credentials for each account. + +The path to the file must be configured in the catalog properties file: + +```text +hive.config.resources= +``` + +One way to find your account key is to ask for the connection string for the +storage account. The `abfsexample.dfs.core.windows.net` account refers to the +storage account. The connection string contains the account key: + +```text +az storage account show-connection-string --name abfswales1 +{ + "connectionString": "DefaultEndpointsProtocol=https;EndpointSuffix=core.windows.net;AccountName=abfsexample;AccountKey=examplekey..." +} +``` + +When you have the account access key, you can add it to your `core-site.xml` +or Java cryptography extension (JCEKS) file. Alternatively, you can have your +cluster management tool to set the option +`fs.azure.account.key.STORAGE-ACCOUNT` to the account key value: + +```text + + fs.azure.account.key.abfsexample.dfs.core.windows.net + examplekey... + +``` + +For more information, see [Hadoop Azure Support: ABFS](https://hadoop.apache.org/docs/stable/hadoop-azure/abfs.html). + +## Accessing Azure Storage data + +### URI scheme to reference data + +Consistent with other FileSystem implementations within Hadoop, the Azure +Standard Blob and Azure Data Lake Storage Gen2 (ABFS) drivers define their own +URI scheme so that resources (directories and files) may be distinctly +addressed. You can access both primary and secondary storage accounts linked to +the cluster with the same URI scheme. Following are example URIs for the +different systems. + +ABFS URI: + +```text +abfs[s]://@.dfs.core.windows.net/// +``` + +ADLS Gen1 URI: + +```text +adl://.azuredatalakestore.net// +``` + +Azure Standard Blob URI: + +```text +wasb[s]://@.blob.core.windows.net/// +``` + +### Querying Azure Storage + +You can query tables already configured in your Hive metastore used in your Hive +catalog. To access Azure Storage data that is not yet mapped in the Hive +metastore, you need to provide the schema of the data, the file format, and the +data location. + +For example, if you have ORC or Parquet files in an ABFS `file_system`, you +need to execute a query: + +``` +-- select schema in which the table is to be defined, must already exist +USE hive.default; + +-- create table +CREATE TABLE orders ( + orderkey BIGINT, + custkey BIGINT, + orderstatus VARCHAR(1), + totalprice DOUBLE, + orderdate DATE, + orderpriority VARCHAR(15), + clerk VARCHAR(15), + shippriority INTEGER, + comment VARCHAR(79) +) WITH ( + external_location = 'abfs[s]://@.dfs.core.windows.net///', + format = 'ORC' -- or 'PARQUET' +); +``` + +Now you can query the newly mapped table: + +``` +SELECT * FROM orders; +``` + +## Writing data + +### Prerequisites + +Before you attempt to write data to Azure Storage, make sure you have configured +everything necessary to read data from the storage. + +### Create a write schema + +If the Hive metastore contains schema(s) mapped to Azure storage filesystems, +you can use them to write data to Azure storage. + +If you don't want to use existing schemas, or there are no appropriate schemas +in the Hive metastore, you need to create a new one: + +``` +CREATE SCHEMA hive.abfs_export +WITH (location = 'abfs[s]://file_system@account_name.dfs.core.windows.net/'); +``` + +### Write data to Azure Storage + +Once you have a schema pointing to a location where you want to write the data, +you can issue a `CREATE TABLE AS` statement and select your desired file +format. The data will be written to one or more files within the +`abfs[s]://file_system@account_name.dfs.core.windows.net//my_table` +namespace. Example: + +``` +CREATE TABLE hive.abfs_export.orders_abfs +WITH (format = 'ORC') +AS SELECT * FROM tpch.sf1.orders; +``` diff --git a/430/_sources/connector/hive-caching.md.txt b/430/_sources/connector/hive-caching.md.txt new file mode 100644 index 000000000..edc3cf3f5 --- /dev/null +++ b/430/_sources/connector/hive-caching.md.txt @@ -0,0 +1,195 @@ +# Hive connector storage caching + +Querying object storage with the {doc}`/connector/hive` is a +very common use case for Trino. It often involves the transfer of large amounts +of data. The objects are retrieved from HDFS, or any other supported object +storage, by multiple workers and processed on these workers. Repeated queries +with different parameters, or even different queries from different users, often +access, and therefore transfer, the same objects. + +## Benefits + +Enabling caching can result in significant benefits: + +**Reduced load on object storage** + +Every retrieved and cached object avoids repeated retrieval from the storage in +subsequent queries. As a result the object storage system does not have to +provide the object again and again. + +For example, if your query accesses 100MB of objects from the storage, the first +time the query runs 100MB are downloaded and cached. Any following query uses +these objects. If your users run another 100 queries accessing the same objects, +your storage system does not have to do any significant work. Without caching it +has to provide the same objects again and again, resulting in 10GB of total +storage to serve. + +This reduced load on the object storage can also impact the sizing, and +therefore the cost, of the object storage system. + +**Increased query performance** + +Caching can provide significant performance benefits, by avoiding the repeated +network transfers and instead accessing copies of the objects from a local +cache. Performance gains are more significant if the performance of directly +accessing the object storage is low compared to accessing the cache. + +For example, if you access object storage in a different network, different data +center or even different cloud-provider region query performance is slow. Adding +caching using fast, local storage has a significant impact and makes your +queries much faster. + +On the other hand, if your object storage is already running at very high +performance for I/O and network access, and your local cache storage is at +similar speeds, or even slower, performance benefits can be minimal. + +**Reduced query costs** + +A result of the reduced load on the object storage, mentioned earlier, is +significantly reduced network traffic. Network traffic however is a considerable +cost factor in an setup, specifically also when hosted in public cloud provider +systems. + +## Architecture + +Caching can operate in two modes. The async mode provides the queried data +directly and caches any objects asynchronously afterwards. Async is the default +and recommended mode. The query doesn't pay the cost of warming up the cache. +The cache is populated in the background and the query bypasses the cache if the +cache is not already populated. Any following queries requesting the cached +objects are served directly from the cache. + +The other mode is a read-through cache. In this mode, if an object is not found +in the cache, it is read from the storage, placed in the cache, and then provided +to the requesting query. In read-through mode, the query always reads from cache +and must wait for the cache to be populated. + +In both modes, objects are cached on local storage of each worker. Workers can +request cached objects from other workers to avoid requests from the object +storage. + +The cache chunks are 1MB in size and are well suited for ORC or Parquet file +formats. + +## Configuration + +The caching feature is part of the {doc}`/connector/hive` and +can be activated in the catalog properties file: + +```text +connector.name=hive +hive.cache.enabled=true +hive.cache.location=/opt/hive-cache +``` + +The cache operates on the coordinator and all workers accessing the object +storage. The used networking ports for the managing BookKeeper and the data +transfer, by default 8898 and 8899, need to be available. + +To use caching on multiple catalogs, you need to configure different caching +directories and different BookKeeper and data-transfer ports. + +```{eval-rst} +.. list-table:: **Cache Configuration Parameters** + :widths: 25, 63, 12 + :header-rows: 1 + + * - Property + - Description + - Default + * - ``hive.cache.enabled`` + - Toggle to enable or disable caching + - ``false`` + * - ``hive.cache.location`` + - Required directory location to use for the cache storage on each worker. + Separate multiple directories, which can be mount points for separate + drives, with commas. More tips can be found in the :ref:`recommendations + `. Example: + ``hive.cache.location=/var/lib/trino/cache1,/var/lib/trino/cache2`` + - + * - ``hive.cache.data-transfer-port`` + - The TCP/IP port used to transfer data managed by the cache. + - ``8898`` + * - ``hive.cache.bookkeeper-port`` + - The TCP/IP port used by the BookKeeper managing the cache. + - ``8899`` + * - ``hive.cache.read-mode`` + - Operational mode for the cache as described earlier in the architecture + section. ``async`` and ``read-through`` are the supported modes. + - ``async`` + * - ``hive.cache.ttl`` + - Time to live for objects in the cache. Objects, which have not been + requested for the TTL value, are removed from the cache. + - ``7d`` + * - ``hive.cache.disk-usage-percentage`` + - Percentage of disk space used for cached data. + - 80 +``` + +(hive-cache-recommendations)= + +## Recommendations + +The speed of the local cache storage is crucial to the performance of the cache. +The most common and cost efficient approach is to attach high performance SSD +disk or equivalents. Fast cache performance can be also be achieved with a RAM +disk used as in-memory. + +In all cases, you should avoid using the root partition and disk of the node and +instead attach at multiple dedicated storage devices for the cache on each node. +The cache uses the disk up to a configurable percentage. Storage should be local +on each coordinator and worker node. The directory needs to exist before Trino +starts. We recommend using multiple devices to improve performance of the cache. + +The capacity of the attached storage devices should be about 20-30% larger than +the size of the queried object storage workload. For example, your current query +workload typically accesses partitions in your HDFS storage that encapsulate +data for the last 3 months. The overall size of these partitions is currently at +1TB. As a result your cache drives have to have a total capacity of 1.2 TB or +more. + +Your deployment method for Trino decides how to create the directory for +caching. Typically you need to connect a fast storage system, like an SSD drive, +and ensure that is it mounted on the configured path. Kubernetes, CFT and other +systems allow this via volumes. + +## Object storage systems + +The following object storage systems are tested: + +- HDFS +- {doc}`Amazon S3 and S3-compatible systems ` +- {doc}`Azure storage systems ` +- Google Cloud Storage + +## Metrics + +In order to verify how caching works on your system you can take multiple +approaches: + +- Inspect the disk usage on the cache storage drives on all nodes +- Query the metrics of the caching system exposed by JMX + +The implementation of the cache exposes a [number of metrics](https://rubix.readthedocs.io/en/latest/metrics.html) via JMX. You can +{doc}`inspect these and other metrics directly in Trino with the JMX connector +or in external tools `. + +Basic caching statistics for the catalog are available in the +`jmx.current."rubix:catalog=,name=stats"` table. +The table `jmx.current."rubix:catalog=,type=detailed,name=stats` +contains more detailed statistics. + +The following example query returns the average cache hit ratio for the `hive` catalog: + +```sql +SELECT avg(cache_hit) +FROM jmx.current."rubix:catalog=hive,name=stats" +WHERE NOT is_nan(cache_hit); +``` + +## Limitations + +Caching does not support user impersonation and cannot be used with HDFS secured by Kerberos. +It does not take any user-specific access rights to the object storage into account. +The cached objects are simply transparent binary blobs to the caching system and full +access to all content is available. diff --git a/430/_sources/connector/hive-cos.md.txt b/430/_sources/connector/hive-cos.md.txt new file mode 100644 index 000000000..b9b9a83e7 --- /dev/null +++ b/430/_sources/connector/hive-cos.md.txt @@ -0,0 +1,98 @@ +# Hive connector with IBM Cloud Object Storage + +Configure the {doc}`hive` to support [IBM Cloud Object Storage COS](https://www.ibm.com/cloud/object-storage) access. + +## Configuration + +To use COS, you need to configure a catalog file to use the Hive +connector. For example, create a file `etc/ibmcos.properties` and +specify the path to the COS service config file with the +`hive.cos.service-config` property. + +```properties +connector.name=hive +hive.cos.service-config=etc/cos-service.properties +``` + +The service configuration file contains the access and secret keys, as well as +the endpoints for one or multiple COS services: + +```properties +service1.access-key= +service1.secret-key= +service1.endpoint= +service2.access-key= +service2.secret-key= +service2.endpoint= +``` + +The endpoints property is optional. `service1` and `service2` are +placeholders for unique COS service names. The granularity for providing access +credentials is at the COS service level. + +To use IBM COS service, specify the service name, for example: `service1` in +the COS path. The general URI path pattern is +`cos://./object(s)`. + +``` +cos://example-bucket.service1/orders_tiny +``` + +Trino translates the COS path, and uses the `service1` endpoint and +credentials from `cos-service.properties` to access +`cos://example-bucket.service1/object`. + +The Hive Metastore (HMS) does not support the IBM COS filesystem, by default. +The [Stocator library](https://github.com/CODAIT/stocator) is a possible +solution to this problem. Download the [Stocator JAR](https://repo1.maven.org/maven2/com/ibm/stocator/stocator/1.1.4/stocator-1.1.4.jar), +and place it in Hadoop PATH. The [Stocator IBM COS configuration](https://github.com/CODAIT/stocator#reference-stocator-in-the-core-sitexml) +should be placed in `core-site.xml`. For example: + +``` + + fs.stocator.scheme.list + cos + + + fs.cos.impl + com.ibm.stocator.fs.ObjectStoreFileSystem + + + fs.stocator.cos.impl + com.ibm.stocator.fs.cos.COSAPIClient + + + fs.stocator.cos.scheme + cos + + + fs.cos.service1.endpoint + http://s3.eu-de.cloud-object-storage.appdomain.cloud + + + fs.cos.service1.access.key + access-key + + + fs.cos.service1.secret.key + secret-key + +``` + +## Alternative configuration using S3 compatibility + +Use the S3 properties for the Hive connector in the catalog file. If only one +IBM Cloud Object Storage endpoint is used, then the configuration can be +simplified: + +``` +hive.s3.endpoint=http://s3.eu-de.cloud-object-storage.appdomain.cloud +hive.s3.aws-access-key=access-key +hive.s3.aws-secret-key=secret-key +``` + +Use `s3` protocol instead of `cos` for the table location: + +``` +s3://example-bucket/object/ +``` diff --git a/430/_sources/connector/hive-gcs-tutorial.md.txt b/430/_sources/connector/hive-gcs-tutorial.md.txt new file mode 100644 index 000000000..3c5c3a9fa --- /dev/null +++ b/430/_sources/connector/hive-gcs-tutorial.md.txt @@ -0,0 +1,81 @@ +# Google Cloud Storage + +Object storage connectors can access +[Google Cloud Storage](https://cloud.google.com/storage/) data using the +`gs://` URI prefix. + +## Requirements + +To use Google Cloud Storage with non-anonymous access objects, you need: + +- A [Google Cloud service account](https://console.cloud.google.com/projectselector2/iam-admin/serviceaccounts) +- The key for the service account in JSON format + +(hive-google-cloud-storage-configuration)= + +## Configuration + +The use of Google Cloud Storage as a storage location for an object storage +catalog requires setting a configuration property that defines the +[authentication method for any non-anonymous access object](https://cloud.google.com/storage/docs/authentication). Access methods cannot +be combined. + +The default root path used by the `gs:\\` prefix is set in the catalog by the +contents of the specified key file, or the key file used to create the OAuth +token. + +```{eval-rst} +.. list-table:: Google Cloud Storage configuration properties + :widths: 35, 65 + :header-rows: 1 + + * - Property Name + - Description + * - ``hive.gcs.json-key-file-path`` + - JSON key file used to authenticate your Google Cloud service account + with Google Cloud Storage. + * - ``hive.gcs.use-access-token`` + - Use client-provided OAuth token to access Google Cloud Storage. +``` + +The following uses the Delta Lake connector in an example of a minimal +configuration file for an object storage catalog using a JSON key file: + +```properties +connector.name=delta_lake +hive.metastore.uri=thrift://example.net:9083 +hive.gcs.json-key-file-path=${ENV:GCP_CREDENTIALS_FILE_PATH} +``` + +## General usage + +Create a schema to use if one does not already exist, as in the following +example: + +```sql +CREATE SCHEMA storage_catalog.sales_data_in_gcs WITH (location = 'gs://example_location'); +``` + +Once you have created a schema, you can create tables in the schema, as in the +following example: + +```sql +CREATE TABLE storage_catalog.sales_data_in_gcs.orders ( + orderkey BIGINT, + custkey BIGINT, + orderstatus VARCHAR(1), + totalprice DOUBLE, + orderdate DATE, + orderpriority VARCHAR(15), + clerk VARCHAR(15), + shippriority INTEGER, + comment VARCHAR(79) +); +``` + +This statement creates the folder `gs://sales_data_in_gcs/orders` in the root +folder defined in the JSON key file. + +Your table is now ready to populate with data using `INSERT` statements. +Alternatively, you can use `CREATE TABLE AS` statements to create and +populate the table in a single statement. diff --git a/430/_sources/connector/hive-s3.md.txt b/430/_sources/connector/hive-s3.md.txt new file mode 100644 index 000000000..8cfdfb450 --- /dev/null +++ b/430/_sources/connector/hive-s3.md.txt @@ -0,0 +1,314 @@ +# Hive connector with Amazon S3 + +The {doc}`hive` can read and write tables that are stored in +[Amazon S3](https://aws.amazon.com/s3/) or S3-compatible systems. +This is accomplished by having a table or database location that +uses an S3 prefix, rather than an HDFS prefix. + +Trino uses its own S3 filesystem for the URI prefixes +`s3://`, `s3n://` and `s3a://`. + +(hive-s3-configuration)= + +## S3 configuration properties + +```{eval-rst} +.. list-table:: + :widths: 35, 65 + :header-rows: 1 + + * - Property name + - Description + * - ``hive.s3.aws-access-key`` + - Default AWS access key to use. + * - ``hive.s3.aws-secret-key`` + - Default AWS secret key to use. + * - ``hive.s3.iam-role`` + - IAM role to assume. + * - ``hive.s3.external-id`` + - External ID for the IAM role trust policy. + * - ``hive.s3.endpoint`` + - The S3 storage endpoint server. This can be used to connect to an + S3-compatible storage system instead of AWS. When using v4 signatures, + it is recommended to set this to the AWS region-specific endpoint + (e.g., ``http[s]://s3..amazonaws.com``). + * - ``hive.s3.region`` + - Optional property to force the S3 client to connect to the specified + region only. + * - ``hive.s3.storage-class`` + - The S3 storage class to use when writing the data. Currently only + ``STANDARD`` and ``INTELLIGENT_TIERING`` storage classes are supported. + Default storage class is ``STANDARD`` + * - ``hive.s3.signer-type`` + - Specify a different signer type for S3-compatible storage. + Example: ``S3SignerType`` for v2 signer type + * - ``hive.s3.signer-class`` + - Specify a different signer class for S3-compatible storage. + * - ``hive.s3.path-style-access`` + - Use path-style access for all requests to the S3-compatible storage. + This is for S3-compatible storage that doesn't support + virtual-hosted-style access, defaults to ``false``. + * - ``hive.s3.staging-directory`` + - Local staging directory for data written to S3. This defaults to the + Java temporary directory specified by the JVM system property + ``java.io.tmpdir``. + * - ``hive.s3.pin-client-to-current-region`` + - Pin S3 requests to the same region as the EC2 instance where Trino is + running, defaults to ``false``. + * - ``hive.s3.ssl.enabled`` + - Use HTTPS to communicate with the S3 API, defaults to ``true``. + * - ``hive.s3.sse.enabled`` + - Use S3 server-side encryption, defaults to ``false``. + * - ``hive.s3.sse.type`` + - The type of key management for S3 server-side encryption. Use ``S3`` + for S3 managed or ``KMS`` for KMS-managed keys, defaults to ``S3``. + * - ``hive.s3.sse.kms-key-id`` + - The KMS Key ID to use for S3 server-side encryption with KMS-managed + keys. If not set, the default key is used. + * - ``hive.s3.kms-key-id`` + - If set, use S3 client-side encryption and use the AWS KMS to store + encryption keys and use the value of this property as the KMS Key ID for + newly created objects. + * - ``hive.s3.encryption-materials-provider`` + - If set, use S3 client-side encryption and use the value of this property + as the fully qualified name of a Java class which implements the AWS + SDK's ``EncryptionMaterialsProvider`` interface. If the class also + implements ``Configurable`` from the Hadoop API, the Hadoop + configuration will be passed in after the object has been created. + * - ``hive.s3.upload-acl-type`` + - Canned ACL to use while uploading files to S3, defaults to ``PRIVATE``. + If the files are to be uploaded to an S3 bucket owned by a different AWS + user, the canned ACL has to be set to one of the following: + ``AUTHENTICATED_READ``, ``AWS_EXEC_READ``, ``BUCKET_OWNER_FULL_CONTROL``, + ``BUCKET_OWNER_READ``, ``LOG_DELIVERY_WRITE``, ``PUBLIC_READ``, + ``PUBLIC_READ_WRITE``. Refer to the `AWS canned ACL `_ + guide to understand each option's definition. + * - ``hive.s3.skip-glacier-objects`` + - Ignore Glacier objects rather than failing the query. This skips data + that may be expected to be part of the table or partition. Defaults to + ``false``. + * - ``hive.s3.streaming.enabled`` + - Use S3 multipart upload API to upload file in streaming way, without + staging file to be created in the local file system. + * - ``hive.s3.streaming.part-size`` + - The part size for S3 streaming upload. Defaults to ``16MB``. + * - ``hive.s3.proxy.host`` + - Proxy host to use if connecting through a proxy. + * - ``hive.s3.proxy.port`` + - Proxy port to use if connecting through a proxy. + * - ``hive.s3.proxy.protocol`` + - Proxy protocol. HTTP or HTTPS , defaults to ``HTTPS``. + * - ``hive.s3.proxy.non-proxy-hosts`` + - Hosts list to access without going through the proxy. + * - ``hive.s3.proxy.username`` + - Proxy user name to use if connecting through a proxy. + * - ``hive.s3.proxy.password`` + - Proxy password to use if connecting through a proxy. + * - ``hive.s3.proxy.preemptive-basic-auth`` + - Whether to attempt to authenticate preemptively against proxy when using + base authorization, defaults to ``false``. + * - ``hive.s3.sts.endpoint`` + - Optional override for the sts endpoint given that IAM role based + authentication via sts is used. + * - ``hive.s3.sts.region`` + - Optional override for the sts region given that IAM role based + authentication via sts is used. +``` + +(hive-s3-credentials)= + +## S3 credentials + +If you are running Trino on Amazon EC2, using EMR or another facility, +it is recommended that you use IAM Roles for EC2 to govern access to S3. +To enable this, your EC2 instances need to be assigned an IAM Role which +grants appropriate access to the data stored in the S3 bucket(s) you wish +to use. It is also possible to configure an IAM role with `hive.s3.iam-role` +that is used for accessing any S3 bucket. This is much cleaner than +setting AWS access and secret keys in the `hive.s3.aws-access-key` +and `hive.s3.aws-secret-key` settings, and also allows EC2 to automatically +rotate credentials on a regular basis without any additional work on your part. + +## Custom S3 credentials provider + +You can configure a custom S3 credentials provider by setting the configuration +property `trino.s3.credentials-provider` to the fully qualified class name of +a custom AWS credentials provider implementation. The property must be set in +the Hadoop configuration files referenced by the `hive.config.resources` Hive +connector property. + +The class must implement the +[AWSCredentialsProvider](http://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/auth/AWSCredentialsProvider.html) +interface and provide a two-argument constructor that takes a +`java.net.URI` and a Hadoop `org.apache.hadoop.conf.Configuration` +as arguments. A custom credentials provider can be used to provide +temporary credentials from STS (using `STSSessionCredentialsProvider`), +IAM role-based credentials (using `STSAssumeRoleSessionCredentialsProvider`), +or credentials for a specific use case (e.g., bucket/user specific credentials). + +(hive-s3-security-mapping)= + +## S3 security mapping + +Trino supports flexible security mapping for S3, allowing for separate +credentials or IAM roles for specific users or buckets/paths. The IAM role +for a specific query can be selected from a list of allowed roles by providing +it as an *extra credential*. + +Each security mapping entry may specify one or more match criteria. If multiple +criteria are specified, all criteria must match. Available match criteria: + +- `user`: Regular expression to match against username. Example: `alice|bob` +- `group`: Regular expression to match against any of the groups that the user + belongs to. Example: `finance|sales` +- `prefix`: S3 URL prefix. It can specify an entire bucket or a path within a + bucket. The URL must start with `s3://` but will also match `s3a` or `s3n`. + Example: `s3://bucket-name/abc/xyz/` + +The security mapping must provide one or more configuration settings: + +- `accessKey` and `secretKey`: AWS access key and secret key. This overrides + any globally configured credentials, such as access key or instance credentials. +- `iamRole`: IAM role to use if no user provided role is specified as an + extra credential. This overrides any globally configured IAM role. This role + is allowed to be specified as an extra credential, although specifying it + explicitly has no effect, as it would be used anyway. +- `roleSessionName`: Optional role session name to use with `iamRole`. This can only + be used when `iamRole` is specified. If `roleSessionName` includes the string + `${USER}`, then the `${USER}` portion of the string will be replaced with the + current session's username. If `roleSessionName` is not specified, it defaults + to `trino-session`. +- `allowedIamRoles`: IAM roles that are allowed to be specified as an extra + credential. This is useful because a particular AWS account may have permissions + to use many roles, but a specific user should only be allowed to use a subset + of those roles. +- `kmsKeyId`: ID of KMS-managed key to be used for client-side encryption. +- `allowedKmsKeyIds`: KMS-managed key IDs that are allowed to be specified as an extra + credential. If list cotains "\*", then any key can be specified via extra credential. + +The security mapping entries are processed in the order listed in the configuration +JSON. More specific mappings should thus be specified before less specific mappings. +For example, the mapping list might have URL prefix `s3://abc/xyz/` followed by +`s3://abc/` to allow different configuration for a specific path within a bucket +than for other paths within the bucket. You can set default configuration by not +including any match criteria for the last entry in the list. + +In addition to the rules above, the default mapping can contain the optional +`useClusterDefault` boolean property with the following behavior: + +- `false` - (is set by default) property is ignored. + +- `true` - This causes the default cluster role to be used as a fallback option. + It can not be used with the following configuration properties: + + - `accessKey` + - `secretKey` + - `iamRole` + - `allowedIamRoles` + +If no mapping entry matches and no default is configured, the access is denied. + +The configuration JSON can either be retrieved from a file or REST-endpoint specified via +`hive.s3.security-mapping.config-file`. + +Example JSON configuration: + +```json +{ + "mappings": [ + { + "prefix": "s3://bucket-name/abc/", + "iamRole": "arn:aws:iam::123456789101:role/test_path" + }, + { + "user": "bob|charlie", + "iamRole": "arn:aws:iam::123456789101:role/test_default", + "allowedIamRoles": [ + "arn:aws:iam::123456789101:role/test1", + "arn:aws:iam::123456789101:role/test2", + "arn:aws:iam::123456789101:role/test3" + ] + }, + { + "prefix": "s3://special-bucket/", + "accessKey": "AKIAxxxaccess", + "secretKey": "iXbXxxxsecret" + }, + { + "prefix": "s3://encrypted-bucket/", + "kmsKeyId": "kmsKey_10", + }, + { + "user": "test.*", + "iamRole": "arn:aws:iam::123456789101:role/test_users" + }, + { + "group": "finance", + "iamRole": "arn:aws:iam::123456789101:role/finance_users" + }, + { + "iamRole": "arn:aws:iam::123456789101:role/default" + } + ] +} +``` + +| Property name | Description | +| ----------------------------------------------------- | -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| `hive.s3.security-mapping.config-file` | The JSON configuration file or REST-endpoint URI containing security mappings. | +| `hive.s3.security-mapping.json-pointer` | A JSON pointer (RFC 6901) to mappings inside the JSON retrieved from the config file or REST-endpont. The whole document ("") by default. | +| `hive.s3.security-mapping.iam-role-credential-name` | The name of the *extra credential* used to provide the IAM role. | +| `hive.s3.security-mapping.kms-key-id-credential-name` | The name of the *extra credential* used to provide the KMS-managed key ID. | +| `hive.s3.security-mapping.refresh-period` | How often to refresh the security mapping configuration. | +| `hive.s3.security-mapping.colon-replacement` | The character or characters to be used in place of the colon (`:`) character when specifying an IAM role name as an extra credential. Any instances of this replacement value in the extra credential value will be converted to a colon. Choose a value that is not used in any of your IAM ARNs. | + +(hive-s3-tuning-configuration)= + +## Tuning properties + +The following tuning properties affect the behavior of the client +used by the Trino S3 filesystem when communicating with S3. +Most of these parameters affect settings on the `ClientConfiguration` +object associated with the `AmazonS3Client`. + +| Property name | Description | Default | +| --------------------------------- | ------------------------------------------------------------------------------------------------- | -------------------------- | +| `hive.s3.max-error-retries` | Maximum number of error retries, set on the S3 client. | `10` | +| `hive.s3.max-client-retries` | Maximum number of read attempts to retry. | `5` | +| `hive.s3.max-backoff-time` | Use exponential backoff starting at 1 second up to this maximum value when communicating with S3. | `10 minutes` | +| `hive.s3.max-retry-time` | Maximum time to retry communicating with S3. | `10 minutes` | +| `hive.s3.connect-timeout` | TCP connect timeout. | `5 seconds` | +| `hive.s3.connect-ttl` | TCP connect TTL, which affects connection reusage. | Connections do not expire. | +| `hive.s3.socket-timeout` | TCP socket read timeout. | `5 seconds` | +| `hive.s3.max-connections` | Maximum number of simultaneous open connections to S3. | `500` | +| `hive.s3.multipart.min-file-size` | Minimum file size before multi-part upload to S3 is used. | `16 MB` | +| `hive.s3.multipart.min-part-size` | Minimum multi-part upload part size. | `5 MB` | + +(hive-s3-data-encryption)= + +## S3 data encryption + +Trino supports reading and writing encrypted data in S3 using both +server-side encryption with S3 managed keys and client-side encryption using +either the Amazon KMS or a software plugin to manage AES encryption keys. + +With [S3 server-side encryption](http://docs.aws.amazon.com/AmazonS3/latest/dev/serv-side-encryption.html), +called *SSE-S3* in the Amazon documentation, the S3 infrastructure takes care of all encryption and decryption +work. One exception is SSL to the client, assuming you have `hive.s3.ssl.enabled` set to `true`. +S3 also manages all the encryption keys for you. To enable this, set `hive.s3.sse.enabled` to `true`. + +With [S3 client-side encryption](http://docs.aws.amazon.com/AmazonS3/latest/dev/UsingClientSideEncryption.html), +S3 stores encrypted data and the encryption keys are managed outside of the S3 infrastructure. Data is encrypted +and decrypted by Trino instead of in the S3 infrastructure. In this case, encryption keys can be managed +either by using the AWS KMS, or your own key management system. To use the AWS KMS for key management, set +`hive.s3.kms-key-id` to the UUID of a KMS key. Your AWS credentials or EC2 IAM role will need to be +granted permission to use the given key as well. + +To use a custom encryption key management system, set `hive.s3.encryption-materials-provider` to the +fully qualified name of a class which implements the +[EncryptionMaterialsProvider](http://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/services/s3/model/EncryptionMaterialsProvider.html) +interface from the AWS Java SDK. This class has to be accessible to the Hive Connector through the +classpath and must be able to communicate with your custom key management system. If this class also implements +the `org.apache.hadoop.conf.Configurable` interface from the Hadoop Java API, then the Hadoop configuration +is passed in after the object instance is created, and before it is asked to provision or retrieve any +encryption keys. diff --git a/430/_sources/connector/hive-security.md.txt b/430/_sources/connector/hive-security.md.txt new file mode 100644 index 000000000..9ebdd748b --- /dev/null +++ b/430/_sources/connector/hive-security.md.txt @@ -0,0 +1,407 @@ +# Hive connector security configuration + +(hive-security-impersonation)= + +## Overview + +The Hive connector supports both authentication and authorization. + +Trino can impersonate the end user who is running a query. In the case of a +user running a query from the command line interface, the end user is the +username associated with the Trino CLI process or argument to the optional +`--user` option. + +Authentication can be configured with or without user impersonation on +Kerberized Hadoop clusters. + +## Requirements + +End user authentication limited to Kerberized Hadoop clusters. Authentication +user impersonation is available for both Kerberized and non-Kerberized clusters. + +You must ensure that you meet the Kerberos, user impersonation and keytab +requirements described in this section that apply to your configuration. + +(hive-security-kerberos-support)= + +### Kerberos + +In order to use the Hive connector with a Hadoop cluster that uses `kerberos` +authentication, you must configure the connector to work with two services on +the Hadoop cluster: + +- The Hive metastore Thrift service +- The Hadoop Distributed File System (HDFS) + +Access to these services by the Hive connector is configured in the properties +file that contains the general Hive connector configuration. + +Kerberos authentication by ticket cache is not yet supported. + +:::{note} +If your `krb5.conf` location is different from `/etc/krb5.conf` you +must set it explicitly using the `java.security.krb5.conf` JVM property +in `jvm.config` file. + +Example: `-Djava.security.krb5.conf=/example/path/krb5.conf`. +::: + +:::{warning} +Access to the Trino coordinator must be secured e.g., using Kerberos or +password authentication, when using Kerberos authentication to Hadoop services. +Failure to secure access to the Trino coordinator could result in unauthorized +access to sensitive data on the Hadoop cluster. Refer to {doc}`/security` for +further information. + +See {doc}`/security/kerberos` for information on setting up Kerberos authentication. +::: + +(hive-security-additional-keytab)= + +#### Keytab files + +Keytab files contain encryption keys that are used to authenticate principals +to the Kerberos {abbr}`KDC (Key Distribution Center)`. These encryption keys +must be stored securely; you must take the same precautions to protect them +that you take to protect ssh private keys. + +In particular, access to keytab files must be limited to only the accounts +that must use them to authenticate. In practice, this is the user that +the Trino process runs as. The ownership and permissions on keytab files +must be set to prevent other users from reading or modifying the files. + +Keytab files must be distributed to every node running Trino. Under common +deployment situations, the Hive connector configuration is the same on all +nodes. This means that the keytab needs to be in the same location on every +node. + +You must ensure that the keytab files have the correct permissions on every +node after distributing them. + +(configuring-hadoop-impersonation)= + +### Impersonation in Hadoop + +In order to use impersonation, the Hadoop cluster must be +configured to allow the user or principal that Trino is running as to +impersonate the users who log in to Trino. Impersonation in Hadoop is +configured in the file {file}`core-site.xml`. A complete description of the +configuration options can be found in the [Hadoop documentation](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/Superusers.html#Configurations). + +## Authentication + +The default security configuration of the {doc}`/connector/hive` does not use +authentication when connecting to a Hadoop cluster. All queries are executed as +the user who runs the Trino process, regardless of which user submits the +query. + +The Hive connector provides additional security options to support Hadoop +clusters that have been configured to use {ref}`Kerberos +`. + +When accessing {abbr}`HDFS (Hadoop Distributed File System)`, Trino can +{ref}`impersonate` the end user who is running the +query. This can be used with HDFS permissions and {abbr}`ACLs (Access Control +Lists)` to provide additional security for data. + +### Hive metastore Thrift service authentication + +In a Kerberized Hadoop cluster, Trino connects to the Hive metastore Thrift +service using {abbr}`SASL (Simple Authentication and Security Layer)` and +authenticates using Kerberos. Kerberos authentication for the metastore is +configured in the connector's properties file using the following optional +properties: + +```{eval-rst} +.. list-table:: Hive metastore Thrift service authentication properties + :widths: 30, 55, 15 + :header-rows: 1 + + * - Property value + - Description + - Default + * - ``hive.metastore.authentication.type`` + - Hive metastore authentication type. One of ``NONE`` or ``KERBEROS``. When + using the default value of ``NONE``, Kerberos authentication is disabled, + and no other properties must be configured. + + When set to ``KERBEROS`` the Hive connector connects to the Hive metastore + Thrift service using SASL and authenticate using Kerberos. + - ``NONE`` + * - ``hive.metastore.thrift.impersonation.enabled`` + - Enable Hive metastore end user impersonation. See + :ref:`hive-security-metastore-impersonation` for more information. + - ``false`` + * - ``hive.metastore.service.principal`` + - The Kerberos principal of the Hive metastore service. The coordinator + uses this to authenticate the Hive metastore. + + The ``_HOST`` placeholder can be used in this property value. When + connecting to the Hive metastore, the Hive connector substitutes in the + hostname of the **metastore** server it is connecting to. This is useful + if the metastore runs on multiple hosts. + + Example: ``hive/hive-server-host@EXAMPLE.COM`` or + ``hive/_HOST@EXAMPLE.COM``. + - + * - ``hive.metastore.client.principal`` + - The Kerberos principal that Trino uses when connecting to the Hive + metastore service. + + Example: ``trino/trino-server-node@EXAMPLE.COM`` or + ``trino/_HOST@EXAMPLE.COM``. + + The ``_HOST`` placeholder can be used in this property value. When + connecting to the Hive metastore, the Hive connector substitutes in the + hostname of the **worker** node Trino is running on. This is useful if + each worker node has its own Kerberos principal. + + Unless :ref:`hive-security-metastore-impersonation` is enabled, + the principal specified by ``hive.metastore.client.principal`` must have + sufficient privileges to remove files and directories within the + ``hive/warehouse`` directory. + + **Warning:** If the principal does have sufficient permissions, only the + metadata is removed, and the data continues to consume disk space. This + occurs because the Hive metastore is responsible for deleting the + internal table data. When the metastore is configured to use Kerberos + authentication, all of the HDFS operations performed by the metastore are + impersonated. Errors deleting data are silently ignored. + - + * - ``hive.metastore.client.keytab`` + - The path to the keytab file that contains a key for the principal + specified by ``hive.metastore.client.principal``. This file must be + readable by the operating system user running Trino. + - +``` + +#### Configuration examples + +The following sections describe the configuration properties and values needed +for the various authentication configurations needed to use the Hive metastore +Thrift service with the Hive connector. + +##### Default `NONE` authentication without impersonation + +```text +hive.metastore.authentication.type=NONE +``` + +The default authentication type for the Hive metastore is `NONE`. When the +authentication type is `NONE`, Trino connects to an unsecured Hive +metastore. Kerberos is not used. + +(hive-security-metastore-impersonation)= + +##### `KERBEROS` authentication with impersonation + +```text +hive.metastore.authentication.type=KERBEROS +hive.metastore.thrift.impersonation.enabled=true +hive.metastore.service.principal=hive/hive-metastore-host.example.com@EXAMPLE.COM +hive.metastore.client.principal=trino@EXAMPLE.COM +hive.metastore.client.keytab=/etc/trino/hive.keytab +``` + +When the authentication type for the Hive metastore Thrift service is +`KERBEROS`, Trino connects as the Kerberos principal specified by the +property `hive.metastore.client.principal`. Trino authenticates this +principal using the keytab specified by the `hive.metastore.client.keytab` +property, and verifies that the identity of the metastore matches +`hive.metastore.service.principal`. + +When using `KERBEROS` Metastore authentication with impersonation, the +principal specified by the `hive.metastore.client.principal` property must be +allowed to impersonate the current Trino user, as discussed in the section +{ref}`configuring-hadoop-impersonation`. + +Keytab files must be distributed to every node in the cluster that runs Trino. + +{ref}`Additional Information About Keytab Files.` + +### HDFS authentication + +In a Kerberized Hadoop cluster, Trino authenticates to HDFS using Kerberos. +Kerberos authentication for HDFS is configured in the connector's properties +file using the following optional properties: + +```{eval-rst} +.. list-table:: HDFS authentication properties + :widths: 30, 55, 15 + :header-rows: 1 + + * - Property value + - Description + - Default + * - ``hive.hdfs.authentication.type`` + - HDFS authentication type; one of ``NONE`` or ``KERBEROS``. When using the + default value of ``NONE``, Kerberos authentication is disabled, and no + other properties must be configured. + + When set to ``KERBEROS``, the Hive connector authenticates to HDFS using + Kerberos. + - ``NONE`` + * - ``hive.hdfs.impersonation.enabled`` + - Enable HDFS end-user impersonation. Impersonating the end user can provide + additional security when accessing HDFS if HDFS permissions or ACLs are + used. + + HDFS Permissions and ACLs are explained in the `HDFS Permissions Guide + `_. + - ``false`` + * - ``hive.hdfs.trino.principal`` + - The Kerberos principal Trino uses when connecting to HDFS. + + Example: ``trino-hdfs-superuser/trino-server-node@EXAMPLE.COM`` or + ``trino-hdfs-superuser/_HOST@EXAMPLE.COM``. + + The ``_HOST`` placeholder can be used in this property value. When + connecting to HDFS, the Hive connector substitutes in the hostname of the + **worker** node Trino is running on. This is useful if each worker node + has its own Kerberos principal. + - + * - ``hive.hdfs.trino.keytab`` + - The path to the keytab file that contains a key for the principal + specified by ``hive.hdfs.trino.principal``. This file must be readable by + the operating system user running Trino. + - + * - ``hive.hdfs.wire-encryption.enabled`` + - Enable HDFS wire encryption. In a Kerberized Hadoop cluster that uses HDFS + wire encryption, this must be set to ``true`` to enable Trino to access + HDFS. Note that using wire encryption may impact query execution + performance. + - +``` + +#### Configuration examples + +The following sections describe the configuration properties and values needed +for the various authentication configurations with HDFS and the Hive connector. + +(hive-security-simple)= + +##### Default `NONE` authentication without impersonation + +```text +hive.hdfs.authentication.type=NONE +``` + +The default authentication type for HDFS is `NONE`. When the authentication +type is `NONE`, Trino connects to HDFS using Hadoop's simple authentication +mechanism. Kerberos is not used. + +(hive-security-simple-impersonation)= + +##### `NONE` authentication with impersonation + +```text +hive.hdfs.authentication.type=NONE +hive.hdfs.impersonation.enabled=true +``` + +When using `NONE` authentication with impersonation, Trino impersonates +the user who is running the query when accessing HDFS. The user Trino is +running as must be allowed to impersonate this user, as discussed in the +section {ref}`configuring-hadoop-impersonation`. Kerberos is not used. + +(hive-security-kerberos)= + +##### `KERBEROS` authentication without impersonation + +```text +hive.hdfs.authentication.type=KERBEROS +hive.hdfs.trino.principal=hdfs@EXAMPLE.COM +hive.hdfs.trino.keytab=/etc/trino/hdfs.keytab +``` + +When the authentication type is `KERBEROS`, Trino accesses HDFS as the +principal specified by the `hive.hdfs.trino.principal` property. Trino +authenticates this principal using the keytab specified by the +`hive.hdfs.trino.keytab` keytab. + +Keytab files must be distributed to every node in the cluster that runs Trino. + +{ref}`Additional Information About Keytab Files.` + +(hive-security-kerberos-impersonation)= + +##### `KERBEROS` authentication with impersonation + +```text +hive.hdfs.authentication.type=KERBEROS +hive.hdfs.impersonation.enabled=true +hive.hdfs.trino.principal=trino@EXAMPLE.COM +hive.hdfs.trino.keytab=/etc/trino/hdfs.keytab +``` + +When using `KERBEROS` authentication with impersonation, Trino impersonates +the user who is running the query when accessing HDFS. The principal +specified by the `hive.hdfs.trino.principal` property must be allowed to +impersonate the current Trino user, as discussed in the section +{ref}`configuring-hadoop-impersonation`. Trino authenticates +`hive.hdfs.trino.principal` using the keytab specified by +`hive.hdfs.trino.keytab`. + +Keytab files must be distributed to every node in the cluster that runs Trino. + +{ref}`Additional Information About Keytab Files.` + +## Authorization + +You can enable authorization checks for the {doc}`hive` by setting +the `hive.security` property in the Hive catalog properties file. This +property must be one of the following values: + +```{eval-rst} +.. list-table:: ``hive.security`` property values + :widths: 30, 60 + :header-rows: 1 + + * - Property value + - Description + * - ``legacy`` (default value) + - Few authorization checks are enforced, thus allowing most operations. The + config properties ``hive.allow-drop-table``, ``hive.allow-rename-table``, + ``hive.allow-add-column``, ``hive.allow-drop-column`` and + ``hive.allow-rename-column`` are used. + * - ``read-only`` + - Operations that read data or metadata, such as ``SELECT``, are permitted, + but none of the operations that write data or metadata, such as + ``CREATE``, ``INSERT`` or ``DELETE``, are allowed. + * - ``file`` + - Authorization checks are enforced using a catalog-level access control + configuration file whose path is specified in the ``security.config-file`` + catalog configuration property. See + :ref:`catalog-file-based-access-control` for details. + * - ``sql-standard`` + - Users are permitted to perform the operations as long as they have the + required privileges as per the SQL standard. In this mode, Trino enforces + the authorization checks for queries based on the privileges defined in + Hive metastore. To alter these privileges, use the :doc:`/sql/grant` and + :doc:`/sql/revoke` commands. + + See the :ref:`hive-sql-standard-based-authorization` section for details. + * - ``allow-all`` + - No authorization checks are enforced. +``` + +(hive-sql-standard-based-authorization)= + +### SQL standard based authorization + +When `sql-standard` security is enabled, Trino enforces the same SQL +standard-based authorization as Hive does. + +Since Trino's `ROLE` syntax support matches the SQL standard, and +Hive does not exactly follow the SQL standard, there are the following +limitations and differences: + +- `CREATE ROLE role WITH ADMIN` is not supported. +- The `admin` role must be enabled to execute `CREATE ROLE`, `DROP ROLE` or `CREATE SCHEMA`. +- `GRANT role TO user GRANTED BY someone` is not supported. +- `REVOKE role FROM user GRANTED BY someone` is not supported. +- By default, all a user's roles, except `admin`, are enabled in a new user session. +- One particular role can be selected by executing `SET ROLE role`. +- `SET ROLE ALL` enables all of a user's roles except `admin`. +- The `admin` role must be enabled explicitly by executing `SET ROLE admin`. +- `GRANT privilege ON SCHEMA schema` is not supported. Schema ownership can be changed with `ALTER SCHEMA schema SET AUTHORIZATION user` diff --git a/430/_sources/connector/hive.md.txt b/430/_sources/connector/hive.md.txt new file mode 100644 index 000000000..ceb15abaf --- /dev/null +++ b/430/_sources/connector/hive.md.txt @@ -0,0 +1,1332 @@ +# Hive connector + +```{raw} html + +``` + +```{toctree} +:hidden: true +:maxdepth: 1 + +Metastores +Security +Amazon S3 +Azure Storage +Google Cloud Storage +IBM Cloud Object Storage +Storage Caching +Alluxio +Object storage file formats +``` + +The Hive connector allows querying data stored in an +[Apache Hive](https://hive.apache.org/) +data warehouse. Hive is a combination of three components: + +- Data files in varying formats, that are typically stored in the + Hadoop Distributed File System (HDFS) or in object storage systems + such as Amazon S3. +- Metadata about how the data files are mapped to schemas and tables. This + metadata is stored in a database, such as MySQL, and is accessed via the Hive + metastore service. +- A query language called HiveQL. This query language is executed on a + distributed computing framework such as MapReduce or Tez. + +Trino only uses the first two components: the data and the metadata. +It does not use HiveQL or any part of Hive's execution environment. + +## Requirements + +The Hive connector requires a +{ref}`Hive metastore service ` (HMS), or a compatible +implementation of the Hive metastore, such as +{ref}`AWS Glue `. + +Apache Hadoop HDFS 2.x and 3.x are supported. + +Many distributed storage systems including HDFS, +{doc}`Amazon S3 ` or S3-compatible systems, +[Google Cloud Storage](hive-gcs-tutorial), +{doc}`Azure Storage `, and +{doc}`IBM Cloud Object Storage` can be queried with the Hive +connector. + +The coordinator and all workers must have network access to the Hive metastore +and the storage system. Hive metastore access with the Thrift protocol defaults +to using port 9083. + +Data files must be in a supported file format. Some file formats can be +configured using file format configuration properties per catalog: + +- {ref}`ORC ` +- {ref}`Parquet ` +- Avro +- RCText (RCFile using ColumnarSerDe) +- RCBinary (RCFile using LazyBinaryColumnarSerDe) +- SequenceFile +- JSON (using org.apache.hive.hcatalog.data.JsonSerDe) +- CSV (using org.apache.hadoop.hive.serde2.OpenCSVSerde) +- TextFile + +## General configuration + +To configure the Hive connector, create a catalog properties file +`etc/catalog/example.properties` that references the `hive` +connector and defines a metastore. You must configure a metastore for table +metadata. If you are using a {ref}`Hive metastore `, +`hive.metastore.uri` must be configured: + +```properties +connector.name=hive +hive.metastore.uri=thrift://example.net:9083 +``` + +If you are using {ref}`AWS Glue ` as your metastore, you +must instead set `hive.metastore` to `glue`: + +```properties +connector.name=hive +hive.metastore=glue +``` + +Each metastore type has specific configuration properties along with +{ref}`general metastore configuration properties `. + +### Multiple Hive clusters + +You can have as many catalogs as you need, so if you have additional +Hive clusters, simply add another properties file to `etc/catalog` +with a different name, making sure it ends in `.properties`. For +example, if you name the property file `sales.properties`, Trino +creates a catalog named `sales` using the configured connector. + +### HDFS configuration + +For basic setups, Trino configures the HDFS client automatically and +does not require any configuration files. In some cases, such as when using +federated HDFS or NameNode high availability, it is necessary to specify +additional HDFS client options in order to access your HDFS cluster. To do so, +add the `hive.config.resources` property to reference your HDFS config files: + +```text +hive.config.resources=/etc/hadoop/conf/core-site.xml,/etc/hadoop/conf/hdfs-site.xml +``` + +Only specify additional configuration files if necessary for your setup. +We recommend reducing the configuration files to have the minimum +set of required properties, as additional properties may cause problems. + +The configuration files must exist on all Trino nodes. If you are +referencing existing Hadoop config files, make sure to copy them to +any Trino nodes that are not running Hadoop. + +### HDFS username and permissions + +Before running any `CREATE TABLE` or `CREATE TABLE AS` statements +for Hive tables in Trino, you must check that the user Trino is +using to access HDFS has access to the Hive warehouse directory. The Hive +warehouse directory is specified by the configuration variable +`hive.metastore.warehouse.dir` in `hive-site.xml`, and the default +value is `/user/hive/warehouse`. + +When not using Kerberos with HDFS, Trino accesses HDFS using the +OS user of the Trino process. For example, if Trino is running as +`nobody`, it accesses HDFS as `nobody`. You can override this +username by setting the `HADOOP_USER_NAME` system property in the +Trino {ref}`jvm-config`, replacing `hdfs_user` with the +appropriate username: + +```text +-DHADOOP_USER_NAME=hdfs_user +``` + +The `hive` user generally works, since Hive is often started with +the `hive` user and this user has access to the Hive warehouse. + +Whenever you change the user Trino is using to access HDFS, remove +`/tmp/presto-*` on HDFS, as the new user may not have access to +the existing temporary directories. + +(hive-configuration-properties)= + +### Hive general configuration properties + +The following table lists general configuration properties for the Hive +connector. There are additional sets of configuration properties throughout the +Hive connector documentation. + +```{eval-rst} +.. list-table:: Hive general configuration properties + :widths: 35, 50, 15 + :header-rows: 1 + + * - Property Name + - Description + - Default + * - ``hive.config.resources`` + - An optional comma-separated list of HDFS configuration files. These + files must exist on the machines running Trino. Only specify this if + absolutely necessary to access HDFS. Example: ``/etc/hdfs-site.xml`` + - + * - ``hive.recursive-directories`` + - Enable reading data from subdirectories of table or partition locations. + If disabled, subdirectories are ignored. This is equivalent to the + ``hive.mapred.supports.subdirectories`` property in Hive. + - ``false`` + * - ``hive.ignore-absent-partitions`` + - Ignore partitions when the file system location does not exist rather + than failing the query. This skips data that may be expected to be part + of the table. + - ``false`` + * - ``hive.storage-format`` + - The default file format used when creating new tables. + - ``ORC`` + * - ``hive.compression-codec`` + - The compression codec to use when writing files. Possible values are + ``NONE``, ``SNAPPY``, ``LZ4``, ``ZSTD``, or ``GZIP``. + - ``GZIP`` + * - ``hive.force-local-scheduling`` + - Force splits to be scheduled on the same node as the Hadoop DataNode + process serving the split data. This is useful for installations where + Trino is collocated with every DataNode. + - ``false`` + * - ``hive.respect-table-format`` + - Should new partitions be written using the existing table format or the + default Trino format? + - ``true`` + * - ``hive.immutable-partitions`` + - Can new data be inserted into existing partitions? If ``true`` then + setting ``hive.insert-existing-partitions-behavior`` to ``APPEND`` is + not allowed. This also affects the ``insert_existing_partitions_behavior`` + session property in the same way. + - ``false`` + * - ``hive.insert-existing-partitions-behavior`` + - What happens when data is inserted into an existing partition? Possible + values are + + * ``APPEND`` - appends data to existing partitions + * ``OVERWRITE`` - overwrites existing partitions + * ``ERROR`` - modifying existing partitions is not allowed + - ``APPEND`` + * - ``hive.target-max-file-size`` + - Best effort maximum size of new files. + - ``1GB`` + * - ``hive.create-empty-bucket-files`` + - Should empty files be created for buckets that have no data? + - ``false`` + * - ``hive.validate-bucketing`` + - Enables validation that data is in the correct bucket when reading + bucketed tables. + - ``true`` + * - ``hive.partition-statistics-sample-size`` + - Specifies the number of partitions to analyze when computing table + statistics. + - 100 + * - ``hive.max-partitions-per-writers`` + - Maximum number of partitions per writer. + - 100 + * - ``hive.max-partitions-for-eager-load`` + - The maximum number of partitions for a single table scan to load eagerly + on the coordinator. Certain optimizations are not possible without eager + loading. + - 100,000 + * - ``hive.max-partitions-per-scan`` + - Maximum number of partitions for a single table scan. + - 1,000,000 + * - ``hive.dfs.replication`` + - Hadoop file system replication factor. + - + * - ``hive.security`` + - See :doc:`hive-security`. + - + * - ``security.config-file`` + - Path of config file to use when ``hive.security=file``. See + :ref:`catalog-file-based-access-control` for details. + - + * - ``hive.non-managed-table-writes-enabled`` + - Enable writes to non-managed (external) Hive tables. + - ``false`` + * - ``hive.non-managed-table-creates-enabled`` + - Enable creating non-managed (external) Hive tables. + - ``true`` + * - ``hive.collect-column-statistics-on-write`` + - Enables automatic column level statistics collection on write. See + `Table Statistics <#table-statistics>`__ for details. + - ``true`` + * - ``hive.file-status-cache-tables`` + - Cache directory listing for specific tables. Examples: + + * ``fruit.apple,fruit.orange`` to cache listings only for tables + ``apple`` and ``orange`` in schema ``fruit`` + * ``fruit.*,vegetable.*`` to cache listings for all tables + in schemas ``fruit`` and ``vegetable`` + * ``*`` to cache listings for all tables in all schemas + - + * - ``hive.file-status-cache.max-retained-size`` + - Maximum retained size of cached file status entries. + - ``1GB`` + * - ``hive.file-status-cache-expire-time`` + - How long a cached directory listing is considered valid. + - ``1m`` + * - ``hive.per-transaction-file-status-cache.max-retained-size`` + - Maximum retained size of all entries in per transaction file status cache. + Retained size limit is shared across all running queries. + - ``100MB`` + * - ``hive.rcfile.time-zone`` + - Adjusts binary encoded timestamp values to a specific time zone. For + Hive 3.1+, this must be set to UTC. + - JVM default + * - ``hive.timestamp-precision`` + - Specifies the precision to use for Hive columns of type ``TIMESTAMP``. + Possible values are ``MILLISECONDS``, ``MICROSECONDS`` and ``NANOSECONDS``. + Values with higher precision than configured are rounded. + - ``MILLISECONDS`` + * - ``hive.temporary-staging-directory-enabled`` + - Controls whether the temporary staging directory configured at + ``hive.temporary-staging-directory-path`` is used for write + operations. Temporary staging directory is never used for writes to + non-sorted tables on S3, encrypted HDFS or external location. Writes to + sorted tables will utilize this path for staging temporary files during + sorting operation. When disabled, the target storage will be used for + staging while writing sorted tables which can be inefficient when + writing to object stores like S3. + - ``true`` + * - ``hive.temporary-staging-directory-path`` + - Controls the location of temporary staging directory that is used for + write operations. The ``${USER}`` placeholder can be used to use a + different location for each user. + - ``/tmp/presto-${USER}`` + * - ``hive.hive-views.enabled`` + - Enable translation for :ref:`Hive views `. + - ``false`` + * - ``hive.hive-views.legacy-translation`` + - Use the legacy algorithm to translate :ref:`Hive views `. + You can use the ``hive_views_legacy_translation`` catalog session + property for temporary, catalog specific use. + - ``false`` + * - ``hive.parallel-partitioned-bucketed-writes`` + - Improve parallelism of partitioned and bucketed table writes. When + disabled, the number of writing threads is limited to number of buckets. + - ``true`` + * - ``hive.fs.new-directory-permissions`` + - Controls the permissions set on new directories created for tables. It + must be either 'skip' or an octal number, with a leading 0. If set to + 'skip', permissions of newly created directories will not be set by + Trino. + - ``0777`` + * - ``hive.fs.cache.max-size`` + - Maximum number of cached file system objects. + - 1000 + * - ``hive.query-partition-filter-required`` + - Set to ``true`` to force a query to use a partition filter. You can use + the ``query_partition_filter_required`` catalog session property for + temporary, catalog specific use. + - ``false`` + * - ``hive.table-statistics-enabled`` + - Enables :doc:`/optimizer/statistics`. The equivalent + :doc:`catalog session property ` is + ``statistics_enabled`` for session specific use. Set to ``false`` to + disable statistics. Disabling statistics means that + :doc:`/optimizer/cost-based-optimizations` can not make smart decisions + about the query plan. + - ``true`` + * - ``hive.auto-purge`` + - Set the default value for the auto_purge table property for managed + tables. See the :ref:`hive-table-properties` for more information on + auto_purge. + - ``false`` + * - ``hive.partition-projection-enabled`` + - Enables Athena partition projection support + - ``false`` + * - ``hive.max-partition-drops-per-query`` + - Maximum number of partitions to drop in a single query. + - 100,000 + * - ``hive.single-statement-writes`` + - Enables auto-commit for all writes. This can be used to disallow + multi-statement write transactions. + - ``false`` +``` + +## Storage + +The Hive connector supports the following storage options: + +- {doc}`Amazon S3 ` +- {doc}`Azure Storage ` +- {doc}`Google Cloud Storage ` +- {doc}`IBM Cloud Object Storage ` + +The Hive connector also supports {doc}`storage caching `. + +## Security + +Please see the {doc}`/connector/hive-security` section for information on the +security options available for the Hive connector. + +(hive-sql-support)= + +## SQL support + +The connector provides read access and write access to data and metadata in the +configured object storage system and metadata stores: + +- {ref}`Globally available statements `; see also + {ref}`Globally available statements ` + +- {ref}`Read operations ` + +- {ref}`sql-write-operations`: + + - {ref}`sql-data-management`; see also + {ref}`Hive-specific data management ` + - {ref}`sql-schema-table-management`; see also + {ref}`Hive-specific schema and table management ` + - {ref}`sql-view-management`; see also + {ref}`Hive-specific view management ` + +- {ref}`sql-security-operations`: see also + {ref}`SQL standard-based authorization for object storage ` + +- {ref}`sql-transactions` + +Refer to {doc}`the migration guide ` for practical advice +on migrating from Hive to Trino. + +The following sections provide Hive-specific information regarding SQL support. + +(hive-examples)= + +### Basic usage examples + +The examples shown here work on Google Cloud Storage by replacing `s3://` with +`gs://`. + +Create a new Hive table named `page_views` in the `web` schema +that is stored using the ORC file format, partitioned by date and +country, and bucketed by user into `50` buckets. Note that Hive +requires the partition columns to be the last columns in the table: + +``` +CREATE TABLE example.web.page_views ( + view_time TIMESTAMP, + user_id BIGINT, + page_url VARCHAR, + ds DATE, + country VARCHAR +) +WITH ( + format = 'ORC', + partitioned_by = ARRAY['ds', 'country'], + bucketed_by = ARRAY['user_id'], + bucket_count = 50 +) +``` + +Create a new Hive schema named `web` that stores tables in an +S3 bucket named `my-bucket`: + +``` +CREATE SCHEMA example.web +WITH (location = 's3://my-bucket/') +``` + +Drop a schema: + +``` +DROP SCHEMA example.web +``` + +Drop a partition from the `page_views` table: + +``` +DELETE FROM example.web.page_views +WHERE ds = DATE '2016-08-09' + AND country = 'US' +``` + +Query the `page_views` table: + +``` +SELECT * FROM example.web.page_views +``` + +List the partitions of the `page_views` table: + +``` +SELECT * FROM example.web."page_views$partitions" +``` + +Create an external Hive table named `request_logs` that points at +existing data in S3: + +``` +CREATE TABLE example.web.request_logs ( + request_time TIMESTAMP, + url VARCHAR, + ip VARCHAR, + user_agent VARCHAR +) +WITH ( + format = 'TEXTFILE', + external_location = 's3://my-bucket/data/logs/' +) +``` + +Collect statistics for the `request_logs` table: + +``` +ANALYZE example.web.request_logs; +``` + +Drop the external table `request_logs`. This only drops the metadata +for the table. The referenced data directory is not deleted: + +``` +DROP TABLE example.web.request_logs +``` + +- {doc}`/sql/create-table-as` can be used to create transactional tables in ORC format like this: + + ``` + CREATE TABLE + WITH ( + format='ORC', + transactional=true + ) + AS + ``` + +Add an empty partition to the `page_views` table: + +``` +CALL system.create_empty_partition( + schema_name => 'web', + table_name => 'page_views', + partition_columns => ARRAY['ds', 'country'], + partition_values => ARRAY['2016-08-09', 'US']); +``` + +Drop stats for a partition of the `page_views` table: + +``` +CALL system.drop_stats( + schema_name => 'web', + table_name => 'page_views', + partition_values => ARRAY[ARRAY['2016-08-09', 'US']]); +``` + +(hive-procedures)= + +### Procedures + +Use the {doc}`/sql/call` statement to perform data manipulation or +administrative tasks. Procedures must include a qualified catalog name, if your +Hive catalog is called `web`: + +``` +CALL web.system.example_procedure() +``` + +The following procedures are available: + +- `system.create_empty_partition(schema_name, table_name, partition_columns, partition_values)` + + Create an empty partition in the specified table. + +- `system.sync_partition_metadata(schema_name, table_name, mode, case_sensitive)` + + Check and update partitions list in metastore. There are three modes available: + + - `ADD` : add any partitions that exist on the file system, but not in the metastore. + - `DROP`: drop any partitions that exist in the metastore, but not on the file system. + - `FULL`: perform both `ADD` and `DROP`. + + The `case_sensitive` argument is optional. The default value is `true` for compatibility + with Hive's `MSCK REPAIR TABLE` behavior, which expects the partition column names in + file system paths to use lowercase (e.g. `col_x=SomeValue`). Partitions on the file system + not conforming to this convention are ignored, unless the argument is set to `false`. + +- `system.drop_stats(schema_name, table_name, partition_values)` + + Drops statistics for a subset of partitions or the entire table. The partitions are specified as an + array whose elements are arrays of partition values (similar to the `partition_values` argument in + `create_empty_partition`). If `partition_values` argument is omitted, stats are dropped for the + entire table. + +(register-partition)= + +- `system.register_partition(schema_name, table_name, partition_columns, partition_values, location)` + + Registers existing location as a new partition in the metastore for the specified table. + + When the `location` argument is omitted, the partition location is + constructed using `partition_columns` and `partition_values`. + + Due to security reasons, the procedure is enabled only when `hive.allow-register-partition-procedure` + is set to `true`. + +(unregister-partition)= + +- `system.unregister_partition(schema_name, table_name, partition_columns, partition_values)` + + Unregisters given, existing partition in the metastore for the specified table. + The partition data is not deleted. + +(hive-flush-metadata-cache)= + +- `system.flush_metadata_cache()` + + Flush all Hive metadata caches. + +- `system.flush_metadata_cache(schema_name => ..., table_name => ...)` + + Flush Hive metadata caches entries connected with selected table. + Procedure requires named parameters to be passed + +- `system.flush_metadata_cache(schema_name => ..., table_name => ..., partition_columns => ARRAY[...], partition_values => ARRAY[...])` + + Flush Hive metadata cache entries connected with selected partition. + Procedure requires named parameters to be passed. + +(hive-data-management)= + +### Data management + +Some {ref}`data management ` statements may be affected by +the Hive catalog's authorization check policy. In the default `legacy` policy, +some statements are disabled by default. See {doc}`hive-security` for more +information. + +The {ref}`sql-data-management` functionality includes support for `INSERT`, +`UPDATE`, `DELETE`, and `MERGE` statements, with the exact support +depending on the storage system, file format, and metastore. + +When connecting to a Hive metastore version 3.x, the Hive connector supports +reading from and writing to insert-only and ACID tables, with full support for +partitioning and bucketing. + +{doc}`/sql/delete` applied to non-transactional tables is only supported if the +table is partitioned and the `WHERE` clause matches entire partitions. +Transactional Hive tables with ORC format support "row-by-row" deletion, in +which the `WHERE` clause may match arbitrary sets of rows. + +{doc}`/sql/update` is only supported for transactional Hive tables with format +ORC. `UPDATE` of partition or bucket columns is not supported. + +{doc}`/sql/merge` is only supported for ACID tables. + +ACID tables created with [Hive Streaming Ingest](https://cwiki.apache.org/confluence/display/Hive/Streaming+Data+Ingest) +are not supported. + +(hive-schema-and-table-management)= + +### Schema and table management + +The Hive connector supports querying and manipulating Hive tables and schemas +(databases). While some uncommon operations must be performed using +Hive directly, most operations can be performed using Trino. + +#### Schema evolution + +Hive allows the partitions in a table to have a different schema than the +table. This occurs when the column types of a table are changed after +partitions already exist (that use the original column types). The Hive +connector supports this by allowing the same conversions as Hive: + +- `VARCHAR` to and from `TINYINT`, `SMALLINT`, `INTEGER` and `BIGINT` +- `REAL` to `DOUBLE` +- Widening conversions for integers, such as `TINYINT` to `SMALLINT` + +Any conversion failure results in null, which is the same behavior +as Hive. For example, converting the string `'foo'` to a number, +or converting the string `'1234'` to a `TINYINT` (which has a +maximum value of `127`). + +(hive-avro-schema)= + +#### Avro schema evolution + +Trino supports querying and manipulating Hive tables with the Avro storage +format, which has the schema set based on an Avro schema file/literal. Trino is +also capable of creating the tables in Trino by infering the schema from a +valid Avro schema file located locally, or remotely in HDFS/Web server. + +To specify that the Avro schema should be used for interpreting table data, use +the `avro_schema_url` table property. + +The schema can be placed in the local file system or remotely in the following +locations: + +- HDFS (e.g. `avro_schema_url = 'hdfs://user/avro/schema/avro_data.avsc'`) +- S3 (e.g. `avro_schema_url = 's3n:///schema_bucket/schema/avro_data.avsc'`) +- A web server (e.g. `avro_schema_url = 'http://example.org/schema/avro_data.avsc'`) + +The URL, where the schema is located, must be accessible from the Hive metastore +and Trino coordinator/worker nodes. + +Alternatively, you can use the table property `avro_schema_literal` to define +the Avro schema. + +The table created in Trino using the `avro_schema_url` or +`avro_schema_literal` property behaves the same way as a Hive table with +`avro.schema.url` or `avro.schema.literal` set. + +Example: + +``` +CREATE TABLE example.avro.avro_data ( + id BIGINT + ) +WITH ( + format = 'AVRO', + avro_schema_url = '/usr/local/avro_data.avsc' +) +``` + +The columns listed in the DDL (`id` in the above example) is ignored if `avro_schema_url` is specified. +The table schema matches the schema in the Avro schema file. Before any read operation, the Avro schema is +accessed so the query result reflects any changes in schema. Thus Trino takes advantage of Avro's backward compatibility abilities. + +If the schema of the table changes in the Avro schema file, the new schema can still be used to read old data. +Newly added/renamed fields *must* have a default value in the Avro schema file. + +The schema evolution behavior is as follows: + +- Column added in new schema: + Data created with an older schema produces a *default* value when table is using the new schema. +- Column removed in new schema: + Data created with an older schema no longer outputs the data from the column that was removed. +- Column is renamed in the new schema: + This is equivalent to removing the column and adding a new one, and data created with an older schema + produces a *default* value when table is using the new schema. +- Changing type of column in the new schema: + If the type coercion is supported by Avro or the Hive connector, then the conversion happens. + An error is thrown for incompatible types. + +##### Limitations + +The following operations are not supported when `avro_schema_url` is set: + +- `CREATE TABLE AS` is not supported. +- Bucketing(`bucketed_by`) columns are not supported in `CREATE TABLE`. +- `ALTER TABLE` commands modifying columns are not supported. + +(hive-alter-table-execute)= + +#### ALTER TABLE EXECUTE + +The connector supports the following commands for use with {ref}`ALTER TABLE +EXECUTE `. + +```{include} optimize.fragment +``` + +The `optimize` command is disabled by default, and can be enabled for a +catalog with the `.non_transactional_optimize_enabled` +session property: + +```sql +SET SESSION .non_transactional_optimize_enabled=true +``` + +:::{warning} +Because Hive tables are non-transactional, take note of the following possible +outcomes: + +- If queries are run against tables that are currently being optimized, + duplicate rows may be read. +- In rare cases where exceptions occur during the `optimize` operation, + a manual cleanup of the table directory is needed. In this situation, refer + to the Trino logs and query failure messages to see which files must be + deleted. +::: + +(hive-table-properties)= + +#### Table properties + +Table properties supply or set metadata for the underlying tables. This +is key for {doc}`/sql/create-table-as` statements. Table properties are passed +to the connector using a {doc}`WITH ` clause: + +``` +CREATE TABLE tablename +WITH (format='CSV', + csv_escape = '"') +``` + +```{eval-rst} +.. list-table:: Hive connector table properties + :widths: 20, 60, 20 + :header-rows: 1 + + * - Property name + - Description + - Default + * - ``auto_purge`` + - Indicates to the configured metastore to perform a purge when a table or + partition is deleted instead of a soft deletion using the trash. + - + * - ``avro_schema_url`` + - The URI pointing to :ref:`hive-avro-schema` for the table. + - + * - ``bucket_count`` + - The number of buckets to group data into. Only valid if used with + ``bucketed_by``. + - 0 + * - ``bucketed_by`` + - The bucketing column for the storage table. Only valid if used with + ``bucket_count``. + - ``[]`` + * - ``bucketing_version`` + - Specifies which Hive bucketing version to use. Valid values are ``1`` + or ``2``. + - + * - ``csv_escape`` + - The CSV escape character. Requires CSV format. + - + * - ``csv_quote`` + - The CSV quote character. Requires CSV format. + - + * - ``csv_separator`` + - The CSV separator character. Requires CSV format. You can use other + separators such as ``|`` or use Unicode to configure invisible separators + such tabs with ``U&'\0009'``. + - ``,`` + * - ``external_location`` + - The URI for an external Hive table on S3, Azure Blob Storage, etc. See the + :ref:`hive-examples` for more information. + - + * - ``format`` + - The table file format. Valid values include ``ORC``, ``PARQUET``, + ``AVRO``, ``RCBINARY``, ``RCTEXT``, ``SEQUENCEFILE``, ``JSON``, + ``TEXTFILE``, ``CSV``, and ``REGEX``. The catalog property + ``hive.storage-format`` sets the default value and can change it to a + different default. + - + * - ``null_format`` + - The serialization format for ``NULL`` value. Requires TextFile, RCText, + or SequenceFile format. + - + * - ``orc_bloom_filter_columns`` + - Comma separated list of columns to use for ORC bloom filter. It improves + the performance of queries using range predicates when reading ORC files. + Requires ORC format. + - ``[]`` + * - ``orc_bloom_filter_fpp`` + - The ORC bloom filters false positive probability. Requires ORC format. + - 0.05 + * - ``partitioned_by`` + - The partitioning column for the storage table. The columns listed in the + ``partitioned_by`` clause must be the last columns as defined in the DDL. + - ``[]`` + * - ``skip_footer_line_count`` + - The number of footer lines to ignore when parsing the file for data. + Requires TextFile or CSV format tables. + - + * - ``skip_header_line_count`` + - The number of header lines to ignore when parsing the file for data. + Requires TextFile or CSV format tables. + - + * - ``sorted_by`` + - The column to sort by to determine bucketing for row. Only valid if + ``bucketed_by`` and ``bucket_count`` are specified as well. + - ``[]`` + * - ``textfile_field_separator`` + - Allows the use of custom field separators, such as '|', for TextFile + formatted tables. + - + * - ``textfile_field_separator_escape`` + - Allows the use of a custom escape character for TextFile formatted tables. + - + * - ``transactional`` + - Set this property to ``true`` to create an ORC ACID transactional table. + Requires ORC format. This property may be shown as true for insert-only + tables created using older versions of Hive. + - + * - ``partition_projection_enabled`` + - Enables partition projection for selected table. + Mapped from AWS Athena table property + `projection.enabled `_. + - + * - ``partition_projection_ignore`` + - Ignore any partition projection properties stored in the metastore for + the selected table. This is a Trino-only property which allows you to + work around compatibility issues on a specific table, and if enabled, + Trino ignores all other configuration options related to partition + projection. + - + * - ``partition_projection_location_template`` + - Projected partition location template, such as + ``s3a://test/name=${name}/``. Mapped from the AWS Athena table property + `storage.location.template `_ + - ``${table_location}/${partition_name}`` + * - ``extra_properties`` + - Additional properties added to a Hive table. The properties are not used by Trino, + and are available in the ``$properties`` metadata table. + The properties are not included in the output of ``SHOW CREATE TABLE`` statements. + - +``` + +(hive-special-tables)= + +#### Metadata tables + +The raw Hive table properties are available as a hidden table, containing a +separate column per table property, with a single row containing the property +values. + +##### `$properties` table + +The properties table name is composed with the table name and `$properties` appended. +It exposes the parameters of the table in the metastore. + +You can inspect the property names and values with a simple query: + +``` +SELECT * FROM example.web."page_views$properties"; +``` + +```text + stats_generated_via_stats_task | auto.purge | presto_query_id | presto_version | transactional +---------------------------------------------+------------+-----------------------------+----------------+--------------- + workaround for potential lack of HIVE-12730 | false | 20230705_152456_00001_nfugi | 423 | false +``` + +##### `$partitions` table + +The `$partitions` table provides a list of all partition values +of a partitioned table. + +The following example query returns all partition values from the +`page_views` table in the `web` schema of the `example` catalog: + +``` +SELECT * FROM example.web."page_views$partitions"; +``` + +```text + day | country +------------+--------- + 2023-07-01 | POL + 2023-07-02 | POL + 2023-07-03 | POL + 2023-03-01 | USA + 2023-03-02 | USA +``` + +(hive-column-properties)= + +#### Column properties + +```{eval-rst} +.. list-table:: Hive connector column properties + :widths: 20, 60, 20 + :header-rows: 1 + + * - Property name + - Description + - Default + * - ``partition_projection_type`` + - Defines the type of partition projection to use on this column. + May be used only on partition columns. Available types: + ``ENUM``, ``INTEGER``, ``DATE``, ``INJECTED``. + Mapped from the AWS Athena table property + `projection.${columnName}.type `_. + - + * - ``partition_projection_values`` + - Used with ``partition_projection_type`` set to ``ENUM``. Contains a static + list of values used to generate partitions. + Mapped from the AWS Athena table property + `projection.${columnName}.values `_. + - + * - ``partition_projection_range`` + - Used with ``partition_projection_type`` set to ``INTEGER`` or ``DATE`` to + define a range. It is a two-element array, describing the minimum and + maximum range values used to generate partitions. Generation starts from + the minimum, then increments by the defined + ``partition_projection_interval`` to the maximum. For example, the format + is ``['1', '4']`` for a ``partition_projection_type`` of ``INTEGER`` and + ``['2001-01-01', '2001-01-07']`` or ``['NOW-3DAYS', 'NOW']`` for a + ``partition_projection_type`` of ``DATE``. Mapped from the AWS Athena + table property + `projection.${columnName}.range `_. + - + * - ``partition_projection_interval`` + - Used with ``partition_projection_type`` set to ``INTEGER`` or ``DATE``. It + represents the interval used to generate partitions within + the given range ``partition_projection_range``. Mapped from the AWS Athena + table property + `projection.${columnName}.interval `_. + - + * - ``partition_projection_digits`` + - Used with ``partition_projection_type`` set to ``INTEGER``. + The number of digits to be used with integer column projection. + Mapped from the AWS Athena table property + `projection.${columnName}.digits `_. + - + * - ``partition_projection_format`` + - Used with ``partition_projection_type`` set to ``DATE``. + The date column projection format, defined as a string such as ``yyyy MM`` + or ``MM-dd-yy HH:mm:ss`` for use with the + `Java DateTimeFormatter class `_. + Mapped from the AWS Athena table property + `projection.${columnName}.format `_. + - + * - ``partition_projection_interval_unit`` + - Used with ``partition_projection_type=DATA``. + The date column projection range interval unit + given in ``partition_projection_interval``. + Mapped from the AWS Athena table property + `projection.${columnName}.interval.unit `_. + - +``` + +(hive-special-columns)= + +#### Metadata columns + +In addition to the defined columns, the Hive connector automatically exposes +metadata in a number of hidden columns in each table: + +- `$bucket`: Bucket number for this row +- `$path`: Full file system path name of the file for this row +- `$file_modified_time`: Date and time of the last modification of the file for this row +- `$file_size`: Size of the file for this row +- `$partition`: Partition name for this row + +You can use these columns in your SQL statements like any other column. They +can be selected directly, or used in conditional statements. For example, you +can inspect the file size, location and partition for each record: + +``` +SELECT *, "$path", "$file_size", "$partition" +FROM example.web.page_views; +``` + +Retrieve all records that belong to files stored in the partition +`ds=2016-08-09/country=US`: + +``` +SELECT *, "$path", "$file_size" +FROM example.web.page_views +WHERE "$partition" = 'ds=2016-08-09/country=US' +``` + +(hive-sql-view-management)= + +### View management + +Trino allows reading from Hive materialized views, and can be configured to +support reading Hive views. + +#### Materialized views + +The Hive connector supports reading from Hive materialized views. +In Trino, these views are presented as regular, read-only tables. + +(hive-views)= + +#### Hive views + +Hive views are defined in HiveQL and stored in the Hive Metastore Service. They +are analyzed to allow read access to the data. + +The Hive connector includes support for reading Hive views with three different +modes. + +- Disabled +- Legacy +- Experimental + +If using Hive views from Trino is required, you must compare results in Hive and +Trino for each view definition to ensure identical results. Use the experimental +mode whenever possible. Avoid using the legacy mode. Leave Hive views support +disabled, if you are not accessing any Hive views from Trino. + +You can configure the behavior in your catalog properties file. + +By default, Hive views are executed with the `RUN AS DEFINER` security mode. +Set the `hive.hive-views.run-as-invoker` catalog configuration property to +`true` to use `RUN AS INVOKER` semantics. + +**Disabled** + +The default behavior is to ignore Hive views. This means that your business +logic and data encoded in the views is not available in Trino. + +**Legacy** + +A very simple implementation to execute Hive views, and therefore allow read +access to the data in Trino, can be enabled with +`hive.hive-views.enabled=true` and +`hive.hive-views.legacy-translation=true`. + +For temporary usage of the legacy behavior for a specific catalog, you can set +the `hive_views_legacy_translation` {doc}`catalog session property +` to `true`. + +This legacy behavior interprets any HiveQL query that defines a view as if it +is written in SQL. It does not do any translation, but instead relies on the +fact that HiveQL is very similar to SQL. + +This works for very simple Hive views, but can lead to problems for more complex +queries. For example, if a HiveQL function has an identical signature but +different behaviors to the SQL version, the returned results may differ. In more +extreme cases the queries might fail, or not even be able to be parsed and +executed. + +**Experimental** + +The new behavior is better engineered and has the potential to become a lot +more powerful than the legacy implementation. It can analyze, process, and +rewrite Hive views and contained expressions and statements. + +It supports the following Hive view functionality: + +- `UNION [DISTINCT]` and `UNION ALL` against Hive views +- Nested `GROUP BY` clauses +- `current_user()` +- `LATERAL VIEW OUTER EXPLODE` +- `LATERAL VIEW [OUTER] EXPLODE` on array of struct +- `LATERAL VIEW json_tuple` + +You can enable the experimental behavior with +`hive.hive-views.enabled=true`. Remove the +`hive.hive-views.legacy-translation` property or set it to `false` to make +sure legacy is not enabled. + +Keep in mind that numerous features are not yet implemented when experimenting +with this feature. The following is an incomplete list of **missing** +functionality: + +- HiveQL `current_date`, `current_timestamp`, and others +- Hive function calls including `translate()`, window functions, and others +- Common table expressions and simple case expressions +- Honor timestamp precision setting +- Support all Hive data types and correct mapping to Trino types +- Ability to process custom UDFs + +(hive-fte-support)= + +## Fault-tolerant execution support + +The connector supports {doc}`/admin/fault-tolerant-execution` of query +processing. Read and write operations are both supported with any retry policy +on non-transactional tables. + +Read operations are supported with any retry policy on transactional tables. +Write operations and `CREATE TABLE ... AS` operations are not supported with +any retry policy on transactional tables. + +## Performance + +The connector includes a number of performance improvements, detailed in the +following sections. + +### Table statistics + +The Hive connector supports collecting and managing {doc}`table statistics +` to improve query processing performance. + +When writing data, the Hive connector always collects basic statistics +(`numFiles`, `numRows`, `rawDataSize`, `totalSize`) +and by default will also collect column level statistics: + +```{eval-rst} +.. list-table:: Available table statistics + :widths: 35, 65 + :header-rows: 1 + + * - Column type + - Collectible statistics + * - ``TINYINT`` + - Number of nulls, number of distinct values, min/max values + * - ``SMALLINT`` + - Number of nulls, number of distinct values, min/max values + * - ``INTEGER`` + - Number of nulls, number of distinct values, min/max values + * - ``BIGINT`` + - Number of nulls, number of distinct values, min/max values + * - ``DOUBLE`` + - Number of nulls, number of distinct values, min/max values + * - ``REAL`` + - Number of nulls, number of distinct values, min/max values + * - ``DECIMAL`` + - Number of nulls, number of distinct values, min/max values + * - ``DATE`` + - Number of nulls, number of distinct values, min/max values + * - ``TIMESTAMP`` + - Number of nulls, number of distinct values, min/max values + * - ``VARCHAR`` + - Number of nulls, number of distinct values + * - ``CHAR`` + - Number of nulls, number of distinct values + * - ``VARBINARY`` + - Number of nulls + * - ``BOOLEAN`` + - Number of nulls, number of true/false values +``` + +(hive-analyze)= + +#### Updating table and partition statistics + +If your queries are complex and include joining large data sets, +running {doc}`/sql/analyze` on tables/partitions may improve query performance +by collecting statistical information about the data. + +When analyzing a partitioned table, the partitions to analyze can be specified +via the optional `partitions` property, which is an array containing +the values of the partition keys in the order they are declared in the table schema: + +``` +ANALYZE table_name WITH ( + partitions = ARRAY[ + ARRAY['p1_value1', 'p1_value2'], + ARRAY['p2_value1', 'p2_value2']]) +``` + +This query will collect statistics for two partitions with keys +`p1_value1, p1_value2` and `p2_value1, p2_value2`. + +On wide tables, collecting statistics for all columns can be expensive and can have a +detrimental effect on query planning. It is also typically unnecessary - statistics are +only useful on specific columns, like join keys, predicates, grouping keys. One can +specify a subset of columns to be analyzed via the optional `columns` property: + +``` +ANALYZE table_name WITH ( + partitions = ARRAY[ARRAY['p2_value1', 'p2_value2']], + columns = ARRAY['col_1', 'col_2']) +``` + +This query collects statistics for columns `col_1` and `col_2` for the partition +with keys `p2_value1, p2_value2`. + +Note that if statistics were previously collected for all columns, they must be dropped +before re-analyzing just a subset: + +``` +CALL system.drop_stats('schema_name', 'table_name') +``` + +You can also drop statistics for selected partitions only: + +``` +CALL system.drop_stats( + schema_name => 'schema', + table_name => 'table', + partition_values => ARRAY[ARRAY['p2_value1', 'p2_value2']]) +``` + +(hive-dynamic-filtering)= + +### Dynamic filtering + +The Hive connector supports the {doc}`dynamic filtering ` optimization. +Dynamic partition pruning is supported for partitioned tables stored in any file format +for broadcast as well as partitioned joins. +Dynamic bucket pruning is supported for bucketed tables stored in any file format for +broadcast joins only. + +For tables stored in ORC or Parquet file format, dynamic filters are also pushed into +local table scan on worker nodes for broadcast joins. Dynamic filter predicates +pushed into the ORC and Parquet readers are used to perform stripe or row-group pruning +and save on disk I/O. Sorting the data within ORC or Parquet files by the columns used in +join criteria significantly improves the effectiveness of stripe or row-group pruning. +This is because grouping similar data within the same stripe or row-group +greatly improves the selectivity of the min/max indexes maintained at stripe or +row-group level. + +#### Delaying execution for dynamic filters + +It can often be beneficial to wait for the collection of dynamic filters before starting +a table scan. This extra wait time can potentially result in significant overall savings +in query and CPU time, if dynamic filtering is able to reduce the amount of scanned data. + +For the Hive connector, a table scan can be delayed for a configured amount of +time until the collection of dynamic filters by using the configuration property +`hive.dynamic-filtering.wait-timeout` in the catalog file or the catalog +session property `.dynamic_filtering_wait_timeout`. + +(hive-table-redirection)= + +### Table redirection + +```{include} table-redirection.fragment +``` + +The connector supports redirection from Hive tables to Iceberg +and Delta Lake tables with the following catalog configuration properties: + +- `hive.iceberg-catalog-name` for redirecting the query to {doc}`/connector/iceberg` +- `hive.delta-lake-catalog-name` for redirecting the query to {doc}`/connector/delta-lake` + +(hive-performance-tuning-configuration)= + +### Performance tuning configuration properties + +The following table describes performance tuning properties for the Hive +connector. + +:::{warning} +Performance tuning configuration properties are considered expert-level +features. Altering these properties from their default values is likely to +cause instability and performance degradation. +::: + +```{eval-rst} +.. list-table:: + :widths: 30, 50, 20 + :header-rows: 1 + + * - Property name + - Description + - Default value + * - ``hive.max-outstanding-splits`` + - The target number of buffered splits for each table scan in a query, + before the scheduler tries to pause. + - ``1000`` + * - ``hive.max-outstanding-splits-size`` + - The maximum size allowed for buffered splits for each table scan + in a query, before the query fails. + - ``256 MB`` + * - ``hive.max-splits-per-second`` + - The maximum number of splits generated per second per table scan. This + can be used to reduce the load on the storage system. By default, there + is no limit, which results in Trino maximizing the parallelization of + data access. + - + * - ``hive.max-initial-splits`` + - For each table scan, the coordinator first assigns file sections of up + to ``max-initial-split-size``. After ``max-initial-splits`` have been + assigned, ``max-split-size`` is used for the remaining splits. + - ``200`` + * - ``hive.max-initial-split-size`` + - The size of a single file section assigned to a worker until + ``max-initial-splits`` have been assigned. Smaller splits results in + more parallelism, which gives a boost to smaller queries. + - ``32 MB`` + * - ``hive.max-split-size`` + - The largest size of a single file section assigned to a worker. Smaller + splits result in more parallelism and thus can decrease latency, but + also have more overhead and increase load on the system. + - ``64 MB`` +``` + +## Hive 3-related limitations + +- For security reasons, the `sys` system catalog is not accessible. +- Hive's `timestamp with local zone` data type is mapped to + `timestamp with time zone` with UTC timezone. It only supports reading + values - writing to tables with columns of this type is not supported. +- Due to Hive issues [HIVE-21002](https://issues.apache.org/jira/browse/HIVE-21002) + and [HIVE-22167](https://issues.apache.org/jira/browse/HIVE-22167), Trino does + not correctly read `TIMESTAMP` values from Parquet, RCBinary, or Avro + file formats created by Hive 3.1 or later. When reading from these file formats, + Trino returns different results than Hive. +- Trino does not support gathering table statistics for Hive transactional tables. + You must use Hive to gather table statistics with + [ANALYZE statement](https://cwiki.apache.org/confluence/display/hive/statsdev#StatsDev-ExistingTables%E2%80%93ANALYZE) + after table creation. diff --git a/430/_sources/connector/hudi.md.txt b/430/_sources/connector/hudi.md.txt new file mode 100644 index 000000000..1236a4b21 --- /dev/null +++ b/430/_sources/connector/hudi.md.txt @@ -0,0 +1,211 @@ +# Hudi connector + +```{raw} html + +``` + +The Hudi connector enables querying [Hudi](https://hudi.apache.org/docs/overview/) tables. + +## Requirements + +To use the Hudi connector, you need: + +- Hudi version 0.12.3 or higher. +- Network access from the Trino coordinator and workers to the Hudi storage. +- Access to a Hive metastore service (HMS). +- Network access from the Trino coordinator to the HMS. +- Data files stored in the Parquet file format. These can be configured using + {ref}`file format configuration properties ` per + catalog. + +## General configuration + +To configure the Hive connector, create a catalog properties file +`etc/catalog/example.properties` that references the `hudi` +connector and defines the HMS to use with the `hive.metastore.uri` +configuration property: + +```properties +connector.name=hudi +hive.metastore.uri=thrift://example.net:9083 +``` + +There are {ref}`HMS configuration properties ` +available for use with the Hudi connector. The connector recognizes Hudi tables +synced to the metastore by the [Hudi sync tool](https://hudi.apache.org/docs/syncing_metastore). + +Additionally, following configuration properties can be set depending on the use-case: + +```{eval-rst} +.. list-table:: Hudi configuration properties + :widths: 30, 55, 15 + :header-rows: 1 + + * - Property name + - Description + - Default + * - ``hudi.columns-to-hide`` + - List of column names that are hidden from the query output. + It can be used to hide Hudi meta fields. By default, no fields are hidden. + - + * - ``hudi.parquet.use-column-names`` + - Access Parquet columns using names from the file. If disabled, then columns + are accessed using the index. Only applicable to Parquet file format. + - ``true`` + * - ``hudi.split-generator-parallelism`` + - Number of threads to generate splits from partitions. + - ``4`` + * - ``hudi.split-loader-parallelism`` + - Number of threads to run background split loader. + A single background split loader is needed per query. + - ``4`` + * - ``hudi.size-based-split-weights-enabled`` + - Unlike uniform splitting, size-based splitting ensures that each batch of splits + has enough data to process. By default, it is enabled to improve performance. + - ``true`` + * - ``hudi.standard-split-weight-size`` + - The split size corresponding to the standard weight (1.0) + when size-based split weights are enabled. + - ``128MB`` + * - ``hudi.minimum-assigned-split-weight`` + - Minimum weight that a split can be assigned + when size-based split weights are enabled. + - ``0.05`` + * - ``hudi.max-splits-per-second`` + - Rate at which splits are queued for processing. + The queue is throttled if this rate limit is breached. + - ``Integer.MAX_VALUE`` + * - ``hudi.max-outstanding-splits`` + - Maximum outstanding splits in a batch enqueued for processing. + - ``1000`` + * - ``hudi.per-transaction-metastore-cache-maximum-size`` + - Maximum number of metastore data objects per transaction in + the Hive metastore cache. + - ``2000`` + +``` + +## SQL support + +The connector provides read access to data in the Hudi table that has been synced to +Hive metastore. The {ref}`globally available ` +and {ref}`read operation ` statements are supported. + +### Basic usage examples + +In the following example queries, `stock_ticks_cow` is the Hudi copy-on-write +table referred to in the Hudi [quickstart guide](https://hudi.apache.org/docs/docker_demo/). + +```sql +USE example.example_schema; + +SELECT symbol, max(ts) +FROM stock_ticks_cow +GROUP BY symbol +HAVING symbol = 'GOOG'; +``` + +```text + symbol | _col1 | +-----------+----------------------+ + GOOG | 2018-08-31 10:59:00 | +(1 rows) +``` + +```sql +SELECT dt, symbol +FROM stock_ticks_cow +WHERE symbol = 'GOOG'; +``` + +```text + dt | symbol | +------------+--------+ + 2018-08-31 | GOOG | +(1 rows) +``` + +```sql +SELECT dt, count(*) +FROM stock_ticks_cow +GROUP BY dt; +``` + +```text + dt | _col1 | +------------+--------+ + 2018-08-31 | 99 | +(1 rows) +``` + +### Schema and table management + +Hudi supports [two types of tables](https://hudi.apache.org/docs/table_types) +depending on how the data is indexed and laid out on the file system. The following +table displays a support matrix of tables types and query types for the connector: + +```{eval-rst} +.. list-table:: Hudi configuration properties + :widths: 45, 55 + :header-rows: 1 + + * - Table type + - Supported query type + * - Copy on write + - Snapshot queries + * - Merge on read + - Read-optimized queries +``` + +(hudi-metadata-tables)= + +#### Metadata tables + +The connector exposes a metadata table for each Hudi table. +The metadata table contains information about the internal structure +of the Hudi table. You can query each metadata table by appending the +metadata table name to the table name: + +``` +SELECT * FROM "test_table$timeline" +``` + +##### `$timeline` table + +The `$timeline` table provides a detailed view of meta-data instants +in the Hudi table. Instants are specific points in time. + +You can retrieve the information about the timeline of the Hudi table +`test_table` by using the following query: + +``` +SELECT * FROM "test_table$timeline" +``` + +```text + timestamp | action | state +--------------------+---------+----------- +8667764846443717831 | commit | COMPLETED +7860805980949777961 | commit | COMPLETED +``` + +The output of the query has the following columns: + +```{eval-rst} +.. list-table:: Timeline columns + :widths: 20, 30, 50 + :header-rows: 1 + + * - Name + - Type + - Description + * - ``timestamp`` + - ``VARCHAR`` + - Instant time is typically a timestamp when the actions performed. + * - ``action`` + - ``VARCHAR`` + - `Type of action `_ performed on the table. + * - ``state`` + - ``VARCHAR`` + - Current state of the instant. +``` diff --git a/430/_sources/connector/iceberg.md.txt b/430/_sources/connector/iceberg.md.txt new file mode 100644 index 000000000..76278a697 --- /dev/null +++ b/430/_sources/connector/iceberg.md.txt @@ -0,0 +1,1479 @@ +# Iceberg connector + +```{raw} html + +``` + +Apache Iceberg is an open table format for huge analytic datasets. The Iceberg +connector allows querying data stored in files written in Iceberg format, as +defined in the [Iceberg Table Spec](https://iceberg.apache.org/spec/). The +connector supports Apache Iceberg table spec versions 1 and 2. + +The table state is maintained in metadata files. All changes to table +state create a new metadata file and replace the old metadata with an atomic +swap. The table metadata file tracks the table schema, partitioning +configuration, custom properties, and snapshots of the table contents. + +Iceberg data files are stored in either Parquet, ORC, or Avro format, as +determined by the `format` property in the table definition. + +Iceberg is designed to improve on the known scalability limitations of Hive, +which stores table metadata in a metastore that is backed by a relational +database such as MySQL. It tracks partition locations in the metastore, but not +individual data files. Trino queries using the {doc}`/connector/hive` must +first call the metastore to get partition locations, then call the underlying +file system to list all data files inside each partition, and then read metadata +from each data file. + +Since Iceberg stores the paths to data files in the metadata files, it only +consults the underlying file system for files that must be read. + +## Requirements + +To use Iceberg, you need: + +- Network access from the Trino coordinator and workers to the distributed + object storage. + +- Access to a {ref}`Hive metastore service (HMS) `, an + {ref}`AWS Glue catalog `, a {ref}`JDBC catalog + `, a {ref}`REST catalog `, or a + {ref}`Nessie server `. + +- Data files stored in a supported file format. These can be configured using + file format configuration properties per catalog: + + - {ref}`ORC ` + - {ref}`Parquet ` (default) + +## General configuration + +To configure the Iceberg connector, create a catalog properties file +`etc/catalog/example.properties` that references the `iceberg` +connector and defines a metastore type. The Hive metastore catalog is the +default implementation. To use a {ref}`Hive metastore `, +`iceberg.catalog.type` must be set to `hive_metastore` and +`hive.metastore.uri` must be configured: + +```properties +connector.name=iceberg +iceberg.catalog.type=hive_metastore +hive.metastore.uri=thrift://example.net:9083 +``` + +Other metadata catalog types as listed in the requirements section of this topic +are available. Each metastore type has specific configuration properties along +with {ref}`general metastore configuration properties +`. + +The following configuration properties are independent of which catalog +implementation is used: + +```{eval-rst} +.. list-table:: Iceberg general configuration properties + :widths: 30, 58, 12 + :header-rows: 1 + + * - Property name + - Description + - Default + * - ``iceberg.catalog.type`` + - Define the metastore type to use. Possible values are: + + * ``hive_metastore`` + * ``glue`` + * ``jdbc`` + * ``rest`` + * ``nessie`` + - + * - ``iceberg.file-format`` + - Define the data storage file format for Iceberg tables. + Possible values are: + + * ``PARQUET`` + * ``ORC`` + * ``AVRO`` + - ``PARQUET`` + * - ``iceberg.compression-codec`` + - The compression codec used when writing files. + Possible values are: + + * ``NONE`` + * ``SNAPPY`` + * ``LZ4`` + * ``ZSTD`` + * ``GZIP`` + - ``ZSTD`` + * - ``iceberg.use-file-size-from-metadata`` + - Read file sizes from metadata instead of file system. This property must + only be used as a workaround for `this issue + `_. The problem was fixed + in Iceberg version 0.11.0. + - ``true`` + * - ``iceberg.max-partitions-per-writer`` + - Maximum number of partitions handled per writer. + - ``100`` + * - ``iceberg.target-max-file-size`` + - Target maximum size of written files; the actual size may be larger. + - ``1GB`` + * - ``iceberg.unique-table-location`` + - Use randomized, unique table locations. + - ``true`` + * - ``iceberg.dynamic-filtering.wait-timeout`` + - Maximum duration to wait for completion of dynamic filters during split + generation. + - ``0s`` + * - ``iceberg.delete-schema-locations-fallback`` + - Whether schema locations are deleted when Trino can't determine whether + they contain external files. + - ``false`` + * - ``iceberg.minimum-assigned-split-weight`` + - A decimal value in the range (0, 1] used as a minimum for weights assigned + to each split. A low value may improve performance on tables with small + files. A higher value may improve performance for queries with highly + skewed aggregations or joins. + - 0.05 + * - ``iceberg.table-statistics-enabled`` + - Enables :doc:`/optimizer/statistics`. The equivalent :doc:`catalog session + property ` is ``statistics_enabled`` for session + specific use. Set to ``false`` to disable statistics. Disabling statistics + means that :doc:`/optimizer/cost-based-optimizations` cannot make better + decisions about the query plan. + - ``true`` + * - ``iceberg.projection-pushdown-enabled`` + - Enable :doc:`projection pushdown ` + - ``true`` + * - ``iceberg.hive-catalog-name`` + - Catalog to redirect to when a Hive table is referenced. + - + * - ``iceberg.materialized-views.storage-schema`` + - Schema for creating materialized views storage tables. When this property + is not configured, storage tables are created in the same schema as the + materialized view definition. When the ``storage_schema`` materialized + view property is specified, it takes precedence over this catalog + property. + - Empty + * - ``iceberg.register-table-procedure.enabled`` + - Enable to allow user to call ``register_table`` procedure. + - ``false`` + * - ``iceberg.query-partition-filter-required`` + - Set to ``true`` to force a query to use a partition filter. + You can use the ``query_partition_filter_required`` catalog session property for temporary, catalog specific use. + - ``false`` +``` + +## Type mapping + +The connector reads and writes data into the supported data file formats Avro, +ORC, and Parquet, following the Iceberg specification. + +Because Trino and Iceberg each support types that the other does not, this +connector {ref}`modifies some types ` when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction. + +The Iceberg specification includes supported data types and the mapping to the +formating in the Avro, ORC, or Parquet files: + +- [Iceberg to Avro](https://iceberg.apache.org/spec/#avro) +- [Iceberg to ORC](https://iceberg.apache.org/spec/#orc) +- [Iceberg to Parquet](https://iceberg.apache.org/spec/#parquet) + +### Iceberg to Trino type mapping + +The connector maps Iceberg types to the corresponding Trino types according to +the following table: + +```{eval-rst} +.. list-table:: Iceberg to Trino type mapping + :widths: 40, 60 + :header-rows: 1 + + * - Iceberg type + - Trino type + * - ``BOOLEAN`` + - ``BOOLEAN`` + * - ``INT`` + - ``INTEGER`` + * - ``LONG`` + - ``BIGINT`` + * - ``FLOAT`` + - ``REAL`` + * - ``DOUBLE`` + - ``DOUBLE`` + * - ``DECIMAL(p,s)`` + - ``DECIMAL(p,s)`` + * - ``DATE`` + - ``DATE`` + * - ``TIME`` + - ``TIME(6)`` + * - ``TIMESTAMP`` + - ``TIMESTAMP(6)`` + * - ``TIMESTAMPTZ`` + - ``TIMESTAMP(6) WITH TIME ZONE`` + * - ``STRING`` + - ``VARCHAR`` + * - ``UUID`` + - ``UUID`` + * - ``BINARY`` + - ``VARBINARY`` + * - ``FIXED (L)`` + - ``VARBINARY`` + * - ``STRUCT(...)`` + - ``ROW(...)`` + * - ``LIST(e)`` + - ``ARRAY(e)`` + * - ``MAP(k,v)`` + - ``MAP(k,v)`` +``` + +No other types are supported. + +### Trino to Iceberg type mapping + +The connector maps Trino types to the corresponding Iceberg types according to +the following table: + +```{eval-rst} +.. list-table:: Trino to Iceberg type mapping + :widths: 40, 60 + :header-rows: 1 + + * - Trino type + - Iceberg type + * - ``BOOLEAN`` + - ``BOOLEAN`` + * - ``INTEGER`` + - ``INT`` + * - ``BIGINT`` + - ``LONG`` + * - ``REAL`` + - ``FLOAT`` + * - ``DOUBLE`` + - ``DOUBLE`` + * - ``DECIMAL(p,s)`` + - ``DECIMAL(p,s)`` + * - ``DATE`` + - ``DATE`` + * - ``TIME(6)`` + - ``TIME`` + * - ``TIMESTAMP(6)`` + - ``TIMESTAMP`` + * - ``TIMESTAMP(6) WITH TIME ZONE`` + - ``TIMESTAMPTZ`` + * - ``VARCHAR`` + - ``STRING`` + * - ``UUID`` + - ``UUID`` + * - ``VARBINARY`` + - ``BINARY`` + * - ``ROW(...)`` + - ``STRUCT(...)`` + * - ``ARRAY(e)`` + - ``LIST(e)`` + * - ``MAP(k,v)`` + - ``MAP(k,v)`` +``` + +No other types are supported. + +## Security + +The Iceberg connector allows you to choose one of several means of providing +authorization at the catalog level. + +(iceberg-authorization)= + +### Authorization checks + +You can enable authorization checks for the connector by setting the +`iceberg.security` property in the catalog properties file. This property must +be one of the following values: + +```{eval-rst} +.. list-table:: Iceberg security values + :widths: 30, 60 + :header-rows: 1 + + * - Property value + - Description + * - ``ALLOW_ALL`` + - No authorization checks are enforced. + * - ``SYSTEM`` + - The connector relies on system-level access control. + * - ``READ_ONLY`` + - Operations that read data or metadata, such as :doc:`/sql/select` are + permitted. No operations that write data or metadata, such as + :doc:`/sql/create-table`, :doc:`/sql/insert`, or :doc:`/sql/delete` are + allowed. + * - ``FILE`` + - Authorization checks are enforced using a catalog-level access control + configuration file whose path is specified in the ``security.config-file`` + catalog configuration property. See + :ref:`catalog-file-based-access-control` for information on the + authorization configuration file. +``` + +(iceberg-sql-support)= + +## SQL support + +This connector provides read access and write access to data and metadata in +Iceberg. In addition to the {ref}`globally available ` +and {ref}`read operation ` statements, the connector +supports the following features: + +- {ref}`sql-write-operations`: + + - {ref}`iceberg-schema-table-management` and {ref}`iceberg-tables` + - {ref}`iceberg-data-management` + - {ref}`sql-view-management` + - {ref}`sql-materialized-view-management`, see also {ref}`iceberg-materialized-views` + +### Basic usage examples + +The connector supports creating schemas. You can create a schema with or without +a specified location. + +You can create a schema with the {doc}`/sql/create-schema` statement and the +`location` schema property. The tables in this schema, which have no explicit +`location` set in {doc}`/sql/create-table` statement, are located in a +subdirectory under the directory corresponding to the schema location. + +Create a schema on S3: + +``` +CREATE SCHEMA example.example_s3_schema +WITH (location = 's3://my-bucket/a/path/'); +``` + +Create a schema on an S3-compatible object storage such as MinIO: + +``` +CREATE SCHEMA example.example_s3a_schema +WITH (location = 's3a://my-bucket/a/path/'); +``` + +Create a schema on HDFS: + +``` +CREATE SCHEMA example.example_hdfs_schema +WITH (location='hdfs://hadoop-master:9000/user/hive/warehouse/a/path/'); +``` + +Optionally, on HDFS, the location can be omitted: + +``` +CREATE SCHEMA example.example_hdfs_schema; +``` + +The Iceberg connector supports creating tables using the {doc}`CREATE TABLE +` syntax. Optionally, specify the {ref}`table properties +` supported by this connector: + +``` +CREATE TABLE example_table ( + c1 INTEGER, + c2 DATE, + c3 DOUBLE +) +WITH ( + format = 'PARQUET', + partitioning = ARRAY['c1', 'c2'], + sorted_by = ARRAY['c3'], + location = 's3://my-bucket/a/path/' +); +``` + +When the `location` table property is omitted, the content of the table is +stored in a subdirectory under the directory corresponding to the schema +location. + +The Iceberg connector supports creating tables using the {doc}`CREATE TABLE AS +` with {doc}`SELECT ` syntax: + +``` +CREATE TABLE tiny_nation +WITH ( + format = 'PARQUET' +) +AS + SELECT * + FROM nation + WHERE nationkey < 10; +``` + +Another flavor of creating tables with {doc}`CREATE TABLE AS +` is with {doc}`VALUES ` syntax: + +``` +CREATE TABLE yearly_clicks ( + year, + clicks +) +WITH ( + partitioning = ARRAY['year'] +) +AS VALUES + (2021, 10000), + (2022, 20000); +``` + +### Procedures + +Use the {doc}`/sql/call` statement to perform data manipulation or +administrative tasks. Procedures are available in the system schema of each +catalog. The following code snippet displays how to call the +`example_procedure` in the `examplecatalog` catalog: + +``` +CALL examplecatalog.system.example_procedure() +``` + +(iceberg-register-table)= + +#### Register table + +The connector can register existing Iceberg tables with the catalog. + +The procedure `system.register_table` allows the caller to register an +existing Iceberg table in the metastore, using its existing metadata and data +files: + +``` +CALL example.system.register_table(schema_name => 'testdb', table_name => 'customer_orders', table_location => 'hdfs://hadoop-master:9000/user/hive/warehouse/customer_orders-581fad8517934af6be1857a903559d44') +``` + +In addition, you can provide a file name to register a table with specific +metadata. This may be used to register the table with some specific table state, +or may be necessary if the connector cannot automatically figure out the +metadata version to use: + +``` +CALL example.system.register_table(schema_name => 'testdb', table_name => 'customer_orders', table_location => 'hdfs://hadoop-master:9000/user/hive/warehouse/customer_orders-581fad8517934af6be1857a903559d44', metadata_file_name => '00003-409702ba-4735-4645-8f14-09537cc0b2c8.metadata.json') +``` + +To prevent unauthorized users from accessing data, this procedure is disabled by +default. The procedure is enabled only when +`iceberg.register-table-procedure.enabled` is set to `true`. + +(iceberg-unregister-table)= + +#### Unregister table + +The connector can unregister existing Iceberg tables from the catalog. + +The procedure `system.unregister_table` allows the caller to unregister an +existing Iceberg table from the metastores without deleting the data: + +``` +CALL example.system.unregister_table(schema_name => 'testdb', table_name => 'customer_orders') +``` + +#### Migrate table + +The connector can read from or write to Hive tables that have been migrated to +Iceberg. + +Use the procedure `system.migrate` to move a table from the Hive format to the +Iceberg format, loaded with the source’s data files. Table schema, partitioning, +properties, and location are copied from the source table. A bucketed Hive table +will be migrated as a non-bucketed Iceberg table. The data files in the Hive table +must use the Parquet, ORC, or Avro file format. + +The procedure must be called for a specific catalog `example` with the +relevant schema and table names supplied with the required parameters +`schema_name` and `table_name`: + +``` +CALL example.system.migrate( + schema_name => 'testdb', + table_name => 'customer_orders') +``` + +Migrate fails if any table partition uses an unsupported file format. + +In addition, you can provide a `recursive_directory` argument to migrate a +Hive table that contains subdirectories: + +``` +CALL example.system.migrate( + schema_name => 'testdb', + table_name => 'customer_orders', + recursive_directory => 'true') +``` + +The default value is `fail`, which causes the migrate procedure to throw an +exception if subdirectories are found. Set the value to `true` to migrate +nested directories, or `false` to ignore them. + +(iceberg-data-management)= + +### Data management + +The {ref}`sql-data-management` functionality includes support for `INSERT`, +`UPDATE`, `DELETE`, and `MERGE` statements. + +(iceberg-delete)= + +#### Deletion by partition + +For partitioned tables, the Iceberg connector supports the deletion of entire +partitions if the `WHERE` clause specifies filters only on the +identity-transformed partitioning columns, that can match entire partitions. +Given the table definition from {ref}`Partitioned Tables ` +section, the following SQL statement deletes all partitions for which +`country` is `US`: + +``` +DELETE FROM example.testdb.customer_orders +WHERE country = 'US' +``` + +A partition delete is performed if the `WHERE` clause meets these conditions. + +#### Row level deletion + +Tables using v2 of the Iceberg specification support deletion of individual rows +by writing position delete files. + +(iceberg-schema-table-management)= + +### Schema and table management + +The {ref}`sql-schema-table-management` functionality includes support for: + +- {doc}`/sql/create-schema` +- {doc}`/sql/drop-schema` +- {doc}`/sql/alter-schema` +- {doc}`/sql/create-table` +- {doc}`/sql/create-table-as` +- {doc}`/sql/drop-table` +- {doc}`/sql/alter-table` +- {doc}`/sql/comment` + +#### Schema evolution + +Iceberg supports schema evolution, with safe column add, drop, reorder, and +rename operations, including in nested structures. Table partitioning can also +be changed and the connector can still query data created before the +partitioning change. + +(iceberg-alter-table-execute)= + +#### ALTER TABLE EXECUTE + +The connector supports the following commands for use with {ref}`ALTER TABLE +EXECUTE `. + +```{include} optimize.fragment +``` + +##### expire_snapshots + +The `expire_snapshots` command removes all snapshots and all related metadata +and data files. Regularly expiring snapshots is recommended to delete data files +that are no longer needed, and to keep the size of table metadata small. The +procedure affects all snapshots that are older than the time period configured +with the `retention_threshold` parameter. + +`expire_snapshots` can be run as follows: + +```sql +ALTER TABLE test_table EXECUTE expire_snapshots(retention_threshold => '7d') +``` + +The value for `retention_threshold` must be higher than or equal to +`iceberg.expire_snapshots.min-retention` in the catalog, otherwise the +procedure fails with a similar message: `Retention specified (1.00d) is shorter +than the minimum retention configured in the system (7.00d)`. The default value +for this property is `7d`. + +##### remove_orphan_files + +The `remove_orphan_files` command removes all files from a table's data +directory that are not linked from metadata files and that are older than the +value of `retention_threshold` parameter. Deleting orphan files from time to +time is recommended to keep size of a table's data directory under control. + +`remove_orphan_files` can be run as follows: + +```sql +ALTER TABLE test_table EXECUTE remove_orphan_files(retention_threshold => '7d') +``` + +The value for `retention_threshold` must be higher than or equal to +`iceberg.remove_orphan_files.min-retention` in the catalog otherwise the +procedure fails with a similar message: `Retention specified (1.00d) is shorter +than the minimum retention configured in the system (7.00d)`. The default value +for this property is `7d`. + +(drop-extended-stats)= + +##### drop_extended_stats + +The `drop_extended_stats` command removes all extended statistics information +from the table. + +`drop_extended_stats` can be run as follows: + +```sql +ALTER TABLE test_table EXECUTE drop_extended_stats +``` + +(iceberg-alter-table-set-properties)= + +#### ALTER TABLE SET PROPERTIES + +The connector supports modifying the properties on existing tables using +{ref}`ALTER TABLE SET PROPERTIES `. + +The following table properties can be updated after a table is created: + +- `format` +- `format_version` +- `partitioning` +- `sorted_by` + +For example, to update a table from v1 of the Iceberg specification to v2: + +```sql +ALTER TABLE table_name SET PROPERTIES format_version = 2; +``` + +Or to set the column `my_new_partition_column` as a partition column on a +table: + +```sql +ALTER TABLE table_name SET PROPERTIES partitioning = ARRAY[, 'my_new_partition_column']; +``` + +The current values of a table's properties can be shown using {doc}`SHOW CREATE +TABLE `. + +(iceberg-table-properties)= + +##### Table properties + +Table properties supply or set metadata for the underlying tables. This is key +for {doc}`/sql/create-table-as` statements. Table properties are passed to the +connector using a {doc}`WITH ` clause. + + +```{eval-rst} +.. list-table:: Iceberg table properties + :widths: 40, 60 + :header-rows: 1 + + * - Property name + - Description + * - ``format`` + - Optionally specifies the format of table data files; either ``PARQUET``, + ``ORC`, or ``AVRO``. Defaults to the value of the ``iceberg.file-format`` + catalog configuration property, which defaults to ``PARQUET``. + * - ``partitioning`` + - Optionally specifies table partitioning. If a table is partitioned by + columns ``c1`` and ``c2``, the partitioning property is ``partitioning = + ARRAY['c1', 'c2']``. + * - ``location`` + - Optionally specifies the file system location URI for the table. + * - ``format_version`` + - Optionally specifies the format version of the Iceberg specification to + use for new tables; either ``1`` or ``2``. Defaults to ``2``. Version + ``2`` is required for row level deletes. + * - ``orc_bloom_filter_columns`` + - Comma-separated list of columns to use for ORC bloom filter. It improves + the performance of queries using Equality and IN predicates when reading + ORC files. Requires ORC format. Defaults to ``[]``. + * - ``orc_bloom_filter_fpp`` + - The ORC bloom filters false positive probability. Requires ORC format. + Defaults to ``0.05``. +``` + +The table definition below specifies to use Parquet files, partitioning by columns +`c1` and `c2`, and a file system location of +`/var/example_tables/test_table`: + +``` +CREATE TABLE test_table ( + c1 INTEGER, + c2 DATE, + c3 DOUBLE) +WITH ( + format = 'PARQUET', + partitioning = ARRAY['c1', 'c2'], + location = '/var/example_tables/test_table') +``` + +The table definition below specifies to use ORC files, bloom filter index by columns +`c1` and `c2`, fpp is 0.05, and a file system location of +`/var/example_tables/test_table`: + +``` +CREATE TABLE test_table ( + c1 INTEGER, + c2 DATE, + c3 DOUBLE) +WITH ( + format = 'ORC', + location = '/var/example_tables/test_table', + orc_bloom_filter_columns = ARRAY['c1', 'c2'], + orc_bloom_filter_fpp = 0.05) +``` + +(iceberg-metadata-tables)= + +#### Metadata tables + +The connector exposes several metadata tables for each Iceberg table. These +metadata tables contain information about the internal structure of the Iceberg +table. You can query each metadata table by appending the metadata table name to +the table name: + +``` +SELECT * FROM "test_table$properties" +``` + +##### `$properties` table + +The `$properties` table provides access to general information about Iceberg +table configuration and any additional metadata key/value pairs that the table +is tagged with. + +You can retrieve the properties of the current snapshot of the Iceberg table +`test_table` by using the following query: + +``` +SELECT * FROM "test_table$properties" +``` + +```text + key | value | +-----------------------+----------+ +write.format.default | PARQUET | +``` + +##### `$history` table + +The `$history` table provides a log of the metadata changes performed on the +Iceberg table. + +You can retrieve the changelog of the Iceberg table `test_table` by using the +following query: + +``` +SELECT * FROM "test_table$history" +``` + +```text + made_current_at | snapshot_id | parent_id | is_current_ancestor +----------------------------------+----------------------+----------------------+-------------------- +2022-01-10 08:11:20 Europe/Vienna | 8667764846443717831 | | true +2022-01-10 08:11:34 Europe/Vienna | 7860805980949777961 | 8667764846443717831 | true +``` + +The output of the query has the following columns: + +```{eval-rst} +.. list-table:: History columns + :widths: 30, 30, 40 + :header-rows: 1 + + * - Name + - Type + - Description + * - ``made_current_at`` + - ``TIMESTAMP(3) WITH TIME ZONE`` + - The time when the snapshot became active. + * - ``snapshot_id`` + - ``BIGINT`` + - The identifier of the snapshot. + * - ``parent_id`` + - ``BIGINT`` + - The identifier of the parent snapshot. + * - ``is_current_ancestor`` + - ``BOOLEAN`` + - Whether or not this snapshot is an ancestor of the current snapshot. +``` + +##### `$snapshots` table + +The `$snapshots` table provides a detailed view of snapshots of the Iceberg +table. A snapshot consists of one or more file manifests, and the complete table +contents are represented by the union of all the data files in those manifests. + +You can retrieve the information about the snapshots of the Iceberg table +`test_table` by using the following query: + +``` +SELECT * FROM "test_table$snapshots" +``` + +```text + committed_at | snapshot_id | parent_id | operation | manifest_list | summary +----------------------------------+----------------------+----------------------+--------------------+------------------------------------------------------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- +2022-01-10 08:11:20 Europe/Vienna | 8667764846443717831 | | append | hdfs://hadoop-master:9000/user/hive/warehouse/test_table/metadata/snap-8667764846443717831-1-100cf97e-6d56-446e-8961-afdaded63bc4.avro | {changed-partition-count=0, total-equality-deletes=0, total-position-deletes=0, total-delete-files=0, total-files-size=0, total-records=0, total-data-files=0} +2022-01-10 08:11:34 Europe/Vienna | 7860805980949777961 | 8667764846443717831 | append | hdfs://hadoop-master:9000/user/hive/warehouse/test_table/metadata/snap-7860805980949777961-1-faa19903-1455-4bb8-855a-61a1bbafbaa7.avro | {changed-partition-count=1, added-data-files=1, total-equality-deletes=0, added-records=1, total-position-deletes=0, added-files-size=442, total-delete-files=0, total-files-size=442, total-records=1, total-data-files=1} +``` + +The output of the query has the following columns: + +```{eval-rst} +.. list-table:: Snapshots columns + :widths: 20, 30, 50 + :header-rows: 1 + + * - Name + - Type + - Description + * - ``committed_at`` + - ``TIMESTAMP(3) WITH TIME ZONE`` + - The time when the snapshot became active. + * - ``snapshot_id`` + - ``BIGINT`` + - The identifier for the snapshot. + * - ``parent_id`` + - ``BIGINT`` + - The identifier for the parent snapshot. + * - ``operation`` + - ``VARCHAR`` + - The type of operation performed on the Iceberg table. The supported + operation types in Iceberg are: + + * ``append`` when new data is appended. + * ``replace`` when files are removed and replaced without changing the + data in the table. + * ``overwrite`` when new data is added to overwrite existing data. + * ``delete`` when data is deleted from the table and no new data is added. + * - ``manifest_list`` + - ``VARCHAR`` + - The list of Avro manifest files containing the detailed information about + the snapshot changes. + * - ``summary`` + - ``map(VARCHAR, VARCHAR)`` + - A summary of the changes made from the previous snapshot to the current + snapshot. +``` + +##### `$manifests` table + +The `$manifests` table provides a detailed overview of the manifests +corresponding to the snapshots performed in the log of the Iceberg table. + +You can retrieve the information about the manifests of the Iceberg table +`test_table` by using the following query: + +``` +SELECT * FROM "test_table$manifests" +``` + +```text + path | length | partition_spec_id | added_snapshot_id | added_data_files_count | added_rows_count | existing_data_files_count | existing_rows_count | deleted_data_files_count | deleted_rows_count | partitions +----------------------------------------------------------------------------------------------------------------+-----------------+----------------------+-----------------------+-------------------------+------------------+-----------------------------+---------------------+-----------------------------+--------------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------- + hdfs://hadoop-master:9000/user/hive/warehouse/test_table/metadata/faa19903-1455-4bb8-855a-61a1bbafbaa7-m0.avro | 6277 | 0 | 7860805980949777961 | 1 | 100 | 0 | 0 | 0 | 0 | {{contains_null=false, contains_nan= false, lower_bound=1, upper_bound=1},{contains_null=false, contains_nan= false, lower_bound=2021-01-12, upper_bound=2021-01-12}} +``` + +The output of the query has the following columns: + +```{eval-rst} +.. list-table:: Manifests columns + :widths: 30, 30, 40 + :header-rows: 1 + + * - Name + - Type + - Description + * - ``path`` + - ``VARCHAR`` + - The manifest file location. + * - ``length`` + - ``BIGINT`` + - The manifest file length. + * - ``partition_spec_id`` + - ``INTEGER`` + - The identifier for the partition specification used to write the manifest + file. + * - ``added_snapshot_id`` + - ``BIGINT`` + - The identifier of the snapshot during which this manifest entry has been + added. + * - ``added_data_files_count`` + - ``INTEGER`` + - The number of data files with status ``ADDED`` in the manifest file. + * - ``added_rows_count`` + - ``BIGINT`` + - The total number of rows in all data files with status ``ADDED`` in the + manifest file. + * - ``existing_data_files_count`` + - ``INTEGER`` + - The number of data files with status ``EXISTING`` in the manifest file. + * - ``existing_rows_count`` + - ``BIGINT`` + - The total number of rows in all data files with status ``EXISTING`` in the + manifest file. + * - ``deleted_data_files_count`` + - ``INTEGER`` + - The number of data files with status ``DELETED`` in the manifest file. + * - ``deleted_rows_count`` + - ``BIGINT`` + - The total number of rows in all data files with status ``DELETED`` in the + manifest file. + * - ``partitions`` + - ``ARRAY(row(contains_null BOOLEAN, contains_nan BOOLEAN, lower_bound VARCHAR, upper_bound VARCHAR))`` + - Partition range metadata. +``` + +##### `$partitions` table + +The `$partitions` table provides a detailed overview of the partitions of the +Iceberg table. + +You can retrieve the information about the partitions of the Iceberg table +`test_table` by using the following query: + +``` +SELECT * FROM "test_table$partitions" +``` + +```text + partition | record_count | file_count | total_size | data +-----------------------+---------------+---------------+---------------+------------------------------------------------------ +{c1=1, c2=2021-01-12} | 2 | 2 | 884 | {c3={min=1.0, max=2.0, null_count=0, nan_count=NULL}} +{c1=1, c2=2021-01-13} | 1 | 1 | 442 | {c3={min=1.0, max=1.0, null_count=0, nan_count=NULL}} +``` + +The output of the query has the following columns: + +```{eval-rst} +.. list-table:: Partitions columns + :widths: 20, 30, 50 + :header-rows: 1 + + * - Name + - Type + - Description + * - ``partition`` + - ``ROW(...)`` + - A row that contains the mapping of the partition column names to the + partition column values. + * - ``record_count`` + - ``BIGINT`` + - The number of records in the partition. + * - ``file_count`` + - ``BIGINT`` + - The number of files mapped in the partition. + * - ``total_size`` + - ``BIGINT`` + - The size of all the files in the partition. + * - ``data`` + - ``ROW(... ROW (min ..., max ... , null_count BIGINT, nan_count BIGINT))`` + - Partition range metadata. +``` + +##### `$files` table + +The `$files` table provides a detailed overview of the data files in current +snapshot of the Iceberg table. + +To retrieve the information about the data files of the Iceberg table +`test_table`, use the following query: + +``` +SELECT * FROM "test_table$files" +``` + +```text + content | file_path | record_count | file_format | file_size_in_bytes | column_sizes | value_counts | null_value_counts | nan_value_counts | lower_bounds | upper_bounds | key_metadata | split_offsets | equality_ids +----------+-------------------------------------------------------------------------------------------------------------------------------+-----------------+---------------+----------------------+----------------------+-------------------+--------------------+-------------------+-----------------------------+-----------------------------+----------------+----------------+--------------- + 0 | hdfs://hadoop-master:9000/user/hive/warehouse/test_table/data/c1=3/c2=2021-01-14/af9872b2-40f3-428f-9c87-186d2750d84e.parquet | 1 | PARQUET | 442 | {1=40, 2=40, 3=44} | {1=1, 2=1, 3=1} | {1=0, 2=0, 3=0} | | {1=3, 2=2021-01-14, 3=1.3} | {1=3, 2=2021-01-14, 3=1.3} | | | +``` + +The output of the query has the following columns: + +```{eval-rst} +.. list-table:: Files columns + :widths: 25, 30, 45 + :header-rows: 1 + + * - Name + - Type + - Description + * - ``content`` + - ``INTEGER`` + - Type of content stored in the file. The supported content types in Iceberg + are: + + * ``DATA(0)`` + * ``POSITION_DELETES(1)`` + * ``EQUALITY_DELETES(2)`` + * - ``file_path`` + - ``VARCHAR`` + - The data file location. + * - ``file_format`` + - ``VARCHAR`` + - The format of the data file. + * - ``record_count`` + - ``BIGINT`` + - The number of entries contained in the data file. + * - ``file_size_in_bytes`` + - ``BIGINT`` + - The data file size + * - ``column_sizes`` + - ``map(INTEGER, BIGINT)`` + - Mapping between the Iceberg column ID and its corresponding size in the + file. + * - ``value_counts`` + - ``map(INTEGER, BIGINT)`` + - Mapping between the Iceberg column ID and its corresponding count of + entries in the file. + * - ``null_value_counts`` + - ``map(INTEGER, BIGINT)`` + - Mapping between the Iceberg column ID and its corresponding count of + ``NULL`` values in the file. + * - ``nan_value_counts`` + - ``map(INTEGER, BIGINT)`` + - Mapping between the Iceberg column ID and its corresponding count of non- + numerical values in the file. + * - ``lower_bounds`` + - ``map(INTEGER, BIGINT)`` + - Mapping between the Iceberg column ID and its corresponding lower bound in + the file. + * - ``upper_bounds`` + - ``map(INTEGER, BIGINT)`` + - Mapping between the Iceberg column ID and its corresponding upper bound in + the file. + * - ``key_metadata`` + - ``VARBINARY`` + - Metadata about the encryption key used to encrypt this file, if applicable. + * - ``split_offsets`` + - ``array(BIGINT)`` + - List of recommended split locations. + * - ``equality_ids`` + - ``array(INTEGER)`` + - The set of field IDs used for equality comparison in equality delete files. +``` + +##### `$refs` table + +The `$refs` table provides information about Iceberg references including +branches and tags. + +You can retrieve the references of the Iceberg table `test_table` by using the +following query: + +``` +SELECT * FROM "test_table$refs" +``` + +```text +name | type | snapshot_id | max_reference_age_in_ms | min_snapshots_to_keep | max_snapshot_age_in_ms | +----------------+--------+-------------+-------------------------+-----------------------+------------------------+ +example_tag | TAG | 10000000000 | 10000 | null | null | +example_branch | BRANCH | 20000000000 | 20000 | 2 | 30000 | +``` + +The output of the query has the following columns: + +```{eval-rst} +.. list-table:: Refs columns + :widths: 20, 30, 50 + :header-rows: 1 + + * - Name + - Type + - Description + * - ``name`` + - ``VARCHAR`` + - Name of the reference. + * - ``type`` + - ``VARCHAR`` + - Type of the reference, either ``BRANCH`` or ``TAG``. + * - ``snapshot_id`` + - ``BIGINT`` + - The snapshot ID of the reference. + * - ``max_reference_age_in_ms`` + - ``BIGINT`` + - The maximum age of the reference before it could be expired. + * - ``min_snapshots_to_keep`` + - ``INTEGER`` + - For branch only, the minimum number of snapshots to keep in a branch. + * - ``max_snapshot_age_in_ms`` + - ``BIGINT`` + - For branch only, the max snapshot age allowed in a branch. Older snapshots + in the branch will be expired. +``` + +(iceberg-metadata-columns)= + +#### Metadata columns + +In addition to the defined columns, the Iceberg connector automatically exposes +path metadata as a hidden column in each table: + +- `$path`: Full file system path name of the file for this row +- `$file_modified_time`: Timestamp of the last modification of the file for + this row + +You can use these columns in your SQL statements like any other column. This can +be selected directly, or used in conditional statements. For example, you can +inspect the file path for each record: + +``` +SELECT *, "$path", "$file_modified_time" +FROM example.web.page_views; +``` + +Retrieve all records that belong to a specific file using `"$path"` filter: + +``` +SELECT * +FROM example.web.page_views +WHERE "$path" = '/usr/iceberg/table/web.page_views/data/file_01.parquet' +``` + +Retrieve all records that belong to a specific file using +`"$file_modified_time"` filter: + +``` +SELECT * +FROM example.web.page_views +WHERE "$file_modified_time" = CAST('2022-07-01 01:02:03.456 UTC' AS TIMESTAMP WIOTH TIMEZONE) +``` + +#### DROP TABLE + +The Iceberg connector supports dropping a table by using the +{doc}`/sql/drop-table` syntax. When the command succeeds, both the data of the +Iceberg table and also the information related to the table in the metastore +service are removed. Dropping tables that have their data/metadata stored in a +different location than the table's corresponding base directory on the object +store is not supported. + +(iceberg-comment)= + +#### COMMENT + +The Iceberg connector supports setting comments on the following objects: + +- tables +- views +- table columns +- materialized view columns + +The `COMMENT` option is supported on both the table and the table columns for +the {doc}`/sql/create-table` operation. + +The `COMMENT` option is supported for adding table columns through the +{doc}`/sql/alter-table` operations. + +The connector supports the command {doc}`COMMENT ` for setting +comments on existing entities. + +(iceberg-tables)= + +#### Partitioned tables + +Iceberg supports partitioning by specifying transforms over the table columns. A +partition is created for each unique tuple value produced by the transforms. +Identity transforms are simply the column name. Other transforms are: + +```{eval-rst} +.. list-table:: Iceberg column transforms + :widths: 40, 60 + :header-rows: 1 + + * - Transform + - Description + * - ``year(ts)`` + - A partition is created for each year. The partition value is the integer + difference in years between ``ts`` and January 1 1970. + * - ``month(ts)`` + - A partition is created for each month of each year. The partition value + is the integer difference in months between ``ts`` and January 1 1970. + * - ``day(ts)`` + - A partition is created for each day of each year. The partition value is + the integer difference in days between ``ts`` and January 1 1970. + * - ``hour(ts)`` + - A partition is created hour of each day. The partition value is a + timestamp with the minutes and seconds set to zero. + * - ``bucket(x, nbuckets)`` + - The data is hashed into the specified number of buckets. The partition + value is an integer hash of ``x``, with a value between 0 and ``nbuckets - + 1`` inclusive. + * - ``truncate(s, nchars)`` + - The partition value is the first ``nchars`` characters of ``s``. +``` + +In this example, the table is partitioned by the month of `order_date`, a hash +of `account_number` (with 10 buckets), and `country`: + +``` +CREATE TABLE example.testdb.customer_orders ( + order_id BIGINT, + order_date DATE, + account_number BIGINT, + customer VARCHAR, + country VARCHAR) +WITH (partitioning = ARRAY['month(order_date)', 'bucket(account_number, 10)', 'country']) +``` + +#### Sorted tables + +The connector supports sorted files as a performance improvement. Data is sorted +during writes within each file based on the specified array of one or more +columns. + +Sorting is particularly beneficial when the sorted columns show a high +cardinality and are used as a filter for selective reads. + +The sort order is configured with the `sorted_by` table property. Specify an +array of one or more columns to use for sorting when creating the table. The +following example configures the `order_date` column of the `orders` table +in the `customers` schema in the `example` catalog: + +``` +CREATE TABLE example.customers.orders ( + order_id BIGINT, + order_date DATE, + account_number BIGINT, + customer VARCHAR, + country VARCHAR) +WITH (sorted_by = ARRAY['order_date']) +``` + +Sorting can be combined with partitioning on the same column. For example: + +``` +CREATE TABLE example.customers.orders ( + order_id BIGINT, + order_date DATE, + account_number BIGINT, + customer VARCHAR, + country VARCHAR) +WITH ( + partitioning = ARRAY['month(order_date)'], + sorted_by = ARRAY['order_date'] +) +``` + +You can disable sorted writing with the session property +`sorted_writing_enabled` set to `false`. + +#### Using snapshots + +Iceberg supports a snapshot model of data, where table snapshots are +identified by a snapshot ID. + +The connector provides a system table exposing snapshot information for every +Iceberg table. Snapshots are identified by `BIGINT` snapshot IDs. For example, +you can find the snapshot IDs for the `customer_orders` table by running the +following query: + +``` +SELECT snapshot_id +FROM example.testdb."customer_orders$snapshots" +ORDER BY committed_at DESC +``` + +(iceberg-time-travel)= + +##### Time travel queries + +The connector offers the ability to query historical data. This allows you to +query the table as it was when a previous snapshot of the table was taken, even +if the data has since been modified or deleted. + +The historical data of the table can be retrieved by specifying the snapshot +identifier corresponding to the version of the table to be retrieved: + +``` +SELECT * +FROM example.testdb.customer_orders FOR VERSION AS OF 8954597067493422955 +``` + +A different approach of retrieving historical data is to specify a point in time +in the past, such as a day or week ago. The latest snapshot of the table taken +before or at the specified timestamp in the query is internally used for +providing the previous state of the table: + +``` +SELECT * +FROM example.testdb.customer_orders FOR TIMESTAMP AS OF TIMESTAMP '2022-03-23 09:59:29.803 Europe/Vienna' +``` + +You can use a date to specify a point a time in the past for using a snapshot of a table in a query. +Assuming that the session time zone is `Europe/Vienna` the following queries are equivalent: + +``` +SELECT * +FROM example.testdb.customer_orders FOR TIMESTAMP AS OF DATE '2022-03-23' +``` + +``` +SELECT * +FROM example.testdb.customer_orders FOR TIMESTAMP AS OF TIMESTAMP '2022-03-23 00:00:00' +``` + +``` +SELECT * +FROM example.testdb.customer_orders FOR TIMESTAMP AS OF TIMESTAMP '2022-03-23 00:00:00.000 Europe/Vienna' +``` + +Iceberg supports named references of snapshots via branches and tags. +Time travel can be performed to branches and tags in the table. + +``` +SELECT * +FROM example.testdb.customer_orders FOR VERSION AS OF 'historical-tag' + +SELECT * +FROM example.testdb.customer_orders FOR VERSION AS OF 'test-branch' +``` + +##### Rolling back to a previous snapshot + +Use the `$snapshots` metadata table to determine the latest snapshot ID of the +table like in the following query: + +``` +SELECT snapshot_id +FROM example.testdb."customer_orders$snapshots" +ORDER BY committed_at DESC LIMIT 1 +``` + +The procedure `system.rollback_to_snapshot` allows the caller to roll back the +state of the table to a previous snapshot id: + +``` +CALL example.system.rollback_to_snapshot('testdb', 'customer_orders', 8954597067493422955) +``` + +#### `NOT NULL` column constraint + +The Iceberg connector supports setting `NOT NULL` constraints on the table +columns. + +The `NOT NULL` constraint can be set on the columns, while creating tables by +using the {doc}`CREATE TABLE ` syntax: + +``` +CREATE TABLE example_table ( + year INTEGER NOT NULL, + name VARCHAR NOT NULL, + age INTEGER, + address VARCHAR +); +``` + +When trying to insert/update data in the table, the query fails if trying to set +`NULL` value on a column having the `NOT NULL` constraint. + +### View management + +Trino allows reading from Iceberg materialized views. + +(iceberg-materialized-views)= + +#### Materialized views + +The Iceberg connector supports {ref}`sql-materialized-view-management`. In the +underlying system, each materialized view consists of a view definition and an +Iceberg storage table. The storage table name is stored as a materialized view +property. The data is stored in that storage table. + +You can use the {ref}`iceberg-table-properties` to control the created storage +table and therefore the layout and performance. For example, you can use the +following clause with {doc}`/sql/create-materialized-view` to use the ORC format +for the data files and partition the storage per day using the column +`_date`: + +``` +WITH ( format = 'ORC', partitioning = ARRAY['event_date'] ) +``` + +By default, the storage table is created in the same schema as the materialized +view definition. The `iceberg.materialized-views.storage-schema` catalog +configuration property or `storage_schema` materialized view property can be +used to specify the schema where the storage table is created. + +Creating a materialized view does not automatically populate it with data. You +must run {doc}`/sql/refresh-materialized-view` to populate data in the +materialized view. + +Updating the data in the materialized view with `REFRESH MATERIALIZED VIEW` +deletes the data from the storage table, and inserts the data that is the result +of executing the materialized view query into the existing table. Data is +replaced atomically, so users can continue to query the materialized view while +it is being refreshed. Refreshing a materialized view also stores the +snapshot-ids of all Iceberg tables that are part of the materialized view's +query in the materialized view metadata. When the materialized view is queried, +the snapshot-ids are used to check if the data in the storage table is up to +date. If the data is outdated, the materialized view behaves like a normal view, +and the data is queried directly from the base tables. Detecting outdated data +is possible only when the materialized view uses Iceberg tables only, or when it +uses a mix of Iceberg and non-Iceberg tables but some Iceberg tables are outdated. +When the materialized view is based on non-Iceberg tables, querying it can +return outdated data, since the connector has no information whether the +underlying non-Iceberg tables have changed. + +Dropping a materialized view with {doc}`/sql/drop-materialized-view` removes +the definition and the storage table. + +(iceberg-fte-support)= + +## Fault-tolerant execution support + +The connector supports {doc}`/admin/fault-tolerant-execution` of query +processing. Read and write operations are both supported with any retry policy. + +## Performance + +The connector includes a number of performance improvements, detailed in the +following sections. + +### Table statistics + +The Iceberg connector can collect column statistics using {doc}`/sql/analyze` +statement. This can be disabled using `iceberg.extended-statistics.enabled` +catalog configuration property, or the corresponding +`extended_statistics_enabled` session property. + +(iceberg-analyze)= + +#### Updating table statistics + +If your queries are complex and include joining large data sets, running +{doc}`/sql/analyze` on tables may improve query performance by collecting +statistical information about the data: + +``` +ANALYZE table_name +``` + +This query collects statistics for all columns. + +On wide tables, collecting statistics for all columns can be expensive. It is +also typically unnecessary - statistics are only useful on specific columns, +like join keys, predicates, or grouping keys. You can specify a subset of +columns to analyzed with the optional `columns` property: + +``` +ANALYZE table_name WITH (columns = ARRAY['col_1', 'col_2']) +``` + +This query collects statistics for columns `col_1` and `col_2`. + +Note that if statistics were previously collected for all columns, they must be +dropped using the {ref}`drop_extended_stats ` command +before re-analyzing. + +(iceberg-table-redirection)= + +### Table redirection + +```{include} table-redirection.fragment +``` + +The connector supports redirection from Iceberg tables to Hive tables with the +`iceberg.hive-catalog-name` catalog configuration property. diff --git a/430/_sources/connector/ignite.md.txt b/430/_sources/connector/ignite.md.txt new file mode 100644 index 000000000..77793bfee --- /dev/null +++ b/430/_sources/connector/ignite.md.txt @@ -0,0 +1,216 @@ +--- +myst: + substitutions: + default_domain_compaction_threshold: '`1000`' +--- + +# Ignite connector + +```{raw} html + +``` + +The Ignite connector allows querying an [Apache Ignite](https://ignite.apache.org/) +database from Trino. + +## Requirements + +To connect to a Ignite server, you need: + +- Ignite version 2.9.0 or latter +- Network access from the Trino coordinator and workers to the Ignite + server. Port 10800 is the default port. +- Specify `--add-opens=java.base/java.nio=ALL-UNNAMED` in the `jvm.config` when starting the Trino server. + +## Configuration + +The Ignite connector expose `public` schema by default. + +The connector can query a Ignite instance. Create a catalog properties file +that specifies the Ignite connector by setting the `connector.name` to +`ignite`. + +For example, to access an instance as `example`, create the file +`etc/catalog/example.properties`. Replace the connection properties as +appropriate for your setup: + +```text +connector.name=ignite +connection-url=jdbc:ignite:thin://host1:10800/ +connection-user=exampleuser +connection-password=examplepassword +``` + +The `connection-url` defines the connection information and parameters to pass +to the Ignite JDBC driver. The parameters for the URL are available in the +[Ignite JDBC driver documentation](https://ignite.apache.org/docs/latest/SQL/JDBC/jdbc-driver). +Some parameters can have adverse effects on the connector behavior or not work +with the connector. + +The `connection-user` and `connection-password` are typically required and +determine the user credentials for the connection, often a service user. You can +use {doc}`secrets ` to avoid actual values in the catalog +properties files. + +### Multiple Ignite servers + +If you have multiple Ignite servers you need to configure one +catalog for each server. To add another catalog: + +- Add another properties file to `etc/catalog` +- Save it with a different name that ends in `.properties` + +For example, if you name the property file `sales.properties`, Trino uses the +configured connector to create a catalog named `sales`. + +```{include} jdbc-common-configurations.fragment +``` + +```{include} query-comment-format.fragment +``` + +```{include} jdbc-domain-compaction-threshold.fragment +``` + +```{include} jdbc-procedures.fragment +``` + +```{include} jdbc-case-insensitive-matching.fragment +``` + +```{include} non-transactional-insert.fragment +``` + +## Table properties + +Table property usage example: + +``` +CREATE TABLE public.person ( + id BIGINT NOT NULL, + birthday DATE NOT NULL, + name VARCHAR(26), + age BIGINT, + logdate DATE +) +WITH ( + primary_key = ARRAY['id', 'birthday'] +); +``` + +The following are supported Ignite table properties from [https://ignite.apache.org/docs/latest/sql-reference/ddl](https://ignite.apache.org/docs/latest/sql-reference/ddl) + +```{eval-rst} +.. list-table:: + :widths: 30, 10, 100 + :header-rows: 1 + + * - Property name + - Required + - Description + * - ``primary_key`` + - No + - ``The primary key of the table, can chose multi columns as the table primary key. Table at least contains one column not in primary key.`` +``` + +### `primary_key` + +This is a list of columns to be used as the table's primary key. If not specified, a `VARCHAR` primary key column named `DUMMY_ID` is generated, +the value is derived from the value generated by the `UUID` function in Ignite. + +(ignite-type-mapping)= + +## Type mapping + +The following are supported Ignite SQL data types from [https://ignite.apache.org/docs/latest/sql-reference/data-types](https://ignite.apache.org/docs/latest/sql-reference/data-types) + +````{eval-rst} +.. list-table:: + :widths: 30, 30, 20 + :header-rows: 1 + + * - Ignite SQL data type name + - Map to Trino type + - Possible values + * - ``BOOLEAN`` + - ``BOOLEAN`` + - ``TRUE`` and ``FALSE`` + * - ``BIGINT`` + - ``BIGINT`` + - ``-9223372036854775808``, ``9223372036854775807``, etc. + * - ``DECIMAL`` + - ``DECIMAL`` + - Data type with fixed precision and scale + * - ``DOUBLE`` + - ``DOUBLE`` + - ``3.14``, ``-10.24``, etc. + * - ``INT`` + - ``INT`` + - ``-2147483648``, ``2147483647``, etc. + * - ``REAL`` + - ``REAL``` + - ``3.14``, ``-10.24``, etc. + * - ``SMALLINT`` + - ``SMALLINT`` + - ``-32768``, ``32767``, etc. + * - ``TINYINT`` + - ``TINYINT`` + - ``-128``, ``127``, etc. + * - ``CHAR`` + - ``CHAR`` + - ``hello``, ``Trino``, etc. + * - ``VARCHAR`` + - ``VARCHAR`` + - ``hello``, ``Trino``, etc. + * - ``DATE`` + - ``DATE`` + - ``1972-01-01``, ``2021-07-15``, etc. + * - ``BINARY`` + - ``VARBINARY`` + - Represents a byte array. +```` + +(ignite-sql-support)= + +## SQL support + +The connector provides read access and write access to data and metadata in +Ignite. In addition to the {ref}`globally available +` and {ref}`read operation ` +statements, the connector supports the following features: + +- {doc}`/sql/insert` +- {doc}`/sql/update` +- {doc}`/sql/delete` +- {doc}`/sql/create-table` +- {doc}`/sql/create-table-as` +- {doc}`/sql/drop-table` +- {doc}`/sql/alter-table` + +```{include} sql-update-limitation.fragment +``` + +```{include} alter-table-limitation.fragment +``` + +(ignite-pushdown)= + +### Pushdown + +The connector supports pushdown for a number of operations: + +- {ref}`join-pushdown` +- {ref}`limit-pushdown` +- {ref}`topn-pushdown` + +{ref}`Aggregate pushdown ` for the following functions: + +- {func}`avg` +- {func}`count` +- {func}`max` +- {func}`min` +- {func}`sum` + + +```{include} no-pushdown-text-type.fragment +``` diff --git a/430/_sources/connector/jmx.md.txt b/430/_sources/connector/jmx.md.txt new file mode 100644 index 000000000..ab6cf6e83 --- /dev/null +++ b/430/_sources/connector/jmx.md.txt @@ -0,0 +1,137 @@ +# JMX connector + +The JMX connector provides the ability to query Java Management Extensions (JMX) +information from all +nodes in a Trino cluster. This is very useful for monitoring or debugging. +JMX provides information about the Java +Virtual Machine and all of the software running inside it. Trino itself +is heavily instrumented via JMX. + +This connector can be configured so that chosen JMX information is +periodically dumped and stored in memory for later access. + +## Configuration + +To configure the JMX connector, create a catalog properties file +`etc/catalog/example.properties` with the following contents: + +```text +connector.name=jmx +``` + +To enable periodical dumps, define the following properties: + +```text +connector.name=jmx +jmx.dump-tables=java.lang:type=Runtime,trino.execution.scheduler:name=NodeScheduler +jmx.dump-period=10s +jmx.max-entries=86400 +``` + +`dump-tables` is a comma separated list of Managed Beans (MBean). It specifies +which MBeans are sampled and stored in memory every `dump-period`. You can +configure the maximum number of history entries with `max-entries` and it +defaults to `86400`. The time between dumps can be configured using +`dump-period` and it defaults to `10s`. + +Commas in MBean names must be escaped using double backslashes (`\\`) in the +following manner: + +```text +connector.name=jmx +jmx.dump-tables=trino.memory:name=general\\,type=memorypool,trino.memory:name=reserved\\,type=memorypool +``` + +Double backslashes are required because a single backslash (`\`) is used to +split the value across multiple lines in the following manner: + +```text +connector.name=jmx +jmx.dump-tables=trino.memory:name=general\\,type=memorypool,\ + trino.memory:name=reserved\\,type=memorypool +``` + +## Querying JMX + +The JMX connector provides two schemas. + +The first one is `current` that contains every MBean from every node in the Trino +cluster. You can see all of the available MBeans by running `SHOW TABLES`: + +``` +SHOW TABLES FROM example.current; +``` + +MBean names map to non-standard table names, and must be quoted with +double quotes when referencing them in a query. For example, the +following query shows the JVM version of every node: + +``` +SELECT node, vmname, vmversion +FROM example.current."java.lang:type=runtime"; +``` + +```text + node | vmname | vmversion +--------------------------------------+-----------------------------------+----------- + ddc4df17-0b8e-4843-bb14-1b8af1a7451a | Java HotSpot(TM) 64-Bit Server VM | 24.60-b09 +(1 row) +``` + +The following query shows the open and maximum file descriptor counts +for each node: + +``` +SELECT openfiledescriptorcount, maxfiledescriptorcount +FROM example.current."java.lang:type=operatingsystem"; +``` + +```text + openfiledescriptorcount | maxfiledescriptorcount +-------------------------+------------------------ + 329 | 10240 +(1 row) +``` + +The wildcard character `*` may be used with table names in the `current` schema. +This allows matching several MBean objects within a single query. The following query +returns information from the different Trino memory pools on each node: + +``` +SELECT freebytes, node, object_name +FROM example.current."trino.memory:*type=memorypool*"; +``` + +```text + freebytes | node | object_name +------------+---------+---------------------------------------------------------- + 214748364 | example | trino.memory:type=MemoryPool,name=reserved + 1073741825 | example | trino.memory:type=MemoryPool,name=general + 858993459 | example | trino.memory:type=MemoryPool,name=system +(3 rows) +``` + +The `history` schema contains the list of tables configured in the connector properties file. +The tables have the same columns as those in the current schema, but with an additional +timestamp column that stores the time at which the snapshot was taken: + +``` +SELECT "timestamp", "uptime" FROM example.history."java.lang:type=runtime"; +``` + +```text + timestamp | uptime +-------------------------+-------- + 2016-01-28 10:18:50.000 | 11420 + 2016-01-28 10:19:00.000 | 21422 + 2016-01-28 10:19:10.000 | 31412 +(3 rows) +``` + +(jmx-sql-support)= + +## SQL support + +The connector provides {ref}`globally available ` and +{ref}`read operation ` statements to access JMX information +on your Trino nodes. diff --git a/430/_sources/connector/kafka-tutorial.md.txt b/430/_sources/connector/kafka-tutorial.md.txt new file mode 100644 index 000000000..918f24b1b --- /dev/null +++ b/430/_sources/connector/kafka-tutorial.md.txt @@ -0,0 +1,591 @@ +# Kafka connector tutorial + +## Introduction + +The {doc}`kafka` for Trino allows access to live topic data from +Apache Kafka using Trino. This tutorial shows how to set up topics, and +how to create the topic description files that back Trino tables. + +## Installation + +This tutorial assumes familiarity with Trino and a working local Trino +installation (see {doc}`/installation/deployment`). It focuses on +setting up Apache Kafka and integrating it with Trino. + +### Step 1: Install Apache Kafka + +Download and extract [Apache Kafka](https://kafka.apache.org/). + +:::{note} +This tutorial was tested with Apache Kafka 0.8.1. +It should work with any 0.8.x version of Apache Kafka. +::: + +Start ZooKeeper and the Kafka server: + +```text +$ bin/zookeeper-server-start.sh config/zookeeper.properties +[2013-04-22 15:01:37,495] INFO Reading configuration from: config/zookeeper.properties (org.apache.zookeeper.server.quorum.QuorumPeerConfig) +... +``` + +```text +$ bin/kafka-server-start.sh config/server.properties +[2013-04-22 15:01:47,028] INFO Verifying properties (kafka.utils.VerifiableProperties) +[2013-04-22 15:01:47,051] INFO Property socket.send.buffer.bytes is overridden to 1048576 (kafka.utils.VerifiableProperties) +... +``` + +This starts Zookeeper on port `2181` and Kafka on port `9092`. + +### Step 2: Load data + +Download the tpch-kafka loader from Maven Central: + +```text +$ curl -o kafka-tpch https://repo1.maven.org/maven2/de/softwareforge/kafka_tpch_0811/1.0/kafka_tpch_0811-1.0.sh +$ chmod 755 kafka-tpch +``` + +Now run the `kafka-tpch` program to preload a number of topics with tpch data: + +```text +$ ./kafka-tpch load --brokers localhost:9092 --prefix tpch. --tpch-type tiny +2014-07-28T17:17:07.594-0700 INFO main io.airlift.log.Logging Logging to stderr +2014-07-28T17:17:07.623-0700 INFO main de.softwareforge.kafka.LoadCommand Processing tables: [customer, orders, lineitem, part, partsupp, supplier, nation, region] +2014-07-28T17:17:07.981-0700 INFO pool-1-thread-1 de.softwareforge.kafka.LoadCommand Loading table 'customer' into topic 'tpch.customer'... +2014-07-28T17:17:07.981-0700 INFO pool-1-thread-2 de.softwareforge.kafka.LoadCommand Loading table 'orders' into topic 'tpch.orders'... +2014-07-28T17:17:07.981-0700 INFO pool-1-thread-3 de.softwareforge.kafka.LoadCommand Loading table 'lineitem' into topic 'tpch.lineitem'... +2014-07-28T17:17:07.982-0700 INFO pool-1-thread-4 de.softwareforge.kafka.LoadCommand Loading table 'part' into topic 'tpch.part'... +2014-07-28T17:17:07.982-0700 INFO pool-1-thread-5 de.softwareforge.kafka.LoadCommand Loading table 'partsupp' into topic 'tpch.partsupp'... +2014-07-28T17:17:07.982-0700 INFO pool-1-thread-6 de.softwareforge.kafka.LoadCommand Loading table 'supplier' into topic 'tpch.supplier'... +2014-07-28T17:17:07.982-0700 INFO pool-1-thread-7 de.softwareforge.kafka.LoadCommand Loading table 'nation' into topic 'tpch.nation'... +2014-07-28T17:17:07.982-0700 INFO pool-1-thread-8 de.softwareforge.kafka.LoadCommand Loading table 'region' into topic 'tpch.region'... +2014-07-28T17:17:10.612-0700 ERROR pool-1-thread-8 kafka.producer.async.DefaultEventHandler Failed to collate messages by topic, partition due to: Failed to fetch topic metadata for topic: tpch.region +2014-07-28T17:17:10.781-0700 INFO pool-1-thread-8 de.softwareforge.kafka.LoadCommand Generated 5 rows for table 'region'. +2014-07-28T17:17:10.797-0700 ERROR pool-1-thread-3 kafka.producer.async.DefaultEventHandler Failed to collate messages by topic, partition due to: Failed to fetch topic metadata for topic: tpch.lineitem +2014-07-28T17:17:10.932-0700 ERROR pool-1-thread-1 kafka.producer.async.DefaultEventHandler Failed to collate messages by topic, partition due to: Failed to fetch topic metadata for topic: tpch.customer +2014-07-28T17:17:11.068-0700 ERROR pool-1-thread-2 kafka.producer.async.DefaultEventHandler Failed to collate messages by topic, partition due to: Failed to fetch topic metadata for topic: tpch.orders +2014-07-28T17:17:11.200-0700 ERROR pool-1-thread-6 kafka.producer.async.DefaultEventHandler Failed to collate messages by topic, partition due to: Failed to fetch topic metadata for topic: tpch.supplier +2014-07-28T17:17:11.319-0700 INFO pool-1-thread-6 de.softwareforge.kafka.LoadCommand Generated 100 rows for table 'supplier'. +2014-07-28T17:17:11.333-0700 ERROR pool-1-thread-4 kafka.producer.async.DefaultEventHandler Failed to collate messages by topic, partition due to: Failed to fetch topic metadata for topic: tpch.part +2014-07-28T17:17:11.466-0700 ERROR pool-1-thread-5 kafka.producer.async.DefaultEventHandler Failed to collate messages by topic, partition due to: Failed to fetch topic metadata for topic: tpch.partsupp +2014-07-28T17:17:11.597-0700 ERROR pool-1-thread-7 kafka.producer.async.DefaultEventHandler Failed to collate messages by topic, partition due to: Failed to fetch topic metadata for topic: tpch.nation +2014-07-28T17:17:11.706-0700 INFO pool-1-thread-7 de.softwareforge.kafka.LoadCommand Generated 25 rows for table 'nation'. +2014-07-28T17:17:12.180-0700 INFO pool-1-thread-1 de.softwareforge.kafka.LoadCommand Generated 1500 rows for table 'customer'. +2014-07-28T17:17:12.251-0700 INFO pool-1-thread-4 de.softwareforge.kafka.LoadCommand Generated 2000 rows for table 'part'. +2014-07-28T17:17:12.905-0700 INFO pool-1-thread-2 de.softwareforge.kafka.LoadCommand Generated 15000 rows for table 'orders'. +2014-07-28T17:17:12.919-0700 INFO pool-1-thread-5 de.softwareforge.kafka.LoadCommand Generated 8000 rows for table 'partsupp'. +2014-07-28T17:17:13.877-0700 INFO pool-1-thread-3 de.softwareforge.kafka.LoadCommand Generated 60175 rows for table 'lineitem'. +``` + +Kafka now has a number of topics that are preloaded with data to query. + +### Step 3: Make the Kafka topics known to Trino + +In your Trino installation, add a catalog properties file +`etc/catalog/kafka.properties` for the Kafka connector. +This file lists the Kafka nodes and topics: + +```text +connector.name=kafka +kafka.nodes=localhost:9092 +kafka.table-names=tpch.customer,tpch.orders,tpch.lineitem,tpch.part,tpch.partsupp,tpch.supplier,tpch.nation,tpch.region +kafka.hide-internal-columns=false +``` + +Now start Trino: + +```text +$ bin/launcher start +``` + +Because the Kafka tables all have the `tpch.` prefix in the configuration, +the tables are in the `tpch` schema. The connector is mounted into the +`kafka` catalog, because the properties file is named `kafka.properties`. + +Start the {doc}`Trino CLI `: + +```text +$ ./trino --catalog kafka --schema tpch +``` + +List the tables to verify that things are working: + +```text +trino:tpch> SHOW TABLES; + Table +---------- + customer + lineitem + nation + orders + part + partsupp + region + supplier +(8 rows) +``` + +### Step 4: Basic data querying + +Kafka data is unstructured, and it has no metadata to describe the format of +the messages. Without further configuration, the Kafka connector can access +the data, and map it in raw form. However there are no actual columns besides the +built-in ones: + +```text +trino:tpch> DESCRIBE customer; + Column | Type | Extra | Comment +-------------------+------------+-------+--------------------------------------------- + _partition_id | bigint | | Partition Id + _partition_offset | bigint | | Offset for the message within the partition + _key | varchar | | Key text + _key_corrupt | boolean | | Key data is corrupt + _key_length | bigint | | Total number of key bytes + _message | varchar | | Message text + _message_corrupt | boolean | | Message data is corrupt + _message_length | bigint | | Total number of message bytes + _timestamp | timestamp | | Message timestamp +(11 rows) + +trino:tpch> SELECT count(*) FROM customer; + _col0 +------- + 1500 + +trino:tpch> SELECT _message FROM customer LIMIT 5; + _message +-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + {"rowNumber":1,"customerKey":1,"name":"Customer#000000001","address":"IVhzIApeRb ot,c,E","nationKey":15,"phone":"25-989-741-2988","accountBalance":711.56,"marketSegment":"BUILDING","comment":"to the even, regular platelets. regular, ironic epitaphs nag e"} + {"rowNumber":3,"customerKey":3,"name":"Customer#000000003","address":"MG9kdTD2WBHm","nationKey":1,"phone":"11-719-748-3364","accountBalance":7498.12,"marketSegment":"AUTOMOBILE","comment":" deposits eat slyly ironic, even instructions. express foxes detect slyly. blithel + {"rowNumber":5,"customerKey":5,"name":"Customer#000000005","address":"KvpyuHCplrB84WgAiGV6sYpZq7Tj","nationKey":3,"phone":"13-750-942-6364","accountBalance":794.47,"marketSegment":"HOUSEHOLD","comment":"n accounts will have to unwind. foxes cajole accor"} + {"rowNumber":7,"customerKey":7,"name":"Customer#000000007","address":"TcGe5gaZNgVePxU5kRrvXBfkasDTea","nationKey":18,"phone":"28-190-982-9759","accountBalance":9561.95,"marketSegment":"AUTOMOBILE","comment":"ainst the ironic, express theodolites. express, even pinto bean + {"rowNumber":9,"customerKey":9,"name":"Customer#000000009","address":"xKiAFTjUsCuxfeleNqefumTrjS","nationKey":8,"phone":"18-338-906-3675","accountBalance":8324.07,"marketSegment":"FURNITURE","comment":"r theodolites according to the requests wake thinly excuses: pending +(5 rows) + +trino:tpch> SELECT sum(cast(json_extract_scalar(_message, '$.accountBalance') AS DOUBLE)) FROM customer LIMIT 10; + _col0 +------------ + 6681865.59 +(1 row) +``` + +The data from Kafka can be queried using Trino, but it is not yet in +actual table shape. The raw data is available through the `_message` and +`_key` columns, but it is not decoded into columns. As the sample data is +in JSON format, the {doc}`/functions/json` built into Trino can be used +to slice the data. + +### Step 5: Add a topic description file + +The Kafka connector supports topic description files to turn raw data into +table format. These files are located in the `etc/kafka` folder in the +Trino installation and must end with `.json`. It is recommended that +the file name matches the table name, but this is not necessary. + +Add the following file as `etc/kafka/tpch.customer.json` and restart Trino: + +```json +{ + "tableName": "customer", + "schemaName": "tpch", + "topicName": "tpch.customer", + "key": { + "dataFormat": "raw", + "fields": [ + { + "name": "kafka_key", + "dataFormat": "LONG", + "type": "BIGINT", + "hidden": "false" + } + ] + } +} +``` + +The customer table now has an additional column: `kafka_key`. + +```text +trino:tpch> DESCRIBE customer; + Column | Type | Extra | Comment +-------------------+------------+-------+--------------------------------------------- + kafka_key | bigint | | + _partition_id | bigint | | Partition Id + _partition_offset | bigint | | Offset for the message within the partition + _key | varchar | | Key text + _key_corrupt | boolean | | Key data is corrupt + _key_length | bigint | | Total number of key bytes + _message | varchar | | Message text + _message_corrupt | boolean | | Message data is corrupt + _message_length | bigint | | Total number of message bytes + _timestamp | timestamp | | Message timestamp +(12 rows) + +trino:tpch> SELECT kafka_key FROM customer ORDER BY kafka_key LIMIT 10; + kafka_key +----------- + 0 + 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 +(10 rows) +``` + +The topic definition file maps the internal Kafka key, which is a raw long +in eight bytes, onto a Trino `BIGINT` column. + +### Step 6: Map all the values from the topic message onto columns + +Update the `etc/kafka/tpch.customer.json` file to add fields for the +message, and restart Trino. As the fields in the message are JSON, it uses +the `JSON` data format. This is an example, where different data formats +are used for the key and the message. + +```json +{ + "tableName": "customer", + "schemaName": "tpch", + "topicName": "tpch.customer", + "key": { + "dataFormat": "raw", + "fields": [ + { + "name": "kafka_key", + "dataFormat": "LONG", + "type": "BIGINT", + "hidden": "false" + } + ] + }, + "message": { + "dataFormat": "json", + "fields": [ + { + "name": "row_number", + "mapping": "rowNumber", + "type": "BIGINT" + }, + { + "name": "customer_key", + "mapping": "customerKey", + "type": "BIGINT" + }, + { + "name": "name", + "mapping": "name", + "type": "VARCHAR" + }, + { + "name": "address", + "mapping": "address", + "type": "VARCHAR" + }, + { + "name": "nation_key", + "mapping": "nationKey", + "type": "BIGINT" + }, + { + "name": "phone", + "mapping": "phone", + "type": "VARCHAR" + }, + { + "name": "account_balance", + "mapping": "accountBalance", + "type": "DOUBLE" + }, + { + "name": "market_segment", + "mapping": "marketSegment", + "type": "VARCHAR" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR" + } + ] + } +} +``` + +Now for all the fields in the JSON of the message, columns are defined and +the sum query from earlier can operate on the `account_balance` column directly: + +```text +trino:tpch> DESCRIBE customer; + Column | Type | Extra | Comment +-------------------+------------+-------+--------------------------------------------- + kafka_key | bigint | | + row_number | bigint | | + customer_key | bigint | | + name | varchar | | + address | varchar | | + nation_key | bigint | | + phone | varchar | | + account_balance | double | | + market_segment | varchar | | + comment | varchar | | + _partition_id | bigint | | Partition Id + _partition_offset | bigint | | Offset for the message within the partition + _key | varchar | | Key text + _key_corrupt | boolean | | Key data is corrupt + _key_length | bigint | | Total number of key bytes + _message | varchar | | Message text + _message_corrupt | boolean | | Message data is corrupt + _message_length | bigint | | Total number of message bytes + _timestamp | timestamp | | Message timestamp +(21 rows) + +trino:tpch> SELECT * FROM customer LIMIT 5; + kafka_key | row_number | customer_key | name | address | nation_key | phone | account_balance | market_segment | comment +-----------+------------+--------------+--------------------+---------------------------------------+------------+-----------------+-----------------+----------------+--------------------------------------------------------------------------------------------------------- + 1 | 2 | 2 | Customer#000000002 | XSTf4,NCwDVaWNe6tEgvwfmRchLXak | 13 | 23-768-687-3665 | 121.65 | AUTOMOBILE | l accounts. blithely ironic theodolites integrate boldly: caref + 3 | 4 | 4 | Customer#000000004 | XxVSJsLAGtn | 4 | 14-128-190-5944 | 2866.83 | MACHINERY | requests. final, regular ideas sleep final accou + 5 | 6 | 6 | Customer#000000006 | sKZz0CsnMD7mp4Xd0YrBvx,LREYKUWAh yVn | 20 | 30-114-968-4951 | 7638.57 | AUTOMOBILE | tions. even deposits boost according to the slyly bold packages. final accounts cajole requests. furious + 7 | 8 | 8 | Customer#000000008 | I0B10bB0AymmC, 0PrRYBCP1yGJ8xcBPmWhl5 | 17 | 27-147-574-9335 | 6819.74 | BUILDING | among the slyly regular theodolites kindle blithely courts. carefully even theodolites haggle slyly alon + 9 | 10 | 10 | Customer#000000010 | 6LrEaV6KR6PLVcgl2ArL Q3rqzLzcT1 v2 | 5 | 15-741-346-9870 | 2753.54 | HOUSEHOLD | es regular deposits haggle. fur +(5 rows) + +trino:tpch> SELECT sum(account_balance) FROM customer LIMIT 10; + _col0 +------------ + 6681865.59 +(1 row) +``` + +Now all the fields from the `customer` topic messages are available as +Trino table columns. + +### Step 7: Use live data + +Trino can query live data in Kafka as it arrives. To simulate a live feed +of data, this tutorial sets up a feed of live tweets into Kafka. + +#### Setup a live Twitter feed + +- Download the twistr tool + +```text +$ curl -o twistr https://repo1.maven.org/maven2/de/softwareforge/twistr_kafka_0811/1.2/twistr_kafka_0811-1.2.sh +$ chmod 755 twistr +``` + +- Create a developer account at and set up an + access and consumer token. +- Create a `twistr.properties` file and put the access and consumer key + and secrets into it: + +```text +twistr.access-token-key=... +twistr.access-token-secret=... +twistr.consumer-key=... +twistr.consumer-secret=... +twistr.kafka.brokers=localhost:9092 +``` + +#### Create a tweets table on Trino + +Add the tweets table to the `etc/catalog/kafka.properties` file: + +```text +connector.name=kafka +kafka.nodes=localhost:9092 +kafka.table-names=tpch.customer,tpch.orders,tpch.lineitem,tpch.part,tpch.partsupp,tpch.supplier,tpch.nation,tpch.region,tweets +kafka.hide-internal-columns=false +``` + +Add a topic definition file for the Twitter feed as `etc/kafka/tweets.json`: + +```json +{ + "tableName": "tweets", + "topicName": "twitter_feed", + "dataFormat": "json", + "key": { + "dataFormat": "raw", + "fields": [ + { + "name": "kafka_key", + "dataFormat": "LONG", + "type": "BIGINT", + "hidden": "false" + } + ] + }, + "message": { + "dataFormat":"json", + "fields": [ + { + "name": "text", + "mapping": "text", + "type": "VARCHAR" + }, + { + "name": "user_name", + "mapping": "user/screen_name", + "type": "VARCHAR" + }, + { + "name": "lang", + "mapping": "lang", + "type": "VARCHAR" + }, + { + "name": "created_at", + "mapping": "created_at", + "type": "TIMESTAMP", + "dataFormat": "rfc2822" + }, + { + "name": "favorite_count", + "mapping": "favorite_count", + "type": "BIGINT" + }, + { + "name": "retweet_count", + "mapping": "retweet_count", + "type": "BIGINT" + }, + { + "name": "favorited", + "mapping": "favorited", + "type": "BOOLEAN" + }, + { + "name": "id", + "mapping": "id_str", + "type": "VARCHAR" + }, + { + "name": "in_reply_to_screen_name", + "mapping": "in_reply_to_screen_name", + "type": "VARCHAR" + }, + { + "name": "place_name", + "mapping": "place/full_name", + "type": "VARCHAR" + } + ] + } +} +``` + +As this table does not have an explicit schema name, it is placed +into the `default` schema. + +#### Feed live data + +Start the twistr tool: + +```text +$ java -Dness.config.location=file:$(pwd) -Dness.config=twistr -jar ./twistr +``` + +`twistr` connects to the Twitter API and feeds the "sample tweet" feed +into a Kafka topic called `twitter_feed`. + +Now run queries against live data: + +```text +$ ./trino --catalog kafka --schema default + +trino:default> SELECT count(*) FROM tweets; + _col0 +------- + 4467 +(1 row) + +trino:default> SELECT count(*) FROM tweets; + _col0 +------- + 4517 +(1 row) + +trino:default> SELECT count(*) FROM tweets; + _col0 +------- + 4572 +(1 row) + +trino:default> SELECT kafka_key, user_name, lang, created_at FROM tweets LIMIT 10; + kafka_key | user_name | lang | created_at +--------------------+-----------------+------+------------------------- + 494227746231685121 | burncaniff | en | 2014-07-29 14:07:31.000 + 494227746214535169 | gu8tn | ja | 2014-07-29 14:07:31.000 + 494227746219126785 | pequitamedicen | es | 2014-07-29 14:07:31.000 + 494227746201931777 | josnyS | ht | 2014-07-29 14:07:31.000 + 494227746219110401 | Cafe510 | en | 2014-07-29 14:07:31.000 + 494227746210332673 | Da_JuanAnd_Only | en | 2014-07-29 14:07:31.000 + 494227746193956865 | Smile_Kidrauhl6 | pt | 2014-07-29 14:07:31.000 + 494227750426017793 | CashforeverCD | en | 2014-07-29 14:07:32.000 + 494227750396653569 | FilmArsivimiz | tr | 2014-07-29 14:07:32.000 + 494227750388256769 | jmolas | es | 2014-07-29 14:07:32.000 +(10 rows) +``` + +There is now a live feed into Kafka, which can be queried using Trino. + +### Epilogue: Time stamps + +The tweets feed, that was set up in the last step, contains a timestamp in +RFC 2822 format as `created_at` attribute in each tweet. + +```text +trino:default> SELECT DISTINCT json_extract_scalar(_message, '$.created_at')) AS raw_date + -> FROM tweets LIMIT 5; + raw_date +-------------------------------- + Tue Jul 29 21:07:31 +0000 2014 + Tue Jul 29 21:07:32 +0000 2014 + Tue Jul 29 21:07:33 +0000 2014 + Tue Jul 29 21:07:34 +0000 2014 + Tue Jul 29 21:07:35 +0000 2014 +(5 rows) +``` + +The topic definition file for the tweets table contains a mapping onto a +timestamp using the `rfc2822` converter: + +```text +... +{ + "name": "created_at", + "mapping": "created_at", + "type": "TIMESTAMP", + "dataFormat": "rfc2822" +}, +... +``` + +This allows the raw data to be mapped onto a Trino TIMESTAMP column: + +```text +trino:default> SELECT created_at, raw_date FROM ( + -> SELECT created_at, json_extract_scalar(_message, '$.created_at') AS raw_date + -> FROM tweets) + -> GROUP BY 1, 2 LIMIT 5; + created_at | raw_date +-------------------------+-------------------------------- + 2014-07-29 14:07:20.000 | Tue Jul 29 21:07:20 +0000 2014 + 2014-07-29 14:07:21.000 | Tue Jul 29 21:07:21 +0000 2014 + 2014-07-29 14:07:22.000 | Tue Jul 29 21:07:22 +0000 2014 + 2014-07-29 14:07:23.000 | Tue Jul 29 21:07:23 +0000 2014 + 2014-07-29 14:07:24.000 | Tue Jul 29 21:07:24 +0000 2014 +(5 rows) +``` + +The Kafka connector contains converters for ISO 8601, RFC 2822 text +formats and for number-based timestamps using seconds or miilliseconds +since the epoch. There is also a generic, text-based formatter, which uses +Joda-Time format strings to parse text columns. diff --git a/430/_sources/connector/kafka.md.txt b/430/_sources/connector/kafka.md.txt new file mode 100644 index 000000000..051063df3 --- /dev/null +++ b/430/_sources/connector/kafka.md.txt @@ -0,0 +1,1448 @@ +# Kafka connector + +```{raw} html + +``` + +```{toctree} +:hidden: true +:maxdepth: 1 + +Tutorial +``` + +This connector allows the use of [Apache Kafka](https://kafka.apache.org/) +topics as tables in Trino. Each message is presented as a row in Trino. + +Topics can be live. Rows appear as data arrives, and disappear as +segments get dropped. This can result in strange behavior if accessing the +same table multiple times in a single query (e.g., performing a self join). + +The connector reads and writes message data from Kafka topics in parallel across +workers to achieve a significant performance gain. The size of data sets for this +parallelization is configurable and can therefore be adapted to your specific +needs. + +See the {doc}`kafka-tutorial`. + +(kafka-requirements)= + +## Requirements + +To connect to Kafka, you need: + +- Kafka broker version 0.10.0 or higher. +- Network access from the Trino coordinator and workers to the Kafka nodes. + Port 9092 is the default port. + +When using Protobuf decoder with the {ref}`Confluent table description +supplier`, the following additional steps +must be taken: + +- Copy the `kafka-protobuf-provider` and `kafka-protobuf-types` JAR files + from [Confluent](https://packages.confluent.io/maven/io/confluent/) for + Confluent version 7.3.1 to the Kafka connector plugin directory (`/plugin/kafka`) on all nodes in the cluster. + The plugin directory depends on the {doc}`/installation` method. +- By copying those JARs and using them, you agree to the terms of the [Confluent + Community License Agreement](https://github.com/confluentinc/schema-registry/blob/master/LICENSE-ConfluentCommunity) + under which Confluent makes them available. + +These steps are not required if you are not using Protobuf and Confluent table +description supplier. + +## Configuration + +To configure the Kafka connector, create a catalog properties file +`etc/catalog/example.properties` with the following content, replacing the +properties as appropriate. + +In some cases, such as when using specialized authentication methods, it is necessary to specify +additional Kafka client properties in order to access your Kafka cluster. To do so, +add the `kafka.config.resources` property to reference your Kafka config files. Note that configs +can be overwritten if defined explicitly in `kafka.properties`: + +```text +connector.name=kafka +kafka.table-names=table1,table2 +kafka.nodes=host1:port,host2:port +kafka.config.resources=/etc/kafka-configuration.properties +``` + +### Multiple Kafka clusters + +You can have as many catalogs as you need, so if you have additional +Kafka clusters, simply add another properties file to `etc/catalog` +with a different name (making sure it ends in `.properties`). For +example, if you name the property file `sales.properties`, Trino +creates a catalog named `sales` using the configured connector. + +### Log levels + +Kafka consumer logging can be verbose and pollute Trino logs. To lower the +{ref}`log level `, simply add the following to `etc/log.properties`: + +```text +org.apache.kafka=WARN +``` + +## Configuration properties + +The following configuration properties are available: + +| Property name | Description | +| ----------------------------------------------------- | --------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| `kafka.default-schema` | Default schema name for tables. | +| `kafka.nodes` | List of nodes in the Kafka cluster. | +| `kafka.buffer-size` | Kafka read buffer size. | +| `kafka.hide-internal-columns` | Controls whether internal columns are part of the table schema or not. | +| `kafka.internal-column-prefix` | Prefix for internal columns, defaults to `_` | +| `kafka.messages-per-split` | Number of messages that are processed by each Trino split; defaults to `100000`. | +| `kafka.protobuf-any-support-enabled` | Enable support for encoding Protobuf `any` types to `JSON` by setting the property to `true`, defaults to `false`. | +| `kafka.timestamp-upper-bound-force-push-down-enabled` | Controls if upper bound timestamp pushdown is enabled for topics using `CreateTime` mode. | +| `kafka.security-protocol` | Security protocol for connection to Kafka cluster; defaults to `PLAINTEXT`. | +| `kafka.ssl.keystore.location` | Location of the keystore file. | +| `kafka.ssl.keystore.password` | Password for the keystore file. | +| `kafka.ssl.keystore.type` | File format of the keystore file; defaults to `JKS`. | +| `kafka.ssl.truststore.location` | Location of the truststore file. | +| `kafka.ssl.truststore.password` | Password for the truststore file. | +| `kafka.ssl.truststore.type` | File format of the truststore file; defaults to `JKS`. | +| `kafka.ssl.key.password` | Password for the private key in the keystore file. | +| `kafka.ssl.endpoint-identification-algorithm` | Endpoint identification algorithm used by clients to validate server host name; defaults to `https`. | +| `kafka.config.resources` | A comma-separated list of Kafka client configuration files. These files must exist on the machines running Trino. Only specify this if absolutely necessary to access Kafka. Example: `/etc/kafka-configuration.properties` | + +In addition, you must configure {ref}`table schema and schema registry usage +` with the relevant properties. + +### `kafka.default-schema` + +Defines the schema which contains all tables that were defined without +a qualifying schema name. + +This property is optional; the default is `default`. + +### `kafka.nodes` + +A comma separated list of `hostname:port` pairs for the Kafka data nodes. + +This property is required; there is no default and at least one node must be defined. + +:::{note} +Trino must still be able to connect to all nodes of the cluster +even if only a subset is specified here, as segment files may be +located only on a specific node. +::: + +### `kafka.buffer-size` + +Size of the internal data buffer for reading data from Kafka. The data +buffer must be able to hold at least one message and ideally can hold many +messages. There is one data buffer allocated per worker and data node. + +This property is optional; the default is `64kb`. + +### `kafka.timestamp-upper-bound-force-push-down-enabled` + +The upper bound predicate on `_timestamp` column +is pushed down only for topics using `LogAppendTime` mode. + +For topics using `CreateTime` mode, upper bound pushdown must be explicitly +enabled via `kafka.timestamp-upper-bound-force-push-down-enabled` config property +or `timestamp_upper_bound_force_push_down_enabled` session property. + +This property is optional; the default is `false`. + +### `kafka.hide-internal-columns` + +In addition to the data columns defined in a table description file, the +connector maintains a number of additional columns for each table. If +these columns are hidden, they can still be used in queries but do not +show up in `DESCRIBE ` or `SELECT *`. + +This property is optional; the default is `true`. + +### `kafka.security-protocol` + +Protocol used to communicate with brokers. +Valid values are: `PLAINTEXT`, `SSL`. + +This property is optional; default is `PLAINTEXT`. + +### `kafka.ssl.keystore.location` + +Location of the keystore file used for connection to Kafka cluster. + +This property is optional. + +### `kafka.ssl.keystore.password` + +Password for the keystore file used for connection to Kafka cluster. + +This property is optional, but required when `kafka.ssl.keystore.location` is given. + +### `kafka.ssl.keystore.type` + +File format of the keystore file. +Valid values are: `JKS`, `PKCS12`. + +This property is optional; default is `JKS`. + +### `kafka.ssl.truststore.location` + +Location of the truststore file used for connection to Kafka cluster. + +This property is optional. + +### `kafka.ssl.truststore.password` + +Password for the truststore file used for connection to Kafka cluster. + +This property is optional, but required when `kafka.ssl.truststore.location` is given. + +### `kafka.ssl.truststore.type` + +File format of the truststore file. +Valid values are: JKS, PKCS12. + +This property is optional; default is `JKS`. + +### `kafka.ssl.key.password` + +Password for the private key in the keystore file used for connection to Kafka cluster. + +This property is optional. This is required for clients only if two-way authentication is configured, i.e. `ssl.client.auth=required`. + +### `kafka.ssl.endpoint-identification-algorithm` + +The endpoint identification algorithm used by clients to validate server host name for connection to Kafka cluster. +Kafka uses `https` as default. Use `disabled` to disable server host name validation. + +This property is optional; default is `https`. + +## Internal columns + +The internal column prefix is configurable by `kafka.internal-column-prefix` +configuration property and defaults to `_`. A different prefix affects the +internal column names in the following sections. For example, a value of +`internal_` changes the partition ID column name from `_partition_id` +to `internal_partition_id`. + +For each defined table, the connector maintains the following columns: + +| Column name | Type | Description | +| ------------------- | ------------------------------ | ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| `_partition_id` | BIGINT | ID of the Kafka partition which contains this row. | +| `_partition_offset` | BIGINT | Offset within the Kafka partition for this row. | +| `_segment_start` | BIGINT | Lowest offset in the segment (inclusive) which contains this row. This offset is partition specific. | +| `_segment_end` | BIGINT | Highest offset in the segment (exclusive) which contains this row. The offset is partition specific. This is the same value as `_segment_start` of the next segment (if it exists). | +| `_segment_count` | BIGINT | Running count for the current row within the segment. For an uncompacted topic, `_segment_start + _segment_count` is equal to `_partition_offset`. | +| `_message_corrupt` | BOOLEAN | True if the decoder could not decode the message for this row. When true, data columns mapped from the message should be treated as invalid. | +| `_message` | VARCHAR | Message bytes as a UTF-8 encoded string. This is only useful for a text topic. | +| `_message_length` | BIGINT | Number of bytes in the message. | +| `_headers` | map(VARCHAR, array(VARBINARY)) | Headers of the message where values with the same key are grouped as array. | +| `_key_corrupt` | BOOLEAN | True if the key decoder could not decode the key for this row. When true, data columns mapped from the key should be treated as invalid. | +| `_key` | VARCHAR | Key bytes as a UTF-8 encoded string. This is only useful for textual keys. | +| `_key_length` | BIGINT | Number of bytes in the key. | +| `_timestamp` | TIMESTAMP | Message timestamp. | + +For tables without a table definition file, the `_key_corrupt` and +`_message_corrupt` columns will always be `false`. + +(kafka-table-schema-registry)= + +## Table schema and schema registry usage + +The table schema for the messages can be supplied to the connector with a +configuration file or a schema registry. It also provides a mechanism for the +connector to discover tables. + +You must configure the supplier with the `kafka.table-description-supplier` +property, setting it to `FILE` or `CONFLUENT`. Each table description +supplier has a separate set of configuration properties. + +Refer to the following subsections for more detail. The `FILE` table +description supplier is the default, and the value is case insensitive. + +### File table description supplier + +In order to use the file-based table description supplier, the +`kafka.table-description-supplier` must be set to `FILE`, which is the +default. + +In addition, you must set `kafka.table-names` and +`kafka.table-description-dir` as described in the following sections: + +#### `kafka.table-names` + +Comma-separated list of all tables provided by this catalog. A table name can be +unqualified (simple name), and is placed into the default schema (see +below), or it can be qualified with a schema name +(`.`). + +For each table defined here, a table description file (see below) may exist. If +no table description file exists, the table name is used as the topic name on +Kafka, and no data columns are mapped into the table. The table still contains +all internal columns (see below). + +This property is required; there is no default and at least one table must be +defined. + +#### `kafka.table-description-dir` + +References a folder within Trino deployment that holds one or more JSON files +(must end with `.json`) which contain table description files. + +This property is optional; the default is `etc/kafka`. + +(table-definition-files)= +#### Table definition files + +Kafka maintains topics only as byte messages and leaves it to producers +and consumers to define how a message should be interpreted. For Trino, +this data must be mapped into columns to allow queries against the data. + +:::{note} +For textual topics that contain JSON data, it is entirely possible to not +use any table definition files, but instead use the Trino +{doc}`/functions/json` to parse the `_message` column which contains +the bytes mapped into a UTF-8 string. This is cumbersome and makes it +difficult to write SQL queries. This only works when reading data. +::: + +A table definition file consists of a JSON definition for a table. The +name of the file can be arbitrary but must end in `.json`. Place the +file in the directory configured with the `kafka.table-description-dir` +property. The table definition file must be accessible from all Trino nodes. + +```text +{ + "tableName": ..., + "schemaName": ..., + "topicName": ..., + "key": { + "dataFormat": ..., + "fields": [ + ... + ] + }, + "message": { + "dataFormat": ..., + "fields": [ + ... + ] + } +} +``` + +| Field | Required | Type | Description | +| ------------ | -------- | ----------- | ------------------------------------------------------------------------- | +| `tableName` | required | string | Trino table name defined by this file. | +| `schemaName` | optional | string | Schema containing the table. If omitted, the default schema name is used. | +| `topicName` | required | string | Kafka topic that is mapped. | +| `key` | optional | JSON object | Field definitions for data columns mapped to the message key. | +| `message` | optional | JSON object | Field definitions for data columns mapped to the message itself. | + +#### Key and message in Kafka + +Starting with Kafka 0.8, each message in a topic can have an optional key. +A table definition file contains sections for both key and message to map +the data onto table columns. + +Each of the `key` and `message` fields in the table definition is a +JSON object that must contain two fields: + +| Field | Required | Type | Description | +| ------------ | -------- | ---------- | ------------------------------------------------------------------------------------------- | +| `dataFormat` | required | string | Selects the decoder for this group of fields. | +| `fields` | required | JSON array | A list of field definitions. Each field definition creates a new column in the Trino table. | + +Each field definition is a JSON object: + +```text +{ + "name": ..., + "type": ..., + "dataFormat": ..., + "mapping": ..., + "formatHint": ..., + "hidden": ..., + "comment": ... +} +``` + +| Field | Required | Type | Description | +| ------------ | -------- | ------- | -------------------------------------------------------------------------------------------------------------------- | +| `name` | required | string | Name of the column in the Trino table. | +| `type` | required | string | Trino type of the column. | +| `dataFormat` | optional | string | Selects the column decoder for this field. Defaults to the default decoder for this row data format and column type. | +| `dataSchema` | optional | string | The path or URL where the Avro schema resides. Used only for Avro decoder. | +| `mapping` | optional | string | Mapping information for the column. This is decoder specific, see below. | +| `formatHint` | optional | string | Sets a column-specific format hint to the column decoder. | +| `hidden` | optional | boolean | Hides the column from `DESCRIBE
` and `SELECT *`. Defaults to `false`. | +| `comment` | optional | string | Adds a column comment, which is shown with `DESCRIBE
`. | + +There is no limit on field descriptions for either key or message. + +(confluent-table-description-supplier)= + +### Confluent table description supplier + +The Confluent table description supplier uses the [Confluent Schema Registry](https://docs.confluent.io/1.0/schema-registry/docs/intro.html) to discover +table definitions. It is only tested to work with the Confluent Schema +Registry. + +The benefits of using the Confluent table description supplier over the file +table description supplier are: + +- New tables can be defined without a cluster restart. +- Schema updates are detected automatically. +- There is no need to define tables manually. +- Some Protobuf specific types like `oneof` and `any` are supported and mapped to JSON. + +When using Protobuf decoder with the Confluent table description supplier, some +additional steps are necessary. For details, refer to {ref}`kafka-requirements`. + +Set `kafka.table-description-supplier` to `CONFLUENT` to use the +schema registry. You must also configure the additional properties in the following table: + +:::{note} +Inserts are not supported, and the only data format supported is AVRO. +::: + +```{eval-rst} +.. list-table:: Confluent table description supplier properties + :widths: 30, 55, 15 + :header-rows: 1 + + * - Property name + - Description + - Default value + * - ``kafka.confluent-schema-registry-url`` + - Comma-separated list of URL addresses for the Confluent schema registry. + For example, ``http://schema-registry-1.example.org:8081,http://schema-registry-2.example.org:8081`` + - + * - ``kafka.confluent-schema-registry-client-cache-size`` + - The maximum number of subjects that can be stored in the local cache. The + cache stores the schemas locally by subjectId, and is provided by the + Confluent ``CachingSchemaRegistry`` client. + - 1000 + * - ``kafka.empty-field-strategy`` + - Avro allows empty struct fields, but this is not allowed in Trino. + There are three strategies for handling empty struct fields: + + * ``IGNORE`` - Ignore structs with no fields. This propagates to parents. + For example, an array of structs with no fields is ignored. + * ``FAIL`` - Fail the query if a struct with no fields is defined. + * ``MARK`` - Add a marker field named ``$empty_field_marker``, which of type boolean with a null value. + This may be desired if the struct represents a marker field. + + This can also be modified via the ``empty_field_strategy`` session property. + - ``IGNORE`` + * - ``kafka.confluent-subjects-cache-refresh-interval`` + - The interval used for refreshing the list of subjects and the definition + of the schema for the subject in the subject's cache. + - ``1s`` + +``` + +#### Confluent subject to table name mapping + +The [subject naming strategy](https://docs.confluent.io/platform/current/schema-registry/serdes-develop/index.html#sr-schemas-subject-name-strategy) +determines how a subject is resolved from the table name. + +The default strategy is the `TopicNameStrategy`, where the key subject is +defined as `-key` and the value subject is defined as +`-value`. If other strategies are used there is no way to +determine the subject name beforehand, so it must be specified manually in the +table name. + +To manually specify the key and value subjects, append to the topic name, +for example: `&key-subject=&value-subject=`. Both the `key-subject` and `value-subject` parameters are +optional. If neither is specified, then the default `TopicNameStrategy` is +used to resolve the subject name via the topic name. Note that a case +insensitive match must be done, as identifiers cannot contain upper case +characters. + +#### Protobuf-specific type handling in Confluent table description supplier + +When using the Confluent table description supplier, the following Protobuf +specific types are supported in addition to the {ref}`normally supported types +`: + +##### oneof + +Protobuf schemas containing `oneof` fields are mapped to a `JSON` field in +Trino. + +For example, given the following Protobuf schema: + +```text +syntax = "proto3"; + +message schema { + oneof test_oneof_column { + string string_column = 1; + uint32 integer_column = 2; + uint64 long_column = 3; + double double_column = 4; + float float_column = 5; + bool boolean_column = 6; + } +} +``` + +The corresponding Trino row is a `JSON` field `test_oneof_column` +containing a JSON object with a single key. The value of the key matches +the name of the `oneof` type that is present. + +In the above example, if the Protobuf message has the +`test_oneof_column` containing `string_column` set to a value `Trino` +then the corresponding Trino row includes a column named +`test_oneof_column` with the value `JSON '{"string_column": "Trino"}'`. + +(kafka-sql-inserts)= + +## Kafka inserts + +The Kafka connector supports the use of {doc}`/sql/insert` statements to write +data to a Kafka topic. Table column data is mapped to Kafka messages as defined +in the [table definition file](#table-definition-files). There are +five supported data formats for key and message encoding: + +- [raw format](raw-encoder) +- [CSV format](csv-encoder) +- [JSON format](json-encoder) +- [Avro format](avro-encoder) +- [Protobuf format](kafka-protobuf-encoding) + +These data formats each have an encoder that maps column values into bytes to be +sent to a Kafka topic. + +Trino supports at-least-once delivery for Kafka producers. This means that +messages are guaranteed to be sent to Kafka topics at least once. If a producer +acknowledgement times out, or if the producer receives an error, it might retry +sending the message. This could result in a duplicate message being sent to the +Kafka topic. + +The Kafka connector does not allow the user to define which partition will be +used as the target for a message. If a message includes a key, the producer will +use a hash algorithm to choose the target partition for the message. The same +key will always be assigned the same partition. + +(kafka-type-mapping)= + +## Type mapping + +Because Trino and Kafka each support types that the other does not, this +connector {ref}`maps some types ` when reading +({ref}`decoding `) or writing ({ref}`encoding +`) data. Type mapping depends on the format (Raw, Avro, +JSON, CSV). + +(kafka-row-encoding)= + +### Row encoding + +Encoding is required to allow writing data; it defines how table columns in +Trino map to Kafka keys and message data. + +The Kafka connector contains the following encoders: + +- [raw encoder](raw-encoder) - Table columns are mapped to a Kafka + message as raw bytes. +- [CSV encoder](csv-encoder) - Kafka message is formatted as a + comma-separated value. +- [JSON encoder](json-encoder) - Table columns are mapped to JSON + fields. +- [Avro encoder](avro-encoder) - Table columns are mapped to Avro + fields based on an Avro schema. +- [Protobuf encoder](kafka-protobuf-encoding) - Table columns are mapped to + Protobuf fields based on a Protobuf schema. + +:::{note} +A [table definition file](#table-definition-files) must be defined +for the encoder to work. +::: + +(raw-encoder)= +#### Raw encoder + +The raw encoder formats the table columns as raw bytes using the mapping +information specified in the +[table definition file](#table-definition-files). + +The following field attributes are supported: + +- `dataFormat` - Specifies the width of the column data type. +- `type` - Trino data type. +- `mapping` - start and optional end position of bytes to convert + (specified as `start` or `start:end`). + +The `dataFormat` attribute selects the number of bytes converted. +If absent, `BYTE` is assumed. All values are signed. + +Supported values: + +- `BYTE` - one byte +- `SHORT` - two bytes (big-endian) +- `INT` - four bytes (big-endian) +- `LONG` - eight bytes (big-endian) +- `FLOAT` - four bytes (IEEE 754 format, big-endian) +- `DOUBLE` - eight bytes (IEEE 754 format, big-endian) + +The `type` attribute defines the Trino data type. + +Different values of `dataFormat` are supported, depending on the Trino data +type: + +| Trino data type | `dataFormat` values | +| ------------------------ | ------------------------------ | +| `BIGINT` | `BYTE`, `SHORT`, `INT`, `LONG` | +| `INTEGER` | `BYTE`, `SHORT`, `INT` | +| `SMALLINT` | `BYTE`, `SHORT` | +| `TINYINT` | `BYTE` | +| `REAL` | `FLOAT` | +| `DOUBLE` | `FLOAT`, `DOUBLE` | +| `BOOLEAN` | `BYTE`, `SHORT`, `INT`, `LONG` | +| `VARCHAR` / `VARCHAR(x)` | `BYTE` | + +No other types are supported. + +The `mapping` attribute specifies the range of bytes in a key or +message used for encoding. + +:::{note} +Both a start and end position must be defined for `VARCHAR` types. +Otherwise, there is no way to know how many bytes the message contains. The +raw format mapping information is static and cannot be dynamically changed +to fit the variable width of some Trino data types. +::: + +If only a start position is given: + +- For fixed width types, the appropriate number of bytes are used for the + specified `dataFormat` (see above). + +If both a start and end position are given, then: + +- For fixed width types, the size must be equal to number of bytes used by + specified `dataFormat`. +- All bytes between start (inclusive) and end (exclusive) are used. + +:::{note} +All mappings must include a start position for encoding to work. +::: + +The encoding for numeric data types (`BIGINT`, `INTEGER`, `SMALLINT`, +`TINYINT`, `REAL`, `DOUBLE`) is straightforward. All numeric types use +big-endian. Floating point types use IEEE 754 format. + +Example raw field definition in a [table definition file](#table-definition-files) +for a Kafka message: + +```json +{ + "tableName": "example_table_name", + "schemaName": "example_schema_name", + "topicName": "example_topic_name", + "key": { "..." }, + "message": { + "dataFormat": "raw", + "fields": [ + { + "name": "field1", + "type": "BIGINT", + "dataFormat": "LONG", + "mapping": "0" + }, + { + "name": "field2", + "type": "INTEGER", + "dataFormat": "INT", + "mapping": "8" + }, + { + "name": "field3", + "type": "SMALLINT", + "dataFormat": "LONG", + "mapping": "12" + }, + { + "name": "field4", + "type": "VARCHAR(6)", + "dataFormat": "BYTE", + "mapping": "20:26" + } + ] + } +} +``` + +Columns should be defined in the same order they are mapped. There can be no +gaps or overlaps between column mappings. The width of the column as defined by +the column mapping must be equivalent to the width of the `dataFormat` for all +types except for variable width types. + +Example insert query for the above table definition: + +``` +INSERT INTO example_raw_table (field1, field2, field3, field4) + VALUES (123456789, 123456, 1234, 'abcdef'); +``` + +:::{note} +The raw encoder requires the field size to be known ahead of time, including +for variable width data types like `VARCHAR`. It also disallows inserting +values that do not match the width defined in the table definition +file. This is done to ensure correctness, as otherwise longer values are +truncated, and shorter values are read back incorrectly due to an undefined +padding character. +::: + +(csv-encoder)= +#### CSV encoder + +The CSV encoder formats the values for each row as a line of +comma-separated-values (CSV) using UTF-8 encoding. The CSV line is formatted +with a comma `,` as the column delimiter. + +The `type` and `mapping` attributes must be defined for each field: + +- `type` - Trino data type +- `mapping` - The integer index of the column in the CSV line (the first + column is 0, the second is 1, and so on) + +`dataFormat` and `formatHint` are not supported and must be omitted. + +The following Trino data types are supported by the CSV encoder: + +- `BIGINT` +- `INTEGER` +- `SMALLINT` +- `TINYINT` +- `DOUBLE` +- `REAL` +- `BOOLEAN` +- `VARCHAR` / `VARCHAR(x)` + +No other types are supported. + +Column values are converted to strings before they are formatted as a CSV line. + +The following is an example CSV field definition in a [table definition file](#table-definition-files) for a Kafka message: + +```json +{ + "tableName": "example_table_name", + "schemaName": "example_schema_name", + "topicName": "example_topic_name", + "key": { "..." }, + "message": { + "dataFormat": "csv", + "fields": [ + { + "name": "field1", + "type": "BIGINT", + "mapping": "0" + }, + { + "name": "field2", + "type": "VARCHAR", + "mapping": "1" + }, + { + "name": "field3", + "type": "BOOLEAN", + "mapping": "2" + } + ] + } +} +``` + +Example insert query for the above table definition: + +``` +INSERT INTO example_csv_table (field1, field2, field3) + VALUES (123456789, 'example text', TRUE); +``` + +(json-encoder)= +#### JSON encoder + +The JSON encoder maps table columns to JSON fields defined in the +[table definition file](#table-definition-files) according to +{rfc}`4627`. + +For fields, the following attributes are supported: + +- `type` - Trino data type of column. +- `mapping` - A slash-separated list of field names to select a field from the + JSON object. +- `dataFormat` - Name of formatter. Required for temporal types. +- `formatHint` - Pattern to format temporal data. Only use with + `custom-date-time` formatter. + +The following Trino data types are supported by the JSON encoder: + +- `BIGINT` +- `INTEGER` +- `SMALLINT` +- `TINYINT` +- `DOUBLE` +- `REAL` +- `BOOLEAN` +- `VARCHAR` +- `DATE` +- `TIME` +- `TIME WITH TIME ZONE` +- `TIMESTAMP` +- `TIMESTAMP WITH TIME ZONE` + +No other types are supported. + +The following `dataFormats` are available for temporal data: + +- `iso8601` +- `rfc2822` +- `custom-date-time` - Formats temporal data according to + [Joda Time](https://www.joda.org/joda-time/key_format.html) + pattern given by `formatHint` field. +- `milliseconds-since-epoch` +- `seconds-since-epoch` + +All temporal data in Kafka supports milliseconds precision. + +The following table defines which temporal data types are supported by +`dataFormats`: + +| Trino data type | Decoding rules | +| -------------------------- | ------------------------------------------------------------------------------------------- | +| `DATE` | `custom-date-time`, `iso8601` | +| `TIME` | `custom-date-time`, `iso8601`, `milliseconds-since-epoch`, `seconds-since-epoch` | +| `TIME WITH TIME ZONE` | `custom-date-time`, `iso8601` | +| `TIMESTAMP` | `custom-date-time`, `iso8601`, `rfc2822`, `milliseconds-since-epoch`, `seconds-since-epoch` | +| `TIMESTAMP WITH TIME ZONE` | `custom-date-time`, `iso8601`, `rfc2822`, `milliseconds-since-epoch`, `seconds-since-epoch` | + +The following is an example JSON field definition in a [table definition file](#table-definition-files) for a Kafka message: + +```json +{ + "tableName": "example_table_name", + "schemaName": "example_schema_name", + "topicName": "example_topic_name", + "key": { "..." }, + "message": { + "dataFormat": "json", + "fields": [ + { + "name": "field1", + "type": "BIGINT", + "mapping": "field1" + }, + { + "name": "field2", + "type": "VARCHAR", + "mapping": "field2" + }, + { + "name": "field3", + "type": "TIMESTAMP", + "dataFormat": "custom-date-time", + "formatHint": "yyyy-dd-MM HH:mm:ss.SSS", + "mapping": "field3" + } + ] + } +} +``` + +The following shows an example insert query for the preceding table definition: + +``` +INSERT INTO example_json_table (field1, field2, field3) + VALUES (123456789, 'example text', TIMESTAMP '2020-07-15 01:02:03.456'); +``` + +(avro-encoder)= +#### Avro encoder + +The Avro encoder serializes rows to Avro records as defined by the +[Avro schema](https://avro.apache.org/docs/current/). +Trino does not support schemaless Avro encoding. + +:::{note} +The Avro schema is encoded with the table column values in each Kafka message. +::: + +The `dataSchema` must be defined in the table definition file to use the Avro +encoder. It points to the location of the Avro schema file for the key or message. + +Avro schema files can be retrieved via HTTP or HTTPS from remote server with the +syntax: + +`"dataSchema": "http://example.org/schema/avro_data.avsc"` + +Local files need to be available on all Trino nodes and use an absolute path in +the syntax, for example: + +`"dataSchema": "/usr/local/schema/avro_data.avsc"` + +The following field attributes are supported: + +- `name` - Name of the column in the Trino table. +- `type` - Trino data type of column. +- `mapping` - A slash-separated list of field names to select a field from the + Avro schema. If the field specified in `mapping` does not exist + in the original Avro schema, then a write operation fails. + +The following table lists supported Trino data types, which can be used in `type` +for the equivalent Avro field type. + +| Trino data type | Avro data type | +| ------------------------ | ----------------- | +| `BIGINT` | `INT`, `LONG` | +| `REAL` | `FLOAT` | +| `DOUBLE` | `FLOAT`, `DOUBLE` | +| `BOOLEAN` | `BOOLEAN` | +| `VARCHAR` / `VARCHAR(x)` | `STRING` | + +No other types are supported. + +The following example shows an Avro field definition in a [kafka.table-description-dirition file](#table-definition-files) for a Kafka message: + +```json +{ + "tableName": "example_table_name", + "schemaName": "example_schema_name", + "topicName": "example_topic_name", + "key": { "..." }, + "message": + { + "dataFormat": "avro", + "dataSchema": "/avro_message_schema.avsc", + "fields": + [ + { + "name": "field1", + "type": "BIGINT", + "mapping": "field1" + }, + { + "name": "field2", + "type": "VARCHAR", + "mapping": "field2" + }, + { + "name": "field3", + "type": "BOOLEAN", + "mapping": "field3" + } + ] + } +} +``` + +In the following example, an Avro schema definition for the preceding table +definition is shown: + +```json +{ + "type" : "record", + "name" : "example_avro_message", + "namespace" : "io.trino.plugin.kafka", + "fields" : + [ + { + "name":"field1", + "type":["null", "long"], + "default": null + }, + { + "name": "field2", + "type":["null", "string"], + "default": null + }, + { + "name":"field3", + "type":["null", "boolean"], + "default": null + } + ], + "doc:" : "A basic avro schema" +} +``` + +The following is an example insert query for the preceding table definition: + +> INSERT INTO example_avro_table (field1, field2, field3) +> +> : VALUES (123456789, 'example text', FALSE); + +(kafka-protobuf-encoding)= + +#### Protobuf encoder + +The Protobuf encoder serializes rows to Protobuf DynamicMessages as defined by +the [Protobuf schema](https://developers.google.com/protocol-buffers/docs/overview). + +:::{note} +The Protobuf schema is encoded with the table column values in each Kafka message. +::: + +The `dataSchema` must be defined in the table definition file to use the +Protobuf encoder. It points to the location of the `proto` file for the key +or message. + +Protobuf schema files can be retrieved via HTTP or HTTPS from a remote server +with the syntax: + +`"dataSchema": "http://example.org/schema/schema.proto"` + +Local files need to be available on all Trino nodes and use an absolute path in +the syntax, for example: + +`"dataSchema": "/usr/local/schema/schema.proto"` + +The following field attributes are supported: + +- `name` - Name of the column in the Trino table. +- `type` - Trino type of column. +- `mapping` - slash-separated list of field names to select a field from the + Protobuf schema. If the field specified in `mapping` does not exist in the + original Protobuf schema, then a write operation fails. + +The following table lists supported Trino data types, which can be used in `type` +for the equivalent Protobuf field type. + +| Trino data type | Protobuf data type | +| ------------------------ | -------------------------------------------------- | +| `BOOLEAN` | `bool` | +| `INTEGER` | `int32`, `uint32`, `sint32`, `fixed32`, `sfixed32` | +| `BIGINT` | `int64`, `uint64`, `sint64`, `fixed64`, `sfixed64` | +| `DOUBLE` | `double` | +| `REAL` | `float` | +| `VARCHAR` / `VARCHAR(x)` | `string` | +| `VARBINARY` | `bytes` | +| `ROW` | `Message` | +| `ARRAY` | Protobuf type with `repeated` field | +| `MAP` | `Map` | +| `TIMESTAMP` | `Timestamp`, predefined in `timestamp.proto` | + +The following example shows a Protobuf field definition in a [table definition +file](#table-definition-files) for a Kafka message: + +```json +{ + "tableName": "example_table_name", + "schemaName": "example_schema_name", + "topicName": "example_topic_name", + "key": { "..." }, + "message": + { + "dataFormat": "protobuf", + "dataSchema": "/message_schema.proto", + "fields": + [ + { + "name": "field1", + "type": "BIGINT", + "mapping": "field1" + }, + { + "name": "field2", + "type": "VARCHAR", + "mapping": "field2" + }, + { + "name": "field3", + "type": "BOOLEAN", + "mapping": "field3" + } + ] + } +} +``` + +In the following example, a Protobuf schema definition for the preceding table +definition is shown: + +```text +syntax = "proto3"; + +message schema { + uint64 field1 = 1 ; + string field2 = 2; + bool field3 = 3; +} +``` + +The following is an example insert query for the preceding table definition: + +```sql +INSERT INTO example_protobuf_table (field1, field2, field3) + VALUES (123456789, 'example text', FALSE); +``` + +(kafka-row-decoding)= + +### Row decoding + +For key and message, a decoder is used to map message and key data onto table columns. + +The Kafka connector contains the following decoders: + +- `raw` - Kafka message is not interpreted; ranges of raw message bytes are mapped to table columns. +- `csv` - Kafka message is interpreted as comma separated message, and fields are mapped to table columns. +- `json` - Kafka message is parsed as JSON, and JSON fields are mapped to table columns. +- `avro` - Kafka message is parsed based on an Avro schema, and Avro fields are mapped to table columns. +- `protobuf` - Kafka message is parsed based on a Protobuf schema, and Protobuf fields are mapped to table columns. + +:::{note} +If no table definition file exists for a table, the `dummy` decoder is used, +which does not expose any columns. +::: + +#### Raw decoder + +The raw decoder supports reading of raw byte-based values from Kafka message +or key, and converting it into Trino columns. + +For fields, the following attributes are supported: + +- `dataFormat` - Selects the width of the data type converted. +- `type` - Trino data type. See table later min this document for list of + supported data types. +- `mapping` - `[:]` - Start and end position of bytes to convert (optional). + +The `dataFormat` attribute selects the number of bytes converted. +If absent, `BYTE` is assumed. All values are signed. + +Supported values are: + +- `BYTE` - one byte +- `SHORT` - two bytes (big-endian) +- `INT` - four bytes (big-endian) +- `LONG` - eight bytes (big-endian) +- `FLOAT` - four bytes (IEEE 754 format) +- `DOUBLE` - eight bytes (IEEE 754 format) + +The `type` attribute defines the Trino data type on which the value is mapped. + +Depending on the Trino type assigned to a column, different values of dataFormat can be used: + +| Trino data type | Allowed `dataFormat` values | +| ------------------------ | ------------------------------ | +| `BIGINT` | `BYTE`, `SHORT`, `INT`, `LONG` | +| `INTEGER` | `BYTE`, `SHORT`, `INT` | +| `SMALLINT` | `BYTE`, `SHORT` | +| `TINYINT` | `BYTE` | +| `DOUBLE` | `DOUBLE`, `FLOAT` | +| `BOOLEAN` | `BYTE`, `SHORT`, `INT`, `LONG` | +| `VARCHAR` / `VARCHAR(x)` | `BYTE` | + +No other types are supported. + +The `mapping` attribute specifies the range of the bytes in a key or +message used for decoding. It can be one or two numbers separated by a colon (`[:]`). + +If only a start position is given: + +- For fixed width types, the column will use the appropriate number of bytes for the specified `dataFormat` (see above). +- When `VARCHAR` value is decoded, all bytes from start position till the end of the message will be used. + +If start and end position are given: + +- For fixed width types, the size must be equal to number of bytes used by specified `dataFormat`. +- For `VARCHAR` all bytes between start (inclusive) and end (exclusive) are used. + +If no `mapping` attribute is specified, it is equivalent to setting start position to 0 and leaving end position undefined. + +The decoding scheme of numeric data types (`BIGINT`, `INTEGER`, `SMALLINT`, `TINYINT`, `DOUBLE`) is straightforward. +A sequence of bytes is read from input message and decoded according to either: + +- big-endian encoding (for integer types) +- IEEE 754 format for (for `DOUBLE`). + +Length of decoded byte sequence is implied by the `dataFormat`. + +For `VARCHAR` data type a sequence of bytes is interpreted according to UTF-8 +encoding. + +#### CSV decoder + +The CSV decoder converts the bytes representing a message or key into a +string using UTF-8 encoding and then interprets the result as a CSV +(comma-separated value) line. + +For fields, the `type` and `mapping` attributes must be defined: + +- `type` - Trino data type. See the following table for a list of supported data types. +- `mapping` - The index of the field in the CSV record. + +The `dataFormat` and `formatHint` attributes are not supported and must be omitted. + +Table below lists supported Trino types, which can be used in `type` and decoding scheme: + +:::{list-table} +:header-rows: 1 + +* - Trino data type + - Decoding rules +* - `BIGINT`, `INTEGER`, `SMALLINT`, `TINYINT` + - Decoded using Java `Long.parseLong()` +* - `DOUBLE` + - Decoded using Java `Double.parseDouble()` +* - `BOOLEAN` + - "true" character sequence maps to `true`; Other character sequences map to `false` +* - `VARCHAR`, `VARCHAR(x)` + - Used as is +::: + +No other types are supported. + +#### JSON decoder + +The JSON decoder converts the bytes representing a message or key into a +JSON according to {rfc}`4627`. Note that the message or key *MUST* convert +into a JSON object, not an array or simple type. + +For fields, the following attributes are supported: + +- `type` - Trino data type of column. +- `dataFormat` - Field decoder to be used for column. +- `mapping` - slash-separated list of field names to select a field from the JSON object. +- `formatHint` - Only for `custom-date-time`. + +The JSON decoder supports multiple field decoders, with `_default` being +used for standard table columns and a number of decoders for date- and +time-based types. + +The following table lists Trino data types, which can be used as in `type`, and matching field decoders, +which can be specified via `dataFormat` attribute. + +:::{list-table} +:header-rows: 1 + +* - Trino data type + - Allowed `dataFormat` values +* - `BIGINT`, `INTEGER`, `SMALLINT`, `TINYINT`, `DOUBLE`, `BOOLEAN`, `VARCHAR`, `VARCHAR(x)` + - Default field decoder (omitted `dataFormat` attribute) +* - `DATE` + - `custom-date-time`, `iso8601` +* - `TIME` + - `custom-date-time`, `iso8601`, `milliseconds-since-epoch`, `seconds-since-epoch` +* - `TIME WITH TIME ZONE` + - `custom-date-time`, `iso8601` +* - `TIMESTAMP` + - `custom-date-time`, `iso8601`, `rfc2822`, `milliseconds-since-epoch`, `seconds-since-epoch` +* - `TIMESTAMP WITH TIME ZONE` + - `custom-date-time`, `iso8601`, `rfc2822`, `milliseconds-since-epoch` `seconds-since-epoch` +::: + +No other types are supported. + +##### Default field decoder + +This is the standard field decoder, supporting all the Trino physical data +types. A field value is transformed under JSON conversion rules into +boolean, long, double or string values. For non-date/time based columns, +this decoder should be used. + +##### Date and time decoders + +To convert values from JSON objects into Trino `DATE`, `TIME`, `TIME WITH TIME ZONE`, +`TIMESTAMP` or `TIMESTAMP WITH TIME ZONE` columns, special decoders must be selected using the +`dataFormat` attribute of a field definition. + +- `iso8601` - Text based, parses a text field as an ISO 8601 timestamp. +- `rfc2822` - Text based, parses a text field as an {rfc}`2822` timestamp. +- `custom-date-time` - Text based, parses a text field according to Joda format pattern + : specified via `formatHint` attribute. Format pattern should conform + to . +- `milliseconds-since-epoch` - Number-based; interprets a text or number as number of milliseconds since the epoch. +- `seconds-since-epoch` - Number-based; interprets a text or number as number of milliseconds since the epoch. + +For `TIMESTAMP WITH TIME ZONE` and `TIME WITH TIME ZONE` data types, if timezone information is present in decoded value, it will +be used as Trino value. Otherwise result time zone will be set to `UTC`. + +#### Avro decoder + +The Avro decoder converts the bytes representing a message or key in +Avro format based on a schema. The message must have the Avro schema embedded. +Trino does not support schemaless Avro decoding. + +For key/message, using `avro` decoder, the `dataSchema` must be defined. +This should point to the location of a valid Avro schema file of the message which needs to be decoded. This location can be a remote web server +(e.g.: `dataSchema: 'http://example.org/schema/avro_data.avsc'`) or local file system(e.g.: `dataSchema: '/usr/local/schema/avro_data.avsc'`). +The decoder fails if this location is not accessible from the Trino coordinator node. + +For fields, the following attributes are supported: + +- `name` - Name of the column in the Trino table. +- `type` - Trino data type of column. +- `mapping` - A slash-separated list of field names to select a field from the Avro schema. If field specified in `mapping` does not exist in the original Avro schema, then a read operation returns `NULL`. + +The following table lists the supported Trino types which can be used in `type` for the equivalent Avro field types: + +| Trino data type | Allowed Avro data type | +| ------------------------ | ---------------------- | +| `BIGINT` | `INT`, `LONG` | +| `DOUBLE` | `DOUBLE`, `FLOAT` | +| `BOOLEAN` | `BOOLEAN` | +| `VARCHAR` / `VARCHAR(x)` | `STRING` | +| `VARBINARY` | `FIXED`, `BYTES` | +| `ARRAY` | `ARRAY` | +| `MAP` | `MAP` | + +No other types are supported. + +##### Avro schema evolution + +The Avro decoder supports schema evolution feature with backward compatibility. With backward compatibility, +a newer schema can be used to read Avro data created with an older schema. Any change in the Avro schema must also be +reflected in Trino's topic definition file. Newly added/renamed fields *must* have a default value in the Avro schema file. + +The schema evolution behavior is as follows: + +- Column added in new schema: + Data created with an older schema produces a *default* value when the table is using the new schema. +- Column removed in new schema: + Data created with an older schema no longer outputs the data from the column that was removed. +- Column is renamed in the new schema: + This is equivalent to removing the column and adding a new one, and data created with an older schema + produces a *default* value when table is using the new schema. +- Changing type of column in the new schema: + If the type coercion is supported by Avro, then the conversion happens. An + error is thrown for incompatible types. + +(kafka-protobuf-decoding)= + +#### Protobuf decoder + +The Protobuf decoder converts the bytes representing a message or key in +Protobuf formatted message based on a schema. + +For key/message, using the `protobuf` decoder, the `dataSchema` must be +defined. It points to the location of a valid `proto` file of the message +which needs to be decoded. This location can be a remote web server, +`dataSchema: 'http://example.org/schema/schema.proto'`, or local file, +`dataSchema: '/usr/local/schema/schema.proto'`. The decoder fails if the +location is not accessible from the coordinator. + +For fields, the following attributes are supported: + +- `name` - Name of the column in the Trino table. +- `type` - Trino data type of column. +- `mapping` - slash-separated list of field names to select a field from the + Protobuf schema. If field specified in `mapping` does not exist in the + original `proto` file then a read operation returns NULL. + +The following table lists the supported Trino types which can be used in +`type` for the equivalent Protobuf field types: + +| Trino data type | Allowed Protobuf data type | +| ------------------------ | -------------------------------------------------- | +| `BOOLEAN` | `bool` | +| `INTEGER` | `int32`, `uint32`, `sint32`, `fixed32`, `sfixed32` | +| `BIGINT` | `int64`, `uint64`, `sint64`, `fixed64`, `sfixed64` | +| `DOUBLE` | `double` | +| `REAL` | `float` | +| `VARCHAR` / `VARCHAR(x)` | `string` | +| `VARBINARY` | `bytes` | +| `ROW` | `Message` | +| `ARRAY` | Protobuf type with `repeated` field | +| `MAP` | `Map` | +| `TIMESTAMP` | `Timestamp`, predefined in `timestamp.proto` | +| `JSON` | `oneof` (Confluent table supplier only), `Any` | + +##### any + +Message types with an [Any](https://protobuf.dev/programming-guides/proto3/#any) +field contain an arbitrary serialized message as bytes and a type URL to resolve +that message's type with a scheme of `file://`, `http://`, or `https://`. +The connector reads the contents of the URL to create the type descriptor +for the `Any` message and convert the message to JSON. This behavior is enabled +by setting `kafka.protobuf-any-support-enabled` to `true`. + +The descriptors for each distinct URL are cached for performance reasons and +any modifications made to the type returned by the URL requires a restart of +Trino. + +For example, given the following Protobuf schema which defines `MyMessage` +with three columns: + +```text +syntax = "proto3"; + +message MyMessage { + string stringColumn = 1; + uint32 integerColumn = 2; + uint64 longColumn = 3; +} +``` + +And a separate schema which uses an `Any` type which is a packed message +of the above type and a valid URL: + +```text +syntax = "proto3"; + +import "google/protobuf/any.proto"; + +message schema { + google.protobuf.Any any_message = 1; +} +``` + +The corresponding Trino column is named `any_message` of type `JSON` +containing a JSON-serialized representation of the Protobuf message: + +```text +{ + "@type":"file:///path/to/schemas/MyMessage", + "longColumn":"493857959588286460", + "numberColumn":"ONE", + "stringColumn":"Trino" +} +``` + +##### Protobuf schema evolution + +The Protobuf decoder supports the schema evolution feature with backward +compatibility. With backward compatibility, a newer schema can be used to read +Protobuf data created with an older schema. Any change in the Protobuf schema +*must* also be reflected in the topic definition file. + +The schema evolution behavior is as follows: + +- Column added in new schema: + Data created with an older schema produces a *default* value when the table is using the new schema. +- Column removed in new schema: + Data created with an older schema no longer outputs the data from the column that was removed. +- Column is renamed in the new schema: + This is equivalent to removing the column and adding a new one, and data created with an older schema + produces a *default* value when table is using the new schema. +- Changing type of column in the new schema: + If the type coercion is supported by Protobuf, then the conversion happens. An error is thrown for incompatible types. + +##### Protobuf limitations + +- Protobuf Timestamp has a nanosecond precision but Trino supports + decoding/encoding at microsecond precision. + +(kafka-sql-support)= + +## SQL support + +The connector provides read and write access to data and metadata in Trino +tables populated by Kafka topics. See {ref}`kafka-row-decoding` for more +information. + +In addition to the {ref}`globally available ` +and {ref}`read operation ` statements, the connector +supports the following features: + +- {doc}`/sql/insert`, encoded to a specified data format. See also + {ref}`kafka-sql-inserts`. diff --git a/430/_sources/connector/kinesis.md.txt b/430/_sources/connector/kinesis.md.txt new file mode 100644 index 000000000..116cbfd0e --- /dev/null +++ b/430/_sources/connector/kinesis.md.txt @@ -0,0 +1,276 @@ +# Kinesis connector + +```{raw} html + +``` + +[Kinesis](https://aws.amazon.com/kinesis/) is Amazon's fully managed cloud-based service for real-time processing of large, distributed data streams. + +This connector allows the use of Kinesis streams as tables in Trino, such that each data-blob/message +in a Kinesis stream is presented as a row in Trino. A flexible table mapping approach lets us +treat fields of the messages as columns in the table. + +Under the hood, a Kinesis +[shard iterator](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetShardIterator.html) +is used to retrieve the records, along with a series of +[GetRecords](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.html) calls. +The shard iterator starts by default 24 hours before the current time, and works its way forward. +To be able to query a stream, table mappings are needed. These table definitions can be +stored on Amazon S3 (preferred), or stored in a local directory on each Trino node. + +This connector is a **read-only** connector. It can only fetch data from Kinesis streams, +but cannot create streams or push data into existing streams. + +To configure the Kinesis connector, create a catalog properties file +`etc/catalog/example.properties` with the following contents, replacing the +properties as appropriate: + +```text +connector.name=kinesis +kinesis.access-key=XXXXXX +kinesis.secret-key=XXXXXX +``` + +## Configuration properties + +The following configuration properties are available: + +| Property name | Description | +| -------------------------------------------- | --------------------------------------------------------------------- | +| `kinesis.access-key` | Access key to AWS account or blank to use default provider chain | +| `kinesis.secret-key` | Secret key to AWS account or blank to use default provider chain | +| `kinesis.aws-region` | AWS region to be used to read kinesis stream from | +| `kinesis.default-schema` | Default schema name for tables | +| `kinesis.table-description-location` | Directory containing table description files | +| `kinesis.table-description-refresh-interval` | How often to get the table description from S3 | +| `kinesis.hide-internal-columns` | Controls whether internal columns are part of the table schema or not | +| `kinesis.batch-size` | Maximum number of records to return in one batch | +| `kinesis.fetch-attempts` | Read attempts made when no records returned and not caught up | +| `kinesis.max-batches` | Maximum batches to read from Kinesis in one single query | +| `kinesis.sleep-time` | Time for thread to sleep waiting to make next attempt to fetch batch | +| `kinesis.iterator-from-timestamp` | Begin iterating from a given timestamp instead of the trim horizon | +| `kinesis.iterator-offset-seconds` | Number of seconds before current time to start iterating | + +### `kinesis.access-key` + +Defines the access key ID for AWS root account or IAM roles, which is used to sign programmatic requests to AWS Kinesis. + +This property is optional; if not defined, the connector tries to follow `Default-Credential-Provider-Chain` provided by AWS in the following order: + +- Environment Variable: Load credentials from environment variables `AWS_ACCESS_KEY_ID` and `AWS_SECRET_ACCESS_KEY`. +- Java System Variable: Load from java system as `aws.accessKeyId` and `aws.secretKey`. +- Profile Credentials File: Load from file typically located at `~/.aws/credentials`. +- Instance profile credentials: These credentials can be used on EC2 instances, and are delivered through the Amazon EC2 metadata service. + +### `kinesis.secret-key` + +Defines the secret key for AWS root account or IAM roles, which together with Access Key ID, is used to sign programmatic requests to AWS Kinesis. + +This property is optional; if not defined, connector will try to follow `Default-Credential-Provider-Chain` same as above. + +### `kinesis.aws-region` + +Defines AWS Kinesis regional endpoint. Selecting appropriate region may reduce latency in fetching data. + +This field is optional; The default region is `us-east-1` referring to end point 'kinesis.us-east-1.amazonaws.com'. + +See [Kinesis Data Streams regions](https://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region) +for a current list of available regions. + +### `kinesis.default-schema` + +Defines the schema which contains all tables that were defined without a qualifying schema name. + +This property is optional; the default is `default`. + +### `kinesis.table-description-location` + +References an S3 URL or a folder within Trino deployment that holds one or more JSON files ending with `.json`, which contain table description files. +The S3 bucket and folder will be checked every 10 minutes for updates and changed files. + +This property is optional; the default is `etc/kinesis`. + +### `kinesis.table-description-refresh-interval` + +This property controls how often the table description is refreshed from S3. + +This property is optional; the default is `10m`. + +### `kinesis.batch-size` + +Defines the maximum number of records to return in one request to Kinesis Streams. Maximum limit is `10000` records. + +This field is optional; the default value is `10000`. + +### `kinesis.max-batches` + +The maximum number of batches to read in a single query. The default value is `1000`. + +### `kinesis.fetch-attempts` + +Defines the number of attempts made to read a batch from Kinesis Streams, when no records are returned and the *millis behind latest* +parameter shows we are not yet caught up. When records are returned no additional attempts are necessary. +`GetRecords` has been observed to return no records even though the shard is not empty. +That is why multiple attempts need to be made. + +This field is optional; the default value is `2`. + +### `kinesis.sleep-time` + +Defines the duration for which a thread needs to sleep between `kinesis.fetch-attempts` made to fetch data. + +This field is optional; the default value is `1000ms`. + +### `kinesis.iterator-from-timestamp` + +Use an initial shard iterator type of `AT_TIMESTAMP` starting `kinesis.iterator-offset-seconds` before the current time. +When this is false, an iterator type of `TRIM_HORIZON` is used, meaning it starts from the oldest record in the stream. + +The default is true. + +### `kinesis.iterator-offset-seconds` + +When `kinesis.iterator-from-timestamp` is true, the shard iterator starts at `kinesis.iterator-offset-seconds` before the current time. + +The default is `86400` seconds (24 hours). + +### `kinesis.hide-internal-columns` + +In addition to the data columns defined in a table description file, the connector maintains a number of additional columns for each table. +If these columns are hidden, they can still be used in queries, but they do not show up in `DESCRIBE ` or `SELECT *`. + +This property is optional; the default is true. + +## Internal columns + +For each defined table, the connector maintains the following columns: + +| Column name | Type | Description | +| -------------------- | ----------- | ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| `_shard_id` | `VARCHAR` | ID of the Kinesis stream shard which contains this row. | +| `_shard_sequence_id` | `VARCHAR` | Sequence id within the Kinesis shard for this row. | +| `_segment_start` | `BIGINT` | Lowest offset in the segment (inclusive) which contains this row. This offset is partition specific. | +| `_segment_end` | `BIGINT` | Highest offset in the segment (exclusive) which contains this row. The offset is partition specific. This is the same value as `_segment_start` of the next segment (if it exists). | +| `_segment_count` | `BIGINT` | Running count for the current row within the segment. For an uncompacted topic, `_segment_start + _segment_count` is equal to `_partition_offset`. | +| `_message_valid` | `BOOLEAN` | True if the decoder could decode the message successfully for this row. When false, data columns mapped from the message should be treated as invalid. | +| `_message` | `VARCHAR` | Message bytes as an UTF-8 encoded string. This is only useful for a text topic. | +| `_message_length` | `BIGINT` | Number of bytes in the message. | +| `_message_timestamp` | `TIMESTAMP` | Approximate arrival time of the message (milliseconds granularity). | +| `_key` | `VARCHAR` | Key bytes as an UTF-8 encoded string. This is only useful for textual keys. | +| `_partition_key` | `VARCHAR` | Partition Key bytes as a UTF-8 encoded string. | + +For tables without a table definition file, the `_message_valid` column is always `true`. + +## Table definition + +A table definition file consists of a JSON definition for a table, which corresponds to one stream in Kinesis. +The name of the file can be arbitrary but must end in `.json`. The structure of the table definition is as follows: + +```text +{ + "tableName": ..., + "schemaName": ..., + "streamName": ..., + "message": { + "dataFormat": ..., + "fields": [ + ... + ] + } + } +``` + +| Field | Required | Type | Description | +| ------------ | -------- | ----------- | ----------------------------------------------------------------------------- | +| `tableName` | required | string | Trino table name defined by this file. | +| `schemaName` | optional | string | Schema which contains the table. If omitted, the default schema name is used. | +| `streamName` | required | string | Name of the Kinesis Stream that is mapped | +| `message` | optional | JSON object | Field definitions for data columns mapped to the message itself. | + +Every message in a Kinesis stream can be decoded using the definition provided in the message object. +The JSON object message in the table definition contains two fields: + +| Field | Required | Type | Description | +| ------------ | -------- | ---------- | ------------------------------------------------------------------------------------------- | +| `dataFormat` | required | string | Selects the decoder for this group of fields. | +| `fields` | required | JSON array | A list of field definitions. Each field definition creates a new column in the Trino table. | + +Each field definition is a JSON object. At a minimum, a name, type, and mapping must be provided. +The overall structure looks like this: + +```text +{ + "name": ..., + "type": ..., + "dataFormat": ..., + "mapping": ..., + "formatHint": ..., + "hidden": ..., + "comment": ... +} +``` + +| Field | Required | Type | Description | +| ------------ | -------- | ------- | -------------------------------------------------------------------------------------------------------------------- | +| `name` | required | string | Name of the column in the Trino table. | +| `type` | required | string | Trino type of the column. | +| `dataFormat` | optional | string | Selects the column decoder for this field. Defaults to the default decoder for this row data format and column type. | +| `mapping` | optional | string | Mapping information for the column. This is decoder specific -- see below. | +| `formatHint` | optional | string | Sets a column specific format hint to the column decoder. | +| `hidden` | optional | boolean | Hides the column from `DESCRIBE
` and `SELECT *`. Defaults to `false`. | +| `comment` | optional | string | Adds a column comment which is shown with `DESCRIBE
`. | + +The name field is exposed to Trino as the column name, while the mapping field is the portion of the message that gets +mapped to that column. For JSON object messages, this refers to the field name of an object, and can be a path that drills +into the object structure of the message. Additionally, you can map a field of the JSON object to a string column type, +and if it is a more complex type (JSON array or JSON object) then the JSON itself becomes the field value. + +There is no limit on field descriptions for either key or message. + +(kinesis-type-mapping)= + +## Type mapping + +Because Trino and Kinesis each support types that the other does not, this +connector {ref}`maps some types ` when reading data. Type +mapping depends on the RAW, CSV, JSON, and AVRO file formats. + +### Row decoding + +A decoder is used to map data to table columns. + +The connector contains the following decoders: + +- `raw`: Message is not interpreted; ranges of raw message bytes are mapped + to table columns. +- `csv`: Message is interpreted as comma separated message, and fields are + mapped to table columns. +- `json`: Message is parsed as JSON, and JSON fields are mapped to table + columns. +- `avro`: Message is parsed based on an Avro schema, and Avro fields are + mapped to table columns. + +:::{note} +If no table definition file exists for a table, the `dummy` decoder is +used, which does not expose any columns. +::: + +```{include} raw-decoder.fragment +``` + +```{include} csv-decoder.fragment +``` + +```{include} json-decoder.fragment +``` + +```{include} avro-decoder.fragment +``` + +(kinesis-sql-support)= + +## SQL support + +The connector provides {ref}`globally available ` and +{ref}`read operation ` statements to access data and +metadata from Kinesis streams. diff --git a/430/_sources/connector/kudu.md.txt b/430/_sources/connector/kudu.md.txt new file mode 100644 index 000000000..70fab26ba --- /dev/null +++ b/430/_sources/connector/kudu.md.txt @@ -0,0 +1,610 @@ +# Kudu connector + +```{raw} html + +``` + +The Kudu connector allows querying, inserting and deleting data in [Apache Kudu]. + +## Requirements + +To connect to Kudu, you need: + +- Kudu version 1.13.0 or higher. +- Network access from the Trino coordinator and workers to Kudu. Port 7051 is + the default port. + +## Configuration + +To configure the Kudu connector, create a catalog properties file +`etc/catalog/kudu.properties` with the following contents, +replacing the properties as appropriate: + +```properties +connector.name=kudu + +## Defaults to NONE +kudu.authentication.type = NONE + +## List of Kudu master addresses, at least one is needed (comma separated) +## Supported formats: example.com, example.com:7051, 192.0.2.1, 192.0.2.1:7051, +## [2001:db8::1], [2001:db8::1]:7051, 2001:db8::1 +kudu.client.master-addresses=localhost + +## Kudu does not support schemas, but the connector can emulate them optionally. +## By default, this feature is disabled, and all tables belong to the default schema. +## For more details see connector documentation. +#kudu.schema-emulation.enabled=false + +## Prefix to use for schema emulation (only relevant if `kudu.schema-emulation.enabled=true`) +## The standard prefix is `presto::`. Empty prefix is also supported. +## For more details see connector documentation. +#kudu.schema-emulation.prefix= + +########################################### +### Advanced Kudu Java client configuration +########################################### + +## Default timeout used for administrative operations (e.g. createTable, deleteTable, etc.) +#kudu.client.default-admin-operation-timeout = 30s + +## Default timeout used for user operations +#kudu.client.default-operation-timeout = 30s + +## Disable Kudu client's collection of statistics. +#kudu.client.disable-statistics = false + +## Assign Kudu splits to replica host if worker and kudu share the same cluster +#kudu.allow-local-scheduling = false +``` + +## Kerberos support + +In order to connect to a kudu cluster that uses `kerberos` +authentication, you need to configure the following kudu properties: + +```properties +kudu.authentication.type = KERBEROS + +## The kerberos client principal name +kudu.authentication.client.principal = clientprincipalname + +## The path to the kerberos keytab file +## The configured client principal must exist in this keytab file +kudu.authentication.client.keytab = /path/to/keytab/file.keytab + +## The path to the krb5.conf kerberos config file +kudu.authentication.config = /path/to/kerberos/krb5.conf + +## Optional and defaults to "kudu" +## If kudu is running with a custom SPN this needs to be configured +kudu.authentication.server.principal.primary = kudu +``` + +## Querying data + +Apache Kudu does not support schemas, i.e. namespaces for tables. +The connector can optionally emulate schemas by table naming conventions. + +### Default behaviour (without schema emulation) + +The emulation of schemas is disabled by default. +In this case all Kudu tables are part of the `default` schema. + +For example, a Kudu table named `orders` can be queried in Trino +with `SELECT * FROM example.default.orders` or simple with `SELECT * FROM orders` +if catalog and schema are set to `kudu` and `default` respectively. + +Table names can contain any characters in Kudu. In this case, use double quotes. +E.g. To query a Kudu table named `special.table!` use `SELECT * FROM example.default."special.table!"`. + +#### Example + +- Create a users table in the default schema: + + ``` + CREATE TABLE example.default.users ( + user_id int WITH (primary_key = true), + first_name VARCHAR, + last_name VARCHAR + ) WITH ( + partition_by_hash_columns = ARRAY['user_id'], + partition_by_hash_buckets = 2 + ); + ``` + + On creating a Kudu table you must/can specify additional information about + the primary key, encoding, and compression of columns and hash or range + partitioning. For details see the {ref}`kudu-create-table` section. + +- Describe the table: + + ``` + DESCRIBE example.default.users; + ``` + + ```text + Column | Type | Extra | Comment + ------------+---------+-------------------------------------------------+--------- + user_id | integer | primary_key, encoding=auto, compression=default | + first_name | varchar | nullable, encoding=auto, compression=default | + last_name | varchar | nullable, encoding=auto, compression=default | + (3 rows) + ``` + +- Insert some data: + + ``` + INSERT INTO example.default.users VALUES (1, 'Donald', 'Duck'), (2, 'Mickey', 'Mouse'); + ``` + +- Select the inserted data: + + ``` + SELECT * FROM example.default.users; + ``` + +(behavior-with-schema-emulation)= + +### Behavior with schema emulation + +If schema emulation has been enabled in the connector properties, i.e. +`etc/catalog/example.properties`, tables are mapped to schemas depending on +some conventions. + +- With `kudu.schema-emulation.enabled=true` and `kudu.schema-emulation.prefix=`, + the mapping works like: + + | Kudu table name | Trino qualified name | + | --------------- | --------------------- | + | `orders` | `kudu.default.orders` | + | `part1.part2` | `kudu.part1.part2` | + | `x.y.z` | `kudu.x."y.z"` | + + As schemas are not directly supported by Kudu, a special table named + `$schemas` is created for managing the schemas. + +- With `kudu.schema-emulation.enabled=true` and `kudu.schema-emulation.prefix=presto::`, + the mapping works like: + + | Kudu table name | Trino qualified name | + | --------------------- | ---------------------------- | + | `orders` | `kudu.default.orders` | + | `part1.part2` | `kudu.default."part1.part2"` | + | `x.y.z` | `kudu.default."x.y.z"` | + | `presto::part1.part2` | `kudu.part1.part2` | + | `presto:x.y.z` | `kudu.x."y.z"` | + + As schemas are not directly supported by Kudu, a special table named + `presto::$schemas` is created for managing the schemas. + +(kudu-type-mapping)= + +## Type mapping + +Because Trino and Kudu each support types that the other does not, this +connector {ref}`modifies some types ` when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction. + +### Kudu type to Trino type mapping + +The connector maps Kudu types to the corresponding Trino types following +this table: + +```{eval-rst} +.. list-table:: Kudu type to Trino type mapping + :widths: 30, 20 + :header-rows: 1 + + * - Kudu type + - Trino type + * - ``BOOL`` + - ``BOOLEAN`` + * - ``INT8`` + - ``TINYINT`` + * - ``INT16`` + - ``SMALLINT`` + * - ``INT32`` + - ``INTEGER`` + * - ``INT64`` + - ``BIGINT`` + * - ``FLOAT`` + - ``REAL`` + * - ``DOUBLE`` + - ``DOUBLE`` + * - ``DECIMAL(p,s)`` + - ``DECIMAL(p,s)`` + * - ``STRING`` + - ``VARCHAR`` + * - ``BINARY`` + - ``VARBINARY`` + * - ``UNIXTIME_MICROS`` + - ``TIMESTAMP(3)`` +``` + +No other types are supported. + +### Trino type to Kudu type mapping + +The connector maps Trino types to the corresponding Kudu types following +this table: + +```{eval-rst} +.. list-table:: Trino type to Kudu type mapping + :widths: 30, 20, 50 + :header-rows: 1 + + * - Trino type + - Kudu type + - Notes + * - ``BOOLEAN`` + - ``BOOL`` + - + * - ``TINYINT`` + - ``INT8`` + - + * - ``SMALLINT`` + - ``INT16`` + - + * - ``INTEGER`` + - ``INT32`` + - + * - ``BIGINT`` + - ``INT64`` + - + * - ``REAL`` + - ``FLOAT`` + - + * - ``DOUBLE`` + - ``DOUBLE`` + - + * - ``DECIMAL(p,s)`` + - ``DECIMAL(p,s)`` + - Only supported for Kudu server >= 1.7.0 + * - ``VARCHAR`` + - ``STRING`` + - The optional maximum length is lost + * - ``VARBINARY`` + - ``BINARY`` + - + * - ``DATE`` + - ``STRING`` + - + * - ``TIMESTAMP(3)`` + - ``UNIXTIME_MICROS`` + - µs resolution in Kudu column is reduced to ms resolution +``` + +No other types are supported. + +(kudu-sql-support)= + +## SQL support + +The connector provides read and write access to data and metadata in +Kudu. In addition to the {ref}`globally available +` and {ref}`read operation ` +statements, the connector supports the following features: + +- {doc}`/sql/insert`, see also {ref}`kudu-insert` +- {doc}`/sql/delete` +- {doc}`/sql/merge` +- {doc}`/sql/create-table`, see also {ref}`kudu-create-table` +- {doc}`/sql/create-table-as` +- {doc}`/sql/drop-table` +- {doc}`/sql/alter-table`, see also {ref}`kudu-alter-table` +- {doc}`/sql/create-schema`, see also {ref}`kudu-create-schema` +- {doc}`/sql/drop-schema`, see also {ref}`kudu-drop-schema` + +(kudu-insert)= + +### Inserting into tables + +`INSERT INTO ... values` and `INSERT INTO ... select` behave like +`UPSERT`. + +```{include} sql-delete-limitation.fragment +``` + +(kudu-create-schema)= + +### Creating schemas + +`CREATE SCHEMA` is only allowed if schema emulation is enabled. See the +{ref}`behavior-with-schema-emulation` section. + +(kudu-drop-schema)= + +### Dropping schemas + +`DROP SCHEMA` is only allowed if schema emulation is enabled. See the +{ref}`behavior-with-schema-emulation` section. + +(kudu-create-table)= + +### Creating a table + +On creating a Kudu table, you need to provide the columns and their types, of +course, but Kudu needs information about partitioning and optionally +for column encoding and compression. + +Simple Example: + +``` +CREATE TABLE user_events ( + user_id INTEGER WITH (primary_key = true), + event_name VARCHAR WITH (primary_key = true), + message VARCHAR, + details VARCHAR WITH (nullable = true, encoding = 'plain') +) WITH ( + partition_by_hash_columns = ARRAY['user_id'], + partition_by_hash_buckets = 5, + number_of_replicas = 3 +); +``` + +The primary key consists of `user_id` and `event_name`. The table is partitioned into +five partitions by hash values of the column `user_id`, and the `number_of_replicas` is +explicitly set to 3. + +The primary key columns must always be the first columns of the column list. +All columns used in partitions must be part of the primary key. + +The table property `number_of_replicas` is optional. It defines the +number of tablet replicas, and must be an odd number. If it is not specified, +the default replication factor from the Kudu master configuration is used. + +Kudu supports two different kinds of partitioning: hash and range partitioning. +Hash partitioning distributes rows by hash value into one of many buckets. +Range partitions distributes rows using a totally-ordered range partition key. +The concrete range partitions must be created explicitly. +Kudu also supports multi-level partitioning. A table must have at least one +partitioning, either hash or range. It can have at most one range partitioning, +but multiple hash partitioning 'levels'. + +For more details see [Partitioning design](kudu-partitioning-design). + +(kudu-column-properties)= +### Column properties + +Besides column name and type, you can specify some more properties of a column. + +| Column property name | Type | Description | +| -------------------- | --------- | ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| `primary_key` | `BOOLEAN` | If `true`, the column belongs to primary key columns. The Kudu primary key enforces a uniqueness constraint. Inserting a second row with the same primary key results in updating the existing row ('UPSERT'). See also [Primary Key Design] in the Kudu documentation. | +| `nullable` | `BOOLEAN` | If `true`, the value can be null. Primary key columns must not be nullable. | +| `encoding` | `VARCHAR` | The column encoding can help to save storage space and to improve query performance. Kudu uses an auto encoding depending on the column type if not specified. Valid values are: `'auto'`, `'plain'`, `'bitshuffle'`, `'runlength'`, `'prefix'`, `'dictionary'`, `'group_varint'`. See also [Column encoding] in the Kudu documentation. | +| `compression` | `VARCHAR` | The encoded column values can be compressed. Kudu uses a default compression if not specified. Valid values are: `'default'`, `'no'`, `'lz4'`, `'snappy'`, `'zlib'`. See also [Column compression] in the Kudu documentation. | + +Example: + +```sql +CREATE TABLE example_table ( + name VARCHAR WITH (primary_key = true, encoding = 'dictionary', compression = 'snappy'), + index BIGINT WITH (nullable = true, encoding = 'runlength', compression = 'lz4'), + comment VARCHAR WITH (nullable = true, encoding = 'plain', compression = 'default'), + ... +) WITH (...); +``` + +(kudu-alter-table)= + +### Changing tables + +Adding a column to an existing table uses the SQL statement `ALTER TABLE ... ADD COLUMN ...`. +You can specify the same column properties as on creating a table. + +Example: + +``` +ALTER TABLE example_table ADD COLUMN extraInfo VARCHAR WITH (nullable = true, encoding = 'plain') +``` + +See also [Column properties](kudu-column-properties). + +`ALTER TABLE ... RENAME COLUMN` is only allowed if not part of a primary key. + +`ALTER TABLE ... DROP COLUMN` is only allowed if not part of a primary key. + +## Procedures + +- `CALL example.system.add_range_partition` see {ref}`managing-range-partitions` +- `CALL example.system.drop_range_partition` see {ref}`managing-range-partitions` + + +(kudu-partitioning-design)= +### Partitioning design + +A table must have at least one partitioning (either hash or range). +It can have at most one range partitioning, but multiple hash partitioning 'levels'. +For more details see Apache Kudu documentation: [Partitioning]. + +If you create a Kudu table in Trino, the partitioning design is given by +several table properties. + +#### Hash partitioning + +You can provide the first hash partition group with two table properties: + +The `partition_by_hash_columns` defines the column(s) belonging to the +partition group and `partition_by_hash_buckets` the number of partitions to +split the hash values range into. All partition columns must be part of the +primary key. + +Example: + +``` +CREATE TABLE example_table ( + col1 VARCHAR WITH (primary_key=true), + col2 VARCHAR WITH (primary_key=true), + ... +) WITH ( + partition_by_hash_columns = ARRAY['col1', 'col2'], + partition_by_hash_buckets = 4 +) +``` + +This defines a hash partitioning with the columns `col1` and `col2` +distributed over 4 partitions. + +To define two separate hash partition groups, also use the second pair +of table properties named `partition_by_second_hash_columns` and +`partition_by_second_hash_buckets`. + +Example: + +``` +CREATE TABLE example_table ( + col1 VARCHAR WITH (primary_key=true), + col2 VARCHAR WITH (primary_key=true), + ... +) WITH ( + partition_by_hash_columns = ARRAY['col1'], + partition_by_hash_buckets = 2, + partition_by_second_hash_columns = ARRAY['col2'], + partition_by_second_hash_buckets = 3 +) +``` + +This defines a two-level hash partitioning, with the first hash partition group +over the column `col1` distributed over 2 buckets, and the second +hash partition group over the column `col2` distributed over 3 buckets. +As a result you have table with 2 x 3 = 6 partitions. + +#### Range partitioning + +You can provide at most one range partitioning in Apache Kudu. The columns +are defined with the table property `partition_by_range_columns`. +The ranges themselves are given either in the +table property `range_partitions` on creating the table. +Or alternatively, the procedures `kudu.system.add_range_partition` and +`kudu.system.drop_range_partition` can be used to manage range +partitions for existing tables. For both ways see below for more +details. + +Example: + +``` +CREATE TABLE events ( + rack VARCHAR WITH (primary_key=true), + machine VARCHAR WITH (primary_key=true), + event_time TIMESTAMP WITH (primary_key=true), + ... +) WITH ( + partition_by_hash_columns = ARRAY['rack'], + partition_by_hash_buckets = 2, + partition_by_second_hash_columns = ARRAY['machine'], + partition_by_second_hash_buckets = 3, + partition_by_range_columns = ARRAY['event_time'], + range_partitions = '[{"lower": null, "upper": "2018-01-01T00:00:00"}, + {"lower": "2018-01-01T00:00:00", "upper": null}]' +) +``` + +This defines a tree-level partitioning with two hash partition groups and +one range partitioning on the `event_time` column. +Two range partitions are created with a split at “2018-01-01T00:00:00”. + +### Table property `range_partitions` + +With the `range_partitions` table property you specify the concrete +range partitions to be created. The range partition definition itself +must be given in the table property `partition_design` separately. + +Example: + +``` +CREATE TABLE events ( + serialno VARCHAR WITH (primary_key = true), + event_time TIMESTAMP WITH (primary_key = true), + message VARCHAR +) WITH ( + partition_by_hash_columns = ARRAY['serialno'], + partition_by_hash_buckets = 4, + partition_by_range_columns = ARRAY['event_time'], + range_partitions = '[{"lower": null, "upper": "2017-01-01T00:00:00"}, + {"lower": "2017-01-01T00:00:00", "upper": "2017-07-01T00:00:00"}, + {"lower": "2017-07-01T00:00:00", "upper": "2018-01-01T00:00:00"}]' +); +``` + +This creates a table with a hash partition on column `serialno` with 4 +buckets and range partitioning on column `event_time`. Additionally, +three range partitions are created: + +1. for all event_times before the year 2017, lower bound = `null` means it is unbound +2. for the first half of the year 2017 +3. for the second half the year 2017 + +This means any attempt to add rows with `event_time` of year 2018 or greater fails, as no partition is defined. +The next section shows how to define a new range partition for an existing table. + +(managing-range-partitions)= + +#### Managing range partitions + +For existing tables, there are procedures to add and drop a range +partition. + +- adding a range partition + + ```sql + CALL example.system.add_range_partition(,
, ) + ``` + +- dropping a range partition + + ```sql + CALL example.system.drop_range_partition(,
, ) + ``` + + - ``: schema of the table + + - `
`: table names + + - ``: lower and upper bound of the + range partition as JSON string in the form + `'{"lower": , "upper": }'`, or if the range partition + has multiple columns: + `'{"lower": [,...], "upper": [,...]}'`. The + concrete literal for lower and upper bound values are depending on + the column types. + + Examples: + + | Trino data Type | JSON string example | + | --------------- | ---------------------------------------------------------------------------- | + | `BIGINT` | `‘{“lower”: 0, “upper”: 1000000}’` | + | `SMALLINT` | `‘{“lower”: 10, “upper”: null}’` | + | `VARCHAR` | `‘{“lower”: “A”, “upper”: “M”}’` | + | `TIMESTAMP` | `‘{“lower”: “2018-02-01T00:00:00.000”, “upper”: “2018-02-01T12:00:00.000”}’` | + | `BOOLEAN` | `‘{“lower”: false, “upper”: true}’` | + | `VARBINARY` | values encoded as base64 strings | + + To specified an unbounded bound, use the value `null`. + +Example: + +``` +CALL example.system.add_range_partition('example_schema', 'events', '{"lower": "2018-01-01", "upper": "2018-06-01"}') +``` + +This adds a range partition for a table `events` in the schema +`example_schema` with the lower bound `2018-01-01`, more exactly +`2018-01-01T00:00:00.000`, and the upper bound `2018-07-01`. + +Use the SQL statement `SHOW CREATE TABLE` to query the existing +range partitions (they are shown in the table property +`range_partitions`). + +## Limitations + +- Only lower case table and column names in Kudu are supported. + +[apache kudu]: https://kudu.apache.org/ +[column compression]: https://kudu.apache.org/docs/schema_design.html#compression +[column encoding]: https://kudu.apache.org/docs/schema_design.html#encoding +[partitioning]: https://kudu.apache.org/docs/schema_design.html#partitioning +[primary key design]: http://kudu.apache.org/docs/schema_design.html#primary-keys diff --git a/430/_sources/connector/localfile.md.txt b/430/_sources/connector/localfile.md.txt new file mode 100644 index 000000000..6c3e7663a --- /dev/null +++ b/430/_sources/connector/localfile.md.txt @@ -0,0 +1,34 @@ +# Local file connector + +The local file connector allows querying the HTTP request log files stored on +the local file system of each worker. + +## Configuration + +To configure the local file connector, create a catalog properties file under +`etc/catalog` named, for example, `example.properties` with the following +contents: + +```text +connector.name=localfile +``` + +## Configuration properties + +| Property name | Description | +| -------------------------------------- | ------------------------------------------------------------------------------------------ | +| `trino-logs.http-request-log.location` | Directory or file where HTTP request logs are written | +| `trino-logs.http-request-log.pattern` | If the log location is a directory, this glob is used to match file names in the directory | + +## Local file connector schemas and tables + +The local file connector provides a single schema named `logs`. +You can see all the available tables by running `SHOW TABLES`: + +``` +SHOW TABLES FROM example.logs; +``` + +### `http_request_log` + +This table contains the HTTP request logs from each node on the cluster. diff --git a/430/_sources/connector/mariadb.md.txt b/430/_sources/connector/mariadb.md.txt new file mode 100644 index 000000000..2f78a0d70 --- /dev/null +++ b/430/_sources/connector/mariadb.md.txt @@ -0,0 +1,397 @@ +--- +myst: + substitutions: + default_domain_compaction_threshold: '`32`' +--- + +# MariaDB connector + +```{raw} html + +``` + +The MariaDB connector allows querying and creating tables in an external MariaDB +database. + +## Requirements + +To connect to MariaDB, you need: + +- MariaDB version 10.2 or higher. +- Network access from the Trino coordinator and workers to MariaDB. Port + 3306 is the default port. + +## Configuration + +To configure the MariaDB connector, create a catalog properties file in +`etc/catalog` named, for example, `example.properties`, to mount the MariaDB +connector as the `example` catalog. Create the file with the following +contents, replacing the connection properties as appropriate for your setup: + +```text +connector.name=mariadb +connection-url=jdbc:mariadb://example.net:3306 +connection-user=root +connection-password=secret +``` + +The `connection-user` and `connection-password` are typically required and +determine the user credentials for the connection, often a service user. You can +use {doc}`secrets ` to avoid actual values in the catalog +properties files. + +```{include} jdbc-authentication.fragment +``` + +```{include} jdbc-common-configurations.fragment +``` + +```{include} jdbc-domain-compaction-threshold.fragment +``` + +```{include} jdbc-case-insensitive-matching.fragment +``` + +```{include} non-transactional-insert.fragment +``` + +## Querying MariaDB + +The MariaDB connector provides a schema for every MariaDB *database*. +You can see the available MariaDB databases by running `SHOW SCHEMAS`: + +``` +SHOW SCHEMAS FROM example; +``` + +If you have a MariaDB database named `web`, you can view the tables +in this database by running `SHOW TABLES`: + +``` +SHOW TABLES FROM example.web; +``` + +You can see a list of the columns in the `clicks` table in the `web` +database using either of the following: + +``` +DESCRIBE example.web.clicks; +SHOW COLUMNS FROM example.web.clicks; +``` + +Finally, you can access the `clicks` table in the `web` database: + +``` +SELECT * FROM example.web.clicks; +``` + +If you used a different name for your catalog properties file, use +that catalog name instead of `example` in the above examples. + +% mariadb-type-mapping: + +## Type mapping + +Because Trino and MariaDB each support types that the other does not, this +connector {ref}`modifies some types ` when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction. + +### MariaDB type to Trino type mapping + +The connector maps MariaDB types to the corresponding Trino types according +to the following table: + +```{eval-rst} +.. list-table:: MariaDB type to Trino type mapping + :widths: 30, 30, 50 + :header-rows: 1 + + * - MariaDB type + - Trino type + - Notes + * - ``BOOLEAN`` + - ``TINYINT`` + - ``BOOL`` and ``BOOLEAN`` are aliases of ``TINYINT(1)`` + * - ``TINYINT`` + - ``TINYINT`` + - + * - ``TINYINT UNSIGNED`` + - ``SMALLINT`` + - + * - ``SMALLINT`` + - ``SMALLINT`` + - + * - ``SMALLINT UNSIGNED`` + - ``INTEGER`` + - + * - ``INT`` + - ``INTEGER`` + - + * - ``INT UNSIGNED`` + - ``BIGINT`` + - + * - ``BIGINT`` + - ``BIGINT`` + - + * - ``BIGINT UNSIGNED`` + - ``DECIMAL(20, 0)`` + - + * - ``FLOAT`` + - ``REAL`` + - + * - ``DOUBLE`` + - ``DOUBLE`` + - + * - ``DECIMAL(p,s)`` + - ``DECIMAL(p,s)`` + - + * - ``CHAR(n)`` + - ``CHAR(n)`` + - + * - ``TINYTEXT`` + - ``VARCHAR(255)`` + - + * - ``TEXT`` + - ``VARCHAR(65535)`` + - + * - ``MEDIUMTEXT`` + - ``VARCHAR(16777215)`` + - + * - ``LONGTEXT`` + - ``VARCHAR`` + - + * - ``VARCHAR(n)`` + - ``VARCHAR(n)`` + - + * - ``TINYBLOB`` + - ``VARBINARY`` + - + * - ``BLOB`` + - ``VARBINARY`` + - + * - ``MEDIUMBLOB`` + - ``VARBINARY`` + - + * - ``LONGBLOB`` + - ``VARBINARY`` + - + * - ``VARBINARY(n)`` + - ``VARBINARY`` + - + * - ``DATE`` + - ``DATE`` + - + * - ``TIME(n)`` + - ``TIME(n)`` + - + * - ``TIMESTAMP(n)`` + - ``TIMESTAMP(n)`` + - MariaDB stores the current timestamp by default. Enable + `explicit_defaults_for_timestamp + `_ + to avoid implicit default values and use ``NULL`` as the default value. +``` + +No other types are supported. + +### Trino type mapping to MariaDB type mapping + +The connector maps Trino types to the corresponding MariaDB types according +to the following table: + +```{eval-rst} +.. list-table:: Trino type mapping to MariaDB type mapping + :widths: 30, 25, 50 + :header-rows: 1 + + * - Trino type + - MariaDB type + - Notes + * - ``BOOLEAN`` + - ``BOOLEAN`` + - + * - ``TINYINT`` + - ``TINYINT`` + - + * - ``SMALLINT`` + - ``SMALLINT`` + - + * - ``INTEGER`` + - ``INT`` + - + * - ``BIGINT`` + - ``BIGINT`` + - + * - ``REAL`` + - ``FLOAT`` + - + * - ``DOUBLE`` + - ``DOUBLE`` + - + * - ``DECIMAL(p,s)`` + - ``DECIMAL(p,s)`` + - + * - ``CHAR(n)`` + - ``CHAR(n)`` + - + * - ``VARCHAR(255)`` + - ``TINYTEXT`` + - Maps on ``VARCHAR`` of length 255 or less. + * - ``VARCHAR(65535)`` + - ``TEXT`` + - Maps on ``VARCHAR`` of length between 256 and 65535, inclusive. + * - ``VARCHAR(16777215)`` + - ``MEDIUMTEXT`` + - Maps on ``VARCHAR`` of length between 65536 and 16777215, inclusive. + * - ``VARCHAR`` + - ``LONGTEXT`` + - ``VARCHAR`` of length greater than 16777215 and unbounded ``VARCHAR`` map + to ``LONGTEXT``. + * - ``VARBINARY`` + - ``MEDIUMBLOB`` + - + * - ``DATE`` + - ``DATE`` + - + * - ``TIME(n)`` + - ``TIME(n)`` + - + * - ``TIMESTAMP(n)`` + - ``TIMESTAMP(n)`` + - MariaDB stores the current timestamp by default. Enable + `explicit_defaults_for_timestamp + `_ + to avoid implicit default values and use ``NULL`` as the default value. +``` + +No other types are supported. + +Complete list of [MariaDB data types](https://mariadb.com/kb/en/data-types/). + +```{include} jdbc-type-mapping.fragment +``` + +(mariadb-sql-support)= + +## SQL support + +The connector provides read access and write access to data and metadata in +a MariaDB database. In addition to the {ref}`globally available +` and {ref}`read operation ` +statements, the connector supports the following features: + +- {doc}`/sql/insert` +- {doc}`/sql/update` +- {doc}`/sql/delete` +- {doc}`/sql/truncate` +- {doc}`/sql/create-table` +- {doc}`/sql/create-table-as` +- {doc}`/sql/drop-table` +- {doc}`/sql/alter-table` +- {doc}`/sql/create-schema` +- {doc}`/sql/drop-schema` + +```{include} sql-update-limitation.fragment +``` + +```{include} sql-delete-limitation.fragment +``` + +## Table functions + +The connector provides specific {doc}`table functions ` to +access MariaDB. + +(mariadb-query-function)= + +### `query(varchar) -> table` + +The `query` function allows you to query the underlying database directly. It +requires syntax native to MariaDB, because the full query is pushed down and +processed in MariaDB. This can be useful for accessing native features which are +not available in Trino or for improving query performance in situations where +running a query natively may be faster. + +```{include} query-passthrough-warning.fragment +``` + +As an example, query the `example` catalog and select the age of employees by +using `TIMESTAMPDIFF` and `CURDATE`: + +``` +SELECT + age +FROM + TABLE( + example.system.query( + query => 'SELECT + TIMESTAMPDIFF( + YEAR, + date_of_birth, + CURDATE() + ) AS age + FROM + tiny.employees' + ) + ); +``` + +```{include} query-table-function-ordering.fragment +``` + +## Performance + +The connector includes a number of performance improvements, detailed in the +following sections. + +(mariadb-table-statistics)= +### Table statistics + +The MariaDB connector can use [table and column +statistics](/optimizer/statistics) for [cost based +optimizations](/optimizer/cost-based-optimizations) to improve query processing +performance based on the actual data in the data source. + +The statistics are collected by MariaDB and retrieved by the connector. + +To collect statistics for a table, execute the following statement in +MariaDB. + +```text +ANALYZE TABLE table_name; +``` + +Refer to [MariaDB documentation](https://mariadb.com/kb/en/analyze-table/) for +additional information. + +(mariadb-pushdown)= + +### Pushdown + +The connector supports pushdown for a number of operations: + +- {ref}`join-pushdown` +- {ref}`limit-pushdown` +- {ref}`topn-pushdown` + +{ref}`Aggregate pushdown ` for the following functions: + +- {func}`avg` +- {func}`count` +- {func}`max` +- {func}`min` +- {func}`sum` +- {func}`stddev` +- {func}`stddev_pop` +- {func}`stddev_samp` +- {func}`variance` +- {func}`var_pop` +- {func}`var_samp` + +```{include} pushdown-correctness-behavior.fragment +``` + +```{include} no-pushdown-text-type.fragment +``` diff --git a/430/_sources/connector/memory.md.txt b/430/_sources/connector/memory.md.txt new file mode 100644 index 000000000..1a4fcdb62 --- /dev/null +++ b/430/_sources/connector/memory.md.txt @@ -0,0 +1,104 @@ +# Memory connector + +The Memory connector stores all data and metadata in RAM on workers +and both are discarded when Trino restarts. + +## Configuration + +To configure the Memory connector, create a catalog properties file +`etc/catalog/example.properties` with the following contents: + +```text +connector.name=memory +memory.max-data-per-node=128MB +``` + +`memory.max-data-per-node` defines memory limit for pages stored in this +connector per each node (default value is 128MB). + +## Examples + +Create a table using the Memory connector: + +``` +CREATE TABLE example.default.nation AS +SELECT * from tpch.tiny.nation; +``` + +Insert data into a table in the Memory connector: + +``` +INSERT INTO example.default.nation +SELECT * FROM tpch.tiny.nation; +``` + +Select from the Memory connector: + +``` +SELECT * FROM example.default.nation; +``` + +Drop table: + +``` +DROP TABLE example.default.nation; +``` + +(memory-type-mapping)= + +## Type mapping + +Trino supports all data types used within the Memory schemas so no mapping is +required. + +(memory-sql-support)= + +## SQL support + +The connector provides read and write access to temporary data and metadata +stored in memory. In addition to the {ref}`globally available +` and {ref}`read operation ` +statements, the connector supports the following features: + +- {doc}`/sql/insert` +- {doc}`/sql/create-table` +- {doc}`/sql/create-table-as` +- {doc}`/sql/drop-table` +- {doc}`/sql/create-schema` +- {doc}`/sql/drop-schema` +- {doc}`/sql/comment` + +### DROP TABLE + +Upon execution of a `DROP TABLE` operation, memory is not released +immediately. It is instead released after the next write operation to the +catalog. + +(memory-dynamic-filtering)= + +## Dynamic filtering + +The Memory connector supports the {doc}`dynamic filtering ` optimization. +Dynamic filters are pushed into local table scan on worker nodes for broadcast joins. + +### Delayed execution for dynamic filters + +For the Memory connector, a table scan is delayed until the collection of dynamic filters. +This can be disabled by using the configuration property `memory.enable-lazy-dynamic-filtering` +in the catalog file. + +## Limitations + +- When one worker fails/restarts, all data that was stored in its + memory is lost. To prevent silent data loss the + connector throws an error on any read access to such + corrupted table. +- When a query fails for any reason during writing to memory table, + the table enters an undefined state. The table should be dropped + and recreated manually. Reading attempts from the table may fail, + or may return partial data. +- When the coordinator fails/restarts, all metadata about tables is + lost. The tables remain on the workers, but become inaccessible. +- This connector does not work properly with multiple + coordinators, since each coordinator has different + metadata. diff --git a/430/_sources/connector/metastores.md.txt b/430/_sources/connector/metastores.md.txt new file mode 100644 index 000000000..986117228 --- /dev/null +++ b/430/_sources/connector/metastores.md.txt @@ -0,0 +1,469 @@ +# Metastores + +Object storage access is mediated through a *metastore*. Metastores provide +information on directory structure, file format, and metadata about the stored +data. Object storage connectors support the use of one or more metastores. A +supported metastore is required to use any object storage connector. + +Additional configuration is required in order to access tables with Athena +partition projection metadata or implement first class support for Avro tables. +These requirements are discussed later in this topic. + +(general-metastore-properties)= + +## General metastore configuration properties + +The following table describes general metastore configuration properties, most +of which are used with either metastore. + +At a minimum, each Delta Lake, Hive or Hudi object storage catalog file must set +the `hive.metastore` configuration property to define the type of metastore to +use. Iceberg catalogs instead use the `iceberg.catalog.type` configuration +property to define the type of metastore to use. + +Additional configuration properties specific to the Thrift and Glue Metastores +are also available. They are discussed later in this topic. + +```{eval-rst} +.. list-table:: General metastore configuration properties + :widths: 35, 50, 15 + :header-rows: 1 + + * - Property Name + - Description + - Default + * - ``hive.metastore`` + - The type of Hive metastore to use. Trino currently supports the default + Hive Thrift metastore (``thrift``), and the AWS Glue Catalog (``glue``) + as metadata sources. You must use this for all object storage catalogs + except Iceberg. + - ``thrift`` + * - ``iceberg.catalog.type`` + - The Iceberg table format manages most metadata in metadata files in the + object storage itself. A small amount of metadata, however, still + requires the use of a metastore. In the Iceberg ecosystem, these smaller + metastores are called Iceberg metadata catalogs, or just catalogs. The + examples in each subsection depict the contents of a Trino catalog file + that uses the the Iceberg connector to configures different Iceberg + metadata catalogs. + + You must set this property in all Iceberg catalog property files. + Valid values are ``HIVE_METASTORE``, ``GLUE``, ``JDBC``, ``REST``, and + ``NESSIE``. + - + * - ``hive.metastore-cache.cache-partitions`` + - Enable caching for partition metadata. You can disable caching to avoid + inconsistent behavior that results from it. + - ``true`` + * - ``hive.metastore-cache-ttl`` + - Duration of how long cached metastore data is considered valid. + - ``0s`` + * - ``hive.metastore-stats-cache-ttl`` + - Duration of how long cached metastore statistics are considered valid. + If ``hive.metastore-cache-ttl`` is larger then it takes precedence + over ``hive.metastore-stats-cache-ttl``. + - ``5m`` + * - ``hive.metastore-cache-maximum-size`` + - Maximum number of metastore data objects in the Hive metastore cache. + - ``10000`` + * - ``hive.metastore-refresh-interval`` + - Asynchronously refresh cached metastore data after access if it is older + than this but is not yet expired, allowing subsequent accesses to see + fresh data. + - + * - ``hive.metastore-refresh-max-threads`` + - Maximum threads used to refresh cached metastore data. + - ``10`` + * - ``hive.metastore-timeout`` + - Timeout for Hive metastore requests. + - ``10s`` + * - ``hive.hide-delta-lake-tables`` + - Controls whether to hide Delta Lake tables in table listings. Currently + applies only when using the AWS Glue metastore. + - ``false`` +``` + +(hive-thrift-metastore)= + +## Thrift metastore configuration properties + +In order to use a Hive Thrift metastore, you must configure the metastore with +`hive.metastore=thrift` and provide further details with the following +properties: + +```{eval-rst} +.. list-table:: Thrift metastore configuration properties + :widths: 35, 50, 15 + :header-rows: 1 + + * - Property name + - Description + - Default + * - ``hive.metastore.uri`` + - The URIs of the Hive metastore to connect to using the Thrift protocol. + If a comma-separated list of URIs is provided, the first URI is used by + default, and the rest of the URIs are fallback metastores. This property + is required. Example: ``thrift://192.0.2.3:9083`` or + ``thrift://192.0.2.3:9083,thrift://192.0.2.4:9083`` + - + * - ``hive.metastore.username`` + - The username Trino uses to access the Hive metastore. + - + * - ``hive.metastore.authentication.type`` + - Hive metastore authentication type. Possible values are ``NONE`` or + ``KERBEROS``. + - ``NONE`` + * - ``hive.metastore.thrift.impersonation.enabled`` + - Enable Hive metastore end user impersonation. + - + * - ``hive.metastore.thrift.use-spark-table-statistics-fallback`` + - Enable usage of table statistics generated by Apache Spark when Hive + table statistics are not available. + - ``true`` + * - ``hive.metastore.thrift.delegation-token.cache-ttl`` + - Time to live delegation token cache for metastore. + - ``1h`` + * - ``hive.metastore.thrift.delegation-token.cache-maximum-size`` + - Delegation token cache maximum size. + - ``1000`` + * - ``hive.metastore.thrift.client.ssl.enabled`` + - Use SSL when connecting to metastore. + - ``false`` + * - ``hive.metastore.thrift.client.ssl.key`` + - Path to private key and client certification (key store). + - + * - ``hive.metastore.thrift.client.ssl.key-password`` + - Password for the private key. + - + * - ``hive.metastore.thrift.client.ssl.trust-certificate`` + - Path to the server certificate chain (trust store). Required when SSL is + enabled. + - + * - ``hive.metastore.thrift.client.ssl.trust-certificate-password`` + - Password for the trust store. + - + * - ``hive.metastore.thrift.batch-fetch.enabled`` + - Enable fetching tables and views from all schemas in a single request. + - ``true`` + * - ``hive.metastore.service.principal`` + - The Kerberos principal of the Hive metastore service. + - + * - ``hive.metastore.client.principal`` + - The Kerberos principal that Trino uses when connecting to the Hive + metastore service. + - + * - ``hive.metastore.client.keytab`` + - Hive metastore client keytab location. + - + * - ``hive.metastore.thrift.delete-files-on-drop`` + - Actively delete the files for managed tables when performing drop table + or partition operations, for cases when the metastore does not delete the + files. + - ``false`` + * - ``hive.metastore.thrift.assume-canonical-partition-keys`` + - Allow the metastore to assume that the values of partition columns can be + converted to string values. This can lead to performance improvements in + queries which apply filters on the partition columns. Partition keys with + a ``TIMESTAMP`` type do not get canonicalized. + - ``false`` + * - ``hive.metastore.thrift.client.socks-proxy`` + - SOCKS proxy to use for the Thrift Hive metastore. + - + * - ``hive.metastore.thrift.client.max-retries`` + - Maximum number of retry attempts for metastore requests. + - ``9`` + * - ``hive.metastore.thrift.client.backoff-scale-factor`` + - Scale factor for metastore request retry delay. + - ``2.0`` + * - ``hive.metastore.thrift.client.max-retry-time`` + - Total allowed time limit for a metastore request to be retried. + - ``30s`` + * - ``hive.metastore.thrift.client.min-backoff-delay`` + - Minimum delay between metastore request retries. + - ``1s`` + * - ``hive.metastore.thrift.client.max-backoff-delay`` + - Maximum delay between metastore request retries. + - ``1s`` + * - ``hive.metastore.thrift.txn-lock-max-wait`` + - Maximum time to wait to acquire hive transaction lock. + - ``10m`` +``` + +(hive-glue-metastore)= + +## AWS Glue catalog configuration properties + +In order to use an AWS Glue catalog, you must configure your catalog file as +follows: + +`hive.metastore=glue` and provide further details with the following +properties: + +```{eval-rst} +.. list-table:: AWS Glue catalog configuration properties + :widths: 35, 50, 15 + :header-rows: 1 + + * - Property Name + - Description + - Default + * - ``hive.metastore.glue.region`` + - AWS region of the Glue Catalog. This is required when not running in + EC2, or when the catalog is in a different region. Example: + ``us-east-1`` + - + * - ``hive.metastore.glue.endpoint-url`` + - Glue API endpoint URL (optional). Example: + ``https://glue.us-east-1.amazonaws.com`` + - + * - ``hive.metastore.glue.sts.region`` + - AWS region of the STS service to authenticate with. This is required + when running in a GovCloud region. Example: ``us-gov-east-1`` + - + * - ``hive.metastore.glue.proxy-api-id`` + - The ID of the Glue Proxy API, when accessing Glue via an VPC endpoint in + API Gateway. + - + * - ``hive.metastore.glue.sts.endpoint`` + - STS endpoint URL to use when authenticating to Glue (optional). Example: + ``https://sts.us-gov-east-1.amazonaws.com`` + - + * - ``hive.metastore.glue.pin-client-to-current-region`` + - Pin Glue requests to the same region as the EC2 instance where Trino is + running. + - ``false`` + * - ``hive.metastore.glue.max-connections`` + - Max number of concurrent connections to Glue. + - ``30`` + * - ``hive.metastore.glue.max-error-retries`` + - Maximum number of error retries for the Glue client. + - ``10`` + * - ``hive.metastore.glue.default-warehouse-dir`` + - Default warehouse directory for schemas created without an explicit + ``location`` property. + - + * - ``hive.metastore.glue.aws-credentials-provider`` + - Fully qualified name of the Java class to use for obtaining AWS + credentials. Can be used to supply a custom credentials provider. + - + * - ``hive.metastore.glue.aws-access-key`` + - AWS access key to use to connect to the Glue Catalog. If specified along + with ``hive.metastore.glue.aws-secret-key``, this parameter takes + precedence over ``hive.metastore.glue.iam-role``. + - + * - ``hive.metastore.glue.aws-secret-key`` + - AWS secret key to use to connect to the Glue Catalog. If specified along + with ``hive.metastore.glue.aws-access-key``, this parameter takes + precedence over ``hive.metastore.glue.iam-role``. + - + * - ``hive.metastore.glue.catalogid`` + - The ID of the Glue Catalog in which the metadata database resides. + - + * - ``hive.metastore.glue.iam-role`` + - ARN of an IAM role to assume when connecting to the Glue Catalog. + - + * - ``hive.metastore.glue.external-id`` + - External ID for the IAM role trust policy when connecting to the Glue + Catalog. + - + * - ``hive.metastore.glue.partitions-segments`` + - Number of segments for partitioned Glue tables. + - ``5`` + * - ``hive.metastore.glue.get-partition-threads`` + - Number of threads for parallel partition fetches from Glue. + - ``20`` + * - ``hive.metastore.glue.read-statistics-threads`` + - Number of threads for parallel statistic fetches from Glue. + - ``5`` + * - ``hive.metastore.glue.write-statistics-threads`` + - Number of threads for parallel statistic writes to Glue. + - ``5`` +``` + +(iceberg-glue-catalog)= + +### Iceberg-specific Glue catalog configuration properties + +When using the Glue catalog, the Iceberg connector supports the same +{ref}`general Glue configuration properties ` as previously +described with the following additional property: + +```{eval-rst} +.. list-table:: Iceberg Glue catalog configuration property + :widths: 35, 50, 15 + :header-rows: 1 + + * - Property name + - Description + - Default + * - ``iceberg.glue.skip-archive`` + - Skip archiving an old table version when creating a new version in a + commit. See `AWS Glue Skip Archive + `_. + - ``false`` +``` + +## Iceberg-specific metastores + +The Iceberg table format manages most metadata in metadata files in the object +storage itself. A small amount of metadata, however, still requires the use of a +metastore. In the Iceberg ecosystem, these smaller metastores are called Iceberg +metadata catalogs, or just catalogs. + +You can use a general metastore such as an HMS or AWS Glue, or you can use the +Iceberg-specific REST, Nessie or JDBC metadata catalogs, as discussed in this +section. + +(iceberg-rest-catalog)= + +### REST catalog + +In order to use the Iceberg REST catalog, configure the catalog type +with `iceberg.catalog.type=rest`, and provide further details with the +following properties: + +```{eval-rst} +.. list-table:: Iceberg REST catalog configuration properties + :widths: 40, 60 + :header-rows: 1 + + * - Property name + - Description + * - ``iceberg.rest-catalog.uri`` + - REST server API endpoint URI (required). + Example: ``http://iceberg-with-rest:8181`` + * - ``iceberg.rest-catalog.warehouse`` + - Warehouse identifier/location for the catalog (optional). + Example: ``s3://my_bucket/warehouse_location`` + * - ``iceberg.rest-catalog.security`` + - The type of security to use (default: ``NONE``). ``OAUTH2`` requires + either a ``token`` or ``credential``. Example: ``OAUTH2`` + * - ``iceberg.rest-catalog.session`` + - Session information included when communicating with the REST Catalog. + Options are ``NONE`` or ``USER`` (default: ``NONE``). + * - ``iceberg.rest-catalog.oauth2.token`` + - The bearer token used for interactions with the server. A + ``token`` or ``credential`` is required for ``OAUTH2`` security. + Example: ``AbCdEf123456`` + * - ``iceberg.rest-catalog.oauth2.credential`` + - The credential to exchange for a token in the OAuth2 client credentials + flow with the server. A ``token`` or ``credential`` is required for + ``OAUTH2`` security. Example: ``AbCdEf123456`` +``` + +The following example shows a minimal catalog configuration using an Iceberg +REST metadata catalog: + +```properties +connector.name=iceberg +iceberg.catalog.type=rest +iceberg.rest-catalog.uri=http://iceberg-with-rest:8181 +``` + +The REST catalog does not support {doc}`views` or +{doc}`materialized views`. + +(iceberg-jdbc-catalog)= + +### JDBC catalog + +The Iceberg REST catalog is supported for the Iceberg connector. At a minimum, +`iceberg.jdbc-catalog.driver-class`, `iceberg.jdbc-catalog.connection-url` +and `iceberg.jdbc-catalog.catalog-name` must be configured. When using any +database besides PostgreSQL, a JDBC driver jar file must be placed in the plugin +directory. + +:::{warning} +The JDBC catalog may have compatibility issues if Iceberg introduces breaking +changes in the future. Consider the {ref}`REST catalog +` as an alternative solution. +::: + +At a minimum, `iceberg.jdbc-catalog.driver-class`, +`iceberg.jdbc-catalog.connection-url`, and +`iceberg.jdbc-catalog.catalog-name` must be configured. When using any +database besides PostgreSQL, a JDBC driver jar file must be placed in the plugin +directory. The following example shows a minimal catalog configuration using an +Iceberg REST metadata catalog: + +```text +connector.name=iceberg +iceberg.catalog.type=jdbc +iceberg.jdbc-catalog.catalog-name=test +iceberg.jdbc-catalog.driver-class=org.postgresql.Driver +iceberg.jdbc-catalog.connection-url=jdbc:postgresql://example.net:5432/database +iceberg.jdbc-catalog.connection-user=admin +iceberg.jdbc-catalog.connection-password=test +iceberg.jdbc-catalog.default-warehouse-dir=s3://bucket +``` + +The JDBC catalog does not support {doc}`views` or +{doc}`materialized views`. + +(iceberg-nessie-catalog)= + +### Nessie catalog + +In order to use a Nessie catalog, configure the catalog type with +`iceberg.catalog.type=nessie` and provide further details with the following +properties: + +```{eval-rst} +.. list-table:: Nessie catalog configuration properties + :widths: 40, 60 + :header-rows: 1 + + * - Property name + - Description + * - ``iceberg.nessie-catalog.uri`` + - Nessie API endpoint URI (required). + Example: ``https://localhost:19120/api/v1`` + * - ``iceberg.nessie-catalog.ref`` + - The branch/tag to use for Nessie, defaults to ``main``. + * - ``iceberg.nessie-catalog.default-warehouse-dir`` + - Default warehouse directory for schemas created without an explicit + ``location`` property. Example: ``/tmp`` +``` + +```text +connector.name=iceberg +iceberg.catalog.type=nessie +iceberg.nessie-catalog.uri=https://localhost:19120/api/v1 +iceberg.nessie-catalog.default-warehouse-dir=/tmp +``` + +(partition-projection)= + +## Access tables with Athena partition projection metadata + +[Partition projection](https://docs.aws.amazon.com/athena/latest/ug/partition-projection.html) +is a feature of AWS Athena often used to speed up query processing with highly +partitioned tables when using the Hive connector. + +Trino supports partition projection table properties stored in the Hive +metastore or Glue catalog, and it reimplements this functionality. Currently, +there is a limitation in comparison to AWS Athena for date projection, as it +only supports intervals of `DAYS`, `HOURS`, `MINUTES`, and `SECONDS`. + +If there are any compatibility issues blocking access to a requested table when +partition projection is enabled, set the +`partition_projection_ignore` table property to `true` for a table to bypass +any errors. + +Refer to {ref}`hive-table-properties` and {ref}`hive-column-properties` for +configuration of partition projection. + +## Configure metastore for Avro + +For catalogs using the Hive connector, you must add the following property +definition to the Hive metastore configuration file `hive-site.xml` and +restart the metastore service to enable first-class support for Avro tables when +using Hive 3.x: + +```xml + + + metastore.storage.schema.reader.impl + org.apache.hadoop.hive.metastore.SerDeStorageSchemaReader + +``` diff --git a/430/_sources/connector/mongodb.md.txt b/430/_sources/connector/mongodb.md.txt new file mode 100644 index 000000000..d0953d936 --- /dev/null +++ b/430/_sources/connector/mongodb.md.txt @@ -0,0 +1,510 @@ +# MongoDB connector + +```{raw} html + +``` + +The `mongodb` connector allows the use of [MongoDB](https://www.mongodb.com/) collections as tables in Trino. + +## Requirements + +To connect to MongoDB, you need: + +- MongoDB 4.2 or higher. +- Network access from the Trino coordinator and workers to MongoDB. + Port 27017 is the default port. +- Write access to the {ref}`schema information collection ` + in MongoDB. + +## Configuration + +To configure the MongoDB connector, create a catalog properties file +`etc/catalog/example.properties` with the following contents, +replacing the properties as appropriate: + +```text +connector.name=mongodb +mongodb.connection-url=mongodb://user:pass@sample.host:27017/ +``` + +### Multiple MongoDB clusters + +You can have as many catalogs as you need, so if you have additional +MongoDB clusters, simply add another properties file to `etc/catalog` +with a different name, making sure it ends in `.properties`). For +example, if you name the property file `sales.properties`, Trino +will create a catalog named `sales` using the configured connector. + +## Configuration properties + +The following configuration properties are available: + +| Property name | Description | +| ---------------------------------------- | -------------------------------------------------------------------------- | +| `mongodb.connection-url` | The connection url that the driver uses to connect to a MongoDB deployment | +| `mongodb.schema-collection` | A collection which contains schema information | +| `mongodb.case-insensitive-name-matching` | Match database and collection names case insensitively | +| `mongodb.min-connections-per-host` | The minimum size of the connection pool per host | +| `mongodb.connections-per-host` | The maximum size of the connection pool per host | +| `mongodb.max-wait-time` | The maximum wait time | +| `mongodb.max-connection-idle-time` | The maximum idle time of a pooled connection | +| `mongodb.connection-timeout` | The socket connect timeout | +| `mongodb.socket-timeout` | The socket timeout | +| `mongodb.tls.enabled` | Use TLS/SSL for connections to mongod/mongos | +| `mongodb.tls.keystore-path` | Path to the or JKS key store | +| `mongodb.tls.truststore-path` | Path to the or JKS trust store | +| `mongodb.tls.keystore-password` | Password for the key store | +| `mongodb.tls.truststore-password` | Password for the trust store | +| `mongodb.read-preference` | The read preference | +| `mongodb.write-concern` | The write concern | +| `mongodb.required-replica-set` | The required replica set name | +| `mongodb.cursor-batch-size` | The number of elements to return in a batch | + +### `mongodb.connection-url` + +A connection string containing the protocol, credential, and host info for use +inconnection to your MongoDB deployment. + +For example, the connection string may use the format +`mongodb://:@:/?` or +`mongodb+srv://:@/?`, depending on the protocol +used. The user/pass credentials must be for a user with write access to the +{ref}`schema information collection `. + +See the [MongoDB Connection URI](https://docs.mongodb.com/drivers/java/sync/current/fundamentals/connection/#connection-uri) for more information. + +This property is required; there is no default. A connection URL must be +provided to connect to a MongoDB deployment. + +### `mongodb.schema-collection` + +As MongoDB is a document database, there is no fixed schema information in the system. So a special collection in each MongoDB database should define the schema of all tables. Please refer the {ref}`table-definition-label` section for the details. + +At startup, the connector tries to guess the data type of fields based on the {ref}`type mapping `. + +The initial guess can be incorrect for your specific collection. In that case, you need to modify it manually. Please refer the {ref}`table-definition-label` section for the details. + +Creating new tables using `CREATE TABLE` and `CREATE TABLE AS SELECT` automatically create an entry for you. + +This property is optional; the default is `_schema`. + +### `mongodb.case-insensitive-name-matching` + +Match database and collection names case insensitively. + +This property is optional; the default is `false`. + +### `mongodb.min-connections-per-host` + +The minimum number of connections per host for this MongoClient instance. Those connections are kept in a pool when idle, and the pool ensures over time that it contains at least this minimum number. + +This property is optional; the default is `0`. + +### `mongodb.connections-per-host` + +The maximum number of connections allowed per host for this MongoClient instance. Those connections are kept in a pool when idle. Once the pool is exhausted, any operation requiring a connection blocks waiting for an available connection. + +This property is optional; the default is `100`. + +### `mongodb.max-wait-time` + +The maximum wait time in milliseconds, that a thread may wait for a connection to become available. +A value of `0` means that it does not wait. A negative value means to wait indefinitely for a connection to become available. + +This property is optional; the default is `120000`. + +### `mongodb.max-connection-idle-time` + +The maximum idle time of a pooled connection in milliseconds. A value of `0` indicates no limit to the idle time. +A pooled connection that has exceeded its idle time will be closed and replaced when necessary by a new connection. + +This property is optional; the default is `0`. + +### `mongodb.connection-timeout` + +The connection timeout in milliseconds. A value of `0` means no timeout. It is used solely when establishing a new connection. + +This property is optional; the default is `10000`. + +### `mongodb.socket-timeout` + +The socket timeout in milliseconds. It is used for I/O socket read and write operations. + +This property is optional; the default is `0` and means no timeout. + +### `mongodb.tls.enabled` + +This flag enables TLS connections to MongoDB servers. + +This property is optional; the default is `false`. + +### `mongodb.tls.keystore-path` + +The path to the {doc}`PEM ` or +{doc}`JKS ` key store. + +This property is optional. + +### `mongodb.tls.truststore-path` + +The path to {doc}`PEM ` or +{doc}`JKS ` trust store. + +This property is optional. + +### `mongodb.tls.keystore-password` + +The key password for the key store specified by `mongodb.tls.keystore-path`. + +This property is optional. + +### `mongodb.tls.truststore-password` + +The key password for the trust store specified by `mongodb.tls.truststore-path`. + +This property is optional. + +### `mongodb.read-preference` + +The read preference to use for queries, map-reduce, aggregation, and count. +The available values are `PRIMARY`, `PRIMARY_PREFERRED`, `SECONDARY`, `SECONDARY_PREFERRED` and `NEAREST`. + +This property is optional; the default is `PRIMARY`. + +### `mongodb.write-concern` + +The write concern to use. The available values are +`ACKNOWLEDGED`, `JOURNALED`, `MAJORITY` and `UNACKNOWLEDGED`. + +This property is optional; the default is `ACKNOWLEDGED`. + +### `mongodb.required-replica-set` + +The required replica set name. With this option set, the MongoClient instance performs the following actions: + +``` +#. Connect in replica set mode, and discover all members of the set based on the given servers +#. Make sure that the set name reported by all members matches the required set name. +#. Refuse to service any requests, if authenticated user is not part of a replica set with the required name. +``` + +This property is optional; no default value. + +### `mongodb.cursor-batch-size` + +Limits the number of elements returned in one batch. A cursor typically fetches a batch of result objects and stores them locally. +If batchSize is 0, Driver's default are used. +If batchSize is positive, it represents the size of each batch of objects retrieved. It can be adjusted to optimize performance and limit data transfer. +If batchSize is negative, it limits the number of objects returned, that fit within the max batch size limit (usually 4MB), and the cursor is closed. For example if batchSize is -10, then the server returns a maximum of 10 documents, and as many as can fit in 4MB, then closes the cursor. + +:::{note} +Do not use a batch size of `1`. +::: + +This property is optional; the default is `0`. + +(table-definition-label)= + +## Table definition + +MongoDB maintains table definitions on the special collection where `mongodb.schema-collection` configuration value specifies. + +:::{note} +The plugin cannot detect that a collection has been deleted. You must +delete the entry by executing `db.getCollection("_schema").remove( { table: +deleted_table_name })` in the MongoDB Shell. You can also drop a collection in +Trino by running `DROP TABLE table_name`. +::: + +A schema collection consists of a MongoDB document for a table. + +```text +{ + "table": ..., + "fields": [ + { "name" : ..., + "type" : "varchar|bigint|boolean|double|date|array(bigint)|...", + "hidden" : false }, + ... + ] + } +} +``` + +The connector quotes the fields for a row type when auto-generating the schema; +however, the auto-generated schema must be corrected manually in the collection +to match the information in the tables. + +Manually altered fields must be explicitly quoted, for example, `row("UpperCase" +varchar)`. + +| Field | Required | Type | Description | +| -------- | -------- | ------ | ------------------------------------------------------------------------------------------- | +| `table` | required | string | Trino table name | +| `fields` | required | array | A list of field definitions. Each field definition creates a new column in the Trino table. | + +Each field definition: + +```text +{ + "name": ..., + "type": ..., + "hidden": ... +} +``` + +| Field | Required | Type | Description | +| -------- | -------- | ------- | ---------------------------------------------------------------------------------- | +| `name` | required | string | Name of the column in the Trino table. | +| `type` | required | string | Trino type of the column. | +| `hidden` | optional | boolean | Hides the column from `DESCRIBE
` and `SELECT *`. Defaults to `false`. | + +There is no limit on field descriptions for either key or message. + +## ObjectId + +MongoDB collection has the special field `_id`. The connector tries to follow the same rules for this special field, so there will be hidden field `_id`. + +```sql +CREATE TABLE IF NOT EXISTS orders ( + orderkey BIGINT, + orderstatus VARCHAR, + totalprice DOUBLE, + orderdate DATE +); + +INSERT INTO orders VALUES(1, 'bad', 50.0, current_date); +INSERT INTO orders VALUES(2, 'good', 100.0, current_date); +SELECT _id, * FROM orders; +``` + +```text + _id | orderkey | orderstatus | totalprice | orderdate +-------------------------------------+----------+-------------+------------+------------ + 55 b1 51 63 38 64 d6 43 8c 61 a9 ce | 1 | bad | 50.0 | 2015-07-23 + 55 b1 51 67 38 64 d6 43 8c 61 a9 cf | 2 | good | 100.0 | 2015-07-23 +(2 rows) +``` + +```sql +SELECT _id, * FROM orders WHERE _id = ObjectId('55b151633864d6438c61a9ce'); +``` + +```text + _id | orderkey | orderstatus | totalprice | orderdate +-------------------------------------+----------+-------------+------------+------------ + 55 b1 51 63 38 64 d6 43 8c 61 a9 ce | 1 | bad | 50.0 | 2015-07-23 +(1 row) +``` + +You can render the `_id` field to readable values with a cast to `VARCHAR`: + +```sql +SELECT CAST(_id AS VARCHAR), * FROM orders WHERE _id = ObjectId('55b151633864d6438c61a9ce'); +``` + +```text + _id | orderkey | orderstatus | totalprice | orderdate +---------------------------+----------+-------------+------------+------------ + 55b151633864d6438c61a9ce | 1 | bad | 50.0 | 2015-07-23 +(1 row) +``` + +### ObjectId timestamp functions + +The first four bytes of each [ObjectId](https://docs.mongodb.com/manual/reference/method/ObjectId) represent +an embedded timestamp of its creation time. Trino provides a couple of functions to take advantage of this MongoDB feature. + +```{eval-rst} +.. function:: objectid_timestamp(ObjectId) -> timestamp + + Extracts the TIMESTAMP WITH TIME ZONE from a given ObjectId:: + + SELECT objectid_timestamp(ObjectId('507f191e810c19729de860ea')); + -- 2012-10-17 20:46:22.000 UTC +``` + +```{eval-rst} +.. function:: timestamp_objectid(timestamp) -> ObjectId + + Creates an ObjectId from a TIMESTAMP WITH TIME ZONE:: + + SELECT timestamp_objectid(TIMESTAMP '2021-08-07 17:51:36 +00:00'); + -- 61 0e c8 28 00 00 00 00 00 00 00 00 +``` + +In MongoDB, you can filter all the documents created after `2021-08-07 17:51:36` +with a query like this: + +```text +db.collection.find({"_id": {"$gt": ObjectId("610ec8280000000000000000")}}) +``` + +In Trino, the same can be achieved with this query: + +```sql +SELECT * +FROM collection +WHERE _id > timestamp_objectid(TIMESTAMP '2021-08-07 17:51:36 +00:00'); +``` + +(mongodb-type-mapping)= + +## Type mapping + +Because Trino and MongoDB each support types that the other does not, this +connector {ref}`modifies some types ` when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction. + +### MongoDB to Trino type mapping + +The connector maps MongoDB types to the corresponding Trino types following +this table: + +```{eval-rst} +.. list-table:: MongoDB to Trino type mapping + :widths: 30, 20, 50 + :header-rows: 1 + + * - MongoDB type + - Trino type + - Notes + * - ``Boolean`` + - ``BOOLEAN`` + - + * - ``Int32`` + - ``BIGINT`` + - + * - ``Int64`` + - ``BIGINT`` + - + * - ``Double`` + - ``DOUBLE`` + - + * - ``Decimal128`` + - ``DECIMAL(p, s)`` + - + * - ``Date`` + - ``TIMESTAMP(3)`` + - + * - ``String`` + - ``VARCHAR`` + - + * - ``Binary`` + - ``VARBINARY`` + - + * - ``ObjectId`` + - ``ObjectId`` + - + * - ``Object`` + - ``ROW`` + - + * - ``Array`` + - ``ARRAY`` + - Map to ``ROW`` if the element type is not unique. + * - ``DBRef`` + - ``ROW`` + - +``` + +No other types are supported. + +### Trino to MongoDB type mapping + +The connector maps Trino types to the corresponding MongoDB types following +this table: + +```{eval-rst} +.. list-table:: Trino to MongoDB type mapping + :widths: 30, 20 + :header-rows: 1 + + * - Trino type + - MongoDB type + * - ``BOOLEAN`` + - ``Boolean`` + * - ``BIGINT`` + - ``Int64`` + * - ``DOUBLE`` + - ``Double`` + * - ``DECIMAL(p, s)`` + - ``Decimal128`` + * - ``TIMESTAMP(3)`` + - ``Date`` + * - ``VARCHAR`` + - ``String`` + * - ``VARBINARY`` + - ``Binary`` + * - ``ObjectId`` + - ``ObjectId`` + * - ``ROW`` + - ``Object`` + * - ``ARRAY`` + - ``Array`` +``` + +No other types are supported. + +(mongodb-sql-support)= + +## SQL support + +The connector provides read and write access to data and metadata in +MongoDB. In addition to the {ref}`globally available +` and {ref}`read operation ` +statements, the connector supports the following features: + +- {doc}`/sql/insert` +- {doc}`/sql/delete` +- {doc}`/sql/create-table` +- {doc}`/sql/create-table-as` +- {doc}`/sql/drop-table` +- {doc}`/sql/alter-table` +- {doc}`/sql/create-schema` +- {doc}`/sql/drop-schema` +- {doc}`/sql/comment` + +### ALTER TABLE + +The connector supports `ALTER TABLE RENAME TO`, `ALTER TABLE ADD COLUMN` +and `ALTER TABLE DROP COLUMN` operations. +Other uses of `ALTER TABLE` are not supported. + +(mongodb-fte-support)= + +## Fault-tolerant execution support + +The connector supports {doc}`/admin/fault-tolerant-execution` of query +processing. Read and write operations are both supported with any retry policy. + +## Table functions + +The connector provides specific {doc}`table functions ` to +access MongoDB. + +(mongodb-query-function)= + +### `query(database, collection, filter) -> table` + +The `query` function allows you to query the underlying MongoDB directly. It +requires syntax native to MongoDB, because the full query is pushed down and +processed by MongoDB. This can be useful for accessing native features which are +not available in Trino or for improving query performance in situations where +running a query natively may be faster. + +For example, get all rows where `regionkey` field is 0: + +``` +SELECT + * +FROM + TABLE( + example.system.query( + database => 'tpch', + collection => 'region', + filter => '{ regionkey: 0 }' + ) + ); +``` diff --git a/430/_sources/connector/mysql.md.txt b/430/_sources/connector/mysql.md.txt new file mode 100644 index 000000000..a2e9fd42d --- /dev/null +++ b/430/_sources/connector/mysql.md.txt @@ -0,0 +1,487 @@ +--- +myst: + substitutions: + default_domain_compaction_threshold: '`32`' +--- + +# MySQL connector + +```{raw} html + +``` + +The MySQL connector allows querying and creating tables in an external +[MySQL](https://www.mysql.com/) instance. This can be used to join data between different +systems like MySQL and Hive, or between two different MySQL instances. + +## Requirements + +To connect to MySQL, you need: + +- MySQL 5.7, 8.0 or higher. +- Network access from the Trino coordinator and workers to MySQL. + Port 3306 is the default port. + +## Configuration + +To configure the MySQL connector, create a catalog properties file in +`etc/catalog` named, for example, `example.properties`, to mount the MySQL +connector as the `mysql` catalog. Create the file with the following contents, +replacing the connection properties as appropriate for your setup: + +```text +connector.name=mysql +connection-url=jdbc:mysql://example.net:3306 +connection-user=root +connection-password=secret +``` + +The `connection-url` defines the connection information and parameters to pass +to the MySQL JDBC driver. The supported parameters for the URL are +available in the [MySQL Developer Guide](https://dev.mysql.com/doc/connector-j/8.0/en/connector-j-reference-configuration-properties.html). + +For example, the following `connection-url` allows you to require encrypted +connections to the MySQL server: + +```text +connection-url=jdbc:mysql://example.net:3306?sslMode=REQUIRED +``` + +The `connection-user` and `connection-password` are typically required and +determine the user credentials for the connection, often a service user. You can +use {doc}`secrets ` to avoid actual values in the catalog +properties files. + +(mysql-tls)= + +### Connection security + +If you have TLS configured with a globally-trusted certificate installed on your +data source, you can enable TLS between your cluster and the data +source by appending a parameter to the JDBC connection string set in the +`connection-url` catalog configuration property. + +For example, with version 8.0 of MySQL Connector/J, use the `sslMode` +parameter to secure the connection with TLS. By default the parameter is set to +`PREFERRED` which secures the connection if enabled by the server. You can +also set this parameter to `REQUIRED` which causes the connection to fail if +TLS is not established. + +You can set the `sslMode` parameter in the catalog configuration file by +appending it to the `connection-url` configuration property: + +```properties +connection-url=jdbc:mysql://example.net:3306/?sslMode=REQUIRED +``` + +For more information on TLS configuration options, see the [MySQL JDBC security +documentation](https://dev.mysql.com/doc/connector-j/8.0/en/connector-j-connp-props-security.html#cj-conn-prop_sslMode). + +```{include} jdbc-authentication.fragment +``` + +### Multiple MySQL servers + +You can have as many catalogs as you need, so if you have additional +MySQL servers, simply add another properties file to `etc/catalog` +with a different name, making sure it ends in `.properties`. For +example, if you name the property file `sales.properties`, Trino +creates a catalog named `sales` using the configured connector. + +```{include} jdbc-common-configurations.fragment +``` + +```{include} query-comment-format.fragment +``` + +```{include} jdbc-domain-compaction-threshold.fragment +``` + +```{include} jdbc-procedures.fragment +``` + +```{include} jdbc-case-insensitive-matching.fragment +``` + +```{include} non-transactional-insert.fragment +``` + +(mysql-type-mapping)= + +## Type mapping + +Because Trino and MySQL each support types that the other does not, this +connector {ref}`modifies some types ` when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction. + +### MySQL to Trino type mapping + +The connector maps MySQL types to the corresponding Trino types following +this table: + +```{eval-rst} +.. list-table:: MySQL to Trino type mapping + :widths: 30, 20, 50 + :header-rows: 1 + + * - MySQL database type + - Trino type + - Notes + * - ``BIT`` + - ``BOOLEAN`` + - + * - ``BOOLEAN`` + - ``TINYINT`` + - + * - ``TINYINT`` + - ``TINYINT`` + - + * - ``TINYINT UNSIGNED`` + - ``SMALLINT`` + - + * - ``SMALLINT`` + - ``SMALLINT`` + - + * - ``SMALLINT UNSIGNED`` + - ``INTEGER`` + - + * - ``INTEGER`` + - ``INTEGER`` + - + * - ``INTEGER UNSIGNED`` + - ``BIGINT`` + - + * - ``BIGINT`` + - ``BIGINT`` + - + * - ``BIGINT UNSIGNED`` + - ``DECIMAL(20, 0)`` + - + * - ``DOUBLE PRECISION`` + - ``DOUBLE`` + - + * - ``FLOAT`` + - ``REAL`` + - + * - ``REAL`` + - ``REAL`` + - + * - ``DECIMAL(p, s)`` + - ``DECIMAL(p, s)`` + - See :ref:`MySQL DECIMAL type handling ` + * - ``CHAR(n)`` + - ``CHAR(n)`` + - + * - ``VARCHAR(n)`` + - ``VARCHAR(n)`` + - + * - ``TINYTEXT`` + - ``VARCHAR(255)`` + - + * - ``TEXT`` + - ``VARCHAR(65535)`` + - + * - ``MEDIUMTEXT`` + - ``VARCHAR(16777215)`` + - + * - ``LONGTEXT`` + - ``VARCHAR`` + - + * - ``ENUM(n)`` + - ``VARCHAR(n)`` + - + * - ``BINARY``, ``VARBINARY``, ``TINYBLOB``, ``BLOB``, ``MEDIUMBLOB``, ``LONGBLOB`` + - ``VARBINARY`` + - + * - ``JSON`` + - ``JSON`` + - + * - ``DATE`` + - ``DATE`` + - + * - ``TIME(n)`` + - ``TIME(n)`` + - + * - ``DATETIME(n)`` + - ``TIMESTAMP(n)`` + - + * - ``TIMESTAMP(n)`` + - ``TIMESTAMP(n) WITH TIME ZONE`` + - +``` + +No other types are supported. + +### Trino to MySQL type mapping + +The connector maps Trino types to the corresponding MySQL types following +this table: + +```{eval-rst} +.. list-table:: Trino to MySQL type mapping + :widths: 30, 20, 50 + :header-rows: 1 + + * - Trino type + - MySQL type + - Notes + * - ``BOOLEAN`` + - ``TINYINT`` + - + * - ``TINYINT`` + - ``TINYINT`` + - + * - ``SMALLINT`` + - ``SMALLINT`` + - + * - ``INTEGER`` + - ``INTEGER`` + - + * - ``BIGINT`` + - ``BIGINT`` + - + * - ``REAL`` + - ``REAL`` + - + * - ``DOUBLE`` + - ``DOUBLE PRECISION`` + - + * - ``DECIMAL(p, s)`` + - ``DECIMAL(p, s)`` + - :ref:`MySQL DECIMAL type handling ` + * - ``CHAR(n)`` + - ``CHAR(n)`` + - + * - ``VARCHAR(n)`` + - ``VARCHAR(n)`` + - + * - ``JSON`` + - ``JSON`` + - + * - ``DATE`` + - ``DATE`` + - + * - ``TIME(n)`` + - ``TIME(n)`` + - + * - ``TIMESTAMP(n)`` + - ``DATETIME(n)`` + - + * - ``TIMESTAMP(n) WITH TIME ZONE`` + - ``TIMESTAMP(n)`` + - +``` + +No other types are supported. + +### Timestamp type handling + +MySQL `TIMESTAMP` types are mapped to Trino `TIMESTAMP WITH TIME ZONE`. +To preserve time instants, Trino sets the session time zone +of the MySQL connection to match the JVM time zone. +As a result, error messages similar to the following example occur when +a timezone from the JVM does not exist on the MySQL server: + +``` +com.mysql.cj.exceptions.CJException: Unknown or incorrect time zone: 'UTC' +``` + +To avoid the errors, you must use a time zone that is known on both systems, +or [install the missing time zone on the MySQL server](https://dev.mysql.com/doc/refman/8.0/en/time-zone-support.html#time-zone-installation). + +(mysql-decimal-handling)= + +```{include} decimal-type-handling.fragment +``` + +```{include} jdbc-type-mapping.fragment +``` + +## Querying MySQL + +The MySQL connector provides a schema for every MySQL *database*. +You can see the available MySQL databases by running `SHOW SCHEMAS`: + +``` +SHOW SCHEMAS FROM example; +``` + +If you have a MySQL database named `web`, you can view the tables +in this database by running `SHOW TABLES`: + +``` +SHOW TABLES FROM example.web; +``` + +You can see a list of the columns in the `clicks` table in the `web` database +using either of the following: + +``` +DESCRIBE example.web.clicks; +SHOW COLUMNS FROM example.web.clicks; +``` + +Finally, you can access the `clicks` table in the `web` database: + +``` +SELECT * FROM example.web.clicks; +``` + +If you used a different name for your catalog properties file, use +that catalog name instead of `example` in the above examples. + +(mysql-sql-support)= + +## SQL support + +The connector provides read access and write access to data and metadata in the +MySQL database. In addition to the {ref}`globally available ` and +{ref}`read operation ` statements, the connector supports +the following statements: + +- {doc}`/sql/insert` +- {doc}`/sql/update` +- {doc}`/sql/delete` +- {doc}`/sql/truncate` +- {doc}`/sql/create-table` +- {doc}`/sql/create-table-as` +- {doc}`/sql/drop-table` +- {doc}`/sql/create-schema` +- {doc}`/sql/drop-schema` + +```{include} sql-update-limitation.fragment +``` + +```{include} sql-delete-limitation.fragment +``` + +(mysql-fte-support)= + +## Fault-tolerant execution support + +The connector supports {doc}`/admin/fault-tolerant-execution` of query +processing. Read and write operations are both supported with any retry policy. + +## Table functions + +The connector provides specific {doc}`table functions ` to +access MySQL. + +(mysql-query-function)= + +### `query(varchar) -> table` + +The `query` function allows you to query the underlying database directly. It +requires syntax native to MySQL, because the full query is pushed down and +processed in MySQL. This can be useful for accessing native features which are +not available in Trino or for improving query performance in situations where +running a query natively may be faster. + +```{include} query-passthrough-warning.fragment +``` + +For example, query the `example` catalog and group and concatenate all +employee IDs by manager ID: + +``` +SELECT + * +FROM + TABLE( + example.system.query( + query => 'SELECT + manager_id, GROUP_CONCAT(employee_id) + FROM + company.employees + GROUP BY + manager_id' + ) + ); +``` + +```{include} query-table-function-ordering.fragment +``` + +## Performance + +The connector includes a number of performance improvements, detailed in the +following sections. + +(mysql-table-statistics)= + +### Table statistics + +The MySQL connector can use {doc}`table and column statistics +` for {doc}`cost based optimizations +`, to improve query processing performance +based on the actual data in the data source. + +The statistics are collected by MySQL and retrieved by the connector. + +The table-level statistics are based on MySQL's `INFORMATION_SCHEMA.TABLES` +table. The column-level statistics are based on MySQL's index statistics +`INFORMATION_SCHEMA.STATISTICS` table. The connector can return column-level +statistics only when the column is the first column in some index. + +MySQL database can automatically update its table and index statistics. In some +cases, you may want to force statistics update, for example after creating new +index, or after changing data in the table. You can do that by executing the +following statement in MySQL Database. + +```text +ANALYZE TABLE table_name; +``` + +:::{note} +MySQL and Trino may use statistics information in different ways. For this +reason, the accuracy of table and column statistics returned by the MySQL +connector might be lower than than that of others connectors. +::: + +**Improving statistics accuracy** + +You can improve statistics accuracy with histogram statistics (available since +MySQL 8.0). To create histogram statistics execute the following statement in +MySQL Database. + +```text +ANALYZE TABLE table_name UPDATE HISTOGRAM ON column_name1, column_name2, ...; +``` + +Refer to MySQL documentation for information about options, limitations +and additional considerations. + +(mysql-pushdown)= + +### Pushdown + +The connector supports pushdown for a number of operations: + +- {ref}`join-pushdown` +- {ref}`limit-pushdown` +- {ref}`topn-pushdown` + +{ref}`Aggregate pushdown ` for the following functions: + +- {func}`avg` +- {func}`count` +- {func}`max` +- {func}`min` +- {func}`sum` +- {func}`stddev` +- {func}`stddev_pop` +- {func}`stddev_samp` +- {func}`variance` +- {func}`var_pop` +- {func}`var_samp` + +```{include} pushdown-correctness-behavior.fragment +``` + +```{include} join-pushdown-enabled-true.fragment +``` + +```{include} no-pushdown-text-type.fragment +``` diff --git a/430/_sources/connector/object-storage-file-formats.md.txt b/430/_sources/connector/object-storage-file-formats.md.txt new file mode 100644 index 000000000..cfa5c2371 --- /dev/null +++ b/430/_sources/connector/object-storage-file-formats.md.txt @@ -0,0 +1,105 @@ +# Object storage file formats + +Object storage connectors support one or more file formats specified by the +underlying data source. + +In the case of serializable formats, only specific +[SerDes](https://www.wikipedia.org/wiki/SerDes) are allowed: + +- RCText - RCFile `ColumnarSerDe` +- RCBinary - RCFile `LazyBinaryColumnarSerDe` +- JSON - `org.apache.hive.hcatalog.data.JsonSerDe` +- CSV - `org.apache.hadoop.hive.serde2.OpenCSVSerde` + +(hive-orc-configuration)= + +## ORC format configuration properties + +The following properties are used to configure the read and write operations +with ORC files performed by supported object storage connectors: + +```{eval-rst} +.. list-table:: ORC format configuration properties + :widths: 30, 50, 20 + :header-rows: 1 + + * - Property Name + - Description + - Default + * - ``hive.orc.time-zone`` + - Sets the default time zone for legacy ORC files that did not declare a + time zone. + - JVM default + * - ``hive.orc.use-column-names`` + - Access ORC columns by name. By default, columns in ORC files are + accessed by their ordinal position in the Hive table definition. The + equivalent catalog session property is ``orc_use_column_names``. + - ``false`` + * - ``hive.orc.bloom-filters.enabled`` + - Enable bloom filters for predicate pushdown. + - ``false`` + * - ``hive.orc.read-legacy-short-zone-id`` + - Allow reads on ORC files with short zone ID in the stripe footer. + - ``false`` +``` + +(hive-parquet-configuration)= + +## Parquet format configuration properties + +The following properties are used to configure the read and write operations +with Parquet files performed by supported object storage connectors: + +```{eval-rst} +.. list-table:: Parquet format configuration properties + :widths: 30, 50, 20 + :header-rows: 1 + + * - Property Name + - Description + - Default + * - ``hive.parquet.time-zone`` + - Adjusts timestamp values to a specific time zone. For Hive 3.1+, set + this to UTC. + - JVM default + * - ``hive.parquet.use-column-names`` + - Access Parquet columns by name by default. Set this property to + ``false`` to access columns by their ordinal position in the Hive table + definition. The equivalent catalog session property is + ``parquet_use_column_names``. + - ``true`` + * - ``parquet.writer.validation-percentage`` + - Percentage of parquet files to validate after write by re-reading the whole file. + The equivalent catalog session property is ``parquet_optimized_writer_validation_percentage``. + Validation can be turned off by setting this property to ``0``. + - ``5`` + * - ``parquet.writer.page-size`` + - Maximum page size for the Parquet writer. + - ``1 MB`` + * - ``parquet.writer.block-size`` + - Maximum row group size for the Parquet writer. + - ``128 MB`` + * - ``parquet.writer.batch-size`` + - Maximum number of rows processed by the parquet writer in a batch. + - ``10000`` + * - ``parquet.use-bloom-filter`` + - Whether bloom filters are used for predicate pushdown when reading + Parquet files. Set this property to ``false`` to disable the usage of + bloom filters by default. The equivalent catalog session property is + ``parquet_use_bloom_filter``. + - ``true`` + * - ``parquet.use-column-index`` + - Skip reading Parquet pages by using Parquet column indices. The + equivalent catalog session property is ``parquet_use_column_index``. + Only supported by the Delta Lake and Hive connectors. + - ``true`` + * - ``parquet.max-read-block-row-count`` + - Sets the maximum number of rows read in a batch. The equivalent catalog + session property is named ``parquet_max_read_block_row_count`` and + supported by the Delta Lake, Hive, and Iceberg connectors. + - ``8192`` + * - ``parquet.small-file-threshold`` + - :ref:`Data size ` below which a Parquet file is + read entirely. The equivalent catalog session property is named + ``parquet_small_file_threshold``. + - ``3MB`` diff --git a/430/_sources/connector/oracle.md.txt b/430/_sources/connector/oracle.md.txt new file mode 100644 index 000000000..4c42bf339 --- /dev/null +++ b/430/_sources/connector/oracle.md.txt @@ -0,0 +1,608 @@ +--- +myst: + substitutions: + default_domain_compaction_threshold: '`32`' +--- + +# Oracle connector + +```{raw} html + +``` + +The Oracle connector allows querying and creating tables in an external Oracle +database. Connectors let Trino join data provided by different databases, +like Oracle and Hive, or different Oracle database instances. + +## Requirements + +To connect to Oracle, you need: + +- Oracle 12 or higher. +- Network access from the Trino coordinator and workers to Oracle. + Port 1521 is the default port. + +## Configuration + +To configure the Oracle connector as the `example` catalog, create a file +named `example.properties` in `etc/catalog`. Include the following +connection properties in the file: + +```text +connector.name=oracle +# The correct syntax of the connection-url varies by Oracle version and +# configuration. The following example URL connects to an Oracle SID named +# "orcl". +connection-url=jdbc:oracle:thin:@example.net:1521:orcl +connection-user=root +connection-password=secret +``` + +The `connection-url` defines the connection information and parameters to pass +to the JDBC driver. The Oracle connector uses the Oracle JDBC Thin driver, +and the syntax of the URL may be different depending on your Oracle +configuration. For example, the connection URL is different if you are +connecting to an Oracle SID or an Oracle service name. See the [Oracle +Database JDBC driver documentation](https://docs.oracle.com/en/database/oracle/oracle-database/21/jjdbc/data-sources-and-URLs.html#GUID-088B1600-C6C2-4F19-A020-2DAF8FE1F1C3) +for more information. + +The `connection-user` and `connection-password` are typically required and +determine the user credentials for the connection, often a service user. You can +use {doc}`secrets ` to avoid actual values in the catalog +properties files. + +:::{note} +Oracle does not expose metadata comment via `REMARKS` column by default +in JDBC driver. You can enable it using `oracle.remarks-reporting.enabled` +config option. See [Additional Oracle Performance Extensions](https://docs.oracle.com/en/database/oracle/oracle-database/19/jjdbc/performance-extensions.html#GUID-96A38C6D-A288-4E0B-9F03-E711C146632B) +for more details. +::: + +By default, the Oracle connector uses connection pooling for performance +improvement. The below configuration shows the typical default values. To update +them, change the properties in the catalog configuration file: + +```properties +oracle.connection-pool.max-size=30 +oracle.connection-pool.min-size=1 +oracle.connection-pool.inactive-timeout=20m +``` + +To disable connection pooling, update properties to include the following: + +```text +oracle.connection-pool.enabled=false +``` + +```{include} jdbc-authentication.fragment +``` + +### Multiple Oracle servers + +If you want to connect to multiple Oracle servers, configure another instance of +the Oracle connector as a separate catalog. + +To add another Oracle catalog, create a new properties file. For example, if +you name the property file `sales.properties`, Trino creates a catalog named +`sales`. + +```{include} jdbc-common-configurations.fragment +``` + +```{include} query-comment-format.fragment +``` + +```{include} jdbc-domain-compaction-threshold.fragment +``` + +```{include} jdbc-procedures.fragment +``` + +```{include} jdbc-case-insensitive-matching.fragment +``` + +```{include} non-transactional-insert.fragment +``` + +## Querying Oracle + +The Oracle connector provides a schema for every Oracle database. + +Run `SHOW SCHEMAS` to see the available Oracle databases: + +``` +SHOW SCHEMAS FROM example; +``` + +If you used a different name for your catalog properties file, use that catalog +name instead of `example`. + +:::{note} +The Oracle user must have access to the table in order to access it from Trino. +The user configuration, in the connection properties file, determines your +privileges in these schemas. +::: + +### Examples + +If you have an Oracle database named `web`, run `SHOW TABLES` to see the +tables it contains: + +``` +SHOW TABLES FROM example.web; +``` + +To see a list of the columns in the `clicks` table in the `web` +database, run either of the following: + +``` +DESCRIBE example.web.clicks; +SHOW COLUMNS FROM example.web.clicks; +``` + +To access the clicks table in the web database, run the following: + +``` +SELECT * FROM example.web.clicks; +``` + +(oracle-type-mapping)= + +## Type mapping + +Because Trino and Oracle each support types that the other does not, this +connector {ref}`modifies some types ` when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction. + +### Oracle to Trino type mapping + +Trino supports selecting Oracle database types. This table shows the Oracle to +Trino data type mapping: + +```{eval-rst} +.. list-table:: Oracle to Trino type mapping + :widths: 30, 25, 50 + :header-rows: 1 + + * - Oracle database type + - Trino type + - Notes + * - ``NUMBER(p, s)`` + - ``DECIMAL(p, s)`` + - See :ref:`oracle-number-mapping` + * - ``NUMBER(p)`` + - ``DECIMAL(p, 0)`` + - See :ref:`oracle-number-mapping` + * - ``FLOAT[(p)]`` + - ``DOUBLE`` + - + * - ``BINARY_FLOAT`` + - ``REAL`` + - + * - ``BINARY_DOUBLE`` + - ``DOUBLE`` + - + * - ``VARCHAR2(n CHAR)`` + - ``VARCHAR(n)`` + - + * - ``VARCHAR2(n BYTE)`` + - ``VARCHAR(n)`` + - + * - ``NVARCHAR2(n)`` + - ``VARCHAR(n)`` + - + * - ``CHAR(n)`` + - ``CHAR(n)`` + - + * - ``NCHAR(n)`` + - ``CHAR(n)`` + - + * - ``CLOB`` + - ``VARCHAR`` + - + * - ``NCLOB`` + - ``VARCHAR`` + - + * - ``RAW(n)`` + - ``VARBINARY`` + - + * - ``BLOB`` + - ``VARBINARY`` + - + * - ``DATE`` + - ``TIMESTAMP(0)`` + - See :ref:`oracle-datetime-mapping` + * - ``TIMESTAMP(p)`` + - ``TIMESTAMP(p)`` + - See :ref:`oracle-datetime-mapping` + * - ``TIMESTAMP(p) WITH TIME ZONE`` + - ``TIMESTAMP WITH TIME ZONE`` + - See :ref:`oracle-datetime-mapping` +``` + +No other types are supported. + +### Trino to Oracle type mapping + +Trino supports creating tables with the following types in an Oracle database. +The table shows the mappings from Trino to Oracle data types: + +:::{note} +For types not listed in the table below, Trino can't perform the `CREATE +TABLE
AS SELECT` operations. When data is inserted into existing +tables, `Oracle to Trino` type mapping is used. +::: + +```{eval-rst} +.. list-table:: Trino to Oracle Type Mapping + :widths: 30, 25, 50 + :header-rows: 1 + + * - Trino type + - Oracle database type + - Notes + * - ``TINYINT`` + - ``NUMBER(3)`` + - + * - ``SMALLINT`` + - ``NUMBER(5)`` + - + * - ``INTEGER`` + - ``NUMBER(10)`` + - + * - ``BIGINT`` + - ``NUMBER(19)`` + - + * - ``DECIMAL(p, s)`` + - ``NUMBER(p, s)`` + - + * - ``REAL`` + - ``BINARY_FLOAT`` + - + * - ``DOUBLE`` + - ``BINARY_DOUBLE`` + - + * - ``VARCHAR`` + - ``NCLOB`` + - + * - ``VARCHAR(n)`` + - ``VARCHAR2(n CHAR)`` or ``NCLOB`` + - See :ref:`oracle-character-mapping` + * - ``CHAR(n)`` + - ``CHAR(n CHAR)`` or ``NCLOB`` + - See :ref:`oracle-character-mapping` + * - ``VARBINARY`` + - ``BLOB`` + - + * - ``DATE`` + - ``DATE`` + - See :ref:`oracle-datetime-mapping` + * - ``TIMESTAMP`` + - ``TIMESTAMP(3)`` + - See :ref:`oracle-datetime-mapping` + * - ``TIMESTAMP WITH TIME ZONE`` + - ``TIMESTAMP(3) WITH TIME ZONE`` + - See :ref:`oracle-datetime-mapping` +``` + +No other types are supported. + +(oracle-number-mapping)= +### Mapping numeric types + +An Oracle `NUMBER(p, s)` maps to Trino's `DECIMAL(p, s)` except in these +conditions: + +- No precision is specified for the column (example: `NUMBER` or + `NUMBER(*)`), unless `oracle.number.default-scale` is set. +- Scale (`s` ) is greater than precision. +- Precision (`p` ) is greater than 38. +- Scale is negative and the difference between `p` and `s` is greater than + 38, unless `oracle.number.rounding-mode` is set to a different value than + `UNNECESSARY`. + +If `s` is negative, `NUMBER(p, s)` maps to `DECIMAL(p + s, 0)`. + +For Oracle `NUMBER` (without precision and scale), you can change +`oracle.number.default-scale=s` and map the column to `DECIMAL(38, s)`. + +(oracle-datetime-mapping)= +### Mapping datetime types + +Writing a timestamp with fractional second precision (`p`) greater than 9 +rounds the fractional seconds to nine digits. + +Oracle `DATE` type stores hours, minutes, and seconds, so it is mapped +to Trino `TIMESTAMP(0)`. + +:::{warning} +Due to date and time differences in the libraries used by Trino and the +Oracle JDBC driver, attempting to insert or select a datetime value earlier +than `1582-10-15` results in an incorrect date inserted. +::: + +(oracle-character-mapping)= +### Mapping character types + +Trino's `VARCHAR(n)` maps to `VARCHAR2(n CHAR)` if `n` is no greater +than 4000. A larger or unbounded `VARCHAR` maps to `NCLOB`. + +Trino's `CHAR(n)` maps to `CHAR(n CHAR)` if `n` is no greater than 2000. +A larger `CHAR` maps to `NCLOB`. + +Using `CREATE TABLE AS` to create an `NCLOB` column from a `CHAR` value +removes the trailing spaces from the initial values for the column. Inserting +`CHAR` values into existing `NCLOB` columns keeps the trailing spaces. For +example: + +``` +CREATE TABLE vals AS SELECT CAST('A' as CHAR(2001)) col; +INSERT INTO vals (col) VALUES (CAST('BB' as CHAR(2001))); +SELECT LENGTH(col) FROM vals; +``` + +```text + _col0 +------- + 2001 + 1 +(2 rows) +``` + +Attempting to write a `CHAR` that doesn't fit in the column's actual size +fails. This is also true for the equivalent `VARCHAR` types. + +```{include} jdbc-type-mapping.fragment +``` + +### Number to decimal configuration properties + +```{eval-rst} +.. list-table:: + :widths: 20, 20, 50, 10 + :header-rows: 1 + + * - Configuration property name + - Session property name + - Description + - Default + * - ``oracle.number.default-scale`` + - ``number_default_scale`` + - Default Trino ``DECIMAL`` scale for Oracle ``NUMBER`` (without precision + and scale) date type. When not set then such column is treated as not + supported. + - not set + * - ``oracle.number.rounding-mode`` + - ``number_rounding_mode`` + - Rounding mode for the Oracle ``NUMBER`` data type. This is useful when + Oracle ``NUMBER`` data type specifies higher scale than is supported in + Trino. Possible values are: + + - ``UNNECESSARY`` - Rounding mode to assert that the + requested operation has an exact result, + hence no rounding is necessary. + - ``CEILING`` - Rounding mode to round towards + positive infinity. + - ``FLOOR`` - Rounding mode to round towards negative + infinity. + - ``HALF_DOWN`` - Rounding mode to round towards + ``nearest neighbor`` unless both neighbors are + equidistant, in which case rounding down is used. + - ``HALF_EVEN`` - Rounding mode to round towards the + ``nearest neighbor`` unless both neighbors are equidistant, + in which case rounding towards the even neighbor is + performed. + - ``HALF_UP`` - Rounding mode to round towards + ``nearest neighbor`` unless both neighbors are + equidistant, in which case rounding up is used + - ``UP`` - Rounding mode to round towards zero. + - ``DOWN`` - Rounding mode to round towards zero. + + - ``UNNECESSARY`` +``` + +(oracle-sql-support)= + +## SQL support + +The connector provides read access and write access to data and metadata in +Oracle. In addition to the {ref}`globally available ` +and {ref}`read operation ` statements, the connector +supports the following statements: + +- {doc}`/sql/insert` +- {doc}`/sql/update` +- {doc}`/sql/delete` +- {doc}`/sql/truncate` +- {doc}`/sql/create-table` +- {doc}`/sql/create-table-as` +- {doc}`/sql/drop-table` +- {doc}`/sql/alter-table` +- {doc}`/sql/comment` + +```{include} sql-update-limitation.fragment +``` + +```{include} sql-delete-limitation.fragment +``` + +```{include} alter-table-limitation.fragment +``` + +(oracle-fte-support)= + +## Fault-tolerant execution support + +The connector supports {doc}`/admin/fault-tolerant-execution` of query +processing. Read and write operations are both supported with any retry policy. + +## Table functions + +The connector provides specific {doc}`table functions ` to +access Oracle. + +(oracle-query-function)= + +### `query(varchar) -> table` + +The `query` function allows you to query the underlying database directly. It +requires syntax native to Oracle, because the full query is pushed down and +processed in Oracle. This can be useful for accessing native features which are +not available in Trino or for improving query performance in situations where +running a query natively may be faster. + +```{include} query-passthrough-warning.fragment +``` + +As a simple example, query the `example` catalog and select an entire table: + +``` +SELECT + * +FROM + TABLE( + example.system.query( + query => 'SELECT + * + FROM + tpch.nation' + ) + ); +``` + +As a practical example, you can use the +[MODEL clause from Oracle SQL](https://docs.oracle.com/cd/B19306_01/server.102/b14223/sqlmodel.htm): + +``` +SELECT + SUBSTR(country, 1, 20) country, + SUBSTR(product, 1, 15) product, + year, + sales +FROM + TABLE( + example.system.query( + query => 'SELECT + * + FROM + sales_view + MODEL + RETURN UPDATED ROWS + MAIN + simple_model + PARTITION BY + country + MEASURES + sales + RULES + (sales['Bounce', 2001] = 1000, + sales['Bounce', 2002] = sales['Bounce', 2001] + sales['Bounce', 2000], + sales['Y Box', 2002] = sales['Y Box', 2001]) + ORDER BY + country' + ) + ); +``` + +```{include} query-table-function-ordering.fragment +``` + +## Performance + +The connector includes a number of performance improvements, detailed in the +following sections. + +### Synonyms + +Based on performance reasons, Trino disables support for Oracle `SYNONYM`. To +include `SYNONYM`, add the following configuration property: + +```text +oracle.synonyms.enabled=true +``` + +(oracle-pushdown)= + +### Pushdown + +The connector supports pushdown for a number of operations: + +- {ref}`join-pushdown` +- {ref}`limit-pushdown` +- {ref}`topn-pushdown` + +In addition, the connector supports {ref}`aggregation-pushdown` for the +following functions: + +- {func}`avg()` +- {func}`count()`, also `count(distinct x)` +- {func}`max()` +- {func}`min()` +- {func}`sum()` + +Pushdown is only supported for `DOUBLE` type columns with the +following functions: + +- {func}`stddev()` and {func}`stddev_samp()` +- {func}`stddev_pop()` +- {func}`var_pop()` +- {func}`variance()` and {func}`var_samp()` + +Pushdown is only supported for `REAL` or `DOUBLE` type column +with the following functions: + +- {func}`covar_samp()` +- {func}`covar_pop()` + + +```{include} pushdown-correctness-behavior.fragment +``` + +```{include} join-pushdown-enabled-false.fragment +``` + +(oracle-predicate-pushdown)= + +#### Predicate pushdown support + +The connector does not support pushdown of any predicates on columns that use +the `CLOB`, `NCLOB`, `BLOB`, or `RAW(n)` Oracle database types, or Trino +data types that {ref}`map ` to these Oracle database types. + +In the following example, the predicate is not pushed down for either query +since `name` is a column of type `VARCHAR`, which maps to `NCLOB` in +Oracle: + +```sql +SHOW CREATE TABLE nation; + +-- Create Table +---------------------------------------- +-- CREATE TABLE oracle.trino_test.nation ( +-- name VARCHAR +-- ) +-- (1 row) + +SELECT * FROM nation WHERE name > 'CANADA'; +SELECT * FROM nation WHERE name = 'CANADA'; +``` + +In the following example, the predicate is pushed down for both queries +since `name` is a column of type `VARCHAR(25)`, which maps to +`VARCHAR2(25)` in Oracle: + +```sql +SHOW CREATE TABLE nation; + +-- Create Table +---------------------------------------- +-- CREATE TABLE oracle.trino_test.nation ( +-- name VARCHAR(25) +-- ) +-- (1 row) + +SELECT * FROM nation WHERE name > 'CANADA'; +SELECT * FROM nation WHERE name = 'CANADA'; +``` diff --git a/430/_sources/connector/phoenix.md.txt b/430/_sources/connector/phoenix.md.txt new file mode 100644 index 000000000..850368af1 --- /dev/null +++ b/430/_sources/connector/phoenix.md.txt @@ -0,0 +1,290 @@ +--- +myst: + substitutions: + default_domain_compaction_threshold: '`5000`' +--- + +# Phoenix connector + +```{raw} html + +``` + +The Phoenix connector allows querying data stored in +[Apache HBase](https://hbase.apache.org/) using +[Apache Phoenix](https://phoenix.apache.org/). + +## Requirements + +To query HBase data through Phoenix, you need: + +- Network access from the Trino coordinator and workers to the ZooKeeper + servers. The default port is 2181. +- A compatible version of Phoenix: all 5.x versions starting from 5.1.0 are supported. + +## Configuration + +To configure the Phoenix connector, create a catalog properties file +`etc/catalog/example.properties` with the following contents, +replacing `host1,host2,host3` with a comma-separated list of the ZooKeeper +nodes used for discovery of the HBase cluster: + +```text +connector.name=phoenix5 +phoenix.connection-url=jdbc:phoenix:host1,host2,host3:2181:/hbase +phoenix.config.resources=/path/to/hbase-site.xml +``` + +The optional paths to Hadoop resource files, such as `hbase-site.xml` are used +to load custom Phoenix client connection properties. + +The following Phoenix-specific configuration properties are available: + +| Property name | Required | Description | +| ----------------------------- | -------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ | +| `phoenix.connection-url` | Yes | `jdbc:phoenix[:zk_quorum][:zk_port][:zk_hbase_path]`. The `zk_quorum` is a comma separated list of ZooKeeper servers. The `zk_port` is the ZooKeeper port. The `zk_hbase_path` is the HBase root znode path, that is configurable using `hbase-site.xml`. By default the location is `/hbase` | +| `phoenix.config.resources` | No | Comma-separated list of configuration files (e.g. `hbase-site.xml`) to use for connection properties. These files must exist on the machines running Trino. | +| `phoenix.max-scans-per-split` | No | Maximum number of HBase scans that will be performed in a single split. Default is 20. Lower values will lead to more splits in Trino. Can also be set via session propery `max_scans_per_split`. For details see: [https://phoenix.apache.org/update_statistics.html](https://phoenix.apache.org/update_statistics.html). (This setting has no effect when guideposts are disabled in Phoenix.) | + +```{include} jdbc-common-configurations.fragment +``` + +```{include} query-comment-format.fragment +``` + +```{include} jdbc-domain-compaction-threshold.fragment +``` + +```{include} jdbc-procedures.fragment +``` + +```{include} jdbc-case-insensitive-matching.fragment +``` + +```{include} non-transactional-insert.fragment +``` + +## Querying Phoenix tables + +The default empty schema in Phoenix maps to a schema named `default` in Trino. +You can see the available Phoenix schemas by running `SHOW SCHEMAS`: + +``` +SHOW SCHEMAS FROM example; +``` + +If you have a Phoenix schema named `web`, you can view the tables +in this schema by running `SHOW TABLES`: + +``` +SHOW TABLES FROM example.web; +``` + +You can see a list of the columns in the `clicks` table in the `web` schema +using either of the following: + +``` +DESCRIBE example.web.clicks; +SHOW COLUMNS FROM example.web.clicks; +``` + +Finally, you can access the `clicks` table in the `web` schema: + +``` +SELECT * FROM example.web.clicks; +``` + +If you used a different name for your catalog properties file, use +that catalog name instead of `example` in the above examples. + +(phoenix-type-mapping)= + +## Type mapping + +Because Trino and Phoenix each support types that the other does not, this +connector {ref}`modifies some types ` when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction. + +### Phoenix type to Trino type mapping + +The connector maps Phoenix types to the corresponding Trino types following this +table: + +```{eval-rst} +.. list-table:: Phoenix type to Trino type mapping + :widths: 30, 20 + :header-rows: 1 + + * - Phoenix database type + - Trino type + * - ``BOOLEAN`` + - ``BOOLEAN`` + * - ``TINYINT`` + - ``TINYINT`` + * - ``UNSIGNED_TINYINT`` + - ``TINYINT`` + * - ``SMALLINT`` + - ``SMALLINT`` + * - ``UNSIGNED_SMALLINT`` + - ``SMALLINT`` + * - ``INTEGER`` + - ``INTEGER`` + * - ``UNSIGNED_INT`` + - ``INTEGER`` + * - ``BIGINT`` + - ``BIGINT`` + * - ``UNSIGNED_LONG`` + - ``BIGINT`` + * - ``FLOAT`` + - ``REAL`` + * - ``UNSIGNED_FLOAT`` + - ``REAL`` + * - ``DOUBLE`` + - ``DOUBLE`` + * - ``UNSIGNED_DOUBLE`` + - ``DOUBLE`` + * - ``DECIMAL(p,s)`` + - ``DECIMAL(p,s)`` + * - ``CHAR(n)`` + - ``CHAR(n)`` + * - ``VARCHAR(n)`` + - ``VARCHAR(n)`` + * - ``BINARY`` + - ``VARBINARY`` + * - ``VARBINARY`` + - ``VARBINARY`` + * - ``DATE`` + - ``DATE`` + * - ``UNSIGNED_DATE`` + - ``DATE`` + * - ``ARRAY`` + - ``ARRAY`` +``` + +No other types are supported. + +### Trino type to Phoenix type mapping + +The Phoenix fixed length `BINARY` data type is mapped to the Trino variable +length `VARBINARY` data type. There is no way to create a Phoenix table in +Trino that uses the `BINARY` data type, as Trino does not have an equivalent +type. + +The connector maps Trino types to the corresponding Phoenix types following this +table: + +```{eval-rst} +.. list-table:: Trino type to Phoenix type mapping + :widths: 30, 20 + :header-rows: 1 + + * - Trino database type + - Phoenix type + * - ``BOOLEAN`` + - ``BOOLEAN`` + * - ``TINYINT`` + - ``TINYINT`` + * - ``SMALLINT`` + - ``SMALLINT`` + * - ``INTEGER`` + - ``INTEGER`` + * - ``BIGINT`` + - ``BIGINT`` + * - ``REAL`` + - ``FLOAT`` + * - ``DOUBLE`` + - ``DOUBLE`` + * - ``DECIMAL(p,s)`` + - ``DECIMAL(p,s)`` + * - ``CHAR(n)`` + - ``CHAR(n)`` + * - ``VARCHAR(n)`` + - ``VARCHAR(n)`` + * - ``VARBINARY`` + - ``VARBINARY`` + * - ``TIME`` + - ``TIME`` + * - ``DATE`` + - ``DATE`` + * - ``ARRAY`` + - ``ARRAY`` +``` + +No other types are supported. + +```{include} decimal-type-handling.fragment +``` + +```{include} jdbc-type-mapping.fragment +``` + +## Table properties - Phoenix + +Table property usage example: + +``` +CREATE TABLE example_schema.scientists ( + recordkey VARCHAR, + birthday DATE, + name VARCHAR, + age BIGINT +) +WITH ( + rowkeys = 'recordkey,birthday', + salt_buckets = 10 +); +``` + +The following are supported Phoenix table properties from [https://phoenix.apache.org/language/index.html#options](https://phoenix.apache.org/language/index.html#options) + +| Property name | Default value | Description | +| ----------------------- | ------------- | --------------------------------------------------------------------------------------------------------------------- | +| `rowkeys` | `ROWKEY` | Comma-separated list of primary key columns. See further description below | +| `split_on` | (none) | List of keys to presplit the table on. See [Split Point](https://phoenix.apache.org/language/index.html#split_point). | +| `salt_buckets` | (none) | Number of salt buckets for this table. | +| `disable_wal` | false | Whether to disable WAL writes in HBase for this table. | +| `immutable_rows` | false | Declares whether this table has rows which are write-once, append-only. | +| `default_column_family` | `0` | Default column family name to use for this table. | + +### `rowkeys` + +This is a comma-separated list of columns to be used as the table's primary key. If not specified, a `BIGINT` primary key column named `ROWKEY` is generated +, as well as a sequence with the same name as the table suffixed with `_seq` (i.e. `.
_seq`) +, which is used to automatically populate the `ROWKEY` for each row during insertion. + +## Table properties - HBase + +The following are the supported HBase table properties that are passed through by Phoenix during table creation. +Use them in the same way as above: in the `WITH` clause of the `CREATE TABLE` statement. + +| Property name | Default value | Description | +| --------------------- | ------------- | ---------------------------------------------------------------------------------------------------------------------- | +| `versions` | `1` | The maximum number of versions of each cell to keep. | +| `min_versions` | `0` | The minimum number of cell versions to keep. | +| `compression` | `NONE` | Compression algorithm to use. Valid values are `NONE` (default), `SNAPPY`, `LZO`, `LZ4`, or `GZ`. | +| `data_block_encoding` | `FAST_DIFF` | Block encoding algorithm to use. Valid values are: `NONE`, `PREFIX`, `DIFF`, `FAST_DIFF` (default), or `ROW_INDEX_V1`. | +| `ttl` | `FOREVER` | Time To Live for each cell. | +| `bloomfilter` | `NONE` | Bloomfilter to use. Valid values are `NONE` (default), `ROW`, or `ROWCOL`. | + +(phoenix-sql-support)= + +## SQL support + +The connector provides read and write access to data and metadata in +Phoenix. In addition to the {ref}`globally available +` and {ref}`read operation ` +statements, the connector supports the following features: + +- {doc}`/sql/insert` +- {doc}`/sql/delete` +- {doc}`/sql/merge` +- {doc}`/sql/create-table` +- {doc}`/sql/create-table-as` +- {doc}`/sql/drop-table` +- {doc}`/sql/create-schema` +- {doc}`/sql/drop-schema` + +```{include} sql-delete-limitation.fragment +``` diff --git a/430/_sources/connector/pinot.md.txt b/430/_sources/connector/pinot.md.txt new file mode 100644 index 000000000..e6013f161 --- /dev/null +++ b/430/_sources/connector/pinot.md.txt @@ -0,0 +1,228 @@ +# Pinot connector + +```{raw} html + +``` + +The Pinot connector allows Trino to query data stored in +[Apache Pinot™](https://pinot.apache.org/). + +## Requirements + +To connect to Pinot, you need: + +- Pinot 0.11.0 or higher. +- Network access from the Trino coordinator and workers to the Pinot controller + nodes. Port 8098 is the default port. + +## Configuration + +To configure the Pinot connector, create a catalog properties file +e.g. `etc/catalog/example.properties` with at least the following contents: + +```text +connector.name=pinot +pinot.controller-urls=host1:8098,host2:8098 +``` + +Replace `host1:8098,host2:8098` with a comma-separated list of Pinot controller nodes. +This can be the ip or the FDQN, the url scheme (`http://`) is optional. + +## Configuration properties + +### General configuration properties + +| Property name | Required | Description | +| ------------------------------------------------------ | -------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| `pinot.controller-urls` | Yes | A comma separated list of controller hosts. If Pinot is deployed via [Kubernetes](https://kubernetes.io/) this needs to point to the controller service endpoint. The Pinot broker and server must be accessible via DNS as Pinot returns hostnames and not IP addresses. | +| `pinot.connection-timeout` | No | Pinot connection timeout, default is `15s`. | +| `pinot.metadata-expiry` | No | Pinot metadata expiration time, default is `2m`. | +| `pinot.controller.authentication.type` | No | Pinot authentication method for controller requests. Allowed values are `NONE` and `PASSWORD` - defaults to `NONE` which is no authentication. | +| `pinot.controller.authentication.user` | No | Controller username for basic authentication method. | +| `pinot.controller.authentication.password` | No | Controller password for basic authentication method. | +| `pinot.broker.authentication.type` | No | Pinot authentication method for broker requests. Allowed values are `NONE` and `PASSWORD` - defaults to `NONE` which is no authentication. | +| `pinot.broker.authentication.user` | No | Broker username for basic authentication method. | +| `pinot.broker.authentication.password` | No | Broker password for basic authentication method. | +| `pinot.max-rows-per-split-for-segment-queries` | No | Fail query if Pinot server split returns more rows than configured, default to `50,000` for non-gRPC connection, `2,147,483,647` for gRPC connection. | +| `pinot.estimated-size-in-bytes-for-non-numeric-column` | No | Estimated byte size for non-numeric column for page pre-allocation in non-gRPC connection, default is `20`. | +| `pinot.prefer-broker-queries` | No | Pinot query plan prefers to query Pinot broker, default is `true`. | +| `pinot.forbid-segment-queries` | No | Forbid parallel querying and force all querying to happen via the broker, default is `false`. | +| `pinot.segments-per-split` | No | The number of segments processed in a split. Setting this higher reduces the number of requests made to Pinot. This is useful for smaller Pinot clusters, default is `1`. | +| `pinot.fetch-retry-count` | No | Retry count for retriable Pinot data fetch calls, default is `2`. | +| `pinot.non-aggregate-limit-for-broker-queries` | No | Max limit for non aggregate queries to the Pinot broker, default is `25,000`. | +| `pinot.max-rows-for-broker-queries` | No | Max rows for a broker query can return, default is `50,000`. | +| `pinot.aggregation-pushdown.enabled` | No | Push down aggregation queries, default is `true`. | +| `pinot.count-distinct-pushdown.enabled` | No | Push down count distinct queries to Pinot, default is `true`. | +| `pinot.target-segment-page-size` | No | Max allowed page size for segment query, default is `1MB`. | +| `pinot.proxy.enabled` | No | Use Pinot Proxy for controller and broker requests, default is `false`. | + +If `pinot.controller.authentication.type` is set to `PASSWORD` then both `pinot.controller.authentication.user` and +`pinot.controller.authentication.password` are required. + +If `pinot.broker.authentication.type` is set to `PASSWORD` then both `pinot.broker.authentication.user` and +`pinot.broker.authentication.password` are required. + +If `pinot.controller-urls` uses `https` scheme then TLS is enabled for all connections including brokers. + +### gRPC configuration properties + +| Property name | Required | Description | +| ------------------------------------- | -------- | -------------------------------------------------------------------- | +| `pinot.grpc.enabled` | No | Use gRPC endpoint for Pinot server queries, default is `true`. | +| `pinot.grpc.port` | No | Pinot gRPC port, default to `8090`. | +| `pinot.grpc.max-inbound-message-size` | No | Max inbound message bytes when init gRPC client, default is `128MB`. | +| `pinot.grpc.use-plain-text` | No | Use plain text for gRPC communication, default to `true`. | +| `pinot.grpc.tls.keystore-type` | No | TLS keystore type for gRPC connection, default is `JKS`. | +| `pinot.grpc.tls.keystore-path` | No | TLS keystore file location for gRPC connection, default is empty. | +| `pinot.grpc.tls.keystore-password` | No | TLS keystore password, default is empty. | +| `pinot.grpc.tls.truststore-type` | No | TLS truststore type for gRPC connection, default is `JKS`. | +| `pinot.grpc.tls.truststore-path` | No | TLS truststore file location for gRPC connection, default is empty. | +| `pinot.grpc.tls.truststore-password` | No | TLS truststore password, default is empty. | +| `pinot.grpc.tls.ssl-provider` | No | SSL provider, default is `JDK`. | +| `pinot.grpc.proxy-uri` | No | Pinot Rest Proxy gRPC endpoint URI, default is null. | + +For more Apache Pinot TLS configurations, please also refer to [Configuring TLS/SSL](https://docs.pinot.apache.org/operators/tutorials/configuring-tls-ssl). + +You can use {doc}`secrets ` to avoid actual values in the catalog properties files. + +## Querying Pinot tables + +The Pinot connector automatically exposes all tables in the default schema of the catalog. +You can list all tables in the pinot catalog with the following query: + +``` +SHOW TABLES FROM example.default; +``` + +You can list columns in the flight_status table: + +``` +DESCRIBE example.default.flight_status; +SHOW COLUMNS FROM example.default.flight_status; +``` + +Queries written with SQL are fully supported and can include filters and limits: + +``` +SELECT foo +FROM pinot_table +WHERE bar = 3 AND baz IN ('ONE', 'TWO', 'THREE') +LIMIT 25000; +``` + +## Dynamic tables + +To leverage Pinot's fast aggregation, a Pinot query written in PQL can be used as the table name. +Filters and limits in the outer query are pushed down to Pinot. +Let's look at an example query: + +``` +SELECT * +FROM example.default."SELECT MAX(col1), COUNT(col2) FROM pinot_table GROUP BY col3, col4" +WHERE col3 IN ('FOO', 'BAR') AND col4 > 50 +LIMIT 30000 +``` + +Filtering and limit processing is pushed down to Pinot. + +The queries are routed to the broker and are more suitable to aggregate queries. + +For `SELECT` queries without aggregates it is more performant to issue a regular SQL query. +Processing is routed directly to the servers that store the data. + +The above query is translated to the following Pinot PQL query: + +``` +SELECT MAX(col1), COUNT(col2) +FROM pinot_table +WHERE col3 IN('FOO', 'BAR') and col4 > 50 +TOP 30000 +``` + +(pinot-type-mapping)= + +## Type mapping + +Because Trino and Pinot each support types that the other does not, this +connector {ref}`maps some types ` when reading data. + +### Pinot type to Trino type mapping + +The connector maps Pinot types to the corresponding Trino types +according to the following table: + +```{eval-rst} +.. list-table:: Pinot type to Trino type mapping + :widths: 75,60 + :header-rows: 1 + + * - Pinot type + - Trino type + * - ``INT`` + - ``INTEGER`` + * - ``LONG`` + - ``BIGINT`` + * - ``FLOAT`` + - ``REAL`` + * - ``DOUBLE`` + - ``DOUBLE`` + * - ``STRING`` + - ``VARCHAR`` + * - ``BYTES`` + - ``VARBINARY`` + * - ``JSON`` + - ``JSON`` + * - ``TIMESTAMP`` + - ``TIMESTAMP`` + * - ``INT_ARRAY`` + - ``VARCHAR`` + * - ``LONG_ARRAY`` + - ``VARCHAR`` + * - ``FLOAT_ARRAY`` + - ``VARCHAR`` + * - ``DOUBLE_ARRAY`` + - ``VARCHAR`` + * - ``STRING_ARRAY`` + - ``VARCHAR`` +``` + +Pinot does not allow null values in any data type. + +No other types are supported. + +(pinot-sql-support)= + +## SQL support + +The connector provides {ref}`globally available ` and +{ref}`read operation ` statements to access data and +metadata in Pinot. + +(pinot-pushdown)= + +## Pushdown + +The connector supports pushdown for a number of operations: + +- {ref}`limit-pushdown` + +{ref}`Aggregate pushdown ` for the following functions: + +- {func}`avg` +- {func}`approx_distinct` +- `count(*)` and `count(distinct)` variations of {func}`count` +- {func}`max` +- {func}`min` +- {func}`sum` + +Aggregate function pushdown is enabled by default, but can be disabled with the +catalog property `pinot.aggregation-pushdown.enabled` or the catalog session +property `aggregation_pushdown_enabled`. + +A `count(distint)` pushdown may cause Pinot to run a full table scan with +significant performance impact. If you encounter this problem, you can disable +it with the catalog property `pinot.count-distinct-pushdown.enabled` or the +catalog session property `count_distinct_pushdown_enabled`. + +```{include} pushdown-correctness-behavior.fragment +``` diff --git a/430/_sources/connector/postgresql.md.txt b/430/_sources/connector/postgresql.md.txt new file mode 100644 index 000000000..9ca7f46fe --- /dev/null +++ b/430/_sources/connector/postgresql.md.txt @@ -0,0 +1,535 @@ +--- +myst: + substitutions: + default_domain_compaction_threshold: '`32`' +--- + +# PostgreSQL connector + +```{raw} html + +``` + +The PostgreSQL connector allows querying and creating tables in an +external [PostgreSQL](https://www.postgresql.org/) database. This can be used to join data between +different systems like PostgreSQL and Hive, or between different +PostgreSQL instances. + +## Requirements + +To connect to PostgreSQL, you need: + +- PostgreSQL 11.x or higher. +- Network access from the Trino coordinator and workers to PostgreSQL. + Port 5432 is the default port. + +## Configuration + +The connector can query a database on a PostgreSQL server. Create a catalog +properties file that specifies the PostgreSQL connector by setting the +`connector.name` to `postgresql`. + +For example, to access a database as the `example` catalog, create the file +`etc/catalog/example.properties`. Replace the connection properties as +appropriate for your setup: + +```text +connector.name=postgresql +connection-url=jdbc:postgresql://example.net:5432/database +connection-user=root +connection-password=secret +``` + +The `connection-url` defines the connection information and parameters to pass +to the PostgreSQL JDBC driver. The parameters for the URL are available in the +[PostgreSQL JDBC driver documentation](https://jdbc.postgresql.org/documentation/use/#connecting-to-the-database). +Some parameters can have adverse effects on the connector behavior or not work +with the connector. + +The `connection-user` and `connection-password` are typically required and +determine the user credentials for the connection, often a service user. You can +use {doc}`secrets ` to avoid actual values in the catalog +properties files. + +### Access to system tables + +The PostgreSQL connector supports reading [PostgreSQ catalog +tables](https://www.postgresql.org/docs/current/catalogs.html), such as +`pg_namespace`. The functionality is turned off by default, and can be enabled +using the `postgresql.include-system-tables` configuration property. + +You can see more details in the `pg_catalog` schema in the `example` catalog, +for example about the `pg_namespace` system table: + +```sql +SHOW TABLES FROM example.pg_catalog; +SELECT * FROM example.pg_catalog.pg_namespace; +``` + +(postgresql-tls)= + +### Connection security + +If you have TLS configured with a globally-trusted certificate installed on your +data source, you can enable TLS between your cluster and the data +source by appending a parameter to the JDBC connection string set in the +`connection-url` catalog configuration property. + +For example, with version 42 of the PostgreSQL JDBC driver, enable TLS by +appending the `ssl=true` parameter to the `connection-url` configuration +property: + +```properties +connection-url=jdbc:postgresql://example.net:5432/database?ssl=true +``` + +For more information on TLS configuration options, see the [PostgreSQL JDBC +driver documentation](https://jdbc.postgresql.org/documentation/use/#connecting-to-the-database). + +```{include} jdbc-authentication.fragment +``` + +### Multiple PostgreSQL databases or servers + +The PostgreSQL connector can only access a single database within +a PostgreSQL server. Thus, if you have multiple PostgreSQL databases, +or want to connect to multiple PostgreSQL servers, you must configure +multiple instances of the PostgreSQL connector. + +To add another catalog, simply add another properties file to `etc/catalog` +with a different name, making sure it ends in `.properties`. For example, +if you name the property file `sales.properties`, Trino creates a +catalog named `sales` using the configured connector. + +```{include} jdbc-common-configurations.fragment +``` + +```{include} query-comment-format.fragment +``` + +```{include} jdbc-domain-compaction-threshold.fragment +``` + +```{include} jdbc-procedures.fragment +``` + +```{include} jdbc-case-insensitive-matching.fragment +``` + +```{include} non-transactional-insert.fragment +``` + +(postgresql-type-mapping)= + +## Type mapping + +Because Trino and PostgreSQL each support types that the other does not, this +connector {ref}`modifies some types ` when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction. + +### PostgreSQL type to Trino type mapping + +The connector maps PostgreSQL types to the corresponding Trino types following +this table: + +```{eval-rst} +.. list-table:: PostgreSQL type to Trino type mapping + :widths: 30, 20, 50 + :header-rows: 1 + + * - PostgreSQL type + - Trino type + - Notes + * - ``BIT`` + - ``BOOLEAN`` + - + * - ``BOOLEAN`` + - ``BOOLEAN`` + - + * - ``SMALLINT`` + - ``SMALLINT`` + - + * - ``INTEGER`` + - ``INTEGER`` + - + * - ``BIGINT`` + - ``BIGINT`` + - + * - ``REAL`` + - ``REAL`` + - + * - ``DOUBLE`` + - ``DOUBLE`` + - + * - ``NUMERIC(p, s)`` + - ``DECIMAL(p, s)`` + - ``DECIMAL(p, s)`` is an alias of ``NUMERIC(p, s)``. See + :ref:`postgresql-decimal-type-handling` for more information. + * - ``CHAR(n)`` + - ``CHAR(n)`` + - + * - ``VARCHAR(n)`` + - ``VARCHAR(n)`` + - + * - ``ENUM`` + - ``VARCHAR`` + - + * - ``BYTEA`` + - ``VARBINARY`` + - + * - ``DATE`` + - ``DATE`` + - + * - ``TIME(n)`` + - ``TIME(n)`` + - + * - ``TIMESTAMP(n)`` + - ``TIMESTAMP(n)`` + - + * - ``TIMESTAMPTZ(n)`` + - ``TIMESTAMP(n) WITH TIME ZONE`` + - + * - ``MONEY`` + - ``VARCHAR`` + - + * - ``UUID`` + - ``UUID`` + - + * - ``JSON`` + - ``JSON`` + - + * - ``JSONB`` + - ``JSON`` + - + * - ``HSTORE`` + - ``MAP(VARCHAR, VARCHAR)`` + - + * - ``ARRAY`` + - Disabled, ``ARRAY``, or ``JSON`` + - See :ref:`postgresql-array-type-handling` for more information. +``` + +No other types are supported. + +### Trino type to PostgreSQL type mapping + +The connector maps Trino types to the corresponding PostgreSQL types following +this table: + +```{eval-rst} +.. list-table:: Trino type to PostgreSQL type mapping + :widths: 30, 20, 50 + :header-rows: 1 + + * - Trino type + - PostgreSQL type + - Notes + * - ``BOOLEAN`` + - ``BOOLEAN`` + - + * - ``SMALLINT`` + - ``SMALLINT`` + - + * - ``TINYINT`` + - ``SMALLINT`` + - + * - ``INTEGER`` + - ``INTEGER`` + - + * - ``BIGINT`` + - ``BIGINT`` + - + * - ``DOUBLE`` + - ``DOUBLE`` + - + * - ``DECIMAL(p, s)`` + - ``NUMERIC(p, s)`` + - ``DECIMAL(p, s)`` is an alias of ``NUMERIC(p, s)``. See + :ref:`postgresql-decimal-type-handling` for more information. + * - ``CHAR(n)`` + - ``CHAR(n)`` + - + * - ``VARCHAR(n)`` + - ``VARCHAR(n)`` + - + * - ``VARBINARY`` + - ``BYTEA`` + - + * - ``DATE`` + - ``DATE`` + - + * - ``TIME(n)`` + - ``TIME(n)`` + - + * - ``TIMESTAMP(n)`` + - ``TIMESTAMP(n)`` + - + * - ``TIMESTAMP(n) WITH TIME ZONE`` + - ``TIMESTAMPTZ(n)`` + - + * - ``UUID`` + - ``UUID`` + - + * - ``JSON`` + - ``JSONB`` + - + * - ``ARRAY`` + - ``ARRAY`` + - See :ref:`postgresql-array-type-handling` for more information. +``` + +No other types are supported. + +(postgresql-decimal-type-handling)= + +```{include} decimal-type-handling.fragment +``` + +(postgresql-array-type-handling)= + +### Array type handling + +The PostgreSQL array implementation does not support fixed dimensions whereas Trino +support only arrays with fixed dimensions. +You can configure how the PostgreSQL connector handles arrays with the `postgresql.array-mapping` configuration property in your catalog file +or the `array_mapping` session property. +The following values are accepted for this property: + +- `DISABLED` (default): array columns are skipped. +- `AS_ARRAY`: array columns are interpreted as Trino `ARRAY` type, for array columns with fixed dimensions. +- `AS_JSON`: array columns are interpreted as Trino `JSON` type, with no constraint on dimensions. + +```{include} jdbc-type-mapping.fragment +``` + +## Querying PostgreSQL + +The PostgreSQL connector provides a schema for every PostgreSQL schema. +You can see the available PostgreSQL schemas by running `SHOW SCHEMAS`: + +``` +SHOW SCHEMAS FROM example; +``` + +If you have a PostgreSQL schema named `web`, you can view the tables +in this schema by running `SHOW TABLES`: + +``` +SHOW TABLES FROM example.web; +``` + +You can see a list of the columns in the `clicks` table in the `web` database +using either of the following: + +``` +DESCRIBE example.web.clicks; +SHOW COLUMNS FROM example.web.clicks; +``` + +Finally, you can access the `clicks` table in the `web` schema: + +``` +SELECT * FROM example.web.clicks; +``` + +If you used a different name for your catalog properties file, use +that catalog name instead of `example` in the above examples. + +(postgresql-sql-support)= + +## SQL support + +The connector provides read access and write access to data and metadata in +PostgreSQL. In addition to the {ref}`globally available +` and {ref}`read operation ` +statements, the connector supports the following features: + +- {doc}`/sql/insert` +- {doc}`/sql/update` +- {doc}`/sql/delete` +- {doc}`/sql/truncate` +- {ref}`sql-schema-table-management` + +```{include} sql-update-limitation.fragment +``` + +```{include} sql-delete-limitation.fragment +``` + +```{include} alter-table-limitation.fragment +``` + +```{include} alter-schema-limitation.fragment +``` + +(postgresql-fte-support)= + +## Fault-tolerant execution support + +The connector supports {doc}`/admin/fault-tolerant-execution` of query +processing. Read and write operations are both supported with any retry policy. + +## Table functions + +The connector provides specific {doc}`table functions ` to +access PostgreSQL. + +(postgresql-query-function)= + +### `query(varchar) -> table` + +The `query` function allows you to query the underlying database directly. It +requires syntax native to PostgreSQL, because the full query is pushed down and +processed in PostgreSQL. This can be useful for accessing native features which +are not available in Trino or for improving query performance in situations +where running a query natively may be faster. + +```{include} query-passthrough-warning.fragment +``` + +As a simple example, query the `example` catalog and select an entire table: + +``` +SELECT + * +FROM + TABLE( + example.system.query( + query => 'SELECT + * + FROM + tpch.nation' + ) + ); +``` + +As a practical example, you can leverage +[frame exclusion from PostgresQL](https://www.postgresql.org/docs/current/sql-expressions.html#SYNTAX-WINDOW-FUNCTIONS) +when using window functions: + +``` +SELECT + * +FROM + TABLE( + example.system.query( + query => 'SELECT + *, + array_agg(week) OVER ( + ORDER BY + week + ROWS + BETWEEN 2 PRECEDING + AND 2 FOLLOWING + EXCLUDE GROUP + ) AS week, + array_agg(week) OVER ( + ORDER BY + day + ROWS + BETWEEN 2 PRECEDING + AND 2 FOLLOWING + EXCLUDE GROUP + ) AS all + FROM + test.time_data' + ) + ); +``` + +```{include} query-table-function-ordering.fragment +``` + +## Performance + +The connector includes a number of performance improvements, detailed in the +following sections. + +(postgresql-table-statistics)= + +### Table statistics + +The PostgreSQL connector can use {doc}`table and column statistics +` for {doc}`cost based optimizations +`, to improve query processing performance +based on the actual data in the data source. + +The statistics are collected by PostgreSQL and retrieved by the connector. + +To collect statistics for a table, execute the following statement in +PostgreSQL. + +```text +ANALYZE table_schema.table_name; +``` + +Refer to PostgreSQL documentation for additional `ANALYZE` options. + +(postgresql-pushdown)= + +### Pushdown + +The connector supports pushdown for a number of operations: + +- {ref}`join-pushdown` +- {ref}`limit-pushdown` +- {ref}`topn-pushdown` + +{ref}`Aggregate pushdown ` for the following functions: + +- {func}`avg` +- {func}`count` +- {func}`max` +- {func}`min` +- {func}`sum` +- {func}`stddev` +- {func}`stddev_pop` +- {func}`stddev_samp` +- {func}`variance` +- {func}`var_pop` +- {func}`var_samp` +- {func}`covar_pop` +- {func}`covar_samp` +- {func}`corr` +- {func}`regr_intercept` +- {func}`regr_slope` + +```{include} pushdown-correctness-behavior.fragment +``` + +```{include} join-pushdown-enabled-true.fragment +``` + +### Predicate pushdown support + +Predicates are pushed down for most types, including `UUID` and temporal +types, such as `DATE`. + +The connector does not support pushdown of range predicates, such as `>`, +`<`, or `BETWEEN`, on columns with {ref}`character string types +` like `CHAR` or `VARCHAR`. Equality predicates, such as +`IN` or `=`, and inequality predicates, such as `!=` on columns with +textual types are pushed down. This ensures correctness of results since the +remote data source may sort strings differently than Trino. + +In the following example, the predicate of the first query is not pushed down +since `name` is a column of type `VARCHAR` and `>` is a range predicate. +The other queries are pushed down. + +```sql +-- Not pushed down +SELECT * FROM nation WHERE name > 'CANADA'; +-- Pushed down +SELECT * FROM nation WHERE name != 'CANADA'; +SELECT * FROM nation WHERE name = 'CANADA'; +``` + +There is experimental support to enable pushdown of range predicates on columns +with character string types which can be enabled by setting the +`postgresql.experimental.enable-string-pushdown-with-collate` catalog +configuration property or the corresponding +`enable_string_pushdown_with_collate` session property to `true`. +Enabling this configuration will make the predicate of all the queries in the +above example get pushed down. diff --git a/430/_sources/connector/prometheus.md.txt b/430/_sources/connector/prometheus.md.txt new file mode 100644 index 000000000..357c68236 --- /dev/null +++ b/430/_sources/connector/prometheus.md.txt @@ -0,0 +1,132 @@ +# Prometheus connector + +```{raw} html + +``` + +The Prometheus connector allows reading +[Prometheus](https://prometheus.io/) +metrics as tables in Trino. + +The mechanism for querying Prometheus is to use the Prometheus HTTP API. Specifically, all queries are resolved to Prometheus Instant queries +with a form like: . +In this case the `up` metric is taken from the Trino query table name, `21d` is the duration of the query. The Prometheus `time` value +corresponds to the `TIMESTAMP` field. Trino queries are translated from their use of the `TIMESTAMP` field to a duration and time value +as needed. Trino splits are generated by dividing the query range into attempted equal chunks. + +## Requirements + +To query Prometheus, you need: + +- Network access from the Trino coordinator and workers to the Prometheus + server. The default port is 9090. +- Prometheus version 2.15.1 or later. + +## Configuration + +Create `etc/catalog/example.properties` to mount the Prometheus connector as +the `example` catalog, replacing the properties as appropriate: + +```text +connector.name=prometheus +prometheus.uri=http://localhost:9090 +prometheus.query.chunk.size.duration=1d +prometheus.max.query.range.duration=21d +prometheus.cache.ttl=30s +prometheus.bearer.token.file=/path/to/bearer/token/file +prometheus.read-timeout=10s +``` + +## Configuration properties + +The following configuration properties are available: + +| Property name | Description | +| ------------------------------------------- | -------------------------------------------------------------------------------------------- | +| `prometheus.uri` | Where to find Prometheus coordinator host | +| `prometheus.query.chunk.size.duration` | The duration of each query to Prometheus | +| `prometheus.max.query.range.duration` | Width of overall query to Prometheus, will be divided into query-chunk-size-duration queries | +| `prometheus.cache.ttl` | How long values from this config file are cached | +| `prometheus.auth.user` | Username for basic authentication | +| `prometheus.auth.password` | Password for basic authentication | +| `prometheus.bearer.token.file` | File holding bearer token if needed for access to Prometheus | +| `prometheus.read-timeout` | How much time a query to Prometheus has before timing out | +| `prometheus.case-insensitive-name-matching` | Match Prometheus metric names case insensitively. Defaults to `false` | + +## Not exhausting your Trino available heap + +The `prometheus.query.chunk.size.duration` and `prometheus.max.query.range.duration` are values to protect Trino from +too much data coming back from Prometheus. The `prometheus.max.query.range.duration` is the item of +particular interest. + +On a Prometheus instance that has been running for awhile and depending +on data retention settings, `21d` might be far too much. Perhaps `1h` might be a more reasonable setting. +In the case of `1h` it might be then useful to set `prometheus.query.chunk.size.duration` to `10m`, dividing the +query window into 6 queries each of which can be handled in a Trino split. + +Primarily query issuers can limit the amount of data returned by Prometheus by taking +advantage of `WHERE` clause limits on `TIMESTAMP`, setting an upper bound and lower bound that define +a relatively small window. For example: + +```sql +SELECT * FROM example.default.up WHERE TIMESTAMP > (NOW() - INTERVAL '10' second); +``` + +If the query does not include a WHERE clause limit, these config +settings are meant to protect against an unlimited query. + +## Bearer token authentication + +Prometheus can be setup to require a Authorization header with every query. The value in +`prometheus.bearer.token.file` allows for a bearer token to be read from the configured file. This file +is optional and not required unless your Prometheus setup requires it. + +(prometheus-type-mapping)= + +## Type mapping + +Because Trino and Prometheus each support types that the other does not, this +connector {ref}`modifies some types ` when reading data. + +The connector returns fixed columns that have a defined mapping to Trino types +according to the following table: + +```{eval-rst} +.. list-table:: Prometheus column to Trino type mapping + :widths: 50, 50 + :header-rows: 1 + + * - Prometheus column + - Trino type + * - ``labels`` + - ``MAP(VARCHAR,VARCHAR)`` + * - ``TIMESTAMP`` + - ``TIMESTAMP(3) WITH TIMEZONE`` + * - ``value`` + - ``DOUBLE`` +``` + +No other types are supported. + +The following example query result shows how the Prometheus `up` metric is +represented in Trino: + +```sql +SELECT * FROM example.default.up; +``` + +```text + labels | timestamp | value +--------------------------------------------------------+--------------------------------+------- +{instance=localhost:9090, job=prometheus, __name__=up} | 2022-09-01 06:18:54.481 +09:00 | 1.0 +{instance=localhost:9090, job=prometheus, __name__=up} | 2022-09-01 06:19:09.446 +09:00 | 1.0 +(2 rows) +``` + +(prometheus-sql-support)= + +## SQL support + +The connector provides {ref}`globally available ` and +{ref}`read operation ` statements to access data and +metadata in Prometheus. diff --git a/430/_sources/connector/redis.md.txt b/430/_sources/connector/redis.md.txt new file mode 100644 index 000000000..2b3051cf0 --- /dev/null +++ b/430/_sources/connector/redis.md.txt @@ -0,0 +1,316 @@ +# Redis connector + +```{raw} html + +``` + +The Redis connector allows querying of live data stored in [Redis](https://redis.io/). This can be +used to join data between different systems like Redis and Hive. + +Each Redis key/value pair is presented as a single row in Trino. Rows can be +broken down into cells by using table definition files. + +Currently, only Redis key of string and zset types are supported, only Redis value of +string and hash types are supported. + +## Requirements + +Requirements for using the connector in a catalog to connect to a Redis data +source are: + +- Redis 2.8.0 or higher (Redis Cluster is not supported) +- Network access, by default on port 6379, from the Trino coordinator and + workers to Redis. + +## Configuration + +To configure the Redis connector, create a catalog properties file +`etc/catalog/example.properties` with the following content, replacing the +properties as appropriate: + +```text +connector.name=redis +redis.table-names=schema1.table1,schema1.table2 +redis.nodes=host:port +``` + +### Multiple Redis servers + +You can have as many catalogs as you need. If you have additional +Redis servers, simply add another properties file to `etc/catalog` +with a different name, making sure it ends in `.properties`. + +## Configuration properties + +The following configuration properties are available: + +| Property name | Description | +| ----------------------------------- | ------------------------------------------------------------------------------------------------- | +| `redis.table-names` | List of all tables provided by the catalog | +| `redis.default-schema` | Default schema name for tables | +| `redis.nodes` | Location of the Redis server | +| `redis.scan-count` | Redis parameter for scanning of the keys | +| `redis.max-keys-per-fetch` | Get values associated with the specified number of keys in the redis command such as MGET(key...) | +| `redis.key-prefix-schema-table` | Redis keys have schema-name:table-name prefix | +| `redis.key-delimiter` | Delimiter separating schema_name and table_name if redis.key-prefix-schema-table is used | +| `redis.table-description-dir` | Directory containing table description files | +| `redis.table-description-cache-ttl` | The cache time for table description files | +| `redis.hide-internal-columns` | Controls whether internal columns are part of the table schema or not | +| `redis.database-index` | Redis database index | +| `redis.user` | Redis server username | +| `redis.password` | Redis server password | + +### `redis.table-names` + +Comma-separated list of all tables provided by this catalog. A table name +can be unqualified (simple name) and is placed into the default schema +(see below), or qualified with a schema name (`.`). + +For each table defined, a table description file (see below) may +exist. If no table description file exists, the +table only contains internal columns (see below). + +This property is optional; the connector relies on the table description files +specified in the `redis.table-description-dir` property. + +### `redis.default-schema` + +Defines the schema which will contain all tables that were defined without +a qualifying schema name. + +This property is optional; the default is `default`. + +### `redis.nodes` + +The `hostname:port` pair for the Redis server. + +This property is required; there is no default. + +Redis Cluster is not supported. + +### `redis.scan-count` + +The internal COUNT parameter for the Redis SCAN command when connector is using +SCAN to find keys for the data. This parameter can be used to tune performance +of the Redis connector. + +This property is optional; the default is `100`. + +### `redis.max-keys-per-fetch` + +The internal number of keys for the Redis MGET command and Pipeline HGETALL command +when connector is using these commands to find values of keys. This parameter can be +used to tune performance of the Redis connector. + +This property is optional; the default is `100`. + +### `redis.key-prefix-schema-table` + +If true, only keys prefixed with the `schema-name:table-name` are scanned +for a table, and all other keys are filtered out. If false, all keys are +scanned. + +This property is optional; the default is `false`. + +### `redis.key-delimiter` + +The character used for separating `schema-name` and `table-name` when +`redis.key-prefix-schema-table` is `true` + +This property is optional; the default is `:`. + +### `redis.table-description-dir` + +References a folder within Trino deployment that holds one or more JSON +files, which must end with `.json` and contain table description files. + +Note that the table description files will only be used by the Trino coordinator +node. + +This property is optional; the default is `etc/redis`. + +### `redis.table-description-cache-ttl` + +The Redis connector dynamically loads the table description files after waiting +for the time specified by this property. Therefore, there is no need to update +the `redis.table-names` property and restart the Trino service when adding, +updating, or deleting a file end with `.json` to `redis.table-description-dir` +folder. + +This property is optional; the default is `5m`. + +### `redis.hide-internal-columns` + +In addition to the data columns defined in a table description file, the +connector maintains a number of additional columns for each table. If +these columns are hidden, they can still be used in queries, but they do not +show up in `DESCRIBE ` or `SELECT *`. + +This property is optional; the default is `true`. + +### `redis.database-index` + +The Redis database to query. + +This property is optional; the default is `0`. + +### `redis.user` + +The username for Redis server. + +This property is optional; the default is `null`. + +### `redis.password` + +The password for password-protected Redis server. + +This property is optional; the default is `null`. + +## Internal columns + +For each defined table, the connector maintains the following columns: + +| Column name | Type | Description | +| ---------------- | ------- | ------------------------------------------------------------------------------------------------------------------------------------------ | +| `_key` | VARCHAR | Redis key. | +| `_value` | VARCHAR | Redis value corresponding to the key. | +| `_key_length` | BIGINT | Number of bytes in the key. | +| `_value_length` | BIGINT | Number of bytes in the value. | +| `_key_corrupt` | BOOLEAN | True if the decoder could not decode the key for this row. When true, data columns mapped from the key should be treated as invalid. | +| `_value_corrupt` | BOOLEAN | True if the decoder could not decode the message for this row. When true, data columns mapped from the value should be treated as invalid. | + +For tables without a table definition file, the `_key_corrupt` and +`_value_corrupt` columns are `false`. + +## Table definition files + +With the Redis connector it is possible to further reduce Redis key/value pairs into +granular cells, provided the key/value string follows a particular format. This process +defines new columns that can be further queried from Trino. + +A table definition file consists of a JSON definition for a table. The +name of the file can be arbitrary, but must end in `.json`. + +```text +{ + "tableName": ..., + "schemaName": ..., + "key": { + "dataFormat": ..., + "fields": [ + ... + ] + }, + "value": { + "dataFormat": ..., + "fields": [ + ... + ] + } +} +``` + +| Field | Required | Type | Description | +| ------------ | -------- | ----------- | --------------------------------------------------------------------------------- | +| `tableName` | required | string | Trino table name defined by this file. | +| `schemaName` | optional | string | Schema which will contain the table. If omitted, the default schema name is used. | +| `key` | optional | JSON object | Field definitions for data columns mapped to the value key. | +| `value` | optional | JSON object | Field definitions for data columns mapped to the value itself. | + +Please refer to the [Kafka connector](/connector/kafka) page for the description of the `dataFormat` as well as various available decoders. + +In addition to the above Kafka types, the Redis connector supports `hash` type for the `value` field which represent data stored in the Redis hash. + +```text +{ + "tableName": ..., + "schemaName": ..., + "value": { + "dataFormat": "hash", + "fields": [ + ... + ] + } +} +``` + +## Type mapping + +Because Trino and Redis each support types that the other does not, this +connector {ref}`maps some types ` when reading data. Type +mapping depends on the RAW, CSV, JSON, and AVRO file formats. + +### Row decoding + +A decoder is used to map data to table columns. + +The connector contains the following decoders: + +- `raw`: Message is not interpreted; ranges of raw message bytes are mapped + to table columns. +- `csv`: Message is interpreted as comma separated message, and fields are + mapped to table columns. +- `json`: Message is parsed as JSON, and JSON fields are mapped to table + columns. +- `avro`: Message is parsed based on an Avro schema, and Avro fields are + mapped to table columns. + +:::{note} +If no table definition file exists for a table, the `dummy` decoder is +used, which does not expose any columns. +::: + +```{include} raw-decoder.fragment +``` + +```{include} csv-decoder.fragment +``` + +```{include} json-decoder.fragment +``` + +```{include} avro-decoder.fragment +``` + +(redis-sql-support)= + +## SQL support + +The connector provides {ref}`globally available ` and +{ref}`read operation ` statements to access data and +metadata in Redis. + +## Performance + +The connector includes a number of performance improvements, detailed in the +following sections. + +(redis-pushdown)= + +### Pushdown + +```{include} pushdown-correctness-behavior.fragment +``` + +(redis-predicate-pushdown)= + +#### Predicate pushdown support + +The connector supports pushdown of keys of `string` type only, the `zset` +type is not supported. Key pushdown is not supported when multiple key fields +are defined in the table definition file. + +The connector supports pushdown of equality predicates, such as `IN` or `=`. +Inequality predicates, such as `!=`, and range predicates, such as `>`, +`<`, or `BETWEEN` are not pushed down. + +In the following example, the predicate of the first query is not pushed down +since `>` is a range predicate. The other queries are pushed down: + +```sql +-- Not pushed down +SELECT * FROM nation WHERE redis_key > 'CANADA'; +-- Pushed down +SELECT * FROM nation WHERE redis_key = 'CANADA'; +SELECT * FROM nation WHERE redis_key IN ('CANADA', 'POLAND'); +``` diff --git a/430/_sources/connector/redshift.md.txt b/430/_sources/connector/redshift.md.txt new file mode 100644 index 000000000..84d9a1405 --- /dev/null +++ b/430/_sources/connector/redshift.md.txt @@ -0,0 +1,210 @@ +--- +myst: + substitutions: + default_domain_compaction_threshold: '`32`' +--- + +# Redshift connector + +```{raw} html + +``` + +The Redshift connector allows querying and creating tables in an +external [Amazon Redshift](https://aws.amazon.com/redshift/) cluster. This can be used to join data between +different systems like Redshift and Hive, or between two different +Redshift clusters. + +## Requirements + +To connect to Redshift, you need: + +- Network access from the Trino coordinator and workers to Redshift. + Port 5439 is the default port. + +## Configuration + +To configure the Redshift connector, create a catalog properties file in +`etc/catalog` named, for example, `example.properties`, to mount the +Redshift connector as the `example` catalog. Create the file with the +following contents, replacing the connection properties as appropriate for your +setup: + +```text +connector.name=redshift +connection-url=jdbc:redshift://example.net:5439/database +connection-user=root +connection-password=secret +``` + +The `connection-user` and `connection-password` are typically required and +determine the user credentials for the connection, often a service user. You can +use {doc}`secrets ` to avoid actual values in the catalog +properties files. + +(redshift-tls)= + +### Connection security + +If you have TLS configured with a globally-trusted certificate installed on your +data source, you can enable TLS between your cluster and the data +source by appending a parameter to the JDBC connection string set in the +`connection-url` catalog configuration property. + +For example, on version 2.1 of the Redshift JDBC driver, TLS/SSL is enabled by +default with the `SSL` parameter. You can disable or further configure TLS +by appending parameters to the `connection-url` configuration property: + +```properties +connection-url=jdbc:redshift://example.net:5439/database;SSL=TRUE; +``` + +For more information on TLS configuration options, see the [Redshift JDBC driver +documentation](https://docs.aws.amazon.com/redshift/latest/mgmt/jdbc20-configuration-options.html#jdbc20-ssl-option). + +```{include} jdbc-authentication.fragment +``` + +### Multiple Redshift databases or clusters + +The Redshift connector can only access a single database within +a Redshift cluster. Thus, if you have multiple Redshift databases, +or want to connect to multiple Redshift clusters, you must configure +multiple instances of the Redshift connector. + +To add another catalog, simply add another properties file to `etc/catalog` +with a different name, making sure it ends in `.properties`. For example, +if you name the property file `sales.properties`, Trino creates a +catalog named `sales` using the configured connector. + +```{include} jdbc-common-configurations.fragment +``` + +```{include} query-comment-format.fragment +``` + +```{include} jdbc-domain-compaction-threshold.fragment +``` + +```{include} jdbc-procedures.fragment +``` + +```{include} jdbc-case-insensitive-matching.fragment +``` + +```{include} non-transactional-insert.fragment +``` + +## Querying Redshift + +The Redshift connector provides a schema for every Redshift schema. +You can see the available Redshift schemas by running `SHOW SCHEMAS`: + +``` +SHOW SCHEMAS FROM example; +``` + +If you have a Redshift schema named `web`, you can view the tables +in this schema by running `SHOW TABLES`: + +``` +SHOW TABLES FROM example.web; +``` + +You can see a list of the columns in the `clicks` table in the `web` database +using either of the following: + +``` +DESCRIBE example.web.clicks; +SHOW COLUMNS FROM example.web.clicks; +``` + +Finally, you can access the `clicks` table in the `web` schema: + +``` +SELECT * FROM example.web.clicks; +``` + +If you used a different name for your catalog properties file, use that catalog +name instead of `example` in the above examples. + +(redshift-type-mapping)= + +## Type mapping + +```{include} jdbc-type-mapping.fragment +``` + +(redshift-sql-support)= + +## SQL support + +The connector provides read access and write access to data and metadata in +Redshift. In addition to the {ref}`globally available +` and {ref}`read operation ` +statements, the connector supports the following features: + +- {doc}`/sql/insert` +- {doc}`/sql/update` +- {doc}`/sql/delete` +- {doc}`/sql/truncate` +- {ref}`sql-schema-table-management` + +```{include} sql-update-limitation.fragment +``` + +```{include} sql-delete-limitation.fragment +``` + +```{include} alter-table-limitation.fragment +``` + +```{include} alter-schema-limitation.fragment +``` + +(redshift-fte-support)= + +## Fault-tolerant execution support + +The connector supports {doc}`/admin/fault-tolerant-execution` of query +processing. Read and write operations are both supported with any retry policy. + +## Table functions + +The connector provides specific {doc}`table functions ` to +access Redshift. + +(redshift-query-function)= + +### `query(varchar) -> table` + +The `query` function allows you to query the underlying database directly. It +requires syntax native to Redshift, because the full query is pushed down and +processed in Redshift. This can be useful for accessing native features which +are not implemented in Trino or for improving query performance in situations +where running a query natively may be faster. + +```{include} query-passthrough-warning.fragment +``` + +For example, query the `example` catalog and select the top 10 nations by +population: + +``` +SELECT + * +FROM + TABLE( + example.system.query( + query => 'SELECT + TOP 10 * + FROM + tpch.nation + ORDER BY + population DESC' + ) + ); +``` + +```{include} query-table-function-ordering.fragment +``` diff --git a/430/_sources/connector/singlestore.md.txt b/430/_sources/connector/singlestore.md.txt new file mode 100644 index 000000000..b3512d9f0 --- /dev/null +++ b/430/_sources/connector/singlestore.md.txt @@ -0,0 +1,369 @@ +--- +myst: + substitutions: + default_domain_compaction_threshold: '`32`' +--- + +# SingleStore connector + +```{raw} html + +``` + +The SingleStore (formerly known as MemSQL) connector allows querying and +creating tables in an external SingleStore database. + +## Requirements + +To connect to SingleStore, you need: + +- SingleStore version 7.1.4 or higher. +- Network access from the Trino coordinator and workers to SingleStore. Port + 3306 is the default port. + +(singlestore-configuration)= + +## Configuration + +To configure the SingleStore connector, create a catalog properties file in +`etc/catalog` named, for example, `example.properties`, to mount the +SingleStore connector as the `example` catalog. Create the file with the +following contents, replacing the connection properties as appropriate for your +setup: + +```text +connector.name=singlestore +connection-url=jdbc:singlestore://example.net:3306 +connection-user=root +connection-password=secret +``` + +The `connection-url` defines the connection information and parameters to pass +to the SingleStore JDBC driver. The supported parameters for the URL are +available in the [SingleStore JDBC driver documentation](https://docs.singlestore.com/db/v7.6/en/developer-resources/connect-with-application-development-tools/connect-with-java-jdbc/the-singlestore-jdbc-driver.html#connection-string-parameters). + +The `connection-user` and `connection-password` are typically required and +determine the user credentials for the connection, often a service user. You can +use {doc}`secrets ` to avoid actual values in the catalog +properties files. + +(singlestore-tls)= + +### Connection security + +If you have TLS configured with a globally-trusted certificate installed on your +data source, you can enable TLS between your cluster and the data +source by appending a parameter to the JDBC connection string set in the +`connection-url` catalog configuration property. + +Enable TLS between your cluster and SingleStore by appending the `useSsl=true` +parameter to the `connection-url` configuration property: + +```properties +connection-url=jdbc:singlestore://example.net:3306/?useSsl=true +``` + +For more information on TLS configuration options, see the [JDBC driver +documentation](https://docs.singlestore.com/db/v7.6/en/developer-resources/connect-with-application-development-tools/connect-with-java-jdbc/the-singlestore-jdbc-driver.html#tls-parameters). + +### Multiple SingleStore servers + +You can have as many catalogs as you need, so if you have additional +SingleStore servers, simply add another properties file to `etc/catalog` +with a different name (making sure it ends in `.properties`). For +example, if you name the property file `sales.properties`, Trino +will create a catalog named `sales` using the configured connector. + +```{include} jdbc-common-configurations.fragment +``` + +```{include} query-comment-format.fragment +``` + +```{include} jdbc-domain-compaction-threshold.fragment +``` + +```{include} jdbc-procedures.fragment +``` + +```{include} jdbc-case-insensitive-matching.fragment +``` + +```{include} non-transactional-insert.fragment +``` + +## Querying SingleStore + +The SingleStore connector provides a schema for every SingleStore *database*. +You can see the available SingleStore databases by running `SHOW SCHEMAS`: + +``` +SHOW SCHEMAS FROM example; +``` + +If you have a SingleStore database named `web`, you can view the tables +in this database by running `SHOW TABLES`: + +``` +SHOW TABLES FROM example.web; +``` + +You can see a list of the columns in the `clicks` table in the `web` +database using either of the following: + +``` +DESCRIBE example.web.clicks; +SHOW COLUMNS FROM example.web.clicks; +``` + +Finally, you can access the `clicks` table in the `web` database: + +``` +SELECT * FROM example.web.clicks; +``` + +If you used a different name for your catalog properties file, use +that catalog name instead of `example` in the above examples. + +(singlestore-type-mapping)= + +## Type mapping + +Because Trino and Singlestore each support types that the other does not, this +connector {ref}`modifies some types ` when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction. + +### Singlestore to Trino type mapping + +The connector maps Singlestore types to the corresponding Trino types following +this table: + +```{eval-rst} +.. list-table:: Singlestore to Trino type mapping + :widths: 30, 20, 50 + :header-rows: 1 + + * - Singlestore type + - Trino type + - Notes + * - ``BIT`` + - ``BOOLEAN`` + - + * - ``BOOLEAN`` + - ``BOOLEAN`` + - + * - ``TINYINT`` + - ``TINYINT`` + - + * - ``TINYINT UNSIGNED`` + - ``SMALLINT`` + - + * - ``SMALLINT`` + - ``SMALLINT`` + - + * - ``SMALLINT UNSIGNED`` + - ``INTEGER`` + - + * - ``INTEGER`` + - ``INTEGER`` + - + * - ``INTEGER UNSIGNED`` + - ``BIGINT`` + - + * - ``BIGINT`` + - ``BIGINT`` + - + * - ``BIGINT UNSIGNED`` + - ``DECIMAL(20, 0)`` + - + * - ``DOUBLE`` + - ``DOUBLE`` + - + * - ``REAL`` + - ``DOUBLE`` + - + * - ``DECIMAL(p, s)`` + - ``DECIMAL(p, s)`` + - See :ref:`Singlestore DECIMAL type handling ` + * - ``CHAR(n)`` + - ``CHAR(n)`` + - + * - ``TINYTEXT`` + - ``VARCHAR(255)`` + - + * - ``TEXT`` + - ``VARCHAR(65535)`` + - + * - ``MEDIUMTEXT`` + - ``VARCHAR(16777215)`` + - + * - ``LONGTEXT`` + - ``VARCHAR`` + - + * - ``VARCHAR(n)`` + - ``VARCHAR(n)`` + - + * - ``LONGBLOB`` + - ``VARBINARY`` + - + * - ``DATE`` + - ``DATE`` + - + * - ``TIME`` + - ``TIME(0)`` + - + * - ``TIME(6)`` + - ``TIME(6)`` + - + * - ``DATETIME`` + - ``TIMESTAMP(0)`` + - + * - ``DATETIME(6)`` + - ``TIMESTAMP(6)`` + - + * - ``JSON`` + - ``JSON`` + - +``` + +No other types are supported. + +### Trino to Singlestore type mapping + +The connector maps Trino types to the corresponding Singlestore types following +this table: + +```{eval-rst} +.. list-table:: Trino to Singlestore type mapping + :widths: 30, 20, 50 + :header-rows: 1 + + * - Trino type + - Singlestore type + - Notes + * - ``BOOLEAN`` + - ``BOOLEAN`` + - + * - ``TINYINT`` + - ``TINYINT`` + - + * - ``SMALLINT`` + - ``SMALLINT`` + - + * - ``INTEGER`` + - ``INTEGER`` + - + * - ``BIGINT`` + - ``BIGINT`` + - + * - ``DOUBLE`` + - ``DOUBLE`` + - + * - ``REAL`` + - ``FLOAT`` + - + * - ``DECIMAL(p, s)`` + - ``DECIMAL(p, s)`` + - See :ref:`Singlestore DECIMAL type handling ` + * - ``CHAR(n)`` + - ``CHAR(n)`` + - + * - ``VARCHAR(65535)`` + - ``TEXT`` + - + * - ``VARCHAR(16777215)`` + - ``MEDIUMTEXT`` + - + * - ``VARCHAR`` + - ``LONGTEXT`` + - + * - ``VARCHAR(n)`` + - ``VARCHAR(n)`` + - + * - ``VARBINARY`` + - ``LONGBLOB`` + - + * - ``DATE`` + - ``DATE`` + - + * - ``TIME(0)`` + - ``TIME`` + - + * - ``TIME(6)`` + - ``TIME(6)`` + - + * - ``TIMESTAMP(0)`` + - ``DATETIME`` + - + * - ``TIMESTAMP(6)`` + - ``DATETIME(6)`` + - + * - ``JSON`` + - ``JSON`` + - +``` + +No other types are supported. + +(singlestore-decimal-handling)= + +```{include} decimal-type-handling.fragment +``` + +```{include} jdbc-type-mapping.fragment +``` + +(singlestore-sql-support)= + +## SQL support + +The connector provides read access and write access to data and metadata in +a SingleStore database. In addition to the {ref}`globally available +` and {ref}`read operation ` +statements, the connector supports the following features: + +- {doc}`/sql/insert` +- {doc}`/sql/update` +- {doc}`/sql/delete` +- {doc}`/sql/truncate` +- {doc}`/sql/create-table` +- {doc}`/sql/create-table-as` +- {doc}`/sql/drop-table` +- {doc}`/sql/alter-table` +- {doc}`/sql/create-schema` +- {doc}`/sql/drop-schema` + +```{include} sql-update-limitation.fragment +``` + +```{include} sql-delete-limitation.fragment +``` + +```{include} alter-table-limitation.fragment +``` + +## Performance + +The connector includes a number of performance improvements, detailed in the +following sections. + +(singlestore-pushdown)= + +### Pushdown + +The connector supports pushdown for a number of operations: + +- {ref}`join-pushdown` +- {ref}`limit-pushdown` +- {ref}`topn-pushdown` + +```{include} pushdown-correctness-behavior.fragment +``` + +```{include} join-pushdown-enabled-false.fragment +``` + +```{include} no-pushdown-text-type.fragment +``` diff --git a/430/_sources/connector/sqlserver.md.txt b/430/_sources/connector/sqlserver.md.txt new file mode 100644 index 000000000..592141a76 --- /dev/null +++ b/430/_sources/connector/sqlserver.md.txt @@ -0,0 +1,581 @@ +--- +myst: + substitutions: + default_domain_compaction_threshold: '`32`' +--- + +# SQL Server connector + +```{raw} html + +``` + +The SQL Server connector allows querying and creating tables in an external +[Microsoft SQL Server](https://www.microsoft.com/sql-server/) database. This +can be used to join data between different systems like SQL Server and Hive, or +between two different SQL Server instances. + +## Requirements + +To connect to SQL Server, you need: + +- SQL Server 2012 or higher, or Azure SQL Database. +- Network access from the Trino coordinator and workers to SQL Server. + Port 1433 is the default port. + +## Configuration + +The connector can query a single database on a given SQL Server instance. Create +a catalog properties file that specifies the SQL server connector by setting the +`connector.name` to `sqlserver`. + +For example, to access a database as `example`, create the file +`etc/catalog/example.properties`. Replace the connection properties as +appropriate for your setup: + +```properties +connector.name=sqlserver +connection-url=jdbc:sqlserver://:;databaseName=;encrypt=false +connection-user=root +connection-password=secret +``` + +The `connection-url` defines the connection information and parameters to pass +to the SQL Server JDBC driver. The supported parameters for the URL are +available in the [SQL Server JDBC driver documentation](https://docs.microsoft.com/sql/connect/jdbc/building-the-connection-url). + +The `connection-user` and `connection-password` are typically required and +determine the user credentials for the connection, often a service user. You can +use {doc}`secrets ` to avoid actual values in the catalog +properties files. + +(sqlserver-tls)= + +### Connection security + +The JDBC driver, and therefore the connector, automatically use Transport Layer +Security (TLS) encryption and certificate validation. This requires a suitable +TLS certificate configured on your SQL Server database host. + +If you do not have the necessary configuration established, you can disable +encryption in the connection string with the `encrypt` property: + +```properties +connection-url=jdbc:sqlserver://:;databaseName=;encrypt=false +``` + +Further parameters like `trustServerCertificate`, `hostNameInCertificate`, +`trustStore`, and `trustStorePassword` are details in the [TLS section of +SQL Server JDBC driver documentation](https://docs.microsoft.com/sql/connect/jdbc/using-ssl-encryption). + +```{include} jdbc-authentication.fragment +``` + +### Multiple SQL Server databases or servers + +The SQL Server connector can only access a single SQL Server database +within a single catalog. Thus, if you have multiple SQL Server databases, +or want to connect to multiple SQL Server instances, you must configure +multiple instances of the SQL Server connector. + +To add another catalog, simply add another properties file to `etc/catalog` +with a different name, making sure it ends in `.properties`. For example, +if you name the property file `sales.properties`, Trino creates a +catalog named `sales` using the configured connector. + +```{include} jdbc-common-configurations.fragment +``` + +```{include} query-comment-format.fragment +``` + +```{include} jdbc-domain-compaction-threshold.fragment +``` + +### Specific configuration properties + +The SQL Server connector supports additional catalog properties to configure the +behavior of the connector and the issues queries to the database. + +```{eval-rst} +.. list-table:: + :widths: 45, 55 + :header-rows: 1 + + * - Property name + - Description + * - ``sqlserver.snapshot-isolation.disabled`` + - Control the automatic use of snapshot isolation for transactions issued by + Trino in SQL Server. Defaults to ``false``, which means that snapshot + isolation is enabled. +``` + +```{include} jdbc-procedures.fragment +``` + +```{include} jdbc-case-insensitive-matching.fragment +``` + +```{include} non-transactional-insert.fragment +``` + +## Querying SQL Server + +The SQL Server connector provides access to all schemas visible to the specified +user in the configured database. For the following examples, assume the SQL +Server catalog is `example`. + +You can see the available schemas by running `SHOW SCHEMAS`: + +``` +SHOW SCHEMAS FROM example; +``` + +If you have a schema named `web`, you can view the tables +in this schema by running `SHOW TABLES`: + +``` +SHOW TABLES FROM example.web; +``` + +You can see a list of the columns in the `clicks` table in the `web` database +using either of the following: + +``` +DESCRIBE example.web.clicks; +SHOW COLUMNS FROM example.web.clicks; +``` + +Finally, you can query the `clicks` table in the `web` schema: + +``` +SELECT * FROM example.web.clicks; +``` + +If you used a different name for your catalog properties file, use +that catalog name instead of `example` in the above examples. + +(sqlserver-type-mapping)= + +## Type mapping + +Because Trino and SQL Server each support types that the other does not, this +connector {ref}`modifies some types ` when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction. + +### SQL Server type to Trino type mapping + +The connector maps SQL Server types to the corresponding Trino types following this table: + +```{eval-rst} +.. list-table:: SQL Server type to Trino type mapping + :widths: 30, 20, 50 + :header-rows: 1 + + * - SQL Server database type + - Trino type + - Notes + * - ``BIT`` + - ``BOOLEAN`` + - + * - ``TINYINT`` + - ``SMALLINT`` + - SQL Server ``TINYINT`` is actually ``unsigned TINYINT`` + * - ``SMALLINT`` + - ``SMALLINT`` + - + * - ``INTEGER`` + - ``INTEGER`` + - + * - ``BIGINT`` + - ``BIGINT`` + - + * - ``DOUBLE PRECISION`` + - ``DOUBLE`` + - + * - ``FLOAT[(n)]`` + - ``REAL`` or ``DOUBLE`` + - See :ref:`sqlserver-numeric-mapping` + * - ``REAL`` + - ``REAL`` + - + * - ``DECIMAL[(p[, s])]``, ``NUMERIC[(p[, s])]`` + - ``DECIMAL(p, s)`` + - + * - ``CHAR[(n)]`` + - ``CHAR(n)`` + - ``1 <= n <= 8000`` + * - ``NCHAR[(n)]`` + - ``CHAR(n)`` + - ``1 <= n <= 4000`` + * - ``VARCHAR[(n | max)]``, ``NVARCHAR[(n | max)]`` + - ``VARCHAR(n)`` + - ``1 <= n <= 8000``, ``max = 2147483647`` + * - ``TEXT`` + - ``VARCHAR(2147483647)`` + - + * - ``NTEXT`` + - ``VARCHAR(1073741823)`` + - + * - ``VARBINARY[(n | max)]`` + - ``VARBINARY`` + - ``1 <= n <= 8000``, ``max = 2147483647`` + * - ``DATE`` + - ``DATE`` + - + * - ``TIME[(n)]`` + - ``TIME(n)`` + - ``0 <= n <= 7`` + * - ``DATETIME2[(n)]`` + - ``TIMESTAMP(n)`` + - ``0 <= n <= 7`` + * - ``SMALLDATETIME`` + - ``TIMESTAMP(0)`` + - + * - ``DATETIMEOFFSET[(n)]`` + - ``TIMESTAMP(n) WITH TIME ZONE`` + - ``0 <= n <= 7`` +``` + +### Trino type to SQL Server type mapping + +The connector maps Trino types to the corresponding SQL Server types following this table: + +```{eval-rst} +.. list-table:: Trino type to SQL Server type mapping + :widths: 30, 20, 50 + :header-rows: 1 + + * - Trino type + - SQL Server type + - Notes + * - ``BOOLEAN`` + - ``BIT`` + - + * - ``TINYINT`` + - ``TINYINT`` + - Trino only supports writing values belonging to ``[0, 127]`` + * - ``SMALLINT`` + - ``SMALLINT`` + - + * - ``INTEGER`` + - ``INTEGER`` + - + * - ``BIGINT`` + - ``BIGINT`` + - + * - ``REAL`` + - ``REAL`` + - + * - ``DOUBLE`` + - ``DOUBLE PRECISION`` + - + * - ``DECIMAL(p, s)`` + - ``DECIMAL(p, s)`` + - + * - ``CHAR(n)`` + - ``NCHAR(n)`` or ``NVARCHAR(max)`` + - See :ref:`sqlserver-character-mapping` + * - ``VARCHAR(n)`` + - ``NVARCHAR(n)`` or ``NVARCHAR(max)`` + - See :ref:`sqlserver-character-mapping` + * - ``VARBINARY`` + - ``VARBINARY(max)`` + - + * - ``DATE`` + - ``DATE`` + - + * - ``TIME(n)`` + - ``TIME(n)`` + - ``0 <= n <= 7`` + * - ``TIMESTAMP(n)`` + - ``DATETIME2(n)`` + - ``0 <= n <= 7`` +``` + +Complete list of [SQL Server data types](https://msdn.microsoft.com/library/ms187752.aspx). + +(sqlserver-numeric-mapping)= + +### Numeric type mapping + +For SQL Server `FLOAT[(n)]`: + +- If `n` is not specified maps to Trino `Double` +- If `1 <= n <= 24` maps to Trino `REAL` +- If `24 < n <= 53` maps to Trino `DOUBLE` + +(sqlserver-character-mapping)= + +### Character type mapping + +For Trino `CHAR(n)`: + +- If `1 <= n <= 4000` maps SQL Server `NCHAR(n)` +- If `n > 4000` maps SQL Server `NVARCHAR(max)` + +For Trino `VARCHAR(n)`: + +- If `1 <= n <= 4000` maps SQL Server `NVARCHAR(n)` +- If `n > 4000` maps SQL Server `NVARCHAR(max)` + +```{include} jdbc-type-mapping.fragment +``` + +(sqlserver-sql-support)= + +## SQL support + +The connector provides read access and write access to data and metadata in SQL +Server. In addition to the {ref}`globally available ` +and {ref}`read operation ` statements, the connector +supports the following features: + +- {doc}`/sql/insert` +- {doc}`/sql/update` +- {doc}`/sql/delete` +- {doc}`/sql/truncate` +- {ref}`sql-schema-table-management` + +```{include} sql-update-limitation.fragment +``` + +```{include} sql-delete-limitation.fragment +``` + +```{include} alter-table-limitation.fragment +``` + +(sqlserver-fte-support)= + +## Fault-tolerant execution support + +The connector supports {doc}`/admin/fault-tolerant-execution` of query +processing. Read and write operations are both supported with any retry policy. + +## Table functions + +The connector provides specific {doc}`table functions ` to +access SQL Server. + +(sqlserver-query-function)= + +### `query(varchar) -> table` + +The `query` function allows you to query the underlying database directly. It +requires syntax native to SQL Server, because the full query is pushed down and +processed in SQL Server. This can be useful for accessing native features which +are not implemented in Trino or for improving query performance in situations +where running a query natively may be faster. + +```{include} query-passthrough-warning.fragment +``` + +For example, query the `example` catalog and select the top 10 percent of +nations by population: + +``` +SELECT + * +FROM + TABLE( + example.system.query( + query => 'SELECT + TOP(10) PERCENT * + FROM + tpch.nation + ORDER BY + population DESC' + ) + ); +``` + +(sqlserver-procedure-function)= + +### `procedure(varchar) -> table` + +The `procedure` function allows you to run stored procedures on the underlying +database directly. It requires syntax native to SQL Server, because the full query +is pushed down and processed in SQL Server. In order to use this table function set +`sqlserver.experimental.stored-procedure-table-function-enabled` to `true`. + +:::{note} +The `procedure` function does not support running StoredProcedures that return multiple statements, +use a non-select statement, use output parameters, or use conditional statements. +::: + +:::{warning} +This feature is experimental only. The function has security implication and syntax might change and +be backward incompatible. +::: + +The follow example runs the stored procedure `employee_sp` in the `example` catalog and the +`example_schema` schema in the underlying SQL Server database: + +``` +SELECT + * +FROM + TABLE( + example.system.procedure( + query => 'EXECUTE example_schema.employee_sp' + ) + ); +``` + +If the stored procedure `employee_sp` requires any input +append the parameter value to the procedure statement: + +``` +SELECT + * +FROM + TABLE( + example.system.procedure( + query => 'EXECUTE example_schema.employee_sp 0' + ) + ); +``` + +```{include} query-table-function-ordering.fragment +``` + +## Performance + +The connector includes a number of performance improvements, detailed in the +following sections. + +(sqlserver-table-statistics)= + +### Table statistics + +The SQL Server connector can use {doc}`table and column statistics +` for {doc}`cost based optimizations +`, to improve query processing performance +based on the actual data in the data source. + +The statistics are collected by SQL Server and retrieved by the connector. + +The connector can use information stored in single-column statistics. SQL Server +Database can automatically create column statistics for certain columns. If +column statistics are not created automatically for a certain column, you can +create them by executing the following statement in SQL Server Database. + +```sql +CREATE STATISTICS example_statistics_name ON table_schema.table_name (column_name); +``` + +SQL Server Database routinely updates the statistics. In some cases, you may +want to force statistics update (e.g. after defining new column statistics or +after changing data in the table). You can do that by executing the following +statement in SQL Server Database. + +```sql +UPDATE STATISTICS table_schema.table_name; +``` + +Refer to SQL Server documentation for information about options, limitations and +additional considerations. + +(sqlserver-pushdown)= + +### Pushdown + +The connector supports pushdown for a number of operations: + +- {ref}`join-pushdown` +- {ref}`limit-pushdown` +- {ref}`topn-pushdown` + +{ref}`Aggregate pushdown ` for the following functions: + +- {func}`avg` +- {func}`count` +- {func}`max` +- {func}`min` +- {func}`sum` +- {func}`stddev` +- {func}`stddev_pop` +- {func}`stddev_samp` +- {func}`variance` +- {func}`var_pop` +- {func}`var_samp` + +```{include} pushdown-correctness-behavior.fragment +``` + +```{include} join-pushdown-enabled-true.fragment +``` + +#### Predicate pushdown support + +The connector supports pushdown of predicates on `VARCHAR` and `NVARCHAR` +columns if the underlying columns in SQL Server use a case-sensitive [collation](https://learn.microsoft.com/en-us/sql/relational-databases/collations/collation-and-unicode-support?view=sql-server-ver16). + +The following operators are pushed down: + +- `=` +- `<>` +- `IN` +- `NOT IN` + +To ensure correct results, operators are not pushed down for columns using a +case-insensitive collation. + +(sqlserver-bulk-insert)= + +### Bulk insert + +You can optionally use the [bulk copy API](https://docs.microsoft.com/sql/connect/jdbc/use-bulk-copy-api-batch-insert-operation) +to drastically speed up write operations. + +Enable bulk copying and a lock on the destination table to meet [minimal +logging requirements](https://docs.microsoft.com/sql/relational-databases/import-export/prerequisites-for-minimal-logging-in-bulk-import). + +The following table shows the relevant catalog configuration properties and +their default values: + +```{eval-rst} +.. list-table:: Bulk load properties + :widths: 30, 60, 10 + :header-rows: 1 + + * - Property name + - Description + - Default + * - ``sqlserver.bulk-copy-for-write.enabled`` + - Use the SQL Server bulk copy API for writes. The corresponding catalog + session property is ``bulk_copy_for_write``. + - ``false`` + * - ``sqlserver.bulk-copy-for-write.lock-destination-table`` + - Obtain a bulk update lock on the destination table for write operations. + The corresponding catalog session property is + ``bulk_copy_for_write_lock_destination_table``. Setting is only used when + ``bulk-copy-for-write.enabled=true``. + - ``false`` +``` + +Limitations: + +- Column names with leading and trailing spaces are not supported. + +## Data compression + +You can specify the [data compression policy for SQL Server tables](https://docs.microsoft.com/sql/relational-databases/data-compression/data-compression) +with the `data_compression` table property. Valid policies are `NONE`, `ROW` or `PAGE`. + +Example: + +``` +CREATE TABLE example_schema.scientists ( + recordkey VARCHAR, + name VARCHAR, + age BIGINT, + birthday DATE +) +WITH ( + data_compression = 'ROW' +); +``` diff --git a/430/_sources/connector/system.md.txt b/430/_sources/connector/system.md.txt new file mode 100644 index 000000000..a4f034bbe --- /dev/null +++ b/430/_sources/connector/system.md.txt @@ -0,0 +1,162 @@ +# System connector + +The System connector provides information and metrics about the currently +running Trino cluster. It makes this available via normal SQL queries. + +## Configuration + +The System connector doesn't need to be configured: it is automatically +available via a catalog named `system`. + +## Using the System connector + +List the available system schemas: + +``` +SHOW SCHEMAS FROM system; +``` + +List the tables in one of the schemas: + +``` +SHOW TABLES FROM system.runtime; +``` + +Query one of the tables: + +``` +SELECT * FROM system.runtime.nodes; +``` + +Kill a running query: + +``` +CALL system.runtime.kill_query(query_id => '20151207_215727_00146_tx3nr', message => 'Using too many resources'); +``` + +## System connector tables + +### `metadata.catalogs` + +The catalogs table contains the list of available catalogs. + +### `metadata.schema_properties` + +The schema properties table contains the list of available properties +that can be set when creating a new schema. + +### `metadata.table_properties` + +The table properties table contains the list of available properties +that can be set when creating a new table. + +(system-metadata-materialized-views)= + +### `metadata.materialized_views` + +The materialized views table contains the following information about all +{ref}`materialized views `: + +```{eval-rst} +.. list-table:: Metadata for materialized views + :widths: 30, 70 + :header-rows: 1 + + * - Column + - Description + * - ``catalog_name`` + - Name of the catalog containing the materialized view. + * - ``schema_name`` + - Name of the schema in ``catalog_name`` containing the materialized view. + * - ``name`` + - Name of the materialized view. + * - ``storage_catalog`` + - Name of the catalog used for the storage table backing the materialized + view. + * - ``storage_schema`` + - Name of the schema in ``storage_catalog`` used for the storage table + backing the materialized view. + * - ``storage_table`` + - Name of the storage table backing the materialized view. + * - ``freshness`` + - Freshness of data in the storage table. Queries on the + materialized view access the storage table if not ``STALE``, otherwise + the ``definition`` is used to access the underlying data in the source + tables. + * - ``owner`` + - Username of the creator and owner of the materialized view. + * - ``comment`` + - User supplied text about the materialized view. + * - ``definition`` + - SQL query that defines the data provided by the materialized view. +``` + +### `metadata.materialized_view_properties` + +The materialized view properties table contains the list of available properties +that can be set when creating a new materialized view. + +### `metadata.table_comments` + +The table comments table contains the list of table comment. + +### `runtime.nodes` + +The nodes table contains the list of visible nodes in the Trino +cluster along with their status. + +(optimizer-rule-stats)= + +### `runtime.optimizer_rule_stats` + +The `optimizer_rule_stats` table contains the statistics for optimizer +rule invocations during the query planning phase. The statistics are +aggregated over all queries since the server start-up. The table contains +information about invocation frequency, failure rates and performance for +optimizer rules. For example, you can look at the multiplication of columns +`invocations` and `average_time` to get an idea about which rules +generally impact query planning times the most. + +### `runtime.queries` + +The queries table contains information about currently and recently +running queries on the Trino cluster. From this table you can find out +the original query SQL text, the identity of the user who ran the query, +and performance information about the query, including how long the query +was queued and analyzed. + +### `runtime.tasks` + +The tasks table contains information about the tasks involved in a +Trino query, including where they were executed, and how many rows +and bytes each task processed. + +### `runtime.transactions` + +The transactions table contains the list of currently open transactions +and related metadata. This includes information such as the create time, +idle time, initialization parameters, and accessed catalogs. + +## System connector procedures + +```{eval-rst} +.. function:: runtime.kill_query(query_id, message) + + Kill the query identified by ``query_id``. The query failure message + includes the specified ``message``. ``message`` is optional. +``` + +(system-type-mapping)= + +## Type mapping + +Trino supports all data types used within the System schemas so no mapping +is required. + +(system-sql-support)= + +## SQL support + +The connector provides {ref}`globally available ` and +{ref}`read operation ` statements to access Trino system +data and metadata. diff --git a/430/_sources/connector/thrift.md.txt b/430/_sources/connector/thrift.md.txt new file mode 100644 index 000000000..db9a5dbd5 --- /dev/null +++ b/430/_sources/connector/thrift.md.txt @@ -0,0 +1,108 @@ +# Thrift connector + +The Thrift connector makes it possible to integrate with external storage systems +without a custom Trino connector implementation by using +[Apache Thrift](https://thrift.apache.org/) on these servers. It is therefore +generic and can provide access to any backend, as long as it exposes the expected +API by using Thrift. + +In order to use the Thrift connector with an external system, you need to implement +the `TrinoThriftService` interface, found below. Next, you configure the Thrift connector +to point to a set of machines, called Thrift servers, that implement the interface. +As part of the interface implementation, the Thrift servers provide metadata, +splits and data. The connector randomly chooses a server to talk to from the available +instances for metadata calls, or for data calls unless the splits include a list of addresses. +All requests are assumed to be idempotent and can be retried freely among any server. + +## Requirements + +To connect to your custom servers with the Thrift protocol, you need: + +- Network access from the Trino coordinator and workers to the Thrift servers. +- A {ref}`trino-thrift-service` for your system. + +## Configuration + +To configure the Thrift connector, create a catalog properties file +`etc/catalog/example.properties` with the following content, replacing the +properties as appropriate: + +```text +connector.name=trino_thrift +trino.thrift.client.addresses=host:port,host:port +``` + +### Multiple Thrift systems + +You can have as many catalogs as you need, so if you have additional +Thrift systems to connect to, simply add another properties file to `etc/catalog` +with a different name, making sure it ends in `.properties`. + +## Configuration properties + +The following configuration properties are available: + +| Property name | Description | +| ------------------------------------------ | -------------------------------------------------------- | +| `trino.thrift.client.addresses` | Location of Thrift servers | +| `trino-thrift.max-response-size` | Maximum size of data returned from Thrift server | +| `trino-thrift.metadata-refresh-threads` | Number of refresh threads for metadata cache | +| `trino.thrift.client.max-retries` | Maximum number of retries for failed Thrift requests | +| `trino.thrift.client.max-backoff-delay` | Maximum interval between retry attempts | +| `trino.thrift.client.min-backoff-delay` | Minimum interval between retry attempts | +| `trino.thrift.client.max-retry-time` | Maximum duration across all attempts of a Thrift request | +| `trino.thrift.client.backoff-scale-factor` | Scale factor for exponential back off | +| `trino.thrift.client.connect-timeout` | Connect timeout | +| `trino.thrift.client.request-timeout` | Request timeout | +| `trino.thrift.client.socks-proxy` | SOCKS proxy address | +| `trino.thrift.client.max-frame-size` | Maximum size of a raw Thrift response | +| `trino.thrift.client.transport` | Thrift transport type (`UNFRAMED`, `FRAMED`, `HEADER`) | +| `trino.thrift.client.protocol` | Thrift protocol type (`BINARY`, `COMPACT`, `FB_COMPACT`) | + +### `trino.thrift.client.addresses` + +Comma-separated list of thrift servers in the form of `host:port`. For example: + +```text +trino.thrift.client.addresses=192.0.2.3:7777,192.0.2.4:7779 +``` + +This property is required; there is no default. + +### `trino-thrift.max-response-size` + +Maximum size of a data response that the connector accepts. This value is sent +by the connector to the Thrift server when requesting data, allowing it to size +the response appropriately. + +This property is optional; the default is `16MB`. + +### `trino-thrift.metadata-refresh-threads` + +Number of refresh threads for metadata cache. + +This property is optional; the default is `1`. + +(trino-thrift-service)= + +## TrinoThriftService implementation + +The following IDL describes the `TrinoThriftService` that must be implemented: + +```{literalinclude} /include/TrinoThriftService.thrift +:language: thrift +``` + +(thrift-type-mapping)= + +## Type mapping + +The Thrift service defines data type support and mappings to Trino data types. + +(thrift-sql-support)= + +## SQL support + +The connector provides {ref}`globally available ` and +{ref}`read operation ` statements to access data and +metadata in your Thrift service. diff --git a/430/_sources/connector/tpcds.md.txt b/430/_sources/connector/tpcds.md.txt new file mode 100644 index 000000000..ac1db6560 --- /dev/null +++ b/430/_sources/connector/tpcds.md.txt @@ -0,0 +1,72 @@ +# TPCDS connector + +The TPCDS connector provides a set of schemas to support the +[TPC Benchmark™ DS (TPC-DS)](http://www.tpc.org/tpcds/). TPC-DS is a database +benchmark used to measure the performance of complex decision support databases. + +This connector can be used to test the capabilities and query +syntax of Trino without configuring access to an external data +source. When you query a TPCDS schema, the connector generates the +data on the fly using a deterministic algorithm. + +## Configuration + +To configure the TPCDS connector, create a catalog properties file +`etc/catalog/example.properties` with the following contents: + +```text +connector.name=tpcds +``` + +## TPCDS schemas + +The TPCDS connector supplies several schemas: + +``` +SHOW SCHEMAS FROM example; +``` + +```text + Schema +-------------------- + information_schema + sf1 + sf10 + sf100 + sf1000 + sf10000 + sf100000 + sf300 + sf3000 + sf30000 + tiny +(11 rows) +``` + +Ignore the standard schema `information_schema`, which exists in every +catalog, and is not directly provided by the TPCDS connector. + +Every TPCDS schema provides the same set of tables. Some tables are +identical in all schemas. The *scale factor* of the tables in a particular +schema is determined from the schema name. For example, the schema +`sf1` corresponds to scale factor `1` and the schema `sf300` +corresponds to scale factor `300`. Every unit in the scale factor +corresponds to a gigabyte of data. For example, for scale factor `300`, +a total of `300` gigabytes are generated. The `tiny` schema is an +alias for scale factor `0.01`, which is a very small data set useful for +testing. + +(tpcds-type-mapping)= + +## Type mapping + +Trino supports all data types used within the TPCDS schemas so no mapping is +required. + +(tpcds-sql-support)= + +## SQL support + +The connector provides {ref}`globally available ` and +{ref}`read operation ` statements to access data and +metadata in the TPC-DS dataset. diff --git a/430/_sources/connector/tpch.md.txt b/430/_sources/connector/tpch.md.txt new file mode 100644 index 000000000..c55de00b9 --- /dev/null +++ b/430/_sources/connector/tpch.md.txt @@ -0,0 +1,80 @@ +# TPCH connector + +The TPCH connector provides a set of schemas to support the +[TPC Benchmark™ H (TPC-H)](http://www.tpc.org/tpch/). TPC-H is a database +benchmark used to measure the performance of highly-complex decision support databases. + +This connector can be used to test the capabilities and query +syntax of Trino without configuring access to an external data +source. When you query a TPCH schema, the connector generates the +data on the fly using a deterministic algorithm. + +## Configuration + +To configure the TPCH connector, create a catalog properties file +`etc/catalog/example.properties` with the following contents: + +```text +connector.name=tpch +``` + +In the TPC-H specification, each column is assigned a prefix based on its +corresponding table name, such as `l_` for the `lineitem` table. By default, the +TPCH connector simplifies column names by excluding these prefixes with the +default of `tpch.column-naming` to `SIMPLIFIED`. To use the long, standard +column names, use the configuration in the catalog properties file: + +```text +tpch.column-naming=STANDARD +``` + +## TPCH schemas + +The TPCH connector supplies several schemas: + +``` +SHOW SCHEMAS FROM example; +``` + +```text + Schema +-------------------- + information_schema + sf1 + sf100 + sf1000 + sf10000 + sf100000 + sf300 + sf3000 + sf30000 + tiny +(11 rows) +``` + +Ignore the standard schema `information_schema`, which exists in every +catalog, and is not directly provided by the TPCH connector. + +Every TPCH schema provides the same set of tables. Some tables are +identical in all schemas. Other tables vary based on the *scale factor*, +which is determined based on the schema name. For example, the schema +`sf1` corresponds to scale factor `1` and the schema `sf300` +corresponds to scale factor `300`. The TPCH connector provides an +infinite number of schemas for any scale factor, not just the few common +ones listed by `SHOW SCHEMAS`. The `tiny` schema is an alias for scale +factor `0.01`, which is a very small data set useful for testing. + +(tpch-type-mapping)= + +## Type mapping + +Trino supports all data types used within the TPCH schemas so no mapping +is required. + +(tpch-sql-support)= + +## SQL support + +The connector provides {ref}`globally available ` and +{ref}`read operation ` statements to access data and +metadata in the TPC-H dataset. diff --git a/430/_sources/develop.md.txt b/430/_sources/develop.md.txt new file mode 100644 index 000000000..5f3667c50 --- /dev/null +++ b/430/_sources/develop.md.txt @@ -0,0 +1,24 @@ +# Developer guide + +This guide is intended for Trino contributors and plugin developers. + +```{toctree} +:maxdepth: 1 + +develop/spi-overview +develop/connectors +develop/example-http +develop/example-jdbc +develop/insert +develop/supporting-merge +develop/types +develop/functions +develop/table-functions +develop/system-access-control +develop/password-authenticator +develop/certificate-authenticator +develop/header-authenticator +develop/group-provider +develop/event-listener +develop/client-protocol +``` diff --git a/430/_sources/develop/certificate-authenticator.md.txt b/430/_sources/develop/certificate-authenticator.md.txt new file mode 100644 index 000000000..722773e86 --- /dev/null +++ b/430/_sources/develop/certificate-authenticator.md.txt @@ -0,0 +1,41 @@ +# Certificate authenticator + +Trino supports TLS-based authentication with X509 certificates via a custom +certificate authenticator that extracts the principal from a client certificate. + +## Implementation + +`CertificateAuthenticatorFactory` is responsible for creating a +`CertificateAuthenticator` instance. It also defines the name of this +authenticator which is used by the administrator in a Trino configuration. + +`CertificateAuthenticator` contains a single method, `authenticate()`, +which authenticates the client certificate and returns a `Principal`, which is then +authorized by the {doc}`system-access-control`. + +The implementation of `CertificateAuthenticatorFactory` must be wrapped +as a plugin and installed on the Trino cluster. + +## Configuration + +After a plugin that implements `CertificateAuthenticatorFactory` has been +installed on the coordinator, it is configured using an +`etc/certificate-authenticator.properties` file. All of the +properties other than `certificate-authenticator.name` are specific to the +`CertificateAuthenticatorFactory` implementation. + +The `certificate-authenticator.name` property is used by Trino to find a +registered `CertificateAuthenticatorFactory` based on the name returned by +`CertificateAuthenticatorFactory.getName()`. The remaining properties are +passed as a map to `CertificateAuthenticatorFactory.create()`. + +Example configuration file: + +```text +certificate-authenticator.name=custom +custom-property1=custom-value1 +custom-property2=custom-value2 +``` + +Additionally, the coordinator must be configured to use certificate authentication +and have HTTPS enabled (or HTTPS forwarding enabled). diff --git a/430/_sources/develop/client-protocol.md.txt b/430/_sources/develop/client-protocol.md.txt new file mode 100644 index 000000000..93fffc9dd --- /dev/null +++ b/430/_sources/develop/client-protocol.md.txt @@ -0,0 +1,278 @@ +# Trino client REST API + +The REST API allows clients to submit SQL queries to Trino and receive the +results. Clients include the CLI, the JDBC driver, and others provided by +the community. The preferred method to interact with Trino is using these +existing clients. This document provides details about the API for reference. +It can also be used to implement your own client, if necessary. + +## HTTP methods + +- A `POST` to `/v1/statement` runs the query string in the `POST` body, + and returns a JSON document containing the query results. If there are more + results, the JSON document contains a `nextUri` URL attribute. +- A `GET` to the `nextUri` attribute returns the next batch of query results. +- A `DELETE` to `nextUri` terminates a running query. + +## Overview of query processing + +A Trino client request is initiated by an HTTP `POST` to the endpoint +`/v1/statement`, with a `POST` body consisting of the SQL query string. +The caller may set various {ref}`client-request-headers`. The headers are +only required on the initial `POST` request, and not when following the +`nextUri` links. + +If the client request returns an HTTP 502, 503 or 504, that means there was +intermittent problem processing request and the client should try again +in 50-100 milliseconds. Trino does not generate those codes by itself +but those can be generated by gateways/load balancers in front of Trino. +Any HTTP status other than 502, 503, 504 or 200 means that query processing +has failed. + +The `/v1/statement` `POST` request returns a JSON document of type +`QueryResults`, as well as a collection of response headers. The +`QueryResults` document contains an `error` field of type +`QueryError` if the query has failed, and if that object is not present, +the query succeeded. Important members of `QueryResults` are documented +in the following sections. + +If the `data` field of the JSON document is set, it contains a list of the +rows of data. The `columns` field is set to a list of the +names and types of the columns returned by the query. Most of the response +headers are treated like browser cookies by the client, and echoed back +as request headers in subsequent client requests, as documented below. + +If the JSON document returned by the `POST` to `/v1/statement` does not +contain a `nextUri` link, the query has completed, either successfully or +unsuccessfully, and no additional requests need to be made. If the +`nextUri` link is present in the document, there are more query results +to be fetched. The client should loop executing a `GET` request +to the `nextUri` returned in the `QueryResults` response object until +`nextUri` is absent from the response. + +The `status` field of the JSON document is for human consumption only, and +provides a hint about the query state. It can not be used to tell if the +query is finished. + +## Important `QueryResults` attributes + +The most important attributes of the `QueryResults` JSON document returned by +the REST API endpoints are listed in this table. For more details, refer to the +class `io.trino.client.QueryResults` in module `trino-client` in the +`client` directory of the Trino source code. + +```{eval-rst} +.. list-table:: ``QueryResults attributes`` + :widths: 25, 55 + :header-rows: 1 + + * - Attribute + - Description + * - ``id`` + - The ID of the query. + * - ``nextUri`` + - If present, the URL to use for subsequent ``GET`` or + ``DELETE`` requests. If not present, the query is complete or + ended in error. + * - ``columns`` + - A list of the names and types of the columns returned by the query. + * - ``data`` + - The ``data`` attribute contains a list of the rows returned by the + query request. Each row is itself a list that holds values of the + columns in the row, in the order specified by the ``columns`` + attribute. + * - ``updateType`` + - A human-readable string representing the operation. For a + ``CREATE TABLE`` request, the ``updateType`` is + "CREATE TABLE"; for ``SET SESSION`` it is "SET SESSION"; etc. + * - ``error`` + - If query failed, the ``error`` attribute contains a ``QueryError`` object. + That object contains a ``message``, an ``errorCode`` and other information + about the error. See the ``io.trino.client.QueryError`` class in module + ``trino-client`` in the ``client`` directory for more details. + +``` + +## `QueryResults` diagnostic attributes + +These `QueryResults` data members may be useful in tracking down problems: + +```{eval-rst} +.. list-table:: ``QueryResults diagnostic attributes`` + :widths: 20, 20, 40 + :header-rows: 1 + + * - Attribute + - Type + - Description + * - ``queryError`` + - ``QueryError`` + - Non-null only if the query resulted in an error. + * - ``failureInfo`` + - ``FailureInfo`` + - ``failureInfo`` has detail on the reason for the failure, including + a stack trace, and ``FailureInfo.errorLocation``, providing the + query line number and column number where the failure was detected. + * - ``warnings`` + - ``List`` + - A usually-empty list of warnings. + * - ``statementStats`` + - ``StatementStats`` + - A class containing statistics about the query execution. Of + particular interest is ``StatementStats.rootStage``, of type + ``StageStats``, providing statistics on the execution of each of + the stages of query processing. +``` + +(client-request-headers)= + +## Client request headers + +This table lists all supported client request headers. Many of the +headers can be updated in the client as response headers, and supplied +in subsequent requests, just like browser cookies. + +```{eval-rst} +.. list-table:: Client request headers + :widths: 30, 50 + :header-rows: 1 + + * - Header name + - Description + * - ``X-Trino-User`` + - Specifies the session user. If not supplied, the session user is + automatically determined via :doc:`/security/user-mapping`. + * - ``X-Trino-Original-User`` + - Specifies the session's original user. + * - ``X-Trino-Source`` + - For reporting purposes, this supplies the name of the software + that submitted the query. + * - ``X-Trino-Catalog`` + - The catalog context for query processing. Set by response + header ``X-Trino-Set-Catalog``. + * - ``X-Trino-Schema`` + - The schema context for query processing. Set by response + header ``X-Trino-Set-Schema``. + * - ``X-Trino-Time-Zone`` + - The timezone for query processing. Defaults to the timezone + of the Trino cluster, and not the timezone of the client. + * - ``X-Trino-Language`` + - The language to use when processing the query and formatting + results, formatted as a Java ``Locale`` string, e.g., ``en-US`` + for US English. The language of the + session can be set on a per-query basis using the + ``X-Trino-Language`` HTTP header. + * - ``X-Trino-Trace-Token`` + - Supplies a trace token to the Trino engine to help identify + log lines that originate with this query request. + * - ``X-Trino-Session`` + - Supplies a comma-separated list of name=value pairs as session + properties. When the Trino client run a + ``SET SESSION name=value`` query, the name=value pair + is returned in the ``X-Set-Trino-Session`` response header, + and added to the client's list of session properties. + If the response header ``X-Trino-Clear-Session`` is returned, + its value is the name of a session property that is + removed from the client's accumulated list. + * - ``X-Trino-Role`` + - Sets the "role" for query processing. A "role" represents + a collection of permissions. Set by response header + ``X-Trino-Set-Role``. See :doc:`/sql/create-role` to + understand roles. + * - ``X-Trino-Prepared-Statement`` + - A comma-separated list of the name=value pairs, where the + names are names of previously prepared SQL statements, and + the values are keys that identify the executable form of the + named prepared statements. + * - ``X-Trino-Transaction-Id`` + - The transaction ID to use for query processing. Set + by response header ``X-Trino-Started-Transaction-Id`` and + cleared by ``X-Trino-Clear-Transaction-Id``. + * - ``X-Trino-Client-Info`` + - Contains arbitrary information about the client program + submitting the query. + * - ``X-Trino-Client-Tags`` + - A comma-separated list of "tag" strings, used to identify + Trino resource groups. + * - ``X-Trino-Resource-Estimate`` + - A comma-separated list of ``resource=value`` type + assigments. The possible choices of ``resource`` are + ``EXECUTION_TIME``, ``CPU_TIME``, ``PEAK_MEMORY`` and + ``PEAK_TASK_MEMORY``. ``EXECUTION_TIME`` and ``CPU_TIME`` + have values specified as airlift ``Duration`` strings + The format is a double precision number followed by + a ``TimeUnit`` string, e.g., of ``s`` for seconds, + ``m`` for minutes, ``h`` for hours, etc. "PEAK_MEMORY" and + "PEAK_TASK_MEMORY" are specified as as airlift ``DataSize`` strings, + whose format is an integer followed by ``B`` for bytes; ``kB`` for + kilobytes; ``mB`` for megabytes, ``gB`` for gigabytes, etc. + * - ``X-Trino-Extra-Credential`` + - Provides extra credentials to the connector. The header is + a name=value string that is saved in the session ``Identity`` + object. The name and value are only meaningful to the connector. +``` + +## Client response headers + +This table lists the supported client response headers. After receiving a +response, a client must update the request headers used in +subsequent requests to be consistent with the response headers received. + +```{eval-rst} +.. list-table:: Client response headers + :widths: 30, 50 + :header-rows: 1 + + * - Header name + - Description + * - ``X-Trino-Set-Catalog`` + - Instructs the client to set the catalog in the + ``X-Trino-Catalog`` request header in subsequent client requests. + * - ``X-Trino-Set-Schema`` + - Instructs the client to set the schema in the + ``X-Trino-Schema`` request header in subsequent client requests. + * - ``X-Trino-Set-Authorization-User`` + - Instructs the client to set the session authorization user in the + ``X-Trino-Authorization-User`` request header in subsequent client requests. + * - ``X-Trino-Reset-Authorization-User`` + - Instructs the client to remove ``X-Trino-Authorization-User`` request header + in subsequent client requests to reset the authorization user back to the + original user. + * - ``X-Trino-Set-Session`` + - The value of the ``X-Trino-Set-Session`` response header is a + string of the form *property* = *value*. It + instructs the client include session property *property* with value + *value* in the ``X-Trino-Session`` header of subsequent + client requests. + * - ``X-Trino-Clear-Session`` + - Instructs the client to remove the session property with the + whose name is the value of the ``X-Trino-Clear-Session`` header + from the list of session properties + in the ``X-Trino-Session`` header in subsequent client requests. + * - ``X-Trino-Set-Role`` + - Instructs the client to set ``X-Trino-Role`` request header to the + catalog role supplied by the ``X-Trino-Set-Role`` header + in subsequent client requests. + * - ``X-Trino-Added-Prepare`` + - Instructs the client to add the name=value pair to the set of + prepared statements in the ``X-Trino-Prepared-Statement`` + request header in subsequent client requests. + * - ``X-Trino-Deallocated-Prepare`` + - Instructs the client to remove the prepared statement whose name + is the value of the ``X-Trino-Deallocated-Prepare`` header from + the client's list of prepared statements sent in the + ``X-Trino-Prepared-Statement`` request header in subsequent client + requests. + * - ``X-Trino-Started-Transaction-Id`` + - Provides the transaction ID that the client should pass back in the + ``X-Trino-Transaction-Id`` request header in subsequent requests. + * - ``X-Trino-Clear-Transaction-Id`` + - Instructs the client to clear the ``X-Trino-Transaction-Id`` request + header in subsequent requests. +``` + +## `ProtocolHeaders` + +Class `io.trino.client.ProtocolHeaders` in module `trino-client` in the +`client` directory of Trino source enumerates all the HTTP request and +response headers allowed by the Trino client REST API. diff --git a/430/_sources/develop/connectors.md.txt b/430/_sources/develop/connectors.md.txt new file mode 100644 index 000000000..b3612eb35 --- /dev/null +++ b/430/_sources/develop/connectors.md.txt @@ -0,0 +1,847 @@ +# Connectors + +Connectors are the source of all data for queries in Trino. Even if your data +source doesn't have underlying tables backing it, as long as you adapt your data +source to the API expected by Trino, you can write queries against this data. + +## ConnectorFactory + +Instances of your connector are created by a `ConnectorFactory` instance which +is created when Trino calls `getConnectorFactory()` on the plugin. The +connector factory is a simple interface responsible for providing the connector +name and creating an instance of a `Connector` object. A basic connector +implementation that only supports reading, but not writing data, should return +instances of the following services: + +- {ref}`connector-metadata` +- {ref}`connector-split-manager` +- {ref}`connector-record-set-provider` or {ref}`connector-page-source-provider` + +### Configuration + +The `create()` method of the connector factory receives a `config` map, +containing all properties from the catalog properties file. It can be used +to configure the connector, but because all the values are strings, they +might require additional processing if they represent other data types. +It also doesn't validate if all the provided properties are known. This +can lead to the connector behaving differently than expected when a +connector ignores a property due to the user making a mistake in +typing the name of the property. + +To make the configuration more robust, define a Configuration class. This +class describes all the available properties, their types, and additional +validation rules. + +```java +import io.airlift.configuration.Config; +import io.airlift.configuration.ConfigDescription; +import io.airlift.configuration.ConfigSecuritySensitive; +import io.airlift.units.Duration; +import io.airlift.units.MaxDuration; +import io.airlift.units.MinDuration; + +import javax.validation.constraints.NotNull; + +public class ExampleConfig +{ + private String secret; + private Duration timeout = Duration.succinctDuration(10, TimeUnit.SECONDS); + + public String getSecret() + { + return secret; + } + + @Config("secret") + @ConfigDescription("Secret required to access the data source") + @ConfigSecuritySensitive + public ExampleConfig setSecret(String secret) + { + this.secret = secret; + return this; + } + + @NotNull + @MaxDuration("10m") + @MinDuration("1ms") + public Duration getTimeout() + { + return timeout; + } + + @Config("timeout") + public ExampleConfig setTimeout(Duration timeout) + { + this.timeout = timeout; + return this; + } +} +``` + +The preceding example defines two configuration properties and makes +the connector more robust by: + +- defining all supported properties, which allows detecting spelling mistakes + in the configuration on server startup +- defining a default timeout value, to prevent connections getting stuck + indefinitely +- preventing invalid timeout values, like 0 ms, that would make + all requests fail +- parsing timeout values in different units, detecting invalid values +- preventing logging the secret value in plain text + +The configuration class needs to be bound in a Guice module: + +```java +import com.google.inject.Binder; +import com.google.inject.Module; + +import static io.airlift.configuration.ConfigBinder.configBinder; + +public class ExampleModule + implements Module +{ + public ExampleModule() + { + } + + @Override + public void configure(Binder binder) + { + configBinder(binder).bindConfig(ExampleConfig.class); + } +} +``` + +And then the module needs to be initialized in the connector factory, when +creating a new instance of the connector: + +```java +@Override +public Connector create(String connectorName, Map config, ConnectorContext context) +{ + requireNonNull(config, "config is null"); + Bootstrap app = new Bootstrap(new ExampleModule()); + Injector injector = app + .doNotInitializeLogging() + .setRequiredConfigurationProperties(config) + .initialize(); + + return injector.getInstance(ExampleConnector.class); +} +``` + +:::{note} +Environment variables in the catalog properties file +(ex. `secret=${ENV:SECRET}`) are resolved only when using +the `io.airlift.bootstrap.Bootstrap` class to initialize the module. +See {doc}`/security/secrets` for more information. +::: + +If you end up needing to define multiple catalogs using the same connector +just to change one property, consider adding support for schema and/or +table properties. That would allow a more fine-grained configuration. +If a connector doesn't support managing the schema, query predicates for +selected columns could be used as a way of passing the required configuration +at run time. + +For example, when building a connector to read commits from a Git repository, +the repository URL could be a configuration property. But this would result +in a catalog being able to return data only from a single repository. +Alternatively, it can be a column, where every select query would require +a predicate for it: + +```sql +SELECT * +FROM git.default.commits +WHERE url = 'https://github.com/trinodb/trino.git' +``` + +(connector-metadata)= + +## ConnectorMetadata + +The connector metadata interface allows Trino to get a lists of schemas, +tables, columns, and other metadata about a particular data source. + +A basic read-only connector should implement the following methods: + +- `listSchemaNames` +- `listTables` +- `streamTableColumns` +- `getTableHandle` +- `getTableMetadata` +- `getColumnHandles` +- `getColumnMetadata` + +If you are interested in seeing strategies for implementing more methods, +look at the {doc}`example-http` and the Cassandra connector. If your underlying +data source supports schemas, tables, and columns, this interface should be +straightforward to implement. If you are attempting to adapt something that +isn't a relational database, as the Example HTTP connector does, you may +need to get creative about how you map your data source to Trino's schema, +table, and column concepts. + +The connector metadata interface allows to also implement other connector +features, like: + +- Schema management, which is creating, altering and dropping schemas, tables, + table columns, views, and materialized views. + +- Support for table and column comments, and properties. + +- Schema, table and view authorization. + +- Executing {doc}`table-functions`. + +- Providing table statistics used by the Cost Based Optimizer (CBO) + and collecting statistics during writes and when analyzing selected tables. + +- Data modification, which is: + + - inserting, updating, and deleting rows in tables, + - refreshing materialized views, + - truncating whole tables, + - and creating tables from query results. + +- Role and grant management. + +- Pushing down: + + - {ref}`Limit and Top N - limit with sort items ` + - {ref}`Predicates ` + - Projections + - Sampling + - Aggregations + - Joins + - Table function invocation + +Note that data modification also requires implementing +a {ref}`connector-page-sink-provider`. + +When Trino receives a `SELECT` query, it parses it into an Intermediate +Representation (IR). Then, during optimization, it checks if connectors +can handle operations related to SQL clauses by calling one of the following +methods of the `ConnectorMetadata` service: + +- `applyLimit` +- `applyTopN` +- `applyFilter` +- `applyProjection` +- `applySample` +- `applyAggregation` +- `applyJoin` +- `applyTableFunction` +- `applyTableScanRedirect` + +Connectors can indicate that they don't support a particular pushdown or that +the action had no effect by returning `Optional.empty()`. Connectors should +expect these methods to be called multiple times during the optimization of +a given query. + +:::{warning} +It's critical for connectors to return `Optional.empty()` if calling +this method has no effect for that invocation, even if the connector generally +supports a particular pushdown. Doing otherwise can cause the optimizer +to loop indefinitely. +::: + +Otherwise, these methods return a result object containing a new table handle. +The new table handle represents the virtual table derived from applying the +operation (filter, project, limit, etc.) to the table produced by the table +scan node. Once the query actually runs, `ConnectorRecordSetProvider` or +`ConnectorPageSourceProvider` can use whatever optimizations were pushed down to +`ConnectorTableHandle`. + +The returned table handle is later passed to other services that the connector +implements, like the `ConnectorRecordSetProvider` or +`ConnectorPageSourceProvider`. + +(connector-limit-pushdown)= + +### Limit and top-N pushdown + +When executing a `SELECT` query with `LIMIT` or `ORDER BY` clauses, +the query plan may contain a `Sort` or `Limit` operations. + +When the plan contains a `Sort` and `Limit` operations, the engine +tries to push down the limit into the connector by calling the `applyTopN` +method of the connector metadata service. If there's no `Sort` operation, but +only a `Limit`, the `applyLimit` method is called, and the connector can +return results in an arbitrary order. + +If the connector could benefit from the information passed to these methods but +can't guarantee that it's be able to produce fewer rows than the provided +limit, it should return a non-empty result containing a new handle for the +derived table and the `limitGuaranteed` (in `LimitApplicationResult`) or +`topNGuaranteed` (in `TopNApplicationResult`) flag set to false. + +If the connector can guarantee to produce fewer rows than the provided +limit, it should return a non-empty result with the "limit guaranteed" or +"topN guaranteed" flag set to true. + +:::{note} +The `applyTopN` is the only method that receives sort items from the +`Sort` operation. +::: + +In a query, the `ORDER BY` section can include any column with any order. +But the data source for the connector might only support limited combinations. +Plugin authors have to decide if the connector should ignore the pushdown, +return all the data and let the engine sort it, or throw an exception +to inform the user that particular order isn't supported, if fetching all +the data would be too expensive or time consuming. When throwing +an exception, use the `TrinoException` class with the `INVALID_ORDER_BY` +error code and an actionable message, to let users know how to write a valid +query. + +(dev-predicate-pushdown)= + +### Predicate pushdown + +When executing a query with a `WHERE` clause, the query plan can +contain a `ScanFilterProject` plan node/node with a predicate constraint. + +A predicate constraint is a description of the constraint imposed on the +results of the stage/fragment as expressed in the `WHERE` clause. For example, +`WHERE x > 5 AND y = 3` translates into a constraint where the +`summary` field means the `x` column's domain must be greater than +`5` and the `y` column domain equals `3`. + +When the query plan contains a `ScanFilterProject` operation, Trino +tries to optimize the query by pushing down the predicate constraint +into the connector by calling the `applyFilter` method of the +connector metadata service. This method receives a table handle with +all optimizations applied thus far, and returns either +`Optional.empty()` or a response with a new table handle derived from +the old one. + +The query optimizer may call `applyFilter` for a single query multiple times, +as it searches for an optimal query plan. Connectors must +return `Optional.empty()` from `applyFilter` if they cannot apply the +constraint for this invocation, even if they support `ScanFilterProject` +pushdown in general. Connectors must also return `Optional.empty()` if the +constraint has already been applied. + +A constraint contains the following elements: + +- A `TupleDomain` defining the mapping between columns and their domains. + A `Domain` is either a list of possible values, or a list of ranges, and + also contains information about nullability. +- Expression for pushing down function calls. +- Map of assignments from variables in the expression to columns. +- (optional) Predicate which tests a map of columns and their values; + it cannot be held on to after the `applyFilter` call returns. +- (optional) Set of columns the predicate depends on; must be present + if predicate is present. + +If both a predicate and a summary are available, the predicate is guaranteed to +be more strict in filtering of values, and can provide a significant boost to +query performance if used. + +However it is not possible to store a predicate in the table handle and use +it later, as the predicate cannot be held on to after the `applyFilter` +call returns. It is used for filtering of entire partitions, and is not pushed +down. The summary can be pushed down instead by storing it in the table handle. + +This overlap between the predicate and summary is due to historical reasons, +as simple comparison pushdown was implemented first via summary, and more +complex filters such as `LIKE` which required more expressive predicates +were added later. + +If a constraint can only be partially pushed down, for example when a connector +for a database that does not support range matching is used in a query with +`WHERE x = 2 AND y > 5`, the `y` column constraint must be +returned in the `ConstraintApplicationResult` from `applyFilter`. +In this case the `y > 5` condition is applied in Trino, +and not pushed down. + +The following is a simple example which only looks at `TupleDomain`: + +```java +@Override +public Optional> applyFilter( + ConnectorSession session, + ConnectorTableHandle tableHandle, + Constraint constraint) +{ + ExampleTableHandle handle = (ExampleTableHandle) tableHandle; + + TupleDomain oldDomain = handle.getConstraint(); + TupleDomain newDomain = oldDomain.intersect(constraint.getSummary()); + if (oldDomain.equals(newDomain)) { + // Nothing has changed, return empty Option + return Optional.empty(); + } + + handle = new ExampleTableHandle(newDomain); + return Optional.of(new ConstraintApplicationResult<>(handle, TupleDomain.all(), false)); +} +``` + +The `TupleDomain` from the constraint is intersected with the `TupleDomain` +already applied to the `TableHandle` to form `newDomain`. +If filtering has not changed, an `Optional.empty()` result is returned to +notify the planner that this optimization path has reached its end. + +In this example, the connector pushes down the `TupleDomain` +with all Trino data types supported with same semantics in the +data source. As a result, no filters are needed in Trino, +and the `ConstraintApplicationResult` sets `remainingFilter` to +`TupleDomain.all()`. + +This pushdown implementation is quite similar to many Trino connectors, +including `MongoMetadata`, `BigQueryMetadata`, `KafkaMetadata`. + +The following, more complex example shows data types from Trino that are +not available directly in the underlying data source, and must be mapped: + +```java +@Override +public Optional> applyFilter( + ConnectorSession session, + ConnectorTableHandle table, + Constraint constraint) +{ + JdbcTableHandle handle = (JdbcTableHandle) table; + + TupleDomain oldDomain = handle.getConstraint(); + TupleDomain newDomain = oldDomain.intersect(constraint.getSummary()); + TupleDomain remainingFilter; + if (newDomain.isNone()) { + newConstraintExpressions = ImmutableList.of(); + remainingFilter = TupleDomain.all(); + remainingExpression = Optional.of(Constant.TRUE); + } + else { + // We need to decide which columns to push down. + // Since this is a base class for many JDBC-based connectors, each + // having different Trino type mappings and comparison semantics + // it needs to be flexible. + + Map domains = newDomain.getDomains().orElseThrow(); + List columnHandles = domains.keySet().stream() + .map(JdbcColumnHandle.class::cast) + .collect(toImmutableList()); + + // Get information about how to push down every column based on its + // JDBC data type + List columnMappings = jdbcClient.toColumnMappings( + session, + columnHandles.stream() + .map(JdbcColumnHandle::getJdbcTypeHandle) + .collect(toImmutableList())); + + // Calculate the domains which can be safely pushed down (supported) + // and those which need to be filtered in Trino (unsupported) + Map supported = new HashMap<>(); + Map unsupported = new HashMap<>(); + for (int i = 0; i < columnHandles.size(); i++) { + JdbcColumnHandle column = columnHandles.get(i); + DomainPushdownResult pushdownResult = + columnMappings.get(i).getPredicatePushdownController().apply( + session, + domains.get(column)); + supported.put(column, pushdownResult.getPushedDown()); + unsupported.put(column, pushdownResult.getRemainingFilter()); + } + + newDomain = TupleDomain.withColumnDomains(supported); + remainingFilter = TupleDomain.withColumnDomains(unsupported); + } + + // Return empty Optional if nothing changed in filtering + if (oldDomain.equals(newDomain)) { + return Optional.empty(); + } + + handle = new JdbcTableHandle( + handle.getRelationHandle(), + newDomain, + ...); + + return Optional.of( + new ConstraintApplicationResult<>( + handle, + remainingFilter)); +} +``` + +This example illustrates implementing a base class for many JDBC connectors +while handling the specific requirements of multiple JDBC-compliant data sources. +It ensures that if a constraint gets pushed down, it works exactly the same in +the underlying data source, and produces the same results as it would in Trino. +For example, in databases where string comparisons are case-insensitive, +pushdown does not work, as string comparison operations in Trino are +case-sensitive. + +The `PredicatePushdownController` interface determines if a column domain can +be pushed down in JDBC-compliant data sources. In the preceding example, it is +called from a `JdbcClient` implementation specific to that database. +In non-JDBC-compliant data sources, type-based push downs are implemented +directly, without going through the `PredicatePushdownController` interface. + +The following example adds expression pushdown enabled by a session flag: + +```java +@Override +public Optional> applyFilter( + ConnectorSession session, + ConnectorTableHandle table, + Constraint constraint) +{ + JdbcTableHandle handle = (JdbcTableHandle) table; + + TupleDomain oldDomain = handle.getConstraint(); + TupleDomain newDomain = oldDomain.intersect(constraint.getSummary()); + List newConstraintExpressions; + TupleDomain remainingFilter; + Optional remainingExpression; + if (newDomain.isNone()) { + newConstraintExpressions = ImmutableList.of(); + remainingFilter = TupleDomain.all(); + remainingExpression = Optional.of(Constant.TRUE); + } + else { + // We need to decide which columns to push down. + // Since this is a base class for many JDBC-based connectors, each + // having different Trino type mappings and comparison semantics + // it needs to be flexible. + + Map domains = newDomain.getDomains().orElseThrow(); + List columnHandles = domains.keySet().stream() + .map(JdbcColumnHandle.class::cast) + .collect(toImmutableList()); + + // Get information about how to push down every column based on its + // JDBC data type + List columnMappings = jdbcClient.toColumnMappings( + session, + columnHandles.stream() + .map(JdbcColumnHandle::getJdbcTypeHandle) + .collect(toImmutableList())); + + // Calculate the domains which can be safely pushed down (supported) + // and those which need to be filtered in Trino (unsupported) + Map supported = new HashMap<>(); + Map unsupported = new HashMap<>(); + for (int i = 0; i < columnHandles.size(); i++) { + JdbcColumnHandle column = columnHandles.get(i); + DomainPushdownResult pushdownResult = + columnMappings.get(i).getPredicatePushdownController().apply( + session, + domains.get(column)); + supported.put(column, pushdownResult.getPushedDown()); + unsupported.put(column, pushdownResult.getRemainingFilter()); + } + + newDomain = TupleDomain.withColumnDomains(supported); + remainingFilter = TupleDomain.withColumnDomains(unsupported); + + // Do we want to handle expression pushdown? + if (isComplexExpressionPushdown(session)) { + List newExpressions = new ArrayList<>(); + List remainingExpressions = new ArrayList<>(); + // Each expression can be broken down into a list of conjuncts + // joined with AND. We handle each conjunct separately. + for (ConnectorExpression expression : extractConjuncts(constraint.getExpression())) { + // Try to convert the conjunct into something which is + // understood by the underlying JDBC data source + Optional converted = jdbcClient.convertPredicate( + session, + expression, + constraint.getAssignments()); + if (converted.isPresent()) { + newExpressions.add(converted.get()); + } + else { + remainingExpressions.add(expression); + } + } + // Calculate which parts of the expression can be pushed down + // and which need to be calculated in Trino engine + newConstraintExpressions = ImmutableSet.builder() + .addAll(handle.getConstraintExpressions()) + .addAll(newExpressions) + .build().asList(); + remainingExpression = Optional.of(and(remainingExpressions)); + } + else { + newConstraintExpressions = ImmutableList.of(); + remainingExpression = Optional.empty(); + } + } + + // Return empty Optional if nothing changed in filtering + if (oldDomain.equals(newDomain) && + handle.getConstraintExpressions().equals(newConstraintExpressions)) { + return Optional.empty(); + } + + handle = new JdbcTableHandle( + handle.getRelationHandle(), + newDomain, + newConstraintExpressions, + ...); + + return Optional.of( + remainingExpression.isPresent() + ? new ConstraintApplicationResult<>( + handle, + remainingFilter, + remainingExpression.get()) + : new ConstraintApplicationResult<>( + handle, + remainingFilter)); +} +``` + +`ConnectorExpression` is split similarly to `TupleDomain`. +Each expression can be broken down into independent *conjuncts*. Conjuncts are +smaller expressions which, if joined together using an `AND` operator, are +equivalent to the original expression. Every conjunct can be handled +individually. Each one is converted using connector-specific rules, as defined +by the `JdbcClient` implementation, to be more flexible. Unconverted +conjuncts are returned as `remainingExpression` and are evaluated by +the Trino engine. + +(connector-split-manager)= + +## ConnectorSplitManager + +The split manager partitions the data for a table into the individual chunks +that Trino distributes to workers for processing. For example, the Hive +connector lists the files for each Hive partition and creates one or more +splits per file. For data sources that don't have partitioned data, a good +strategy here is to simply return a single split for the entire table. This is +the strategy employed by the Example HTTP connector. + +(connector-record-set-provider)= + +## ConnectorRecordSetProvider + +Given a split, a table handle, and a list of columns, the record set provider +is responsible for delivering data to the Trino execution engine. + +The table and column handles represent a virtual table. They're created by the +connector's metadata service, called by Trino during query planning and +optimization. Such a virtual table doesn't have to map directly to a single +collection in the connector's data source. If the connector supports pushdowns, +there can be multiple virtual tables derived from others, presenting a different +view of the underlying data. + +The provider creates a `RecordSet`, which in turn creates a `RecordCursor` +that's used by Trino to read the column values for each row. + +The provided record set must only include requested columns in the order +matching the list of column handles passed to the +`ConnectorRecordSetProvider.getRecordSet()` method. The record set must return +all the rows contained in the "virtual table" represented by the TableHandle +associated with the TableScan operation. + +For simple connectors, where performance isn't critical, the record set +provider can return an instance of `InMemoryRecordSet`. The in-memory record +set can be built using lists of values for every row, which can be simpler than +implementing a `RecordCursor`. + +A `RecordCursor` implementation needs to keep track of the current record. +It return values for columns by a numerical position, in the data type matching +the column definition in the table. When the engine is done reading the current +record it calls `advanceNextPosition` on the cursor. + +### Type mapping + +The built-in SQL data types use different Java types as carrier types. + +```{eval-rst} +.. list-table:: SQL type to carrier type mapping + :widths: 45, 55 + :header-rows: 1 + + * - SQL type + - Java type + * - ``BOOLEAN`` + - ``boolean`` + * - ``TINYINT`` + - ``long`` + * - ``SMALLINT`` + - ``long`` + * - ``INTEGER`` + - ``long`` + * - ``BIGINT`` + - ``long`` + * - ``REAL`` + - ``double`` + * - ``DOUBLE`` + - ``double`` + * - ``DECIMAL`` + - ``long`` for precision up to 19, inclusive; + ``Int128`` for precision greater than 19 + * - ``VARCHAR`` + - ``Slice`` + * - ``CHAR`` + - ``Slice`` + * - ``VARBINARY`` + - ``Slice`` + * - ``JSON`` + - ``Slice`` + * - ``DATE`` + - ``long`` + * - ``TIME(P)`` + - ``long`` + * - ``TIME WITH TIME ZONE`` + - ``long`` for precision up to 9; + ``LongTimeWithTimeZone`` for precision greater than 9 + * - ``TIMESTAMP(P)`` + - ``long`` for precision up to 6; + ``LongTimestamp`` for precision greater than 6 + * - ``TIMESTAMP(P) WITH TIME ZONE`` + - ``long`` for precision up to 3; + ``LongTimestampWithTimeZone`` for precision greater than 3 + * - ``INTERVAL YEAR TO MONTH`` + - ``long`` + * - ``INTERVAL DAY TO SECOND`` + - ``long`` + * - ``ARRAY`` + - ``Block`` + * - ``MAP`` + - ``Block`` + * - ``ROW`` + - ``Block`` + * - ``IPADDRESS`` + - ``Slice`` + * - ``UUID`` + - ``Slice`` + * - ``HyperLogLog`` + - ``Slice`` + * - ``P4HyperLogLog`` + - ``Slice`` + * - ``SetDigest`` + - ``Slice`` + * - ``QDigest`` + - ``Slice`` + * - ``TDigest`` + - ``TDigest`` +``` + +The `RecordCursor.getType(int field)` method returns the SQL type for a field +and the field value is returned by one of the following methods, matching +the carrier type: + +- `getBoolean(int field)` +- `getLong(int field)` +- `getDouble(int field)` +- `getSlice(int field)` +- `getObject(int field)` + +Values for the `timestamp(p) with time zone` and `time(p) with time zone` +types of regular precision can be converted into `long` using static methods +from the `io.trino.spi.type.DateTimeEncoding` class, like `pack()` or +`packDateTimeWithZone()`. + +UTF-8 encoded strings can be converted to Slices using +the `Slices.utf8Slice()` static method. + +:::{note} +The `Slice` class is provided by the `io.airlift:slice` package. +::: + +`Int128` objects can be created using the `Int128.valueOf()` method. + +The following example creates a block for an `array(varchar)` column: + +```java +private Block encodeArray(List names) +{ + BlockBuilder builder = VARCHAR.createBlockBuilder(null, names.size()); + for (String name : names) { + if (name == null) { + builder.appendNull(); + } + else { + VARCHAR.writeString(builder, name); + } + } + return builder.build(); +} +``` + +The following example creates a block for a `map(varchar, varchar)` column: + +```java +private Block encodeMap(Map map) +{ + MapType mapType = typeManager.getType(TypeSignature.mapType( + VARCHAR.getTypeSignature(), + VARCHAR.getTypeSignature())); + BlockBuilder values = mapType.createBlockBuilder(null, map != null ? map.size() : 0); + if (map == null) { + values.appendNull(); + return values.build().getObject(0, Block.class); + } + BlockBuilder builder = values.beginBlockEntry(); + for (Map.Entry entry : map.entrySet()) { + VARCHAR.writeString(builder, entry.getKey()); + Object value = entry.getValue(); + if (value == null) { + builder.appendNull(); + } + else { + VARCHAR.writeString(builder, value.toString()); + } + } + values.closeEntry(); + return values.build().getObject(0, Block.class); +} +``` + +(connector-page-source-provider)= + +## ConnectorPageSourceProvider + +Given a split, a table handle, and a list of columns, the page source provider +is responsible for delivering data to the Trino execution engine. It creates +a `ConnectorPageSource`, which in turn creates `Page` objects that are used +by Trino to read the column values. + +If not implemented, a default `RecordPageSourceProvider` is used. +Given a record set provider, it returns an instance of `RecordPageSource` +that builds `Page` objects from records in a record set. + +A connector should implement a page source provider instead of a record set +provider when it's possible to create pages directly. The conversion of +individual records from a record set provider into pages adds overheads during +query execution. + +(connector-page-sink-provider)= + +## ConnectorPageSinkProvider + +Given an insert table handle, the page sink provider is responsible for +consuming data from the Trino execution engine. +It creates a `ConnectorPageSink`, which in turn accepts `Page` objects +that contains the column values. + +Example that shows how to iterate over the page to access single values: + +```java +@Override +public CompletableFuture appendPage(Page page) +{ + for (int channel = 0; channel < page.getChannelCount(); channel++) { + Block block = page.getBlock(channel); + for (int position = 0; position < page.getPositionCount(); position++) { + if (block.isNull(position)) { + // or handle this differently + continue; + } + + // channel should match the column number in the table + // use it to determine the expected column type + String value = VARCHAR.getSlice(block, position).toStringUtf8(); + // TODO do something with the value + } + } + return NOT_BLOCKED; +} +``` diff --git a/430/_sources/develop/event-listener.md.txt b/430/_sources/develop/event-listener.md.txt new file mode 100644 index 000000000..72f22d644 --- /dev/null +++ b/430/_sources/develop/event-listener.md.txt @@ -0,0 +1,59 @@ +# Event listener + +Trino supports custom event listeners that are invoked for the following +events: + +- Query creation +- Query completion (success or failure) +- Split completion (success or failure) + +Event details include session, query execution, resource utilization, timeline, +and more. + +This functionality enables development of custom logging, debugging and +performance analysis plugins. + +## Implementation + +`EventListenerFactory` is responsible for creating an +`EventListener` instance. It also defines an `EventListener` +name which is used by the administrator in a Trino configuration. +Implementations of `EventListener` implement methods for the event types +they are interested in handling. + +The implementation of `EventListener` and `EventListenerFactory` +must be wrapped as a plugin and installed on the Trino cluster. + +## Configuration + +After a plugin that implements `EventListener` and +`EventListenerFactory` has been installed on the coordinator, it is +configured using an `etc/event-listener.properties` file. All of the +properties other than `event-listener.name` are specific to the +`EventListener` implementation. + +The `event-listener.name` property is used by Trino to find a registered +`EventListenerFactory` based on the name returned by +`EventListenerFactory.getName()`. The remaining properties are passed +as a map to `EventListenerFactory.create()`. + +Example configuration file: + +```text +event-listener.name=custom-event-listener +custom-property1=custom-value1 +custom-property2=custom-value2 +``` + +(multiple-listeners)= + +## Multiple event listeners + +Trino supports multiple instances of the same or different event listeners. +Install and configure multiple instances by setting +`event-listener.config-files` in {ref}`config-properties` to a comma-separated +list of the event listener configuration files: + +```text +event-listener.config-files=etc/event-listener.properties,etc/event-listener-second.properties +``` diff --git a/430/_sources/develop/example-http.md.txt b/430/_sources/develop/example-http.md.txt new file mode 100644 index 000000000..d68d5bb00 --- /dev/null +++ b/430/_sources/develop/example-http.md.txt @@ -0,0 +1,102 @@ +# Example HTTP connector + +The Example HTTP connector has a simple goal: it reads comma-separated +data over HTTP. For example, if you have a large amount of data in a +CSV format, you can point the example HTTP connector at this data and +write a query to process it. + +## Code + +The Example HTTP connector can be found in the [trino-example-http](https://github.com/trinodb/trino/tree/master/plugin/trino-example-http) +directory within the Trino source tree. + +## Plugin implementation + +The plugin implementation in the Example HTTP connector looks very +similar to other plugin implementations. Most of the implementation is +devoted to handling optional configuration and the only function of +interest is the following: + +```java +@Override +public Iterable getConnectorFactories() +{ + return ImmutableList.of(new ExampleConnectorFactory()); +} +``` + +Note that the `ImmutableList` class is a utility class from Guava. + +As with all connectors, this plugin overrides the `getConnectorFactories()` method +and returns an `ExampleConnectorFactory`. + +## ConnectorFactory implementation + +In Trino, the primary object that handles the connection between +Trino and a particular type of data source is the `Connector` object, +which are created using `ConnectorFactory`. + +This implementation is available in the class `ExampleConnectorFactory`. +The first thing the connector factory implementation does is specify the +name of this connector. This is the same string used to reference this +connector in Trino configuration. + +```java +@Override +public String getName() +{ + return "example_http"; +} +``` + +The real work in a connector factory happens in the `create()` +method. In the `ExampleConnectorFactory` class, the `create()` method +configures the connector and then asks Guice to create the object. +This is the meat of the `create()` method without parameter validation +and exception handling: + +```java +// A plugin is not required to use Guice; it is just very convenient +Bootstrap app = new Bootstrap( + new JsonModule(), + new ExampleModule(catalogName)); + +Injector injector = app + .doNotInitializeLogging() + .setRequiredConfigurationProperties(requiredConfig) + .initialize(); + +return injector.getInstance(ExampleConnector.class); +``` + +### Connector: ExampleConnector + +This class allows Trino to obtain references to the various services +provided by the connector. + +### Metadata: ExampleMetadata + +This class is responsible for reporting table names, table metadata, +column names, column metadata and other information about the schemas +that are provided by this connector. `ConnectorMetadata` is also called +by Trino to ensure that a particular connector can understand and +handle a given table name. + +The `ExampleMetadata` implementation delegates many of these calls to +`ExampleClient`, a class that implements much of the core functionality +of the connector. + +### Split manager: ExampleSplitManager + +The split manager partitions the data for a table into the individual +chunks that Trino will distribute to workers for processing. +In the case of the Example HTTP connector, each table contains one or +more URIs pointing at the actual data. One split is created per URI. + +### Record set provider: ExampleRecordSetProvider + +The record set provider creates a record set which in turn creates a +record cursor that returns the actual data to Trino. +`ExampleRecordCursor` reads data from a URI via HTTP. Each line +corresponds to a single row. Lines are split on comma into individual +field values which are then returned to Trino. diff --git a/430/_sources/develop/example-jdbc.md.txt b/430/_sources/develop/example-jdbc.md.txt new file mode 100644 index 000000000..a3b231458 --- /dev/null +++ b/430/_sources/develop/example-jdbc.md.txt @@ -0,0 +1,64 @@ +# Example JDBC connector + +The Example JDBC connector shows how to extend the base `JdbcPlugin` +to read data from a source using a JDBC driver, without having +to implement different Trino SPI services, like `ConnectorMetadata` +or `ConnectorRecordSetProvider`. + +:::{note} +This connector is just an example. It supports a very limited set of data +types and does not support any advanced functions, like predicacte or other +kind of pushdowns. +::: + +## Code + +The Example JDBC connector can be found in the [trino-example-jdbc](https://github.com/trinodb/trino/tree/master/plugin/trino-example-jdbc) +directory within the Trino source tree. + +## Plugin implementation + +The plugin implementation in the Example JDBC connector extends +the `JdbcPlugin` class and uses the `ExampleClientModule`. + +The module: + +- binds the `ExampleClient` class so it can be used by the base JDBC + connector; +- provides a connection factory that will create new connections using a JDBC + driver based on the JDBC URL specified in configuration properties. + +## JdbcClient implementation + +The base JDBC plugin maps the Trino SPI calls to the JDBC API. Operations like +reading table and columns names are well defined in JDBC so the base JDBC plugin +can implement it in a way that works for most JDBC drivers. + +One behavior that is not implemented by default is mapping of the data types +when reading and writing data. The Example JDBC connector implements +the `JdbcClient` interface in the `ExampleClient` class that extends +the `BaseJdbcClient` and implements two methods. + +### toColumnMapping + +`toColumnMapping` is used when reading data from the connector. +Given a `ConnectorSession`, `Connection` and a `JdbcTypeHandle`, +it returns a `ColumnMapping`, if there is a matching data type. + +The column mapping includes: + +- a Trino type, +- a write function, used to set query parameter values when preparing a + JDBC statement to execute in the data source, +- and a read function, used to read a value from the JDBC statement result set, + and return it using an internal Trino representation (for example, a Slice). + +### toWriteMapping + +`toWriteMapping` is used when writing data to the connector. Given a +`ConnectorSession` and a Trino type, it returns a `WriteMapping`. + +The mapping includes: + +- a data type name +- a write function diff --git a/430/_sources/develop/functions.md.txt b/430/_sources/develop/functions.md.txt new file mode 100644 index 000000000..26555000b --- /dev/null +++ b/430/_sources/develop/functions.md.txt @@ -0,0 +1,320 @@ +# Functions + +## Plugin implementation + +The function framework is used to implement SQL functions. Trino includes a +number of built-in functions. In order to implement new functions, you can +write a plugin that returns one or more functions from `getFunctions()`: + +```java +public class ExampleFunctionsPlugin + implements Plugin +{ + @Override + public Set> getFunctions() + { + return ImmutableSet.>builder() + .add(ExampleNullFunction.class) + .add(IsNullFunction.class) + .add(IsEqualOrNullFunction.class) + .add(ExampleStringFunction.class) + .add(ExampleAverageFunction.class) + .build(); + } +} +``` + +Note that the `ImmutableSet` class is a utility class from Guava. +The `getFunctions()` method contains all of the classes for the functions +that we will implement below in this tutorial. + +For a full example in the codebase, see either the `trino-ml` module for +machine learning functions or the `trino-teradata-functions` module for +Teradata-compatible functions, both in the `plugin` directory of the Trino +source. + +## Scalar function implementation + +The function framework uses annotations to indicate relevant information +about functions, including name, description, return type and parameter +types. Below is a sample function which implements `is_null`: + +```java +public class ExampleNullFunction +{ + @ScalarFunction("is_null", deterministic = true) + @Description("Returns TRUE if the argument is NULL") + @SqlType(StandardTypes.BOOLEAN) + public static boolean isNull( + @SqlNullable @SqlType(StandardTypes.VARCHAR) Slice string) + { + return (string == null); + } +} +``` + +The function `is_null` takes a single `VARCHAR` argument and returns a +`BOOLEAN` indicating if the argument was `NULL`. Note that the argument to +the function is of type `Slice`. `VARCHAR` uses `Slice`, which is essentially +a wrapper around `byte[]`, rather than `String` for its native container type. + +The `deterministic` argument indicates that a function has no side effects and, +for subsequent calls with the same argument(s), the function returns the exact +same value(s). + +In Trino, deterministic functions don't rely on any changing state +and don't modify any state. The `deterministic` flag is optional and defaults +to `true`. + +For example, the function {func}`shuffle` is non-deterministic, since it uses random +values. On the other hand, {func}`now` is deterministic, because subsequent calls in a +single query return the same timestamp. + +Any function with non-deterministic behavior is required to set `deterministic = false` +to avoid unexpected results. + +- `@SqlType`: + + The `@SqlType` annotation is used to declare the return type and the argument + types. Note that the return type and arguments of the Java code must match + the native container types of the corresponding annotations. + +- `@SqlNullable`: + + The `@SqlNullable` annotation indicates that the argument may be `NULL`. Without + this annotation the framework assumes that all functions return `NULL` if + any of their arguments are `NULL`. When working with a `Type` that has a + primitive native container type, such as `BigintType`, use the object wrapper for the + native container type when using `@SqlNullable`. The method must be annotated with + `@SqlNullable` if it can return `NULL` when the arguments are non-null. + +## Parametric scalar functions + +Scalar functions that have type parameters have some additional complexity. +To make our previous example work with any type we need the following: + +```java +@ScalarFunction(name = "is_null") +@Description("Returns TRUE if the argument is NULL") +public final class IsNullFunction +{ + @TypeParameter("T") + @SqlType(StandardTypes.BOOLEAN) + public static boolean isNullSlice(@SqlNullable @SqlType("T") Slice value) + { + return (value == null); + } + + @TypeParameter("T") + @SqlType(StandardTypes.BOOLEAN) + public static boolean isNullLong(@SqlNullable @SqlType("T") Long value) + { + return (value == null); + } + + @TypeParameter("T") + @SqlType(StandardTypes.BOOLEAN) + public static boolean isNullDouble(@SqlNullable @SqlType("T") Double value) + { + return (value == null); + } + + // ...and so on for each native container type +} +``` + +- `@TypeParameter`: + + The `@TypeParameter` annotation is used to declare a type parameter which can + be used in the argument types `@SqlType` annotation, or return type of the function. + It can also be used to annotate a parameter of type `Type`. At runtime, the engine + will bind the concrete type to this parameter. `@OperatorDependency` may be used + to declare that an additional function for operating on the given type parameter is needed. + For example, the following function will only bind to types which have an equals function + defined: + +```java +@ScalarFunction(name = "is_equal_or_null") +@Description("Returns TRUE if arguments are equal or both NULL") +public final class IsEqualOrNullFunction +{ + @TypeParameter("T") + @SqlType(StandardTypes.BOOLEAN) + public static boolean isEqualOrNullSlice( + @OperatorDependency( + operator = OperatorType.EQUAL, + returnType = StandardTypes.BOOLEAN, + argumentTypes = {"T", "T"}) MethodHandle equals, + @SqlNullable @SqlType("T") Slice value1, + @SqlNullable @SqlType("T") Slice value2) + { + if (value1 == null && value2 == null) { + return true; + } + if (value1 == null || value2 == null) { + return false; + } + return (boolean) equals.invokeExact(value1, value2); + } + + // ...and so on for each native container type +} +``` + +## Another scalar function example + +The `lowercaser` function takes a single `VARCHAR` argument and returns a +`VARCHAR`, which is the argument converted to lower case: + +```java +public class ExampleStringFunction +{ + @ScalarFunction("lowercaser") + @Description("Converts the string to alternating case") + @SqlType(StandardTypes.VARCHAR) + public static Slice lowercaser(@SqlType(StandardTypes.VARCHAR) Slice slice) + { + String argument = slice.toStringUtf8(); + return Slices.utf8Slice(argument.toLowerCase()); + } +} +``` + +Note that for most common string functions, including converting a string to +lower case, the Slice library also provides implementations that work directly +on the underlying `byte[]`, which have much better performance. This function +has no `@SqlNullable` annotations, meaning that if the argument is `NULL`, +the result will automatically be `NULL` (the function will not be called). + +## Aggregation function implementation + +Aggregation functions use a similar framework to scalar functions, but are +a bit more complex. + +- `AccumulatorState`: + + All aggregation functions accumulate input rows into a state object; this + object must implement `AccumulatorState`. For simple aggregations, just + extend `AccumulatorState` into a new interface with the getters and setters + you want, and the framework will generate all the implementations and + serializers for you. If you need a more complex state object, you will need + to implement `AccumulatorStateFactory` and `AccumulatorStateSerializer` + and provide these via the `AccumulatorStateMetadata` annotation. + +The following code implements the aggregation function `avg_double` which computes the +average of a `DOUBLE` column: + +```java +@AggregationFunction("avg_double") +public class AverageAggregation +{ + @InputFunction + public static void input( + LongAndDoubleState state, + @SqlType(StandardTypes.DOUBLE) double value) + { + state.setLong(state.getLong() + 1); + state.setDouble(state.getDouble() + value); + } + + @CombineFunction + public static void combine( + LongAndDoubleState state, + LongAndDoubleState otherState) + { + state.setLong(state.getLong() + otherState.getLong()); + state.setDouble(state.getDouble() + otherState.getDouble()); + } + + @OutputFunction(StandardTypes.DOUBLE) + public static void output(LongAndDoubleState state, BlockBuilder out) + { + long count = state.getLong(); + if (count == 0) { + out.appendNull(); + } + else { + double value = state.getDouble(); + DOUBLE.writeDouble(out, value / count); + } + } +} +``` + +The average has two parts: the sum of the `DOUBLE` in each row of the column +and the `LONG` count of the number of rows seen. `LongAndDoubleState` is an interface +which extends `AccumulatorState`: + +```java +public interface LongAndDoubleState + extends AccumulatorState +{ + long getLong(); + + void setLong(long value); + + double getDouble(); + + void setDouble(double value); +} +``` + +As stated above, for simple `AccumulatorState` objects, it is sufficient to +just to define the interface with the getters and setters, and the framework +will generate the implementation for you. + +An in-depth look at the various annotations relevant to writing an aggregation +function follows: + +- `@InputFunction`: + + The `@InputFunction` annotation declares the function which accepts input + rows and stores them in the `AccumulatorState`. Similar to scalar functions + you must annotate the arguments with `@SqlType`. Note that, unlike in the above + scalar example where `Slice` is used to hold `VARCHAR`, the primitive + `double` type is used for the argument to input. In this example, the input + function simply keeps track of the running count of rows (via `setLong()`) + and the running sum (via `setDouble()`). + +- `@CombineFunction`: + + The `@CombineFunction` annotation declares the function used to combine two + state objects. This function is used to merge all the partial aggregation states. + It takes two state objects, and merges the results into the first one (in the + above example, just by adding them together). + +- `@OutputFunction`: + + The `@OutputFunction` is the last function called when computing an + aggregation. It takes the final state object (the result of merging all + partial states) and writes the result to a `BlockBuilder`. + +- Where does serialization happen, and what is `GroupedAccumulatorState`? + + The `@InputFunction` is usually run on a different worker from the + `@CombineFunction`, so the state objects are serialized and transported + between these workers by the aggregation framework. `GroupedAccumulatorState` + is used when performing a `GROUP BY` aggregation, and an implementation + will be automatically generated for you, if you don't specify a + `AccumulatorStateFactory` + +## Deprecated function + +The `@Deprecated` annotation has to be used on any function that should no longer be +used. The annotation causes Trino to generate a warning whenever SQL statements +use a deprecated function. When a function is deprecated, the `@Description` +needs to be replaced with a note about the deprecation and the replacement function: + +```java +public class ExampleDeprecatedFunction +{ + @Deprecated + @ScalarFunction("bad_function") + @Description("(DEPRECATED) Use good_function() instead") + @SqlType(StandardTypes.BOOLEAN) + public static boolean bad_function() + { + return false; + } +} +``` diff --git a/430/_sources/develop/group-provider.md.txt b/430/_sources/develop/group-provider.md.txt new file mode 100644 index 000000000..63915ce80 --- /dev/null +++ b/430/_sources/develop/group-provider.md.txt @@ -0,0 +1,40 @@ +# Group provider + +Trino can map user names onto groups for easier access control management. +This mapping is performed by a `GroupProvider` implementation. + +## Implementation + +`GroupProviderFactory` is responsible for creating a `GroupProvider` instance. +It also defines the name of the group provider as used in the configuration file. + +`GroupProvider` contains a one method, `getGroups(String user)` +which returns a `Set` of group names. +This set of group names becomes part of the `Identity` and `ConnectorIdentity` +objects representing the user, and can then be used by {doc}`system-access-control`. + +The implementation of `GroupProvider` and its corresponding `GroupProviderFactory` +must be wrapped as a Trino plugin and installed on the cluster. + +## Configuration + +After a plugin that implements `GroupProviderFactory` has been installed on the coordinator, +it is configured using an `etc/group-provider.properties` file. +All of the properties other than `group-provider.name` are specific to +the `GroupProviderFactory` implementation. + +The `group-provider.name` property is used by Trino to find a registered +`GroupProviderFactory` based on the name returned by `GroupProviderFactory.getName()`. +The remaining properties are passed as a map to +`GroupProviderFactory.create(Map)`. + +Example configuration file: + +```text +group-provider.name=custom-group-provider +custom-property1=custom-value1 +custom-property2=custom-value2 +``` + +With that file in place, Trino will attempt user group name resolution, +and will be able to use the group names while evaluating access control rules. diff --git a/430/_sources/develop/header-authenticator.md.txt b/430/_sources/develop/header-authenticator.md.txt new file mode 100644 index 000000000..ec8777222 --- /dev/null +++ b/430/_sources/develop/header-authenticator.md.txt @@ -0,0 +1,42 @@ +# Header authenticator + +Trino supports header authentication over TLS via a custom header authenticator +that extracts the principal from a predefined header(s), performs any validation it needs and creates +an authenticated principal. + +## Implementation + +`HeaderAuthenticatorFactory` is responsible for creating a +`HeaderAuthenticator` instance. It also defines the name of this +authenticator which is used by the administrator in a Trino configuration. + +`HeaderAuthenticator` contains a single method, `createAuthenticatedPrincipal()`, +which validates the request headers wrapped by the Headers interface; has the method getHeader(String name) +and returns a `Principal`, which is then authorized by the {doc}`system-access-control`. + +The implementation of `HeaderAuthenticatorFactory` must be wrapped +as a plugin and installed on the Trino cluster. + +## Configuration + +After a plugin that implements `HeaderAuthenticatorFactory` has been +installed on the coordinator, it is configured using an +`etc/header-authenticator.properties` file. All of the +properties other than `header-authenticator.name` are specific to the +`HeaderAuthenticatorFactory` implementation. + +The `header-authenticator.name` property is used by Trino to find a +registered `HeaderAuthenticatorFactory` based on the name returned by +`HeaderAuthenticatorFactory.getName()`. The remaining properties are +passed as a map to `HeaderAuthenticatorFactory.create()`. + +Example configuration file: + +```none +header-authenticator.name=custom +custom-property1=custom-value1 +custom-property2=custom-value2 +``` + +Additionally, the coordinator must be configured to use header authentication +and have HTTPS enabled (or HTTPS forwarding enabled). diff --git a/430/_sources/develop/insert.md.txt b/430/_sources/develop/insert.md.txt new file mode 100644 index 000000000..cab389ee4 --- /dev/null +++ b/430/_sources/develop/insert.md.txt @@ -0,0 +1,28 @@ +# Supporting `INSERT` and `CREATE TABLE AS` + +To support `INSERT`, a connector must implement: + +- `beginInsert()` and `finishInsert()` from the `ConnectorMetadata` + interface; +- a `ConnectorPageSinkProvider` that receives a table handle and returns + a `ConnectorPageSink`. + +When executing an `INSERT` statement, the engine calls the `beginInsert()` +method in the connector, which receives a table handle and a list of columns. +It should return a `ConnectorInsertTableHandle`, that can carry any +connector specific information, and it's passed to the page sink provider. +The `PageSinkProvider` creates a page sink, that accepts `Page` objects. + +When all the pages for a specific split have been processed, Trino calls +`ConnectorPageSink.finish()`, which returns a `Collection` +of fragments representing connector-specific information about the processed +rows. + +When all pages for all splits have been processed, Trino calls +`ConnectorMetadata.finishInsert()`, passing a collection containing all +the fragments from all the splits. The connector does what is required +to finalize the operation, for example, committing the transaction. + +To support `CREATE TABLE AS`, the `ConnectorPageSinkProvider` must also +return a page sink when receiving a `ConnectorOutputTableHandle`. This handle +is returned from `ConnectorMetadata.beginCreateTable()`. diff --git a/430/_sources/develop/password-authenticator.md.txt b/430/_sources/develop/password-authenticator.md.txt new file mode 100644 index 000000000..8095e39ab --- /dev/null +++ b/430/_sources/develop/password-authenticator.md.txt @@ -0,0 +1,41 @@ +# Password authenticator + +Trino supports authentication with a username and password via a custom +password authenticator that validates the credentials and creates a principal. + +## Implementation + +`PasswordAuthenticatorFactory` is responsible for creating a +`PasswordAuthenticator` instance. It also defines the name of this +authenticator which is used by the administrator in a Trino configuration. + +`PasswordAuthenticator` contains a single method, `createAuthenticatedPrincipal()`, +that validates the credential and returns a `Principal`, which is then +authorized by the {doc}`system-access-control`. + +The implementation of `PasswordAuthenticatorFactory` must be wrapped +as a plugin and installed on the Trino cluster. + +## Configuration + +After a plugin that implements `PasswordAuthenticatorFactory` has been +installed on the coordinator, it is configured using an +`etc/password-authenticator.properties` file. All of the +properties other than `password-authenticator.name` are specific to the +`PasswordAuthenticatorFactory` implementation. + +The `password-authenticator.name` property is used by Trino to find a +registered `PasswordAuthenticatorFactory` based on the name returned by +`PasswordAuthenticatorFactory.getName()`. The remaining properties are +passed as a map to `PasswordAuthenticatorFactory.create()`. + +Example configuration file: + +```text +password-authenticator.name=custom-access-control +custom-property1=custom-value1 +custom-property2=custom-value2 +``` + +Additionally, the coordinator must be configured to use password authentication +and have HTTPS enabled (or HTTPS forwarding enabled). diff --git a/430/_sources/develop/spi-overview.md.txt b/430/_sources/develop/spi-overview.md.txt new file mode 100644 index 000000000..872619f2d --- /dev/null +++ b/430/_sources/develop/spi-overview.md.txt @@ -0,0 +1,113 @@ +# SPI overview + +When you implement a new Trino plugin, you implement interfaces and +override methods defined by the Service Provider Interface (SPI). + +Plugins can provide additional: + +- {doc}`connectors`, +- block encodings, +- {doc}`types`, +- {doc}`functions`, +- {doc}`system-access-control`, +- {doc}`group-provider`, +- {doc}`password-authenticator`, +- {doc}`header-authenticator`, +- {doc}`certificate-authenticator`, +- {doc}`event-listener`, +- resource group configuration managers, +- session property configuration managers, +- and exchange managers. + +In particular, connectors are the source of all data for queries in +Trino: they back each catalog available to Trino. + +## Code + +The SPI source can be found in the `core/trino-spi` directory in the Trino +source tree. + +## Plugin metadata + +Each plugin identifies an entry point: an implementation of the +`Plugin` interface. This class name is provided to Trino via +the standard Java `ServiceLoader` interface: the classpath contains +a resource file named `io.trino.spi.Plugin` in the +`META-INF/services` directory. The content of this file is a +single line listing the name of the plugin class: + +```text +com.example.plugin.ExamplePlugin +``` + +For a built-in plugin that is included in the Trino source code, +this resource file is created whenever the `pom.xml` file of a plugin +contains the following line: + +```xml +trino-plugin +``` + +## Plugin + +The `Plugin` interface is a good starting place for developers looking +to understand the Trino SPI. It contains access methods to retrieve +various classes that a Plugin can provide. For example, the `getConnectorFactories()` +method is a top-level function that Trino calls to retrieve a `ConnectorFactory` when Trino +is ready to create an instance of a connector to back a catalog. There are similar +methods for `Type`, `ParametricType`, `Function`, `SystemAccessControl`, and +`EventListenerFactory` objects. + +## Building plugins via Maven + +Plugins depend on the SPI from Trino: + +```xml + + io.trino + trino-spi + provided + +``` + +The plugin uses the Maven `provided` scope because Trino provides +the classes from the SPI at runtime and thus the plugin should not +include them in the plugin assembly. + +There are a few other dependencies that are provided by Trino, +including Slice and Jackson annotations. In particular, Jackson is +used for serializing connector handles and thus plugins must use the +annotations version provided by Trino. + +All other dependencies are based on what the plugin needs for its +own implementation. Plugins are loaded in a separate class loader +to provide isolation and to allow plugins to use a different version +of a library that Trino uses internally. + +For an example `pom.xml` file, see the example HTTP connector in the +`plugin/trino-example-http` directory in the Trino source tree. + +## Deploying a custom plugin + +Because Trino plugins use the `trino-plugin` packaging type, building +a plugin will create a ZIP file in the `target` directory. This file +contains the plugin JAR and all its dependencies JAR files. + +In order to add a custom plugin to a Trino installation, extract the plugin +ZIP file and move the extracted directory into the Trino plugin directory. +For example, for a plugin called `my-functions`, with a version of 1.0, +you would extract `my-functions-1.0.zip` and then move `my-functions-1.0` +to `my-functions` in the Trino plugin directory. + +:::{note} +Every Trino plugin should be in a separate directory. Do not put JAR files +directly into the `plugin` directory. Plugins should only contain JAR files, +so any subdirectories will not be traversed and will be ignored. +::: + +By default, the plugin directory is the `plugin` directory relative to the +directory in which Trino is installed, but it is configurable using the +configuration variable `plugin.dir`. In order for Trino to pick up +the new plugin, you must restart Trino. + +Plugins must be installed on all nodes in the Trino cluster (coordinator and workers). diff --git a/430/_sources/develop/supporting-merge.md.txt b/430/_sources/develop/supporting-merge.md.txt new file mode 100644 index 000000000..b33d2b5e1 --- /dev/null +++ b/430/_sources/develop/supporting-merge.md.txt @@ -0,0 +1,431 @@ +# Supporting `MERGE` + +The Trino engine provides APIs to support row-level SQL `MERGE`. +To implement `MERGE`, a connector must provide the following: + +- An implementation of `ConnectorMergeSink`, which is typically + layered on top of a `ConnectorPageSink`. +- Methods in `ConnectorMetadata` to get a "rowId" column handle, get the + row change paradigm, and to start and complete the `MERGE` operation. + +The Trino engine machinery used to implement SQL `MERGE` is also used to +support SQL `DELETE` and `UPDATE`. This means that all a connector needs to +do is implement support for SQL `MERGE`, and the connector gets all the Data +Modification Language (DML) operations. + +## Standard SQL `MERGE` + +Different query engines support varying definitions of SQL `MERGE`. +Trino supports the strict SQL specification `ISO/IEC 9075`, published +in 2016. As a simple example, given tables `target_table` and +`source_table` defined as: + +``` +CREATE TABLE accounts ( + customer VARCHAR, + purchases DECIMAL, + address VARCHAR); +INSERT INTO accounts (customer, purchases, address) VALUES ...; +CREATE TABLE monthly_accounts_update ( + customer VARCHAR, + purchases DECIMAL, + address VARCHAR); +INSERT INTO monthly_accounts_update (customer, purchases, address) VALUES ...; +``` + +Here is a possible `MERGE` operation, from `monthly_accounts_update` to +`accounts`: + +``` +MERGE INTO accounts t USING monthly_accounts_update s + ON (t.customer = s.customer) + WHEN MATCHED AND s.address = 'Berkeley' THEN + DELETE + WHEN MATCHED AND s.customer = 'Joe Shmoe' THEN + UPDATE SET purchases = purchases + 100.0 + WHEN MATCHED THEN + UPDATE + SET purchases = s.purchases + t.purchases, address = s.address + WHEN NOT MATCHED THEN + INSERT (customer, purchases, address) + VALUES (s.customer, s.purchases, s.address); +``` + +SQL `MERGE` tries to match each `WHEN` clause in source order. When +a match is found, the corresponding `DELETE`, `INSERT` or `UPDATE` +is executed and subsequent `WHEN` clauses are ignored. + +SQL `MERGE` supports two operations on the target table and source +when a row from the source table or query matches a row in the target table: + +- `UPDATE`, in which the columns in the target row are updated. +- `DELETE`, in which the target row is deleted. + +In the `NOT MATCHED` case, SQL `MERGE` supports only `INSERT` +operations. The values inserted are arbitrary but usually come from +the unmatched row of the source table or query. + +## `RowChangeParadigm` + +Different connectors have different ways of representing row updates, +imposed by the underlying storage systems. The Trino engine classifies +these different paradigms as elements of the `RowChangeParadigm` +enumeration, returned by enumeration, returned by method +`ConnectorMetadata.getRowChangeParadigm(...)`. + +The `RowChangeParadigm` enumeration values are: + +- `CHANGE_ONLY_UPDATED_COLUMNS`, intended for connectors that can update + individual columns of rows identified by a `rowId`. The corresponding + merge processor class is `ChangeOnlyUpdatedColumnsMergeProcessor`. +- `DELETE_ROW_AND_INSERT_ROW`, intended for connectors that represent a + row change as a row deletion paired with a row insertion. The corresponding + merge processor class is `DeleteAndInsertMergeProcessor`. + +## Overview of `MERGE` processing + +A `MERGE` statement is processed by creating a `RIGHT JOIN` between the +target table and the source, on the `MERGE` criteria. The source may be +a table or an arbitrary query. For each row in the source table or query, +`MERGE` produces a `ROW` object containing: + +- the data column values from the `UPDATE` or `INSERT` cases. For the + `DELETE` cases, only the partition columns, which determine + partitioning and bucketing, are non-null. +- a boolean column containing `true` for source rows that matched some + target row, and `false` otherwise. +- an integer that identifies whether the merge case operation is `UPDATE`, + `DELETE` or `INSERT`, or a source row for which no case matched. If a + source row doesn't match any merge case, all data column values except + those that determine distribution are null, and the operation number + is -1. + +A `SearchedCaseExpression` is constructed from `RIGHT JOIN` result +to represent the `WHEN` clauses of the `MERGE`. In the example preceding +the `MERGE` is executed as if the `SearchedCaseExpression` were written as: + +``` +SELECT + CASE + WHEN present AND s.address = 'Berkeley' THEN + -- Null values for delete; present=true; operation DELETE=2, case_number=0 + row(null, null, null, true, 2, 0) + WHEN present AND s.customer = 'Joe Shmoe' THEN + -- Update column values; present=true; operation UPDATE=3, case_number=1 + row(t.customer, t.purchases + 100.0, t.address, true, 3, 1) + WHEN present THEN + -- Update column values; present=true; operation UPDATE=3, case_number=2 + row(t.customer, s.purchases + t.purchases, s.address, true, 3, 2) + WHEN (present IS NULL) THEN + -- Insert column values; present=false; operation INSERT=1, case_number=3 + row(s.customer, s.purchases, s.address, false, 1, 3) + ELSE + -- Null values for no case matched; present=false; operation=-1, + -- case_number=-1 + row(null, null, null, false, -1, -1) + END + FROM (SELECT *, true AS present FROM target_table) t + RIGHT JOIN source_table s ON s.customer = t.customer; +``` + +The Trino engine executes the `RIGHT JOIN` and `CASE` expression, +and ensures that no target table row matches more than one source expression +row, and ultimately creates a sequence of pages to be routed to the node that +runs the `ConnectorMergeSink.storeMergedRows(...)` method. + +Like `DELETE` and `UPDATE`, `MERGE` target table rows are identified by +a connector-specific `rowId` column handle. For `MERGE`, the `rowId` +handle is returned by `ConnectorMetadata.getMergeRowIdColumnHandle(...)`. + +## `MERGE` redistribution + +The Trino `MERGE` implementation allows `UPDATE` to change +the values of columns that determine partitioning and/or bucketing, and so +it must "redistribute" rows from the `MERGE` operation to the worker +nodes responsible for writing rows with the merged partitioning and/or +bucketing columns. + +Since the `MERGE` process in general requires redistribution of +merged rows among Trino nodes, the order of rows in pages to be stored +are indeterminate. Connectors like Hive that depend on an ascending +rowId order for deleted rows must sort the deleted rows before storing +them. + +To ensure that all inserted rows for a given partition end up on a +single node, the redistribution hash on the partition key/bucket columns +is applied to the page partition keys. As a result of the hash, all +rows for a specific partition/bucket hash together, whether they +were `MATCHED` rows or `NOT MATCHED` rows. + +For connectors whose `RowChangeParadigm` is `DELETE_ROW_AND_INSERT_ROW`, +inserted rows are distributed using the layout supplied by +`ConnectorMetadata.getInsertLayout()`. For some connectors, the same +layout is used for updated rows. Other connectors require a special +layout for updated rows, supplied by `ConnectorMetadata.getUpdateLayout()`. + +### Connector support for `MERGE` + +To start `MERGE` processing, the Trino engine calls: + +- `ConnectorMetadata.getMergeRowIdColumnHandle(...)` to get the + `rowId` column handle. +- `ConnectorMetadata.getRowChangeParadigm(...)` to get the paradigm + supported by the connector for changing existing table rows. +- `ConnectorMetadata.beginMerge(...)` to get the a + `ConnectorMergeTableHandle` for the merge operation. That + `ConnectorMergeTableHandle` object contains whatever information the + connector needs to specify the `MERGE` operation. +- `ConnectorMetadata.getInsertLayout(...)`, from which it extracts the + the list of partition or table columns that impact write redistribution. +- `ConnectorMetadata.getUpdateLayout(...)`. If that layout is non-empty, + it is used to distribute updated rows resulting from the `MERGE` + operation. + +On nodes that are targets of the hash, the Trino engine calls +`ConnectorPageSinkProvider.createMergeSink(...)` to create a +`ConnectorMergeSink`. + +To write out each page of merged rows, the Trino engine calls +`ConnectorMergeSink.storeMergedRows(Page)`. The `storeMergedRows(Page)` +method iterates over the rows in the page, performing updates and deletes +in the `MATCHED` cases, and inserts in the `NOT MATCHED` cases. + +When using `RowChangeParadigm.DELETE_ROW_AND_INSERT_ROW`, the engine +translates `UPDATE` operations into a pair of `DELETE` and `INSERT` +operations before `storeMergedRows(Page)` is called. + +To complete the `MERGE` operation, the Trino engine calls +`ConnectorMetadata.finishMerge(...)`, passing the table handle +and a collection of JSON objects encoded as `Slice` instances. These +objects contain connector-specific information specifying what was changed +by the `MERGE` operation. Typically this JSON object contains the files +written and table and partition statistics generated by the `MERGE` +operation. The connector takes appropriate actions, if any. + +## `RowChangeProcessor` implementation for `MERGE` + +In the `MERGE` implementation, each `RowChangeParadigm` +corresponds to an internal Trino engine class that implements interface +`RowChangeProcessor`. `RowChangeProcessor` has one interesting method: +`Page transformPage(Page)`. The format of the output page depends +on the `RowChangeParadigm`. + +The connector has no access to the `RowChangeProcessor` instance -- it +is used inside the Trino engine to transform the merge page rows into rows +to be stored, based on the connector's choice of `RowChangeParadigm`. + +The page supplied to `transformPage()` consists of: + +- The write redistribution columns if any +- For partitioned or bucketed tables, a long hash value column. +- The `rowId` column for the row from the target table if matched, or + null if not matched +- The merge case `RowBlock` +- The integer case number block +- The byte `is_distinct` block, with value 0 if not distinct. + +The merge case `RowBlock` has the following layout: + +- Blocks for each column in the table, including partition columns, in + table column order. +- A block containing the boolean "present" value which is true if the + source row matched a target row, and false otherwise. +- A block containing the `MERGE` case operation number, encoded as + `INSERT` = 1, `DELETE` = 2, `UPDATE` = 3 and if no `MERGE` + case matched, -1. +- A block containing the number, starting with 0, for the + `WHEN` clause that matched for the row, or -1 if no clause + matched. + +The page returned from `transformPage` consists of: + +- All table columns, in table column order. +- The merge case operation block. +- The rowId block. +- A byte block containing 1 if the row is an insert derived from an + update operation, and 0 otherwise. This block is used to correctly + calculate the count of rows changed for connectors that represent + updates and deletes plus inserts. + +`transformPage` +must ensure that there are no rows whose operation number is -1 in +the page it returns. + +## Detecting duplicate matching target rows + +The SQL `MERGE` specification requires that in each `MERGE` case, +a single target table row must match at most one source row, after +applying the `MERGE` case condition expression. The first step +toward finding these error is done by labeling each row in the target +table with a unique id, using an `AssignUniqueId` node above the +target table scan. The projected results from the `RIGHT JOIN` +have these unique ids for matched target table rows as well as +the `WHEN` clause number. A `MarkDistinct` node adds an +`is_distinct` column which is true if no other row has the same +unique id and `WHEN` clause number, and false otherwise. If +any row has `is_distinct` equal to false, a +`MERGE_TARGET_ROW_MULTIPLE_MATCHES` exception is raised and +the `MERGE` operation fails. + +## `ConnectorMergeTableHandle` API + +Interface `ConnectorMergeTableHandle` defines one method, +`getTableHandle()` to retrieve the `ConnectorTableHandle` +originally passed to `ConnectorMetadata.beginMerge()`. + +## `ConnectorPageSinkProvider` API + +To support SQL `MERGE`, `ConnectorPageSinkProvider` must implement +the method that creates the `ConnectorMergeSink`: + +- `createMergeSink`: + + ``` + ConnectorMergeSink createMergeSink( + ConnectorTransactionHandle transactionHandle, + ConnectorSession session, + ConnectorMergeTableHandle mergeHandle) + ``` + +## `ConnectorMergeSink` API + +To support `MERGE`, the connector must define an +implementation of `ConnectorMergeSink`, usually layered over the +connector's `ConnectorPageSink`. + +The `ConnectorMergeSink` is created by a call to +`ConnectorPageSinkProvider.createMergeSink()`. + +The only interesting methods are: + +- `storeMergedRows`: + + ``` + void storeMergedRows(Page page) + ``` + + The Trino engine calls the `storeMergedRows(Page)` method of the + `ConnectorMergeSink` instance returned by + `ConnectorPageSinkProvider.createMergeSink()`, passing the page + generated by the `RowChangeProcessor.transformPage()` method. + That page consists of all table columns, in table column order, + followed by the `TINYINT` operation column, followed by the rowId column. + + The job of `storeMergedRows()` is iterate over the rows in the page, + and process them based on the value of the operation column, `INSERT`, + `DELETE`, `UPDATE`, or ignore the row. By choosing appropriate + paradigm, the connector can request that the UPDATE operation be + transformed into `DELETE` and `INSERT` operations. + +- `finish`: + + ``` + CompletableFuture> finish() + ``` + + The Trino engine calls `finish()` when all the data has been processed by + a specific `ConnectorMergeSink` instance. The connector returns a future + containing a collection of `Slice`, representing connector-specific + information about the rows processed. Usually this includes the row count, + and might include information like the files or partitions created or + changed. + +## `ConnectorMetadata` `MERGE` API + +A connector implementing `MERGE` must implement these `ConnectorMetadata` +methods. + +- `getRowChangeParadigm()`: + + ``` + RowChangeParadigm getRowChangeParadigm( + ConnectorSession session, + ConnectorTableHandle tableHandle) + ``` + + This method is called as the engine starts processing a `MERGE` statement. + The connector must return a `RowChangeParadigm` enumeration instance. If + the connector doesn't support `MERGE`, then it should throw a + `NOT_SUPPORTED` exception to indicate that SQL `MERGE` isn't supported by + the connector. Note that the default implementation already throws this + exception when the method isn't implemented. + +- `getMergeRowIdColumnHandle()`: + + ``` + ColumnHandle getMergeRowIdColumnHandle( + ConnectorSession session, + ConnectorTableHandle tableHandle) + ``` + + This method is called in the early stages of query planning for `MERGE` + statements. The ColumnHandle returned provides the `rowId` used by the + connector to identify rows to be merged, as well as any other fields of + the row that the connector needs to complete the `MERGE` operation. + +- `getInsertLayout()`: + + ``` + Optional getInsertLayout( + ConnectorSession session, + ConnectorTableHandle tableHandle) + ``` + + This method is called during query planning to get the table layout to be + used for rows inserted by the `MERGE` operation. For some connectors, + this layout is used for rows deleted as well. + +- `getUpdateLayout()`: + + ``` + Optional getUpdateLayout( + ConnectorSession session, + ConnectorTableHandle tableHandle) + ``` + + This method is called during query planning to get the table layout to be + used for rows deleted by the `MERGE` operation. If the optional return + value is present, the Trino engine uses the layout for updated rows. + Otherwise, it uses the result of `ConnectorMetadata.getInsertLayout` to + distribute updated rows. + +- `beginMerge()`: + + ``` + ConnectorMergeTableHandle beginMerge( + ConnectorSession session, + ConnectorTableHandle tableHandle) + ``` + + As the last step in creating the `MERGE` execution plan, the connector's + `beginMerge()` method is called, passing the `session`, and the + `tableHandle`. + + `beginMerge()` performs any orchestration needed in the connector to + start processing the `MERGE`. This orchestration varies from connector + to connector. In the case of Hive connector operating on transactional tables, + for example, `beginMerge()` checks that the table is transactional and + starts a Hive Metastore transaction. + + `beginMerge()` returns a `ConnectorMergeTableHandle` with any added + information the connector needs when the handle is passed back to + `finishMerge()` and the split generation machinery. For most + connectors, the returned table handle contains at least a flag identifying + the table handle as a table handle for a `MERGE` operation. + +- `finishMerge()`: + + ``` + void finishMerge( + ConnectorSession session, + ConnectorMergeTableHandle tableHandle, + Collection fragments) + ``` + + During `MERGE` processing, the Trino engine accumulates the `Slice` + collections returned by `ConnectorMergeSink.finish()`. The engine calls + `finishMerge()`, passing the table handle and that collection of + `Slice` fragments. In response, the connector takes appropriate actions + to complete the `MERGE` operation. Those actions might include + committing an underlying transaction, if any, or freeing any other + resources. diff --git a/430/_sources/develop/system-access-control.md.txt b/430/_sources/develop/system-access-control.md.txt new file mode 100644 index 000000000..54050491d --- /dev/null +++ b/430/_sources/develop/system-access-control.md.txt @@ -0,0 +1,49 @@ +# System access control + +Trino separates the concept of the principal who authenticates to the +coordinator from the username that is responsible for running queries. When +running the Trino CLI, for example, the Trino username can be specified using +the `--user` option. + +By default, the Trino coordinator allows any principal to run queries as any +Trino user. In a secure environment, this is probably not desirable behavior +and likely requires customization. + +## Implementation + +`SystemAccessControlFactory` is responsible for creating a +`SystemAccessControl` instance. It also defines a `SystemAccessControl` +name which is used by the administrator in a Trino configuration. + +`SystemAccessControl` implementations have several responsibilities: + +- Verifying whether or not a given principal is authorized to execute queries as a specific user. +- Determining whether or not a given user can alter values for a given system property. +- Performing access checks across all catalogs. These access checks happen before + any connector specific checks and thus can deny permissions that would otherwise + be allowed by `ConnectorAccessControl`. + +The implementation of `SystemAccessControl` and `SystemAccessControlFactory` +must be wrapped as a plugin and installed on the Trino cluster. + +## Configuration + +After a plugin that implements `SystemAccessControl` and +`SystemAccessControlFactory` has been installed on the coordinator, it is +configured using the file(s) specified by the `access-control.config-files` +property (the default is a single `etc/access-control.properties` file). +All of the properties other than `access-control.name` are specific to +the `SystemAccessControl` implementation. + +The `access-control.name` property is used by Trino to find a registered +`SystemAccessControlFactory` based on the name returned by +`SystemAccessControlFactory.getName()`. The remaining properties are passed +as a map to `SystemAccessControlFactory.create()`. + +Example configuration file: + +```text +access-control.name=custom-access-control +custom-property1=custom-value1 +custom-property2=custom-value2 +``` diff --git a/430/_sources/develop/table-functions.md.txt b/430/_sources/develop/table-functions.md.txt new file mode 100644 index 000000000..7b5048fa2 --- /dev/null +++ b/430/_sources/develop/table-functions.md.txt @@ -0,0 +1,274 @@ +# Table functions + +Table functions return tables. They allow users to dynamically invoke custom +logic from within the SQL query. They are invoked in the `FROM` clause of a +query, and the calling convention is similar to a scalar function call. For +description of table functions usage, see +{doc}`table functions`. + +Trino supports adding custom table functions. They are declared by connectors +through implementing dedicated interfaces. + +## Table function declaration + +To declare a table function, you need to implement `ConnectorTableFunction`. +Subclassing `AbstractConnectorTableFunction` is a convenient way to do it. +The connector's `getTableFunctions()` method must return a set of your +implementations. + +### The constructor + +```java +public class MyFunction + extends AbstractConnectorTableFunction +{ + public MyFunction() + { + super( + "system", + "my_function", + List.of( + ScalarArgumentSpecification.builder() + .name("COLUMN_COUNT") + .type(INTEGER) + .defaultValue(2) + .build(), + ScalarArgumentSpecification.builder() + .name("ROW_COUNT") + .type(INTEGER) + .build()), + GENERIC_TABLE); + } +} +``` + +The constructor takes the following arguments: + +- **schema name** + +The schema name helps you organize functions, and it is used for function +resolution. When a table function is invoked, the right implementation is +identified by the catalog name, the schema name, and the function name. + +The function can use the schema name, for example to use data from the +indicated schema, or ignore it. + +- **function name** +- **list of expected arguments** + +Three different types of arguments are supported: scalar arguments, descriptor +arguments, and table arguments. See {ref}`tf-argument-types` for details. You can +specify default values for scalar and descriptor arguments. The arguments with +specified default can be skipped during table function invocation. + +- **returned row type** + +It describes the row type produced by the table function. + +If a table function takes table arguments, it can additionally pass the columns +of the input tables to output using the *pass-through mechanism*. The returned +row type is supposed to describe only the columns produced by the function, as +opposed to the pass-through columns. + +In the example, the returned row type is `GENERIC_TABLE`, which means that +the row type is not known statically, and it is determined dynamically based on +the passed arguments. + +When the returned row type is known statically, you can declare it using: + +```java +new DescribedTable(descriptor) +``` + +If a table function does not produce any columns, and it only outputs the +pass-through columns, use `ONLY_PASS_THROUGH` as the returned row type. + +:::{note} +A table function must return at least one column. It can either be a proper +column, i.e. produced by the function, or a pass-through column. +::: + +(tf-argument-types)= + +### Argument types + +Table functions take three types of arguments: +{ref}`scalar arguments`, +{ref}`descriptor arguments`, and +{ref}`table arguments`. + +(tf-scalar-arguments)= + +#### Scalar arguments + +They can be of any supported data type. You can specify a default value. + +```java +ScalarArgumentSpecification.builder() + .name("COLUMN_COUNT") + .type(INTEGER) + .defaultValue(2) + .build() +``` + +```java +ScalarArgumentSpecification.builder() + .name("ROW_COUNT") + .type(INTEGER) + .build() +``` + +(tf-descriptor-arguments)= + +#### Descriptor arguments + +Descriptors consist of fields with names and optional data types. They are a +convenient way to pass the required result row type to the function, or for +example inform the function which input columns it should use. You can specify +default values for descriptor arguments. Descriptor argument can be `null`. + +```java +DescriptorArgumentSpecification.builder() + .name("SCHEMA") + .defaultValue(null) + .build() +``` + +(tf-table-arguments)= + +#### Table arguments + +A table function can take any number of input relations. It allows you to +process multiple data sources simultaneously. + +When declaring a table argument, you must specify characteristics to determine +how the input table is processed. Also note that you cannot specify a default +value for a table argument. + +```java +TableArgumentSpecification.builder() + .name("INPUT") + .rowSemantics() + .pruneWhenEmpty() + .passThroughColumns() + .build() +``` + +(tf-set-or-row-semantics)= + +##### Set or row semantics + +Set semantics is the default for table arguments. A table argument with set +semantics is processed on a partition-by-partition basis. During function +invocation, the user can specify partitioning and ordering for the argument. If +no partitioning is specified, the argument is processed as a single partition. + +A table argument with row semantics is processed on a row-by-row basis. +Partitioning or ordering is not applicable. + +##### Prune or keep when empty + +The *prune when empty* property indicates that if the given table argument is +empty, the function returns empty result. This property is used to optimize +queries involving table functions. The *keep when empty* property indicates +that the function should be executed even if the table argument is empty. The +user can override this property when invoking the function. Using the *keep +when empty* property can negatively affect performance when the table argument +is not empty. + +##### Pass-through columns + +If a table argument has *pass-through columns*, all of its columns are passed +on output. For a table argument without this property, only the partitioning +columns are passed on output. + +### The `analyze()` method + +In order to provide all the necessary information to the Trino engine, the +class must implement the `analyze()` method. This method is called by the +engine during the analysis phase of query processing. The `analyze()` method +is also the place to perform custom checks on the arguments: + +```java +@Override +public TableFunctionAnalysis analyze(ConnectorSession session, ConnectorTransactionHandle transaction, Map arguments) +{ + long columnCount = (long) ((ScalarArgument) arguments.get("COLUMN_COUNT")).getValue(); + long rowCount = (long) ((ScalarArgument) arguments.get("ROW_COUNT")).getValue(); + + // custom validation of arguments + if (columnCount < 1 || columnCount > 3) { + throw new TrinoException(INVALID_FUNCTION_ARGUMENT, "column_count must be in range [1, 3]"); + } + + if (rowCount < 1) { + throw new TrinoException(INVALID_FUNCTION_ARGUMENT, "row_count must be positive"); + } + + // determine the returned row type + List fields = List.of("col_a", "col_b", "col_c").subList(0, (int) columnCount).stream() + .map(name -> new Descriptor.Field(name, Optional.of(BIGINT))) + .collect(toList()); + + Descriptor returnedType = new Descriptor(fields); + + return TableFunctionAnalysis.builder() + .returnedType(returnedType) + .handle(new MyHandle(columnCount, rowCount)) + .build(); +} +``` + +The `analyze()` method returns a `TableFunctionAnalysis` object, which +comprises all the information required by the engine to analyze, plan, and +execute the table function invocation: + +- The returned row type, specified as an optional `Descriptor`. It should be + passed if and only if the table function is declared with the + `GENERIC_TABLE` returned type. +- Required columns from the table arguments, specified as a map of table + argument names to lists of column indexes. +- Any information gathered during analysis that is useful during planning or + execution, in the form of a `ConnectorTableFunctionHandle`. + `ConnectorTableFunctionHandle` is a marker interface intended to carry + information throughout subsequent phases of query processing in a manner that + is opaque to the engine. + +## Table function execution + +There are two paths of execution available for table functions. + +1. Pushdown to the connector + +The connector that provides the table function implements the +`applyTableFunction()` method. This method is called during the optimization +phase of query processing. It returns a `ConnectorTableHandle` and a list of +`ColumnHandle` s representing the table function result. The table function +invocation is then replaced with a `TableScanNode`. + +This execution path is convenient for table functions whose results are easy to +represent as a `ConnectorTableHandle`, for example query pass-through. It +only supports scalar and descriptor arguments. + +2. Execution by operator + +Trino has a dedicated operator for table functions. It can handle table +functions with any number of table arguments as well as scalar and descriptor +arguments. To use this execution path, you provide an implementation of a +processor. + +If your table function has one or more table arguments, you must implement +`TableFunctionDataProcessor`. It processes pages of input data. + +If your table function is a source operator (it does not have table arguments), +you must implement `TableFunctionSplitProcessor`. It processes splits. The +connector that provides the function must provide a `ConnectorSplitSource` +for the function. With splits, the task can be divided so that each split +represents a subtask. + +## Access control + +The access control for table functions can be provided both on system and +connector level. It is based on the fully qualified table function name, +which consists of the catalog name, the schema name, and the function name, +in the syntax of `catalog.schema.function`. diff --git a/430/_sources/develop/types.md.txt b/430/_sources/develop/types.md.txt new file mode 100644 index 000000000..1ae357d27 --- /dev/null +++ b/430/_sources/develop/types.md.txt @@ -0,0 +1,36 @@ +# Types + +The `Type` interface in Trino is used to implement a type in the SQL language. +Trino ships with a number of built-in types, like `VarcharType` and `BigintType`. +The `ParametricType` interface is used to provide type parameters for types, to +allow types like `VARCHAR(10)` or `DECIMAL(22, 5)`. A `Plugin` can provide +new `Type` objects by returning them from `getTypes()` and new `ParametricType` +objects by returning them from `getParametricTypes()`. + +Below is a high level overview of the `Type` interface. For more details, see the +JavaDocs for `Type`. + +## Native container type + +All types define the `getJavaType()` method, frequently referred to as the +"native container type". This is the Java type used to hold values during execution +and to store them in a `Block`. For example, this is the type used in +the Java code that implements functions that produce or consume this `Type`. + +## Native encoding + +The interpretation of a value in its native container type form is defined by its +`Type`. For some types, such as `BigintType`, it matches the Java +interpretation of the native container type (64bit 2's complement). However, for other +types such as `TimestampWithTimeZoneType`, which also uses `long` for its +native container type, the value stored in the `long` is a 8byte binary value +combining the timezone and the milliseconds since the unix epoch. In particular, +this means that you cannot compare two native values and expect a meaningful +result, without knowing the native encoding. + +## Type signature + +The signature of a type defines its identity, and also encodes some general +information about the type, such as its type parameters (if it's parametric), +and its literal parameters. The literal parameters are used in types like +`VARCHAR(10)`. diff --git a/430/_sources/functions.md.txt b/430/_sources/functions.md.txt new file mode 100644 index 000000000..3478e1a92 --- /dev/null +++ b/430/_sources/functions.md.txt @@ -0,0 +1,60 @@ +# Functions and operators + +This section describes the built-in SQL functions and operators supported by +Trino. They allow you to implement complex capabilities and behavior of the +queries executed by Trino operating on the underlying data sources. + +Refer to the following sections for further details: + +* [SQL data types and other general aspects](/language) +* [SQL statement and syntax reference](/sql) + +## Functions by name + +If you are looking for a specific function or operator by name use +[](/sql/show-functions), or refer the to the following resources: + +:::{toctree} +:maxdepth: 1 + +functions/list +functions/list-by-topic +::: + +## Functions per topic + +```{toctree} +:maxdepth: 1 + +Aggregate +Array +Binary +Bitwise +Color +Comparison +Conditional +Conversion +Date and time +Decimal +Geospatial +HyperLogLog +IP Address +JSON +Lambda +Logical +Machine learning +Map +Math +Quantile digest +Regular expression +Session +Set Digest +String +System +Table +Teradata +T-Digest +URL +UUID +Window +``` diff --git a/430/_sources/functions/aggregate.md.txt b/430/_sources/functions/aggregate.md.txt new file mode 100644 index 000000000..fc59ce710 --- /dev/null +++ b/430/_sources/functions/aggregate.md.txt @@ -0,0 +1,606 @@ +# Aggregate functions + +Aggregate functions operate on a set of values to compute a single result. + +Except for {func}`count`, {func}`count_if`, {func}`max_by`, {func}`min_by` and +{func}`approx_distinct`, all of these aggregate functions ignore null values +and return null for no input rows or when all values are null. For example, +{func}`sum` returns null rather than zero and {func}`avg` does not include null +values in the count. The `coalesce` function can be used to convert null into +zero. + +(aggregate-function-ordering-during-aggregation)= + +## Ordering during aggregation + +Some aggregate functions such as {func}`array_agg` produce different results +depending on the order of input values. This ordering can be specified by writing +an {ref}`order-by-clause` within the aggregate function: + +``` +array_agg(x ORDER BY y DESC) +array_agg(x ORDER BY x, y, z) +``` + +(aggregate-function-filtering-during-aggregation)= + +## Filtering during aggregation + +The `FILTER` keyword can be used to remove rows from aggregation processing +with a condition expressed using a `WHERE` clause. This is evaluated for each +row before it is used in the aggregation and is supported for all aggregate +functions. + +```text +aggregate_function(...) FILTER (WHERE ) +``` + +A common and very useful example is to use `FILTER` to remove nulls from +consideration when using `array_agg`: + +``` +SELECT array_agg(name) FILTER (WHERE name IS NOT NULL) +FROM region; +``` + +As another example, imagine you want to add a condition on the count for Iris +flowers, modifying the following query: + +``` +SELECT species, + count(*) AS count +FROM iris +GROUP BY species; +``` + +```text +species | count +-----------+------- +setosa | 50 +virginica | 50 +versicolor | 50 +``` + +If you just use a normal `WHERE` statement you lose information: + +``` +SELECT species, + count(*) AS count +FROM iris +WHERE petal_length_cm > 4 +GROUP BY species; +``` + +```text +species | count +-----------+------- +virginica | 50 +versicolor | 34 +``` + +Using a filter you retain all information: + +``` +SELECT species, + count(*) FILTER (where petal_length_cm > 4) AS count +FROM iris +GROUP BY species; +``` + +```text +species | count +-----------+------- +virginica | 50 +setosa | 0 +versicolor | 34 +``` + +## General aggregate functions + +:::{function} any_value(x) -> [same as input] +Returns an arbitrary non-null value `x`, if one exists. `x` can be any +valid expression. This allows you to return values from columns that are not +directly part of the aggregation, inluding expressions using these columns, +in a query. + +For example, the following query returns the customer name from the `name` +column, and returns the sum of all total prices as customer spend. The +aggregation however uses the rows grouped by the customer identifier +`custkey` a required, since only that column is guaranteed to be unique: + +``` +SELECT sum(o.totalprice) as spend, + any_value(c.name) +FROM tpch.tiny.orders o +JOIN tpch.tiny.customer c +ON o.custkey = c.custkey +GROUP BY c.custkey; +ORDER BY spend; +``` +::: + +:::{function} arbitrary(x) -> [same as input] +Returns an arbitrary non-null value of `x`, if one exists. Identical to +{func}`any_value`. +::: + +:::{function} array_agg(x) -> array<[same as input]> +Returns an array created from the input `x` elements. +::: + +:::{function} avg(x) -> double +Returns the average (arithmetic mean) of all input values. +::: + +:::{function} avg(time interval type) -> time interval type +:noindex: true + +Returns the average interval length of all input values. +::: + +:::{function} bool_and(boolean) -> boolean +Returns `TRUE` if every input value is `TRUE`, otherwise `FALSE`. +::: + +:::{function} bool_or(boolean) -> boolean +Returns `TRUE` if any input value is `TRUE`, otherwise `FALSE`. +::: + +:::{function} checksum(x) -> varbinary +Returns an order-insensitive checksum of the given values. +::: + +:::{function} count(*) -> bigint +Returns the number of input rows. +::: + +:::{function} count(x) -> bigint +:noindex: true + +Returns the number of non-null input values. +::: + +:::{function} count_if(x) -> bigint +Returns the number of `TRUE` input values. +This function is equivalent to `count(CASE WHEN x THEN 1 END)`. +::: + +:::{function} every(boolean) -> boolean +This is an alias for {func}`bool_and`. +::: + +:::{function} geometric_mean(x) -> double +Returns the geometric mean of all input values. +::: + +:::{function} listagg(x, separator) -> varchar +Returns the concatenated input values, separated by the `separator` string. + +Synopsis: + +``` +LISTAGG( expression [, separator] [ON OVERFLOW overflow_behaviour]) + WITHIN GROUP (ORDER BY sort_item, [...]) +``` + +If `separator` is not specified, the empty string will be used as `separator`. + +In its simplest form the function looks like: + +``` +SELECT listagg(value, ',') WITHIN GROUP (ORDER BY value) csv_value +FROM (VALUES 'a', 'c', 'b') t(value); +``` + +and results in: + +``` +csv_value +----------- +'a,b,c' +``` + +The overflow behaviour is by default to throw an error in case that the length of the output +of the function exceeds `1048576` bytes: + +``` +SELECT listagg(value, ',' ON OVERFLOW ERROR) WITHIN GROUP (ORDER BY value) csv_value +FROM (VALUES 'a', 'b', 'c') t(value); +``` + +There exists also the possibility to truncate the output `WITH COUNT` or `WITHOUT COUNT` +of omitted non-null values in case that the length of the output of the +function exceeds `1048576` bytes: + +``` +SELECT LISTAGG(value, ',' ON OVERFLOW TRUNCATE '.....' WITH COUNT) WITHIN GROUP (ORDER BY value) +FROM (VALUES 'a', 'b', 'c') t(value); +``` + +If not specified, the truncation filler string is by default `'...'`. + +This aggregation function can be also used in a scenario involving grouping: + +``` +SELECT id, LISTAGG(value, ',') WITHIN GROUP (ORDER BY o) csv_value +FROM (VALUES + (100, 1, 'a'), + (200, 3, 'c'), + (200, 2, 'b') +) t(id, o, value) +GROUP BY id +ORDER BY id; +``` + +results in: + +```text + id | csv_value +-----+----------- + 100 | a + 200 | b,c +``` + +The current implementation of `LISTAGG` function does not support window frames. +::: + +:::{function} max(x) -> [same as input] +Returns the maximum value of all input values. +::: + +:::{function} max(x, n) -> array<[same as x]> +:noindex: true + +Returns `n` largest values of all input values of `x`. +::: + +:::{function} max_by(x, y) -> [same as x] +Returns the value of `x` associated with the maximum value of `y` over all input values. +::: + +:::{function} max_by(x, y, n) -> array<[same as x]> +:noindex: true + +Returns `n` values of `x` associated with the `n` largest of all input values of `y` +in descending order of `y`. +::: + +:::{function} min(x) -> [same as input] +Returns the minimum value of all input values. +::: + +:::{function} min(x, n) -> array<[same as x]> +:noindex: true + +Returns `n` smallest values of all input values of `x`. +::: + +:::{function} min_by(x, y) -> [same as x] +Returns the value of `x` associated with the minimum value of `y` over all input values. +::: + +:::{function} min_by(x, y, n) -> array<[same as x]> +:noindex: true + +Returns `n` values of `x` associated with the `n` smallest of all input values of `y` +in ascending order of `y`. +::: + +:::{function} sum(x) -> [same as input] +Returns the sum of all input values. +::: + +## Bitwise aggregate functions + +:::{function} bitwise_and_agg(x) -> bigint +Returns the bitwise AND of all input values in 2's complement representation. +::: + +:::{function} bitwise_or_agg(x) -> bigint +Returns the bitwise OR of all input values in 2's complement representation. +::: + +## Map aggregate functions + +:::{function} histogram(x) -> map +Returns a map containing the count of the number of times each input value occurs. +::: + +:::{function} map_agg(key, value) -> map +Returns a map created from the input `key` / `value` pairs. +::: + +:::{function} map_union(x(K,V)) -> map +Returns the union of all the input maps. If a key is found in multiple +input maps, that key's value in the resulting map comes from an arbitrary input map. + +For example, take the following histogram function that creates multiple maps from the Iris dataset: + +``` +SELECT histogram(floor(petal_length_cm)) petal_data +FROM memory.default.iris +GROUP BY species; + + petal_data +-- {4.0=6, 5.0=33, 6.0=11} +-- {4.0=37, 5.0=2, 3.0=11} +-- {1.0=50} +``` + +You can combine these maps using `map_union`: + +``` +SELECT map_union(petal_data) petal_data_union +FROM ( + SELECT histogram(floor(petal_length_cm)) petal_data + FROM memory.default.iris + GROUP BY species + ); + + petal_data_union +--{4.0=6, 5.0=2, 6.0=11, 1.0=50, 3.0=11} +``` +::: + +:::{function} multimap_agg(key, value) -> map +Returns a multimap created from the input `key` / `value` pairs. +Each key can be associated with multiple values. +::: + +## Approximate aggregate functions + +:::{function} approx_distinct(x) -> bigint +Returns the approximate number of distinct input values. +This function provides an approximation of `count(DISTINCT x)`. +Zero is returned if all input values are null. + +This function should produce a standard error of 2.3%, which is the +standard deviation of the (approximately normal) error distribution over +all possible sets. It does not guarantee an upper bound on the error for +any specific input set. +::: + +:::{function} approx_distinct(x, e) -> bigint +:noindex: true + +Returns the approximate number of distinct input values. +This function provides an approximation of `count(DISTINCT x)`. +Zero is returned if all input values are null. + +This function should produce a standard error of no more than `e`, which +is the standard deviation of the (approximately normal) error distribution +over all possible sets. It does not guarantee an upper bound on the error +for any specific input set. The current implementation of this function +requires that `e` be in the range of `[0.0040625, 0.26000]`. +::: + +:::{function} approx_most_frequent(buckets, value, capacity) -> map<[same as value], bigint> +Computes the top frequent values up to `buckets` elements approximately. +Approximate estimation of the function enables us to pick up the frequent +values with less memory. Larger `capacity` improves the accuracy of +underlying algorithm with sacrificing the memory capacity. The returned +value is a map containing the top elements with corresponding estimated +frequency. + +The error of the function depends on the permutation of the values and its +cardinality. We can set the capacity same as the cardinality of the +underlying data to achieve the least error. + +`buckets` and `capacity` must be `bigint`. `value` can be numeric +or string type. + +The function uses the stream summary data structure proposed in the paper +[Efficient Computation of Frequent and Top-k Elements in Data Streams](https://www.cse.ust.hk/~raywong/comp5331/References/EfficientComputationOfFrequentAndTop-kElementsInDataStreams.pdf) +by A. Metwalley, D. Agrawl and A. Abbadi. +::: + +:::{function} approx_percentile(x, percentage) -> [same as x] +Returns the approximate percentile for all input values of `x` at the +given `percentage`. The value of `percentage` must be between zero and +one and must be constant for all input rows. +::: + +:::{function} approx_percentile(x, percentages) -> array<[same as x]> +:noindex: true + +Returns the approximate percentile for all input values of `x` at each of +the specified percentages. Each element of the `percentages` array must be +between zero and one, and the array must be constant for all input rows. +::: + +:::{function} approx_percentile(x, w, percentage) -> [same as x] +:noindex: true + +Returns the approximate weighed percentile for all input values of `x` +using the per-item weight `w` at the percentage `percentage`. Weights must be +greater or equal to 1. Integer-value weights can be thought of as a replication +count for the value `x` in the percentile set. The value of `percentage` must be +between zero and one and must be constant for all input rows. +::: + +:::{function} approx_percentile(x, w, percentages) -> array<[same as x]> +:noindex: true + +Returns the approximate weighed percentile for all input values of `x` +using the per-item weight `w` at each of the given percentages specified +in the array. Weights must be greater or equal to 1. Integer-value weights can +be thought of as a replication count for the value `x` in the percentile +set. Each element of the `percentages` array must be between zero and one, and the array +must be constant for all input rows. +::: + +:::{function} approx_set(x) -> HyperLogLog +:noindex: true + +See {doc}`hyperloglog`. +::: + +:::{function} merge(x) -> HyperLogLog +:noindex: true + +See {doc}`hyperloglog`. +::: + +:::{function} merge(qdigest(T)) -> qdigest(T) +:noindex: true + +See {doc}`qdigest`. +::: + +:::{function} merge(tdigest) -> tdigest +:noindex: true + +See {doc}`tdigest`. +::: + +:::{function} numeric_histogram(buckets, value) -> map +:noindex: true + +Computes an approximate histogram with up to `buckets` number of buckets +for all `value`s. This function is equivalent to the variant of +{func}`numeric_histogram` that takes a `weight`, with a per-item weight of `1`. +::: + +:::{function} numeric_histogram(buckets, value, weight) -> map +Computes an approximate histogram with up to `buckets` number of buckets +for all `value`s with a per-item weight of `weight`. The algorithm +is based loosely on: + +```text +Yael Ben-Haim and Elad Tom-Tov, "A streaming parallel decision tree algorithm", +J. Machine Learning Research 11 (2010), pp. 849--872. +``` + +`buckets` must be a `bigint`. `value` and `weight` must be numeric. +::: + +:::{function} qdigest_agg(x) -> qdigest([same as x]) +:noindex: true + +See {doc}`qdigest`. +::: + +:::{function} qdigest_agg(x, w) -> qdigest([same as x]) +:noindex: true + +See {doc}`qdigest`. +::: + +:::{function} qdigest_agg(x, w, accuracy) -> qdigest([same as x]) +:noindex: true + +See {doc}`qdigest`. +::: + +:::{function} tdigest_agg(x) -> tdigest +:noindex: true + +See {doc}`tdigest`. +::: + +:::{function} tdigest_agg(x, w) -> tdigest +:noindex: true + +See {doc}`tdigest`. +::: + +## Statistical aggregate functions + +:::{function} corr(y, x) -> double +Returns correlation coefficient of input values. +::: + +:::{function} covar_pop(y, x) -> double +Returns the population covariance of input values. +::: + +:::{function} covar_samp(y, x) -> double +Returns the sample covariance of input values. +::: + +:::{function} kurtosis(x) -> double +Returns the excess kurtosis of all input values. Unbiased estimate using +the following expression: + +```text +kurtosis(x) = n(n+1)/((n-1)(n-2)(n-3))sum[(x_i-mean)^4]/stddev(x)^4-3(n-1)^2/((n-2)(n-3)) +``` +::: + +:::{function} regr_intercept(y, x) -> double +Returns linear regression intercept of input values. `y` is the dependent +value. `x` is the independent value. +::: + +:::{function} regr_slope(y, x) -> double +Returns linear regression slope of input values. `y` is the dependent +value. `x` is the independent value. +::: + +:::{function} skewness(x) -> double +Returns the Fisher’s moment coefficient of [skewness](https://wikipedia.org/wiki/Skewness) of all input values. +::: + +:::{function} stddev(x) -> double +This is an alias for {func}`stddev_samp`. +::: + +:::{function} stddev_pop(x) -> double +Returns the population standard deviation of all input values. +::: + +:::{function} stddev_samp(x) -> double +Returns the sample standard deviation of all input values. +::: + +:::{function} variance(x) -> double +This is an alias for {func}`var_samp`. +::: + +:::{function} var_pop(x) -> double +Returns the population variance of all input values. +::: + +:::{function} var_samp(x) -> double +Returns the sample variance of all input values. +::: + +## Lambda aggregate functions + +:::{function} reduce_agg(inputValue T, initialState S, inputFunction(S, T, S), combineFunction(S, S, S)) -> S +Reduces all input values into a single value. `inputFunction` will be invoked +for each non-null input value. In addition to taking the input value, `inputFunction` +takes the current state, initially `initialState`, and returns the new state. +`combineFunction` will be invoked to combine two states into a new state. +The final state is returned: + +``` +SELECT id, reduce_agg(value, 0, (a, b) -> a + b, (a, b) -> a + b) +FROM ( + VALUES + (1, 3), + (1, 4), + (1, 5), + (2, 6), + (2, 7) +) AS t(id, value) +GROUP BY id; +-- (1, 12) +-- (2, 13) + +SELECT id, reduce_agg(value, 1, (a, b) -> a * b, (a, b) -> a * b) +FROM ( + VALUES + (1, 3), + (1, 4), + (1, 5), + (2, 6), + (2, 7) +) AS t(id, value) +GROUP BY id; +-- (1, 60) +-- (2, 42) +``` + +The state type must be a boolean, integer, floating-point, or date/time/interval. +::: diff --git a/430/_sources/functions/array.md.txt b/430/_sources/functions/array.md.txt new file mode 100644 index 000000000..0b4252284 --- /dev/null +++ b/430/_sources/functions/array.md.txt @@ -0,0 +1,423 @@ +# Array functions and operators + +(subscript-operator)= + +## Subscript operator: \[\] + +The `[]` operator is used to access an element of an array and is indexed starting from one: + +``` +SELECT my_array[1] AS first_element +``` + +(concatenation-operator)= + +## Concatenation operator: || + +The `||` operator is used to concatenate an array with an array or an element of the same type: + +``` +SELECT ARRAY[1] || ARRAY[2]; +-- [1, 2] + +SELECT ARRAY[1] || 2; +-- [1, 2] + +SELECT 2 || ARRAY[1]; +-- [2, 1] +``` + +## Array functions + +:::{function} all_match(array(T), function(T,boolean)) -> boolean +Returns whether all elements of an array match the given predicate. Returns `true` if all the elements +match the predicate (a special case is when the array is empty); `false` if one or more elements don't +match; `NULL` if the predicate function returns `NULL` for one or more elements and `true` for all +other elements. +::: + +:::{function} any_match(array(T), function(T,boolean)) -> boolean +Returns whether any elements of an array match the given predicate. Returns `true` if one or more +elements match the predicate; `false` if none of the elements matches (a special case is when the +array is empty); `NULL` if the predicate function returns `NULL` for one or more elements and `false` +for all other elements. +::: + +:::{function} array_distinct(x) -> array +Remove duplicate values from the array `x`. +::: + +:::{function} array_intersect(x, y) -> array +Returns an array of the elements in the intersection of `x` and `y`, without duplicates. +::: + +:::{function} array_union(x, y) -> array +Returns an array of the elements in the union of `x` and `y`, without duplicates. +::: + +:::{function} array_except(x, y) -> array +Returns an array of elements in `x` but not in `y`, without duplicates. +::: + +:::{function} array_histogram(x) -> map +Returns a map where the keys are the unique elements in the input array +`x` and the values are the number of times that each element appears in +`x`. Null values are ignored. + +``` +SELECT array_histogram(ARRAY[42, 7, 42, NULL]); +-- {42=2, 7=1} +``` + +Returns an empty map if the input array has no non-null elements. + +``` +SELECT array_histogram(ARRAY[NULL, NULL]); +-- {} +``` +::: + +:::{function} array_join(x, delimiter, null_replacement) -> varchar +Concatenates the elements of the given array using the delimiter and an optional string to replace nulls. +::: + +:::{function} array_max(x) -> x +Returns the maximum value of input array. +::: + +:::{function} array_min(x) -> x +Returns the minimum value of input array. +::: + +:::{function} array_position(x, element) -> bigint +Returns the position of the first occurrence of the `element` in array `x` (or 0 if not found). +::: + +:::{function} array_remove(x, element) -> array +Remove all elements that equal `element` from array `x`. +::: + +:::{function} array_sort(x) -> array +Sorts and returns the array `x`. The elements of `x` must be orderable. +Null elements will be placed at the end of the returned array. +::: + +:::{function} array_sort(array(T), function(T,T,int)) -> array(T) +:noindex: true + +Sorts and returns the `array` based on the given comparator `function`. +The comparator will take two nullable arguments representing two nullable +elements of the `array`. It returns -1, 0, or 1 as the first nullable +element is less than, equal to, or greater than the second nullable element. +If the comparator function returns other values (including `NULL`), the +query will fail and raise an error. + +``` +SELECT array_sort(ARRAY[3, 2, 5, 1, 2], + (x, y) -> IF(x < y, 1, IF(x = y, 0, -1))); +-- [5, 3, 2, 2, 1] + +SELECT array_sort(ARRAY['bc', 'ab', 'dc'], + (x, y) -> IF(x < y, 1, IF(x = y, 0, -1))); +-- ['dc', 'bc', 'ab'] + + +SELECT array_sort(ARRAY[3, 2, null, 5, null, 1, 2], + -- sort null first with descending order + (x, y) -> CASE WHEN x IS NULL THEN -1 + WHEN y IS NULL THEN 1 + WHEN x < y THEN 1 + WHEN x = y THEN 0 + ELSE -1 END); +-- [null, null, 5, 3, 2, 2, 1] + +SELECT array_sort(ARRAY[3, 2, null, 5, null, 1, 2], + -- sort null last with descending order + (x, y) -> CASE WHEN x IS NULL THEN 1 + WHEN y IS NULL THEN -1 + WHEN x < y THEN 1 + WHEN x = y THEN 0 + ELSE -1 END); +-- [5, 3, 2, 2, 1, null, null] + +SELECT array_sort(ARRAY['a', 'abcd', 'abc'], + -- sort by string length + (x, y) -> IF(length(x) < length(y), -1, + IF(length(x) = length(y), 0, 1))); +-- ['a', 'abc', 'abcd'] + +SELECT array_sort(ARRAY[ARRAY[2, 3, 1], ARRAY[4, 2, 1, 4], ARRAY[1, 2]], + -- sort by array length + (x, y) -> IF(cardinality(x) < cardinality(y), -1, + IF(cardinality(x) = cardinality(y), 0, 1))); +-- [[1, 2], [2, 3, 1], [4, 2, 1, 4]] +``` +::: + +:::{function} arrays_overlap(x, y) -> boolean +Tests if arrays `x` and `y` have any non-null elements in common. +Returns null if there are no non-null elements in common but either array contains null. +::: + +:::{function} cardinality(x) -> bigint +Returns the cardinality (size) of the array `x`. +::: + +:::{function} concat(array1, array2, ..., arrayN) -> array +:noindex: true + +Concatenates the arrays `array1`, `array2`, `...`, `arrayN`. +This function provides the same functionality as the SQL-standard concatenation operator (`||`). +::: + +:::{function} combinations(array(T), n) -> array(array(T)) +Returns n-element sub-groups of input array. If the input array has no duplicates, +`combinations` returns n-element subsets. + +``` +SELECT combinations(ARRAY['foo', 'bar', 'baz'], 2); +-- [['foo', 'bar'], ['foo', 'baz'], ['bar', 'baz']] + +SELECT combinations(ARRAY[1, 2, 3], 2); +-- [[1, 2], [1, 3], [2, 3]] + +SELECT combinations(ARRAY[1, 2, 2], 2); +-- [[1, 2], [1, 2], [2, 2]] +``` + +Order of sub-groups is deterministic but unspecified. Order of elements within +a sub-group deterministic but unspecified. `n` must be not be greater than 5, +and the total size of sub-groups generated must be smaller than 100,000. +::: + +:::{function} contains(x, element) -> boolean +Returns true if the array `x` contains the `element`. +::: + +:::{function} contains_sequence(x, seq) -> boolean +Return true if array `x` contains all of array `seq` as a subsequence (all values in the same consecutive order). +::: + +:::{function} element_at(array(E), index) -> E +Returns element of `array` at given `index`. +If `index` > 0, this function provides the same functionality as the SQL-standard subscript operator (`[]`), +except that the function returns `NULL` when accessing an `index` larger than array length, whereas +the subscript operator would fail in such a case. +If `index` \< 0, `element_at` accesses elements from the last to the first. +::: + +:::{function} filter(array(T), function(T,boolean)) -> array(T) +Constructs an array from those elements of `array` for which `function` returns true: + +``` +SELECT filter(ARRAY[], x -> true); +-- [] + +SELECT filter(ARRAY[5, -6, NULL, 7], x -> x > 0); +-- [5, 7] + +SELECT filter(ARRAY[5, NULL, 7, NULL], x -> x IS NOT NULL); +-- [5, 7] +``` +::: + +:::{function} flatten(x) -> array +Flattens an `array(array(T))` to an `array(T)` by concatenating the contained arrays. +::: + +:::{function} ngrams(array(T), n) -> array(array(T)) +Returns `n`-grams (sub-sequences of adjacent `n` elements) for the `array`. +The order of the `n`-grams in the result is unspecified. + +``` +SELECT ngrams(ARRAY['foo', 'bar', 'baz', 'foo'], 2); +-- [['foo', 'bar'], ['bar', 'baz'], ['baz', 'foo']] + +SELECT ngrams(ARRAY['foo', 'bar', 'baz', 'foo'], 3); +-- [['foo', 'bar', 'baz'], ['bar', 'baz', 'foo']] + +SELECT ngrams(ARRAY['foo', 'bar', 'baz', 'foo'], 4); +-- [['foo', 'bar', 'baz', 'foo']] + +SELECT ngrams(ARRAY['foo', 'bar', 'baz', 'foo'], 5); +-- [['foo', 'bar', 'baz', 'foo']] + +SELECT ngrams(ARRAY[1, 2, 3, 4], 2); +-- [[1, 2], [2, 3], [3, 4]] +``` +::: + +:::{function} none_match(array(T), function(T,boolean)) -> boolean +Returns whether no elements of an array match the given predicate. Returns `true` if none of the elements +matches the predicate (a special case is when the array is empty); `false` if one or more elements match; +`NULL` if the predicate function returns `NULL` for one or more elements and `false` for all other elements. +::: + +:::{function} reduce(array(T), initialState S, inputFunction(S,T,S), outputFunction(S,R)) -> R +Returns a single value reduced from `array`. `inputFunction` will +be invoked for each element in `array` in order. In addition to taking +the element, `inputFunction` takes the current state, initially +`initialState`, and returns the new state. `outputFunction` will be +invoked to turn the final state into the result value. It may be the +identity function (`i -> i`). + +``` +SELECT reduce(ARRAY[], 0, + (s, x) -> s + x, + s -> s); +-- 0 + +SELECT reduce(ARRAY[5, 20, 50], 0, + (s, x) -> s + x, + s -> s); +-- 75 + +SELECT reduce(ARRAY[5, 20, NULL, 50], 0, + (s, x) -> s + x, + s -> s); +-- NULL + +SELECT reduce(ARRAY[5, 20, NULL, 50], 0, + (s, x) -> s + coalesce(x, 0), + s -> s); +-- 75 + +SELECT reduce(ARRAY[5, 20, NULL, 50], 0, + (s, x) -> IF(x IS NULL, s, s + x), + s -> s); +-- 75 + +SELECT reduce(ARRAY[2147483647, 1], BIGINT '0', + (s, x) -> s + x, + s -> s); +-- 2147483648 + +-- calculates arithmetic average +SELECT reduce(ARRAY[5, 6, 10, 20], + CAST(ROW(0.0, 0) AS ROW(sum DOUBLE, count INTEGER)), + (s, x) -> CAST(ROW(x + s.sum, s.count + 1) AS + ROW(sum DOUBLE, count INTEGER)), + s -> IF(s.count = 0, NULL, s.sum / s.count)); +-- 10.25 +``` +::: + +:::{function} repeat(element, count) -> array +Repeat `element` for `count` times. +::: + +:::{function} reverse(x) -> array +:noindex: true + +Returns an array which has the reversed order of array `x`. +::: + +:::{function} sequence(start, stop) -> array(bigint) +Generate a sequence of integers from `start` to `stop`, incrementing +by `1` if `start` is less than or equal to `stop`, otherwise `-1`. +::: + +:::{function} sequence(start, stop, step) -> array(bigint) +:noindex: true + +Generate a sequence of integers from `start` to `stop`, incrementing by `step`. +::: + +:::{function} sequence(start, stop) -> array(date) +:noindex: true + +Generate a sequence of dates from `start` date to `stop` date, incrementing +by `1` day if `start` date is less than or equal to `stop` date, otherwise `-1` day. +::: + +:::{function} sequence(start, stop, step) -> array(date) +:noindex: true + +Generate a sequence of dates from `start` to `stop`, incrementing by `step`. +The type of `step` can be either `INTERVAL DAY TO SECOND` or `INTERVAL YEAR TO MONTH`. +::: + +:::{function} sequence(start, stop, step) -> array(timestamp) +:noindex: true + +Generate a sequence of timestamps from `start` to `stop`, incrementing by `step`. +The type of `step` can be either `INTERVAL DAY TO SECOND` or `INTERVAL YEAR TO MONTH`. +::: + +:::{function} shuffle(x) -> array +Generate a random permutation of the given array `x`. +::: + +:::{function} slice(x, start, length) -> array +Subsets array `x` starting from index `start` (or starting from the end +if `start` is negative) with a length of `length`. +::: + +:::{function} trim_array(x, n) -> array +Remove `n` elements from the end of array: + +``` +SELECT trim_array(ARRAY[1, 2, 3, 4], 1); +-- [1, 2, 3] + +SELECT trim_array(ARRAY[1, 2, 3, 4], 2); +-- [1, 2] +``` +::: + +:::{function} transform(array(T), function(T,U)) -> array(U) +Returns an array that is the result of applying `function` to each element of `array`: + +``` +SELECT transform(ARRAY[], x -> x + 1); +-- [] + +SELECT transform(ARRAY[5, 6], x -> x + 1); +-- [6, 7] + +SELECT transform(ARRAY[5, NULL, 6], x -> coalesce(x, 0) + 1); +-- [6, 1, 7] + +SELECT transform(ARRAY['x', 'abc', 'z'], x -> x || '0'); +-- ['x0', 'abc0', 'z0'] + +SELECT transform(ARRAY[ARRAY[1, NULL, 2], ARRAY[3, NULL]], + a -> filter(a, x -> x IS NOT NULL)); +-- [[1, 2], [3]] +``` +::: + +:::{function} zip(array1, array2[, ...]) -> array(row) +Merges the given arrays, element-wise, into a single array of rows. The M-th element of +the N-th argument will be the N-th field of the M-th output element. +If the arguments have an uneven length, missing values are filled with `NULL`. + +``` +SELECT zip(ARRAY[1, 2], ARRAY['1b', null, '3b']); +-- [ROW(1, '1b'), ROW(2, null), ROW(null, '3b')] +``` +::: + +:::{function} zip_with(array(T), array(U), function(T,U,R)) -> array(R) +Merges the two given arrays, element-wise, into a single array using `function`. +If one array is shorter, nulls are appended at the end to match the length of the +longer array, before applying `function`. + +``` +SELECT zip_with(ARRAY[1, 3, 5], ARRAY['a', 'b', 'c'], + (x, y) -> (y, x)); +-- [ROW('a', 1), ROW('b', 3), ROW('c', 5)] + +SELECT zip_with(ARRAY[1, 2], ARRAY[3, 4], + (x, y) -> x + y); +-- [4, 6] + +SELECT zip_with(ARRAY['a', 'b', 'c'], ARRAY['d', 'e', 'f'], + (x, y) -> concat(x, y)); +-- ['ad', 'be', 'cf'] + +SELECT zip_with(ARRAY['a'], ARRAY['d', null, 'f'], + (x, y) -> coalesce(x, y)); +-- ['a', null, 'f'] +``` +::: diff --git a/430/_sources/functions/binary.md.txt b/430/_sources/functions/binary.md.txt new file mode 100644 index 000000000..8d86f06bd --- /dev/null +++ b/430/_sources/functions/binary.md.txt @@ -0,0 +1,203 @@ +# Binary functions and operators + +## Binary operators + +The `||` operator performs concatenation. + +## Binary functions + +:::{function} concat(binary1, ..., binaryN) -> varbinary +:noindex: true + +Returns the concatenation of `binary1`, `binary2`, `...`, `binaryN`. +This function provides the same functionality as the +SQL-standard concatenation operator (`||`). +::: + +:::{function} length(binary) -> bigint +:noindex: true + +Returns the length of `binary` in bytes. +::: + +:::{function} lpad(binary, size, padbinary) -> varbinary +:noindex: true + +Left pads `binary` to `size` bytes with `padbinary`. +If `size` is less than the length of `binary`, the result is +truncated to `size` characters. `size` must not be negative +and `padbinary` must be non-empty. +::: + +:::{function} rpad(binary, size, padbinary) -> varbinary +:noindex: true + +Right pads `binary` to `size` bytes with `padbinary`. +If `size` is less than the length of `binary`, the result is +truncated to `size` characters. `size` must not be negative +and `padbinary` must be non-empty. +::: + +:::{function} substr(binary, start) -> varbinary +:noindex: true + +Returns the rest of `binary` from the starting position `start`, +measured in bytes. Positions start with `1`. A negative starting position +is interpreted as being relative to the end of the string. +::: + +:::{function} substr(binary, start, length) -> varbinary +:noindex: true + +Returns a substring from `binary` of length `length` from the starting +position `start`, measured in bytes. Positions start with `1`. A +negative starting position is interpreted as being relative to the end of +the string. +::: + +(function-reverse-varbinary)= + +:::{function} reverse(binary) -> varbinary +:noindex: true + +Returns `binary` with the bytes in reverse order. +::: + +## Base64 encoding functions + +The Base64 functions implement the encoding specified in {rfc}`4648`. + +:::{function} from_base64(string) -> varbinary +Decodes binary data from the base64 encoded `string`. +::: + +:::{function} to_base64(binary) -> varchar +Encodes `binary` into a base64 string representation. +::: + +:::{function} from_base64url(string) -> varbinary +Decodes binary data from the base64 encoded `string` using the URL safe alphabet. +::: + +:::{function} to_base64url(binary) -> varchar +Encodes `binary` into a base64 string representation using the URL safe alphabet. +::: + +:::{function} from_base32(string) -> varbinary +Decodes binary data from the base32 encoded `string`. +::: + +:::{function} to_base32(binary) -> varchar +Encodes `binary` into a base32 string representation. +::: + +## Hex encoding functions + +:::{function} from_hex(string) -> varbinary +Decodes binary data from the hex encoded `string`. +::: + +:::{function} to_hex(binary) -> varchar +Encodes `binary` into a hex string representation. +::: + +## Integer encoding functions + +:::{function} from_big_endian_32(binary) -> integer +Decodes the 32-bit two's complement big-endian `binary`. +The input must be exactly 4 bytes. +::: + +:::{function} to_big_endian_32(integer) -> varbinary +Encodes `integer` into a 32-bit two's complement big-endian format. +::: + +:::{function} from_big_endian_64(binary) -> bigint +Decodes the 64-bit two's complement big-endian `binary`. +The input must be exactly 8 bytes. +::: + +:::{function} to_big_endian_64(bigint) -> varbinary +Encodes `bigint` into a 64-bit two's complement big-endian format. +::: + +## Floating-point encoding functions + +:::{function} from_ieee754_32(binary) -> real +Decodes the 32-bit big-endian `binary` in IEEE 754 single-precision floating-point format. +The input must be exactly 4 bytes. +::: + +:::{function} to_ieee754_32(real) -> varbinary +Encodes `real` into a 32-bit big-endian binary according to IEEE 754 single-precision floating-point format. +::: + +:::{function} from_ieee754_64(binary) -> double +Decodes the 64-bit big-endian `binary` in IEEE 754 double-precision floating-point format. +The input must be exactly 8 bytes. +::: + +:::{function} to_ieee754_64(double) -> varbinary +Encodes `double` into a 64-bit big-endian binary according to IEEE 754 double-precision floating-point format. +::: + +## Hashing functions + +:::{function} crc32(binary) -> bigint +Computes the CRC-32 of `binary`. For general purpose hashing, use +{func}`xxhash64`, as it is much faster and produces a better quality hash. +::: + +:::{function} md5(binary) -> varbinary +Computes the MD5 hash of `binary`. +::: + +:::{function} sha1(binary) -> varbinary +Computes the SHA1 hash of `binary`. +::: + +:::{function} sha256(binary) -> varbinary +Computes the SHA256 hash of `binary`. +::: + +:::{function} sha512(binary) -> varbinary +Computes the SHA512 hash of `binary`. +::: + +:::{function} spooky_hash_v2_32(binary) -> varbinary +Computes the 32-bit SpookyHashV2 hash of `binary`. +::: + +:::{function} spooky_hash_v2_64(binary) -> varbinary +Computes the 64-bit SpookyHashV2 hash of `binary`. +::: + +:::{function} xxhash64(binary) -> varbinary +Computes the xxHash64 hash of `binary`. +::: + +:::{function} murmur3(binary) -> varbinary +Computes the 128-bit [MurmurHash3](https://wikipedia.org/wiki/MurmurHash) +hash of `binary`. + +> SELECT murmur3(from_base64('aaaaaa')); +> -- ba 58 55 63 55 69 b4 2f 49 20 37 2c a0 e3 96 ef +::: + +## HMAC functions + +:::{function} hmac_md5(binary, key) -> varbinary +Computes HMAC with MD5 of `binary` with the given `key`. +::: + +:::{function} hmac_sha1(binary, key) -> varbinary +Computes HMAC with SHA1 of `binary` with the given `key`. +::: + +:::{function} hmac_sha256(binary, key) -> varbinary +Computes HMAC with SHA256 of `binary` with the given `key`. +::: + +:::{function} hmac_sha512(binary, key) -> varbinary +Computes HMAC with SHA512 of `binary` with the given `key`. +::: diff --git a/430/_sources/functions/bitwise.md.txt b/430/_sources/functions/bitwise.md.txt new file mode 100644 index 000000000..246d481d8 --- /dev/null +++ b/430/_sources/functions/bitwise.md.txt @@ -0,0 +1,139 @@ +# Bitwise functions + +:::{function} bit_count(x, bits) -> bigint +Count the number of bits set in `x` (treated as `bits`-bit signed +integer) in 2's complement representation: + +``` +SELECT bit_count(9, 64); -- 2 +SELECT bit_count(9, 8); -- 2 +SELECT bit_count(-7, 64); -- 62 +SELECT bit_count(-7, 8); -- 6 +``` +::: + +:::{function} bitwise_and(x, y) -> bigint +Returns the bitwise AND of `x` and `y` in 2's complement representation. + +Bitwise AND of `19` (binary: `10011`) and `25` (binary: `11001`) results in +`17` (binary: `10001`): + +``` +SELECT bitwise_and(19,25); -- 17 +``` +::: + +:::{function} bitwise_not(x) -> bigint +Returns the bitwise NOT of `x` in 2's complement representation +(`NOT x = -x - 1`): + +``` +SELECT bitwise_not(-12); -- 11 +SELECT bitwise_not(19); -- -20 +SELECT bitwise_not(25); -- -26 +``` +::: + +:::{function} bitwise_or(x, y) -> bigint +Returns the bitwise OR of `x` and `y` in 2's complement representation. + +Bitwise OR of `19` (binary: `10011`) and `25` (binary: `11001`) results in +`27` (binary: `11011`): + +``` +SELECT bitwise_or(19,25); -- 27 +``` +::: + +:::{function} bitwise_xor(x, y) -> bigint +Returns the bitwise XOR of `x` and `y` in 2's complement representation. + +Bitwise XOR of `19` (binary: `10011`) and `25` (binary: `11001`) results in +`10` (binary: `01010`): + +``` +SELECT bitwise_xor(19,25); -- 10 +``` +::: + +:::{function} bitwise_left_shift(value, shift) -> [same as value] +Returns the left shifted value of `value`. + +Shifting `1` (binary: `001`) by two bits results in `4` (binary: `00100`): + +``` +SELECT bitwise_left_shift(1, 2); -- 4 +``` + +Shifting `5` (binary: `0101`) by two bits results in `20` (binary: `010100`): + +``` +SELECT bitwise_left_shift(5, 2); -- 20 +``` + +Shifting a `value` by `0` always results in the original `value`: + +``` +SELECT bitwise_left_shift(20, 0); -- 20 +SELECT bitwise_left_shift(42, 0); -- 42 +``` + +Shifting `0` by a `shift` always results in `0`: + +``` +SELECT bitwise_left_shift(0, 1); -- 0 +SELECT bitwise_left_shift(0, 2); -- 0 +``` +::: + +:::{function} bitwise_right_shift(value, shift) -> [same as value] +Returns the logical right shifted value of `value`. + +Shifting `8` (binary: `1000`) by three bits results in `1` (binary: `001`): + +``` +SELECT bitwise_right_shift(8, 3); -- 1 +``` + +Shifting `9` (binary: `1001`) by one bit results in `4` (binary: `100`): + +``` +SELECT bitwise_right_shift(9, 1); -- 4 +``` + +Shifting a `value` by `0` always results in the original `value`: + +``` +SELECT bitwise_right_shift(20, 0); -- 20 +SELECT bitwise_right_shift(42, 0); -- 42 +``` + +Shifting a `value` by `64` or more bits results in `0`: + +``` +SELECT bitwise_right_shift( 12, 64); -- 0 +SELECT bitwise_right_shift(-45, 64); -- 0 +``` + +Shifting `0` by a `shift` always results in `0`: + +``` +SELECT bitwise_right_shift(0, 1); -- 0 +SELECT bitwise_right_shift(0, 2); -- 0 +``` +::: + +:::{function} bitwise_right_shift_arithmetic(value, shift) -> [same as value] +Returns the arithmetic right shifted value of `value`. + +Returns the same values as {func}`bitwise_right_shift` when shifting by less than +`64` bits. Shifting by `64` or more bits results in `0` for a positive and +`-1` for a negative `value`: + +``` +SELECT bitwise_right_shift_arithmetic( 12, 64); -- 0 +SELECT bitwise_right_shift_arithmetic(-45, 64); -- -1 +``` +::: + +See also {func}`bitwise_and_agg` and {func}`bitwise_or_agg`. diff --git a/430/_sources/functions/color.md.txt b/430/_sources/functions/color.md.txt new file mode 100644 index 000000000..746a2f7e8 --- /dev/null +++ b/430/_sources/functions/color.md.txt @@ -0,0 +1,75 @@ +# Color functions + +:::{function} bar(x, width) -> varchar +Renders a single bar in an ANSI bar chart using a default +`low_color` of red and a `high_color` of green. For example, +if `x` of 25% and width of 40 are passed to this function. A +10-character red bar will be drawn followed by 30 spaces to create +a bar of 40 characters. +::: + +::::{function} bar(x, width, low_color, high_color) -> varchar +:noindex: true + +Renders a single line in an ANSI bar chart of the specified +`width`. The parameter `x` is a double value between 0 and 1. +Values of `x` that fall outside the range \[0, 1\] will be +truncated to either a 0 or a 1 value. The `low_color` and +`high_color` capture the color to use for either end of +the horizontal bar chart. For example, if `x` is 0.5, `width` +is 80, `low_color` is 0xFF0000, and `high_color` is 0x00FF00 +this function will return a 40 character bar that varies from red +(0xFF0000) and yellow (0xFFFF00) and the remainder of the 80 +character bar will be padded with spaces. + +:::{figure} ../images/functions_color_bar.png +:align: center +::: +:::: + +:::{function} color(string) -> color +Returns a color capturing a decoded RGB value from a 4-character +string of the format "#000". The input string should be varchar +containing a CSS-style short rgb string or one of `black`, +`red`, `green`, `yellow`, `blue`, `magenta`, `cyan`, +`white`. +::: + +:::{function} color(x, low, high, low_color, high_color) -> color +:noindex: true + +Returns a color interpolated between `low_color` and +`high_color` using the double parameters `x`, `low`, and +`high` to calculate a fraction which is then passed to the +`color(fraction, low_color, high_color)` function shown below. +If `x` falls outside the range defined by `low` and `high` +its value is truncated to fit within this range. +::: + +:::{function} color(x, low_color, high_color) -> color +:noindex: true + +Returns a color interpolated between `low_color` and +`high_color` according to the double argument `x` between 0 +and 1. The parameter `x` is a double value between 0 and 1. +Values of `x` that fall outside the range \[0, 1\] will be +truncated to either a 0 or a 1 value. +::: + +:::{function} render(x, color) -> varchar +Renders value `x` using the specific color using ANSI +color codes. `x` can be either a double, bigint, or varchar. +::: + +:::{function} render(b) -> varchar +:noindex: true + +Accepts boolean value `b` and renders a green true or a red +false using ANSI color codes. +::: + +:::{function} rgb(red, green, blue) -> color +Returns a color value capturing the RGB value of three +component color values supplied as int parameters ranging from 0 +to 255: `red`, `green`, `blue`. +::: diff --git a/430/_sources/functions/comparison.md.txt b/430/_sources/functions/comparison.md.txt new file mode 100644 index 000000000..bb137c267 --- /dev/null +++ b/430/_sources/functions/comparison.md.txt @@ -0,0 +1,303 @@ +# Comparison functions and operators + +(comparison-operators)= + +## Comparison operators + +:::{list-table} +:widths: 30, 70 +:header-rows: 1 + +* - Operator + - Description +* - `<` + - Less than +* - `>` + - Greater than +* - `<=` + - Less than or equal to +* - `>=` + - Greater than or equal to +* - `=` + - Equal +* - `<>` + - Not equal +* - `!=` + - Not equal (non-standard but popular syntax) +::: + +(range-operator)= + +## Range operator: BETWEEN + +The `BETWEEN` operator tests if a value is within a specified range. +It uses the syntax `value BETWEEN min AND max`: + +``` +SELECT 3 BETWEEN 2 AND 6; +``` + +The statement shown above is equivalent to the following statement: + +``` +SELECT 3 >= 2 AND 3 <= 6; +``` + +To test if a value does not fall within the specified range +use `NOT BETWEEN`: + +``` +SELECT 3 NOT BETWEEN 2 AND 6; +``` + +The statement shown above is equivalent to the following statement: + +``` +SELECT 3 < 2 OR 3 > 6; +``` + +A `NULL` in a `BETWEEN` or `NOT BETWEEN` statement is evaluated +using the standard `NULL` evaluation rules applied to the equivalent +expression above: + +``` +SELECT NULL BETWEEN 2 AND 4; -- null + +SELECT 2 BETWEEN NULL AND 6; -- null + +SELECT 2 BETWEEN 3 AND NULL; -- false + +SELECT 8 BETWEEN NULL AND 6; -- false +``` + +The `BETWEEN` and `NOT BETWEEN` operators can also be used to +evaluate any orderable type. For example, a `VARCHAR`: + +``` +SELECT 'Paul' BETWEEN 'John' AND 'Ringo'; -- true +``` + +Note that the value, min, and max parameters to `BETWEEN` and `NOT +BETWEEN` must be the same type. For example, Trino will produce an +error if you ask it if John is between 2.3 and 35.2. + +(is-null-operator)= + +## IS NULL and IS NOT NULL + +The `IS NULL` and `IS NOT NULL` operators test whether a value +is null (undefined). Both operators work for all data types. + +Using `NULL` with `IS NULL` evaluates to true: + +``` +select NULL IS NULL; -- true +``` + +But any other constant does not: + +``` +SELECT 3.0 IS NULL; -- false +``` + +(is-distinct-operator)= + +## IS DISTINCT FROM and IS NOT DISTINCT FROM + +In SQL a `NULL` value signifies an unknown value, so any comparison +involving a `NULL` will produce `NULL`. The `IS DISTINCT FROM` +and `IS NOT DISTINCT FROM` operators treat `NULL` as a known value +and both operators guarantee either a true or false outcome even in +the presence of `NULL` input: + +``` +SELECT NULL IS DISTINCT FROM NULL; -- false + +SELECT NULL IS NOT DISTINCT FROM NULL; -- true +``` + +In the example shown above, a `NULL` value is not considered +distinct from `NULL`. When you are comparing values which may +include `NULL` use these operators to guarantee either a `TRUE` or +`FALSE` result. + +The following truth table demonstrate the handling of `NULL` in +`IS DISTINCT FROM` and `IS NOT DISTINCT FROM`: + +| a | b | a = b | a \<> b | a DISTINCT b | a NOT DISTINCT b | +| ------ | ------ | ------- | ------- | ------------ | ---------------- | +| `1` | `1` | `TRUE` | `FALSE` | `FALSE` | `TRUE` | +| `1` | `2` | `FALSE` | `TRUE` | `TRUE` | `FALSE` | +| `1` | `NULL` | `NULL` | `NULL` | `TRUE` | `FALSE` | +| `NULL` | `NULL` | `NULL` | `NULL` | `FALSE` | `TRUE` | + +## GREATEST and LEAST + +These functions are not in the SQL standard, but are a common extension. +Like most other functions in Trino, they return null if any argument is +null. Note that in some other databases, such as PostgreSQL, they only +return null if all arguments are null. + +The following types are supported: +`DOUBLE`, +`BIGINT`, +`VARCHAR`, +`TIMESTAMP`, +`TIMESTAMP WITH TIME ZONE`, +`DATE` + +:::{function} greatest(value1, value2, ..., valueN) -> [same as input] +Returns the largest of the provided values. +::: + +:::{function} least(value1, value2, ..., valueN) -> [same as input] +Returns the smallest of the provided values. +::: + +(quantified-comparison-predicates)= + +## Quantified comparison predicates: ALL, ANY and SOME + +The `ALL`, `ANY` and `SOME` quantifiers can be used together with comparison operators in the +following way: + +```text +expression operator quantifier ( subquery ) +``` + +For example: + +``` +SELECT 'hello' = ANY (VALUES 'hello', 'world'); -- true + +SELECT 21 < ALL (VALUES 19, 20, 21); -- false + +SELECT 42 >= SOME (SELECT 41 UNION ALL SELECT 42 UNION ALL SELECT 43); -- true +``` + +Here are the meanings of some quantifier and comparison operator combinations: + +:::{list-table} +:widths: 40, 60 +:header-rows: 1 + +* - Expression + - Meaning +* - `A = ALL (...)` + - Evaluates to `true` when `A` is equal to all values. +* - `A <> ALL (...)` + - Evaluates to `true` when `A` doesn't match any value. +* - `A < ALL (...)` + - Evaluates to `true` when `A` is smaller than the smallest value. +* - `A = ANY (...)` + - Evaluates to `true` when `A` is equal to any of the values. This form + is equivalent to `A IN (...)`. +* - `A <> ANY (...)` + - Evaluates to `true` when `A` doesn't match one or more values. +* - `A < ANY (...)` + - Evaluates to `true` when `A` is smaller than the biggest value. +::: + +`ANY` and `SOME` have the same meaning and can be used interchangeably. + +(like-operator)= + +## Pattern comparison: LIKE + +The `LIKE` operator can be used to compare values with a pattern: + +``` +... column [NOT] LIKE 'pattern' ESCAPE 'character'; +``` + +Matching characters is case sensitive, and the pattern supports two symbols for +matching: + +- `_` matches any single character +- `%` matches zero or more characters + +Typically it is often used as a condition in `WHERE` statements. An example is +a query to find all continents starting with `E`, which returns `Europe`: + +``` +SELECT * FROM (VALUES 'America', 'Asia', 'Africa', 'Europe', 'Australia', 'Antarctica') AS t (continent) +WHERE continent LIKE 'E%'; +``` + +You can negate the result by adding `NOT`, and get all other continents, all +not starting with `E`: + +``` +SELECT * FROM (VALUES 'America', 'Asia', 'Africa', 'Europe', 'Australia', 'Antarctica') AS t (continent) +WHERE continent NOT LIKE 'E%'; +``` + +If you only have one specific character to match, you can use the `_` symbol +for each character. The following query uses two underscores and produces only +`Asia` as result: + +``` +SELECT * FROM (VALUES 'America', 'Asia', 'Africa', 'Europe', 'Australia', 'Antarctica') AS t (continent) +WHERE continent LIKE 'A__A'; +``` + +The wildcard characters `_` and `%` must be escaped to allow you to match +them as literals. This can be achieved by specifying the `ESCAPE` character to +use: + +``` +SELECT 'South_America' LIKE 'South\_America' ESCAPE '\'; +``` + +The above query returns `true` since the escaped underscore symbol matches. If +you need to match the used escape character as well, you can escape it. + +If you want to match for the chosen escape character, you simply escape itself. +For example, you can use `\\` to match for `\`. + +(in-operator)= + +## Row comparison: IN + +The `IN` operator can be used in a `WHERE` clause to compare column values with +a list of values. The list of values can be supplied by a subquery or directly +as static values in an array: + +```sql +... WHERE column [NOT] IN ('value1','value2'); +... WHERE column [NOT] IN ( subquery ); +``` + +Use the optional `NOT` keyword to negate the condition. + +The following example shows a simple usage with a static array: + +```sql +SELECT * FROM region WHERE name IN ('AMERICA', 'EUROPE'); +``` + +The values in the clause are used for multiple comparisons that are combined as +a logical `OR`. The preceding query is equivalent to the following query: + +```sql +SELECT * FROM region WHERE name = 'AMERICA' OR name = 'EUROPE'; +``` + +You can negate the comparisons by adding `NOT`, and get all other regions +except the values in list: + +```sql +SELECT * FROM region WHERE name NOT IN ('AMERICA', 'EUROPE'); +``` + +When using a subquery to determine the values to use in the comparison, the +subquery must return a single column and one or more rows. + +```sql +SELECT name +FROM nation +WHERE regionkey IN ( + SELECT starts_with(regionkey,"A") AS regionkey + FROM region +); +``` diff --git a/430/_sources/functions/conditional.md.txt b/430/_sources/functions/conditional.md.txt new file mode 100644 index 000000000..3e07ed293 --- /dev/null +++ b/430/_sources/functions/conditional.md.txt @@ -0,0 +1,201 @@ +# Conditional expressions + +(case-expression)= + +## CASE + +The standard SQL `CASE` expression has two forms. +The "simple" form searches each `value` expression from left to right +until it finds one that equals `expression`: + +```text +CASE expression + WHEN value THEN result + [ WHEN ... ] + [ ELSE result ] +END +``` + +The `result` for the matching `value` is returned. +If no match is found, the `result` from the `ELSE` clause is +returned if it exists, otherwise null is returned. Example: + +``` +SELECT a, + CASE a + WHEN 1 THEN 'one' + WHEN 2 THEN 'two' + ELSE 'many' + END +``` + +The "searched" form evaluates each boolean `condition` from left +to right until one is true and returns the matching `result`: + +```text +CASE + WHEN condition THEN result + [ WHEN ... ] + [ ELSE result ] +END +``` + +If no conditions are true, the `result` from the `ELSE` clause is +returned if it exists, otherwise null is returned. Example: + +``` +SELECT a, b, + CASE + WHEN a = 1 THEN 'aaa' + WHEN b = 2 THEN 'bbb' + ELSE 'ccc' + END +``` + +(if-function)= + +## IF + +The `IF` expression has two forms, one supplying only a +`true_value` and the other supplying both a `true_value` and a +`false_value`: + +:::{function} if(condition, true_value) +Evaluates and returns `true_value` if `condition` is true, +otherwise null is returned and `true_value` is not evaluated. +::: + +:::{function} if(condition, true_value, false_value) +:noindex: true + +Evaluates and returns `true_value` if `condition` is true, +otherwise evaluates and returns `false_value`. +::: + +The following `IF` and `CASE` expressions are equivalent: + +```sql +SELECT + orderkey, + totalprice, + IF(totalprice >= 150000, 'High Value', 'Low Value') +FROM tpch.sf1.orders; +``` + +```sql +SELECT + orderkey, + totalprice, + CASE + WHEN totalprice >= 150000 THEN 'High Value' + ELSE 'Low Value' + END +FROM tpch.sf1.orders; +``` + +(coalesce-function)= + +## COALESCE + +:::{function} coalesce(value1, value2[, ...]) +Returns the first non-null `value` in the argument list. +Like a `CASE` expression, arguments are only evaluated if necessary. +::: + +(nullif-function)= + +## NULLIF + +:::{function} nullif(value1, value2) +Returns null if `value1` equals `value2`, otherwise returns `value1`. +::: + +(try-function)= + +## TRY + +:::{function} try(expression) +Evaluate an expression and handle certain types of errors by returning +`NULL`. +::: + +In cases where it is preferable that queries produce `NULL` or default values +instead of failing when corrupt or invalid data is encountered, the `TRY` +function may be useful. To specify default values, the `TRY` function can be +used in conjunction with the `COALESCE` function. + +The following errors are handled by `TRY`: + +- Division by zero +- Invalid cast or function argument +- Numeric value out of range + +### Examples + +Source table with some invalid data: + +```sql +SELECT * FROM shipping; +``` + +```text + origin_state | origin_zip | packages | total_cost +--------------+------------+----------+------------ + California | 94131 | 25 | 100 + California | P332a | 5 | 72 + California | 94025 | 0 | 155 + New Jersey | 08544 | 225 | 490 +(4 rows) +``` + +Query failure without `TRY`: + +```sql +SELECT CAST(origin_zip AS BIGINT) FROM shipping; +``` + +```text +Query failed: Cannot cast 'P332a' to BIGINT +``` + +`NULL` values with `TRY`: + +```sql +SELECT TRY(CAST(origin_zip AS BIGINT)) FROM shipping; +``` + +```text + origin_zip +------------ + 94131 + NULL + 94025 + 08544 +(4 rows) +``` + +Query failure without `TRY`: + +```sql +SELECT total_cost / packages AS per_package FROM shipping; +``` + +```text +Query failed: Division by zero +``` + +Default values with `TRY` and `COALESCE`: + +```sql +SELECT COALESCE(TRY(total_cost / packages), 0) AS per_package FROM shipping; +``` + +```text + per_package +------------- + 4 + 14 + 0 + 19 +(4 rows) +``` diff --git a/430/_sources/functions/conversion.md.txt b/430/_sources/functions/conversion.md.txt new file mode 100644 index 000000000..7de546ab7 --- /dev/null +++ b/430/_sources/functions/conversion.md.txt @@ -0,0 +1,123 @@ +# Conversion functions + +Trino will implicitly convert numeric and character values to the +correct type if such a conversion is possible. Trino will not convert +between character and numeric types. For example, a query that expects +a varchar will not automatically convert a bigint value to an +equivalent varchar. + +When necessary, values can be explicitly cast to a particular type. + +## Conversion functions + +:::{function} cast(value AS type) -> type +Explicitly cast a value as a type. This can be used to cast a +varchar to a numeric value type and vice versa. +::: + +:::{function} try_cast(value AS type) -> type +Like {func}`cast`, but returns null if the cast fails. +::: + +## Formatting + +:::{function} format(format, args...) -> varchar +Returns a formatted string using the specified [format string](https://docs.oracle.com/en/java/javase/17/docs/api/java.base/java/util/Formatter.html#syntax) +and arguments: + +``` +SELECT format('%s%%', 123); +-- '123%' + +SELECT format('%.5f', pi()); +-- '3.14159' + +SELECT format('%03d', 8); +-- '008' + +SELECT format('%,.2f', 1234567.89); +-- '1,234,567.89' + +SELECT format('%-7s,%7s', 'hello', 'world'); +-- 'hello , world' + +SELECT format('%2$s %3$s %1$s', 'a', 'b', 'c'); +-- 'b c a' + +SELECT format('%1$tA, %1$tB %1$te, %1$tY', date '2006-07-04'); +-- 'Tuesday, July 4, 2006' +``` +::: + +:::{function} format_number(number) -> varchar +Returns a formatted string using a unit symbol: + +``` +SELECT format_number(123456); -- '123K' +SELECT format_number(1000000); -- '1M' +``` +::: + +## Data size + +The `parse_data_size` function supports the following units: + +:::{list-table} +:widths: 30, 40, 30 +:header-rows: 1 + +* - Unit + - Description + - Value +* - ``B`` + - Bytes + - 1 +* - ``kB`` + - Kilobytes + - 1024 +* - ``MB`` + - Megabytes + - 1024{sup}`2` +* - ``GB`` + - Gigabytes + - 1024{sup}`3` +* - ``TB`` + - Terabytes + - 1024{sup}`4` +* - ``PB`` + - Petabytes + - 1024{sup}`5` +* - ``EB`` + - Exabytes + - 1024{sup}`6` +* - ``ZB`` + - Zettabytes + - 1024{sup}`7` +* - ``YB`` + - Yottabytes + - 1024{sup}`8` +::: + +:::{function} parse_data_size(string) -> decimal(38) +Parses `string` of format `value unit` into a number, where +`value` is the fractional number of `unit` values: + +``` +SELECT parse_data_size('1B'); -- 1 +SELECT parse_data_size('1kB'); -- 1024 +SELECT parse_data_size('1MB'); -- 1048576 +SELECT parse_data_size('2.3MB'); -- 2411724 +``` +::: + +## Miscellaneous + +:::{function} typeof(expr) -> varchar +Returns the name of the type of the provided expression: + +``` +SELECT typeof(123); -- integer +SELECT typeof('cat'); -- varchar(3) +SELECT typeof(cos(2) + 1.5); -- double +``` +::: diff --git a/430/_sources/functions/datetime.md.txt b/430/_sources/functions/datetime.md.txt new file mode 100644 index 000000000..d599874e8 --- /dev/null +++ b/430/_sources/functions/datetime.md.txt @@ -0,0 +1,575 @@ +# Date and time functions and operators + +These functions and operators operate on {ref}`date and time data types `. + +## Date and time operators + +| Operator | Example | Result | +| -------- | --------------------------------------------------- | ------------------------- | +| `+` | `date '2012-08-08' + interval '2' day` | `2012-08-10` | +| `+` | `time '01:00' + interval '3' hour` | `04:00:00.000` | +| `+` | `timestamp '2012-08-08 01:00' + interval '29' hour` | `2012-08-09 06:00:00.000` | +| `+` | `timestamp '2012-10-31 01:00' + interval '1' month` | `2012-11-30 01:00:00.000` | +| `+` | `interval '2' day + interval '3' hour` | `2 03:00:00.000` | +| `+` | `interval '3' year + interval '5' month` | `3-5` | +| `-` | `date '2012-08-08' - interval '2' day` | `2012-08-06` | +| `-` | `time '01:00' - interval '3' hour` | `22:00:00.000` | +| `-` | `timestamp '2012-08-08 01:00' - interval '29' hour` | `2012-08-06 20:00:00.000` | +| `-` | `timestamp '2012-10-31 01:00' - interval '1' month` | `2012-09-30 01:00:00.000` | +| `-` | `interval '2' day - interval '3' hour` | `1 21:00:00.000` | +| `-` | `interval '3' year - interval '5' month` | `2-7` | + +(at-time-zone-operator)= + +## Time zone conversion + +The `AT TIME ZONE` operator sets the time zone of a timestamp: + +``` +SELECT timestamp '2012-10-31 01:00 UTC'; +-- 2012-10-31 01:00:00.000 UTC + +SELECT timestamp '2012-10-31 01:00 UTC' AT TIME ZONE 'America/Los_Angeles'; +-- 2012-10-30 18:00:00.000 America/Los_Angeles +``` + +## Date and time functions + +:::{data} current_date +Returns the current date as of the start of the query. +::: + +:::{data} current_time +Returns the current time with time zone as of the start of the query. +::: + +:::{data} current_timestamp +Returns the current timestamp with time zone as of the start of the query, +with `3` digits of subsecond precision, +::: + +:::{data} current_timestamp(p) +:noindex: true + +Returns the current {ref}`timestamp with time zone +` as of the start of the query, with +`p` digits of subsecond precision: + +``` +SELECT current_timestamp(6); +-- 2020-06-24 08:25:31.759993 America/Los_Angeles +``` +::: + +:::{function} current_timezone() -> varchar +Returns the current time zone in the format defined by IANA +(e.g., `America/Los_Angeles`) or as fixed offset from UTC (e.g., `+08:35`) +::: + +:::{function} date(x) -> date +This is an alias for `CAST(x AS date)`. +::: + +:::{function} last_day_of_month(x) -> date +Returns the last day of the month. +::: + +:::{function} from_iso8601_timestamp(string) -> timestamp(3) with time zone +Parses the ISO 8601 formatted date `string`, optionally with time and time +zone, into a `timestamp(3) with time zone`. The time defaults to +`00:00:00.000`, and the time zone defaults to the session time zone: + +``` +SELECT from_iso8601_timestamp('2020-05-11'); +-- 2020-05-11 00:00:00.000 America/Vancouver + +SELECT from_iso8601_timestamp('2020-05-11T11:15:05'); +-- 2020-05-11 11:15:05.000 America/Vancouver + +SELECT from_iso8601_timestamp('2020-05-11T11:15:05.055+01:00'); +-- 2020-05-11 11:15:05.055 +01:00 +``` +::: + +:::{function} from_iso8601_timestamp_nanos(string) -> timestamp(9) with time zone +Parses the ISO 8601 formatted date and time `string`. The time zone +defaults to the session time zone: + +``` +SELECT from_iso8601_timestamp_nanos('2020-05-11T11:15:05'); +-- 2020-05-11 11:15:05.000000000 America/Vancouver + +SELECT from_iso8601_timestamp_nanos('2020-05-11T11:15:05.123456789+01:00'); +-- 2020-05-11 11:15:05.123456789 +01:00 +``` +::: + +:::{function} from_iso8601_date(string) -> date +Parses the ISO 8601 formatted date `string` into a `date`. The date can +be a calendar date, a week date using ISO week numbering, or year and day +of year combined: + +``` +SELECT from_iso8601_date('2020-05-11'); +-- 2020-05-11 + +SELECT from_iso8601_date('2020-W10'); +-- 2020-03-02 + +SELECT from_iso8601_date('2020-123'); +-- 2020-05-02 +``` +::: + +:::{function} at_timezone(timestamp(p), zone) -> timestamp(p) with time zone +Returns the timestamp specified in `timestamp` with the time zone +converted from the session time zone to the time zone specified in `zone` +with precision `p`. In the following example, the session time zone is set +to `America/New_York`, which is three hours ahead of +`America/Los_Angeles`: + +``` +SELECT current_timezone() +-- America/New_York + +SELECT at_timezone(TIMESTAMP '2022-11-01 09:08:07.321', 'America/Los_Angeles') +-- 2022-11-01 06:08:07.321 America/Los_Angeles +``` +::: + +:::{function} with_timezone(timestamp(p), zone) -> timestamp(p) with time zone +Returns the timestamp specified in `timestamp` with the time zone +specified in `zone` with precision `p`: + +``` +SELECT current_timezone() +-- America/New_York + +SELECT with_timezone(TIMESTAMP '2022-11-01 09:08:07.321', 'America/Los_Angeles') +-- 2022-11-01 09:08:07.321 America/Los_Angeles +``` +::: + +:::{function} from_unixtime(unixtime) -> timestamp(3) with time zone +Returns the UNIX timestamp `unixtime` as a timestamp with time zone. `unixtime` is the +number of seconds since `1970-01-01 00:00:00 UTC`. +::: + +:::{function} from_unixtime(unixtime, zone) -> timestamp(3) with time zone +:noindex: true + +Returns the UNIX timestamp `unixtime` as a timestamp with time zone +using `zone` for the time zone. `unixtime` is the number of seconds +since `1970-01-01 00:00:00 UTC`. +::: + +:::{function} from_unixtime(unixtime, hours, minutes) -> timestamp(3) with time zone +:noindex: true + +Returns the UNIX timestamp `unixtime` as a timestamp with time zone +using `hours` and `minutes` for the time zone offset. `unixtime` is +the number of seconds since `1970-01-01 00:00:00` in `double` data type. +::: + +:::{function} from_unixtime_nanos(unixtime) -> timestamp(9) with time zone +Returns the UNIX timestamp `unixtime` as a timestamp with time zone. `unixtime` is the +number of nanoseconds since `1970-01-01 00:00:00.000000000 UTC`: + +``` +SELECT from_unixtime_nanos(100); +-- 1970-01-01 00:00:00.000000100 UTC + +SELECT from_unixtime_nanos(DECIMAL '1234'); +-- 1970-01-01 00:00:00.000001234 UTC + +SELECT from_unixtime_nanos(DECIMAL '1234.499'); +-- 1970-01-01 00:00:00.000001234 UTC + +SELECT from_unixtime_nanos(DECIMAL '-1234'); +-- 1969-12-31 23:59:59.999998766 UTC +``` +::: + +:::{data} localtime +Returns the current time as of the start of the query. +::: + +:::{data} localtimestamp +Returns the current timestamp as of the start of the query, with `3` +digits of subsecond precision. +::: + +:::{data} localtimestamp(p) +:noindex: true + +Returns the current {ref}`timestamp ` as of the start +of the query, with `p` digits of subsecond precision: + +``` +SELECT localtimestamp(6); +-- 2020-06-10 15:55:23.383628 +``` +::: + +:::{function} now() -> timestamp(3) with time zone +This is an alias for `current_timestamp`. +::: + +:::{function} to_iso8601(x) -> varchar +Formats `x` as an ISO 8601 string. `x` can be date, timestamp, or +timestamp with time zone. +::: + +:::{function} to_milliseconds(interval) -> bigint +Returns the day-to-second `interval` as milliseconds. +::: + +:::{function} to_unixtime(timestamp) -> double +Returns `timestamp` as a UNIX timestamp. +::: + +:::{note} +The following SQL-standard functions do not use parenthesis: + +- `current_date` +- `current_time` +- `current_timestamp` +- `localtime` +- `localtimestamp` +::: + +## Truncation function + +The `date_trunc` function supports the following units: + +| Unit | Example Truncated Value | +| --------- | ------------------------- | +| `second` | `2001-08-22 03:04:05.000` | +| `minute` | `2001-08-22 03:04:00.000` | +| `hour` | `2001-08-22 03:00:00.000` | +| `day` | `2001-08-22 00:00:00.000` | +| `week` | `2001-08-20 00:00:00.000` | +| `month` | `2001-08-01 00:00:00.000` | +| `quarter` | `2001-07-01 00:00:00.000` | +| `year` | `2001-01-01 00:00:00.000` | + +The above examples use the timestamp `2001-08-22 03:04:05.321` as the input. + +:::{function} date_trunc(unit, x) -> [same as input] +Returns `x` truncated to `unit`: + +``` +SELECT date_trunc('day' , TIMESTAMP '2022-10-20 05:10:00'); +-- 2022-10-20 00:00:00.000 + +SELECT date_trunc('month' , TIMESTAMP '2022-10-20 05:10:00'); +-- 2022-10-01 00:00:00.000 + +SELECT date_trunc('year', TIMESTAMP '2022-10-20 05:10:00'); +-- 2022-01-01 00:00:00.000 +``` +::: + +(datetime-interval-functions)= + +## Interval functions + +The functions in this section support the following interval units: + +| Unit | Description | +| ------------- | ------------------ | +| `millisecond` | Milliseconds | +| `second` | Seconds | +| `minute` | Minutes | +| `hour` | Hours | +| `day` | Days | +| `week` | Weeks | +| `month` | Months | +| `quarter` | Quarters of a year | +| `year` | Years | + +:::{function} date_add(unit, value, timestamp) -> [same as input] +Adds an interval `value` of type `unit` to `timestamp`. +Subtraction can be performed by using a negative value: + +``` +SELECT date_add('second', 86, TIMESTAMP '2020-03-01 00:00:00'); +-- 2020-03-01 00:01:26.000 + +SELECT date_add('hour', 9, TIMESTAMP '2020-03-01 00:00:00'); +-- 2020-03-01 09:00:00.000 + +SELECT date_add('day', -1, TIMESTAMP '2020-03-01 00:00:00 UTC'); +-- 2020-02-29 00:00:00.000 UTC +``` +::: + +:::{function} date_diff(unit, timestamp1, timestamp2) -> bigint +Returns `timestamp2 - timestamp1` expressed in terms of `unit`: + +``` +SELECT date_diff('second', TIMESTAMP '2020-03-01 00:00:00', TIMESTAMP '2020-03-02 00:00:00'); +-- 86400 + +SELECT date_diff('hour', TIMESTAMP '2020-03-01 00:00:00 UTC', TIMESTAMP '2020-03-02 00:00:00 UTC'); +-- 24 + +SELECT date_diff('day', DATE '2020-03-01', DATE '2020-03-02'); +-- 1 + +SELECT date_diff('second', TIMESTAMP '2020-06-01 12:30:45.000000000', TIMESTAMP '2020-06-02 12:30:45.123456789'); +-- 86400 + +SELECT date_diff('millisecond', TIMESTAMP '2020-06-01 12:30:45.000000000', TIMESTAMP '2020-06-02 12:30:45.123456789'); +-- 86400123 +``` +::: + +## Duration function + +The `parse_duration` function supports the following units: + +| Unit | Description | +| ---- | ------------ | +| `ns` | Nanoseconds | +| `us` | Microseconds | +| `ms` | Milliseconds | +| `s` | Seconds | +| `m` | Minutes | +| `h` | Hours | +| `d` | Days | + +:::{function} parse_duration(string) -> interval +Parses `string` of format `value unit` into an interval, where +`value` is fractional number of `unit` values: + +``` +SELECT parse_duration('42.8ms'); +-- 0 00:00:00.043 + +SELECT parse_duration('3.81 d'); +-- 3 19:26:24.000 + +SELECT parse_duration('5m'); +-- 0 00:05:00.000 +``` +::: + +:::{function} human_readable_seconds(double) -> varchar +Formats the double value of `seconds` into a human readable string containing +`weeks`, `days`, `hours`, `minutes`, and `seconds`: + +``` +SELECT human_readable_seconds(96); +-- 1 minute, 36 seconds + +SELECT human_readable_seconds(3762); +-- 1 hour, 2 minutes, 42 seconds + +SELECT human_readable_seconds(56363463); +-- 93 weeks, 1 day, 8 hours, 31 minutes, 3 seconds +``` +::: + +## MySQL date functions + +The functions in this section use a format string that is compatible with +the MySQL `date_parse` and `str_to_date` functions. The following table, +based on the MySQL manual, describes the format specifiers: + +| Specifier | Description | +| --------- | ------------------------------------------------------------------------------------------------------------------- | +| `%a` | Abbreviated weekday name (`Sun` .. `Sat`) | +| `%b` | Abbreviated month name (`Jan` .. `Dec`) | +| `%c` | Month, numeric (`1` .. `12`), this specifier does not support `0` as a month. | +| `%D` | Day of the month with English suffix (`0th`, `1st`, `2nd`, `3rd`, ...) | +| `%d` | Day of the month, numeric (`01` .. `31`), this specifier does not support `0` as a month or day. | +| `%e` | Day of the month, numeric (`1` .. `31`), this specifier does not support `0` as a day. | +| `%f` | Fraction of second (6 digits for printing: `000000` .. `999000`; 1 - 9 digits for parsing: `0` .. `999999999`), timestamp is truncated to milliseconds. | +| `%H` | Hour (`00` .. `23`) | +| `%h` | Hour (`01` .. `12`) | +| `%I` | Hour (`01` .. `12`) | +| `%i` | Minutes, numeric (`00` .. `59`) | +| `%j` | Day of year (`001` .. `366`) | +| `%k` | Hour (`0` .. `23`) | +| `%l` | Hour (`1` .. `12`) | +| `%M` | Month name (`January` .. `December`) | +| `%m` | Month, numeric (`01` .. `12`), this specifier does not support `0` as a month. | +| `%p` | `AM` or `PM` | +| `%r` | Time of day, 12-hour (equivalent to `%h:%i:%s %p`) | +| `%S` | Seconds (`00` .. `59`) | +| `%s` | Seconds (`00` .. `59`) | +| `%T` | Time of day, 24-hour (equivalent to `%H:%i:%s`) | +| `%U` | Week (`00` .. `53`), where Sunday is the first day of the week | +| `%u` | Week (`00` .. `53`), where Monday is the first day of the week | +| `%V` | Week (`01` .. `53`), where Sunday is the first day of the week; used with `%X` | +| `%v` | Week (`01` .. `53`), where Monday is the first day of the week; used with `%x` | +| `%W` | Weekday name (`Sunday` .. `Saturday`) | +| `%w` | Day of the week (`0` .. `6`), where Sunday is the first day of the week, this specifier is not supported,consider using {func}`day_of_week` (it uses `1-7` instead of `0-6`). | +| `%X` | Year for the week where Sunday is the first day of the week, numeric, four digits; used with `%V` | +| `%x` | Year for the week, where Monday is the first day of the week, numeric, four digits; used with `%v` | +| `%Y` | Year, numeric, four digits | +| `%y` | Year, numeric (two digits), when parsing, two-digit year format assumes range `1970` .. `2069`, so "70" will result in year `1970` but "69" will produce `2069`. | +| `%%` | A literal `%` character | +| `%x` | `x`, for any `x` not listed above | + +:::{warning} +The following specifiers are not currently supported: `%D %U %u %V %w %X` +::: + +:::{function} date_format(timestamp, format) -> varchar +Formats `timestamp` as a string using `format`: + +``` +SELECT date_format(TIMESTAMP '2022-10-20 05:10:00', '%m-%d-%Y %H'); +-- 10-20-2022 05 +``` +::: + +:::{function} date_parse(string, format) -> timestamp(3) +Parses `string` into a timestamp using `format`: + +``` +SELECT date_parse('2022/10/20/05', '%Y/%m/%d/%H'); +-- 2022-10-20 05:00:00.000 +``` +::: + +## Java date functions + +The functions in this section use a format string that is compatible with +JodaTime's [DateTimeFormat] pattern format. + +:::{function} format_datetime(timestamp, format) -> varchar +Formats `timestamp` as a string using `format`. +::: + +:::{function} parse_datetime(string, format) -> timestamp with time zone +Parses `string` into a timestamp with time zone using `format`. +::: + +## Extraction function + +The `extract` function supports the following fields: + +| Field | Description | +| ----------------- | ----------------------- | +| `YEAR` | {func}`year` | +| `QUARTER` | {func}`quarter` | +| `MONTH` | {func}`month` | +| `WEEK` | {func}`week` | +| `DAY` | {func}`day` | +| `DAY_OF_MONTH` | {func}`day` | +| `DAY_OF_WEEK` | {func}`day_of_week` | +| `DOW` | {func}`day_of_week` | +| `DAY_OF_YEAR` | {func}`day_of_year` | +| `DOY` | {func}`day_of_year` | +| `YEAR_OF_WEEK` | {func}`year_of_week` | +| `YOW` | {func}`year_of_week` | +| `HOUR` | {func}`hour` | +| `MINUTE` | {func}`minute` | +| `SECOND` | {func}`second` | +| `TIMEZONE_HOUR` | {func}`timezone_hour` | +| `TIMEZONE_MINUTE` | {func}`timezone_minute` | + +The types supported by the `extract` function vary depending on the +field to be extracted. Most fields support all date and time types. + +::::{function} extract(field FROM x) -> bigint +Returns `field` from `x`: + +``` +SELECT extract(YEAR FROM TIMESTAMP '2022-10-20 05:10:00'); +-- 2022 +``` + +:::{note} +This SQL-standard function uses special syntax for specifying the arguments. +::: +:::: + +## Convenience extraction functions + +:::{function} day(x) -> bigint +Returns the day of the month from `x`. +::: + +:::{function} day_of_month(x) -> bigint +This is an alias for {func}`day`. +::: + +:::{function} day_of_week(x) -> bigint +Returns the ISO day of the week from `x`. +The value ranges from `1` (Monday) to `7` (Sunday). +::: + +:::{function} day_of_year(x) -> bigint +Returns the day of the year from `x`. +The value ranges from `1` to `366`. +::: + +:::{function} dow(x) -> bigint +This is an alias for {func}`day_of_week`. +::: + +:::{function} doy(x) -> bigint +This is an alias for {func}`day_of_year`. +::: + +:::{function} hour(x) -> bigint +Returns the hour of the day from `x`. +The value ranges from `0` to `23`. +::: + +:::{function} millisecond(x) -> bigint +Returns the millisecond of the second from `x`. +::: + +:::{function} minute(x) -> bigint +Returns the minute of the hour from `x`. +::: + +:::{function} month(x) -> bigint +Returns the month of the year from `x`. +::: + +:::{function} quarter(x) -> bigint +Returns the quarter of the year from `x`. +The value ranges from `1` to `4`. +::: + +:::{function} second(x) -> bigint +Returns the second of the minute from `x`. +::: + +:::{function} timezone_hour(timestamp) -> bigint +Returns the hour of the time zone offset from `timestamp`. +::: + +:::{function} timezone_minute(timestamp) -> bigint +Returns the minute of the time zone offset from `timestamp`. +::: + +:::{function} week(x) -> bigint +Returns the [ISO week] of the year from `x`. +The value ranges from `1` to `53`. +::: + +:::{function} week_of_year(x) -> bigint +This is an alias for {func}`week`. +::: + +:::{function} year(x) -> bigint +Returns the year from `x`. +::: + +:::{function} year_of_week(x) -> bigint +Returns the year of the [ISO week] from `x`. +::: + +:::{function} yow(x) -> bigint +This is an alias for {func}`year_of_week`. +::: + +[datetimeformat]: http://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html +[iso week]: https://wikipedia.org/wiki/ISO_week_date diff --git a/430/_sources/functions/decimal.md.txt b/430/_sources/functions/decimal.md.txt new file mode 100644 index 000000000..d6371f309 --- /dev/null +++ b/430/_sources/functions/decimal.md.txt @@ -0,0 +1,89 @@ +# Decimal functions and operators + +(decimal-literal)= + +## Decimal literals + +Use the `DECIMAL 'xxxxxxx.yyyyyyy'` syntax to define a decimal literal. + +The precision of a decimal type for a literal will be equal to the number of digits +in the literal (including trailing and leading zeros). The scale will be equal +to the number of digits in the fractional part (including trailing zeros). + +:::{list-table} +:widths: 50, 50 +:header-rows: 1 + +* - Example literal + - Data type +* - `DECIMAL '0'` + - `DECIMAL(1)` +* - `DECIMAL '12345'` + - `DECIMAL(5)` +* - `DECIMAL '0000012345.1234500000'` + - `DECIMAL(20, 10)` +::: + +## Binary arithmetic decimal operators + +Standard mathematical operators are supported. The table below explains +precision and scale calculation rules for result. +Assuming `x` is of type `DECIMAL(xp, xs)` and `y` is of type `DECIMAL(yp, ys)`. + +:::{list-table} +:widths: 30, 40, 30 +:header-rows: 1 + +* - Operation + - Result type precision + - Result type scale +* - `x + y` and `x - y` + - + ``` + min(38, + 1 + + max(xs, ys) + + max(xp - xs, yp - ys) + ) + ``` + - `max(xs, ys)` +* - `x * y` + - ``` + min(38, xp + yp) + ``` + - `xs + ys` +* - `x / y` + - + ``` + min(38, + xp + ys-xs + + max(0, ys-xs) + ) + ``` + - `max(xs, ys)` +* - `x % y` + - ``` + min(xp - xs, yp - ys) + + max(xs, bs) + ``` + - `max(xs, ys)` +::: + +If the mathematical result of the operation is not exactly representable with +the precision and scale of the result data type, +then an exception condition is raised: `Value is out of range`. + +When operating on decimal types with different scale and precision, the values are +first coerced to a common super type. For types near the largest representable precision (38), +this can result in Value is out of range errors when one of the operands doesn't fit +in the common super type. For example, the common super type of decimal(38, 0) and +decimal(38, 1) is decimal(38, 1), but certain values that fit in decimal(38, 0) +cannot be represented as a decimal(38, 1). + +## Comparison operators + +All standard {doc}`comparison` work for the decimal type. + +## Unary decimal operators + +The `-` operator performs negation. The type of result is same as type of argument. diff --git a/430/_sources/functions/geospatial.md.txt b/430/_sources/functions/geospatial.md.txt new file mode 100644 index 000000000..448bc8f2e --- /dev/null +++ b/430/_sources/functions/geospatial.md.txt @@ -0,0 +1,499 @@ +# Geospatial functions + +Trino Geospatial functions that begin with the `ST_` prefix support the SQL/MM specification +and are compliant with the Open Geospatial Consortium’s (OGC) OpenGIS Specifications. +As such, many Trino Geospatial functions require, or more accurately, assume that +geometries that are operated on are both simple and valid. For example, it does not +make sense to calculate the area of a polygon that has a hole defined outside of the +polygon, or to construct a polygon from a non-simple boundary line. + +Trino Geospatial functions support the Well-Known Text (WKT) and Well-Known Binary (WKB) form of spatial objects: + +- `POINT (0 0)` +- `LINESTRING (0 0, 1 1, 1 2)` +- `POLYGON ((0 0, 4 0, 4 4, 0 4, 0 0), (1 1, 2 1, 2 2, 1 2, 1 1))` +- `MULTIPOINT (0 0, 1 2)` +- `MULTILINESTRING ((0 0, 1 1, 1 2), (2 3, 3 2, 5 4))` +- `MULTIPOLYGON (((0 0, 4 0, 4 4, 0 4, 0 0), (1 1, 2 1, 2 2, 1 2, 1 1)), ((-1 -1, -1 -2, -2 -2, -2 -1, -1 -1)))` +- `GEOMETRYCOLLECTION (POINT(2 3), LINESTRING (2 3, 3 4))` + +Use {func}`ST_GeometryFromText` and {func}`ST_GeomFromBinary` functions to create geometry +objects from WKT or WKB. + +The `SphericalGeography` type provides native support for spatial features represented on +*geographic* coordinates (sometimes called *geodetic* coordinates, or *lat/lon*, or *lon/lat*). +Geographic coordinates are spherical coordinates expressed in angular units (degrees). + +The basis for the `Geometry` type is a plane. The shortest path between two points on the plane is a +straight line. That means calculations on geometries (areas, distances, lengths, intersections, etc) +can be calculated using cartesian mathematics and straight line vectors. + +The basis for the `SphericalGeography` type is a sphere. The shortest path between two points on the +sphere is a great circle arc. That means that calculations on geographies (areas, distances, +lengths, intersections, etc) must be calculated on the sphere, using more complicated mathematics. +More accurate measurements that take the actual spheroidal shape of the world into account are not +supported. + +Values returned by the measurement functions {func}`ST_Distance` and {func}`ST_Length` are in the unit of meters; +values returned by {func}`ST_Area` are in square meters. + +Use {func}`to_spherical_geography()` function to convert a geometry object to geography object. + +For example, `ST_Distance(ST_Point(-71.0882, 42.3607), ST_Point(-74.1197, 40.6976))` returns +`3.4577` in the unit of the passed-in values on the euclidean plane, while +`ST_Distance(to_spherical_geography(ST_Point(-71.0882, 42.3607)), to_spherical_geography(ST_Point(-74.1197, 40.6976)))` +returns `312822.179` in meters. + +## Constructors + +:::{function} ST_AsBinary(Geometry) -> varbinary +Returns the WKB representation of the geometry. +::: + +:::{function} ST_AsText(Geometry) -> varchar +Returns the WKT representation of the geometry. For empty geometries, +`ST_AsText(ST_LineFromText('LINESTRING EMPTY'))` will produce `'MULTILINESTRING EMPTY'` +and `ST_AsText(ST_Polygon('POLYGON EMPTY'))` will produce `'MULTIPOLYGON EMPTY'`. +::: + +:::{function} ST_GeometryFromText(varchar) -> Geometry +Returns a geometry type object from WKT representation. +::: + +:::{function} ST_GeomFromBinary(varbinary) -> Geometry +Returns a geometry type object from WKB representation. +::: + +:::{function} geometry_from_hadoop_shape(varbinary) -> Geometry +Returns a geometry type object from Spatial Framework for Hadoop representation. +::: + +:::{function} ST_LineFromText(varchar) -> LineString +Returns a geometry type linestring object from WKT representation. +::: + +:::{function} ST_LineString(array(Point)) -> LineString +Returns a LineString formed from an array of points. If there are fewer than +two non-empty points in the input array, an empty LineString will be returned. +Array elements must not be `NULL` or the same as the previous element. +The returned geometry may not be simple, e.g. may self-intersect or may contain +duplicate vertexes depending on the input. +::: + +:::{function} ST_MultiPoint(array(Point)) -> MultiPoint +Returns a MultiPoint geometry object formed from the specified points. Returns `NULL` if input array is empty. +Array elements must not be `NULL` or empty. +The returned geometry may not be simple and may contain duplicate points if input array has duplicates. +::: + +:::{function} ST_Point(double, double) -> Point +Returns a geometry type point object with the given coordinate values. +::: + +:::{function} ST_Polygon(varchar) -> Polygon +Returns a geometry type polygon object from WKT representation. +::: + +:::{function} to_spherical_geography(Geometry) -> SphericalGeography +Converts a Geometry object to a SphericalGeography object on the sphere of the Earth's radius. This +function is only applicable to `POINT`, `MULTIPOINT`, `LINESTRING`, `MULTILINESTRING`, +`POLYGON`, `MULTIPOLYGON` geometries defined in 2D space, or `GEOMETRYCOLLECTION` of such +geometries. For each point of the input geometry, it verifies that `point.x` is within +`[-180.0, 180.0]` and `point.y` is within `[-90.0, 90.0]`, and uses them as (longitude, latitude) +degrees to construct the shape of the `SphericalGeography` result. +::: + +:::{function} to_geometry(SphericalGeography) -> Geometry +Converts a SphericalGeography object to a Geometry object. +::: + +## Relationship tests + +:::{function} ST_Contains(Geometry, Geometry) -> boolean +Returns `true` if and only if no points of the second geometry lie in the exterior +of the first geometry, and at least one point of the interior of the first geometry +lies in the interior of the second geometry. +::: + +:::{function} ST_Crosses(Geometry, Geometry) -> boolean +Returns `true` if the supplied geometries have some, but not all, interior points in common. +::: + +:::{function} ST_Disjoint(Geometry, Geometry) -> boolean +Returns `true` if the give geometries do not *spatially intersect* -- +if they do not share any space together. +::: + +:::{function} ST_Equals(Geometry, Geometry) -> boolean +Returns `true` if the given geometries represent the same geometry. +::: + +:::{function} ST_Intersects(Geometry, Geometry) -> boolean +Returns `true` if the given geometries spatially intersect in two dimensions +(share any portion of space) and `false` if they do not (they are disjoint). +::: + +:::{function} ST_Overlaps(Geometry, Geometry) -> boolean +Returns `true` if the given geometries share space, are of the same dimension, +but are not completely contained by each other. +::: + +:::{function} ST_Relate(Geometry, Geometry) -> boolean +Returns `true` if first geometry is spatially related to second geometry. +::: + +:::{function} ST_Touches(Geometry, Geometry) -> boolean +Returns `true` if the given geometries have at least one point in common, +but their interiors do not intersect. +::: + +:::{function} ST_Within(Geometry, Geometry) -> boolean +Returns `true` if first geometry is completely inside second geometry. +::: + +## Operations + +:::{function} geometry_nearest_points(Geometry, Geometry) -> row(Point, Point) +Returns the points on each geometry nearest the other. If either geometry +is empty, return `NULL`. Otherwise, return a row of two Points that have +the minimum distance of any two points on the geometries. The first Point +will be from the first Geometry argument, the second from the second Geometry +argument. If there are multiple pairs with the minimum distance, one pair +is chosen arbitrarily. +::: + +:::{function} geometry_union(array(Geometry)) -> Geometry +Returns a geometry that represents the point set union of the input geometries. Performance +of this function, in conjunction with {func}`array_agg` to first aggregate the input geometries, +may be better than {func}`geometry_union_agg`, at the expense of higher memory utilization. +::: + +:::{function} ST_Boundary(Geometry) -> Geometry +Returns the closure of the combinatorial boundary of this geometry. +::: + +:::{function} ST_Buffer(Geometry, distance) -> Geometry +Returns the geometry that represents all points whose distance from the specified geometry +is less than or equal to the specified distance. +::: + +:::{function} ST_Difference(Geometry, Geometry) -> Geometry +Returns the geometry value that represents the point set difference of the given geometries. +::: + +:::{function} ST_Envelope(Geometry) -> Geometry +Returns the bounding rectangular polygon of a geometry. +::: + +:::{function} ST_EnvelopeAsPts(Geometry) -> array(Geometry) +Returns an array of two points: the lower left and upper right corners of the bounding +rectangular polygon of a geometry. Returns `NULL` if input geometry is empty. +::: + +:::{function} ST_ExteriorRing(Geometry) -> Geometry +Returns a line string representing the exterior ring of the input polygon. +::: + +:::{function} ST_Intersection(Geometry, Geometry) -> Geometry +Returns the geometry value that represents the point set intersection of two geometries. +::: + +:::{function} ST_SymDifference(Geometry, Geometry) -> Geometry +Returns the geometry value that represents the point set symmetric difference of two geometries. +::: + +:::{function} ST_Union(Geometry, Geometry) -> Geometry +Returns a geometry that represents the point set union of the input geometries. + +See also: {func}`geometry_union`, {func}`geometry_union_agg` +::: + +## Accessors + +:::{function} ST_Area(Geometry) -> double +Returns the 2D Euclidean area of a geometry. + +For Point and LineString types, returns 0.0. +For GeometryCollection types, returns the sum of the areas of the individual +geometries. +::: + +:::{function} ST_Area(SphericalGeography) -> double +:noindex: true + +Returns the area of a polygon or multi-polygon in square meters using a spherical model for Earth. +::: + +:::{function} ST_Centroid(Geometry) -> Geometry +Returns the point value that is the mathematical centroid of a geometry. +::: + +:::{function} ST_ConvexHull(Geometry) -> Geometry +Returns the minimum convex geometry that encloses all input geometries. +::: + +:::{function} ST_CoordDim(Geometry) -> bigint +Returns the coordinate dimension of the geometry. +::: + +:::{function} ST_Dimension(Geometry) -> bigint +Returns the inherent dimension of this geometry object, which must be +less than or equal to the coordinate dimension. +::: + +:::{function} ST_Distance(Geometry, Geometry) -> double +:noindex: true + +Returns the 2-dimensional cartesian minimum distance (based on spatial ref) +between two geometries in projected units. +::: + +:::{function} ST_Distance(SphericalGeography, SphericalGeography) -> double +Returns the great-circle distance in meters between two SphericalGeography points. +::: + +:::{function} ST_GeometryN(Geometry, index) -> Geometry +Returns the geometry element at a given index (indices start at 1). +If the geometry is a collection of geometries (e.g., GEOMETRYCOLLECTION or MULTI\*), +returns the geometry at a given index. +If the given index is less than 1 or greater than the total number of elements in the collection, +returns `NULL`. +Use {func}`ST_NumGeometries` to find out the total number of elements. +Singular geometries (e.g., POINT, LINESTRING, POLYGON), are treated as collections of one element. +Empty geometries are treated as empty collections. +::: + +:::{function} ST_InteriorRingN(Geometry, index) -> Geometry +Returns the interior ring element at the specified index (indices start at 1). If +the given index is less than 1 or greater than the total number of interior rings +in the input geometry, returns `NULL`. The input geometry must be a polygon. +Use {func}`ST_NumInteriorRing` to find out the total number of elements. +::: + +:::{function} ST_GeometryType(Geometry) -> varchar +Returns the type of the geometry. +::: + +:::{function} ST_IsClosed(Geometry) -> boolean +Returns `true` if the linestring's start and end points are coincident. +::: + +:::{function} ST_IsEmpty(Geometry) -> boolean +Returns `true` if this Geometry is an empty geometrycollection, polygon, point etc. +::: + +:::{function} ST_IsSimple(Geometry) -> boolean +Returns `true` if this Geometry has no anomalous geometric points, such as self intersection or self tangency. +::: + +:::{function} ST_IsRing(Geometry) -> boolean +Returns `true` if and only if the line is closed and simple. +::: + +:::{function} ST_IsValid(Geometry) -> boolean +Returns `true` if and only if the input geometry is well formed. +Use {func}`geometry_invalid_reason` to determine why the geometry is not well formed. +::: + +:::{function} ST_Length(Geometry) -> double +Returns the length of a linestring or multi-linestring using Euclidean measurement on a +two dimensional plane (based on spatial ref) in projected units. +::: + +:::{function} ST_Length(SphericalGeography) -> double +:noindex: true + +Returns the length of a linestring or multi-linestring on a spherical model of the Earth. +This is equivalent to the sum of great-circle distances between adjacent points on the linestring. +::: + +:::{function} ST_PointN(LineString, index) -> Point +Returns the vertex of a linestring at a given index (indices start at 1). +If the given index is less than 1 or greater than the total number of elements in the collection, +returns `NULL`. +Use {func}`ST_NumPoints` to find out the total number of elements. +::: + +:::{function} ST_Points(Geometry) -> array(Point) +Returns an array of points in a linestring. +::: + +:::{function} ST_XMax(Geometry) -> double +Returns X maxima of a bounding box of a geometry. +::: + +:::{function} ST_YMax(Geometry) -> double +Returns Y maxima of a bounding box of a geometry. +::: + +:::{function} ST_XMin(Geometry) -> double +Returns X minima of a bounding box of a geometry. +::: + +:::{function} ST_YMin(Geometry) -> double +Returns Y minima of a bounding box of a geometry. +::: + +:::{function} ST_StartPoint(Geometry) -> point +Returns the first point of a LineString geometry as a Point. +This is a shortcut for `ST_PointN(geometry, 1)`. +::: + +:::{function} simplify_geometry(Geometry, double) -> Geometry +Returns a "simplified" version of the input geometry using the Douglas-Peucker algorithm. +Will avoid creating derived geometries (polygons in particular) that are invalid. +::: + +:::{function} ST_EndPoint(Geometry) -> point +Returns the last point of a LineString geometry as a Point. +This is a shortcut for `ST_PointN(geometry, ST_NumPoints(geometry))`. +::: + +:::{function} ST_X(Point) -> double +Returns the X coordinate of the point. +::: + +:::{function} ST_Y(Point) -> double +Returns the Y coordinate of the point. +::: + +:::{function} ST_InteriorRings(Geometry) -> array(Geometry) +Returns an array of all interior rings found in the input geometry, or an empty +array if the polygon has no interior rings. Returns `NULL` if the input geometry +is empty. The input geometry must be a polygon. +::: + +:::{function} ST_NumGeometries(Geometry) -> bigint +Returns the number of geometries in the collection. +If the geometry is a collection of geometries (e.g., GEOMETRYCOLLECTION or MULTI\*), +returns the number of geometries, +for single geometries returns 1, +for empty geometries returns 0. +::: + +:::{function} ST_Geometries(Geometry) -> array(Geometry) +Returns an array of geometries in the specified collection. Returns a one-element array +if the input geometry is not a multi-geometry. Returns `NULL` if input geometry is empty. +::: + +:::{function} ST_NumPoints(Geometry) -> bigint +Returns the number of points in a geometry. This is an extension to the SQL/MM +`ST_NumPoints` function which only applies to point and linestring. +::: + +:::{function} ST_NumInteriorRing(Geometry) -> bigint +Returns the cardinality of the collection of interior rings of a polygon. +::: + +:::{function} line_interpolate_point(LineString, double) -> Geometry +Returns a Point interpolated along a LineString at the fraction given. The fraction +must be between 0 and 1, inclusive. +::: + +:::{function} line_interpolate_points(LineString, double, repeated) -> array(Geometry) +Returns an array of Points interpolated along a LineString. The fraction must be +between 0 and 1, inclusive. +::: + +:::{function} line_locate_point(LineString, Point) -> double +Returns a float between 0 and 1 representing the location of the closest point on +the LineString to the given Point, as a fraction of total 2d line length. + +Returns `NULL` if a LineString or a Point is empty or `NULL`. +::: + +:::{function} geometry_invalid_reason(Geometry) -> varchar +Returns the reason for why the input geometry is not valid. +Returns `NULL` if the input is valid. +::: + +:::{function} great_circle_distance(latitude1, longitude1, latitude2, longitude2) -> double +Returns the great-circle distance between two points on Earth's surface in kilometers. +::: + +:::{function} to_geojson_geometry(SphericalGeography) -> varchar +Returns the GeoJSON encoded defined by the input spherical geography. +::: + +:::{function} from_geojson_geometry(varchar) -> SphericalGeography +Returns the spherical geography type object from the GeoJSON representation stripping non geometry key/values. +Feature and FeatureCollection are not supported. +::: + +## Aggregations + +:::{function} convex_hull_agg(Geometry) -> Geometry +Returns the minimum convex geometry that encloses all input geometries. +::: + +:::{function} geometry_union_agg(Geometry) -> Geometry +Returns a geometry that represents the point set union of all input geometries. +::: + +## Bing tiles + +These functions convert between geometries and +[Bing tiles](https://msdn.microsoft.com/library/bb259689.aspx). + +:::{function} bing_tile(x, y, zoom_level) -> BingTile +Creates a Bing tile object from XY coordinates and a zoom level. +Zoom levels from 1 to 23 are supported. +::: + +:::{function} bing_tile(quadKey) -> BingTile +:noindex: true + +Creates a Bing tile object from a quadkey. +::: + +:::{function} bing_tile_at(latitude, longitude, zoom_level) -> BingTile +Returns a Bing tile at a given zoom level containing a point at a given latitude +and longitude. Latitude must be within `[-85.05112878, 85.05112878]` range. +Longitude must be within `[-180, 180]` range. Zoom levels from 1 to 23 are supported. +::: + +:::{function} bing_tiles_around(latitude, longitude, zoom_level) -> array(BingTile) +Returns a collection of Bing tiles that surround the point specified +by the latitude and longitude arguments at a given zoom level. +::: + +:::{function} bing_tiles_around(latitude, longitude, zoom_level, radius_in_km) -> array(BingTile) +:noindex: true + +Returns a minimum set of Bing tiles at specified zoom level that cover a circle of specified +radius in km around a specified (latitude, longitude) point. +::: + +:::{function} bing_tile_coordinates(tile) -> row +Returns the XY coordinates of a given Bing tile. +::: + +:::{function} bing_tile_polygon(tile) -> Geometry +Returns the polygon representation of a given Bing tile. +::: + +:::{function} bing_tile_quadkey(tile) -> varchar +Returns the quadkey of a given Bing tile. +::: + +:::{function} bing_tile_zoom_level(tile) -> tinyint +Returns the zoom level of a given Bing tile. +::: + +:::{function} geometry_to_bing_tiles(geometry, zoom_level) -> array(BingTile) +Returns the minimum set of Bing tiles that fully covers a given geometry at +a given zoom level. Zoom levels from 1 to 23 are supported. +::: + +## Encoded polylines + +These functions convert between geometries and +[encoded polylines](https://developers.google.com/maps/documentation/utilities/polylinealgorithm). + +:::{function} to_encoded_polyline(Geometry) -> varchar +Encodes a linestring or multipoint to a polyline. +::: + +:::{function} from_encoded_polyline(varchar) -> Geometry +Decodes a polyline to a linestring. +::: diff --git a/430/_sources/functions/hyperloglog.md.txt b/430/_sources/functions/hyperloglog.md.txt new file mode 100644 index 000000000..9a6a97799 --- /dev/null +++ b/430/_sources/functions/hyperloglog.md.txt @@ -0,0 +1,74 @@ +# HyperLogLog functions + +Trino implements the {func}`approx_distinct` function using the +[HyperLogLog](https://wikipedia.org/wiki/HyperLogLog) data structure. + +## Data structures + +Trino implements HyperLogLog data sketches as a set of 32-bit buckets which +store a *maximum hash*. They can be stored sparsely (as a map from bucket ID +to bucket), or densely (as a contiguous memory block). The HyperLogLog data +structure starts as the sparse representation, switching to dense when it is +more efficient. The P4HyperLogLog structure is initialized densely and +remains dense for its lifetime. + +{ref}`hyperloglog-type` implicitly casts to {ref}`p4hyperloglog-type`, +while one can explicitly cast `HyperLogLog` to `P4HyperLogLog`: + +``` +cast(hll AS P4HyperLogLog) +``` + +## Serialization + +Data sketches can be serialized to and deserialized from `varbinary`. This +allows them to be stored for later use. Combined with the ability to merge +multiple sketches, this allows one to calculate {func}`approx_distinct` of the +elements of a partition of a query, then for the entirety of a query with very +little cost. + +For example, calculating the `HyperLogLog` for daily unique users will allow +weekly or monthly unique users to be calculated incrementally by combining the +dailies. This is similar to computing weekly revenue by summing daily revenue. +Uses of {func}`approx_distinct` with `GROUPING SETS` can be converted to use +`HyperLogLog`. Examples: + +``` +CREATE TABLE visit_summaries ( + visit_date date, + hll varbinary +); + +INSERT INTO visit_summaries +SELECT visit_date, cast(approx_set(user_id) AS varbinary) +FROM user_visits +GROUP BY visit_date; + +SELECT cardinality(merge(cast(hll AS HyperLogLog))) AS weekly_unique_users +FROM visit_summaries +WHERE visit_date >= current_date - interval '7' day; +``` + +## Functions + +:::{function} approx_set(x) -> HyperLogLog +Returns the `HyperLogLog` sketch of the input data set of `x`. This +data sketch underlies {func}`approx_distinct` and can be stored and +used later by calling `cardinality()`. +::: + +:::{function} cardinality(hll) -> bigint +:noindex: true + +This will perform {func}`approx_distinct` on the data summarized by the +`hll` HyperLogLog data sketch. +::: + +:::{function} empty_approx_set() -> HyperLogLog +Returns an empty `HyperLogLog`. +::: + +:::{function} merge(HyperLogLog) -> HyperLogLog +Returns the `HyperLogLog` of the aggregate union of the individual `hll` +HyperLogLog structures. +::: diff --git a/430/_sources/functions/ipaddress.md.txt b/430/_sources/functions/ipaddress.md.txt new file mode 100644 index 000000000..91fe20a98 --- /dev/null +++ b/430/_sources/functions/ipaddress.md.txt @@ -0,0 +1,17 @@ +# IP Address Functions + +(ip-address-contains)= + +:::{function} contains(network, address) -> boolean +:noindex: true + +Returns true if the `address` exists in the CIDR `network`: + +``` +SELECT contains('10.0.0.0/8', IPADDRESS '10.255.255.255'); -- true +SELECT contains('10.0.0.0/8', IPADDRESS '11.255.255.255'); -- false + +SELECT contains('2001:0db8:0:0:0:ff00:0042:8329/128', IPADDRESS '2001:0db8:0:0:0:ff00:0042:8329'); -- true +SELECT contains('2001:0db8:0:0:0:ff00:0042:8329/128', IPADDRESS '2001:0db8:0:0:0:ff00:0042:8328'); -- false +``` +::: diff --git a/430/_sources/functions/json.md.txt b/430/_sources/functions/json.md.txt new file mode 100644 index 000000000..249db0af1 --- /dev/null +++ b/430/_sources/functions/json.md.txt @@ -0,0 +1,1760 @@ +# JSON functions and operators + +The SQL standard describes functions and operators to process JSON data. They +allow you to access JSON data according to its structure, generate JSON data, +and store it persistently in SQL tables. + +Importantly, the SQL standard imposes that there is no dedicated data type to +represent JSON data in SQL. Instead, JSON data is represented as character or +binary strings. Although Trino supports `JSON` type, it is not used or +produced by the following functions. + +Trino supports three functions for querying JSON data: +{ref}`json_exists`, +{ref}`json_query`, and {ref}`json_value`. Each of them +is based on the same mechanism of exploring and processing JSON input using +JSON path. + +Trino also supports two functions for generating JSON data -- +{ref}`json_array`, and {ref}`json_object`. + +(json-path-language)= + +## JSON path language + +The JSON path language is a special language, used exclusively by certain SQL +operators to specify the query to perform on the JSON input. Although JSON path +expressions are embedded in SQL queries, their syntax significantly differs +from SQL. The semantics of predicates, operators, etc. in JSON path expressions +generally follow the semantics of SQL. The JSON path language is case-sensitive +for keywords and identifiers. + +(json-path-syntax-and-semantics)= + +### JSON path syntax and semantics + +JSON path expressions are recursive structures. Although the name "path" +suggests a linear sequence of operations going step by step deeper into the JSON +structure, a JSON path expression is in fact a tree. It can access the input +JSON item multiple times, in multiple ways, and combine the results. Moreover, +the result of a JSON path expression is not a single item, but an ordered +sequence of items. Each of the sub-expressions takes one or more input +sequences, and returns a sequence as the result. + +:::{note} +In the lax mode, most path operations first unnest all JSON arrays in the +input sequence. Any divergence from this rule is mentioned in the following +listing. Path modes are explained in {ref}`json-path-modes`. +::: + +The JSON path language features are divided into: literals, variables, +arithmetic binary expressions, arithmetic unary expressions, and a group of +operators collectively known as accessors. + +#### literals + +- numeric literals + + They include exact and approximate numbers, and are interpreted as if they + were SQL values. + +```text +-1, 1.2e3, NaN +``` + +- string literals + + They are enclosed in double quotes. + +```text +"Some text" +``` + +- boolean literals + +```text +true, false +``` + +- null literal + + It has the semantics of the JSON null, not of SQL null. See {ref}`json-comparison-rules`. + +```text +null +``` + +#### variables + +- context variable + + It refers to the currently processed input of the JSON + function. + +```text +$ +``` + +- named variable + + It refers to a named parameter by its name. + +```text +$param +``` + +- current item variable + + It is used inside the filter expression to refer to the currently processed + item from the input sequence. + +```text +@ +``` + +- last subscript variable + + It refers to the last index of the innermost enclosing array. Array indexes + in JSON path expressions are zero-based. + +```text +last +``` + +#### arithmetic binary expressions + +The JSON path language supports five arithmetic binary operators: + +```text + + + - + * + / + % +``` + +Both operands, `` and ``, are evaluated to sequences of +items. For arithmetic binary operators, each input sequence must contain a +single numeric item. The arithmetic operation is performed according to SQL +semantics, and it returns a sequence containing a single element with the +result. + +The operators follow the same precedence rules as in SQL arithmetic operations, +and parentheses can be used for grouping. + +#### arithmetic unary expressions + +```text ++ +- +``` + +The operand `` is evaluated to a sequence of items. Every item must be +a numeric value. The unary plus or minus is applied to every item in the +sequence, following SQL semantics, and the results form the returned sequence. + +#### member accessor + +The member accessor returns the value of the member with the specified key for +each JSON object in the input sequence. + +```text +.key +."key" +``` + +The condition when a JSON object does not have such a member is called a +structural error. In the lax mode, it is suppressed, and the faulty object is +excluded from the result. + +Let `` return a sequence of three JSON objects: + +```text +{"customer" : 100, "region" : "AFRICA"}, +{"region" : "ASIA"}, +{"customer" : 300, "region" : "AFRICA", "comment" : null} +``` + +the expression `.customer` succeeds in the first and the third object, +but the second object lacks the required member. In strict mode, path +evaluation fails. In lax mode, the second object is silently skipped, and the +resulting sequence is `100, 300`. + +All items in the input sequence must be JSON objects. + +:::{note} +Trino does not support JSON objects with duplicate keys. +::: + +#### wildcard member accessor + +Returns values from all key-value pairs for each JSON object in the input +sequence. All the partial results are concatenated into the returned sequence. + +```text +.* +``` + +Let `` return a sequence of three JSON objects: + +```text +{"customer" : 100, "region" : "AFRICA"}, +{"region" : "ASIA"}, +{"customer" : 300, "region" : "AFRICA", "comment" : null} +``` + +The results is: + +```text +100, "AFRICA", "ASIA", 300, "AFRICA", null +``` + +All items in the input sequence must be JSON objects. + +The order of values returned from a single JSON object is arbitrary. The +sub-sequences from all JSON objects are concatenated in the same order in which +the JSON objects appear in the input sequence. + +(json-descendant-member-accessor)= + +#### descendant member accessor + +Returns the values associated with the specified key in all JSON objects on all +levels of nesting in the input sequence. + +```text +..key +.."key" +``` + +The order of returned values is that of preorder depth first search. First, the +enclosing object is visited, and then all child nodes are visited. + +This method does not perform array unwrapping in the lax mode. The results +are the same in the lax and strict modes. The method traverses into JSON +arrays and JSON objects. Non-structural JSON items are skipped. + +Let `` be a sequence containing a JSON object: + +```text +{ + "id" : 1, + "notes" : [{"type" : 1, "comment" : "foo"}, {"type" : 2, "comment" : null}], + "comment" : ["bar", "baz"] +} +``` + +```text +..comment --> ["bar", "baz"], "foo", null +``` + +#### array accessor + +Returns the elements at the specified indexes for each JSON array in the input +sequence. Indexes are zero-based. + +```text +[ ] +``` + +The `` list contains one or more subscripts. Each subscript +specifies a single index or a range (ends inclusive): + +```text +[, to , ,...] +``` + +In lax mode, any non-array items resulting from the evaluation of the input +sequence are wrapped into single-element arrays. Note that this is an exception +to the rule of automatic array wrapping. + +Each array in the input sequence is processed in the following way: + +- The variable `last` is set to the last index of the array. +- All subscript indexes are computed in order of declaration. For a + singleton subscript ``, the result must be a singleton numeric item. + For a range subscript ` to `, two numeric items are expected. +- The specified array elements are added in order to the output sequence. + +Let `` return a sequence of three JSON arrays: + +```text +[0, 1, 2], ["a", "b", "c", "d"], [null, null] +``` + +The following expression returns a sequence containing the last element from +every array: + +```text +[last] --> 2, "d", null +``` + +The following expression returns the third and fourth element from every array: + +```text +[2 to 3] --> 2, "c", "d" +``` + +Note that the first array does not have the fourth element, and the last array +does not have the third or fourth element. Accessing non-existent elements is a +structural error. In strict mode, it causes the path expression to fail. In lax +mode, such errors are suppressed, and only the existing elements are returned. + +Another example of a structural error is an improper range specification such +as `5 to 3`. + +Note that the subscripts may overlap, and they do not need to follow the +element order. The order in the returned sequence follows the subscripts: + +```text +[1, 0, 0] --> 1, 0, 0, "b", "a", "a", null, null, null +``` + +#### wildcard array accessor + +Returns all elements of each JSON array in the input sequence. + +```text +[*] +``` + +In lax mode, any non-array items resulting from the evaluation of the input +sequence are wrapped into single-element arrays. Note that this is an exception +to the rule of automatic array wrapping. + +The output order follows the order of the original JSON arrays. Also, the order +of elements within the arrays is preserved. + +Let `` return a sequence of three JSON arrays: + +```text +[0, 1, 2], ["a", "b", "c", "d"], [null, null] +[*] --> 0, 1, 2, "a", "b", "c", "d", null, null +``` + +#### filter + +Retrieves the items from the input sequence which satisfy the predicate. + +```text +?( ) +``` + +JSON path predicates are syntactically similar to boolean expressions in SQL. +However, the semantics are different in many aspects: + +- They operate on sequences of items. +- They have their own error handling (they never fail). +- They behave different depending on the lax or strict mode. + +The predicate evaluates to `true`, `false`, or `unknown`. Note that some +predicate expressions involve nested JSON path expression. When evaluating the +nested path, the variable `@` refers to the currently examined item from the +input sequence. + +The following predicate expressions are supported: + +- Conjunction + +```text + && +``` + +- Disjunction + +```text + || +``` + +- Negation + +```text +! +``` + +- `exists` predicate + +```text +exists( ) +``` + +Returns `true` if the nested path evaluates to a non-empty sequence, and +`false` when the nested path evaluates to an empty sequence. If the path +evaluation throws an error, returns `unknown`. + +- `starts with` predicate + +```text + starts with "Some text" + starts with $variable +``` + +The nested `` must evaluate to a sequence of textual items, and the +other operand must evaluate to a single textual item. If evaluating of either +operand throws an error, the result is `unknown`. All items from the sequence +are checked for starting with the right operand. The result is `true` if a +match is found, otherwise `false`. However, if any of the comparisons throws +an error, the result in the strict mode is `unknown`. The result in the lax +mode depends on whether the match or the error was found first. + +- `is unknown` predicate + +```text +( ) is unknown +``` + +Returns `true` if the nested predicate evaluates to `unknown`, and +`false` otherwise. + +- Comparisons + +```text + == + <> + != + < + > + <= + >= +``` + +Both operands of a comparison evaluate to sequences of items. If either +evaluation throws an error, the result is `unknown`. Items from the left and +right sequence are then compared pairwise. Similarly to the `starts with` +predicate, the result is `true` if any of the comparisons returns `true`, +otherwise `false`. However, if any of the comparisons throws an error, for +example because the compared types are not compatible, the result in the strict +mode is `unknown`. The result in the lax mode depends on whether the `true` +comparison or the error was found first. + +(json-comparison-rules)= + +##### Comparison rules + +Null values in the context of comparison behave different than SQL null: + +- null == null --> `true` +- null != null, null \< null, ... --> `false` +- null compared to a scalar value --> `false` +- null compared to a JSON array or a JSON object --> `false` + +When comparing two scalar values, `true` or `false` is returned if the +comparison is successfully performed. The semantics of the comparison is the +same as in SQL. In case of an error, e.g. comparing text and number, +`unknown` is returned. + +Comparing a scalar value with a JSON array or a JSON object, and comparing JSON +arrays/objects is an error, so `unknown` is returned. + +##### Examples of filter + +Let `` return a sequence of three JSON objects: + +```text +{"customer" : 100, "region" : "AFRICA"}, +{"region" : "ASIA"}, +{"customer" : 300, "region" : "AFRICA", "comment" : null} +``` + +```text +?(@.region != "ASIA") --> {"customer" : 100, "region" : "AFRICA"}, + {"customer" : 300, "region" : "AFRICA", "comment" : null} +?(!exists(@.customer)) --> {"region" : "ASIA"} +``` + +The following accessors are collectively referred to as **item methods**. + +#### double() + +Converts numeric or text values into double values. + +```text +.double() +``` + +Let `` return a sequence `-1, 23e4, "5.6"`: + +```text +.double() --> -1e0, 23e4, 5.6e0 +``` + +#### ceiling(), floor(), and abs() + +Gets the ceiling, the floor or the absolute value for every numeric item in the +sequence. The semantics of the operations is the same as in SQL. + +Let `` return a sequence `-1.5, -1, 1.3`: + +```text +.ceiling() --> -1.0, -1, 2.0 +.floor() --> -2.0, -1, 1.0 +.abs() --> 1.5, 1, 1.3 +``` + +#### keyvalue() + +Returns a collection of JSON objects including one object per every member of +the original object for every JSON object in the sequence. + +```text +.keyvalue() +``` + +The returned objects have three members: + +- "name", which is the original key, +- "value", which is the original bound value, +- "id", which is the unique number, specific to an input object. + +Let `` be a sequence of three JSON objects: + +```text +{"customer" : 100, "region" : "AFRICA"}, +{"region" : "ASIA"}, +{"customer" : 300, "region" : "AFRICA", "comment" : null} +``` + +```text +.keyvalue() --> {"name" : "customer", "value" : 100, "id" : 0}, + {"name" : "region", "value" : "AFRICA", "id" : 0}, + {"name" : "region", "value" : "ASIA", "id" : 1}, + {"name" : "customer", "value" : 300, "id" : 2}, + {"name" : "region", "value" : "AFRICA", "id" : 2}, + {"name" : "comment", "value" : null, "id" : 2} +``` + +It is required that all items in the input sequence are JSON objects. + +The order of the returned values follows the order of the original JSON +objects. However, within objects, the order of returned entries is arbitrary. + +#### type() + +Returns a textual value containing the type name for every item in the +sequence. + +```text +.type() +``` + +This method does not perform array unwrapping in the lax mode. + +The returned values are: + +- `"null"` for JSON null, +- `"number"` for a numeric item, +- `"string"` for a textual item, +- `"boolean"` for a boolean item, +- `"date"` for an item of type date, +- `"time without time zone"` for an item of type time, +- `"time with time zone"` for an item of type time with time zone, +- `"timestamp without time zone"` for an item of type timestamp, +- `"timestamp with time zone"` for an item of type timestamp with time zone, +- `"array"` for JSON array, +- `"object"` for JSON object, + +#### size() + +Returns a numeric value containing the size for every JSON array in the +sequence. + +```text +.size() +``` + +This method does not perform array unwrapping in the lax mode. Instead, all +non-array items are wrapped in singleton JSON arrays, so their size is `1`. + +It is required that all items in the input sequence are JSON arrays. + +Let `` return a sequence of three JSON arrays: + +```text +[0, 1, 2], ["a", "b", "c", "d"], [null, null] +.size() --> 3, 4, 2 +``` + +### Limitations + +The SQL standard describes the `datetime()` JSON path item method and the +`like_regex()` JSON path predicate. Trino does not support them. + +(json-path-modes)= + +### JSON path modes + +The JSON path expression can be evaluated in two modes: strict and lax. In the +strict mode, it is required that the input JSON data strictly fits the schema +required by the path expression. In the lax mode, the input JSON data can +diverge from the expected schema. + +The following table shows the differences between the two modes. + +:::{list-table} +:widths: 40 20 40 +:header-rows: 1 + +* - Condition + - strict mode + - lax mode +* - Performing an operation which requires a non-array on an array, e.g.: + + `$.key` requires a JSON object + + `$.floor()` requires a numeric value + - ERROR + - The array is automatically unnested, and the operation is performed on + each array element. +* - Performing an operation which requires an array on an non-array, e.g.: + + `$[0]`, `$[*]`, `$.size()` + - ERROR + - The non-array item is automatically wrapped in a singleton array, and + the operation is performed on the array. +* - A structural error: accessing a non-existent element of an array or a + non-existent member of a JSON object, e.g.: + + `$[-1]` (array index out of bounds) + + `$.key`, where the input JSON object does not have a member `key` + - ERROR + - The error is suppressed, and the operation results in an empty sequence. +::: + +#### Examples of the lax mode behavior + +Let `` return a sequence of three items, a JSON array, a JSON object, +and a scalar numeric value: + +```text +[1, "a", null], {"key1" : 1.0, "key2" : true}, -2e3 +``` + +The following example shows the wildcard array accessor in the lax mode. The +JSON array returns all its elements, while the JSON object and the number are +wrapped in singleton arrays and then unnested, so effectively they appear +unchanged in the output sequence: + +```text +[*] --> 1, "a", null, {"key1" : 1.0, "key2" : true}, -2e3 +``` + +When calling the `size()` method, the JSON object and the number are also +wrapped in singleton arrays: + +```text +.size() --> 3, 1, 1 +``` + +In some cases, the lax mode cannot prevent failure. In the following example, +even though the JSON array is unwrapped prior to calling the `floor()` +method, the item `"a"` causes type mismatch. + +```text +.floor() --> ERROR +``` + +(json-exists)= + +## json_exists + +The `json_exists` function determines whether a JSON value satisfies a JSON +path specification. + +```text +JSON_EXISTS( + json_input [ FORMAT JSON [ ENCODING { UTF8 | UTF16 | UTF32 } ] ], + json_path + [ PASSING json_argument [, ...] ] + [ { TRUE | FALSE | UNKNOWN | ERROR } ON ERROR ] + ) +``` + +The `json_path` is evaluated using the `json_input` as the context variable +(`$`), and the passed arguments as the named variables (`$variable_name`). +The returned value is `true` if the path returns a non-empty sequence, and +`false` if the path returns an empty sequence. If an error occurs, the +returned value depends on the `ON ERROR` clause. The default value returned +`ON ERROR` is `FALSE`. The `ON ERROR` clause is applied for the following +kinds of errors: + +- Input conversion errors, such as malformed JSON +- JSON path evaluation errors, e.g. division by zero + +`json_input` is a character string or a binary string. It should contain +a single JSON item. For a binary string, you can specify encoding. + +`json_path` is a string literal, containing the path mode specification, and +the path expression, following the syntax rules described in +{ref}`json-path-syntax-and-semantics`. + +```text +'strict ($.price + $.tax)?(@ > 99.9)' +'lax $[0 to 1].floor()?(@ > 10)' +``` + +In the `PASSING` clause you can pass arbitrary expressions to be used by the +path expression. + +```text +PASSING orders.totalprice AS O_PRICE, + orders.tax % 10 AS O_TAX +``` + +The passed parameters can be referenced in the path expression by named +variables, prefixed with `$`. + +```text +'lax $?(@.price > $O_PRICE || @.tax > $O_TAX)' +``` + +Additionally to SQL values, you can pass JSON values, specifying the format and +optional encoding: + +```text +PASSING orders.json_desc FORMAT JSON AS o_desc, + orders.binary_record FORMAT JSON ENCODING UTF16 AS o_rec +``` + +Note that the JSON path language is case-sensitive, while the unquoted SQL +identifiers are upper-cased. Therefore, it is recommended to use quoted +identifiers in the `PASSING` clause: + +```text +'lax $.$KeyName' PASSING nation.name AS KeyName --> ERROR; no passed value found +'lax $.$KeyName' PASSING nation.name AS "KeyName" --> correct +``` + +### Examples + +Let `customers` be a table containing two columns: `id:bigint`, +`description:varchar`. + +| id | description | +| --- | ----------------------------------------------------- | +| 101 | '{"comment" : "nice", "children" : \[10, 13, 16\]}' | +| 102 | '{"comment" : "problematic", "children" : \[8, 11\]}' | +| 103 | '{"comment" : "knows best", "children" : \[2\]}' | + +The following query checks which customers have children above the age of 10: + +```text +SELECT + id, + json_exists( + description, + 'lax $.children[*]?(@ > 10)' + ) AS children_above_ten +FROM customers +``` + +| id | children_above_ten | +| --- | ------------------ | +| 101 | true | +| 102 | true | +| 103 | false | + +In the following query, the path mode is strict. We check the third child for +each customer. This should cause a structural error for the customers who do +not have three or more children. This error is handled according to the `ON +ERROR` clause. + +```text +SELECT + id, + json_exists( + description, + 'strict $.children[2]?(@ > 10)' + UNKNOWN ON ERROR + ) AS child_3_above_ten +FROM customers +``` + +| id | child_3_above_ten | +| --- | ----------------- | +| 101 | true | +| 102 | NULL | +| 103 | NULL | + +(json-query)= + +## json_query + +The `json_query` function extracts a JSON value from a JSON value. + +```text +JSON_QUERY( + json_input [ FORMAT JSON [ ENCODING { UTF8 | UTF16 | UTF32 } ] ], + json_path + [ PASSING json_argument [, ...] ] + [ RETURNING type [ FORMAT JSON [ ENCODING { UTF8 | UTF16 | UTF32 } ] ] ] + [ WITHOUT [ ARRAY ] WRAPPER | + WITH [ { CONDITIONAL | UNCONDITIONAL } ] [ ARRAY ] WRAPPER ] + [ { KEEP | OMIT } QUOTES [ ON SCALAR STRING ] ] + [ { ERROR | NULL | EMPTY ARRAY | EMPTY OBJECT } ON EMPTY ] + [ { ERROR | NULL | EMPTY ARRAY | EMPTY OBJECT } ON ERROR ] + ) +``` + +The `json_path` is evaluated using the `json_input` as the context variable +(`$`), and the passed arguments as the named variables (`$variable_name`). + +The returned value is a JSON item returned by the path. By default, it is +represented as a character string (`varchar`). In the `RETURNING` clause, +you can specify other character string type or `varbinary`. With +`varbinary`, you can also specify the desired encoding. + +`json_input` is a character string or a binary string. It should contain +a single JSON item. For a binary string, you can specify encoding. + +`json_path` is a string literal, containing the path mode specification, and +the path expression, following the syntax rules described in +{ref}`json-path-syntax-and-semantics`. + +```text +'strict $.keyvalue()?(@.name == $cust_id)' +'lax $[5 to last]' +``` + +In the `PASSING` clause you can pass arbitrary expressions to be used by the +path expression. + +```text +PASSING orders.custkey AS CUST_ID +``` + +The passed parameters can be referenced in the path expression by named +variables, prefixed with `$`. + +```text +'strict $.keyvalue()?(@.value == $CUST_ID)' +``` + +Additionally to SQL values, you can pass JSON values, specifying the format and +optional encoding: + +```text +PASSING orders.json_desc FORMAT JSON AS o_desc, + orders.binary_record FORMAT JSON ENCODING UTF16 AS o_rec +``` + +Note that the JSON path language is case-sensitive, while the unquoted SQL +identifiers are upper-cased. Therefore, it is recommended to use quoted +identifiers in the `PASSING` clause: + +```text +'lax $.$KeyName' PASSING nation.name AS KeyName --> ERROR; no passed value found +'lax $.$KeyName' PASSING nation.name AS "KeyName" --> correct +``` + +The `ARRAY WRAPPER` clause lets you modify the output by wrapping the results +in a JSON array. `WITHOUT ARRAY WRAPPER` is the default option. `WITH +CONDITIONAL ARRAY WRAPPER` wraps every result which is not a singleton JSON +array or JSON object. `WITH UNCONDITIONAL ARRAY WRAPPER` wraps every result. + +The `QUOTES` clause lets you modify the result for a scalar string by +removing the double quotes being part of the JSON string representation. + +### Examples + +Let `customers` be a table containing two columns: `id:bigint`, +`description:varchar`. + +| id | description | +| --- | ----------------------------------------------------- | +| 101 | '{"comment" : "nice", "children" : \[10, 13, 16\]}' | +| 102 | '{"comment" : "problematic", "children" : \[8, 11\]}' | +| 103 | '{"comment" : "knows best", "children" : \[2\]}' | + +The following query gets the `children` array for each customer: + +```text +SELECT + id, + json_query( + description, + 'lax $.children' + ) AS children +FROM customers +``` + +| id | children | +| --- | -------------- | +| 101 | '\[10,13,16\]' | +| 102 | '\[8,11\]' | +| 103 | '\[2\]' | + +The following query gets the collection of children for each customer. +Note that the `json_query` function can only output a single JSON item. If +you don't use array wrapper, you get an error for every customer with multiple +children. The error is handled according to the `ON ERROR` clause. + +```text +SELECT + id, + json_query( + description, + 'lax $.children[*]' + WITHOUT ARRAY WRAPPER + NULL ON ERROR + ) AS children +FROM customers +``` + +| id | children | +| --- | -------- | +| 101 | NULL | +| 102 | NULL | +| 103 | '2' | + +The following query gets the last child for each customer, wrapped in a JSON +array: + +```text +SELECT + id, + json_query( + description, + 'lax $.children[last]' + WITH ARRAY WRAPPER + ) AS last_child +FROM customers +``` + +| id | last_child | +| --- | ---------- | +| 101 | '\[16\]' | +| 102 | '\[11\]' | +| 103 | '\[2\]' | + +The following query gets all children above the age of 12 for each customer, +wrapped in a JSON array. The second and the third customer don't have children +of this age. Such case is handled according to the `ON EMPTY` clause. The +default value returned `ON EMPTY` is `NULL`. In the following example, +`EMPTY ARRAY ON EMPTY` is specified. + +```text +SELECT + id, + json_query( + description, + 'strict $.children[*]?(@ > 12)' + WITH ARRAY WRAPPER + EMPTY ARRAY ON EMPTY + ) AS children +FROM customers +``` + +| id | children | +| --- | ----------- | +| 101 | '\[13,16\]' | +| 102 | '\[\]' | +| 103 | '\[\]' | + +The following query shows the result of the `QUOTES` clause. Note that `KEEP +QUOTES` is the default. + +```text +SELECT + id, + json_query(description, 'strict $.comment' KEEP QUOTES) AS quoted_comment, + json_query(description, 'strict $.comment' OMIT QUOTES) AS unquoted_comment +FROM customers +``` + +| id | quoted_comment | unquoted_comment | +| --- | --------------- | ---------------- | +| 101 | '"nice"' | 'nice' | +| 102 | '"problematic"' | 'problematic' | +| 103 | '"knows best"' | 'knows best' | + +If an error occurs, the returned value depends on the `ON ERROR` clause. The +default value returned `ON ERROR` is `NULL`. One example of error is +multiple items returned by the path. Other errors caught and handled according +to the `ON ERROR` clause are: + +- Input conversion errors, such as malformed JSON +- JSON path evaluation errors, e.g. division by zero +- Output conversion errors + +(json-value)= + +## json_value + +The `json_value` function extracts a scalar SQL value from a JSON value. + +```text +JSON_VALUE( + json_input [ FORMAT JSON [ ENCODING { UTF8 | UTF16 | UTF32 } ] ], + json_path + [ PASSING json_argument [, ...] ] + [ RETURNING type ] + [ { ERROR | NULL | DEFAULT expression } ON EMPTY ] + [ { ERROR | NULL | DEFAULT expression } ON ERROR ] + ) +``` + +The `json_path` is evaluated using the `json_input` as the context variable +(`$`), and the passed arguments as the named variables (`$variable_name`). + +The returned value is the SQL scalar returned by the path. By default, it is +converted to string (`varchar`). In the `RETURNING` clause, you can specify +other desired type: a character string type, numeric, boolean or datetime type. + +`json_input` is a character string or a binary string. It should contain +a single JSON item. For a binary string, you can specify encoding. + +`json_path` is a string literal, containing the path mode specification, and +the path expression, following the syntax rules described in +{ref}`json-path-syntax-and-semantics`. + +```text +'strict $.price + $tax' +'lax $[last].abs().floor()' +``` + +In the `PASSING` clause you can pass arbitrary expressions to be used by the +path expression. + +```text +PASSING orders.tax AS O_TAX +``` + +The passed parameters can be referenced in the path expression by named +variables, prefixed with `$`. + +```text +'strict $[last].price + $O_TAX' +``` + +Additionally to SQL values, you can pass JSON values, specifying the format and +optional encoding: + +```text +PASSING orders.json_desc FORMAT JSON AS o_desc, + orders.binary_record FORMAT JSON ENCODING UTF16 AS o_rec +``` + +Note that the JSON path language is case-sensitive, while the unquoted SQL +identifiers are upper-cased. Therefore, it is recommended to use quoted +identifiers in the `PASSING` clause: + +```text +'lax $.$KeyName' PASSING nation.name AS KeyName --> ERROR; no passed value found +'lax $.$KeyName' PASSING nation.name AS "KeyName" --> correct +``` + +If the path returns an empty sequence, the `ON EMPTY` clause is applied. The +default value returned `ON EMPTY` is `NULL`. You can also specify the +default value: + +```text +DEFAULT -1 ON EMPTY +``` + +If an error occurs, the returned value depends on the `ON ERROR` clause. The +default value returned `ON ERROR` is `NULL`. One example of error is +multiple items returned by the path. Other errors caught and handled according +to the `ON ERROR` clause are: + +- Input conversion errors, such as malformed JSON +- JSON path evaluation errors, e.g. division by zero +- Returned scalar not convertible to the desired type + +### Examples + +Let `customers` be a table containing two columns: `id:bigint`, +`description:varchar`. + +| id | description | +| --- | ----------------------------------------------------- | +| 101 | '{"comment" : "nice", "children" : \[10, 13, 16\]}' | +| 102 | '{"comment" : "problematic", "children" : \[8, 11\]}' | +| 103 | '{"comment" : "knows best", "children" : \[2\]}' | + +The following query gets the `comment` for each customer as `char(12)`: + +```text +SELECT id, json_value( + description, + 'lax $.comment' + RETURNING char(12) + ) AS comment +FROM customers +``` + +| id | comment | +| --- | -------------- | +| 101 | 'nice ' | +| 102 | 'problematic ' | +| 103 | 'knows best ' | + +The following query gets the first child's age for each customer as +`tinyint`: + +```text +SELECT id, json_value( + description, + 'lax $.children[0]' + RETURNING tinyint + ) AS child +FROM customers +``` + +| id | child | +| --- | ----- | +| 101 | 10 | +| 102 | 8 | +| 103 | 2 | + +The following query gets the third child's age for each customer. In the strict +mode, this should cause a structural error for the customers who do not have +the third child. This error is handled according to the `ON ERROR` clause. + +```text +SELECT id, json_value( + description, + 'strict $.children[2]' + DEFAULT 'err' ON ERROR + ) AS child +FROM customers +``` + +| id | child | +| --- | ----- | +| 101 | '16' | +| 102 | 'err' | +| 103 | 'err' | + +After changing the mode to lax, the structural error is suppressed, and the +customers without a third child produce empty sequence. This case is handled +according to the `ON EMPTY` clause. + +```text +SELECT id, json_value( + description, + 'lax $.children[2]' + DEFAULT 'missing' ON EMPTY + ) AS child +FROM customers +``` + +| id | child | +| --- | --------- | +| 101 | '16' | +| 102 | 'missing' | +| 103 | 'missing' | + +(json-array)= + +## json_array + +The `json_array` function creates a JSON array containing given elements. + +```text +JSON_ARRAY( + [ array_element [, ...] + [ { NULL ON NULL | ABSENT ON NULL } ] ], + [ RETURNING type [ FORMAT JSON [ ENCODING { UTF8 | UTF16 | UTF32 } ] ] ] + ) +``` + +### Argument types + +The array elements can be arbitrary expressions. Each passed value is converted +into a JSON item according to its type, and optional `FORMAT` and +`ENCODING` specification. + +You can pass SQL values of types boolean, numeric, and character string. They +are converted to corresponding JSON literals: + +``` +SELECT json_array(true, 12e-1, 'text') +--> '[true,1.2,"text"]' +``` + +Additionally to SQL values, you can pass JSON values. They are character or +binary strings with a specified format and optional encoding: + +``` +SELECT json_array( + '[ "text" ] ' FORMAT JSON, + X'5B0035005D00' FORMAT JSON ENCODING UTF16 + ) +--> '[["text"],[5]]' +``` + +You can also nest other JSON-returning functions. In that case, the `FORMAT` +option is implicit: + +``` +SELECT json_array( + json_query('{"key" : [ "value" ]}', 'lax $.key') + ) +--> '[["value"]]' +``` + +Other passed values are cast to varchar, and they become JSON text literals: + +``` +SELECT json_array( + DATE '2001-01-31', + UUID '12151fd2-7586-11e9-8f9e-2a86e4085a59' + ) +--> '["2001-01-31","12151fd2-7586-11e9-8f9e-2a86e4085a59"]' +``` + +You can omit the arguments altogether to get an empty array: + +``` +SELECT json_array() --> '[]' +``` + +### Null handling + +If a value passed for an array element is `null`, it is treated according to +the specified null treatment option. If `ABSENT ON NULL` is specified, the +null element is omitted in the result. If `NULL ON NULL` is specified, JSON +`null` is added to the result. `ABSENT ON NULL` is the default +configuration: + +``` +SELECT json_array(true, null, 1) +--> '[true,1]' + +SELECT json_array(true, null, 1 ABSENT ON NULL) +--> '[true,1]' + +SELECT json_array(true, null, 1 NULL ON NULL) +--> '[true,null,1]' +``` + +### Returned type + +The SQL standard imposes that there is no dedicated data type to represent JSON +data in SQL. Instead, JSON data is represented as character or binary strings. +By default, the `json_array` function returns varchar containing the textual +representation of the JSON array. With the `RETURNING` clause, you can +specify other character string type: + +``` +SELECT json_array(true, 1 RETURNING VARCHAR(100)) +--> '[true,1]' +``` + +You can also specify to use varbinary and the required encoding as return type. +The default encoding is UTF8: + +``` +SELECT json_array(true, 1 RETURNING VARBINARY) +--> X'5b 74 72 75 65 2c 31 5d' + +SELECT json_array(true, 1 RETURNING VARBINARY FORMAT JSON ENCODING UTF8) +--> X'5b 74 72 75 65 2c 31 5d' + +SELECT json_array(true, 1 RETURNING VARBINARY FORMAT JSON ENCODING UTF16) +--> X'5b 00 74 00 72 00 75 00 65 00 2c 00 31 00 5d 00' + +SELECT json_array(true, 1 RETURNING VARBINARY FORMAT JSON ENCODING UTF32) +--> X'5b 00 00 00 74 00 00 00 72 00 00 00 75 00 00 00 65 00 00 00 2c 00 00 00 31 00 00 00 5d 00 00 00' +``` + +(json-object)= + +## json_object + +The `json_object` function creates a JSON object containing given key-value pairs. + +```text +JSON_OBJECT( + [ key_value [, ...] + [ { NULL ON NULL | ABSENT ON NULL } ] ], + [ { WITH UNIQUE [ KEYS ] | WITHOUT UNIQUE [ KEYS ] } ] + [ RETURNING type [ FORMAT JSON [ ENCODING { UTF8 | UTF16 | UTF32 } ] ] ] + ) +``` + +### Argument passing conventions + +There are two conventions for passing keys and values: + +``` +SELECT json_object('key1' : 1, 'key2' : true) +--> '{"key1":1,"key2":true}' + +SELECT json_object(KEY 'key1' VALUE 1, KEY 'key2' VALUE true) +--> '{"key1":1,"key2":true}' +``` + +In the second convention, you can omit the `KEY` keyword: + +``` +SELECT json_object('key1' VALUE 1, 'key2' VALUE true) +--> '{"key1":1,"key2":true}' +``` + +### Argument types + +The keys can be arbitrary expressions. They must be of character string type. +Each key is converted into a JSON text item, and it becomes a key in the +created JSON object. Keys must not be null. + +The values can be arbitrary expressions. Each passed value is converted +into a JSON item according to its type, and optional `FORMAT` and +`ENCODING` specification. + +You can pass SQL values of types boolean, numeric, and character string. They +are converted to corresponding JSON literals: + +``` +SELECT json_object('x' : true, 'y' : 12e-1, 'z' : 'text') +--> '{"x":true,"y":1.2,"z":"text"}' +``` + +Additionally to SQL values, you can pass JSON values. They are character or +binary strings with a specified format and optional encoding: + +``` +SELECT json_object( + 'x' : '[ "text" ] ' FORMAT JSON, + 'y' : X'5B0035005D00' FORMAT JSON ENCODING UTF16 + ) +--> '{"x":["text"],"y":[5]}' +``` + +You can also nest other JSON-returning functions. In that case, the `FORMAT` +option is implicit: + +``` +SELECT json_object( + 'x' : json_query('{"key" : [ "value" ]}', 'lax $.key') + ) +--> '{"x":["value"]}' +``` + +Other passed values are cast to varchar, and they become JSON text literals: + +``` +SELECT json_object( + 'x' : DATE '2001-01-31', + 'y' : UUID '12151fd2-7586-11e9-8f9e-2a86e4085a59' + ) +--> '{"x":"2001-01-31","y":"12151fd2-7586-11e9-8f9e-2a86e4085a59"}' +``` + +You can omit the arguments altogether to get an empty object: + +``` +SELECT json_object() --> '{}' +``` + +### Null handling + +The values passed for JSON object keys must not be null. It is allowed to pass +`null` for JSON object values. A null value is treated according to the +specified null treatment option. If `NULL ON NULL` is specified, a JSON +object entry with `null` value is added to the result. If `ABSENT ON NULL` +is specified, the entry is omitted in the result. `NULL ON NULL` is the +default configuration.: + +``` +SELECT json_object('x' : null, 'y' : 1) +--> '{"x":null,"y":1}' + +SELECT json_object('x' : null, 'y' : 1 NULL ON NULL) +--> '{"x":null,"y":1}' + +SELECT json_object('x' : null, 'y' : 1 ABSENT ON NULL) +--> '{"y":1}' +``` + +### Key uniqueness + +If a duplicate key is encountered, it is handled according to the specified key +uniqueness constraint. + +If `WITH UNIQUE KEYS` is specified, a duplicate key results in a query +failure: + +``` +SELECT json_object('x' : null, 'x' : 1 WITH UNIQUE KEYS) +--> failure: "duplicate key passed to JSON_OBJECT function" +``` + +Note that this option is not supported if any of the arguments has a +`FORMAT` specification. + +If `WITHOUT UNIQUE KEYS` is specified, duplicate keys are not supported due +to implementation limitation. `WITHOUT UNIQUE KEYS` is the default +configuration. + +### Returned type + +The SQL standard imposes that there is no dedicated data type to represent JSON +data in SQL. Instead, JSON data is represented as character or binary strings. +By default, the `json_object` function returns varchar containing the textual +representation of the JSON object. With the `RETURNING` clause, you can +specify other character string type: + +``` +SELECT json_object('x' : 1 RETURNING VARCHAR(100)) +--> '{"x":1}' +``` + +You can also specify to use varbinary and the required encoding as return type. +The default encoding is UTF8: + +``` +SELECT json_object('x' : 1 RETURNING VARBINARY) +--> X'7b 22 78 22 3a 31 7d' + +SELECT json_object('x' : 1 RETURNING VARBINARY FORMAT JSON ENCODING UTF8) +--> X'7b 22 78 22 3a 31 7d' + +SELECT json_object('x' : 1 RETURNING VARBINARY FORMAT JSON ENCODING UTF16) +--> X'7b 00 22 00 78 00 22 00 3a 00 31 00 7d 00' + +SELECT json_object('x' : 1 RETURNING VARBINARY FORMAT JSON ENCODING UTF32) +--> X'7b 00 00 00 22 00 00 00 78 00 00 00 22 00 00 00 3a 00 00 00 31 00 00 00 7d 00 00 00' +``` + +:::{warning} +The following functions and operators are not compliant with the SQL +standard, and should be considered deprecated. According to the SQL +standard, there shall be no `JSON` data type. Instead, JSON values +should be represented as string values. The remaining functionality of the +following functions is covered by the functions described previously. +::: + +## Cast to JSON + +The following types can be cast to JSON: + +- `BOOLEAN` +- `TINYINT` +- `SMALLINT` +- `INTEGER` +- `BIGINT` +- `REAL` +- `DOUBLE` +- `VARCHAR` + +Additionally, `ARRAY`, `MAP`, and `ROW` types can be cast to JSON when +the following requirements are met: + +- `ARRAY` types can be cast when the element type of the array is one + of the supported types. +- `MAP` types can be cast when the key type of the map is `VARCHAR` and + the value type of the map is a supported type, +- `ROW` types can be cast when every field type of the row is a supported + type. + +:::{note} +Cast operations with supported {ref}`character string types +` treat the input as a string, not validated as JSON. +This means that a cast operation with a string-type input of invalid JSON +results in a succesful cast to invalid JSON. + +Instead, consider using the {func}`json_parse` function to +create validated JSON from a string. +::: + +The following examples show the behavior of casting to JSON with these types: + +``` +SELECT CAST(NULL AS JSON); +-- NULL + +SELECT CAST(1 AS JSON); +-- JSON '1' + +SELECT CAST(9223372036854775807 AS JSON); +-- JSON '9223372036854775807' + +SELECT CAST('abc' AS JSON); +-- JSON '"abc"' + +SELECT CAST(true AS JSON); +-- JSON 'true' + +SELECT CAST(1.234 AS JSON); +-- JSON '1.234' + +SELECT CAST(ARRAY[1, 23, 456] AS JSON); +-- JSON '[1,23,456]' + +SELECT CAST(ARRAY[1, NULL, 456] AS JSON); +-- JSON '[1,null,456]' + +SELECT CAST(ARRAY[ARRAY[1, 23], ARRAY[456]] AS JSON); +-- JSON '[[1,23],[456]]' + +SELECT CAST(MAP(ARRAY['k1', 'k2', 'k3'], ARRAY[1, 23, 456]) AS JSON); +-- JSON '{"k1":1,"k2":23,"k3":456}' + +SELECT CAST(CAST(ROW(123, 'abc', true) AS + ROW(v1 BIGINT, v2 VARCHAR, v3 BOOLEAN)) AS JSON); +-- JSON '{"v1":123,"v2":"abc","v3":true}' +``` + +Casting from NULL to `JSON` is not straightforward. Casting +from a standalone `NULL` will produce SQL `NULL` instead of +`JSON 'null'`. However, when casting from arrays or map containing +`NULL`s, the produced `JSON` will have `null`s in it. + +## Cast from JSON + +Casting to `BOOLEAN`, `TINYINT`, `SMALLINT`, `INTEGER`, +`BIGINT`, `REAL`, `DOUBLE` or `VARCHAR` is supported. +Casting to `ARRAY` and `MAP` is supported when the element type of +the array is one of the supported types, or when the key type of the map +is `VARCHAR` and value type of the map is one of the supported types. +Behaviors of the casts are shown with the examples below: + +``` +SELECT CAST(JSON 'null' AS VARCHAR); +-- NULL + +SELECT CAST(JSON '1' AS INTEGER); +-- 1 + +SELECT CAST(JSON '9223372036854775807' AS BIGINT); +-- 9223372036854775807 + +SELECT CAST(JSON '"abc"' AS VARCHAR); +-- abc + +SELECT CAST(JSON 'true' AS BOOLEAN); +-- true + +SELECT CAST(JSON '1.234' AS DOUBLE); +-- 1.234 + +SELECT CAST(JSON '[1,23,456]' AS ARRAY(INTEGER)); +-- [1, 23, 456] + +SELECT CAST(JSON '[1,null,456]' AS ARRAY(INTEGER)); +-- [1, NULL, 456] + +SELECT CAST(JSON '[[1,23],[456]]' AS ARRAY(ARRAY(INTEGER))); +-- [[1, 23], [456]] + +SELECT CAST(JSON '{"k1":1,"k2":23,"k3":456}' AS MAP(VARCHAR, INTEGER)); +-- {k1=1, k2=23, k3=456} + +SELECT CAST(JSON '{"v1":123,"v2":"abc","v3":true}' AS + ROW(v1 BIGINT, v2 VARCHAR, v3 BOOLEAN)); +-- {v1=123, v2=abc, v3=true} + +SELECT CAST(JSON '[123,"abc",true]' AS + ROW(v1 BIGINT, v2 VARCHAR, v3 BOOLEAN)); +-- {v1=123, v2=abc, v3=true} +``` + +JSON arrays can have mixed element types and JSON maps can have mixed +value types. This makes it impossible to cast them to SQL arrays and maps in +some cases. To address this, Trino supports partial casting of arrays and maps: + +``` +SELECT CAST(JSON '[[1, 23], 456]' AS ARRAY(JSON)); +-- [JSON '[1,23]', JSON '456'] + +SELECT CAST(JSON '{"k1": [1, 23], "k2": 456}' AS MAP(VARCHAR, JSON)); +-- {k1 = JSON '[1,23]', k2 = JSON '456'} + +SELECT CAST(JSON '[null]' AS ARRAY(JSON)); +-- [JSON 'null'] +``` + +When casting from `JSON` to `ROW`, both JSON array and JSON object are supported. + +## Other JSON functions + +In addition to the functions explained in more details in the preceding +sections, the following functions are available: + +:::{function} is_json_scalar(json) -> boolean +Determine if `json` is a scalar (i.e. a JSON number, a JSON string, `true`, `false` or `null`): + +``` +SELECT is_json_scalar('1'); -- true +SELECT is_json_scalar('[1, 2, 3]'); -- false +``` +::: + +:::{function} json_array_contains(json, value) -> boolean +Determine if `value` exists in `json` (a string containing a JSON array): + +``` +SELECT json_array_contains('[1, 2, 3]', 2); -- true +``` +::: + +::::{function} json_array_get(json_array, index) -> json + +:::{warning} +The semantics of this function are broken. If the extracted element +is a string, it will be converted into an invalid `JSON` value that +is not properly quoted (the value will not be surrounded by quotes +and any interior quotes will not be escaped). + +We recommend against using this function. It cannot be fixed without +impacting existing usages and may be removed in a future release. +::: + +Returns the element at the specified index into the `json_array`. +The index is zero-based: + +``` +SELECT json_array_get('["a", [3, 9], "c"]', 0); -- JSON 'a' (invalid JSON) +SELECT json_array_get('["a", [3, 9], "c"]', 1); -- JSON '[3,9]' +``` + +This function also supports negative indexes for fetching element indexed +from the end of an array: + +``` +SELECT json_array_get('["c", [3, 9], "a"]', -1); -- JSON 'a' (invalid JSON) +SELECT json_array_get('["c", [3, 9], "a"]', -2); -- JSON '[3,9]' +``` + +If the element at the specified index doesn't exist, the function returns null: + +``` +SELECT json_array_get('[]', 0); -- NULL +SELECT json_array_get('["a", "b", "c"]', 10); -- NULL +SELECT json_array_get('["c", "b", "a"]', -10); -- NULL +``` +:::: + +:::{function} json_array_length(json) -> bigint +Returns the array length of `json` (a string containing a JSON array): + +``` +SELECT json_array_length('[1, 2, 3]'); -- 3 +``` +::: + +:::{function} json_extract(json, json_path) -> json +Evaluates the [JSONPath]-like expression `json_path` on `json` +(a string containing JSON) and returns the result as a JSON string: + +``` +SELECT json_extract(json, '$.store.book'); +SELECT json_extract(json, '$.store[book]'); +SELECT json_extract(json, '$.store["book name"]'); +``` + +The {ref}`json_query function` provides a more powerful and +feature-rich alternative to parse and extract JSON data. +::: + +:::{function} json_extract_scalar(json, json_path) -> varchar +Like {func}`json_extract`, but returns the result value as a string (as opposed +to being encoded as JSON). The value referenced by `json_path` must be a +scalar (boolean, number or string). + +``` +SELECT json_extract_scalar('[1, 2, 3]', '$[2]'); +SELECT json_extract_scalar(json, '$.store.book[0].author'); +``` +::: + +::::{function} json_format(json) -> varchar +Returns the JSON text serialized from the input JSON value. +This is inverse function to {func}`json_parse`. + +``` +SELECT json_format(JSON '[1, 2, 3]'); -- '[1,2,3]' +SELECT json_format(JSON '"a"'); -- '"a"' +``` + +:::{note} +{func}`json_format` and `CAST(json AS VARCHAR)` have completely +different semantics. + +{func}`json_format` serializes the input JSON value to JSON text conforming to +{rfc}`7159`. The JSON value can be a JSON object, a JSON array, a JSON string, +a JSON number, `true`, `false` or `null`. + +``` +SELECT json_format(JSON '{"a": 1, "b": 2}'); -- '{"a":1,"b":2}' +SELECT json_format(JSON '[1, 2, 3]'); -- '[1,2,3]' +SELECT json_format(JSON '"abc"'); -- '"abc"' +SELECT json_format(JSON '42'); -- '42' +SELECT json_format(JSON 'true'); -- 'true' +SELECT json_format(JSON 'null'); -- 'null' +``` + +`CAST(json AS VARCHAR)` casts the JSON value to the corresponding SQL VARCHAR value. +For JSON string, JSON number, `true`, `false` or `null`, the cast +behavior is same as the corresponding SQL type. JSON object and JSON array +cannot be cast to VARCHAR. + +``` +SELECT CAST(JSON '{"a": 1, "b": 2}' AS VARCHAR); -- ERROR! +SELECT CAST(JSON '[1, 2, 3]' AS VARCHAR); -- ERROR! +SELECT CAST(JSON '"abc"' AS VARCHAR); -- 'abc' (the double quote is gone) +SELECT CAST(JSON '42' AS VARCHAR); -- '42' +SELECT CAST(JSON 'true' AS VARCHAR); -- 'true' +SELECT CAST(JSON 'null' AS VARCHAR); -- NULL +``` +::: +:::: + +::::{function} json_parse(string) -> json +Returns the JSON value deserialized from the input JSON text. +This is inverse function to {func}`json_format`: + +``` +SELECT json_parse('[1, 2, 3]'); -- JSON '[1,2,3]' +SELECT json_parse('"abc"'); -- JSON '"abc"' +``` + +:::{note} +{func}`json_parse` and `CAST(string AS JSON)` have completely +different semantics. + +{func}`json_parse` expects a JSON text conforming to {rfc}`7159`, and returns +the JSON value deserialized from the JSON text. +The JSON value can be a JSON object, a JSON array, a JSON string, a JSON number, +`true`, `false` or `null`. + +``` +SELECT json_parse('not_json'); -- ERROR! +SELECT json_parse('["a": 1, "b": 2]'); -- JSON '["a": 1, "b": 2]' +SELECT json_parse('[1, 2, 3]'); -- JSON '[1,2,3]' +SELECT json_parse('"abc"'); -- JSON '"abc"' +SELECT json_parse('42'); -- JSON '42' +SELECT json_parse('true'); -- JSON 'true' +SELECT json_parse('null'); -- JSON 'null' +``` + +`CAST(string AS JSON)` takes any VARCHAR value as input, and returns +a JSON string with its value set to input string. + +``` +SELECT CAST('not_json' AS JSON); -- JSON '"not_json"' +SELECT CAST('["a": 1, "b": 2]' AS JSON); -- JSON '"[\"a\": 1, \"b\": 2]"' +SELECT CAST('[1, 2, 3]' AS JSON); -- JSON '"[1, 2, 3]"' +SELECT CAST('"abc"' AS JSON); -- JSON '"\"abc\""' +SELECT CAST('42' AS JSON); -- JSON '"42"' +SELECT CAST('true' AS JSON); -- JSON '"true"' +SELECT CAST('null' AS JSON); -- JSON '"null"' +``` +::: +:::: + +:::{function} json_size(json, json_path) -> bigint +Like {func}`json_extract`, but returns the size of the value. +For objects or arrays, the size is the number of members, +and the size of a scalar value is zero. + +``` +SELECT json_size('{"x": {"a": 1, "b": 2}}', '$.x'); -- 2 +SELECT json_size('{"x": [1, 2, 3]}', '$.x'); -- 3 +SELECT json_size('{"x": {"a": 1, "b": 2}}', '$.x.a'); -- 0 +``` +::: + +[jsonpath]: http://goessner.net/articles/JsonPath/ diff --git a/430/_sources/functions/lambda.md.txt b/430/_sources/functions/lambda.md.txt new file mode 100644 index 000000000..26f36f439 --- /dev/null +++ b/430/_sources/functions/lambda.md.txt @@ -0,0 +1,130 @@ +(lambda-expressions)= + +# Lambda expressions + +Lambda expressions are anonymous functions which are passed as +arguments to higher-order SQL functions. + +Lambda expressions are written with `->`: + +``` +x -> x + 1 +(x, y) -> x + y +x -> regexp_like(x, 'a+') +x -> x[1] / x[2] +x -> IF(x > 0, x, -x) +x -> COALESCE(x, 0) +x -> CAST(x AS JSON) +x -> x + TRY(1 / 0) +``` + +## Limitations + +Most SQL expressions can be used in a lambda body, with a few exceptions: + +- Subqueries are not supported: `x -> 2 + (SELECT 3)` +- Aggregations are not supported: `x -> max(y)` + +## Examples + +Obtain the squared elements of an array column with {func}`transform`: + +``` +SELECT numbers, + transform(numbers, n -> n * n) as squared_numbers +FROM ( + VALUES + (ARRAY[1, 2]), + (ARRAY[3, 4]), + (ARRAY[5, 6, 7]) +) AS t(numbers); +``` + +```text + numbers | squared_numbers +-----------+----------------- + [1, 2] | [1, 4] + [3, 4] | [9, 16] + [5, 6, 7] | [25, 36, 49] +(3 rows) +``` + +The function {func}`transform` can be also employed to safely cast the elements +of an array to strings: + +``` +SELECT transform(prices, n -> TRY_CAST(n AS VARCHAR) || '$') as price_tags +FROM ( + VALUES + (ARRAY[100, 200]), + (ARRAY[30, 4]) +) AS t(prices); +``` + +```text + price_tags +-------------- + [100$, 200$] + [30$, 4$] +(2 rows) +``` + +Besides the array column being manipulated, +other columns can be captured as well within the lambda expression. +The following statement provides a showcase of this feature +for calculating the value of the linear function `f(x) = ax + b` +with {func}`transform`: + +``` +SELECT xvalues, + a, + b, + transform(xvalues, x -> a * x + b) as linear_function_values +FROM ( + VALUES + (ARRAY[1, 2], 10, 5), + (ARRAY[3, 4], 4, 2) +) AS t(xvalues, a, b); +``` + +```text + xvalues | a | b | linear_function_values +---------+----+---+------------------------ + [1, 2] | 10 | 5 | [15, 25] + [3, 4] | 4 | 2 | [14, 18] +(2 rows) +``` + +Find the array elements containing at least one value greater than `100` +with {func}`any_match`: + +``` +SELECT numbers +FROM ( + VALUES + (ARRAY[1,NULL,3]), + (ARRAY[10,20,30]), + (ARRAY[100,200,300]) +) AS t(numbers) +WHERE any_match(numbers, n -> COALESCE(n, 0) > 100); +-- [100, 200, 300] +``` + +Capitalize the first word in a string via {func}`regexp_replace`: + +``` +SELECT regexp_replace('once upon a time ...', '^(\w)(\w*)(\s+.*)$',x -> upper(x[1]) || x[2] || x[3]); +-- Once upon a time ... +``` + +Lambda expressions can be also applied in aggregation functions. +Following statement is a sample the overly complex calculation of the sum of all elements of a column +by making use of {func}`reduce_agg`: + +``` +SELECT reduce_agg(value, 0, (a, b) -> a + b, (a, b) -> a + b) sum_values +FROM ( + VALUES (1), (2), (3), (4), (5) +) AS t(value); +-- 15 +``` diff --git a/430/_sources/functions/list-by-topic.md.txt b/430/_sources/functions/list-by-topic.md.txt new file mode 100644 index 000000000..d55aafbcb --- /dev/null +++ b/430/_sources/functions/list-by-topic.md.txt @@ -0,0 +1,555 @@ +# List of functions by topic + +## Aggregate + +For more details, see {doc}`aggregate` + +- {func}`any_value` +- {func}`approx_distinct` +- {func}`approx_most_frequent` +- {func}`approx_percentile` +- `approx_set()` +- {func}`arbitrary` +- {func}`array_agg` +- {func}`avg` +- {func}`bitwise_and_agg` +- {func}`bitwise_or_agg` +- {func}`bool_and` +- {func}`bool_or` +- {func}`checksum` +- {func}`corr` +- {func}`count` +- {func}`count_if` +- {func}`covar_pop` +- {func}`covar_samp` +- {func}`every` +- {func}`geometric_mean` +- {func}`histogram` +- {func}`kurtosis` +- {func}`map_agg` +- {func}`map_union` +- {func}`max` +- {func}`max_by` +- `merge()` +- {func}`min` +- {func}`min_by` +- {func}`multimap_agg` +- {func}`numeric_histogram` +- `qdigest_agg()` +- {func}`regr_intercept` +- {func}`regr_slope` +- {func}`skewness` +- {func}`sum` +- {func}`stddev` +- {func}`stddev_pop` +- {func}`stddev_samp` +- `tdigest_agg()` +- {func}`variance` +- {func}`var_pop` +- {func}`var_samp` + +## Array + +For more details, see {doc}`array` + +- {func}`all_match` +- {func}`any_match` +- {func}`array_distinct` +- {func}`array_except` +- {func}`array_intersect` +- {func}`array_join` +- {func}`array_max` +- {func}`array_min` +- {func}`array_position` +- {func}`array_remove` +- {func}`array_sort` +- {func}`array_union` +- {func}`arrays_overlap` +- {func}`cardinality` +- {func}`combinations` +- `concat()` +- {func}`contains` +- {func}`element_at` +- {func}`filter` +- {func}`flatten` +- {func}`ngrams` +- {func}`none_match` +- {func}`reduce` +- {func}`repeat` +- `reverse()` +- {func}`sequence` +- {func}`shuffle` +- {func}`slice` +- {func}`transform` +- {func}`trim_array` +- {func}`zip` +- {func}`zip_with` + +## Binary + +For more details, see {doc}`binary` + +- `concat()` +- {func}`crc32` +- {func}`from_base32` +- {func}`from_base64` +- {func}`from_base64url` +- {func}`from_big_endian_32` +- {func}`from_big_endian_64` +- {func}`from_hex` +- {func}`from_ieee754_32` +- {func}`from_ieee754_64` +- {func}`hmac_md5` +- {func}`hmac_sha1` +- {func}`hmac_sha256` +- {func}`hmac_sha512` +- `length()` +- `lpad()` +- {func}`md5` +- {func}`murmur3` +- `reverse()` +- `rpad()` +- {func}`sha1` +- {func}`sha256` +- {func}`sha512` +- {func}`spooky_hash_v2_32` +- {func}`spooky_hash_v2_64` +- `substr()` +- {func}`to_base32` +- {func}`to_base64` +- {func}`to_base64url` +- {func}`to_big_endian_32` +- {func}`to_big_endian_64` +- {func}`to_hex` +- {func}`to_ieee754_32` +- {func}`to_ieee754_64` +- {func}`xxhash64` + +## Bitwise + +For more details, see {doc}`bitwise` + +- {func}`bit_count` +- {func}`bitwise_and` +- {func}`bitwise_left_shift` +- {func}`bitwise_not` +- {func}`bitwise_or` +- {func}`bitwise_right_shift` +- {func}`bitwise_right_shift_arithmetic` +- {func}`bitwise_xor` + +## Color + +For more details, see {doc}`color` + +- {func}`bar` +- {func}`color` +- {func}`render` +- {func}`rgb` + +## Comparison + +For more details, see {doc}`comparison` + +- {func}`greatest` +- {func}`least` + +## Conditional + +For more details, see {doc}`conditional` + +- {ref}`coalesce ` +- {ref}`if ` +- {ref}`nullif ` +- {ref}`try ` + +## Conversion + +For more details, see {doc}`conversion` + +- {func}`cast` +- {func}`format` +- {func}`try_cast` +- {func}`typeof` + +## Date and time + +For more details, see {doc}`datetime` + +- {ref}`AT TIME ZONE ` +- {data}`current_date` +- {data}`current_time` +- {data}`current_timestamp` +- {data}`localtime` +- {data}`localtimestamp` +- {func}`current_timezone` +- {func}`date` +- {func}`date_add` +- {func}`date_diff` +- {func}`date_format` +- {func}`date_parse` +- {func}`date_trunc` +- {func}`format_datetime` +- {func}`from_iso8601_date` +- {func}`from_iso8601_timestamp` +- {func}`from_unixtime` +- {func}`from_unixtime_nanos` +- {func}`human_readable_seconds` +- {func}`last_day_of_month` +- {func}`now` +- {func}`parse_duration` +- {func}`to_iso8601` +- {func}`to_milliseconds` +- {func}`to_unixtime` +- {func}`with_timezone` + +## Geospatial + +For more details, see {doc}`geospatial` + +- {func}`bing_tile` +- {func}`bing_tile_at` +- {func}`bing_tile_coordinates` +- {func}`bing_tile_polygon` +- {func}`bing_tile_quadkey` +- {func}`bing_tile_zoom_level` +- {func}`bing_tiles_around` +- {func}`convex_hull_agg` +- {func}`from_encoded_polyline` +- {func}`from_geojson_geometry` +- {func}`geometry_from_hadoop_shape` +- {func}`geometry_invalid_reason` +- {func}`geometry_nearest_points` +- {func}`geometry_to_bing_tiles` +- {func}`geometry_union` +- {func}`geometry_union_agg` +- {func}`great_circle_distance` +- {func}`line_interpolate_point` +- {func}`line_locate_point` +- {func}`simplify_geometry` +- {func}`ST_Area` +- {func}`ST_AsBinary` +- {func}`ST_AsText` +- {func}`ST_Boundary` +- {func}`ST_Buffer` +- {func}`ST_Centroid` +- {func}`ST_Contains` +- {func}`ST_ConvexHull` +- {func}`ST_CoordDim` +- {func}`ST_Crosses` +- {func}`ST_Difference` +- {func}`ST_Dimension` +- {func}`ST_Disjoint` +- {func}`ST_Distance` +- {func}`ST_EndPoint` +- {func}`ST_Envelope` +- {func}`ST_Equals` +- {func}`ST_ExteriorRing` +- {func}`ST_Geometries` +- {func}`ST_GeometryFromText` +- {func}`ST_GeometryN` +- {func}`ST_GeometryType` +- {func}`ST_GeomFromBinary` +- {func}`ST_InteriorRings` +- {func}`ST_InteriorRingN` +- {func}`ST_Intersects` +- {func}`ST_Intersection` +- {func}`ST_IsClosed` +- {func}`ST_IsEmpty` +- {func}`ST_IsSimple` +- {func}`ST_IsRing` +- {func}`ST_IsValid` +- {func}`ST_Length` +- {func}`ST_LineFromText` +- {func}`ST_LineString` +- {func}`ST_MultiPoint` +- {func}`ST_NumGeometries` +- {func}`ST_NumInteriorRing` +- {func}`ST_NumPoints` +- {func}`ST_Overlaps` +- {func}`ST_Point` +- {func}`ST_PointN` +- {func}`ST_Points` +- {func}`ST_Polygon` +- {func}`ST_Relate` +- {func}`ST_StartPoint` +- {func}`ST_SymDifference` +- {func}`ST_Touches` +- {func}`ST_Union` +- {func}`ST_Within` +- {func}`ST_X` +- {func}`ST_XMax` +- {func}`ST_XMin` +- {func}`ST_Y` +- {func}`ST_YMax` +- {func}`ST_YMin` +- {func}`to_encoded_polyline` +- {func}`to_geojson_geometry` +- {func}`to_geometry` +- {func}`to_spherical_geography` + +## HyperLogLog + +For more details, see {doc}`hyperloglog` + +- {func}`approx_set` +- `cardinality()` +- {func}`empty_approx_set` +- {func}`merge` + +## JSON + +For more details, see {doc}`json` + +- {func}`is_json_scalar` +- {ref}`json_array() ` +- {func}`json_array_contains` +- {func}`json_array_get` +- {func}`json_array_length` +- {ref}`json_exists() ` +- {func}`json_extract` +- {func}`json_extract_scalar` +- {func}`json_format` +- {func}`json_parse` +- {ref}`json_object() ` +- {ref}`json_query() ` +- {func}`json_size` +- {ref}`json_value() ` + +## Lambda + +For more details, see {doc}`lambda` + +- {func}`any_match` +- {func}`reduce_agg` +- {func}`regexp_replace` +- {func}`transform` + +## Machine learning + +For more details, see {doc}`ml` + +- {func}`classify` +- {func}`features` +- {func}`learn_classifier` +- {func}`learn_libsvm_classifier` +- {func}`learn_libsvm_regressor` +- {func}`learn_regressor` +- {func}`regress` + +## Map + +For more details, see {doc}`map` + +- {func}`cardinality` +- {func}`element_at` +- {func}`map` +- {func}`map_concat` +- {func}`map_entries` +- {func}`map_filter` +- {func}`map_from_entries` +- {func}`map_keys` +- {func}`map_values` +- {func}`map_zip_with` +- {func}`multimap_from_entries` +- {func}`transform_keys` +- {func}`transform_values` + +## Math + +For more details, see {doc}`math` + +- {func}`abs` +- {func}`acos` +- {func}`asin` +- {func}`atan` +- {func}`beta_cdf` +- {func}`cbrt` +- {func}`ceil` +- {func}`cos` +- {func}`cosh` +- {func}`cosine_similarity` +- {func}`degrees` +- {func}`e` +- {func}`exp` +- {func}`floor` +- {func}`from_base` +- {func}`infinity` +- {func}`inverse_beta_cdf` +- {func}`inverse_normal_cdf` +- {func}`is_finite` +- {func}`is_nan` +- {func}`ln` +- {func}`log` +- {func}`log2` +- {func}`log10` +- {func}`mod` +- {func}`nan` +- {func}`normal_cdf` +- {func}`pi` +- {func}`pow` +- {func}`power` +- {func}`radians` +- {func}`rand` +- {func}`random` +- {func}`round` +- {func}`sign` +- {func}`sin` +- {func}`sinh` +- {func}`sqrt` +- {func}`tan` +- {func}`tanh` +- {func}`to_base` +- {func}`truncate` +- {func}`width_bucket` +- {func}`wilson_interval_lower` +- {func}`wilson_interval_upper` + +## Quantile digest + +For more details, see {doc}`qdigest` + +- `merge()` +- {func}`qdigest_agg` +- {func}`value_at_quantile` +- {func}`values_at_quantiles` + +## Regular expression + +For more details, see {doc}`regexp` + +- {func}`regexp_count` +- {func}`regexp_extract` +- {func}`regexp_extract_all` +- {func}`regexp_like` +- {func}`regexp_position` +- {func}`regexp_replace` +- {func}`regexp_split` + +## Row pattern recognition expressions + +- {ref}`classifier ` +- {ref}`first ` +- {ref}`last ` +- {ref}`match_number ` +- {ref}`next ` +- {ref}`permute ` +- {ref}`prev ` + +## Session + +For more details, see {doc}`session` + +- {data}`current_catalog` +- {func}`current_groups` +- {data}`current_schema` +- {data}`current_user` + +## Set Digest + +For more details, see {doc}`setdigest` + +- {func}`make_set_digest` +- {func}`merge_set_digest` +- {ref}`cardinality() ` +- {func}`intersection_cardinality` +- {func}`jaccard_index` +- {func}`hash_counts` + +## String + +For more details, see {doc}`string` + +- {func}`chr` +- {func}`codepoint` +- {func}`concat` +- {func}`concat_ws` +- {func}`format` +- {func}`from_utf8` +- {func}`hamming_distance` +- {func}`length` +- {func}`levenshtein_distance` +- {func}`lower` +- {func}`lpad` +- {func}`ltrim` +- {func}`luhn_check` +- {func}`normalize` +- {func}`position` +- {func}`replace` +- {func}`reverse` +- {func}`rpad` +- {func}`rtrim` +- {func}`soundex` +- {func}`split` +- {func}`split_part` +- {func}`split_to_map` +- {func}`split_to_multimap` +- {func}`starts_with` +- {func}`strpos` +- {func}`substr` +- {func}`substring` +- {func}`to_utf8` +- {func}`translate` +- {func}`trim` +- {func}`upper` +- {func}`word_stem` + +## System + +For more details, see {doc}`system` + +- {func}`version` + +## T-Digest + +For more details, see {doc}`tdigest` + +- `merge()` +- {func}`tdigest_agg` +- `value_at_quantile()` + +## Teradata + +For more details, see {doc}`teradata` + +- {func}`char2hexint` +- {func}`index` +- {func}`to_char` +- {func}`to_timestamp` +- {func}`to_date` + +## URL + +For more details, see {doc}`url` + +- {func}`url_decode` +- {func}`url_encode` +- {func}`url_extract_fragment` +- {func}`url_extract_host` +- {func}`url_extract_parameter` +- {func}`url_extract_path` +- {func}`url_extract_port` +- {func}`url_extract_protocol` +- {func}`url_extract_query` + +## UUID + +For more details, see {doc}`uuid` + +- {func}`uuid` + +## Window + +For more details, see {doc}`window` + +- {func}`cume_dist` +- {func}`dense_rank` +- {func}`first_value` +- {func}`lag` +- {func}`last_value` +- {func}`lead` +- {func}`nth_value` +- {func}`ntile` +- {func}`percent_rank` +- {func}`rank` +- {func}`row_number` diff --git a/430/_sources/functions/list.md.txt b/430/_sources/functions/list.md.txt new file mode 100644 index 000000000..e69699448 --- /dev/null +++ b/430/_sources/functions/list.md.txt @@ -0,0 +1,542 @@ +# List of functions and operators + +## \# + +- [\[\] substring operator](subscript-operator) +- [|| concatenation operator](concatenation-operator) +- [< comparison operator](comparison-operators) +- [> comparison operator](comparison-operators) +- [<= comparison operator](comparison-operators) +- [>= comparison operator](comparison-operators) +- [= comparison operator](comparison-operators) +- [<> comparison operator](comparison-operators) +- [!= comparison operator](comparison-operators) +- [-> lambda expression](lambda-expressions) +- [+ mathematical operator](mathematical-operators) +- [- mathematical operator](mathematical-operators) +- [* mathematical operator](mathematical-operators) +- [/ mathematical operator](mathematical-operators) +- [% mathematical operator](mathematical-operators) + +## A + +- {func}`abs` +- {func}`acos` +- [ALL](quantified-comparison-predicates) +- {func}`all_match` +- [AND](logical-operators) +- [ANY](quantified-comparison-predicates) +- {func}`any_match` +- {func}`any_value` +- {func}`approx_distinct` +- {func}`approx_most_frequent` +- {func}`approx_percentile` +- {func}`approx_set` +- {func}`arbitrary` +- {func}`array_agg` +- {func}`array_distinct` +- {func}`array_except` +- {func}`array_intersect` +- {func}`array_join` +- {func}`array_max` +- {func}`array_min` +- {func}`array_position` +- {func}`array_remove` +- {func}`array_sort` +- {func}`array_union` +- {func}`arrays_overlap` +- {func}`asin` +- [AT TIME ZONE](at-time-zone-operator) +- {func}`at_timezone` +- {func}`atan` +- {func}`atan2` +- {func}`avg` + +## B + +- {func}`bar` +- {func}`beta_cdf` +- [BETWEEN](range-operator) +- {func}`bing_tile` +- {func}`bing_tile_at` +- {func}`bing_tile_coordinates` +- {func}`bing_tile_polygon` +- {func}`bing_tile_quadkey` +- {func}`bing_tile_zoom_level` +- {func}`bing_tiles_around` +- {func}`bit_count` +- {func}`bitwise_and` +- {func}`bitwise_and_agg` +- {func}`bitwise_left_shift` +- {func}`bitwise_not` +- {func}`bitwise_or` +- {func}`bitwise_or_agg` +- {func}`bitwise_right_shift` +- {func}`bitwise_right_shift_arithmetic` +- {func}`bitwise_xor` +- {func}`bool_and` +- {func}`bool_or` + +## C + +- {func}`cardinality` +- [CASE](case-expression) +- {func}`cast` +- {func}`cbrt` +- {func}`ceil` +- {func}`ceiling` +- {func}`char2hexint` +- {func}`checksum` +- {func}`chr` +- {func}`classify` +- [classifier](classifier-function) +- [coalesce](coalesce-function) +- {func}`codepoint` +- {func}`color` +- {func}`combinations` +- {func}`concat` +- {func}`concat_ws` +- {func}`contains` +- {func}`contains_sequence` +- {func}`convex_hull_agg` +- {func}`corr` +- {func}`cos` +- {func}`cosh` +- {func}`cosine_similarity` +- {func}`count` +- {func}`count_if` +- {func}`covar_pop` +- {func}`covar_samp` +- {func}`crc32` +- {func}`cume_dist` +- {data}`current_date` +- {func}`current_groups` +- {data}`current_time` +- {data}`current_timestamp` +- {func}`current_timezone` +- {data}`current_user` + +## D + +- {func}`date` +- {func}`date_add` +- {func}`date_diff` +- {func}`date_format` +- {func}`date_parse` +- {func}`date_trunc` +- {func}`day` +- {func}`day_of_month` +- {func}`day_of_week` +- {func}`day_of_year` +- [DECIMAL](decimal-literal) +- {func}`degrees` +- {func}`dense_rank` +- {func}`dow` +- {func}`doy` + +## E + +- {func}`e` +- {func}`element_at` +- {func}`empty_approx_set` +- `evaluate_classifier_predictions` +- {func}`every` +- {func}`exclude_columns` +- {func}`extract` +- {func}`exp` + +## F + +- {func}`features` +- {func}`filter` +- [first](logical-navigation-functions) +- {func}`first_value` +- {func}`flatten` +- {func}`floor` +- {func}`format` +- {func}`format_datetime` +- {func}`format_number` +- {func}`from_base` +- {func}`from_base32` +- {func}`from_base64` +- {func}`from_base64url` +- {func}`from_big_endian_32` +- {func}`from_big_endian_64` +- {func}`from_encoded_polyline` +- `from_geojson_geometry` +- {func}`from_hex` +- {func}`from_ieee754_32` +- {func}`from_ieee754_64` +- {func}`from_iso8601_date` +- {func}`from_iso8601_timestamp` +- {func}`from_iso8601_timestamp_nanos` +- {func}`from_unixtime` +- {func}`from_unixtime_nanos` +- {func}`from_utf8` + +## G + +- {func}`geometric_mean` +- {func}`geometry_from_hadoop_shape` +- {func}`geometry_invalid_reason` +- {func}`geometry_nearest_points` +- {func}`geometry_to_bing_tiles` +- {func}`geometry_union` +- {func}`geometry_union_agg` +- {func}`great_circle_distance` +- {func}`greatest` + +## H + +- {func}`hamming_distance` +- {func}`hash_counts` +- {func}`histogram` +- {func}`hmac_md5` +- {func}`hmac_sha1` +- {func}`hmac_sha256` +- {func}`hmac_sha512` +- {func}`hour` +- {func}`human_readable_seconds` + +## I + +- [if](if-function) +- {func}`index` +- {func}`infinity` +- {func}`intersection_cardinality` +- {func}`inverse_beta_cdf` +- {func}`inverse_normal_cdf` +- {func}`is_finite` +- {func}`is_infinite` +- {func}`is_json_scalar` +- {func}`is_nan` +- [IS NOT DISTINCT](is-distinct-operator) +- [IS NOT NULL](is-null-operator) +- [IS DISTINCT](is-distinct-operator) +- [IS NULL](is-null-operator) + +## J + +- {func}`jaccard_index` +- [json_array()](json-array) +- {func}`json_array_contains` +- {func}`json_array_get` +- {func}`json_array_length` +- [json_exists()](json-exists) +- {func}`json_extract` +- {func}`json_extract_scalar` +- {func}`json_format` +- [json_object()](json-object) +- {func}`json_parse` +- [json_query()](json-query) +- {func}`json_size` +- [json_value()](json-value) + +## K + +- {func}`kurtosis` + +## L + +- {func}`lag` +- [last](logical-navigation-functions) +- {func}`last_day_of_month` +- {func}`last_value` +- {func}`lead` +- {func}`learn_classifier` +- {func}`learn_libsvm_classifier` +- {func}`learn_libsvm_regressor` +- {func}`learn_regressor` +- {func}`least` +- {func}`length` +- {func}`levenshtein_distance` +- {func}`line_interpolate_point` +- {func}`line_interpolate_points` +- {func}`line_locate_point` +- {func}`listagg` +- {func}`ln` +- {data}`localtime` +- {data}`localtimestamp` +- {func}`log` +- {func}`log10` +- {func}`log2` +- {func}`lower` +- {func}`lpad` +- {func}`ltrim` +- {func}`luhn_check` + +## M + +- {func}`make_set_digest` +- {func}`map` +- {func}`map_agg` +- {func}`map_concat` +- {func}`map_entries` +- {func}`map_filter` +- {func}`map_from_entries` +- {func}`map_keys` +- {func}`map_union` +- {func}`map_values` +- {func}`map_zip_with` +- [match_number](match-number-function) +- {func}`max` +- {func}`max_by` +- {func}`md5` +- {func}`merge` +- {func}`merge_set_digest` +- {func}`millisecond` +- {func}`min` +- {func}`min_by` +- {func}`minute` +- {func}`mod` +- {func}`month` +- {func}`multimap_agg` +- {func}`multimap_from_entries` +- {func}`murmur3` + +## N + +- {func}`nan` +- [next](physical-navigation-functions) +- {func}`ngrams` +- {func}`none_match` +- {func}`normal_cdf` +- {func}`normalize` +- [NOT](logical-operators) +- [NOT BETWEEN](range-operator) +- {func}`now` +- {func}`nth_value` +- {func}`ntile` +- [nullif](nullif-function) +- {func}`numeric_histogram` + +## O + +- `objectid` +- {func}`objectid_timestamp` +- [OR](logical-operators) + +## P + +- {func}`parse_datetime` +- {func}`parse_duration` +- {func}`parse_data_size` +- {func}`percent_rank` +- [permute](permute-function) +- {func}`pi` +- {func}`position` +- {func}`pow` +- {func}`power` +- [prev](physical-navigation-functions) + +## Q + +- {func}`qdigest_agg` +- {func}`quarter` + +## R + +- {func}`radians` +- {func}`rand` +- {func}`random` +- {func}`rank` +- {func}`reduce` +- {func}`reduce_agg` +- {func}`regexp_count` +- {func}`regexp_extract` +- {func}`regexp_extract_all` +- {func}`regexp_like` +- {func}`regexp_position` +- {func}`regexp_replace` +- {func}`regexp_split` +- {func}`regress` +- {func}`regr_intercept` +- {func}`regr_slope` +- {func}`render` +- {func}`repeat` +- {func}`replace` +- {func}`reverse` +- {func}`rgb` +- {func}`round` +- {func}`row_number` +- {func}`rpad` +- {func}`rtrim` + +## S + +- {func}`second` +- {func}`sequence` (scalar function) +- [sequence()](sequence-table-function) (table function) +- {func}`sha1` +- {func}`sha256` +- {func}`sha512` +- {func}`shuffle` +- {func}`sign` +- {func}`simplify_geometry` +- {func}`sin` +- {func}`sinh` +- {func}`skewness` +- {func}`slice` +- [SOME](quantified-comparison-predicates) +- {func}`soundex` +- `spatial_partitioning` +- `spatial_partitions` +- {func}`split` +- {func}`split_part` +- {func}`split_to_map` +- {func}`split_to_multimap` +- {func}`spooky_hash_v2_32` +- {func}`spooky_hash_v2_64` +- {func}`sqrt` +- {func}`ST_Area` +- {func}`ST_AsBinary` +- {func}`ST_AsText` +- {func}`ST_Boundary` +- {func}`ST_Buffer` +- {func}`ST_Centroid` +- {func}`ST_Contains` +- {func}`ST_ConvexHull` +- {func}`ST_CoordDim` +- {func}`ST_Crosses` +- {func}`ST_Difference` +- {func}`ST_Dimension` +- {func}`ST_Disjoint` +- {func}`ST_Distance` +- {func}`ST_EndPoint` +- {func}`ST_Envelope` +- {func}`ST_EnvelopeAsPts` +- {func}`ST_Equals` +- {func}`ST_ExteriorRing` +- {func}`ST_Geometries` +- {func}`ST_GeometryFromText` +- {func}`ST_GeometryN` +- {func}`ST_GeometryType` +- {func}`ST_GeomFromBinary` +- {func}`ST_InteriorRingN` +- {func}`ST_InteriorRings` +- {func}`ST_Intersection` +- {func}`ST_Intersects` +- {func}`ST_IsClosed` +- {func}`ST_IsEmpty` +- {func}`ST_IsRing` +- {func}`ST_IsSimple` +- {func}`ST_IsValid` +- {func}`ST_Length` +- {func}`ST_LineFromText` +- {func}`ST_LineString` +- {func}`ST_MultiPoint` +- {func}`ST_NumGeometries` +- `ST_NumInteriorRing` +- {func}`ST_NumPoints` +- {func}`ST_Overlaps` +- {func}`ST_Point` +- {func}`ST_PointN` +- {func}`ST_Points` +- {func}`ST_Polygon` +- {func}`ST_Relate` +- {func}`ST_StartPoint` +- {func}`ST_SymDifference` +- {func}`ST_Touches` +- {func}`ST_Union` +- {func}`ST_Within` +- {func}`ST_X` +- {func}`ST_XMax` +- {func}`ST_XMin` +- {func}`ST_Y` +- {func}`ST_YMax` +- {func}`ST_YMin` +- {func}`starts_with` +- {func}`stddev` +- {func}`stddev_pop` +- {func}`stddev_samp` +- {func}`strpos` +- {func}`substr` +- {func}`substring` +- {func}`sum` + +## T + +- {func}`tan` +- {func}`tanh` +- {func}`tdigest_agg` +- {func}`timestamp_objectid` +- {func}`timezone_hour` +- {func}`timezone_minute` +- {func}`to_base` +- {func}`to_base32` +- {func}`to_base64` +- {func}`to_base64url` +- {func}`to_big_endian_32` +- {func}`to_big_endian_64` +- {func}`to_char` +- {func}`to_date` +- {func}`to_encoded_polyline` +- `to_geojson_geometry` +- {func}`to_geometry` +- {func}`to_hex` +- {func}`to_ieee754_32` +- {func}`to_ieee754_64` +- {func}`to_iso8601` +- {func}`to_milliseconds` +- {func}`to_spherical_geography` +- {func}`to_timestamp` +- {func}`to_unixtime` +- {func}`to_utf8` +- {func}`transform` +- {func}`transform_keys` +- {func}`transform_values` +- {func}`translate` +- {func}`trim` +- {func}`trim_array` +- {func}`truncate` +- [try](try-function) +- {func}`try_cast` +- {func}`typeof` + +## U + +- {func}`upper` +- {func}`url_decode` +- {func}`url_encode` +- {func}`url_extract_fragment` +- {func}`url_extract_host` +- {func}`url_extract_parameter` +- {func}`url_extract_path` +- {func}`url_extract_protocol` +- {func}`url_extract_port` +- {func}`url_extract_query` +- {func}`uuid` + +## V + +- {func}`value_at_quantile` +- {func}`values_at_quantiles` +- {func}`var_pop` +- {func}`var_samp` +- {func}`variance` +- {func}`version` + +## W + +- {func}`week` +- {func}`week_of_year` +- {func}`width_bucket` +- {func}`wilson_interval_lower` +- {func}`wilson_interval_upper` +- {func}`with_timezone` +- {func}`word_stem` + +## X + +- {func}`xxhash64` + +## Y + +- {func}`year` +- {func}`year_of_week` +- {func}`yow` + +## Z + +- {func}`zip` +- {func}`zip_with` diff --git a/430/_sources/functions/logical.md.txt b/430/_sources/functions/logical.md.txt new file mode 100644 index 000000000..7704bfc2a --- /dev/null +++ b/430/_sources/functions/logical.md.txt @@ -0,0 +1,66 @@ +(logical-operators)= + +# Logical operators + +## Logical operators + +| Operator | Description | Example | +| -------- | ---------------------------- | ------- | +| `AND` | True if both values are true | a AND b | +| `OR` | True if either value is true | a OR b | +| `NOT` | True if the value is false | NOT a | + +## Effect of NULL on logical operators + +The result of an `AND` comparison may be `NULL` if one or both +sides of the expression are `NULL`. If at least one side of an +`AND` operator is `FALSE` the expression evaluates to `FALSE`: + +``` +SELECT CAST(null AS boolean) AND true; -- null + +SELECT CAST(null AS boolean) AND false; -- false + +SELECT CAST(null AS boolean) AND CAST(null AS boolean); -- null +``` + +The result of an `OR` comparison may be `NULL` if one or both +sides of the expression are `NULL`. If at least one side of an +`OR` operator is `TRUE` the expression evaluates to `TRUE`: + +``` +SELECT CAST(null AS boolean) OR CAST(null AS boolean); -- null + +SELECT CAST(null AS boolean) OR false; -- null + +SELECT CAST(null AS boolean) OR true; -- true +``` + +The following truth table demonstrates the handling of +`NULL` in `AND` and `OR`: + +| a | b | a AND b | a OR b | +| ------- | ------- | ------- | ------- | +| `TRUE` | `TRUE` | `TRUE` | `TRUE` | +| `TRUE` | `FALSE` | `FALSE` | `TRUE` | +| `TRUE` | `NULL` | `NULL` | `TRUE` | +| `FALSE` | `TRUE` | `FALSE` | `TRUE` | +| `FALSE` | `FALSE` | `FALSE` | `FALSE` | +| `FALSE` | `NULL` | `FALSE` | `NULL` | +| `NULL` | `TRUE` | `NULL` | `TRUE` | +| `NULL` | `FALSE` | `FALSE` | `NULL` | +| `NULL` | `NULL` | `NULL` | `NULL` | + +The logical complement of `NULL` is `NULL` as shown in the following example: + +``` +SELECT NOT CAST(null AS boolean); -- null +``` + +The following truth table demonstrates the handling of `NULL` in `NOT`: + +| a | NOT a | +| ------- | ------- | +| `TRUE` | `FALSE` | +| `FALSE` | `TRUE` | +| `NULL` | `NULL` | diff --git a/430/_sources/functions/map.md.txt b/430/_sources/functions/map.md.txt new file mode 100644 index 000000000..c8cfb508b --- /dev/null +++ b/430/_sources/functions/map.md.txt @@ -0,0 +1,175 @@ +# Map functions and operators + +## Subscript operator: \[\] + +The `[]` operator is used to retrieve the value corresponding to a given key from a map: + +``` +SELECT name_to_age_map['Bob'] AS bob_age; +``` + +## Map functions + +:::{function} cardinality(x) -> bigint +:noindex: true + +Returns the cardinality (size) of the map `x`. +::: + +:::{function} element_at(map(K,V), key) -> V +:noindex: true + +Returns value for given `key`, or `NULL` if the key is not contained in the map. +::: + +:::{function} map() -> map +Returns an empty map. + +``` +SELECT map(); +-- {} +``` +::: + +:::{function} map(array(K), array(V)) -> map(K,V) +:noindex: true + +Returns a map created using the given key/value arrays. + +``` +SELECT map(ARRAY[1,3], ARRAY[2,4]); +-- {1 -> 2, 3 -> 4} +``` + +See also {func}`map_agg` and {func}`multimap_agg` for creating a map as an aggregation. +::: + +:::{function} map_from_entries(array(row(K,V))) -> map(K,V) +Returns a map created from the given array of entries. + +``` +SELECT map_from_entries(ARRAY[(1, 'x'), (2, 'y')]); +-- {1 -> 'x', 2 -> 'y'} +``` +::: + +:::{function} multimap_from_entries(array(row(K,V))) -> map(K,array(V)) +Returns a multimap created from the given array of entries. Each key can be associated with multiple values. + +``` +SELECT multimap_from_entries(ARRAY[(1, 'x'), (2, 'y'), (1, 'z')]); +-- {1 -> ['x', 'z'], 2 -> ['y']} +``` +::: + +:::{function} map_entries(map(K,V)) -> array(row(K,V)) +Returns an array of all entries in the given map. + +``` +SELECT map_entries(MAP(ARRAY[1, 2], ARRAY['x', 'y'])); +-- [ROW(1, 'x'), ROW(2, 'y')] +``` +::: + +:::{function} map_concat(map1(K,V), map2(K,V), ..., mapN(K,V)) -> map(K,V) +Returns the union of all the given maps. If a key is found in multiple given maps, +that key's value in the resulting map comes from the last one of those maps. +::: + +:::{function} map_filter(map(K,V), function(K,V,boolean)) -> map(K,V) +Constructs a map from those entries of `map` for which `function` returns true: + +``` +SELECT map_filter(MAP(ARRAY[], ARRAY[]), (k, v) -> true); +-- {} + +SELECT map_filter(MAP(ARRAY[10, 20, 30], ARRAY['a', NULL, 'c']), + (k, v) -> v IS NOT NULL); +-- {10 -> a, 30 -> c} + +SELECT map_filter(MAP(ARRAY['k1', 'k2', 'k3'], ARRAY[20, 3, 15]), + (k, v) -> v > 10); +-- {k1 -> 20, k3 -> 15} +``` +::: + +:::{function} map_keys(x(K,V)) -> array(K) +Returns all the keys in the map `x`. +::: + +:::{function} map_values(x(K,V)) -> array(V) +Returns all the values in the map `x`. +::: + +:::{function} map_zip_with(map(K,V1), map(K,V2), function(K,V1,V2,V3)) -> map(K,V3) +Merges the two given maps into a single map by applying `function` to the pair of values with the same key. +For keys only presented in one map, NULL will be passed as the value for the missing key. + +``` +SELECT map_zip_with(MAP(ARRAY[1, 2, 3], ARRAY['a', 'b', 'c']), + MAP(ARRAY[1, 2, 3], ARRAY['d', 'e', 'f']), + (k, v1, v2) -> concat(v1, v2)); +-- {1 -> ad, 2 -> be, 3 -> cf} + +SELECT map_zip_with(MAP(ARRAY['k1', 'k2'], ARRAY[1, 2]), + MAP(ARRAY['k2', 'k3'], ARRAY[4, 9]), + (k, v1, v2) -> (v1, v2)); +-- {k1 -> ROW(1, null), k2 -> ROW(2, 4), k3 -> ROW(null, 9)} + +SELECT map_zip_with(MAP(ARRAY['a', 'b', 'c'], ARRAY[1, 8, 27]), + MAP(ARRAY['a', 'b', 'c'], ARRAY[1, 2, 3]), + (k, v1, v2) -> k || CAST(v1 / v2 AS VARCHAR)); +-- {a -> a1, b -> b4, c -> c9} +``` +::: + +:::{function} transform_keys(map(K1,V), function(K1,V,K2)) -> map(K2,V) +Returns a map that applies `function` to each entry of `map` and transforms the keys: + +``` +SELECT transform_keys(MAP(ARRAY[], ARRAY[]), (k, v) -> k + 1); +-- {} + +SELECT transform_keys(MAP(ARRAY [1, 2, 3], ARRAY ['a', 'b', 'c']), + (k, v) -> k + 1); +-- {2 -> a, 3 -> b, 4 -> c} + +SELECT transform_keys(MAP(ARRAY ['a', 'b', 'c'], ARRAY [1, 2, 3]), + (k, v) -> v * v); +-- {1 -> 1, 4 -> 2, 9 -> 3} + +SELECT transform_keys(MAP(ARRAY ['a', 'b'], ARRAY [1, 2]), + (k, v) -> k || CAST(v as VARCHAR)); +-- {a1 -> 1, b2 -> 2} + +SELECT transform_keys(MAP(ARRAY [1, 2], ARRAY [1.0, 1.4]), + (k, v) -> MAP(ARRAY[1, 2], ARRAY['one', 'two'])[k]); +-- {one -> 1.0, two -> 1.4} +``` +::: + +:::{function} transform_values(map(K,V1), function(K,V1,V2)) -> map(K,V2) +Returns a map that applies `function` to each entry of `map` and transforms the values: + +``` +SELECT transform_values(MAP(ARRAY[], ARRAY[]), (k, v) -> v + 1); +-- {} + +SELECT transform_values(MAP(ARRAY [1, 2, 3], ARRAY [10, 20, 30]), + (k, v) -> v + k); +-- {1 -> 11, 2 -> 22, 3 -> 33} + +SELECT transform_values(MAP(ARRAY [1, 2, 3], ARRAY ['a', 'b', 'c']), + (k, v) -> k * k); +-- {1 -> 1, 2 -> 4, 3 -> 9} + +SELECT transform_values(MAP(ARRAY ['a', 'b'], ARRAY [1, 2]), + (k, v) -> k || CAST(v as VARCHAR)); +-- {a -> a1, b -> b2} + +SELECT transform_values(MAP(ARRAY [1, 2], ARRAY [1.0, 1.4]), + (k, v) -> MAP(ARRAY[1, 2], ARRAY['one', 'two'])[k] + || '_' || CAST(v AS VARCHAR)); +-- {1 -> one_1.0, 2 -> two_1.4} +``` +::: diff --git a/430/_sources/functions/math.md.txt b/430/_sources/functions/math.md.txt new file mode 100644 index 000000000..d0a8ece21 --- /dev/null +++ b/430/_sources/functions/math.md.txt @@ -0,0 +1,274 @@ +# Mathematical functions and operators + +(mathematical-operators)= + +## Mathematical operators + +| Operator | Description | +| -------- | ----------------------------------------------- | +| `+` | Addition | +| `-` | Subtraction | +| `*` | Multiplication | +| `/` | Division (integer division performs truncation) | +| `%` | Modulus (remainder) | + +## Mathematical functions + +:::{function} abs(x) -> [same as input] +Returns the absolute value of `x`. +::: + +:::{function} cbrt(x) -> double +Returns the cube root of `x`. +::: + +:::{function} ceil(x) -> [same as input] +This is an alias for {func}`ceiling`. +::: + +:::{function} ceiling(x) -> [same as input] +Returns `x` rounded up to the nearest integer. +::: + +:::{function} degrees(x) -> double +Converts angle `x` in radians to degrees. +::: + +:::{function} e() -> double +Returns the constant Euler's number. +::: + +:::{function} exp(x) -> double +Returns Euler's number raised to the power of `x`. +::: + +:::{function} floor(x) -> [same as input] +Returns `x` rounded down to the nearest integer. +::: + +:::{function} ln(x) -> double +Returns the natural logarithm of `x`. +::: + +:::{function} log(b, x) -> double +Returns the base `b` logarithm of `x`. +::: + +:::{function} log2(x) -> double +Returns the base 2 logarithm of `x`. +::: + +:::{function} log10(x) -> double +Returns the base 10 logarithm of `x`. +::: + +:::{function} mod(n, m) -> [same as input] +Returns the modulus (remainder) of `n` divided by `m`. +::: + +:::{function} pi() -> double +Returns the constant Pi. +::: + +:::{function} pow(x, p) -> double +This is an alias for {func}`power`. +::: + +:::{function} power(x, p) -> double +Returns `x` raised to the power of `p`. +::: + +:::{function} radians(x) -> double +Converts angle `x` in degrees to radians. +::: + +:::{function} round(x) -> [same as input] +Returns `x` rounded to the nearest integer. +::: + +:::{function} round(x, d) -> [same as input] +:noindex: true + +Returns `x` rounded to `d` decimal places. +::: + +:::{function} sign(x) -> [same as input] +Returns the signum function of `x`, that is: + +- 0 if the argument is 0, +- 1 if the argument is greater than 0, +- -1 if the argument is less than 0. + +For double arguments, the function additionally returns: + +- NaN if the argument is NaN, +- 1 if the argument is +Infinity, +- -1 if the argument is -Infinity. +::: + +:::{function} sqrt(x) -> double +Returns the square root of `x`. +::: + +:::{function} truncate(x) -> double +Returns `x` rounded to integer by dropping digits after decimal point. +::: + +:::{function} width_bucket(x, bound1, bound2, n) -> bigint +Returns the bin number of `x` in an equi-width histogram with the +specified `bound1` and `bound2` bounds and `n` number of buckets. +::: + +:::{function} width_bucket(x, bins) -> bigint +:noindex: true + +Returns the bin number of `x` according to the bins specified by the +array `bins`. The `bins` parameter must be an array of doubles and is +assumed to be in sorted ascending order. +::: + +## Random functions + +:::{function} rand() -> double +This is an alias for {func}`random()`. +::: + +:::{function} random() -> double +Returns a pseudo-random value in the range 0.0 \<= x \< 1.0. +::: + +:::{function} random(n) -> [same as input] +:noindex: true + +Returns a pseudo-random number between 0 and n (exclusive). +::: + +:::{function} random(m, n) -> [same as input] +:noindex: true + +Returns a pseudo-random number between m and n (exclusive). +::: + +## Trigonometric functions + +All trigonometric function arguments are expressed in radians. +See unit conversion functions {func}`degrees` and {func}`radians`. + +:::{function} acos(x) -> double +Returns the arc cosine of `x`. +::: + +:::{function} asin(x) -> double +Returns the arc sine of `x`. +::: + +:::{function} atan(x) -> double +Returns the arc tangent of `x`. +::: + +:::{function} atan2(y, x) -> double +Returns the arc tangent of `y / x`. +::: + +:::{function} cos(x) -> double +Returns the cosine of `x`. +::: + +:::{function} cosh(x) -> double +Returns the hyperbolic cosine of `x`. +::: + +:::{function} sin(x) -> double +Returns the sine of `x`. +::: + +:::{function} sinh(x) -> double +Returns the hyperbolic sine of `x`. +::: + +:::{function} tan(x) -> double +Returns the tangent of `x`. +::: + +:::{function} tanh(x) -> double +Returns the hyperbolic tangent of `x`. +::: + +## Floating point functions + +:::{function} infinity() -> double +Returns the constant representing positive infinity. +::: + +:::{function} is_finite(x) -> boolean +Determine if `x` is finite. +::: + +:::{function} is_infinite(x) -> boolean +Determine if `x` is infinite. +::: + +:::{function} is_nan(x) -> boolean +Determine if `x` is not-a-number. +::: + +:::{function} nan() -> double +Returns the constant representing not-a-number. +::: + +## Base conversion functions + +:::{function} from_base(string, radix) -> bigint +Returns the value of `string` interpreted as a base-`radix` number. +::: + +:::{function} to_base(x, radix) -> varchar +Returns the base-`radix` representation of `x`. +::: + +## Statistical functions + +:::{function} cosine_similarity(x, y) -> double +Returns the cosine similarity between the sparse vectors `x` and `y`: + +``` +SELECT cosine_similarity(MAP(ARRAY['a'], ARRAY[1.0]), MAP(ARRAY['a'], ARRAY[2.0])); -- 1.0 +``` +::: + +:::{function} wilson_interval_lower(successes, trials, z) -> double +Returns the lower bound of the Wilson score interval of a Bernoulli trial process +at a confidence specified by the z-score `z`. +::: + +:::{function} wilson_interval_upper(successes, trials, z) -> double +Returns the upper bound of the Wilson score interval of a Bernoulli trial process +at a confidence specified by the z-score `z`. +::: + +## Cumulative distribution functions + +:::{function} beta_cdf(a, b, v) -> double +Compute the Beta cdf with given a, b parameters: P(N \< v; a, b). +The a, b parameters must be positive real numbers and value v must be a real value. +The value v must lie on the interval \[0, 1\]. +::: + +:::{function} inverse_beta_cdf(a, b, p) -> double +Compute the inverse of the Beta cdf with given a, b parameters for the cumulative +probability (p): P(N \< n). The a, b parameters must be positive real values. +The probability p must lie on the interval \[0, 1\]. +::: + +:::{function} inverse_normal_cdf(mean, sd, p) -> double +Compute the inverse of the Normal cdf with given mean and standard +deviation (sd) for the cumulative probability (p): P(N \< n). The mean must be +a real value and the standard deviation must be a real and positive value. +The probability p must lie on the interval (0, 1). +::: + +:::{function} normal_cdf(mean, sd, v) -> double +Compute the Normal cdf with given mean and standard deviation (sd): P(N \< v; mean, sd). +The mean and value v must be real values and the standard deviation must be a real +and positive value. +::: diff --git a/430/_sources/functions/ml.md.txt b/430/_sources/functions/ml.md.txt new file mode 100644 index 000000000..44f4c46f8 --- /dev/null +++ b/430/_sources/functions/ml.md.txt @@ -0,0 +1,157 @@ +# Machine learning functions + +The machine learning plugin provides machine learning functionality +as an aggregation function. It enables you to train Support Vector Machine (SVM) +based classifiers and regressors for the supervised learning problems. + +:::{note} +The machine learning functions are not optimized for distributed processing. +The capability to train large data sets is limited by this execution of the +final training on a single instance. +::: + +## Feature vector + +To solve a problem with the machine learning technique, especially as a +supervised learning problem, it is necessary to represent the data set +with the sequence of pairs of labels and feature vector. A label is a +target value you want to predict from the unseen feature and a feature is a +A N-dimensional vector whose elements are numerical values. In Trino, a +feature vector is represented as a map-type value, whose key is an index +of each feature, so that it can express a sparse vector. +Since classifiers and regressors can recognize the map-type feature +vector, there is a function to construct the feature from the existing +numerical values, {func}`features`: + +``` +SELECT features(1.0, 2.0, 3.0) AS features; +``` + +```text + features +----------------------- + {0=1.0, 1=2.0, 2=3.0} +``` + +The output from {func}`features` can be directly passed to ML functions. + +## Classification + +Classification is a type of supervised learning problem to predict the distinct +label from the given feature vector. The interface looks similar to the +construction of the SVM model from the sequence of pairs of labels and features +implemented in Teradata Aster or [BigQuery ML](https://cloud.google.com/bigquery-ml/docs/bigqueryml-intro). +The function to train a classification model looks like as follows: + +``` +SELECT + learn_classifier( + species, + features(sepal_length, sepal_width, petal_length, petal_width) + ) AS model +FROM + iris +``` + +It returns the trained model in a serialized format. + +```text + model +------------------------------------------------- + 3c 43 6c 61 73 73 69 66 69 65 72 28 76 61 72 63 + 68 61 72 29 3e +``` + +{func}`classify` returns the predicted label by using the trained model. +The trained model can not be saved natively, and needs to be passed in +the format of a nested query: + +``` +SELECT + classify(features(5.9, 3, 5.1, 1.8), model) AS predicted_label +FROM ( + SELECT + learn_classifier(species, features(sepal_length, sepal_width, petal_length, petal_width)) AS model + FROM + iris +) t +``` + +```text + predicted_label +----------------- + Iris-virginica +``` + +As a result you need to run the training process at the same time when predicting values. +Internally, the model is trained by [libsvm](https://www.csie.ntu.edu.tw/~cjlin/libsvm/). +You can use {func}`learn_libsvm_classifier` to control the internal parameters of the model. + +## Regression + +Regression is another type of supervised learning problem, predicting continuous +value, unlike the classification problem. The target must be numerical values that can +be described as `double`. + +The following code shows the creation of the model predicting `sepal_length` +from the other 3 features: + +``` +SELECT + learn_regressor(sepal_length, features(sepal_width, petal_length, petal_width)) AS model +FROM + iris +``` + +The way to use the model is similar to the classification case: + +``` +SELECT + regress(features(3, 5.1, 1.8), model) AS predicted_target +FROM ( + SELECT + learn_regressor(sepal_length, features(sepal_width, petal_length, petal_width)) AS model + FROM iris +) t; +``` + +```text + predicted_target +------------------- + 6.407376822560477 +``` + +Internally, the model is trained by [libsvm](https://www.csie.ntu.edu.tw/~cjlin/libsvm/). +{func}`learn_libsvm_regressor` provides you a way to control the training process. + +## Machine learning functions + +:::{function} features(double, ...) -> map(bigint, double) +Returns the map representing the feature vector. +::: + +:::{function} learn_classifier(label, features) -> Classifier +Returns an SVM-based classifier model, trained with the given label and feature data sets. +::: + +:::{function} learn_libsvm_classifier(label, features, params) -> Classifier +Returns an SVM-based classifier model, trained with the given label and feature data sets. +You can control the training process by libsvm parameters. +::: + +:::{function} classify(features, model) -> label +Returns a label predicted by the given classifier SVM model. +::: + +:::{function} learn_regressor(target, features) -> Regressor +Returns an SVM-based regressor model, trained with the given target and feature data sets. +::: + +:::{function} learn_libsvm_regressor(target, features, params) -> Regressor +Returns an SVM-based regressor model, trained with the given target and feature data sets. +You can control the training process by libsvm parameters. +::: + +:::{function} regress(features, model) -> target +Returns a predicted target value by the given regressor SVM model. +::: diff --git a/430/_sources/functions/qdigest.md.txt b/430/_sources/functions/qdigest.md.txt new file mode 100644 index 000000000..d454ff238 --- /dev/null +++ b/430/_sources/functions/qdigest.md.txt @@ -0,0 +1,55 @@ +# Quantile digest functions + +## Data structures + +A quantile digest is a data sketch which stores approximate percentile +information. The Trino type for this data structure is called `qdigest`, +and it takes a parameter which must be one of `bigint`, `double` or +`real` which represent the set of numbers that may be ingested by the +`qdigest`. They may be merged without losing precision, and for storage +and retrieval they may be cast to/from `VARBINARY`. + +## Functions + +:::{function} merge(qdigest) -> qdigest +:noindex: true + +Merges all input `qdigest`s into a single `qdigest`. +::: + +:::{function} value_at_quantile(qdigest(T), quantile) -> T +Returns the approximate percentile value from the quantile digest given +the number `quantile` between 0 and 1. +::: + +:::{function} quantile_at_value(qdigest(T), T) -> quantile +Returns the approximate `quantile` number between 0 and 1 from the +quantile digest given an input value. Null is returned if the quantile digest +is empty or the input value is outside of the range of the quantile digest. +::: + +:::{function} values_at_quantiles(qdigest(T), quantiles) -> array(T) +Returns the approximate percentile values as an array given the input +quantile digest and array of values between 0 and 1 which +represent the quantiles to return. +::: + +:::{function} qdigest_agg(x) -> qdigest([same as x]) +Returns the `qdigest` which is composed of all input values of `x`. +::: + +:::{function} qdigest_agg(x, w) -> qdigest([same as x]) +:noindex: true + +Returns the `qdigest` which is composed of all input values of `x` using +the per-item weight `w`. +::: + +:::{function} qdigest_agg(x, w, accuracy) -> qdigest([same as x]) +:noindex: true + +Returns the `qdigest` which is composed of all input values of `x` using +the per-item weight `w` and maximum error of `accuracy`. `accuracy` +must be a value greater than zero and less than one, and it must be constant +for all input rows. +::: diff --git a/430/_sources/functions/regexp.md.txt b/430/_sources/functions/regexp.md.txt new file mode 100644 index 000000000..cbc853e0e --- /dev/null +++ b/430/_sources/functions/regexp.md.txt @@ -0,0 +1,189 @@ +# Regular expression functions + +All of the regular expression functions use the [Java pattern] syntax, +with a few notable exceptions: + +- When using multi-line mode (enabled via the `(?m)` flag), + only `\n` is recognized as a line terminator. Additionally, + the `(?d)` flag is not supported and must not be used. + +- Case-insensitive matching (enabled via the `(?i)` flag) is always + performed in a Unicode-aware manner. However, context-sensitive and + local-sensitive matching is not supported. Additionally, the + `(?u)` flag is not supported and must not be used. + +- Surrogate pairs are not supported. For example, `\uD800\uDC00` is + not treated as `U+10000` and must be specified as `\x{10000}`. + +- Boundaries (`\b`) are incorrectly handled for a non-spacing mark + without a base character. + +- `\Q` and `\E` are not supported in character classes + (such as `[A-Z123]`) and are instead treated as literals. + +- Unicode character classes (`\p{prop}`) are supported with + the following differences: + + - All underscores in names must be removed. For example, use + `OldItalic` instead of `Old_Italic`. + + - Scripts must be specified directly, without the + `Is`, `script=` or `sc=` prefixes. + Example: `\p{Hiragana}` + + - Blocks must be specified with the `In` prefix. + The `block=` and `blk=` prefixes are not supported. + Example: `\p{Mongolian}` + + - Categories must be specified directly, without the `Is`, + `general_category=` or `gc=` prefixes. + Example: `\p{L}` + + - Binary properties must be specified directly, without the `Is`. + Example: `\p{NoncharacterCodePoint}` + +:::{function} regexp_count(string, pattern) -> bigint +Returns the number of occurrence of `pattern` in `string`: + +``` +SELECT regexp_count('1a 2b 14m', '\s*[a-z]+\s*'); -- 3 +``` +::: + +:::{function} regexp_extract_all(string, pattern) -> array(varchar) +Returns the substring(s) matched by the regular expression `pattern` +in `string`: + +``` +SELECT regexp_extract_all('1a 2b 14m', '\d+'); -- [1, 2, 14] +``` +::: + +:::{function} regexp_extract_all(string, pattern, group) -> array(varchar) +:noindex: true + +Finds all occurrences of the regular expression `pattern` in `string` +and returns the [capturing group number] `group`: + +``` +SELECT regexp_extract_all('1a 2b 14m', '(\d+)([a-z]+)', 2); -- ['a', 'b', 'm'] +``` +::: + +:::{function} regexp_extract(string, pattern) -> varchar +Returns the first substring matched by the regular expression `pattern` +in `string`: + +``` +SELECT regexp_extract('1a 2b 14m', '\d+'); -- 1 +``` +::: + +:::{function} regexp_extract(string, pattern, group) -> varchar +:noindex: true + +Finds the first occurrence of the regular expression `pattern` in +`string` and returns the [capturing group number] `group`: + +``` +SELECT regexp_extract('1a 2b 14m', '(\d+)([a-z]+)', 2); -- 'a' +``` +::: + +:::{function} regexp_like(string, pattern) -> boolean +Evaluates the regular expression `pattern` and determines if it is +contained within `string`. + +The `pattern` only needs to be contained within +`string`, rather than needing to match all of `string`. In other words, +this performs a *contains* operation rather than a *match* operation. You can +match the entire string by anchoring the pattern using `^` and `$`: + +``` +SELECT regexp_like('1a 2b 14m', '\d+b'); -- true +``` +::: + +:::{function} regexp_position(string, pattern) -> integer +Returns the index of the first occurrence (counting from 1) of `pattern` in `string`. +Returns -1 if not found: + +``` +SELECT regexp_position('I have 23 apples, 5 pears and 13 oranges', '\b\d+\b'); -- 8 +``` +::: + +:::{function} regexp_position(string, pattern, start) -> integer +:noindex: true + +Returns the index of the first occurrence of `pattern` in `string`, +starting from `start` (include `start`). Returns -1 if not found: + +``` +SELECT regexp_position('I have 23 apples, 5 pears and 13 oranges', '\b\d+\b', 5); -- 8 +SELECT regexp_position('I have 23 apples, 5 pears and 13 oranges', '\b\d+\b', 12); -- 19 +``` +::: + +:::{function} regexp_position(string, pattern, start, occurrence) -> integer +:noindex: true + +Returns the index of the nth `occurrence` of `pattern` in `string`, +starting from `start` (include `start`). Returns -1 if not found: + +``` +SELECT regexp_position('I have 23 apples, 5 pears and 13 oranges', '\b\d+\b', 12, 1); -- 19 +SELECT regexp_position('I have 23 apples, 5 pears and 13 oranges', '\b\d+\b', 12, 2); -- 31 +SELECT regexp_position('I have 23 apples, 5 pears and 13 oranges', '\b\d+\b', 12, 3); -- -1 +``` +::: + +:::{function} regexp_replace(string, pattern) -> varchar +Removes every instance of the substring matched by the regular expression +`pattern` from `string`: + +``` +SELECT regexp_replace('1a 2b 14m', '\d+[ab] '); -- '14m' +``` +::: + +:::{function} regexp_replace(string, pattern, replacement) -> varchar +:noindex: true + +Replaces every instance of the substring matched by the regular expression +`pattern` in `string` with `replacement`. [Capturing groups] can be +referenced in `replacement` using `$g` for a numbered group or +`${name}` for a named group. A dollar sign (`$`) may be included in the +replacement by escaping it with a backslash (`\$`): + +``` +SELECT regexp_replace('1a 2b 14m', '(\d+)([ab]) ', '3c$2 '); -- '3ca 3cb 14m' +``` +::: + +:::{function} regexp_replace(string, pattern, function) -> varchar +:noindex: true + +Replaces every instance of the substring matched by the regular expression +`pattern` in `string` using `function`. The {doc}`lambda expression ` +`function` is invoked for each match with the [capturing groups] passed as an +array. Capturing group numbers start at one; there is no group for the entire match +(if you need this, surround the entire expression with parenthesis). + +``` +SELECT regexp_replace('new york', '(\w)(\w*)', x -> upper(x[1]) || lower(x[2])); --'New York' +``` +::: + +:::{function} regexp_split(string, pattern) -> array(varchar) +Splits `string` using the regular expression `pattern` and returns an +array. Trailing empty strings are preserved: + +``` +SELECT regexp_split('1a 2b 14m', '\s*[a-z]+\s*'); -- [1, 2, 14, ] +``` +::: + +[capturing group number]: https://docs.oracle.com/en/java/javase/17/docs/api/java.base/java/util/regex/Pattern.html#gnumber +[capturing groups]: https://docs.oracle.com/en/java/javase/17/docs/api/java.base/java/util/regex/Pattern.html#cg +[java pattern]: https://docs.oracle.com/en/java/javase/17/docs/api/java.base/java/util/regex/Pattern.html diff --git a/430/_sources/functions/session.md.txt b/430/_sources/functions/session.md.txt new file mode 100644 index 000000000..206a333fe --- /dev/null +++ b/430/_sources/functions/session.md.txt @@ -0,0 +1,23 @@ +# Session information + +Functions providing information about the query execution environment. + +:::{data} current_user +Returns the current user running the query. +::: + +:::{function} current_groups +Returns the list of groups for the current user running the query. +::: + +:::{data} current_catalog +Returns a character string that represents the current catalog name. +::: + +::::{data} current_schema +Returns a character string that represents the current unqualified schema name. + +:::{note} +This is part of the SQL standard and does not use parenthesis. +::: +:::: diff --git a/430/_sources/functions/setdigest.md.txt b/430/_sources/functions/setdigest.md.txt new file mode 100644 index 000000000..37bef87b2 --- /dev/null +++ b/430/_sources/functions/setdigest.md.txt @@ -0,0 +1,187 @@ +# Set Digest functions + +Trino offers several functions that deal with the +[MinHash](https://wikipedia.org/wiki/MinHash) technique. + +MinHash is used to quickly estimate the +[Jaccard similarity coefficient](https://wikipedia.org/wiki/Jaccard_index) +between two sets. + +It is commonly used in data mining to detect near-duplicate web pages at scale. +By using this information, the search engines efficiently avoid showing +within the search results two pages that are nearly identical. + +The following example showcases how the Set Digest functions can be +used to naively estimate the similarity between texts. The input texts +are split by using the function {func}`ngrams` to +[4-shingles](https://wikipedia.org/wiki/W-shingling) which are +used as input for creating a set digest of each initial text. +The set digests are compared to each other to get an +approximation of the similarity of their corresponding +initial texts: + +``` +WITH text_input(id, text) AS ( + VALUES + (1, 'The quick brown fox jumps over the lazy dog'), + (2, 'The quick and the lazy'), + (3, 'The quick brown fox jumps over the dog') + ), + text_ngrams(id, ngrams) AS ( + SELECT id, + transform( + ngrams( + split(text, ' '), + 4 + ), + token -> array_join(token, ' ') + ) + FROM text_input + ), + minhash_digest(id, digest) AS ( + SELECT id, + (SELECT make_set_digest(v) FROM unnest(ngrams) u(v)) + FROM text_ngrams + ), + setdigest_side_by_side(id1, digest1, id2, digest2) AS ( + SELECT m1.id as id1, + m1.digest as digest1, + m2.id as id2, + m2.digest as digest2 + FROM (SELECT id, digest FROM minhash_digest) m1 + JOIN (SELECT id, digest FROM minhash_digest) m2 + ON m1.id != m2.id AND m1.id < m2.id + ) +SELECT id1, + id2, + intersection_cardinality(digest1, digest2) AS intersection_cardinality, + jaccard_index(digest1, digest2) AS jaccard_index +FROM setdigest_side_by_side +ORDER BY id1, id2; +``` + +```text + id1 | id2 | intersection_cardinality | jaccard_index +-----+-----+--------------------------+--------------- + 1 | 2 | 0 | 0.0 + 1 | 3 | 4 | 0.6 + 2 | 3 | 0 | 0.0 +``` + +The above result listing points out, as expected, that the texts +with the id `1` and `3` are quite similar. + +One may argue that the text with the id `2` is somewhat similar to +the texts with the id `1` and `3`. Due to the fact in the example above +*4-shingles* are taken into account for measuring the similarity of the texts, +there are no intersections found for the text pairs `1` and `2`, respectively +`3` and `2` and therefore there the similarity index for these text pairs +is `0`. + +## Data structures + +Trino implements Set Digest data sketches by encapsulating the following components: + +- [HyperLogLog](https://wikipedia.org/wiki/HyperLogLog) +- [MinHash with a single hash function](http://wikipedia.org/wiki/MinHash#Variant_with_a_single_hash_function) + +The HyperLogLog structure is used for the approximation of the distinct elements +in the original set. + +The MinHash structure is used to store a low memory footprint signature of the original set. +The similarity of any two sets is estimated by comparing their signatures. + +The Trino type for this data structure is called `setdigest`. +Trino offers the ability to merge multiple Set Digest data sketches. + +## Serialization + +Data sketches can be serialized to and deserialized from `varbinary`. This +allows them to be stored for later use. + +## Functions + +:::{function} make_set_digest(x) -> setdigest +Composes all input values of `x` into a `setdigest`. + +Create a `setdigest` corresponding to a `bigint` array: + +``` +SELECT make_set_digest(value) +FROM (VALUES 1, 2, 3) T(value); +``` + +Create a `setdigest` corresponding to a `varchar` array: + +``` +SELECT make_set_digest(value) +FROM (VALUES 'Trino', 'SQL', 'on', 'everything') T(value); +``` +::: + +:::{function} merge_set_digest(setdigest) -> setdigest +Returns the `setdigest` of the aggregate union of the individual `setdigest` +Set Digest structures. +::: + +(setdigest-cardinality)= + +:::{function} cardinality(setdigest) -> long +:noindex: true + +Returns the cardinality of the set digest from its internal +`HyperLogLog` component. + +Examples: + +``` +SELECT cardinality(make_set_digest(value)) +FROM (VALUES 1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5) T(value); +-- 5 +``` +::: + +:::{function} intersection_cardinality(x,y) -> long +Returns the estimation for the cardinality of the intersection of the two set digests. + +`x` and `y` must be of type `setdigest` + +Examples: + +``` +SELECT intersection_cardinality(make_set_digest(v1), make_set_digest(v2)) +FROM (VALUES (1, 1), (NULL, 2), (2, 3), (3, 4)) T(v1, v2); +-- 3 +``` +::: + +:::{function} jaccard_index(x, y) -> double +Returns the estimation of [Jaccard index](https://wikipedia.org/wiki/Jaccard_index) for +the two set digests. + +`x` and `y` must be of type `setdigest`. + +Examples: + +``` +SELECT jaccard_index(make_set_digest(v1), make_set_digest(v2)) +FROM (VALUES (1, 1), (NULL,2), (2, 3), (NULL, 4)) T(v1, v2); +-- 0.5 +``` +::: + +:::{function} hash_counts(x) -> map(bigint, smallint) +Returns a map containing the [Murmur3Hash128](https://wikipedia.org/wiki/MurmurHash#MurmurHash3) +hashed values and the count of their occurences within +the internal `MinHash` structure belonging to `x`. + +`x` must be of type `setdigest`. + +Examples: + +``` +SELECT hash_counts(make_set_digest(value)) +FROM (VALUES 1, 1, 1, 2, 2) T(value); +-- {19144387141682250=3, -2447670524089286488=2} +``` +::: diff --git a/430/_sources/functions/string.md.txt b/430/_sources/functions/string.md.txt new file mode 100644 index 000000000..81e7b0755 --- /dev/null +++ b/430/_sources/functions/string.md.txt @@ -0,0 +1,341 @@ +# String functions and operators + +## String operators + +The `||` operator performs concatenation. + +The `LIKE` statement can be used for pattern matching and is documented in +{ref}`like-operator`. + +## String functions + +:::{note} +These functions assume that the input strings contain valid UTF-8 encoded +Unicode code points. There are no explicit checks for valid UTF-8 and +the functions may return incorrect results on invalid UTF-8. +Invalid UTF-8 data can be corrected with {func}`from_utf8`. + +Additionally, the functions operate on Unicode code points and not user +visible *characters* (or *grapheme clusters*). Some languages combine +multiple code points into a single user-perceived *character*, the basic +unit of a writing system for a language, but the functions will treat each +code point as a separate unit. + +The {func}`lower` and {func}`upper` functions do not perform +locale-sensitive, context-sensitive, or one-to-many mappings required for +some languages. Specifically, this will return incorrect results for +Lithuanian, Turkish and Azeri. +::: + +:::{function} chr(n) -> varchar +Returns the Unicode code point `n` as a single character string. +::: + +:::{function} codepoint(string) -> integer +Returns the Unicode code point of the only character of `string`. +::: + +:::{function} concat(string1, ..., stringN) -> varchar +Returns the concatenation of `string1`, `string2`, `...`, `stringN`. +This function provides the same functionality as the +SQL-standard concatenation operator (`||`). +::: + +:::{function} concat_ws(string0, string1, ..., stringN) -> varchar +Returns the concatenation of `string1`, `string2`, `...`, `stringN` +using `string0` as a separator. If `string0` is null, then the return +value is null. Any null values provided in the arguments after the +separator are skipped. +::: + +:::{function} concat_ws(string0, array(varchar)) -> varchar +:noindex: true + +Returns the concatenation of elements in the array using `string0` as a +separator. If `string0` is null, then the return value is null. Any +null values in the array are skipped. +::: + +:::{function} format(format, args...) -> varchar +:noindex: true + +See {func}`format`. +::: + +:::{function} hamming_distance(string1, string2) -> bigint +Returns the Hamming distance of `string1` and `string2`, +i.e. the number of positions at which the corresponding characters are different. +Note that the two strings must have the same length. +::: + +:::{function} length(string) -> bigint +Returns the length of `string` in characters. +::: + +:::{function} levenshtein_distance(string1, string2) -> bigint +Returns the Levenshtein edit distance of `string1` and `string2`, +i.e. the minimum number of single-character edits (insertions, +deletions or substitutions) needed to change `string1` into `string2`. +::: + +:::{function} lower(string) -> varchar +Converts `string` to lowercase. +::: + +:::{function} lpad(string, size, padstring) -> varchar +Left pads `string` to `size` characters with `padstring`. +If `size` is less than the length of `string`, the result is +truncated to `size` characters. `size` must not be negative +and `padstring` must be non-empty. +::: + +:::{function} ltrim(string) -> varchar +Removes leading whitespace from `string`. +::: + +:::{function} luhn_check(string) -> boolean +Tests whether a `string` of digits is valid according to the +[Luhn algorithm](https://wikipedia.org/wiki/Luhn_algorithm). + +This checksum function, also known as `modulo 10` or `mod 10`, is +widely applied on credit card numbers and government identification numbers +to distinguish valid numbers from mistyped, incorrect numbers. + +Valid identification number: + +``` +select luhn_check('79927398713'); +-- true +``` + +Invalid identification number: + +``` +select luhn_check('79927398714'); +-- false +``` +::: + +::::{function} position(substring IN string) -> bigint +Returns the starting position of the first instance of `substring` in +`string`. Positions start with `1`. If not found, `0` is returned. + +:::{note} +This SQL-standard function has special syntax and uses the +`IN` keyword for the arguments. See also {func}`strpos`. +::: +:::: + +:::{function} replace(string, search) -> varchar +Removes all instances of `search` from `string`. +::: + +:::{function} replace(string, search, replace) -> varchar +:noindex: true + +Replaces all instances of `search` with `replace` in `string`. +::: + +:::{function} reverse(string) -> varchar +Returns `string` with the characters in reverse order. +::: + +:::{function} rpad(string, size, padstring) -> varchar +Right pads `string` to `size` characters with `padstring`. +If `size` is less than the length of `string`, the result is +truncated to `size` characters. `size` must not be negative +and `padstring` must be non-empty. +::: + +:::{function} rtrim(string) -> varchar +Removes trailing whitespace from `string`. +::: + +:::{function} soundex(char) -> string +`soundex` returns a character string containing the phonetic representation of `char`. + +: It is typically used to evaluate the similarity of two expressions phonetically, that is + how the string sounds when spoken: + + ``` + SELECT name + FROM nation + WHERE SOUNDEX(name) = SOUNDEX('CHYNA'); + + name | + -------+---- + CHINA | + (1 row) + ``` +::: + +:::{function} split(string, delimiter) -> array(varchar) +Splits `string` on `delimiter` and returns an array. +::: + +:::{function} split(string, delimiter, limit) -> array(varchar) +:noindex: true + +Splits `string` on `delimiter` and returns an array of size at most +`limit`. The last element in the array always contain everything +left in the `string`. `limit` must be a positive number. +::: + +:::{function} split_part(string, delimiter, index) -> varchar +Splits `string` on `delimiter` and returns the field `index`. +Field indexes start with `1`. If the index is larger than +the number of fields, then null is returned. +::: + +:::{function} split_to_map(string, entryDelimiter, keyValueDelimiter) -> map +Splits `string` by `entryDelimiter` and `keyValueDelimiter` and returns a map. +`entryDelimiter` splits `string` into key-value pairs. `keyValueDelimiter` splits +each pair into key and value. +::: + +:::{function} split_to_multimap(string, entryDelimiter, keyValueDelimiter) -> map(varchar, array(varchar)) +Splits `string` by `entryDelimiter` and `keyValueDelimiter` and returns a map +containing an array of values for each unique key. `entryDelimiter` splits `string` +into key-value pairs. `keyValueDelimiter` splits each pair into key and value. The +values for each key will be in the same order as they appeared in `string`. +::: + +:::{function} strpos(string, substring) -> bigint +Returns the starting position of the first instance of `substring` in +`string`. Positions start with `1`. If not found, `0` is returned. +::: + +:::{function} strpos(string, substring, instance) -> bigint +:noindex: true + +Returns the position of the N-th `instance` of `substring` in `string`. +When `instance` is a negative number the search will start from the end of `string`. +Positions start with `1`. If not found, `0` is returned. +::: + +:::{function} starts_with(string, substring) -> boolean +Tests whether `substring` is a prefix of `string`. +::: + +:::{function} substr(string, start) -> varchar +This is an alias for {func}`substring`. +::: + +:::{function} substring(string, start) -> varchar +Returns the rest of `string` from the starting position `start`. +Positions start with `1`. A negative starting position is interpreted +as being relative to the end of the string. +::: + +:::{function} substr(string, start, length) -> varchar +:noindex: true + +This is an alias for {func}`substring`. +::: + +:::{function} substring(string, start, length) -> varchar +:noindex: true + +Returns a substring from `string` of length `length` from the starting +position `start`. Positions start with `1`. A negative starting +position is interpreted as being relative to the end of the string. +::: + +:::{function} translate(source, from, to) -> varchar +Returns the `source` string translated by replacing characters found in the +`from` string with the corresponding characters in the `to` string. If the `from` +string contains duplicates, only the first is used. If the `source` character +does not exist in the `from` string, the `source` character will be copied +without translation. If the index of the matching character in the `from` +string is beyond the length of the `to` string, the `source` character will +be omitted from the resulting string. + +Here are some examples illustrating the translate function: + +``` +SELECT translate('abcd', '', ''); -- 'abcd' +SELECT translate('abcd', 'a', 'z'); -- 'zbcd' +SELECT translate('abcda', 'a', 'z'); -- 'zbcdz' +SELECT translate('Palhoça', 'ç','c'); -- 'Palhoca' +SELECT translate('abcd', 'b', U&'\+01F600'); -- a😀cd +SELECT translate('abcd', 'a', ''); -- 'bcd' +SELECT translate('abcd', 'a', 'zy'); -- 'zbcd' +SELECT translate('abcd', 'ac', 'z'); -- 'zbd' +SELECT translate('abcd', 'aac', 'zq'); -- 'zbd' +``` +::: + +:::{function} trim(string) -> varchar +:noindex: true + +Removes leading and trailing whitespace from `string`. +::: + +:::{function} trim( [ [ specification ] [ string ] FROM ] source ) -> varchar +Removes any leading and/or trailing characters as specified up to and +including `string` from `source`: + +``` +SELECT trim('!' FROM '!foo!'); -- 'foo' +SELECT trim(LEADING FROM ' abcd'); -- 'abcd' +SELECT trim(BOTH '$' FROM '$var$'); -- 'var' +SELECT trim(TRAILING 'ER' FROM upper('worker')); -- 'WORK' +``` +::: + +:::{function} upper(string) -> varchar +Converts `string` to uppercase. +::: + +:::{function} word_stem(word) -> varchar +Returns the stem of `word` in the English language. +::: + +:::{function} word_stem(word, lang) -> varchar +:noindex: true + +Returns the stem of `word` in the `lang` language. +::: + +## Unicode functions + +:::{function} normalize(string) -> varchar +Transforms `string` with NFC normalization form. +::: + +::::{function} normalize(string, form) -> varchar +:noindex: true + +Transforms `string` with the specified normalization form. +`form` must be one of the following keywords: + +| Form | Description | +| ------ | -------------------------------------------------------------- | +| `NFD` | Canonical Decomposition | +| `NFC` | Canonical Decomposition, followed by Canonical Composition | +| `NFKD` | Compatibility Decomposition | +| `NFKC` | Compatibility Decomposition, followed by Canonical Composition | + +:::{note} +This SQL-standard function has special syntax and requires +specifying `form` as a keyword, not as a string. +::: +:::: + +:::{function} to_utf8(string) -> varbinary +Encodes `string` into a UTF-8 varbinary representation. +::: + +:::{function} from_utf8(binary) -> varchar +Decodes a UTF-8 encoded string from `binary`. Invalid UTF-8 sequences +are replaced with the Unicode replacement character `U+FFFD`. +::: + +:::{function} from_utf8(binary, replace) -> varchar +:noindex: true + +Decodes a UTF-8 encoded string from `binary`. Invalid UTF-8 sequences +are replaced with `replace`. The replacement string `replace` must either +be a single character or empty (in which case invalid characters are +removed). +::: diff --git a/430/_sources/functions/system.md.txt b/430/_sources/functions/system.md.txt new file mode 100644 index 000000000..c88dd45b5 --- /dev/null +++ b/430/_sources/functions/system.md.txt @@ -0,0 +1,10 @@ +# System information + +Functions providing information about the Trino cluster system environment. More +information is available by querying the various schemas and tables exposed by +the {doc}`/connector/system`. + +:::{function} version() -> varchar +Returns the Trino version used on the cluster. Equivalent to the value of +the `node_version` column in the `system.runtime.nodes` table. +::: diff --git a/430/_sources/functions/table.md.txt b/430/_sources/functions/table.md.txt new file mode 100644 index 000000000..a06fa4f13 --- /dev/null +++ b/430/_sources/functions/table.md.txt @@ -0,0 +1,192 @@ +# Table functions + +A table function is a function returning a table. It can be invoked inside the +`FROM` clause of a query: + +``` +SELECT * FROM TABLE(my_function(1, 100)) +``` + +The row type of the returned table can depend on the arguments passed with +invocation of the function. If different row types can be returned, the +function is a **polymorphic table function**. + +Polymorphic table functions allow you to dynamically invoke custom logic from +within the SQL query. They can be used for working with external systems as +well as for enhancing Trino with capabilities going beyond the SQL standard. + +For the list of built-in table functions available in Trino, see {ref}`built in +table functions`. + +Trino supports adding custom table functions. They are declared by connectors +through implementing dedicated interfaces. For guidance on adding new table +functions, see the {doc}`developer guide`. + +Connectors offer support for different functions on a per-connector basis. For +more information about supported table functions, refer to the {doc}`connector +documentation <../../connector>`. + +(built-in-table-functions)= + +## Built-in table functions + +:::{function} exclude_columns(input => table, columns => descriptor) -> table +Excludes from `table` all columns listed in `descriptor`: + +``` +SELECT * +FROM TABLE(exclude_columns( + input => TABLE(orders), + columns => DESCRIPTOR(clerk, comment))) +``` + +The argument `input` is a table or a query. +The argument `columns` is a descriptor without types. +::: + +(sequence-table-function)= + +:::{function} sequence(start => bigint, stop => bigint, step => bigint) -> table(sequential_number bigint) +:noindex: true + +Returns a single column `sequential_number` containing a sequence of +bigint: + +``` +SELECT * +FROM TABLE(sequence( + start => 1000000, + stop => -2000000, + step => -3)) +``` + +`start` is the first element in te sequence. The default value is `0`. + +`stop` is the end of the range, inclusive. The last element in the +sequence is equal to `stop`, or it is the last value within range, +reachable by steps. + +`step` is the difference between subsequent values. The default value is +`1`. +::: + +:::{note} +The result of the `sequence` table function might not be ordered. +::: + +## Table function invocation + +You invoke a table function in the `FROM` clause of a query. Table function +invocation syntax is similar to a scalar function call. + +### Function resolution + +Every table function is provided by a catalog, and it belongs to a schema in +the catalog. You can qualify the function name with a schema name, or with +catalog and schema names: + +``` +SELECT * FROM TABLE(schema_name.my_function(1, 100)) +SELECT * FROM TABLE(catalog_name.schema_name.my_function(1, 100)) +``` + +Otherwise, the standard Trino name resolution is applied. The connection +between the function and the catalog must be identified, because the function +is executed by the corresponding connector. If the function is not registered +by the specified catalog, the query fails. + +The table function name is resolved case-insensitive, analogically to scalar +function and table resolution in Trino. + +### Arguments + +There are three types of arguments. + +1. Scalar arguments + +They must be constant expressions, and they can be of any SQL type, which is +compatible with the declared argument type: + +``` +factor => 42 +``` + +2. Descriptor arguments + +Descriptors consist of fields with names and optional data types: + +``` +schema => DESCRIPTOR(id BIGINT, name VARCHAR) +columns => DESCRIPTOR(date, status, comment) +``` + +To pass `null` for a descriptor, use: + +``` +schema => CAST(null AS DESCRIPTOR) +``` + +3. Table arguments + +You can pass a table name, or a query. Use the keyword `TABLE`: + +``` +input => TABLE(orders) +data => TABLE(SELECT * FROM region, nation WHERE region.regionkey = nation.regionkey) +``` + +If the table argument is declared as {ref}`set semantics`, +you can specify partitioning and ordering. Each partition is processed +independently by the table function. If you do not specify partitioning, the +argument is processed as a single partition. You can also specify +`PRUNE WHEN EMPTY` or `KEEP WHEN EMPTY`. With `PRUNE WHEN EMPTY` you +declare that you are not interested in the function result if the argument is +empty. This information is used by the Trino engine to optimize the query. The +`KEEP WHEN EMPTY` option indicates that the function should be executed even +if the table argument is empty. Note that by specifying `KEEP WHEN EMPTY` or +`PRUNE WHEN EMPTY`, you override the property set for the argument by the +function author. + +The following example shows how the table argument properties should be ordered: + +``` +input => TABLE(orders) + PARTITION BY orderstatus + KEEP WHEN EMPTY + ORDER BY orderdate +``` + +### Argument passing conventions + +There are two conventions of passing arguments to a table function: + +- **Arguments passed by name**: + + ``` + SELECT * FROM TABLE(my_function(row_count => 100, column_count => 1)) + ``` + +In this convention, you can pass the arguments in arbitrary order. Arguments +declared with default values can be skipped. Argument names are resolved +case-sensitive, and with automatic uppercasing of unquoted names. + +- **Arguments passed positionally**: + + ``` + SELECT * FROM TABLE(my_function(1, 100)) + ``` + +In this convention, you must follow the order in which the arguments are +declared. You can skip a suffix of the argument list, provided that all the +skipped arguments are declared with default values. + +You cannot mix the argument conventions in one invocation. + +You can also use parameters in arguments: + +``` +PREPARE stmt FROM +SELECT * FROM TABLE(my_function(row_count => ? + 1, column_count => ?)); + +EXECUTE stmt USING 100, 1; +``` diff --git a/430/_sources/functions/tdigest.md.txt b/430/_sources/functions/tdigest.md.txt new file mode 100644 index 000000000..84d5d3a1e --- /dev/null +++ b/430/_sources/functions/tdigest.md.txt @@ -0,0 +1,44 @@ +# T-Digest functions + +## Data structures + +A T-digest is a data sketch which stores approximate percentile +information. The Trino type for this data structure is called `tdigest`. +T-digests can be merged, and for storage and retrieval they can be cast +to and from `VARBINARY`. + +## Functions + +:::{function} merge(tdigest) -> tdigest +:noindex: true + +Aggregates all inputs into a single `tdigest`. +::: + +:::{function} value_at_quantile(tdigest, quantile) -> double +:noindex: true + +Returns the approximate percentile value from the T-digest, given +the number `quantile` between 0 and 1. +::: + +:::{function} values_at_quantiles(tdigest, quantiles) -> array(double) +:noindex: true + +Returns the approximate percentile values as an array, given the input +T-digest and an array of values between 0 and 1, which +represent the quantiles to return. +::: + +:::{function} tdigest_agg(x) -> tdigest +Composes all input values of `x` into a `tdigest`. `x` can be +of any numeric type. +::: + +:::{function} tdigest_agg(x, w) -> tdigest +:noindex: true + +Composes all input values of `x` into a `tdigest` using +the per-item weight `w`. `w` must be greater or equal than 1. +`x` and `w` can be of any numeric type. +::: diff --git a/430/_sources/functions/teradata.md.txt b/430/_sources/functions/teradata.md.txt new file mode 100644 index 000000000..d1218deb0 --- /dev/null +++ b/430/_sources/functions/teradata.md.txt @@ -0,0 +1,47 @@ +# Teradata functions + +These functions provide compatibility with Teradata SQL. + +## String functions + +:::{function} char2hexint(string) -> varchar +Returns the hexadecimal representation of the UTF-16BE encoding of the string. +::: + +:::{function} index(string, substring) -> bigint +Alias for {func}`strpos` function. +::: + +## Date functions + +The functions in this section use a format string that is compatible with +the Teradata datetime functions. The following table, based on the +Teradata reference manual, describes the supported format specifiers: + +| Specifier | Description | +| ------------- | ---------------------------------- | +| `- / , . ; :` | Punctuation characters are ignored | +| `dd` | Day of month (1-31) | +| `hh` | Hour of day (1-12) | +| `hh24` | Hour of the day (0-23) | +| `mi` | Minute (0-59) | +| `mm` | Month (01-12) | +| `ss` | Second (0-59) | +| `yyyy` | 4-digit year | +| `yy` | 2-digit year | + +:::{warning} +Case insensitivity is not currently supported. All specifiers must be lowercase. +::: + +:::{function} to_char(timestamp, format) -> varchar +Formats `timestamp` as a string using `format`. +::: + +:::{function} to_timestamp(string, format) -> timestamp +Parses `string` into a `TIMESTAMP` using `format`. +::: + +:::{function} to_date(string, format) -> date +Parses `string` into a `DATE` using `format`. +::: diff --git a/430/_sources/functions/url.md.txt b/430/_sources/functions/url.md.txt new file mode 100644 index 000000000..9455784bd --- /dev/null +++ b/430/_sources/functions/url.md.txt @@ -0,0 +1,74 @@ +# URL functions + +## Extraction functions + +The URL extraction functions extract components from HTTP URLs +(or any valid URIs conforming to {rfc}`2396`). +The following syntax is supported: + +```text +[protocol:][//host[:port]][path][?query][#fragment] +``` + +The extracted components do not contain URI syntax separators +such as `:` or `?`. + +:::{function} url_extract_fragment(url) -> varchar +Returns the fragment identifier from `url`. +::: + +:::{function} url_extract_host(url) -> varchar +Returns the host from `url`. +::: + +:::{function} url_extract_parameter(url, name) -> varchar +Returns the value of the first query string parameter named `name` +from `url`. Parameter extraction is handled in the typical manner +as specified by {rfc}`1866#section-8.2.1`. +::: + +:::{function} url_extract_path(url) -> varchar +Returns the path from `url`. +::: + +:::{function} url_extract_port(url) -> bigint +Returns the port number from `url`. +::: + +:::{function} url_extract_protocol(url) -> varchar +Returns the protocol from `url`: + +``` +SELECT url_extract_protocol('http://localhost:8080/req_path'); +-- http + +SELECT url_extract_protocol('https://127.0.0.1:8080/req_path'); +-- https + +SELECT url_extract_protocol('ftp://path/file'); +-- ftp +``` +::: + +:::{function} url_extract_query(url) -> varchar +Returns the query string from `url`. +::: + +## Encoding functions + +:::{function} url_encode(value) -> varchar +Escapes `value` by encoding it so that it can be safely included in +URL query parameter names and values: + +- Alphanumeric characters are not encoded. +- The characters `.`, `-`, `*` and `_` are not encoded. +- The ASCII space character is encoded as `+`. +- All other characters are converted to UTF-8 and the bytes are encoded + as the string `%XX` where `XX` is the uppercase hexadecimal + value of the UTF-8 byte. +::: + +:::{function} url_decode(value) -> varchar +Unescapes the URL encoded `value`. +This function is the inverse of {func}`url_encode`. +::: diff --git a/430/_sources/functions/uuid.md.txt b/430/_sources/functions/uuid.md.txt new file mode 100644 index 000000000..c0cf1dc34 --- /dev/null +++ b/430/_sources/functions/uuid.md.txt @@ -0,0 +1,5 @@ +# UUID functions + +:::{function} uuid() -> uuid +Returns a pseudo randomly generated {ref}`uuid-type` (type 4). +::: diff --git a/430/_sources/functions/window.md.txt b/430/_sources/functions/window.md.txt new file mode 100644 index 000000000..f4b9bbb1c --- /dev/null +++ b/430/_sources/functions/window.md.txt @@ -0,0 +1,127 @@ +# Window functions + +Window functions perform calculations across rows of the query result. +They run after the `HAVING` clause but before the `ORDER BY` clause. +Invoking a window function requires special syntax using the `OVER` +clause to specify the window. +For example, the following query ranks orders for each clerk by price: + +``` +SELECT orderkey, clerk, totalprice, + rank() OVER (PARTITION BY clerk + ORDER BY totalprice DESC) AS rnk +FROM orders +ORDER BY clerk, rnk +``` + +The window can be specified in two ways (see {ref}`window-clause`): + +- By a reference to a named window specification defined in the `WINDOW` clause, +- By an in-line window specification which allows to define window components + as well as refer to the window components pre-defined in the `WINDOW` clause. + +## Aggregate functions + +All {doc}`aggregate` can be used as window functions by adding the `OVER` +clause. The aggregate function is computed for each row over the rows within +the current row's window frame. + +For example, the following query produces a rolling sum of order prices +by day for each clerk: + +``` +SELECT clerk, orderdate, orderkey, totalprice, + sum(totalprice) OVER (PARTITION BY clerk + ORDER BY orderdate) AS rolling_sum +FROM orders +ORDER BY clerk, orderdate, orderkey +``` + +## Ranking functions + +:::{function} cume_dist() -> bigint +Returns the cumulative distribution of a value in a group of values. +The result is the number of rows preceding or peer with the row in the +window ordering of the window partition divided by the total number of +rows in the window partition. Thus, any tie values in the ordering will +evaluate to the same distribution value. +::: + +:::{function} dense_rank() -> bigint +Returns the rank of a value in a group of values. This is similar to +{func}`rank`, except that tie values do not produce gaps in the sequence. +::: + +:::{function} ntile(n) -> bigint +Divides the rows for each window partition into `n` buckets ranging +from `1` to at most `n`. Bucket values will differ by at most `1`. +If the number of rows in the partition does not divide evenly into the +number of buckets, then the remainder values are distributed one per +bucket, starting with the first bucket. + +For example, with `6` rows and `4` buckets, the bucket values would +be as follows: `1` `1` `2` `2` `3` `4` +::: + +:::{function} percent_rank() -> double +Returns the percentage ranking of a value in group of values. The result +is `(r - 1) / (n - 1)` where `r` is the {func}`rank` of the row and +`n` is the total number of rows in the window partition. +::: + +:::{function} rank() -> bigint +Returns the rank of a value in a group of values. The rank is one plus +the number of rows preceding the row that are not peer with the row. +Thus, tie values in the ordering will produce gaps in the sequence. +The ranking is performed for each window partition. +::: + +:::{function} row_number() -> bigint +Returns a unique, sequential number for each row, starting with one, +according to the ordering of rows within the window partition. +::: + +## Value functions + +By default, null values are respected. If `IGNORE NULLS` is specified, all rows where +`x` is null are excluded from the calculation. If `IGNORE NULLS` is specified and `x` +is null for all rows, the `default_value` is returned, or if it is not specified, +`null` is returned. + +:::{function} first_value(x) -> [same as input] +Returns the first value of the window. +::: + +:::{function} last_value(x) -> [same as input] +Returns the last value of the window. +::: + +:::{function} nth_value(x, offset) -> [same as input] +Returns the value at the specified offset from the beginning of the window. +Offsets start at `1`. The offset can be any scalar +expression. If the offset is null or greater than the number of values in +the window, `null` is returned. It is an error for the offset to be zero or +negative. +::: + +:::{function} lead(x[, offset [, default_value]]) -> [same as input] +Returns the value at `offset` rows after the current row in the window partition. +Offsets start at `0`, which is the current row. The +offset can be any scalar expression. The default `offset` is `1`. If the +offset is null, `null` is returned. If the offset refers to a row that is not +within the partition, the `default_value` is returned, or if it is not specified +`null` is returned. +The {func}`lead` function requires that the window ordering be specified. +Window frame must not be specified. +::: + +:::{function} lag(x[, offset [, default_value]]) -> [same as input] +Returns the value at `offset` rows before the current row in the window partition. +Offsets start at `0`, which is the current row. The +offset can be any scalar expression. The default `offset` is `1`. If the +offset is null, `null` is returned. If the offset refers to a row that is not +within the partition, the `default_value` is returned, or if it is not specified +`null` is returned. +The {func}`lag` function requires that the window ordering be specified. +Window frame must not be specified. +::: diff --git a/430/_sources/glossary.md.txt b/430/_sources/glossary.md.txt new file mode 100644 index 000000000..a47d74674 --- /dev/null +++ b/430/_sources/glossary.md.txt @@ -0,0 +1,213 @@ +# Glossary + +The glossary contains a list of key Trino terms and definitions. + +(glosscatalog)= + +Catalog + +: Catalogs define and name a configuration for connecting to a data source, + allowing users to query the connected data. Each catalog's configuration + specifies a {ref}`connector ` to define which data source + the catalog connects to. For more information about catalogs, see + {ref}`trino-concept-catalog`. + +(glosscert)= + +Certificate + +: A public key [certificate](https://wikipedia.org/wiki/Public_key_certificate) issued by a {ref}`CA + `, sometimes abbreviated as cert, that verifies the ownership of a + server's private keys. Certificate format is specified in the [X.509](https://wikipedia.org/wiki/X.509) standard. + +(glossca)= + +Certificate Authority (CA) + +: A trusted organization that signs and issues certificates. Its signatures + can be used to verify the validity of {ref}`certificates `. + +Cluster + +: A Trino cluster provides the resources to run queries against numerous data + sources. Clusters define the number of nodes, the configuration for the JVM + runtime, configured data sources, and others aspects. For more information, + see {ref}`trino-concept-cluster`. + +(glossconnector)= + +Connector + +: Translates data from a data source into Trino schemas, tables, columns, + rows, and data types. A {doc}`connector ` is specific to a data + source, and is used in {ref}`catalog ` configurations to + define what data source the catalog connects to. A connector is one of many + types of {ref}`plugins ` + +Container + +: A lightweight virtual package of software that contains libraries, binaries, + code, configuration files, and other dependencies needed to deploy an + application. A running container does not include an operating system, + instead using the operating system of the host machine. To learn more, read + read about [containers](https://kubernetes.io/docs/concepts/containers/) + in the Kubernetes documentation. + +(glossdatasource)= + +Data source + +: A system from which data is retrieved - for example, PostgreSQL or Iceberg + on S3 data. In Trino, users query data sources with {ref}`catalogs + ` that connect to each source. See + {ref}`trino-concept-data-sources` for more information. + +(glossdatavirtualization)= + +Data virtualization + +: [Data virtualization](https://wikipedia.org/wiki/Data_virtualization) is a + method of abstracting an interaction with multiple {ref}`heterogeneous data + sources `, without needing to know the distributed nature + of the data, its format, or any other technical details involved in + presenting the data. + +(glossgzip)= + +gzip + +: [gzip](https://wikipedia.org/wiki/Gzip) is a compression format and + software that compresses and decompresses files. This format is used several + ways in Trino, including deployment and compressing files in {ref}`object + storage `. The most common extension for gzip-compressed + files is `.gz`. + +(glosshdfs)= + +HDFS + +: [Hadoop Distributed Filesystem (HDFS)](https://wikipedia.org/wiki/Apache_Hadoop#HDFS) is a scalable {ref}`open + source ` filesystem that was one of the earliest + distributed big data systems created to store large amounts of data for the + [Hadoop ecosystem](https://wikipedia.org/wiki/Apache_Hadoop). + +(glossjks)= + +Java KeyStore (JKS) + +: The system of public key cryptography supported as one part of the Java + security APIs. The legacy JKS system recognizes keys and {ref}`certificates + ` stored in *keystore* files, typically with the `.jks` + extension, and by default relies on a system-level list of {ref}`CAs + ` in *truststore* files installed as part of the current Java + installation. + +Key + +: A cryptographic key specified as a pair of public and private strings + generally used in the context of {ref}`TLS ` to secure public + network traffic. + +(glosslb)= + +Load Balancer (LB) + +: Software or a hardware device that sits on a network edge and accepts + network connections on behalf of servers behind that wall, distributing + traffic across network and server infrastructure to balance the load on + networked services. + +(glossobjectstorage)= + +Object storage + +: [Object storage](https://en.wikipedia.org/wiki/Object_storage) is a file + storage mechanism. Examples of compatible object stores include the + following: + + - [Amazon S3](https://aws.amazon.com/s3) + - [Google Cloud Storage](https://cloud.google.com/storage) + - [Azure Blob Storage](https://azure.microsoft.com/en-us/products/storage/blobs) + - [MinIO](https://min.io/) and other S3-compatible stores + - {ref}`HDFS ` + +(glossopensource)= + +Open-source + +: Typically refers to [open-source software](https://wikipedia.org/wiki/Open-source_software). which is software that + has the source code made available for others to see, use, and contribute + to. Allowed usage varies depending on the license that the software is + licensed under. Trino is licensed under the [Apache license](https://wikipedia.org/wiki/Apache_License), and is therefore maintained + by a community of contributors from all across the globe. + +(glosspem)= + +PEM file format + +: A format for storing and sending cryptographic keys and certificates. PEM + format can contain both a key and its certificate, plus the chain of + certificates from authorities back to the root {ref}`CA `, or back + to a CA vendor's intermediate CA. + +(glosspkcs12)= + +PKCS #12 + +: A binary archive used to store keys and certificates or certificate chains + that validate a key. [PKCS #12](https://wikipedia.org/wiki/PKCS_12) files + have `.p12` or `.pfx` extensions. This format is a less popular + alternative to {ref}`PEM `. + +(glossplugin)= + +Plugin + +: A bundle of code implementing the Trino {doc}`Service Provider Interface + (SPI) ` that is used to add new {ref}`connectors + `, {doc}`data types `, {doc}`functions`, + {doc}`access control implementations `, and + other features of Trino. + +Presto and PrestoSQL + +: The old name for Trino. To learn more about the name change to Trino, read + [the history](). + +Query federation + +: A type of {ref}`data virtualization ` that provides a + common access point and data model across two or more heterogeneous data + sources. A popular data model used by many query federation engines is + translating different data sources to {ref}`SQL ` tables. + +(glossssl)= + +Secure Sockets Layer (SSL) + +: Now superseded by {ref}`TLS `, but still recognized as the term + for what TLS does. + +(glosssql)= + +Structured Query Language (SQL) + +: The standard language used with relational databases. For more information, + see {doc}`SQL `. + +Tarball + +: A common abbreviation for [TAR file](), which is a common software + distribution mechanism. This file format is a collection of multiple files + distributed as a single file, commonly compressed using {ref}`gzip + ` compression. + +(glosstls)= + +Transport Layer Security (TLS) + +: [TLS](https://wikipedia.org/wiki/Transport_Layer_Security) is a security + protocol designed to provide secure communications over a network. It is the + successor to {ref}`SSL `, and used in many applications like + HTTPS, email, and Trino. These security topics use the term TLS to refer to + both TLS and SSL. diff --git a/430/_sources/index.md.txt b/430/_sources/index.md.txt new file mode 100644 index 000000000..8c485a678 --- /dev/null +++ b/430/_sources/index.md.txt @@ -0,0 +1,26 @@ +# Trino documentation + +```{toctree} +:titlesonly: true + +overview +installation +client +security +admin +optimizer +connector +functions +language +sql +develop +glossary +appendix +``` + +```{toctree} +:maxdepth: 1 +:titlesonly: true + +release +``` diff --git a/430/_sources/installation.md.txt b/430/_sources/installation.md.txt new file mode 100644 index 000000000..01c76e095 --- /dev/null +++ b/430/_sources/installation.md.txt @@ -0,0 +1,20 @@ +# Installation + +A Trino server can be installed and deployed on a number of different +platforms. Typically you run a cluster of machines with one coordinator and many +workers. You can find instructions for deploying such a cluster, and related +information, in the following sections: + +```{toctree} +:maxdepth: 1 + +installation/deployment +installation/containers +installation/kubernetes +installation/rpm +installation/query-resiliency +``` + +Once you have a completed the deployment, or if you have access to a running +cluster already, you can proceed to configure your {doc}`client application +`. diff --git a/430/_sources/installation/containers.md.txt b/430/_sources/installation/containers.md.txt new file mode 100644 index 000000000..1ae1cd1db --- /dev/null +++ b/430/_sources/installation/containers.md.txt @@ -0,0 +1,95 @@ +# Trino in a Docker container + +The Trino project provides the [trinodb/trino](https://hub.docker.com/r/trinodb/trino) +Docker image that includes the Trino server and a default configuration. The +Docker image is published to Docker Hub and can be used with the Docker runtime, +among several others. + +## Running the container + +To run Trino in Docker, you must have the Docker engine installed on your +machine. You can download Docker from the [Docker website](https://www.docker.com), +or use the packaging system of your operating systems. + +Use the `docker` command to create a container from the `trinodb/trino` +image. Assign it the `trino` name, to make it easier to reference it later. +Run it in the background, and map the default Trino port, which is 8080, +from inside the container to port 8080 on your workstation. + +```text +docker run --name trino -d -p 8080:8080 trinodb/trino +``` + +Without specifying the container image tag, it defaults to `latest`, +but a number of any released Trino version can be used, for example +`trinodb/trino:|trino_version|`. + +Run `docker ps` to see all the containers running in the background. + +```text +% docker ps +CONTAINER ID IMAGE COMMAND CREATED STATUS PORTS NAMES +955c3b3d3d0a trinodb/trino:390 "/usr/lib/trino/bin/…" 39 hours ago Up 39 hours (healthy) 0.0.0.0:8080->8080/tcp trino +``` + +When Trino is still starting, it shows `(health: starting)`, +and `(healthy)` when it's ready. + +:::{note} +There are multiple ways to use Trino within containers. You can either run +Trino in Docker containers locally, as explained in the following sections, +or use a container orchestration platform like Kubernetes. For the Kubernetes +instructions see {doc}`/installation/kubernetes`. +::: + +## Executing queries + +The image includes the Trino command-line interface (CLI) client, `trino`. +Execute it in the existing container to connect to the Trino server running +inside it. After starting the client, type and execute a query on a table +of the `tpch` catalog, which includes example data: + +```text +$ docker exec -it trino trino +trino> select count(*) from tpch.sf1.nation; + _col0 +------- + 25 +(1 row) + +Query 20181105_001601_00002_e6r6y, FINISHED, 1 node +Splits: 21 total, 21 done (100.00%) +0:06 [25 rows, 0B] [4 rows/s, 0B/s] +``` + +Once you are done with your exploration, enter the `quit` command. + +Alternatively, you can use the Trino CLI installed directly on your workstation. +The default server URL in the CLI of matches the port used +in the command to start the container. More information about using the CLI can +be found in {doc}`/client/cli`. You can also connect with any other client +application using the {doc}`/client/jdbc`. + +## Configuring Trino + +The image already contains a default configuration to get started, and some +catalogs to allow you to explore Trino. You can also use the container with your +custom configuration files in a local `etc` directory structure as created in +the {doc}`/installation/deployment`. If you mount this directory as a volume +in the path `/etc/trino` when starting the container, your configuration +is used instead of the default in the image. + +```text +$ docker run --name trino -d -p 8080:8080 --volume $PWD/etc:/etc/trino trinodb/trino +``` + +To keep the default configuration and only configure catalogs, mount a folder +at `/etc/trino/catalog`, or individual catalog property files in it. + +If you want to use additional plugins, mount them at `/usr/lib/trino/plugin`. + +## Cleaning up + +You can stop and start the container, using the `docker stop trino` and +`docker start trino` commands. To fully remove the stopped container, run +`docker rm trino`. diff --git a/430/_sources/installation/deployment.md.txt b/430/_sources/installation/deployment.md.txt new file mode 100644 index 000000000..c82c6b423 --- /dev/null +++ b/430/_sources/installation/deployment.md.txt @@ -0,0 +1,373 @@ +# Deploying Trino + +(requirements)= + +## Requirements + +(requirements-linux)= + +### Linux operating system + +- 64-bit required + +- newer release preferred, especially when running on containers + +- adequate ulimits for the user that runs the Trino process. These limits may + depend on the specific Linux distribution you are using. The number of open + file descriptors needed for a particular Trino instance scales as roughly the + number of machines in the cluster, times some factor depending on the + workload. The `nofile` limit sets the maximum number of file descriptors + that a process can have, while the `nproc` limit restricts the number of + processes, and therefore threads on the JVM, a user can create. We recommend + setting limits to the following values at a minimum. Typically, this + configuration is located in `/etc/security/limits.conf`: + + ```text + trino soft nofile 131072 + trino hard nofile 131072 + trino soft nproc 128000 + trino hard nproc 128000 + ``` + +% These values are used in core/trino-server-rpm/src/main/resources/dist/etc/init.d/trino + +(requirements-java)= + +### Java runtime environment + +Trino requires a 64-bit version of Java 17, with a minimum required version of 17.0.3. +Earlier major versions such as Java 8 or Java 11 do not work. +Newer major versions such as Java 18 or 19, are not supported -- they may work, but are not tested. + +We recommend using the Eclipse Temurin OpenJDK distribution from +[Adoptium](https://adoptium.net/) as the JDK for Trino, as Trino is tested +against that distribution. Eclipse Temurin is also the JDK used by the [Trino +Docker image](https://hub.docker.com/r/trinodb/trino). + +If you are using Java 17 or 18, the JVM must be configured to use UTF-8 as the default charset by +adding `-Dfile.encoding=UTF-8` to `etc/jvm.config`. Starting with Java 19, the Java default +charset is UTF-8, so this configuration is not needed. + +(requirements-python)= + +### Python + +- version 2.6.x, 2.7.x, or 3.x +- required by the `bin/launcher` script only + +## Installing Trino + +Download the Trino server tarball, {maven_download}`server`, and unpack it. The +tarball contains a single top-level directory, `trino-server-|trino_version|`, +which we call the *installation* directory. + +Trino needs a *data* directory for storing logs, etc. +We recommend creating a data directory outside of the installation directory, +which allows it to be easily preserved when upgrading Trino. + +## Configuring Trino + +Create an `etc` directory inside the installation directory. +This holds the following configuration: + +- Node Properties: environmental configuration specific to each node +- JVM Config: command line options for the Java Virtual Machine +- Config Properties: configuration for the Trino server. See the + {doc}`/admin/properties` for available configuration properties. +- Catalog Properties: configuration for {doc}`/connector` (data sources). + The available catalog configuration properties for a connector are described + in the respective connector documentation. + +(node-properties)= + +### Node properties + +The node properties file, `etc/node.properties`, contains configuration +specific to each node. A *node* is a single installed instance of Trino +on a machine. This file is typically created by the deployment system when +Trino is first installed. The following is a minimal `etc/node.properties`: + +```text +node.environment=production +node.id=ffffffff-ffff-ffff-ffff-ffffffffffff +node.data-dir=/var/trino/data +``` + +The above properties are described below: + +- `node.environment`: + The name of the environment. All Trino nodes in a cluster must have the same + environment name. The name must start with a lowercase alphanumeric character + and only contain lowercase alphanumeric or underscore (`_`) characters. +- `node.id`: + The unique identifier for this installation of Trino. This must be + unique for every node. This identifier should remain consistent across + reboots or upgrades of Trino. If running multiple installations of + Trino on a single machine (i.e. multiple nodes on the same machine), + each installation must have a unique identifier. The identifier must start + with an alphanumeric character and only contain alphanumeric, `-`, or `_` + characters. +- `node.data-dir`: + The location (filesystem path) of the data directory. Trino stores + logs and other data here. + +(jvm-config)= + +### JVM config + +The JVM config file, `etc/jvm.config`, contains a list of command line +options used for launching the Java Virtual Machine. The format of the file +is a list of options, one per line. These options are not interpreted by +the shell, so options containing spaces or other special characters should +not be quoted. + +The following provides a good starting point for creating `etc/jvm.config`: + +```text +-server +-Xmx16G +-XX:InitialRAMPercentage=80 +-XX:MaxRAMPercentage=80 +-XX:G1HeapRegionSize=32M +-XX:+ExplicitGCInvokesConcurrent +-XX:+ExitOnOutOfMemoryError +-XX:+HeapDumpOnOutOfMemoryError +-XX:-OmitStackTraceInFastThrow +-XX:ReservedCodeCacheSize=512M +-XX:PerMethodRecompilationCutoff=10000 +-XX:PerBytecodeRecompilationCutoff=10000 +-Djdk.attach.allowAttachSelf=true +-Djdk.nio.maxCachedBufferSize=2000000 +-XX:+UnlockDiagnosticVMOptions +-XX:+UseAESCTRIntrinsics +-Dfile.encoding=UTF-8 +# Disable Preventive GC for performance reasons (JDK-8293861) +-XX:-G1UsePreventiveGC +# Reduce starvation of threads by GClocker, recommend to set about the number of cpu cores (JDK-8192647) +-XX:GCLockerRetryAllocationCount=32 +``` + +You must adjust the value for the memory used by Trino, specified with `-Xmx` +to the available memory on your nodes. Typically, values representing 70 to 85 +percent of the total available memory is recommended. For example, if all +workers and the coordinator use nodes with 64GB of RAM, you can use `-Xmx54G`. +Trino uses most of the allocated memory for processing, with a small percentage +used by JVM-internal processes such as garbage collection. + +The rest of the available node memory must be sufficient for the operating +system and other running services, as well as off-heap memory used for native +code initiated the JVM process. + +On larger nodes, the percentage value can be lower. Allocation of all memory to +the JVM or using swap space is not supported, and disabling swap space on the +operating system level is recommended. + +Large memory allocation beyond 32GB is recommended for production clusters. + +Because an `OutOfMemoryError` typically leaves the JVM in an +inconsistent state, we write a heap dump, for debugging, and forcibly +terminate the process when this occurs. + +The temporary directory used by the JVM must allow execution of code. +Specifically, the mount must not have the `noexec` flag set. The default +`/tmp` directory is mounted with this flag in some installations, which +prevents Trino from starting. You can workaround this by overriding the +temporary directory by adding `-Djava.io.tmpdir=/path/to/other/tmpdir` to the +list of JVM options. + +We enable `-XX:+UnlockDiagnosticVMOptions` and `-XX:+UseAESCTRIntrinsics` to improve AES performance for S3, etc. on ARM64 ([JDK-8271567](https://bugs.openjdk.java.net/browse/JDK-8271567)) +We disable Preventive GC (`-XX:-G1UsePreventiveGC`) for performance reasons (see [JDK-8293861](https://bugs.openjdk.org/browse/JDK-8293861)) +We set GCLocker retry allocation count (`-XX:GCLockerRetryAllocationCount=32`) to avoid OOM too early (see [JDK-8192647](https://bugs.openjdk.org/browse/JDK-8192647)) + +(config-properties)= + +### Config properties + +The config properties file, `etc/config.properties`, contains the +configuration for the Trino server. Every Trino server can function as both a +coordinator and a worker. A cluster is required to include one coordinator, and +dedicating a machine to only perform coordination work provides the best +performance on larger clusters. Scaling and parallelization is achieved by using +many workers. + +The following is a minimal configuration for the coordinator: + +```text +coordinator=true +node-scheduler.include-coordinator=false +http-server.http.port=8080 +discovery.uri=http://example.net:8080 +``` + +And this is a minimal configuration for the workers: + +```text +coordinator=false +http-server.http.port=8080 +discovery.uri=http://example.net:8080 +``` + +Alternatively, if you are setting up a single machine for testing, that +functions as both a coordinator and worker, use this configuration: + +```text +coordinator=true +node-scheduler.include-coordinator=true +http-server.http.port=8080 +discovery.uri=http://example.net:8080 +``` + +These properties require some explanation: + +- `coordinator`: + Allow this Trino instance to function as a coordinator, so to + accept queries from clients and manage query execution. +- `node-scheduler.include-coordinator`: + Allow scheduling work on the coordinator. + For larger clusters, processing work on the coordinator + can impact query performance because the machine's resources are not + available for the critical task of scheduling, managing and monitoring + query execution. +- `http-server.http.port`: + Specifies the port for the HTTP server. Trino uses HTTP for all + communication, internal and external. +- `discovery.uri`: + The Trino coordinator has a discovery service that is used by all the nodes + to find each other. Every Trino instance registers itself with the discovery + service on startup and continuously heartbeats to keep its registration + active. The discovery service shares the HTTP server with Trino and thus + uses the same port. Replace `example.net:8080` to match the host and + port of the Trino coordinator. If you have disabled HTTP on the coordinator, + the URI scheme must be `https`, not `http`. + +The above configuration properties are a *minimal set* to help you get started. +All additional configuration is optional and varies widely based on the specific +cluster and supported use cases. The {doc}`/admin` and {doc}`/security` sections +contain documentation for many aspects, including {doc}`/admin/resource-groups` +for configuring queuing policies and {doc}`/admin/fault-tolerant-execution`. + +The {doc}`/admin/properties` provides a comprehensive list of the supported +properties for topics such as {doc}`/admin/properties-general`, +{doc}`/admin/properties-resource-management`, +{doc}`/admin/properties-query-management`, +{doc}`/admin/properties-web-interface`, and others. + +(log-levels)= + +### Log levels + +The optional log levels file, `etc/log.properties`, allows setting the +minimum log level for named logger hierarchies. Every logger has a name, +which is typically the fully qualified name of the class that uses the logger. +Loggers have a hierarchy based on the dots in the name, like Java packages. +For example, consider the following log levels file: + +```text +io.trino=INFO +``` + +This would set the minimum level to `INFO` for both +`io.trino.server` and `io.trino.plugin.hive`. +The default minimum level is `INFO`, +thus the above example does not actually change anything. +There are four levels: `DEBUG`, `INFO`, `WARN` and `ERROR`. + +(catalog-properties)= + +### Catalog properties + +Trino accesses data via *connectors*, which are mounted in catalogs. +The connector provides all of the schemas and tables inside of the catalog. +For example, the Hive connector maps each Hive database to a schema. +If the Hive connector is mounted as the `hive` catalog, and Hive +contains a table `clicks` in database `web`, that table can be accessed +in Trino as `hive.web.clicks`. + +Catalogs are registered by creating a catalog properties file +in the `etc/catalog` directory. +For example, create `etc/catalog/jmx.properties` with the following +contents to mount the `jmx` connector as the `jmx` catalog: + +```text +connector.name=jmx +``` + +See {doc}`/connector` for more information about configuring connectors. + +(running-trino)= + +## Running Trino + +The installation provides a `bin/launcher` script, which requires Python in +the `PATH`. The script can be used manually or as a daemon startup script. It +accepts the following commands: + +:::{list-table} `launcher` commands +:widths: 15, 85 +:header-rows: 1 + +* - Command + - Action +* - `run` + - Starts the server in the foreground and leaves it running. To shut down + the server, use Ctrl+C in this terminal or the `stop` command from + another terminal. +* - `start` + - Starts the server as a daemon and returns its process ID. +* - `stop` + - Shuts down a server started with either `start` or `run`. Sends the + SIGTERM signal. +* - `restart` + - Stops then restarts a running server, or starts a stopped server, + assigning a new process ID. +* - `kill` + - Shuts down a possibly hung server by sending the SIGKILL signal. +* - `status` + - Prints a status line, either *Stopped pid* or *Running as pid*. +::: + +A number of additional options allow you to specify configuration file and +directory locations, as well as Java options. Run the launcher with `--help` +to see the supported commands and command line options. + +The `-v` or `--verbose` option for each command prepends the server's +current settings before the command's usual output. + +Trino can be started as a daemon by running the following: + +```text +bin/launcher start +``` + +Alternatively, it can be run in the foreground, with the logs and other +output written to stdout/stderr. Both streams should be captured +if using a supervision system like daemontools: + +```text +bin/launcher run +``` + +The launcher configures default values for the configuration +directory `etc`, configuration files, the data directory `var`, +and log files in the data directory. You can change these values +to adjust your Trino usage to any requirements, such as using a +directory outside the installation directory, specific mount points +or locations, and even using other file names. For example, the Trino +RPM adjusts the used directories to better follow the Linux Filesystem +Hierarchy Standard (FHS). + +After starting Trino, you can find log files in the `log` directory inside +the data directory `var`: + +- `launcher.log`: + This log is created by the launcher and is connected to the stdout + and stderr streams of the server. It contains a few log messages + that occur while the server logging is being initialized, and any + errors or diagnostics produced by the JVM. +- `server.log`: + This is the main log file used by Trino. It typically contains + the relevant information if the server fails during initialization. + It is automatically rotated and compressed. +- `http-request.log`: + This is the HTTP request log which contains every HTTP request + received by the server. It is automatically rotated and compressed. diff --git a/430/_sources/installation/kubernetes.md.txt b/430/_sources/installation/kubernetes.md.txt new file mode 100644 index 000000000..277226c51 --- /dev/null +++ b/430/_sources/installation/kubernetes.md.txt @@ -0,0 +1,386 @@ +# Trino on Kubernetes with Helm + +[Kubernetes](https://kubernetes.io) is a container orchestration platform that +allows you to deploy Trino and other applications in a repeatable manner across +different types of infrastructure. This can range from deploying on your laptop +using tools like [kind](https://kind.sigs.k8s.io), to running on a managed +Kubernetes service on cloud services like +[Amazon Elastic Kubernetes Service](https://aws.amazon.com/eks), +[Google Kubernetes Engine](https://cloud.google.com/kubernetes-engine), +[Azure Kubernetes Service](https://azure.microsoft.com/services/kubernetes-service), +and others. + +The fastest way to run Trino on Kubernetes is to use the +[Trino Helm chart](https://github.com/trinodb/charts). +[Helm](https://helm.sh) is a package manager for Kubernetes applications that +allows for simpler installation and versioning by templating Kubernetes +configuration files. This allows you to prototype on your local or on-premise +cluster and use the same deployment mechanism to deploy to the cloud to scale +up. + +## Requirements + +- A Kubernetes cluster with a + [supported version](https://kubernetes.io/releases/) of Kubernetes. + + - If you don't have a Kubernetes cluster, you can + {ref}`run one locally using kind `. + +- [kubectl](https://kubernetes.io/docs/tasks/tools/#kubectl) with a version + that adheres to the + [Kubernetes version skew policy](https://kubernetes.io/releases/version-skew-policy/) + installed on the machine managing the Kubernetes deployment. + +- [helm](https://helm.sh) with a version that adheres to the + [Helm version skew policy](https://helm.sh/docs/topics/version_skew/) + installed on the machine managing the Kubernetes deployment. + +(running-trino-using-helm)= + +## Running Trino using Helm + +Run the following commands from the system with `helm` and `kubectl` +installed and configured to connect to your running Kubernetes cluster: + +1. Validate `kubectl` is pointing to the correct cluster by running the + command: + + ```text + kubectl cluster-info + ``` + + You should see output that shows the correct Kubernetes control plane + address. + +2. Add the Trino Helm chart repository to Helm if you haven't done so already. + This tells Helm where to find the Trino charts. You can name the repository + whatever you want, `trino` is a good choice. + + ```text + helm repo add trino https://trinodb.github.io/charts + ``` + +3. Install Trino on the Kubernetes cluster using the Helm chart. Start by + running the `install` command to use all default values and create + a cluster called `example-trino-cluster`. + + ```text + helm install example-trino-cluster trino/trino + ``` + + This generates the Kubernetes configuration files by inserting properties + into helm templates. The Helm chart contains + [default values](https://trinodb.github.io/charts/charts/trino/) + that can be overridden by a YAML file to update default settings. + + 1. *(Optional)* To override the default values, + {ref}`create your own YAML configuration ` to + define the parameters of your deployment. To run the install command using + the `example.yaml`, add the `f` parameter in you `install` command. + Be sure to follow + {ref}`best practices and naming conventions ` + for your configuration files. + + ```text + helm install -f example.yaml example-trino-cluster trino/trino + ``` + + You should see output as follows: + + ```text + NAME: example-trino-cluster + LAST DEPLOYED: Tue Sep 13 14:12:09 2022 + NAMESPACE: default + STATUS: deployed + REVISION: 1 + TEST SUITE: None + NOTES: + Get the application URL by running these commands: + export POD_NAME=$(kubectl get pods --namespace default -l "app=trino,release=example-trino-cluster,component=coordinator" -o jsonpath="{.items[0].metadata.name}") + echo "Visit http://127.0.0.1:8080 to use your application" + kubectl port-forward $POD_NAME 8080:8080 + ``` + + This output depends on your configuration and cluster name. For example, the + port `8080` is set by the `.service.port` in the `example.yaml`. + +4. Run the following command to check that all pods, deployments, and services + are running properly. + + ```text + kubectl get all + ``` + + You should expect to see output that shows running pods, deployments, and + replica sets. A good indicator that everything is running properly is to see + all pods are returning a ready status in the `READY` column. + + ```text + NAME READY STATUS RESTARTS AGE + pod/example-trino-cluster-coordinator-bfb74c98d-rnrxd 1/1 Running 0 161m + pod/example-trino-cluster-worker-76f6bf54d6-hvl8n 1/1 Running 0 161m + pod/example-trino-cluster-worker-76f6bf54d6-tcqgb 1/1 Running 0 161m + + NAME TYPE CLUSTER-IP EXTERNAL-IP PORT(S) AGE + service/example-trino-cluster ClusterIP 10.96.25.35 8080/TCP 161m + + NAME READY UP-TO-DATE AVAILABLE AGE + deployment.apps/example-trino-cluster-coordinator 1/1 1 1 161m + deployment.apps/example-trino-cluster-worker 2/2 2 2 161m + + NAME DESIRED CURRENT READY AGE + replicaset.apps/example-trino-cluster-coordinator-bfb74c98d 1 1 1 161m + replicaset.apps/example-trino-cluster-worker-76f6bf54d6 2 2 2 161m + ``` + + The output shows running pods. These include the actual Trino containers. To + better understand this output, check out the following resources: + + 1. [kubectl get command reference](https://kubernetes.io/docs/reference/generated/kubectl/kubectl-commands#get). + 2. [kubectl get command example](https://kubernetes.io/docs/reference/kubectl/docker-cli-to-kubectl/#docker-ps). + 3. [Debugging Kubernetes reference](https://kubernetes.io/docs/tasks/debug/). + +5. If all pods, deployments, and replica sets are running and in the ready + state, Trino has been successfully deployed. + +:::{note} +Unlike some Kubernetes applications, where it's better to have many small +pods, Trino works best with fewer pods each having more resources +available. We strongly recommend to avoid having multiple Trino pods on a +single physical host to avoid contention for resources. +::: + +(executing-queries)= + +## Executing queries + +The pods running the Trino containers are all running on a private network +internal to Kubernetes. In order to access them, specifically the coordinator, +you need to create a tunnel to the coordinator pod and your computer. You can do +this by running the commands generated upon installation. + +1. Store the coordinator pod name in a shell variable called `POD_NAME`. + + ```text + POD_NAME=$(kubectl get pods -l "app=trino,release=example-trino-cluster,component=coordinator" -o name) + ``` + +2. Create the tunnel from the coordinator pod to the client. + + ```text + kubectl port-forward $POD_NAME 8080:8080 + ``` + + Now you can connect to the Trino coordinator at `http://localhost:8080`. + +3. To connect to Trino, you can use the + {doc}`command-line interface `, a + {doc}`JDBC client `, or any of the + {doc}`other clients `. For this example, + {ref}`install the command-line interface `, and connect to + Trino in a new console session. + + ```text + trino --server http://localhost:8080 + ``` + +4. Using the sample data in the `tpch` catalog, type and execute a query on + the `nation` table using the `tiny` schema: + + ```text + trino> select count(*) from tpch.tiny.nation; + _col0 + ------- + 25 + (1 row) + + Query 20181105_001601_00002_e6r6y, FINISHED, 1 node + Splits: 21 total, 21 done (100.00%) + 0:06 [25 rows, 0B] [4 rows/s, 0B/s] + ``` + + Try other SQL queries to explore the data set and test your cluster. + +5. Once you are done with your exploration, enter the `quit` command in the + CLI. + +6. Kill the tunnel to the coordinator pod. The is only available while the + `kubectl` process is running, so you can just kill the `kubectl` process + that's forwarding the port. In most cases that means pressing `CTRL` + + `C` in the terminal where the port-forward command is running. + +## Configuration + +The Helm chart uses the {doc}`Trino container image `. +The Docker image already contains a default configuration to get started, and +some catalogs to allow you to explore Trino. Kubernetes allows you to mimic a +{doc}`traditional deployment ` by supplying +configuration in YAML files. It's important to understand how files such as the +Trino configuration, JVM, and various {doc}`catalog properties ` are +configured in Trino before updating the values. + +(creating-your-own-yaml)= + +### Creating your own YAML configuration + +When you use your own YAML Kubernetes configuration, you only override the values you specify. +The remaining properties use their default values. Add an `example.yaml` with +the following configuration: + +```yaml +image: + tag: "|trino_version|" +server: + workers: 3 +coordinator: + jvm: + maxHeapSize: "8G" +worker: + jvm: + maxHeapSize: "8G" +``` + +These values are higher than the defaults and allow Trino to use more memory +and run more demanding queries. If the values are too high, Kubernetes might +not be able to schedule some Trino pods, depending on other applications +deployed in this cluster and the size of the cluster nodes. + +1. `.image.tag` is set to the current version, |trino_version|. Set + this value if you need to use a specific version of Trino. The default is + `latest`, which is not recommended. Using `latest` will publish a new + version of Trino with each release and a following Kubernetes deployment. +2. `.server.workers` is set to `3`. This value sets the number of + workers, in this case, a coordinator and three worker nodes are deployed. +3. `.coordinator.jvm.maxHeapSize` is set to `8GB`. + This sets the maximum heap size in the JVM of the coordinator. See + {ref}`jvm-config`. +4. `.worker.jvm.maxHeapSize` is set to `8GB`. + This sets the maximum heap size in the JVM of the worker. See + {ref}`jvm-config`. + +:::{warning} +Some memory settings need to be tuned carefully as setting some values +outside of the range of the maximum heap size will cause Trino startup to +fail. See the warnings listed on {doc}`/admin/properties-resource-management`. +::: + +Reference [the full list of properties](https://trinodb.github.io/charts/charts/trino/) +that can be overridden in the Helm chart. + +(kubernetes-configuration-best-practices)= + +:::{note} +Although `example.yaml` is used to refer to the Kubernetes configuration +file in this document, you should use clear naming guidelines for the cluster +and deployment you are managing. For example, +`cluster-example-trino-etl.yaml` might refer to a Trino deployment for a +cluster used primarily for extract-transform-load queries deployed on the +`example` Kubernetes cluster. See +[Configuration Best Practices](https://kubernetes.io/docs/concepts/configuration/overview/) +for more tips on configuring Kubernetes deployments. +::: + +### Adding catalogs + +A common use-case is to add custom catalogs. You can do this by adding values to +the `additionalCatalogs` property in the `example.yaml` file. + +```yaml +additionalCatalogs: + lakehouse: |- + connector.name=iceberg + hive.metastore.uri=thrift://example.net:9083 + rdbms: |- + connector.name=postgresql + connection-url=jdbc:postgresql://example.net:5432/database + connection-user=root + connection-password=secret +``` + +This adds both `lakehouse` and `rdbms` catalogs to the Kubernetes deployment +configuration. + +(running-a-local-kubernetes-cluster-with-kind)= + +## Running a local Kubernetes cluster with kind + +For local deployments, you can use +[kind (Kubernetes in Docker)](https://kind.sigs.k8s.io). Follow the steps +below to run `kind` on your system. + +1. `kind` runs on [Docker](https://www.docker.com), so first check if Docker + is installed: + + ```text + docker --version + ``` + + If this command fails, install Docker by following + [Docker installation instructions](https://docs.docker.com/engine/install/). + +2. Install `kind` by following the + [kind installation instructions](https://kind.sigs.k8s.io/docs/user/quick-start/#installation). + +3. Run a Kubernetes cluster in `kind` by running the command: + + ```text + kind create cluster --name trino + ``` + + :::{note} + The `name` parameter is optional but is used to showcase how the + namespace is applied in future commands. The cluster name defaults to + `kind` if no parameter is added. Use `trino` to make the application + on this cluster obvious. + ::: + +4. Verify that `kubectl` is running against the correct Kubernetes cluster. + + ```text + kubectl cluster-info --context kind-trino + ``` + + If you have multiple Kubernetes clusters already configured within + `~/.kube/config`, you need to pass the `context` parameter to the + `kubectl` commands to operate with the local `kind` cluster. `kubectl` + uses the + [default context](https://kubernetes.io/docs/reference/kubectl/cheatsheet/#kubectl-context-and-configuration) + if this parameter isn't supplied. Notice the context is the name of the + cluster with the `kind-` prefix added. Now you can look at all the + Kubernetes objects running on your `kind` cluster. + +5. Set up Trino by folling the {ref}`running-trino-using-helm` steps. When + running the `kubectl get all` command, add the `context` parameter. + + ```text + kubectl get all --context kind-trino + ``` + +6. Run some queries by following the [Executing queries](#executing-queries) steps. + +7. Once you are done with the cluster using kind, you can delete the cluster. + + ```text + kind delete cluster -n trino + ``` + +## Cleaning up + +To uninstall Trino from the Kubernetes cluster, run the following command: + +```text +helm uninstall my-trino-cluster +``` + +You should expect to see the following output: + +```text +release "my-trino-cluster" uninstalled +``` + +To validate that this worked, you can run this `kubectl` command to make sure +there are no remaining Kubernetes objects related to the Trino cluster. + +```text +kubectl get all +``` diff --git a/430/_sources/installation/query-resiliency.md.txt b/430/_sources/installation/query-resiliency.md.txt new file mode 100644 index 000000000..0fc718bb1 --- /dev/null +++ b/430/_sources/installation/query-resiliency.md.txt @@ -0,0 +1,110 @@ +# Improve query processing resilience + +You can configure Trino to be more resilient against failures during query +processing by enabling fault-tolerant execution. This allows Trino to handle +larger queries such as batch operations without worker node interruptions +causing the query to fail. + +When configured, the Trino cluster buffers data used by workers during query +processing. If processing on a worker node fails for any reason, such as a +network outage or running out of available resources, the coordinator +reschedules processing of the failed piece of work on another worker. This +allows query processing to continue using buffered data. + +## Architecture + +The coordinator node uses a configured exchange manager service that buffers +data during query processing in an external location, such as an S3 object +storage bucket. Worker nodes send data to the buffer as they execute their +query tasks. + +## Best practices and considerations + +A fault-tolerant cluster is best suited for large batch queries. Users may +experience latency or similar behavior if they issue a high volume of +short-running queries on a fault-tolerant cluster. As such, it is recommended to +run a dedicated fault-tolerant cluster for handling batch operations, separate +from a cluster that is designated for a higher query volume. + +Catalogs using the following connectors support fault-tolerant execution of read +and write operations: + +- {doc}`/connector/delta-lake` +- {doc}`/connector/hive` +- {doc}`/connector/iceberg` +- {doc}`/connector/mysql` +- {doc}`/connector/postgresql` +- {doc}`/connector/sqlserver` + +Catalogs using other connectors only support fault-tolerant execution of read +operations. When fault-tolerant execution is enabled on a cluster, write +operations fail on any catalogs that do not support fault-tolerant +execution of those operations. + +The exchange manager may send a large amount of data to the exchange storage, +resulting in high I/O load on that storage. You can configure multiple storage +locations for use by the exchange manager to help balance the I/O load between +them. + +## Configuration + +The following steps describe how to configure a Trino cluster for +fault-tolerant execution with an S3-based exchange: + +1. Set up an S3 bucket to use as the exchange storage. For this example we are + using an AWS S3 bucket, but other storage options are described in the + {doc}`reference documentation ` + as well. You can use multiple S3 buckets for exchange storage. + + For each bucket in AWS, collect the following information: + + - S3 URI location for the bucket, such as `s3://exchange-spooling-bucket` + - Region that the bucket is located in, such as `us-west-1` + - AWS access and secret keys for the bucket + +2. For a {doc}`Kubernetes deployment of Trino `, add + the following exchange manager configuration in the + `server.exchangeManager` and `additionalExchangeManagerProperties` + sections of the Helm chart, using the gathered S3 bucket information: + + ```yaml + server: + exchangeManager: + name=filesystem + base-directories=s3://exchange-spooling-bucket-1,s3://exchange-spooling-bucket-2 + + additionalExchangeManagerProperties: + s3.region=us-west-1 + s3.aws-access-key=example-access-key + s3.aws-secret-key=example-secret-key + ``` + + In non-Kubernetes installations, the same properties must be defined in an + `exchange-manager.properties` configuration file on the coordinator and + all worker nodes. + +3. Add the following configuration for fault-tolerant execution in the + `additionalConfigProperties:` section of the Helm chart: + + ```yaml + additionalConfigProperties: + retry-policy=TASK + ``` + + In non-Kubernetes installations, the same property must be defined in the + `config.properties` file on the coordinator and all worker nodes. + +4. Re-deploy your instance of Trino or, for non-Kubernetes + installations, restart the cluster. + +Your Trino cluster is now configured with fault-tolerant query +execution. If a query run on the cluster would normally fail due to an +interruption of query processing, fault-tolerant execution now resumes the +query processing to ensure successful execution of the query. + +## Next steps + +For more information about fault-tolerant execution, including simple query +retries that do not require an exchange manager and advanced configuration +operations, see the {doc}`reference documentation +`. diff --git a/430/_sources/installation/rpm.md.txt b/430/_sources/installation/rpm.md.txt new file mode 100644 index 000000000..ca2ebe311 --- /dev/null +++ b/430/_sources/installation/rpm.md.txt @@ -0,0 +1,87 @@ +# RPM package + +Users can install Trino using the RPM Package Manager (RPM) on some Linux +distributions that support RPM. + +The RPM archive contains the application, all plugins, the necessary default +configuration files, default setups, and integration with the operating system +to start as a service. + +:::{warning} +It is recommended to deploy Trino with the {doc}`Helm chart ` on +Kubernetes or manually with the {doc}`Docker containers ` or the +{doc}`tar archive `. While the RPM is available for use, it is +discouraged in favor of the tarball or Docker containers. +::: + +## Installing Trino + +Download the Trino server RPM package {maven_download}`server-rpm`. Use the +`rpm` command to install the package: + +```text +rpm -i trino-server-rpm-*.rpm --nodeps +``` + +Installing the {ref}`required Java and Python setup ` must be +managed separately. + +## Service script + +The RPM installation deploys a service script configured with `systemctl` so +that the service can be started automatically on operating system boot. After +installation, you can manage the Trino server with the `service` command: + +```text +service trino [start|stop|restart|status] +``` + +:::{list-table} `service` commands +:widths: 15, 85 +:header-rows: 1 + +* - Command + - Action +* - `start` + - Starts the server as a daemon and returns its process ID. +* - `stop` + - Shuts down a server started with either `start` or `run`. Sends the + SIGTERM signal. +* - `restart` + - Stops and then starts a running server, or starts a stopped server, + assigning a new process ID. +* - `status` + - Prints a status line, either *Stopped pid* or *Running as pid*. +::: + +## Installation directory structure + +The RPM installation places Trino files in accordance with the Linux Filesystem +Hierarchy Standard using the following directory structure: + +- `/usr/lib/trino/lib/`: Contains the various libraries needed to run the + product. Plugins go in a `plugin` subdirectory. +- `/etc/trino`: Contains the general Trino configuration files like + `node.properties`, `jvm.config`, `config.properties`. Catalog + configurations go in a `catalog` subdirectory. +- `/etc/trino/env.sh`: Contains the Java installation path used by Trino, + allows configuring process environment variables, including {doc}`secrets + `. +- `/var/log/trino`: Contains the log files. +- `/var/lib/trino/data`: The location of the data directory. Trino stores logs + and other data here. +- `/etc/rc.d/init.d/trino`: Contains the service scripts for controlling the + server process, and launcher configuration for file paths. + +## Uninstalling + +Uninstalling the RPM is like uninstalling any other RPM, just run: + +```text +rpm -e trino-server-rpm- +``` + +Note: During uninstall, all Trino related files are deleted except for +user-created configuration files, copies of the original configuration files +`node.properties.rpmsave` and `env.sh.rpmsave` located in the `/etc/trino` +directory, and the Trino logs directory `/var/log/trino`. diff --git a/430/_sources/language.md.txt b/430/_sources/language.md.txt new file mode 100644 index 000000000..9c163f925 --- /dev/null +++ b/430/_sources/language.md.txt @@ -0,0 +1,31 @@ +# SQL language + +Trino is an ANSI SQL compliant query engine. This standard compliance allows +Trino users to integrate their favorite data tools, including BI and ETL tools +with any underlying data source. + +Trino validates and translates the received SQL statements into the necessary +operations on the connected data source. + +This section provides a reference to the supported SQL data types and other +general characteristics of the SQL support of Trino. + +Refer to the following sections for further details: + +* [SQL statement and syntax reference](/sql) +* [SQL functions and operators](/functions) + + +```{toctree} +:maxdepth: 2 + +language/sql-support +language/types +``` + +```{toctree} +:maxdepth: 1 + +language/reserved +language/comments +``` diff --git a/430/_sources/language/comments.md.txt b/430/_sources/language/comments.md.txt new file mode 100644 index 000000000..eb1614638 --- /dev/null +++ b/430/_sources/language/comments.md.txt @@ -0,0 +1,26 @@ +# Comments + +## Synopsis + +Comments are part of a SQL statement or script that are ignored for processing. +Comments begin with double dashes and extend to the end of the line. Block +comments begin with `/*` and extend to the next occurrence of `*/`, possibly +spanning over multiple lines. + +## Examples + +The following example displays a comment line, a comment after a valid +statement, and a block comment: + +```sql +-- This is a comment. +SELECT * FROM table; -- This comment is ignored. + +/* This is a block comment + that spans multiple lines + until it is closed. */ +``` + +## See also + +[](/sql/comment) diff --git a/430/_sources/language/reserved.md.txt b/430/_sources/language/reserved.md.txt new file mode 100644 index 000000000..a33d122c8 --- /dev/null +++ b/430/_sources/language/reserved.md.txt @@ -0,0 +1,125 @@ +# Keywords and identifiers + +(language-keywords)= +## Reserved keywords + +The following table lists all of the keywords that are reserved in Trino, +along with their status in the SQL standard. These reserved keywords must +be quoted (using double quotes) in order to be used as an identifier. + +| Keyword | SQL:2016 | SQL-92 | +| ------------------- | -------- | -------- | +| `ALTER` | reserved | reserved | +| `AND` | reserved | reserved | +| `AS` | reserved | reserved | +| `BETWEEN` | reserved | reserved | +| `BY` | reserved | reserved | +| `CASE` | reserved | reserved | +| `CAST` | reserved | reserved | +| `CONSTRAINT` | reserved | reserved | +| `CREATE` | reserved | reserved | +| `CROSS` | reserved | reserved | +| `CUBE` | reserved | | +| `CURRENT_CATALOG` | reserved | | +| `CURRENT_DATE` | reserved | reserved | +| `CURRENT_PATH` | reserved | | +| `CURRENT_ROLE` | reserved | reserved | +| `CURRENT_SCHEMA` | reserved | | +| `CURRENT_TIME` | reserved | reserved | +| `CURRENT_TIMESTAMP` | reserved | reserved | +| `CURRENT_USER` | reserved | | +| `DEALLOCATE` | reserved | reserved | +| `DELETE` | reserved | reserved | +| `DESCRIBE` | reserved | reserved | +| `DISTINCT` | reserved | reserved | +| `DROP` | reserved | reserved | +| `ELSE` | reserved | reserved | +| `END` | reserved | reserved | +| `ESCAPE` | reserved | reserved | +| `EXCEPT` | reserved | reserved | +| `EXECUTE` | reserved | reserved | +| `EXISTS` | reserved | reserved | +| `EXTRACT` | reserved | reserved | +| `FALSE` | reserved | reserved | +| `FOR` | reserved | reserved | +| `FROM` | reserved | reserved | +| `FULL` | reserved | reserved | +| `GROUP` | reserved | reserved | +| `GROUPING` | reserved | | +| `HAVING` | reserved | reserved | +| `IN` | reserved | reserved | +| `INNER` | reserved | reserved | +| `INSERT` | reserved | reserved | +| `INTERSECT` | reserved | reserved | +| `INTO` | reserved | reserved | +| `IS` | reserved | reserved | +| `JOIN` | reserved | reserved | +| `JSON_ARRAY` | reserved | | +| `JSON_EXISTS` | reserved | | +| `JSON_OBJECT` | reserved | | +| `JSON_QUERY` | reserved | | +| `JSON_TABLE` | reserved | | +| `JSON_VALUE` | reserved | | +| `LEFT` | reserved | reserved | +| `LIKE` | reserved | reserved | +| `LISTAGG` | reserved | | +| `LOCALTIME` | reserved | | +| `LOCALTIMESTAMP` | reserved | | +| `NATURAL` | reserved | reserved | +| `NORMALIZE` | reserved | | +| `NOT` | reserved | reserved | +| `NULL` | reserved | reserved | +| `ON` | reserved | reserved | +| `OR` | reserved | reserved | +| `ORDER` | reserved | reserved | +| `OUTER` | reserved | reserved | +| `PREPARE` | reserved | reserved | +| `RECURSIVE` | reserved | | +| `RIGHT` | reserved | reserved | +| `ROLLUP` | reserved | | +| `SELECT` | reserved | reserved | +| `SKIP` | reserved | | +| `TABLE` | reserved | reserved | +| `THEN` | reserved | reserved | +| `TRIM` | reserved | reserved | +| `TRUE` | reserved | reserved | +| `UESCAPE` | reserved | | +| `UNION` | reserved | reserved | +| `UNNEST` | reserved | | +| `USING` | reserved | reserved | +| `VALUES` | reserved | reserved | +| `WHEN` | reserved | reserved | +| `WHERE` | reserved | reserved | +| `WITH` | reserved | reserved | + +(language-identifiers)= +## Identifiers + +Tokens that identify names of catalogs, schemas, tables, columns, functions, or +other objects, are identifiers. + +Identifiers must start with a letter, and subsequently include alphanumeric +characters and underscores. Identifiers with other characters must be delimited +with double quotes (`"`). When delimited with double quotes, identifiers can use +any character. Escape a `"` with another preceding double quote in a delimited +identifier. + +Identifiers are not treated as case sensitive. + +Following are some valid examples: + +```sql +tablename +SchemaName +example_catalog.a_schema."table$partitions" +"identifierWith""double""quotes" +``` + +The following identifiers are invalid in Trino and must be quoted when used: + +```text +table-name +123SchemaName +colum$name@field +``` + diff --git a/430/_sources/language/sql-support.md.txt b/430/_sources/language/sql-support.md.txt new file mode 100644 index 000000000..419ef207c --- /dev/null +++ b/430/_sources/language/sql-support.md.txt @@ -0,0 +1,151 @@ +# SQL statement support + +The SQL statement support in Trino can be categorized into several topics. Many +statements are part of the core engine and therefore available in all use cases. +For example, you can always set session properties or inspect an explain plan +and perform other actions with the {ref}`globally available statements +`. + +However, the details and architecture of the connected data sources can limit +some SQL functionality. For example, if the data source does not support any +write operations, then a {doc}`/sql/delete` statement cannot be executed against +the data source. + +Similarly, if the underlying system does not have any security concepts, SQL +statements like {doc}`/sql/create-role` cannot be supported by Trino and the +connector. + +The categories of these different topics are related to {ref}`read operations +`, {ref}`write operations `, +{ref}`security operations ` and {ref}`transactions +`. + +Details of the support for specific statements is available with the +documentation for each connector. + +(sql-globally-available)= + +## Globally available statements + +The following statements are implemented in the core engine and available with +any connector: + +- {doc}`/sql/call` +- {doc}`/sql/deallocate-prepare` +- {doc}`/sql/describe-input` +- {doc}`/sql/describe-output` +- {doc}`/sql/execute` +- {doc}`/sql/execute-immediate` +- {doc}`/sql/explain` +- {doc}`/sql/explain-analyze` +- {doc}`/sql/prepare` +- {doc}`/sql/reset-session` +- {doc}`/sql/set-session` +- {doc}`/sql/set-time-zone` +- {doc}`/sql/show-functions` +- {doc}`/sql/show-session` +- {doc}`/sql/use` +- {doc}`/sql/values` + +(sql-read-operations)= + +## Read operations + +The following statements provide read access to data and meta data exposed by a +connector accessing a data source. They are supported by all connectors: + +- {doc}`/sql/select` including {doc}`/sql/match-recognize` +- {doc}`/sql/describe` +- {doc}`/sql/show-catalogs` +- {doc}`/sql/show-columns` +- {doc}`/sql/show-create-materialized-view` +- {doc}`/sql/show-create-schema` +- {doc}`/sql/show-create-table` +- {doc}`/sql/show-create-view` +- {doc}`/sql/show-grants` +- {doc}`/sql/show-roles` +- {doc}`/sql/show-schemas` +- {doc}`/sql/show-tables` +- {doc}`/sql/show-stats` + +(sql-write-operations)= + +## Write operations + +The following statements provide write access to data and meta data exposed +by a connector accessing a data source. Availability varies widely from +connector to connector: + +(sql-data-management)= + +### Data management + +- {doc}`/sql/insert` +- {doc}`/sql/update` +- {doc}`/sql/delete` +- {doc}`/sql/truncate` +- {doc}`/sql/merge` + +(sql-materialized-view-management)= + +### Materialized view management + +- {doc}`/sql/create-materialized-view` +- {doc}`/sql/alter-materialized-view` +- {doc}`/sql/drop-materialized-view` +- {doc}`/sql/refresh-materialized-view` + +(sql-schema-table-management)= + +### Schema and table management + +- {doc}`/sql/create-table` +- {doc}`/sql/create-table-as` +- {doc}`/sql/drop-table` +- {doc}`/sql/alter-table` +- {doc}`/sql/create-schema` +- {doc}`/sql/drop-schema` +- {doc}`/sql/alter-schema` +- {doc}`/sql/comment` + +(sql-view-management)= + +### View management + +- {doc}`/sql/create-view` +- {doc}`/sql/drop-view` +- {doc}`/sql/alter-view` + +(sql-security-operations)= + +## Security operations + +The following statements provide security-related operations to security +configuration, data, and meta data exposed by a connector accessing a data +source. Most connectors do not support these operations: + +Connector roles: + +- {doc}`/sql/create-role` +- {doc}`/sql/drop-role` +- {doc}`/sql/grant-roles` +- {doc}`/sql/revoke-roles` +- {doc}`/sql/set-role` +- {doc}`/sql/show-role-grants` + +Grants management: + +- {doc}`/sql/deny` +- {doc}`/sql/grant` +- {doc}`/sql/revoke` + +(sql-transactions)= + +## Transactions + +The following statements manage transactions. Most connectors do not support +transactions: + +- {doc}`/sql/start-transaction` +- {doc}`/sql/commit` +- {doc}`/sql/rollback` diff --git a/430/_sources/language/types.md.txt b/430/_sources/language/types.md.txt new file mode 100644 index 000000000..97bf46da3 --- /dev/null +++ b/430/_sources/language/types.md.txt @@ -0,0 +1,461 @@ +# Data types + +Trino has a set of built-in data types, described below. +Additional types can be provided by plugins. + +(type-mapping-overview)= + +## Trino type support and mapping + +Connectors to data sources are not required to support all Trino data types +described on this page. If there are data types similar to Trino's that are used +on the data source, the connector may map the Trino and remote data types to +each other as needed. + +Depending on the connector and the data source, type mapping may apply +in either direction as follows: + +- **Data source to Trino** mapping applies to any operation where columns in the + data source are read by Trino, such as a {doc}`/sql/select` statement, and the + underlying source data type needs to be represented by a Trino data type. +- **Trino to data source** mapping applies to any operation where the columns + or expressions in Trino need to be translated into data types or expressions + compatible with the underlying data source. For example, + {doc}`/sql/create-table-as` statements specify Trino types that are then + mapped to types on the remote data source. Predicates like `WHERE` also use + these mappings in order to ensure that the predicate is translated to valid + syntax on the remote data source. + +Data type support and mappings vary depending on the connector. Refer to the +{doc}`connector documentation ` for more information. + +(boolean-data-types)= + +## Boolean + +### `BOOLEAN` + +This type captures boolean values `true` and `false`. + +(integer-data-types)= + +## Integer + +Integer numbers can be expressed as numeric literals in the following formats: + +* Decimal integer. Examples are `-7`, `0`, or `3`. +* Hexadecimal integer composed of `0X` or `0x` and the value. Examples are + `0x0A` for decimal `10` or `0x11` for decimal `17`. +* Octal integer composed of `0O` or `0o` and the value. Examples are `0o40` for + decimal `32` or `0o11` for decimal `9`. +* Binary integer composed of `0B` or `0b` and the value. Examples are `0b1001` + for decimal `9` or `0b101010` for decimal `42``. + +Underscore characters are ignored within literal values, and can be used to +increase readability. For example, decimal integer `123_456.789_123` is +equivalent to `123456.789123`. Preceding and trailing underscores are not +permitted. + +Integers are supported by the following data types. + +### `TINYINT` + +A 8-bit signed two's complement integer with a minimum value of +`-2^7` or `-0x80` and a maximum value of `2^7 - 1` or `0x7F`. + +### `SMALLINT` + +A 16-bit signed two's complement integer with a minimum value of +`-2^15` or `-0x8000` and a maximum value of `2^15 - 1` or `0x7FFF`. + +### `INTEGER` or `INT` + +A 32-bit signed two's complement integer with a minimum value of `-2^31` or +`-0x80000000` and a maximum value of `2^31 - 1` or `0x7FFFFFFF`. The names +`INTEGER` and `INT` can both be used for this type. + +### `BIGINT` + +A 64-bit signed two's complement integer with a minimum value of `-2^63` or +`-0x8000000000000000` and a maximum value of `2^63 - 1` or `0x7FFFFFFFFFFFFFFF`. + +(floating-point-data-types)= + +## Floating-point + +Floating-point, fixed-precision numbers can be expressed as numeric literal +using scientific notation such as `1.03e1` and are cast as `DOUBLE` data type. +Underscore characters are ignored within literal values, and can be used to +increase readability. For example, value `123_456.789e4` is equivalent to +`123456.789e4`. Preceding underscores, trailing underscores, and underscores +beside the comma (`.`) are not permitted. + +### `REAL` + +A real is a 32-bit inexact, variable-precision implementing the +IEEE Standard 754 for Binary Floating-Point Arithmetic. + +Example literals: `REAL '10.3'`, `REAL '10.3e0'`, `REAL '1.03e1'` + +### `DOUBLE` + +A double is a 64-bit inexact, variable-precision implementing the +IEEE Standard 754 for Binary Floating-Point Arithmetic. + +Example literals: `DOUBLE '10.3'`, `DOUBLE '1.03e1'`, `10.3e0`, `1.03e1` + +(fixed-precision-data-types)= + +## Fixed-precision + +Fixed-precision numbers can be expressed as numeric literals such as `1.1`, and +are supported by the `DECIMAL` data type. + +Underscore characters are ignored within literal values, and can be used to +increase readability. For example, decimal `123_456.789_123` is equivalent to +`123456.789123`. Preceding underscores, trailing underscores, and underscores +beside the comma (`.`) are not permitted. + +Leading zeros in literal values are permitted and ignored. For example, +`000123.456` is equivalent to `123.456`. + +### `DECIMAL` + +A fixed-precision decimal number. Precision up to 38 digits is supported +but performance is best up to 18 digits. + +The decimal type takes two literal parameters: + +- **precision** - total number of digits +- **scale** - number of digits in fractional part. Scale is optional and defaults to 0. + +Example type definitions: `DECIMAL(10,3)`, `DECIMAL(20)` + +Example literals: `DECIMAL '10.3'`, `DECIMAL '1234567890'`, `1.1` + +(string-data-types)= + +## String + +### `VARCHAR` + +Variable length character data with an optional maximum length. + +Example type definitions: `varchar`, `varchar(20)` + +SQL statements support simple literal, as well as Unicode usage: + +- literal string : `'Hello winter !'` +- Unicode string with default escape character: `U&'Hello winter \2603 !'` +- Unicode string with custom escape character: `U&'Hello winter #2603 !' UESCAPE '#'` + +A Unicode string is prefixed with `U&` and requires an escape character +before any Unicode character usage with 4 digits. In the examples above +`\2603` and `#2603` represent a snowman character. Long Unicode codes +with 6 digits require usage of the plus symbol before the code. For example, +you need to use `\+01F600` for a grinning face emoji. + +### `CHAR` + +Fixed length character data. A `CHAR` type without length specified has a default length of 1. +A `CHAR(x)` value always has `x` characters. For example, casting `dog` to `CHAR(7)` +adds 4 implicit trailing spaces. Leading and trailing spaces are included in comparisons of +`CHAR` values. As a result, two character values with different lengths (`CHAR(x)` and +`CHAR(y)` where `x != y`) will never be equal. + +Example type definitions: `char`, `char(20)` + +### `VARBINARY` + +Variable length binary data. + +SQL statements support usage of binary literal data with the prefix `X` or `x`. +The binary data has to use hexadecimal format. For example, the binary form of +`eh?` is `X'65683F'` as you can confirm with the following statement: + +```sql +SELECT from_utf8(x'65683F'); +``` + +:::{note} +Binary strings with length are not yet supported: `varbinary(n)` +::: + +### `JSON` + +JSON value type, which can be a JSON object, a JSON array, a JSON number, a JSON string, +`true`, `false` or `null`. + +(date-time-data-types)= + +## Date and time + +See also {doc}`/functions/datetime` + +(date-data-type)= + +### `DATE` + +Calendar date (year, month, day). + +Example: `DATE '2001-08-22'` + +### `TIME` + +`TIME` is an alias for `TIME(3)` (millisecond precision). + +### `TIME(P)` + +Time of day (hour, minute, second) without a time zone with `P` digits of precision +for the fraction of seconds. A precision of up to 12 (picoseconds) is supported. + +Example: `TIME '01:02:03.456'` + +### `TIME WITH TIME ZONE` + +Time of day (hour, minute, second, millisecond) with a time zone. +Values of this type are rendered using the time zone from the value. +Time zones are expressed as the numeric UTC offset value: + +``` +SELECT TIME '01:02:03.456 -08:00'; +-- 1:02:03.456-08:00 +``` + +(timestamp-data-type)= + +### `TIMESTAMP` + +`TIMESTAMP` is an alias for `TIMESTAMP(3)` (millisecond precision). + +### `TIMESTAMP(P)` + +Calendar date and time of day without a time zone with `P` digits of precision +for the fraction of seconds. A precision of up to 12 (picoseconds) is supported. +This type is effectively a combination of the `DATE` and `TIME(P)` types. + +`TIMESTAMP(P) WITHOUT TIME ZONE` is an equivalent name. + +Timestamp values can be constructed with the `TIMESTAMP` literal +expression. Alternatively, language constructs such as +`localtimestamp(p)`, or a number of {doc}`date and time functions and +operators ` can return timestamp values. + +Casting to lower precision causes the value to be rounded, and not +truncated. Casting to higher precision appends zeros for the additional +digits. + +The following examples illustrate the behavior: + +``` +SELECT TIMESTAMP '2020-06-10 15:55:23'; +-- 2020-06-10 15:55:23 + +SELECT TIMESTAMP '2020-06-10 15:55:23.383345'; +-- 2020-06-10 15:55:23.383345 + +SELECT typeof(TIMESTAMP '2020-06-10 15:55:23.383345'); +-- timestamp(6) + +SELECT cast(TIMESTAMP '2020-06-10 15:55:23.383345' as TIMESTAMP(1)); + -- 2020-06-10 15:55:23.4 + +SELECT cast(TIMESTAMP '2020-06-10 15:55:23.383345' as TIMESTAMP(12)); +-- 2020-06-10 15:55:23.383345000000 +``` + +(timestamp-with-time-zone-data-type)= + +### `TIMESTAMP WITH TIME ZONE` + +`TIMESTAMP WITH TIME ZONE` is an alias for `TIMESTAMP(3) WITH TIME ZONE` +(millisecond precision). + +### `TIMESTAMP(P) WITH TIME ZONE` + +Instant in time that includes the date and time of day with `P` digits of +precision for the fraction of seconds and with a time zone. Values of this +type are rendered using the time zone from the value. +Time zones can be expressed in the following ways: + +- `UTC`, with `GMT`, `Z`, or `UT` usable as aliases for UTC. +- `+hh:mm` or `-hh:mm` with `hh:mm` as an hour and minute offset from UTC. + Can be written with or without `UTC`, `GMT`, or `UT` as an alias for + UTC. +- An [IANA time zone name](https://www.iana.org/time-zones). + +The following examples demonstrate some of these syntax options: + +``` +SELECT TIMESTAMP '2001-08-22 03:04:05.321 UTC'; +-- 2001-08-22 03:04:05.321 UTC + +SELECT TIMESTAMP '2001-08-22 03:04:05.321 -08:30'; +-- 2001-08-22 03:04:05.321 -08:30 + +SELECT TIMESTAMP '2001-08-22 03:04:05.321 GMT-08:30'; +-- 2001-08-22 03:04:05.321 -08:30 + +SELECT TIMESTAMP '2001-08-22 03:04:05.321 America/New_York'; +-- 2001-08-22 03:04:05.321 America/New_York +``` + +### `INTERVAL YEAR TO MONTH` + +Span of years and months. + +Example: `INTERVAL '3' MONTH` + +### `INTERVAL DAY TO SECOND` + +Span of days, hours, minutes, seconds and milliseconds. + +Example: `INTERVAL '2' DAY` + +(structural-data-types)= + +## Structural + +(array-type)= + +### `ARRAY` + +An array of the given component type. + +Example: `ARRAY[1, 2, 3]` + +(map-type)= + +### `MAP` + +A map between the given component types. + +Example: `MAP(ARRAY['foo', 'bar'], ARRAY[1, 2])` + +(row-type)= + +### `ROW` + +A structure made up of fields that allows mixed types. +The fields may be of any SQL type. + +By default, row fields are not named, but names can be assigned. + +Example: `CAST(ROW(1, 2e0) AS ROW(x BIGINT, y DOUBLE))` + +Named row fields are accessed with field reference operator (`.`). + +Example: `CAST(ROW(1, 2.0) AS ROW(x BIGINT, y DOUBLE)).x` + +Named or unnamed row fields are accessed by position with the subscript +operator (`[]`). The position starts at `1` and must be a constant. + +Example: `ROW(1, 2.0)[1]` + +## Network address + +(ipaddress-type)= + +### `IPADDRESS` + +An IP address that can represent either an IPv4 or IPv6 address. Internally, +the type is a pure IPv6 address. Support for IPv4 is handled using the +*IPv4-mapped IPv6 address* range ({rfc}`4291#section-2.5.5.2`). +When creating an `IPADDRESS`, IPv4 addresses will be mapped into that range. +When formatting an `IPADDRESS`, any address within the mapped range will +be formatted as an IPv4 address. Other addresses will be formatted as IPv6 +using the canonical format defined in {rfc}`5952`. + +Examples: `IPADDRESS '10.0.0.1'`, `IPADDRESS '2001:db8::1'` + +## UUID + +(uuid-type)= + +### `UUID` + +This type represents a UUID (Universally Unique IDentifier), also known as a +GUID (Globally Unique IDentifier), using the format defined in {rfc}`4122`. + +Example: `UUID '12151fd2-7586-11e9-8f9e-2a86e4085a59'` + +## HyperLogLog + +Calculating the approximate distinct count can be done much more cheaply than an exact count using the +[HyperLogLog](https://wikipedia.org/wiki/HyperLogLog) data sketch. See {doc}`/functions/hyperloglog`. + +(hyperloglog-type)= + +### `HyperLogLog` + +A HyperLogLog sketch allows efficient computation of {func}`approx_distinct`. It starts as a +sparse representation, switching to a dense representation when it becomes more efficient. + +(p4hyperloglog-type)= + +### `P4HyperLogLog` + +A P4HyperLogLog sketch is similar to {ref}`hyperloglog-type`, but it starts (and remains) +in the dense representation. + +## SetDigest + +(setdigest-type)= + +### `SetDigest` + +A SetDigest (setdigest) is a data sketch structure used +in calculating [Jaccard similarity coefficient](https://wikipedia.org/wiki/Jaccard_index) +between two sets. + +SetDigest encapsulates the following components: + +- [HyperLogLog](https://wikipedia.org/wiki/HyperLogLog) +- [MinHash with a single hash function](http://wikipedia.org/wiki/MinHash#Variant_with_a_single_hash_function) + +The HyperLogLog structure is used for the approximation of the distinct elements +in the original set. + +The MinHash structure is used to store a low memory footprint signature of the original set. +The similarity of any two sets is estimated by comparing their signatures. + +SetDigests are additive, meaning they can be merged together. + +## Quantile digest + +(qdigest-type)= + +### `QDigest` + +A quantile digest (qdigest) is a summary structure which captures the approximate +distribution of data for a given input set, and can be queried to retrieve approximate +quantile values from the distribution. The level of accuracy for a qdigest +is tunable, allowing for more precise results at the expense of space. + +A qdigest can be used to give approximate answer to queries asking for what value +belongs at a certain quantile. A useful property of qdigests is that they are +additive, meaning they can be merged together without losing precision. + +A qdigest may be helpful whenever the partial results of `approx_percentile` +can be reused. For example, one may be interested in a daily reading of the 99th +percentile values that are read over the course of a week. Instead of calculating +the past week of data with `approx_percentile`, `qdigest`s could be stored +daily, and quickly merged to retrieve the 99th percentile value. + +## T-Digest + +(tdigest-type)= + +### `TDigest` + +A T-digest (tdigest) is a summary structure which, similarly to qdigest, captures the +approximate distribution of data for a given input set. It can be queried to retrieve +approximate quantile values from the distribution. + +TDigest has the following advantages compared to QDigest: + +- higher performance +- lower memory usage +- higher accuracy at high and low percentiles + +T-digests are additive, meaning they can be merged together. diff --git a/430/_sources/optimizer.md.txt b/430/_sources/optimizer.md.txt new file mode 100644 index 000000000..346d0fd82 --- /dev/null +++ b/430/_sources/optimizer.md.txt @@ -0,0 +1,10 @@ +# Query optimizer + +```{toctree} +:maxdepth: 1 + +optimizer/statistics +optimizer/cost-in-explain +optimizer/cost-based-optimizations +optimizer/pushdown +``` diff --git a/430/_sources/optimizer/cost-based-optimizations.md.txt b/430/_sources/optimizer/cost-based-optimizations.md.txt new file mode 100644 index 000000000..f3946e79d --- /dev/null +++ b/430/_sources/optimizer/cost-based-optimizations.md.txt @@ -0,0 +1,124 @@ +# Cost-based optimizations + +Trino supports several cost based optimizations, described below. + +## Join enumeration + +The order in which joins are executed in a query can have a significant impact +on the query's performance. The aspect of join ordering that has the largest +impact on performance is the size of the data being processed and transferred +over the network. If a join which produces a lot of data is performed early in +the query's execution, then subsequent stages need to process large amounts of +data for longer than necessary, increasing the time and resources needed for +processing the query. + +With cost-based join enumeration, Trino uses {doc}`/optimizer/statistics` +provided by connectors to estimate the costs for different join orders and +automatically picks the join order with the lowest computed costs. + +The join enumeration strategy is governed by the `join_reordering_strategy` +{ref}`session property `, with the +`optimizer.join-reordering-strategy` configuration property providing the +default value. + +The possible values are: + +> - `AUTOMATIC` (default) - enable full automatic join enumeration +> - `ELIMINATE_CROSS_JOINS` - eliminate unnecessary cross joins +> - `NONE` - purely syntactic join order + +If you are using `AUTOMATIC` join enumeration and statistics are not +available or a cost can not be computed for any other reason, the +`ELIMINATE_CROSS_JOINS` strategy is used instead. + +## Join distribution selection + +Trino uses a hash-based join algorithm. For each join operator, a hash table +must be created from one join input, referred to as the build side. The other +input, called the probe side, is then iterated on. For each row, the hash table +is queried to find matching rows. + +There are two types of join distributions: + +> - Partitioned: each node participating in the query builds a hash table from +> only a fraction of the data +> - Broadcast: each node participating in the query builds a hash table from all +> of the data. The data is replicated to each node. + +Each type has advantages and disadvantages. Partitioned joins require +redistributing both tables using a hash of the join key. These joins can be much +slower than broadcast joins, but they allow much larger joins overall. Broadcast +joins are faster if the build side is much smaller than the probe side. However, +broadcast joins require that the tables on the build side of the join after +filtering fit in memory on each node, whereas distributed joins only need to fit +in distributed memory across all nodes. + +With cost-based join distribution selection, Trino automatically chooses whether +to use a partitioned or broadcast join. With cost-based join enumeration, Trino +automatically chooses which sides are probe and build. + +The join distribution strategy is governed by the `join_distribution_type` +session property, with the `join-distribution-type` configuration property +providing the default value. + +The valid values are: + +> - `AUTOMATIC` (default) - join distribution type is determined automatically +> for each join +> - `BROADCAST` - broadcast join distribution is used for all joins +> - `PARTITIONED` - partitioned join distribution is used for all join + +### Capping replicated table size + +The join distribution type is automatically chosen when the join reordering +strategy is set to `AUTOMATIC` or when the join distribution type is set to +`AUTOMATIC`. In both cases, it is possible to cap the maximum size of the +replicated table with the `join-max-broadcast-table-size` configuration +property or with the `join_max_broadcast_table_size` session property. This +allows you to improve cluster concurrency and prevent bad plans when the +cost-based optimizer misestimates the size of the joined tables. + +By default, the replicated table size is capped to 100MB. + +## Syntactic join order + +If not using cost-based optimization, Trino defaults to syntactic join ordering. +While there is no formal way to optimize queries for this case, it is possible +to take advantage of how Trino implements joins to make them more performant. + +Trino uses in-memory hash joins. When processing a join statement, Trino loads +the right-most table of the join into memory as the build side, then streams the +next right-most table as the probe side to execute the join. If a query has +multiple joins, the result of this first join stays in memory as the build side, +and the third right-most table is then used as the probe side, and so on for +additional joins. In the case where join order is made more complex, such as +when using parentheses to specify specific parents for joins, Trino may execute +multiple lower-level joins at once, but each step of that process follows the +same logic, and the same applies when the results are ultimately joined +together. + +Because of this behavior, it is optimal to syntactically order joins in your SQL +queries from the largest tables to the smallest, as this minimizes memory usage. + +As an example, if you have a small, medium, and large table and are using left +joins: + +```sql +SELECT + * +FROM + large_table l + LEFT JOIN medium_table m ON l.user_id = m.user_id + LEFT JOIN small_table s ON s.user_id = l.user_id +``` + +:::{warning} +This means of optimization is not a feature of Trino. It is an artifact of +how joins are implemented, and therefore this behavior may change without +notice. +::: + +## Connector implementations + +In order for the Trino optimizer to use the cost based strategies, +the connector implementation must provide {doc}`statistics`. diff --git a/430/_sources/optimizer/cost-in-explain.md.txt b/430/_sources/optimizer/cost-in-explain.md.txt new file mode 100644 index 000000000..5bd36ef9e --- /dev/null +++ b/430/_sources/optimizer/cost-in-explain.md.txt @@ -0,0 +1,43 @@ +# Cost in EXPLAIN + +During planning, the cost associated with each node of the plan is computed +based on the table statistics for the tables in the query. This calculated +cost is printed as part of the output of an {doc}`/sql/explain` statement. + +Cost information is displayed in the plan tree using the format `{rows: XX +(XX), cpu: XX, memory: XX, network: XX}`. `rows` refers to the expected +number of rows output by each plan node during execution. The value in the +parentheses following the number of rows refers to the expected size of the data +output by each plan node in bytes. Other parameters indicate the estimated +amount of CPU, memory, and network utilized by the execution of a plan node. +These values do not represent any actual unit, but are numbers that are used to +compare the relative costs between plan nodes, allowing the optimizer to choose +the best plan for executing a query. If any of the values is not known, a `?` +is printed. + +For example: + +``` +EXPLAIN SELECT comment FROM tpch.sf1.nation WHERE nationkey > 3; +``` + +```text +- Output[comment] => [[comment]] + Estimates: {rows: 22 (1.69kB), cpu: 6148.25, memory: 0.00, network: 1734.25} + - RemoteExchange[GATHER] => [[comment]] + Estimates: {rows: 22 (1.69kB), cpu: 6148.25, memory: 0.00, network: 1734.25} + - ScanFilterProject[table = tpch:nation:sf1.0, filterPredicate = ("nationkey" > BIGINT '3')] => [[comment]] + Estimates: {rows: 25 (1.94kB), cpu: 2207.00, memory: 0.00, network: 0.00}/{rows: 22 (1.69kB), cpu: 4414.00, memory: 0.00, network: 0.00}/{rows: 22 (1.69kB), cpu: 6148.25, memory: 0.00, network: 0.00} + nationkey := tpch:nationkey + comment := tpch:comment +``` + +Generally, there is only one cost printed for each plan node. However, when a +`Scan` operator is combined with a `Filter` and/or `Project` operator, +then multiple cost structures are printed, each corresponding to an +individual logical part of the combined operator. For example, three cost +structures are printed for a `ScanFilterProject` operator, corresponding +to the `Scan`, `Filter`, and `Project` parts of the operator, in that order. + +Estimated cost is also printed in {doc}`/sql/explain-analyze` in addition to actual +runtime statistics. diff --git a/430/_sources/optimizer/pushdown.md.txt b/430/_sources/optimizer/pushdown.md.txt new file mode 100644 index 000000000..960aa1953 --- /dev/null +++ b/430/_sources/optimizer/pushdown.md.txt @@ -0,0 +1,362 @@ +# Pushdown + +Trino can push down the processing of queries, or parts of queries, into the +connected data source. This means that a specific predicate, aggregation +function, or other operation, is passed through to the underlying database or +storage system for processing. + +The results of this pushdown can include the following benefits: + +- Improved overall query performance +- Reduced network traffic between Trino and the data source +- Reduced load on the remote data source + +These benefits often result in significant cost reduction. + +Support for pushdown is specific to each connector and the relevant underlying +database or storage system. + +(predicate-pushdown)= + +## Predicate pushdown + +Predicate pushdown optimizes row-based filtering. It uses the inferred filter, +typically resulting from a condition in a `WHERE` clause to omit unnecessary +rows. The processing is pushed down to the data source by the connector and then +processed by the data source. + +If predicate pushdown for a specific clause is succesful, the `EXPLAIN` plan +for the query does not include a `ScanFilterProject` operation for that +clause. + +(projection-pushdown)= + +## Projection pushdown + +Projection pushdown optimizes column-based filtering. It uses the columns +specified in the `SELECT` clause and other parts of the query to limit access +to these columns. The processing is pushed down to the data source by the +connector and then the data source only reads and returns the neccessary +columns. + +If projection pushdown is succesful, the `EXPLAIN` plan for the query only +accesses the relevant columns in the `Layout` of the `TableScan` operation. + +(dereference-pushdown)= + +## Dereference pushdown + +Projection pushdown and dereference pushdown limit access to relevant columns, +except dereference pushdown is more selective. It limits access to only read the +specified fields within a top level or nested `ROW` data type. + +For example, consider a table in the Hive connector that has a `ROW` type +column with several fields. If a query only accesses one field, dereference +pushdown allows the file reader to read only that single field within the row. +The same applies to fields of a row nested within the top level row. This can +result in significant savings in the amount of data read from the storage +system. + +(aggregation-pushdown)= + +## Aggregation pushdown + +Aggregation pushdown can take place provided the following conditions are satisfied: + +- If aggregation pushdown is generally supported by the connector. +- If pushdown of the specific function or functions is supported by the connector. +- If the query structure allows pushdown to take place. + +You can check if pushdown for a specific query is performed by looking at the +{doc}`EXPLAIN plan ` of the query. If an aggregate function is successfully +pushed down to the connector, the explain plan does **not** show that `Aggregate` operator. +The explain plan only shows the operations that are performed by Trino. + +As an example, we loaded the TPCH data set into a PostgreSQL database and then +queried it using the PostgreSQL connector: + +``` +SELECT regionkey, count(*) +FROM nation +GROUP BY regionkey; +``` + +You can get the explain plan by prepending the above query with `EXPLAIN`: + +``` +EXPLAIN +SELECT regionkey, count(*) +FROM nation +GROUP BY regionkey; +``` + +The explain plan for this query does not show any `Aggregate` operator with +the `count` function, as this operation is now performed by the connector. You +can see the `count(*)` function as part of the PostgreSQL `TableScan` +operator. This shows you that the pushdown was successful. + +```text +Fragment 0 [SINGLE] + Output layout: [regionkey_0, _generated_1] + Output partitioning: SINGLE [] + Output[regionkey, _col1] + │ Layout: [regionkey_0:bigint, _generated_1:bigint] + │ Estimates: {rows: ? (?), cpu: ?, memory: 0B, network: ?} + │ regionkey := regionkey_0 + │ _col1 := _generated_1 + └─ RemoteSource[1] + Layout: [regionkey_0:bigint, _generated_1:bigint] + +Fragment 1 [SOURCE] + Output layout: [regionkey_0, _generated_1] + Output partitioning: SINGLE [] + TableScan[postgresql:tpch.nation tpch.nation columns=[regionkey:bigint:int8, count(*):_generated_1:bigint:bigint] groupingSets=[[regionkey:bigint:int8]], gro + Layout: [regionkey_0:bigint, _generated_1:bigint] + Estimates: {rows: ? (?), cpu: ?, memory: 0B, network: 0B} + _generated_1 := count(*):_generated_1:bigint:bigint + regionkey_0 := regionkey:bigint:int8 +``` + +A number of factors can prevent a push down: + +- adding a condition to the query +- using a different aggregate function that cannot be pushed down into the connector +- using a connector without pushdown support for the specific function + +As a result, the explain plan shows the `Aggregate` operation being performed +by Trino. This is a clear sign that now pushdown to the remote data source is not +performed, and instead Trino performs the aggregate processing. + +```text +Fragment 0 [SINGLE] + Output layout: [regionkey, count] + Output partitioning: SINGLE [] + Output[regionkey, _col1] + │ Layout: [regionkey:bigint, count:bigint] + │ Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?} + │ _col1 := count + └─ RemoteSource[1] + Layout: [regionkey:bigint, count:bigint] + +Fragment 1 [HASH] + Output layout: [regionkey, count] + Output partitioning: SINGLE [] + Aggregate(FINAL)[regionkey] + │ Layout: [regionkey:bigint, count:bigint] + │ Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?} + │ count := count("count_0") + └─ LocalExchange[HASH][$hashvalue] ("regionkey") + │ Layout: [regionkey:bigint, count_0:bigint, $hashvalue:bigint] + │ Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?} + └─ RemoteSource[2] + Layout: [regionkey:bigint, count_0:bigint, $hashvalue_1:bigint] + +Fragment 2 [SOURCE] + Output layout: [regionkey, count_0, $hashvalue_2] + Output partitioning: HASH [regionkey][$hashvalue_2] + Project[] + │ Layout: [regionkey:bigint, count_0:bigint, $hashvalue_2:bigint] + │ Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?} + │ $hashvalue_2 := combine_hash(bigint '0', COALESCE("$operator$hash_code"("regionkey"), 0)) + └─ Aggregate(PARTIAL)[regionkey] + │ Layout: [regionkey:bigint, count_0:bigint] + │ count_0 := count(*) + └─ TableScan[tpch:nation:sf0.01, grouped = false] + Layout: [regionkey:bigint] + Estimates: {rows: 25 (225B), cpu: 225, memory: 0B, network: 0B} + regionkey := tpch:regionkey +``` + +### Limitations + +Aggregation pushdown does not support a number of more complex statements: + +- complex grouping operations such as `ROLLUP`, `CUBE`, or `GROUPING SETS` +- expressions inside the aggregation function call: `sum(a * b)` +- coercions: `sum(integer_column)` +- {ref}`aggregations with ordering ` +- {ref}`aggregations with filter ` + +(join-pushdown)= + +## Join pushdown + +Join pushdown allows the connector to delegate the table join operation to the +underlying data source. This can result in performance gains, and allows Trino +to perform the remaining query processing on a smaller amount of data. + +The specifics for the supported pushdown of table joins varies for each data +source, and therefore for each connector. + +However, there are some generic conditions that must be met in order for a join +to be pushed down: + +- all predicates that are part of the join must be possible to be pushed down +- the tables in the join must be from the same catalog + +You can verify if pushdown for a specific join is performed by looking at the +{doc}`EXPLAIN ` plan of the query. The explain plan does not +show a `Join` operator, if the join is pushed down to the data source by the +connector: + +``` +EXPLAIN SELECT c.custkey, o.orderkey +FROM orders o JOIN customer c ON c.custkey = o.custkey; +``` + +The following plan results from the PostgreSQL connector querying TPCH +data in a PostgreSQL database. It does not show any `Join` operator as a +result of the successful join push down. + +```text +Fragment 0 [SINGLE] + Output layout: [custkey, orderkey] + Output partitioning: SINGLE [] + Output[custkey, orderkey] + │ Layout: [custkey:bigint, orderkey:bigint] + │ Estimates: {rows: ? (?), cpu: ?, memory: 0B, network: ?} + └─ RemoteSource[1] + Layout: [orderkey:bigint, custkey:bigint] + +Fragment 1 [SOURCE] + Output layout: [orderkey, custkey] + Output partitioning: SINGLE [] + TableScan[postgres:Query[SELECT l."orderkey" AS "orderkey_0", l."custkey" AS "custkey_1", r."custkey" AS "custkey_2" FROM (SELECT "orderkey", "custkey" FROM "tpch"."orders") l INNER JOIN (SELECT "custkey" FROM "tpch"."customer") r O + Layout: [orderkey:bigint, custkey:bigint] + Estimates: {rows: ? (?), cpu: ?, memory: 0B, network: 0B} + orderkey := orderkey_0:bigint:int8 + custkey := custkey_1:bigint:int8 +``` + +It is typically beneficial to push down a join. Pushing down a join can also +increase the row count compared to the size of the input to the join. This +may impact performance. + +(limit-pushdown)= + +## Limit pushdown + +A {ref}`limit-clause` reduces the number of returned records for a statement. +Limit pushdown enables a connector to push processing of such queries of +unsorted record to the underlying data source. + +A pushdown of this clause can improve the performance of the query and +significantly reduce the amount of data transferred from the data source to +Trino. + +Queries include sections such as `LIMIT N` or `FETCH FIRST N ROWS`. + +Implementation and support is connector-specific since different data sources have varying capabilities. + +(topn-pushdown)= + +## Top-N pushdown + +The combination of a {ref}`limit-clause` with an {ref}`order-by-clause` creates +a small set of records to return out of a large sorted dataset. It relies on the +order to determine which records need to be returned, and is therefore quite +different to optimize compared to a {ref}`limit-pushdown`. + +The pushdown for such a query is called a Top-N pushdown, since the operation is +returning the top N rows. It enables a connector to push processing of such +queries to the underlying data source, and therefore significantly reduces the +amount of data transferred to and processed by Trino. + +Queries include sections such as `ORDER BY ... LIMIT N` or `ORDER BY ... +FETCH FIRST N ROWS`. + +Implementation and support is connector-specific since different data sources +support different SQL syntax and processing. + +For example, you can find two queries to learn how to identify Top-N pushdown behavior in the following section. + +First, a concrete example of a Top-N pushdown query on top of a PostgreSQL database: + +``` +SELECT id, name +FROM postgresql.public.company +ORDER BY id +LIMIT 5; +``` + +You can get the explain plan by prepending the above query with `EXPLAIN`: + +``` +EXPLAIN SELECT id, name +FROM postgresql.public.company +ORDER BY id +LIMIT 5; +``` + +```text +Fragment 0 [SINGLE] + Output layout: [id, name] + Output partitioning: SINGLE [] + Stage Execution Strategy: UNGROUPED_EXECUTION + Output[id, name] + │ Layout: [id:integer, name:varchar] + │ Estimates: {rows: ? (?), cpu: ?, memory: 0B, network: ?} + └─ RemoteSource[1] + Layout: [id:integer, name:varchar] + +Fragment 1 [SOURCE] + Output layout: [id, name] + Output partitioning: SINGLE [] + Stage Execution Strategy: UNGROUPED_EXECUTION + TableScan[postgresql:public.company public.company sortOrder=[id:integer:int4 ASC NULLS LAST] limit=5, grouped = false] + Layout: [id:integer, name:varchar] + Estimates: {rows: ? (?), cpu: ?, memory: 0B, network: 0B} + name := name:varchar:text + id := id:integer:int4 +``` + +Second, an example of a Top-N query on the `tpch` connector which does not support +Top-N pushdown functionality: + +``` +SELECT custkey, name +FROM tpch.sf1.customer +ORDER BY custkey +LIMIT 5; +``` + +The related query plan: + +```text +Fragment 0 [SINGLE] + Output layout: [custkey, name] + Output partitioning: SINGLE [] + Stage Execution Strategy: UNGROUPED_EXECUTION + Output[custkey, name] + │ Layout: [custkey:bigint, name:varchar(25)] + │ Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?} + └─ TopN[5 by (custkey ASC NULLS LAST)] + │ Layout: [custkey:bigint, name:varchar(25)] + └─ LocalExchange[SINGLE] () + │ Layout: [custkey:bigint, name:varchar(25)] + │ Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?} + └─ RemoteSource[1] + Layout: [custkey:bigint, name:varchar(25)] + +Fragment 1 [SOURCE] + Output layout: [custkey, name] + Output partitioning: SINGLE [] + Stage Execution Strategy: UNGROUPED_EXECUTION + TopNPartial[5 by (custkey ASC NULLS LAST)] + │ Layout: [custkey:bigint, name:varchar(25)] + └─ TableScan[tpch:customer:sf1.0, grouped = false] + Layout: [custkey:bigint, name:varchar(25)] + Estimates: {rows: 150000 (4.58MB), cpu: 4.58M, memory: 0B, network: 0B} + custkey := tpch:custkey + name := tpch:name +``` + +In the preceding query plan, the Top-N operation `TopN[5 by (custkey ASC NULLS LAST)]` +is being applied in the `Fragment 0` by Trino and not by the source database. + +Note that, compared to the query executed on top of the `tpch` connector, +the explain plan of the query applied on top of the `postgresql` connector +is missing the reference to the operation `TopN[5 by (id ASC NULLS LAST)]` +in the `Fragment 0`. +The absence of the `TopN` Trino operator in the `Fragment 0` from the query plan +demonstrates that the query benefits of the Top-N pushdown optimization. diff --git a/430/_sources/optimizer/statistics.md.txt b/430/_sources/optimizer/statistics.md.txt new file mode 100644 index 000000000..f3b026bca --- /dev/null +++ b/430/_sources/optimizer/statistics.md.txt @@ -0,0 +1,32 @@ +# Table statistics + +Trino supports statistics based optimizations for queries. For a query to take +advantage of these optimizations, Trino must have statistical information for +the tables in that query. + +Table statistics are provided to the query planner by connectors. + +## Available statistics + +The following statistics are available in Trino: + +- For a table: + + - **row count**: the total number of rows in the table + +- For each column in a table: + + - **data size**: the size of the data that needs to be read + - **nulls fraction**: the fraction of null values + - **distinct value count**: the number of distinct values + - **low value**: the smallest value in the column + - **high value**: the largest value in the column + +The set of statistics available for a particular query depends on the connector +being used and can also vary by table. For example, the +Hive connector does not currently provide statistics on data size. + +Table statistics can be displayed via the Trino SQL interface using the +{doc}`/sql/show-stats` command. For the Hive connector, refer to the +{ref}`Hive connector ` documentation to learn how to update table +statistics. diff --git a/430/_sources/overview.md.txt b/430/_sources/overview.md.txt new file mode 100644 index 000000000..4367ab6e6 --- /dev/null +++ b/430/_sources/overview.md.txt @@ -0,0 +1,11 @@ +# Overview + +Trino is a distributed SQL query engine designed to query large data sets +distributed over one or more heterogeneous data sources. + +```{toctree} +:maxdepth: 1 + +overview/use-cases +overview/concepts +``` diff --git a/430/_sources/overview/concepts.md.txt b/430/_sources/overview/concepts.md.txt new file mode 100644 index 000000000..b81439e07 --- /dev/null +++ b/430/_sources/overview/concepts.md.txt @@ -0,0 +1,261 @@ +# Trino concepts + +## Overview + +To understand Trino, you must first understand the terms and concepts +used throughout the Trino documentation. + +While it is easy to understand statements and queries, as an end-user +you should have familiarity with concepts such as stages and splits to +take full advantage of Trino to execute efficient queries. As a +Trino administrator or a Trino contributor you should understand how +Trino's concepts of stages map to tasks and how tasks contain a set +of drivers which process data. + +This section provides a solid definition for the core concepts +referenced throughout Trino, and these sections are sorted from most +general to most specific. + +:::{note} +The book [Trino: The Definitive Guide](https://trino.io/trino-the-definitive-guide.html) and the research +paper [Presto: SQL on Everything](https://trino.io/paper.html) can +provide further information about Trino and the concepts in use. +::: + +(trino-concept-architecture)= + +## Architecture + +Trino is a distributed query engine that processes data in parallel across +multiple servers. There are two types of Trino servers, +{ref}`coordinators ` and +{ref}`workers `. The following sections describe these +servers and other components of Trino's architecture. + +(trino-concept-cluster)= + +### Cluster + +A Trino cluster consists of a {ref}`coordinator ` and +many {ref}`workers `. Users connect to the coordinator +with their {ref}`SQL ` query tool. The coordinator collaborates with the +workers. The coordinator and the workers access the connected +{ref}`data sources `. This access is configured in +{ref}`catalogs `. + +Processing each query is a stateful operation. The workload is orchestrated by +the coordinator and spread parallel across all workers in the cluster. Each node +runs Trino in one JVM instance, and processing is parallelized further using +threads. + +(trino-concept-coordinator)= + +### Coordinator + +The Trino coordinator is the server that is responsible for parsing +statements, planning queries, and managing Trino worker nodes. It is +the "brain" of a Trino installation and is also the node to which a +client connects to submit statements for execution. Every Trino +installation must have a Trino coordinator alongside one or more +Trino workers. For development or testing purposes, a single +instance of Trino can be configured to perform both roles. + +The coordinator keeps track of the activity on each worker and +coordinates the execution of a query. The coordinator creates +a logical model of a query involving a series of stages, which is then +translated into a series of connected tasks running on a cluster of +Trino workers. + +Coordinators communicate with workers and clients using a REST API. + +(trino-concept-worker)= + +### Worker + +A Trino worker is a server in a Trino installation, which is responsible +for executing tasks and processing data. Worker nodes fetch data from +connectors and exchange intermediate data with each other. The coordinator +is responsible for fetching results from the workers and returning the +final results to the client. + +When a Trino worker process starts up, it advertises itself to the discovery +server in the coordinator, which makes it available to the Trino coordinator +for task execution. + +Workers communicate with other workers and Trino coordinators +using a REST API. + +(trino-concept-data-sources)= + +## Data sources + +Throughout this documentation, you'll read terms such as connector, +catalog, schema, and table. These fundamental concepts cover Trino's +model of a particular data source and are described in the following +section. + +### Connector + +A connector adapts Trino to a data source such as Hive or a +relational database. You can think of a connector the same way you +think of a driver for a database. It is an implementation of Trino's +{doc}`SPI `, which allows Trino to interact +with a resource using a standard API. + +Trino contains several built-in connectors: a connector for +{doc}`JMX `, a {doc}`System ` +connector which provides access to built-in system tables, +a {doc}`Hive ` connector, and a +{doc}`TPCH ` connector designed to serve TPC-H benchmark +data. Many third-party developers have contributed connectors so that +Trino can access data in a variety of data sources. + +Every catalog is associated with a specific connector. If you examine +a catalog configuration file, you see that each contains a +mandatory property `connector.name`, which is used by the catalog +manager to create a connector for a given catalog. It is possible +to have more than one catalog use the same connector to access two +different instances of a similar database. For example, if you have +two Hive clusters, you can configure two catalogs in a single Trino +cluster that both use the Hive connector, allowing you to query data +from both Hive clusters, even within the same SQL query. + +(trino-concept-catalog)= + +### Catalog + +A Trino catalog contains schemas and references a data source via a +connector. For example, you can configure a JMX catalog to provide +access to JMX information via the JMX connector. When you run SQL +statements in Trino, you are running them against one or more catalogs. +Other examples of catalogs include the Hive catalog to connect to a +Hive data source. + +When addressing a table in Trino, the fully-qualified table name is +always rooted in a catalog. For example, a fully-qualified table name +of `hive.test_data.test` refers to the `test` table in the +`test_data` schema in the `hive` catalog. + +Catalogs are defined in properties files stored in the Trino +configuration directory. + +### Schema + +Schemas are a way to organize tables. Together, a catalog and schema +define a set of tables that can be queried. When accessing Hive or a +relational database such as MySQL with Trino, a schema translates to +the same concept in the target database. Other types of connectors may +choose to organize tables into schemas in a way that makes sense for +the underlying data source. + +### Table + +A table is a set of unordered rows, which are organized into named columns +with types. This is the same as in any relational database. The mapping +from source data to tables is defined by the connector. + +## Query execution model + +Trino executes SQL statements and turns these statements into queries, +that are executed across a distributed cluster of coordinator and workers. + +### Statement + +Trino executes ANSI-compatible SQL statements. When the Trino +documentation refers to a statement, it is referring to statements as +defined in the ANSI SQL standard, which consists of clauses, +expressions, and predicates. + +Some readers might be curious why this section lists separate concepts +for statements and queries. This is necessary because, in Trino, +statements simply refer to the textual representation of a statement written +in SQL. When a statement is executed, Trino creates a query along +with a query plan that is then distributed across a series of Trino +workers. + +### Query + +When Trino parses a statement, it converts it into a query and creates +a distributed query plan, which is then realized as a series of +interconnected stages running on Trino workers. When you retrieve +information about a query in Trino, you receive a snapshot of every +component that is involved in producing a result set in response to a +statement. + +The difference between a statement and a query is simple. A statement +can be thought of as the SQL text that is passed to Trino, while a query +refers to the configuration and components instantiated to execute +that statement. A query encompasses stages, tasks, splits, connectors, +and other components and data sources working in concert to produce a +result. + +(trino-concept-stage)= + +### Stage + +When Trino executes a query, it does so by breaking up the execution +into a hierarchy of stages. For example, if Trino needs to aggregate +data from one billion rows stored in Hive, it does so by creating a +root stage to aggregate the output of several other stages, all of +which are designed to implement different sections of a distributed +query plan. + +The hierarchy of stages that comprises a query resembles a tree. +Every query has a root stage, which is responsible for aggregating +the output from other stages. Stages are what the coordinator uses to +model a distributed query plan, but stages themselves don't run on +Trino workers. + +(trino-concept-task)= + +### Task + +As mentioned in the previous section, stages model a particular +section of a distributed query plan, but stages themselves don't +execute on Trino workers. To understand how a stage is executed, +you need to understand that a stage is implemented as a series of +tasks distributed over a network of Trino workers. + +Tasks are the "work horse" in the Trino architecture as a distributed +query plan is deconstructed into a series of stages, which are then +translated to tasks, which then act upon or process splits. A Trino +task has inputs and outputs, and just as a stage can be executed in +parallel by a series of tasks, a task is executing in parallel with a +series of drivers. + +(trino-concept-splits)= + +### Split + +Tasks operate on splits, which are sections of a larger data +set. Stages at the lowest level of a distributed query plan retrieve +data via splits from connectors, and intermediate stages at a higher +level of a distributed query plan retrieve data from other stages. + +When Trino is scheduling a query, the coordinator queries a +connector for a list of all splits that are available for a table. +The coordinator keeps track of which machines are running which tasks, +and what splits are being processed by which tasks. + +### Driver + +Tasks contain one or more parallel drivers. Drivers act upon data and +combine operators to produce output that is then aggregated by a task +and then delivered to another task in another stage. A driver is a +sequence of operator instances, or you can think of a driver as a +physical set of operators in memory. It is the lowest level of +parallelism in the Trino architecture. A driver has one input and +one output. + +### Operator + +An operator consumes, transforms and produces data. For example, a table +scan fetches data from a connector and produces data that can be consumed +by other operators, and a filter operator consumes data and produces a +subset by applying a predicate over the input data. + +### Exchange + +Exchanges transfer data between Trino nodes for different stages of +a query. Tasks produce data into an output buffer and consume data +from other tasks using an exchange client. diff --git a/430/_sources/overview/use-cases.md.txt b/430/_sources/overview/use-cases.md.txt new file mode 100644 index 000000000..9ab8422a5 --- /dev/null +++ b/430/_sources/overview/use-cases.md.txt @@ -0,0 +1,31 @@ +# Use cases + +This section puts Trino into perspective, so that prospective +administrators and end users know what to expect from Trino. + +## What Trino is not + +Since Trino is being called a *database* by many members of the community, +it makes sense to begin with a definition of what Trino is not. + +Do not mistake the fact that Trino understands SQL with it providing +the features of a standard database. Trino is not a general-purpose +relational database. It is not a replacement for databases like MySQL, +PostgreSQL or Oracle. Trino was not designed to handle Online +Transaction Processing (OLTP). This is also true for many other +databases designed and optimized for data warehousing or analytics. + +## What Trino is + +Trino is a tool designed to efficiently query vast amounts of data +using distributed queries. If you work with terabytes or petabytes of +data, you are likely using tools that interact with Hadoop and HDFS. +Trino was designed as an alternative to tools that query HDFS +using pipelines of MapReduce jobs, such as Hive or Pig, but Trino +is not limited to accessing HDFS. Trino can be and has been extended +to operate over different kinds of data sources, including traditional +relational databases and other data sources such as Cassandra. + +Trino was designed to handle data warehousing and analytics: data analysis, +aggregating large amounts of data and producing reports. These workloads +are often classified as Online Analytical Processing (OLAP). diff --git a/430/_sources/release.md.txt b/430/_sources/release.md.txt new file mode 100644 index 000000000..0788d32f0 --- /dev/null +++ b/430/_sources/release.md.txt @@ -0,0 +1,357 @@ +# Release notes + +(releases-2023)= + +## 2023 + +```{toctree} +:maxdepth: 1 + +release/release-430 +release/release-429 +release/release-428 +release/release-427 +release/release-426 +release/release-425 +release/release-424 +release/release-423 +release/release-422 +release/release-421 +release/release-420 +release/release-419 +release/release-418 +release/release-417 +release/release-416 +release/release-415 +release/release-414 +release/release-413 +release/release-412 +release/release-411 +release/release-410 +release/release-409 +release/release-408 +release/release-407 +release/release-406 +``` + +(releases-2022)= + +## 2022 + +```{toctree} +:maxdepth: 1 + +release/release-405 +release/release-404 +release/release-403 +release/release-402 +release/release-401 +release/release-400 +release/release-399 +release/release-398 +release/release-397 +release/release-396 +release/release-395 +release/release-394 +release/release-393 +release/release-392 +release/release-391 +release/release-390 +release/release-389 +release/release-388 +release/release-387 +release/release-386 +release/release-385 +release/release-384 +release/release-383 +release/release-382 +release/release-381 +release/release-380 +release/release-379 +release/release-378 +release/release-377 +release/release-376 +release/release-375 +release/release-374 +release/release-373 +release/release-372 +release/release-371 +release/release-370 +release/release-369 +release/release-368 +``` + +(releases-2021)= + +## 2021 + +```{toctree} +:maxdepth: 1 + +release/release-367 +release/release-366 +release/release-365 +release/release-364 +release/release-363 +release/release-362 +release/release-361 +release/release-360 +release/release-359 +release/release-358 +release/release-357 +release/release-356 +release/release-355 +release/release-354 +release/release-353 +release/release-352 +release/release-351 +``` + +(releases-2020)= + +## 2020 + +```{toctree} +:maxdepth: 1 + +release/release-350 +release/release-349 +release/release-348 +release/release-347 +release/release-346 +release/release-345 +release/release-344 +release/release-343 +release/release-342 +release/release-341 +release/release-340 +release/release-339 +release/release-338 +release/release-337 +release/release-336 +release/release-335 +release/release-334 +release/release-333 +release/release-332 +release/release-331 +release/release-330 +release/release-329 +release/release-328 +``` + +(releases-2019)= + +## 2019 + +```{toctree} +:maxdepth: 1 + +release/release-327 +release/release-326 +release/release-325 +release/release-324 +release/release-323 +release/release-322 +release/release-321 +release/release-320 +release/release-319 +release/release-318 +release/release-317 +release/release-316 +release/release-315 +release/release-314 +release/release-313 +release/release-312 +release/release-311 +release/release-310 +release/release-309 +release/release-308 +release/release-307 +release/release-306 +release/release-305 +release/release-304 +release/release-303 +release/release-302 +release/release-301 +release/release-300 +``` + +## Before 2019 + +```{toctree} +:maxdepth: 1 + +release/release-0.215 +release/release-0.214 +release/release-0.213 +release/release-0.212 +release/release-0.211 +release/release-0.210 +release/release-0.209 +release/release-0.208 +release/release-0.207 +release/release-0.206 +release/release-0.205 +release/release-0.204 +release/release-0.203 +release/release-0.202 +release/release-0.201 +release/release-0.200 +release/release-0.199 +release/release-0.198 +release/release-0.197 +release/release-0.196 +release/release-0.195 +release/release-0.194 +release/release-0.193 +release/release-0.192 +release/release-0.191 +release/release-0.190 +release/release-0.189 +release/release-0.188 +release/release-0.187 +release/release-0.186 +release/release-0.185 +release/release-0.184 +release/release-0.183 +release/release-0.182 +release/release-0.181 +release/release-0.180 +release/release-0.179 +release/release-0.178 +release/release-0.177 +release/release-0.176 +release/release-0.175 +release/release-0.174 +release/release-0.173 +release/release-0.172 +release/release-0.171 +release/release-0.170 +release/release-0.169 +release/release-0.168 +release/release-0.167 +release/release-0.166 +release/release-0.165 +release/release-0.164 +release/release-0.163 +release/release-0.162 +release/release-0.161 +release/release-0.160 +release/release-0.159 +release/release-0.158 +release/release-0.157.1 +release/release-0.157 +release/release-0.156 +release/release-0.155 +release/release-0.154 +release/release-0.153 +release/release-0.152.3 +release/release-0.152.2 +release/release-0.152.1 +release/release-0.152 +release/release-0.151 +release/release-0.150 +release/release-0.149 +release/release-0.148 +release/release-0.147 +release/release-0.146 +release/release-0.145 +release/release-0.144.7 +release/release-0.144.6 +release/release-0.144.5 +release/release-0.144.4 +release/release-0.144.3 +release/release-0.144.2 +release/release-0.144.1 +release/release-0.144 +release/release-0.143 +release/release-0.142 +release/release-0.141 +release/release-0.140 +release/release-0.139 +release/release-0.138 +release/release-0.137 +release/release-0.136 +release/release-0.135 +release/release-0.134 +release/release-0.133 +release/release-0.132 +release/release-0.131 +release/release-0.130 +release/release-0.129 +release/release-0.128 +release/release-0.127 +release/release-0.126 +release/release-0.125 +release/release-0.124 +release/release-0.123 +release/release-0.122 +release/release-0.121 +release/release-0.120 +release/release-0.119 +release/release-0.118 +release/release-0.117 +release/release-0.116 +release/release-0.115 +release/release-0.114 +release/release-0.113 +release/release-0.112 +release/release-0.111 +release/release-0.110 +release/release-0.109 +release/release-0.108 +release/release-0.107 +release/release-0.106 +release/release-0.105 +release/release-0.104 +release/release-0.103 +release/release-0.102 +release/release-0.101 +release/release-0.100 +release/release-0.99 +release/release-0.98 +release/release-0.97 +release/release-0.96 +release/release-0.95 +release/release-0.94 +release/release-0.93 +release/release-0.92 +release/release-0.91 +release/release-0.90 +release/release-0.89 +release/release-0.88 +release/release-0.87 +release/release-0.86 +release/release-0.85 +release/release-0.84 +release/release-0.83 +release/release-0.82 +release/release-0.81 +release/release-0.80 +release/release-0.79 +release/release-0.78 +release/release-0.77 +release/release-0.76 +release/release-0.75 +release/release-0.74 +release/release-0.73 +release/release-0.72 +release/release-0.71 +release/release-0.70 +release/release-0.69 +release/release-0.68 +release/release-0.67 +release/release-0.66 +release/release-0.65 +release/release-0.64 +release/release-0.63 +release/release-0.62 +release/release-0.61 +release/release-0.60 +release/release-0.59 +release/release-0.58 +release/release-0.57 +release/release-0.56 +release/release-0.55 +release/release-0.54 +``` diff --git a/430/_sources/release/release-0.100.md.txt b/430/_sources/release/release-0.100.md.txt new file mode 100644 index 000000000..5b1c47898 --- /dev/null +++ b/430/_sources/release/release-0.100.md.txt @@ -0,0 +1,25 @@ +# Release 0.100 + +## System connector + +The {doc}`/connector/system` now works like other connectors: global system +tables are only available in the `system` catalog, rather than in a special +schema that is available in every catalog. Additionally, connectors may now +provide system tables that are available within that connector's catalog by +implementing the `getSystemTables()` method on the `Connector` interface. + +## General + +- Fix `%f` specifier in {func}`date_format` and {func}`date_parse`. +- Add `WITH ORDINALITY` support to `UNNEST`. +- Add {func}`array_distinct` function. +- Add {func}`split` function. +- Add {func}`degrees` and {func}`radians` functions. +- Add {func}`to_base` and {func}`from_base` functions. +- Rename config property `task.shard.max-threads` to `task.max-worker-threads`. + This property sets the number of threads used to concurrently process splits. + The old property name is deprecated and will be removed in a future release. +- Fix referencing `NULL` values in {ref}`row-type`. +- Make {ref}`map-type` comparable. +- Fix leak of tasks blocked during query teardown. +- Improve query queue config validation. diff --git a/430/_sources/release/release-0.101.md.txt b/430/_sources/release/release-0.101.md.txt new file mode 100644 index 000000000..d9d6c4c42 --- /dev/null +++ b/430/_sources/release/release-0.101.md.txt @@ -0,0 +1,75 @@ +# Release 0.101 + +## General + +- Add support for {doc}`/sql/create-table` (in addition to {doc}`/sql/create-table-as`). +- Add `IF EXISTS` support to {doc}`/sql/drop-table` and {doc}`/sql/drop-view`. +- Add {func}`array_agg` function. +- Add {func}`array_intersect` function. +- Add {func}`array_position` function. +- Add {func}`regexp_split` function. +- Add support for `millisecond` to {func}`date_diff` and {func}`date_add`. +- Fix excessive memory usage in {func}`map_agg`. +- Fix excessive memory usage in queries that perform partitioned top-N operations + with {func}`row_number`. +- Optimize {ref}`array-type` comparison operators. +- Fix analysis of `UNION` queries for tables with hidden columns. +- Fix `JOIN` associativity to be left-associative instead of right-associative. +- Add `source` column to `runtime.queries` table in {doc}`/connector/system`. +- Add `coordinator` column to `runtime.nodes` table in {doc}`/connector/system`. +- Add `errorCode`, `errorName` and `errorType` to `error` object in REST API + (`errorCode` previously existed but was always zero). +- Fix `DatabaseMetaData.getIdentifierQuoteString()` in JDBC driver. +- Handle thread interruption in JDBC driver `ResultSet`. +- Add `history` command and support for running previous commands via `!n` to the CLI. +- Change Driver to make as much progress as possible before blocking. This improves + responsiveness of some limit queries. +- Add predicate push down support to JMX connector. +- Add support for unary `PLUS` operator. +- Improve scheduling speed by reducing lock contention. +- Extend optimizer to understand physical properties such as local grouping and sorting. +- Add support for streaming execution of window functions. +- Make `UNION` run partitioned, if underlying plan is partitioned. +- Add `hash_partition_count` session property to control hash partitions. + +## Web UI + +The main page of the web UI has been completely rewritten to use ReactJS. It also has +a number of new features, such as the ability to pause auto-refresh via the "Z" key and +also with a toggle in the UI. + +## Hive + +- Add support for connecting to S3 using EC2 instance credentials. + This feature is enabled by default. To disable it, set + `hive.s3.use-instance-credentials=false` in your Hive catalog properties file. +- Treat ORC files as splittable. +- Change PrestoS3FileSystem to use lazy seeks, which improves ORC performance. +- Fix ORC `DOUBLE` statistic for columns containing `NaN`. +- Lower the Hive metadata refresh interval from two minutes to one second. +- Invalidate Hive metadata cache for failed operations. +- Support `s3a` file system scheme. +- Fix discovery of splits to correctly backoff when the queue is full. +- Add support for non-canonical Parquet structs. +- Add support for accessing Parquet columns by name. By default, columns in Parquet + files are accessed by their ordinal position in the Hive table definition. To access + columns based on the names recorded in the Parquet file, set + `hive.parquet.use-column-names=true` in your Hive catalog properties file. +- Add JMX stats to PrestoS3FileSystem. +- Add `hive.recursive-directories` config option to recursively scan + partition directories for data. + +## SPI + +- Add connector callback for rollback of `INSERT` and `CREATE TABLE AS`. +- Introduce an abstraction for representing physical organizations of a table + and describing properties such as partitioning, grouping, predicate and columns. + `ConnectorPartition` and related interfaces are deprecated and will be removed + in a future version. +- Rename `ConnectorColumnHandle` to `ColumnHandle`. + +:::{note} +This is a backwards incompatible change with the previous connector SPI. +If you have written a connector, you will need to update your code +before deploying this release. +::: diff --git a/430/_sources/release/release-0.102.md.txt b/430/_sources/release/release-0.102.md.txt new file mode 100644 index 000000000..edc4c3671 --- /dev/null +++ b/430/_sources/release/release-0.102.md.txt @@ -0,0 +1,47 @@ +# Release 0.102 + +## Unicode support + +All string functions have been updated to support Unicode. The functions assume +that the string contains valid UTF-8 encoded code points. There are no explicit +checks for valid UTF-8, and the functions may return incorrect results on +invalid UTF-8. Invalid UTF-8 data can be corrected with {func}`from_utf8`. + +Additionally, the functions operate on Unicode code points and not user visible +*characters* (or *grapheme clusters*). Some languages combine multiple code points +into a single user-perceived *character*, the basic unit of a writing system for a +language, but the functions will treat each code point as a separate unit. + +## Regular expression functions + +All {doc}`/functions/regexp` have been rewritten to improve performance. +The new versions are often twice as fast and in some cases can be many +orders of magnitude faster (due to removal of quadratic behavior). +This change introduced some minor incompatibilities that are explained +in the documentation for the functions. + +## General + +- Add support for partitioned right outer joins, which allows for larger tables to + be joined on the inner side. +- Add support for full outer joins. +- Support returning booleans as numbers in JDBC driver +- Fix {func}`contains` to return `NULL` if the value was not found, but a `NULL` was. +- Fix nested {ref}`row-type` rendering in `DESCRIBE`. +- Add {func}`array_join`. +- Optimize map subscript operator. +- Add {func}`from_utf8` and {func}`to_utf8` functions. +- Add `task_writer_count` session property to set `task.writer-count`. +- Add cast from `ARRAY(F)` to `ARRAY(T)`. +- Extend implicit coercions to `ARRAY` element types. +- Implement implicit coercions in `VALUES` expressions. +- Fix potential deadlock in scheduler. + +## Hive + +- Collect more metrics from `PrestoS3FileSystem`. +- Retry when seeking in `PrestoS3FileSystem`. +- Ignore `InvalidRange` error in `PrestoS3FileSystem`. +- Implement rename and delete in `PrestoS3FileSystem`. +- Fix assertion failure when running `SHOW TABLES FROM schema`. +- Fix S3 socket leak when reading ORC files. diff --git a/430/_sources/release/release-0.103.md.txt b/430/_sources/release/release-0.103.md.txt new file mode 100644 index 000000000..2fd5589d2 --- /dev/null +++ b/430/_sources/release/release-0.103.md.txt @@ -0,0 +1,50 @@ +# Release 0.103 + +## Cluster resource management + +There is a new cluster resource manager, which can be enabled via the +`experimental.cluster-memory-manager-enabled` flag. Currently, the only +resource that's tracked is memory, and the cluster resource manager guarantees +that the cluster will not deadlock waiting for memory. However, in a low memory +situation it is possible that only one query will make progress. Memory limits can +now be configured via `query.max-memory` which controls the total distributed +memory a query may use and `query.max-memory-per-node` which limits the amount +of memory a query may use on any one node. On each worker, the +`resources.reserved-system-memory` flags controls how much memory is reserved +for internal Presto data structures and temporary allocations. + +## Task parallelism + +Queries involving a large number of aggregations or a large hash table for a +join can be slow due to single threaded execution in the intermediate stages. +This release adds experimental configuration and session properties to execute +this single threaded work in parallel. Depending on the exact query this may +reduce wall time, but will likely increase CPU usage. + +Use the configuration parameter `task.default-concurrency` or the session +property `task_default_concurrency` to set the default number of parallel +workers to use for join probes, hash builds and final aggregations. +Additionally, the session properties `task_join_concurrency`, +`task_hash_build_concurrency` and `task_aggregation_concurrency` can be +used to control the parallelism for each type of work. + +This is an experimental feature and will likely change in a future release. It +is also expected that this will eventually be handled automatically by the +query planner and these options will be removed entirely. + +## Hive + +- Removed the `hive.max-split-iterator-threads` parameter and renamed + `hive.max-global-split-iterator-threads` to `hive.max-split-iterator-threads`. +- Fix excessive object creation when querying tables with a large number of partitions. +- Do not retry requests when an S3 path is not found. + +## General + +- Add {func}`array_remove`. +- Fix NPE in {func}`max_by` and {func}`min_by` caused when few rows were present in the aggregation. +- Reduce memory usage of {func}`map_agg`. +- Change HTTP client defaults: 2 second idle timeout, 10 second request + timeout and 250 connections per host. +- Add SQL command autocompletion to CLI. +- Increase CLI history file size. diff --git a/430/_sources/release/release-0.104.md.txt b/430/_sources/release/release-0.104.md.txt new file mode 100644 index 000000000..043512775 --- /dev/null +++ b/430/_sources/release/release-0.104.md.txt @@ -0,0 +1,26 @@ +# Release 0.104 + +## General + +- Handle thread interruption in StatementClient. +- Fix CLI hang when server becomes unreachable during a query. +- Add {func}`covar_pop`, {func}`covar_samp`, {func}`corr`, {func}`regr_slope`, + and {func}`regr_intercept` functions. +- Fix potential deadlock in cluster memory manager. +- Add a visualization of query execution timeline. +- Allow mixed case in input to {func}`from_hex`. +- Display "BLOCKED" state in web UI. +- Reduce CPU usage in coordinator. +- Fix excess object retention in workers due to long running queries. +- Reduce memory usage of {func}`array_distinct`. +- Add optimizer for projection push down which can + improve the performance of certain query shapes. +- Improve query performance by storing pre-partitioned pages. +- Support `TIMESTAMP` for {func}`first_value`, {func}`last_value`, + {func}`nth_value`, {func}`lead` and {func}`lag`. + +## Hive + +- Upgrade to Parquet 1.6.0. +- Collect request time and retry statistics in `PrestoS3FileSystem`. +- Fix retry attempt counting for S3. diff --git a/430/_sources/release/release-0.105.md.txt b/430/_sources/release/release-0.105.md.txt new file mode 100644 index 000000000..732ed96e6 --- /dev/null +++ b/430/_sources/release/release-0.105.md.txt @@ -0,0 +1,18 @@ +# Release 0.105 + +## General + +- Fix issue which can cause queries to be blocked permanently. +- Close connections correctly in JDBC connectors. +- Add implicit coercions for values of equi-join criteria. +- Fix detection of window function calls without an `OVER` clause. + +## SPI + +- Remove `ordinalPosition` from `ColumnMetadata`. + +:::{note} +This is a backwards incompatible change with the previous connector SPI. +If you have written a connector, you will need to update your code +before deploying this release. +::: diff --git a/430/_sources/release/release-0.106.md.txt b/430/_sources/release/release-0.106.md.txt new file mode 100644 index 000000000..212bc0b9c --- /dev/null +++ b/430/_sources/release/release-0.106.md.txt @@ -0,0 +1,12 @@ +# Release 0.106 + +## General + +- Parallelize startup of table scan task splits. +- Fixed index join driver resource leak. +- Improve memory accounting for JOINs and GROUP BYs. +- Improve CPU efficiency of coordinator. +- Added `Asia/Chita`, `Asia/Srednekolymsk`, and `Pacific/Bougainville` time zones. +- Fix task leak caused by race condition in stage state machine. +- Fix blocking in Hive split source. +- Free resources sooner for queries that finish prematurely. diff --git a/430/_sources/release/release-0.107.md.txt b/430/_sources/release/release-0.107.md.txt new file mode 100644 index 000000000..a10dfaeae --- /dev/null +++ b/430/_sources/release/release-0.107.md.txt @@ -0,0 +1,13 @@ +# Release 0.107 + +## General + +- Added `query_max_memory` session property. Note: this session property cannot + increase the limit above the limit set by the `query.max-memory` configuration option. +- Fixed task leak caused by queries that finish early, such as a `LIMIT` query + or cancelled query, when the cluster is under high load. +- Added `task.info-refresh-max-wait` to configure task info freshness. +- Add support for `DELETE` to language and connector SPI. +- Reenable error classification code for syntax errors. +- Fix out of bounds exception in {func}`lower` and {func}`upper` + when the string contains the code point `U+10FFFF`. diff --git a/430/_sources/release/release-0.108.md.txt b/430/_sources/release/release-0.108.md.txt new file mode 100644 index 000000000..fecf2a03e --- /dev/null +++ b/430/_sources/release/release-0.108.md.txt @@ -0,0 +1,34 @@ +# Release 0.108 + +## General + +- Fix incorrect query results when a window function follows a {func}`row_number` + function and both are partitioned on the same column(s). +- Fix planning issue where queries that apply a `false` predicate + to the result of a non-grouped aggregation produce incorrect results. +- Fix exception when `ORDER BY` clause contains duplicate columns. +- Fix issue where a query (read or write) that should fail can instead + complete successfully with zero rows. +- Add {func}`normalize`, {func}`from_iso8601_timestamp`, {func}`from_iso8601_date` + and {func}`to_iso8601` functions. +- Add support for {func}`position` syntax. +- Add Teradata compatibility functions: {func}`index`, {func}`char2hexint`, + {func}`to_char`, {func}`to_date` and {func}`to_timestamp`. +- Make `ctrl-C` in CLI cancel the query (rather than a partial cancel). +- Allow calling `Connection.setReadOnly(false)` in the JDBC driver. + The read-only status for the connection is currently ignored. +- Add missing `CAST` from `VARCHAR` to `TIMESTAMP WITH TIME ZONE`. +- Allow optional time zone in `CAST` from `VARCHAR` to `TIMESTAMP` and + `TIMESTAMP WITH TIME ZONE`. +- Trim values when converting from `VARCHAR` to date/time types. +- Add support for fixed time zones `+00:00` and `-00:00`. +- Properly account for query memory when using the {func}`row_number` function. +- Skip execution of inner join when the join target is empty. +- Improve query detail UI page. +- Fix printing of table layouts in {doc}`/sql/explain`. +- Add {doc}`/connector/blackhole`. + +## Cassandra + +- Randomly select Cassandra node for split generation. +- Fix handling of `UUID` partition keys. diff --git a/430/_sources/release/release-0.109.md.txt b/430/_sources/release/release-0.109.md.txt new file mode 100644 index 000000000..80cb6abd6 --- /dev/null +++ b/430/_sources/release/release-0.109.md.txt @@ -0,0 +1,23 @@ +# Release 0.109 + +## General + +- Add {func}`slice`, {func}`md5`, {func}`array_min` and {func}`array_max` functions. +- Fix bug that could cause queries submitted soon after startup to hang forever. +- Fix bug that could cause `JOIN` queries to hang forever, if the right side of + the `JOIN` had too little data or skewed data. +- Improve index join planning heuristics to favor streaming execution. +- Improve validation of date/time literals. +- Produce RPM package for Presto server. +- Always redistribute data when writing tables to avoid skew. This can + be disabled by setting the session property `redistribute_writes` + or the config property `redistribute-writes` to false. + +## Remove "Big Query" support + +The experimental support for big queries has been removed in favor of +the new resource manager which can be enabled via the +`experimental.cluster-memory-manager-enabled` config option. +The `experimental_big_query` session property and the following config +options are no longer supported: `experimental.big-query-initial-hash-partitions`, +`experimental.max-concurrent-big-queries` and `experimental.max-queued-big-queries`. diff --git a/430/_sources/release/release-0.110.md.txt b/430/_sources/release/release-0.110.md.txt new file mode 100644 index 000000000..0d6fecabb --- /dev/null +++ b/430/_sources/release/release-0.110.md.txt @@ -0,0 +1,14 @@ +# Release 0.110 + +## General + +- Fix result truncation bug in window function {func}`row_number` when performing a + partitioned top-N that chooses the maximum or minimum `N` rows. For example: + + ``` + SELECT * FROM ( + SELECT row_number() OVER (PARTITION BY orderstatus ORDER BY orderdate) AS rn, + custkey, orderdate, orderstatus + FROM orders + ) WHERE rn <= 5; + ``` diff --git a/430/_sources/release/release-0.111.md.txt b/430/_sources/release/release-0.111.md.txt new file mode 100644 index 000000000..b0efb38bd --- /dev/null +++ b/430/_sources/release/release-0.111.md.txt @@ -0,0 +1,11 @@ +# Release 0.111 + +## General + +- Add {func}`histogram` function. +- Optimize `CASE` expressions on a constant. +- Add basic support for `IF NOT EXISTS` for `CREATE TABLE`. +- Semi-joins are hash-partitioned if `distributed_join` is turned on. +- Add support for partial cast from JSON. For example, `json` can be cast to `array(json)`, `map(varchar, json)`, etc. +- Add implicit coercions for `UNION`. +- Expose query stats in the JDBC driver `ResultSet`. diff --git a/430/_sources/release/release-0.112.md.txt b/430/_sources/release/release-0.112.md.txt new file mode 100644 index 000000000..bd3f5ce00 --- /dev/null +++ b/430/_sources/release/release-0.112.md.txt @@ -0,0 +1,17 @@ +# Release 0.112 + +## General + +- Fix incorrect handling of filters and limits in {func}`row_number` optimizer. + This caused certain query shapes to produce incorrect results. +- Fix non-string object arrays in JMX connector. + +## Hive + +- Tables created using {doc}`/sql/create-table` (not {doc}`/sql/create-table-as`) + had invalid metadata and were not readable. +- Improve performance of `IN` and `OR` clauses when reading `ORC` data. + Previously, the ranges for a column were always compacted into a single range + before being passed to the reader, preventing the reader from taking full + advantage of row skipping. The compaction only happens now if the number of + ranges exceeds the `hive.domain-compaction-threshold` config property. diff --git a/430/_sources/release/release-0.113.md.txt b/430/_sources/release/release-0.113.md.txt new file mode 100644 index 000000000..8905cccb5 --- /dev/null +++ b/430/_sources/release/release-0.113.md.txt @@ -0,0 +1,64 @@ +# Release 0.113 + +:::{warning} +The ORC reader in the Hive connector is broken in this release. +::: + +## Cluster resource management + +The cluster resource manager announced in {doc}`/release/release-0.103` is now enabled by default. +You can disable it with the `experimental.cluster-memory-manager-enabled` flag. +Memory limits can now be configured via `query.max-memory` which controls the total distributed +memory a query may use and `query.max-memory-per-node` which limits the amount +of memory a query may use on any one node. On each worker, the +`resources.reserved-system-memory` config property controls how much memory is reserved +for internal Presto data structures and temporary allocations. + +## Session properties + +All session properties have a type, default value, and description. +The value for {doc}`/sql/set-session` can now be any constant expression, and +the {doc}`/sql/show-session` command prints the current effective value and +default value for all session properties. + +This type safety extends to the {doc}`SPI ` where properties +can be validated and converted to any Java type using +`SessionPropertyMetadata`. For an example, see `HiveSessionProperties`. + +:::{note} +This is a backwards incompatible change with the previous connector SPI. +If you have written a connector that uses session properties, you will need +to update your code to declare the properties in the `Connector` +implementation and callers of `ConnectorSession.getProperty()` will now +need the expected Java type of the property. +::: + +## General + +- Allow using any type with value window functions {func}`first_value`, + {func}`last_value`, {func}`nth_value`, {func}`lead` and {func}`lag`. +- Add {func}`element_at` function. +- Add {func}`url_encode` and {func}`url_decode` functions. +- {func}`concat` now allows arbitrary number of arguments. +- Fix JMX connector. In the previous release it always returned zero rows. +- Fix handling of literal `NULL` in `IS DISTINCT FROM`. +- Fix an issue that caused some specific queries to fail in planning. + +## Hive + +- Fix the Hive metadata cache to properly handle negative responses. + This makes the background refresh work properly by clearing the cached + metadata entries when an object is dropped outside of Presto. + In particular, this fixes the common case where a table is dropped using + Hive but Presto thinks it still exists. +- Fix metastore socket leak when SOCKS connect fails. + +## SPI + +- Changed the internal representation of structural types. + +:::{note} +This is a backwards incompatible change with the previous connector SPI. +If you have written a connector that uses structural types, you will need +to update your code to the new APIs. +::: diff --git a/430/_sources/release/release-0.114.md.txt b/430/_sources/release/release-0.114.md.txt new file mode 100644 index 000000000..cd90ff930 --- /dev/null +++ b/430/_sources/release/release-0.114.md.txt @@ -0,0 +1,10 @@ +# Release 0.114 + +## General + +- Fix `%k` specifier for {func}`date_format` and {func}`date_parse`. + It previously used `24` rather than `0` for the midnight hour. + +## Hive + +- Fix ORC reader for Hive connector. diff --git a/430/_sources/release/release-0.115.md.txt b/430/_sources/release/release-0.115.md.txt new file mode 100644 index 000000000..4328e2e6e --- /dev/null +++ b/430/_sources/release/release-0.115.md.txt @@ -0,0 +1,14 @@ +# Release 0.115 + +## General + +- Fix an issue with hierarchical queue rules where queries could be rejected after being accepted. +- Add {func}`sha1`, {func}`sha256` and {func}`sha512` functions. +- Add {func}`power` as an alias for {func}`pow`. +- Add support for `LIMIT ALL` syntax. + +## Hive + +- Fix a race condition which could cause queries to finish without reading all the data. +- Fix a bug in Parquet reader that causes failures while reading lists that has an element + schema name other than `array_element` in its Parquet-level schema. diff --git a/430/_sources/release/release-0.116.md.txt b/430/_sources/release/release-0.116.md.txt new file mode 100644 index 000000000..b2dde115f --- /dev/null +++ b/430/_sources/release/release-0.116.md.txt @@ -0,0 +1,40 @@ +# Release 0.116 + +## Cast between JSON and VARCHAR + +Casts of both directions between JSON and VARCHAR have been removed. If you +have such casts in your scripts or views, they will fail with a message when +you move to release 0.116. To get the semantics of the current casts, use: + +- `JSON_PARSE(x)` instead of `CAST(x as JSON)` +- `JSON_FORMAT(x)` instead of `CAST(x as VARCHAR)` + +In a future release, we intend to reintroduce casts between JSON and VARCHAR +along with other casts involving JSON. The semantics of the new JSON and +VARCHAR cast will be consistent with the other casts being introduced. But it +will be different from the semantics in 0.115 and before. When that comes, +cast between JSON and VARCHAR in old scripts and views will produce unexpected +result. + +## Cluster memory manager improvements + +The cluster memory manager now has a low memory killer. If the cluster runs low +on memory, the killer will kill queries to improve throughput. It can be enabled +with the `query.low-memory-killer.enabled` config flag, and the delay between +when the cluster runs low on memory and when the killer will be invoked can be +configured with the `query.low-memory-killer.delay` option. + +## General + +- Add {func}`multimap_agg` function. +- Add {func}`checksum` function. +- Add {func}`max` and {func}`min` that takes a second argument and produces + `n` largest or `n` smallest values. +- Add `query_max_run_time` session property and `query.max-run-time` + config. Queries are failed after the specified duration. +- Removed `experimental.cluster-memory-manager-enabled` config. The cluster + memory manager is now always enabled. +- Removed `task.max-memory` config. +- `optimizer.optimize-hash-generation` and `distributed-joins-enabled` are + both enabled by default now. +- Add optimization for `IF` on a constant condition. diff --git a/430/_sources/release/release-0.117.md.txt b/430/_sources/release/release-0.117.md.txt new file mode 100644 index 000000000..84bed955c --- /dev/null +++ b/430/_sources/release/release-0.117.md.txt @@ -0,0 +1,9 @@ +# Release 0.117 + +## General + +- Add back casts between JSON and VARCHAR to provide an easier migration path + to {func}`json_parse` and {func}`json_format`. These will be removed in a + future release. +- Fix bug in semi joins and group bys on a single `BIGINT` column where + 0 could match `NULL`. diff --git a/430/_sources/release/release-0.118.md.txt b/430/_sources/release/release-0.118.md.txt new file mode 100644 index 000000000..ec02e6db6 --- /dev/null +++ b/430/_sources/release/release-0.118.md.txt @@ -0,0 +1,20 @@ +# Release 0.118 + +## General + +- Fix planning error for `UNION` queries that require implicit coercions. +- Fix null pointer exception when using {func}`checksum`. +- Fix completion condition for `SqlTask` that can cause queries to be blocked. + +## Authorization + +We've added experimental support for authorization of SQL queries in Presto. +This is currently only supported by the Hive connector. You can enable Hive +checks by setting the `hive.security` property to `none`, `read-only`, +or `sql-standard`. + +:::{note} +The authentication support is experimental and only lightly tested. We are +actively working on this feature, so expect backwards incompatible changes. +See the `ConnectorAccessControl` interface the SPI for details. +::: diff --git a/430/_sources/release/release-0.119.md.txt b/430/_sources/release/release-0.119.md.txt new file mode 100644 index 000000000..391c17278 --- /dev/null +++ b/430/_sources/release/release-0.119.md.txt @@ -0,0 +1,65 @@ +# Release 0.119 + +## General + +- Add {doc}`/connector/redis`. +- Add {func}`geometric_mean` function. +- Fix restoring interrupt status in `StatementClient`. +- Support getting server version in JDBC driver. +- Improve correctness and compliance of JDBC `DatabaseMetaData`. +- Catalog and schema are now optional on the server. This allows connecting + and executing metadata commands or queries that use fully qualified names. + Previously, the CLI and JDBC driver would use a catalog and schema named + `default` if they were not specified. +- Fix scheduler handling of partially canceled queries. +- Execute views with the permissions of the view owner. +- Replaced the `task.http-notification-threads` config option with two + independent options: `task.http-response-threads` and `task.http-timeout-threads`. +- Improve handling of negated expressions in join criteria. +- Fix {func}`arbitrary`, {func}`max_by` and {func}`min_by` functions when used + with an array, map or row type. +- Fix union coercion when the same constant or column appears more than once on + the same side. +- Support `RENAME COLUMN` in {doc}`/sql/alter-table`. + +## SPI + +- Add more system table distribution modes. +- Add owner to view metadata. + +:::{note} +These are backwards incompatible changes with the previous connector SPI. +If you have written a connector, you may need to update your code to the +new APIs. +::: + +## CLI + +- Fix handling of full width characters. +- Skip printing query URL if terminal is too narrow. +- Allow performing a partial query cancel using `ctrl-P`. +- Allow toggling debug mode during query by pressing `D`. +- Fix handling of query abortion after result has been partially received. +- Fix handling of `ctrl-C` when displaying results without a pager. + +## Verifier + +- Add `expected-double-precision` config to specify the expected level of + precision when comparing double values. +- Return non-zero exit code when there are failures. + +## Cassandra + +- Add support for Cassandra blob types. + +## Hive + +- Support adding and renaming columns using {doc}`/sql/alter-table`. +- Automatically configure the S3 region when running in EC2. +- Allow configuring multiple Hive metastores for high availability. +- Add support for `TIMESTAMP` and `VARBINARY` in Parquet. + +## MySQL and PostgreSQL + +- Enable streaming results instead of buffering everything in memory. +- Fix handling of pattern characters when matching table or column names. diff --git a/430/_sources/release/release-0.120.md.txt b/430/_sources/release/release-0.120.md.txt new file mode 100644 index 000000000..0abeb2ddf --- /dev/null +++ b/430/_sources/release/release-0.120.md.txt @@ -0,0 +1,5 @@ +# Release 0.120 + +:::{warning} +This release is broken and should not be used. +::: diff --git a/430/_sources/release/release-0.121.md.txt b/430/_sources/release/release-0.121.md.txt new file mode 100644 index 000000000..fadd5f1bf --- /dev/null +++ b/430/_sources/release/release-0.121.md.txt @@ -0,0 +1,8 @@ +# Release 0.121 + +## General + +- Fix regression that causes task scheduler to not retry requests in some cases. +- Throttle task info refresher on errors. +- Fix planning failure that prevented the use of large `IN` lists. +- Fix comparison of `array(T)` where `T` is a comparable, non-orderable type. diff --git a/430/_sources/release/release-0.122.md.txt b/430/_sources/release/release-0.122.md.txt new file mode 100644 index 000000000..9fa45215b --- /dev/null +++ b/430/_sources/release/release-0.122.md.txt @@ -0,0 +1,19 @@ +# Release 0.122 + +:::{warning} +There is a bug in this release that will cause queries to fail when the +`optimizer.optimize-hash-generation` config is disabled. +::: + +## General + +- The deprecated casts between JSON and VARCHAR will now fail and provide the + user with instructions to migrate their query. For more details, see + {doc}`/release/release-0.116`. +- Fix `NoSuchElementException` when cross join is used inside `IN` query. +- Fix `GROUP BY` to support maps of structural types. +- The web interface now displays a lock icon next to authenticated users. +- The {func}`min_by` and {func}`max_by` aggregations now have an additional form + that return multiple values. +- Fix incorrect results when using `IN` lists of more than 1000 elements of + `timestamp with time zone`, `time with time zone` or structural types. diff --git a/430/_sources/release/release-0.123.md.txt b/430/_sources/release/release-0.123.md.txt new file mode 100644 index 000000000..6dd65644f --- /dev/null +++ b/430/_sources/release/release-0.123.md.txt @@ -0,0 +1,55 @@ +# Release 0.123 + +## General + +- Remove `node-scheduler.location-aware-scheduling-enabled` config. +- Fixed query failures that occur when the `optimizer.optimize-hash-generation` + config is disabled. +- Fix exception when using the `ResultSet` returned from the + `DatabaseMetaData.getColumns` method in the JDBC driver. +- Increase default value of `failure-detector.threshold` config. +- Fix race in queueing system which could cause queries to fail with + "Entering secondary queue failed". +- Fix issue with {func}`histogram` that can cause failures or incorrect results + when there are more than ten buckets. +- Optimize execution of cross join. +- Run Presto server as `presto` user in RPM init scripts. + +## Table properties + +When creating tables with {doc}`/sql/create-table` or {doc}`/sql/create-table-as`, +you can now add connector specific properties to the new table. For example, when +creating a Hive table you can specify the file format. To list all available table, +properties, run the following query: + +``` +SELECT * FROM system.metadata.table_properties +``` + +## Hive + +We have implemented `INSERT` and `DELETE` for Hive. Both `INSERT` and `CREATE` +statements support partitioned tables. For example, to create a partitioned table +execute the following: + +``` +CREATE TABLE orders ( + order_date VARCHAR, + order_region VARCHAR, + order_id BIGINT, + order_info VARCHAR +) WITH (partitioned_by = ARRAY['order_date', 'order_region']) +``` + +To `DELETE` from a Hive table, you must specify a `WHERE` clause that matches +entire partitions. For example, to delete from the above table, execute the following: + +``` +DELETE FROM orders +WHERE order_date = '2015-10-15' AND order_region = 'APAC' +``` + +:::{note} +Currently, Hive deletion is only supported for partitioned tables. +Additionally, partition keys must be of type VARCHAR. +::: diff --git a/430/_sources/release/release-0.124.md.txt b/430/_sources/release/release-0.124.md.txt new file mode 100644 index 000000000..152cb6300 --- /dev/null +++ b/430/_sources/release/release-0.124.md.txt @@ -0,0 +1,45 @@ +# Release 0.124 + +## General + +- Fix race in memory tracking of `JOIN` which could cause the cluster to become over + committed and possibly crash. +- The {func}`approx_percentile` aggregation now also accepts an array of percentages. +- Allow nested row type references. +- Fix correctness for some queries with `IN` lists. When all constants in the + list are in the range of 32-bit signed integers but the test value can be + outside of the range, `true` may be produced when the correct result should + be `false`. +- Fail queries submitted while coordinator is starting. +- Add JMX stats to track authentication and authorization successes and failures. +- Add configuration support for the system access control plugin. The system access + controller can be selected and configured using `etc/access-control.properties`. + Note that Presto currently does not ship with any system access controller + implementations. +- Add support for `WITH NO DATA` syntax in `CREATE TABLE ... AS SELECT`. +- Fix issue where invalid plans are generated for queries with multiple aggregations + that require input values to be cast in different ways. +- Fix performance issue due to redundant processing in queries involving `DISTINCT` + and `LIMIT`. +- Add optimization that can reduce the amount of data sent over the network + for grouped aggregation queries. This feature can be enabled by + `optimizer.use-intermediate-aggregations` config property or + `task_intermediate_aggregation` session property. + +## Hive + +- Do not count expected exceptions as errors in the Hive metastore client stats. +- Improve performance when reading ORC files with many tiny stripes. + +## Verifier + +- Add support for pre and post control and test queries. + +If you are upgrading, you need to alter your `verifier_queries` table: + +``` +ALTER TABLE verifier_queries ADD COLUMN test_postqueries text; +ALTER TABLE verifier_queries ADD COLUMN test_prequeries text; +ALTER TABLE verifier_queries ADD COLUMN control_postqueries text; +ALTER TABLE verifier_queries ADD COLUMN control_prequeries text; +``` diff --git a/430/_sources/release/release-0.125.md.txt b/430/_sources/release/release-0.125.md.txt new file mode 100644 index 000000000..3134c9d97 --- /dev/null +++ b/430/_sources/release/release-0.125.md.txt @@ -0,0 +1,8 @@ +# Release 0.125 + +## General + +- Fix an issue where certain operations such as `GROUP BY`, `DISTINCT`, etc. on the + output of a `RIGHT` or `FULL OUTER JOIN` can return incorrect results if they reference columns + from the left relation that are also used in the join clause, and not every row from the right relation + has a match. diff --git a/430/_sources/release/release-0.126.md.txt b/430/_sources/release/release-0.126.md.txt new file mode 100644 index 000000000..15d3bd2fb --- /dev/null +++ b/430/_sources/release/release-0.126.md.txt @@ -0,0 +1,47 @@ +# Release 0.126 + +## General + +- Add error location information (line and column number) for semantic errors. +- Fix a CLI crash during tab-completion when no schema is currently selected. +- Fix reset of session properties in CLI when running {doc}`/sql/use`. +- Fix occasional query planning failure due to a bug in the projection + push down optimizer. +- Fix a parsing issue when expressions contain the form `POSITION(x in (y))`. +- Add a new version of {func}`approx_percentile` that takes an `accuracy` + parameter. +- Allow specifying columns names in {doc}`/sql/insert` queries. +- Add `field_length` table property to blackhole connector to control the + size of generated `VARCHAR` and `VARBINARY` fields. +- Bundle Teradata functions plugin in server package. +- Improve handling of physical properties which can increase performance for + queries involving window functions. +- Add ability to control whether index join lookups and caching are shared + within a task. This allows us to optimize for index cache hits or for more + CPU parallelism. This option is toggled by the `task.share-index-loading` + config property or the `task_share_index_loading` session property. +- Add Tableau web connector. +- Improve performance of queries that use an `IN` expression with a large + list of constant values. +- Enable connector predicate push down for all comparable and equatable types. +- Fix query planning failure when using certain operations such as `GROUP BY`, + `DISTINCT`, etc. on the output columns of `UNNEST`. +- In `ExchangeClient` set `maxResponseSize` to be slightly smaller than + the configured value. This reduces the possibility of encountering + `PageTooLargeException`. +- Fix memory leak in coordinator. +- Add validation for names of table properties. + +## Hive + +- Fix reading structural types containing nulls in Parquet. +- Fix writing DATE type when timezone offset is negative. Previous versions + would write the wrong date (off by one day). +- Fix an issue where `VARCHAR` columns added to an existing table could not be + queried. +- Fix over-creation of initial splits. +- Fix `hive.immutable-partitions` config property to also apply to + unpartitioned tables. +- Allow non-`VARCHAR` columns in `DELETE` query. +- Support `DATE` columns as partition columns in parquet tables. +- Improve error message for cases where partition columns are also table columns. diff --git a/430/_sources/release/release-0.127.md.txt b/430/_sources/release/release-0.127.md.txt new file mode 100644 index 000000000..2d0996fec --- /dev/null +++ b/430/_sources/release/release-0.127.md.txt @@ -0,0 +1,6 @@ +# Release 0.127 + +## General + +- Disable index join repartitioning when it disrupts streaming execution. +- Fix memory accounting leak in some `JOIN` queries. diff --git a/430/_sources/release/release-0.128.md.txt b/430/_sources/release/release-0.128.md.txt new file mode 100644 index 000000000..ec2b6d027 --- /dev/null +++ b/430/_sources/release/release-0.128.md.txt @@ -0,0 +1,24 @@ +# Release 0.128 + +## Graceful shutdown + +Workers can now be instructed to shutdown. This is done by submiting a `PUT` +request to `/v1/info/state` with the body `"SHUTTING_DOWN"`. Once instructed +to shutdown, the worker will no longer receive new tasks, and will exit once +all existing tasks have completed. + +## General + +- Fix cast from json to structural types when rows or maps have arrays, + rows, or maps nested in them. +- Fix Example HTTP connector. + It would previously fail with a JSON deserialization error. +- Optimize memory usage in TupleDomain. +- Fix an issue that can occur when an `INNER JOIN` has equi-join clauses that + align with the grouping columns used by a preceding operation such as + `GROUP BY`, `DISTINCT`, etc. When this triggers, the join may fail to + produce some of the output rows. + +## MySQL + +- Fix handling of MySQL database names with underscores. diff --git a/430/_sources/release/release-0.129.md.txt b/430/_sources/release/release-0.129.md.txt new file mode 100644 index 000000000..521c2ba25 --- /dev/null +++ b/430/_sources/release/release-0.129.md.txt @@ -0,0 +1,57 @@ +# Release 0.129 + +:::{warning} +There is a performance regression in this release for `GROUP BY` and `JOIN` +queries when the length of the keys is between 16 and 31 bytes. This is fixed +in {doc}`/release/release-0.130`. +::: + +## General + +- Fix a planner issue that could cause queries involving `OUTER JOIN` to + return incorrect results. +- Some queries, particularly those using {func}`max_by` or {func}`min_by`, now + accurately reflect their true memory usage and thus appear to use more memory + than before. +- Fix {doc}`/sql/show-session` to not show hidden session properties. +- Fix hang in large queries with `ORDER BY` and `LIMIT`. +- Fix an issue when casting empty arrays or arrays containing only `NULL` to + other types. +- Table property names are now properly treated as case-insensitive. +- Minor UI improvements for query detail page. +- Do not display useless stack traces for expected exceptions in verifier. +- Improve performance of queries involving `UNION ALL` that write data. +- Introduce the `P4HyperLogLog` type, which uses an implementation of the HyperLogLog data + structure that trades off accuracy and memory requirements when handling small sets for an + improvement in performance. + +## JDBC driver + +- Throw exception when using {doc}`/sql/set-session` or {doc}`/sql/reset-session` + rather than silently ignoring the command. +- The driver now properly supports non-query statements. + The `Statement` interface supports all variants of the `execute` methods. + It also supports the `getUpdateCount` and `getLargeUpdateCount` methods. + +## CLI + +- Always clear screen when canceling query with `ctrl-C`. +- Make client request timeout configurable. + +## Network topology aware scheduling + +The scheduler can now be configured to take network topology into account when +scheduling splits. This is set using the `node-scheduler.network-topology` +config. See {doc}`/admin/tuning` for more information. + +## Hive + +- The S3 region is no longer automatically configured when running in EC2. + To enable this feature, use `hive.s3.pin-client-to-current-region=true` + in your Hive catalog properties file. Enabling this feature is required + to access S3 data in the China isolated region, but prevents accessing + data outside the current region. +- Server-side encryption is now supported for S3. To enable this feature, + use `hive.s3.sse.enabled=true` in your Hive catalog properties file. +- Add support for the `retention_days` table property. +- Add support for S3 `EncryptionMaterialsProvider`. diff --git a/430/_sources/release/release-0.130.md.txt b/430/_sources/release/release-0.130.md.txt new file mode 100644 index 000000000..7cae49c61 --- /dev/null +++ b/430/_sources/release/release-0.130.md.txt @@ -0,0 +1,14 @@ +# Release 0.130 + +## General + +- Fix a performance regression in `GROUP BY` and `JOIN` queries when the + length of the keys is between 16 and 31 bytes. +- Add {func}`map_concat` function. +- Performance improvements for filters, projections and dictionary encoded data. + This optimization is turned off by default. It can be configured via the + `optimizer.columnar-processing-dictionary` config property or the + `columnar_processing_dictionary` session property. +- Improve performance of aggregation queries with large numbers of groups. +- Improve performance for queries that use {ref}`array-type` type. +- Fix querying remote views in MySQL and PostgreSQL connectors. diff --git a/430/_sources/release/release-0.131.md.txt b/430/_sources/release/release-0.131.md.txt new file mode 100644 index 000000000..c697084bd --- /dev/null +++ b/430/_sources/release/release-0.131.md.txt @@ -0,0 +1,6 @@ +# Release 0.131 + +## General + +- Fix poor performance of transporting dictionary encoded data over the network. +- Fix code generator to prevent "Method code too large" error. diff --git a/430/_sources/release/release-0.132.md.txt b/430/_sources/release/release-0.132.md.txt new file mode 100644 index 000000000..8d5c4d637 --- /dev/null +++ b/430/_sources/release/release-0.132.md.txt @@ -0,0 +1,40 @@ +# Release 0.132 + +:::{warning} +{func}`concat` on {ref}`array-type`, or enabling `columnar_processing_dictionary` +may cause queries to fail in this release. This is fixed in {doc}`/release/release-0.133`. +::: + +## General + +- Fix a correctness issue that can occur when any join depends on the output + of another outer join that has an inner side (or either side for the full outer + case) for which the connector declares that it has no data during planning. +- Improve error messages for unresolved operators. +- Add support for creating constant arrays with more than 255 elements. +- Fix analyzer for queries with `GROUP BY ()` such that errors are raised + during analysis rather than execution. +- Add `resource_overcommit` session property. This disables all memory + limits for the query. Instead it may be killed at any time, if the coordinator + needs to reclaim memory. +- Add support for transactional connectors. +- Add support for non-correlated scalar sub-queries. +- Add support for SQL binary literals. +- Add variant of {func}`random` that produces an integer number between 0 and a + specified upper bound. +- Perform bounds checks when evaluating {func}`abs`. +- Improve accuracy of memory accounting for {func}`map_agg` and {func}`array_agg`. + These functions will now appear to use more memory than before. +- Various performance optimizations for functions operating on {ref}`array-type`. +- Add server version to web UI. + +## CLI + +- Fix sporadic *"Failed to disable interrupt character"* error after exiting pager. + +## Hive + +- Report metastore and namenode latency in milliseconds rather than seconds in + JMX stats. +- Fix `NullPointerException` when inserting a null value for a partition column. +- Improve CPU efficiency when writing data. diff --git a/430/_sources/release/release-0.133.md.txt b/430/_sources/release/release-0.133.md.txt new file mode 100644 index 000000000..6754cee3f --- /dev/null +++ b/430/_sources/release/release-0.133.md.txt @@ -0,0 +1,17 @@ +# Release 0.133 + +## General + +- Add support for calling connector-defined procedures using {doc}`/sql/call`. +- Add {doc}`/connector/system` procedure for killing running queries. +- Properly expire idle transactions that consist of just the start transaction statement + and nothing else. +- Fix possible deadlock in worker communication when task restart is detected. +- Performance improvements for aggregations on dictionary encoded data. + This optimization is turned off by default. It can be configured via the + `optimizer.dictionary-aggregation` config property or the + `dictionary_aggregation` session property. +- Fix race which could cause queries to fail when using {func}`concat` on + {ref}`array-type`, or when enabling `columnar_processing_dictionary`. +- Add sticky headers and the ability to sort the tasks table on the query page + in the web interface. diff --git a/430/_sources/release/release-0.134.md.txt b/430/_sources/release/release-0.134.md.txt new file mode 100644 index 000000000..67032be0c --- /dev/null +++ b/430/_sources/release/release-0.134.md.txt @@ -0,0 +1,24 @@ +# Release 0.134 + +## General + +- Add cumulative memory statistics tracking and expose the stat in the web interface. +- Remove nullability and partition key flags from {doc}`/sql/show-columns`. +- Remove non-standard `is_partition_key` column from `information_schema.columns`. +- Fix performance regression in creation of `DictionaryBlock`. +- Fix rare memory accounting leak in queries with `JOIN`. + +## Hive + +- The comment for partition keys is now prefixed with *"Partition Key"*. + +## SPI + +- Remove legacy partition API methods and classes. + +:::{note} +This is a backwards incompatible change with the previous connector SPI. +If you have written a connector and have not yet updated to the +`TableLayout` API, you will need to update your code before deploying +this release. +::: diff --git a/430/_sources/release/release-0.135.md.txt b/430/_sources/release/release-0.135.md.txt new file mode 100644 index 000000000..566837688 --- /dev/null +++ b/430/_sources/release/release-0.135.md.txt @@ -0,0 +1,10 @@ +# Release 0.135 + +## General + +- Add summary of change in CPU usage to verifier output. +- Add cast between JSON and VARCHAR, BOOLEAN, DOUBLE, BIGINT. For the old + behavior of cast between JSON and VARCHAR (pre-{doc}`/release/release-0.122`), + use {func}`json_parse` and {func}`json_format`. +- Fix bug in 0.134 that prevented query page in web UI from displaying in + Safari. diff --git a/430/_sources/release/release-0.136.md.txt b/430/_sources/release/release-0.136.md.txt new file mode 100644 index 000000000..55b67f439 --- /dev/null +++ b/430/_sources/release/release-0.136.md.txt @@ -0,0 +1,9 @@ +# Release 0.136 + +## General + +- Add `control.query-types` and `test.query-types` to verifier, which can + be used to select the type of queries to run. +- Fix issue where queries with `ORDER BY LIMIT` with a limit greater than + 2147483647 could fail or return incorrect results. +- Add query plan visualization with live stats to the web UI. diff --git a/430/_sources/release/release-0.137.md.txt b/430/_sources/release/release-0.137.md.txt new file mode 100644 index 000000000..c937b1f60 --- /dev/null +++ b/430/_sources/release/release-0.137.md.txt @@ -0,0 +1,34 @@ +# Release 0.137 + +## General + +- Fix `current_date` to return correct results for all time zones. +- Fix invalid plans when scalar subqueries use `GROUP BY`, `DISTINCT` or `JOIN`. +- Do not allow creating views with a column type of `UNKNOWN`. +- Improve expression optimizer to remove some redundant operations. +- Add {func}`bit_count`, {func}`bitwise_not`, {func}`bitwise_and`, + {func}`bitwise_or`, and {func}`bitwise_xor` functions. +- Add {func}`approx_distinct` aggregation support for `VARBINARY` input. +- Add create time to query detail page in UI. +- Add support for `VARCHAR(length)` type. +- Track per-stage peak memory usage. +- Allow using double input for {func}`approx_percentile` with an array of + percentiles. +- Add API to JDBC driver to track query progress. + +## Hive + +- Do not allow inserting into tables when the Hive type does not match + the Presto type. Previously, Presto would insert data that did not + match the table or partition type and that data could not be read by + Hive. For example, Presto would write files containing `BIGINT` + data for a Hive column type of `INT`. +- Add validation to {doc}`/sql/create-table` and {doc}`/sql/create-table-as` + to check that partition keys are the last columns in the table and in the same + order as the table properties. +- Remove `retention_days` table property. This property is not used by Hive. +- Fix Parquet decoding of `MAP` containing a null value. +- Add support for accessing ORC columns by name. By default, columns in ORC + files are accessed by their ordinal position in the Hive table definition. + To access columns based on the names recorded in the ORC file, set + `hive.orc.use-column-names=true` in your Hive catalog properties file. diff --git a/430/_sources/release/release-0.138.md.txt b/430/_sources/release/release-0.138.md.txt new file mode 100644 index 000000000..6735e7f94 --- /dev/null +++ b/430/_sources/release/release-0.138.md.txt @@ -0,0 +1,20 @@ +# Release 0.138 + +## General + +- Fix planning bug with `NULL` literal coercions. +- Reduce query startup time by reducing lock contention in scheduler. + +## New Hive Parquet reader + +We have added a new Parquet reader implementation. The new reader supports vectorized +reads, lazy loading, and predicate push down, all of which make the reader more +efficient and typically reduces wall clock time for a query. Although the new +reader has been heavily tested, it is an extensive rewrite of the Apache Hive +Parquet reader, and may have some latent issues, so it is not enabled by default. +If you are using Parquet we suggest you test out the new reader on a per-query basis +by setting the `.parquet_optimized_reader_enabled` session property, +or you can enable the reader by default by setting the Hive catalog property +`hive.parquet-optimized-reader.enabled=true`. To enable Parquet predicate push down +there is a separate session property `.parquet_predicate_pushdown_enabled` +and configuration property `hive.parquet-predicate-pushdown.enabled=true`. diff --git a/430/_sources/release/release-0.139.md.txt b/430/_sources/release/release-0.139.md.txt new file mode 100644 index 000000000..f907d05a9 --- /dev/null +++ b/430/_sources/release/release-0.139.md.txt @@ -0,0 +1,24 @@ +# Release 0.139 + +## Dynamic split concurrency + +The number of running leaf splits per query is now dynamically adjusted to improve +overall cluster throughput. `task.initial-splits-per-node` can be used to set +the initial number of splits, and `task.split-concurrency-adjustment-interval` +can be used to change how frequently adjustments happen. The session properties +`initial_splits_per_node` and `split_concurrency_adjustment_interval` can +also be used. + +## General + +- Fix planning bug that causes some joins to not be redistributed when + `distributed-joins-enabled` is true. +- Fix rare leak of stage objects and tasks for queries using `LIMIT`. +- Add experimental `task.join-concurrency` config which can be used to increase + concurrency for the probe side of joins. + +## Hive + +- Remove cursor-based readers for ORC and DWRF file formats, as they have been + replaced by page-based readers. +- Fix creating tables on S3 with {doc}`/sql/create-table-as`. diff --git a/430/_sources/release/release-0.140.md.txt b/430/_sources/release/release-0.140.md.txt new file mode 100644 index 000000000..8a64e2421 --- /dev/null +++ b/430/_sources/release/release-0.140.md.txt @@ -0,0 +1,37 @@ +# Release 0.140 + +## General + +- Add the `TRY` function to handle specific data exceptions. See + {doc}`/functions/conditional`. +- Optimize predicate expressions to minimize redundancies. +- Add environment name to UI. +- Fix logging of `failure_host` and `failure_task` fields in + `QueryCompletionEvent`. +- Fix race which can cause queries to fail with a `REMOTE_TASK_ERROR`. +- Optimize {func}`array_distinct` for `array(bigint)`. +- Optimize `>` operator for {ref}`array-type`. +- Fix an optimization issue that could result in non-deterministic functions + being evaluated more than once producing unexpected results. +- Fix incorrect result for rare `IN` lists that contain certain combinations + of non-constant expressions that are null and non-null. +- Improve performance of joins, aggregations, etc. by removing unnecessarily + duplicated columns. +- Optimize `NOT IN` queries to produce more compact predicates. + +## Hive + +- Remove bogus "from deserializer" column comments. +- Change categorization of Hive writer errors to be more specific. +- Add date and timestamp support to new Parquet Reader + +## SPI + +- Remove partition key from `ColumnMetadata`. +- Change return type of `ConnectorTableLayout.getDiscretePredicates()`. + +:::{note} +This is a backwards incompatible change with the previous connector SPI. +If you have written a connector, you will need to update your code +before deploying this release. +::: diff --git a/430/_sources/release/release-0.141.md.txt b/430/_sources/release/release-0.141.md.txt new file mode 100644 index 000000000..02c7dc22d --- /dev/null +++ b/430/_sources/release/release-0.141.md.txt @@ -0,0 +1,5 @@ +# Release 0.141 + +## General + +- Fix server returning an HTTP 500 response for queries with parse errors. diff --git a/430/_sources/release/release-0.142.md.txt b/430/_sources/release/release-0.142.md.txt new file mode 100644 index 000000000..b3b451e2e --- /dev/null +++ b/430/_sources/release/release-0.142.md.txt @@ -0,0 +1,28 @@ +# Release 0.142 + +## General + +- Fix planning bug for `JOIN` criteria that optimizes to a `FALSE` expression. +- Fix planning bug when the output of `UNION` doesn't match the table column order + in `INSERT` queries. +- Fix error when `ORDER BY` clause in window specification refers to the same column multiple times. +- Add support for {ref}`complex grouping operations` + \- `CUBE`, `ROLLUP` and `GROUPING SETS`. +- Add support for `IF NOT EXISTS` in `CREATE TABLE AS` queries. +- Add {func}`substring` function. +- Add `http.server.authentication.krb5.keytab` config option to set the location of the Kerberos + keytab file explicitly. +- Add `optimize_metadata_queries` session property to enable the metadata-only query optimization. +- Improve support for non-equality predicates in `JOIN` criteria. +- Add support for non-correlated subqueries in aggregation queries. +- Improve performance of {func}`json_extract`. + +## Hive + +- Change ORC input format to report actual bytes read as opposed to estimated bytes. +- Fix cache invalidation when renaming tables. +- Fix Parquet reader to handle uppercase column names. +- Fix issue where the `hive.respect-table-format` config option was being ignored. +- Add {doc}`hive.compression-codec ` config option to control + compression used when writing. The default is now `GZIP` for all formats. +- Collect and expose end-to-end execution time JMX metric for requests to AWS services. diff --git a/430/_sources/release/release-0.143.md.txt b/430/_sources/release/release-0.143.md.txt new file mode 100644 index 000000000..47fa04a62 --- /dev/null +++ b/430/_sources/release/release-0.143.md.txt @@ -0,0 +1,26 @@ +# Release 0.143 + +## General + +- Fix race condition in output buffer that can cause a page to be lost. +- Fix case-sensitivity issue when de-referencing row fields. +- Fix bug in phased scheduler that could cause queries to block forever. +- Fix {doc}`/sql/delete` for predicates that optimize to false. +- Add support for scalar subqueries in {doc}`/sql/delete` queries. +- Add config option `query.max-cpu-time` to limit CPU time used by a query. +- Add loading indicator and error message to query detail page in UI. +- Add query teardown to query timeline visualizer. +- Add string padding functions {func}`lpad` and {func}`rpad`. +- Add {func}`width_bucket` function. +- Add {func}`truncate` function. +- Improve query startup time in large clusters. +- Improve error messages for `CAST` and {func}`slice`. + +## Hive + +- Fix native memory leak when reading or writing gzip compressed data. +- Fix performance regression due to complex expressions not being applied + when pruning partitions. +- Fix data corruption in {doc}`/sql/create-table-as` when + `hive.respect-table-format` config is set to false and user-specified + storage format does not match default. diff --git a/430/_sources/release/release-0.144.1.md.txt b/430/_sources/release/release-0.144.1.md.txt new file mode 100644 index 000000000..c923621cc --- /dev/null +++ b/430/_sources/release/release-0.144.1.md.txt @@ -0,0 +1,5 @@ +# Release 0.144.1 + +## Hive + +- Fix bug when grouping on a bucketed column which causes incorrect results. diff --git a/430/_sources/release/release-0.144.2.md.txt b/430/_sources/release/release-0.144.2.md.txt new file mode 100644 index 000000000..d46011e88 --- /dev/null +++ b/430/_sources/release/release-0.144.2.md.txt @@ -0,0 +1,7 @@ +# Release 0.144.2 + +## General + +- Fix potential memory leak in coordinator query history. +- Add `driver.max-page-partitioning-buffer-size` config to control buffer size + used to repartition pages for exchanges. diff --git a/430/_sources/release/release-0.144.3.md.txt b/430/_sources/release/release-0.144.3.md.txt new file mode 100644 index 000000000..f2c51615e --- /dev/null +++ b/430/_sources/release/release-0.144.3.md.txt @@ -0,0 +1,14 @@ +# Release 0.144.3 + +## General + +- Fix bugs in planner where coercions were not taken into account when computing + types. +- Fix compiler failure when `TRY` is a sub-expression. +- Fix compiler failure when `TRY` is called on a constant or an input reference. +- Fix race condition that can cause queries that process data from non-columnar data + sources to fail. + +## Hive + +- Fix reading symlinks when the target is in a different HDFS instance. diff --git a/430/_sources/release/release-0.144.4.md.txt b/430/_sources/release/release-0.144.4.md.txt new file mode 100644 index 000000000..37fd64089 --- /dev/null +++ b/430/_sources/release/release-0.144.4.md.txt @@ -0,0 +1,5 @@ +# Release 0.144.4 + +## General + +- Fix incorrect results for grouping sets for some queries with filters. diff --git a/430/_sources/release/release-0.144.5.md.txt b/430/_sources/release/release-0.144.5.md.txt new file mode 100644 index 000000000..68c33c29d --- /dev/null +++ b/430/_sources/release/release-0.144.5.md.txt @@ -0,0 +1,10 @@ +# Release 0.144.5 + +## General + +- Fix window functions to correctly handle empty frames between unbounded and + bounded in the same direction. For example, a frame such as + `ROWS BETWEEN UNBOUNDED PRECEDING AND 2 PRECEDING` + would incorrectly use the first row as the window frame for the first two + rows rather than using an empty frame. +- Fix correctness issue when grouping on columns that are also arguments to aggregation functions. diff --git a/430/_sources/release/release-0.144.6.md.txt b/430/_sources/release/release-0.144.6.md.txt new file mode 100644 index 000000000..bac0618fb --- /dev/null +++ b/430/_sources/release/release-0.144.6.md.txt @@ -0,0 +1,24 @@ +# Release 0.144.6 + +## General + +This release fixes several problems with large and negative intervals. + +- Fix parsing of negative interval literals. Previously, the sign of each field was treated + independently instead of applying to the entire interval value. For example, the literal + `INTERVAL '-2-3' YEAR TO MONTH` was interpreted as a negative interval of `21` months + rather than `27` months (positive `3` months was added to negative `24` months). +- Fix handling of `INTERVAL DAY TO SECOND` type in REST API. Previously, intervals greater than + `2,147,483,647` milliseconds (about `24` days) were returned as the wrong value. +- Fix handling of `INTERVAL YEAR TO MONTH` type. Previously, intervals greater than + `2,147,483,647` months were returned as the wrong value from the REST API + and parsed incorrectly when specified as a literal. +- Fix formatting of negative intervals in REST API. Previously, negative intervals + had a negative sign before each component and could not be parsed. +- Fix formatting of negative intervals in JDBC `PrestoInterval` classes. + +:::{note} +Older versions of the JDBC driver will misinterpret most negative +intervals from new servers. Make sure to update the JDBC driver +along with the server. +::: diff --git a/430/_sources/release/release-0.144.7.md.txt b/430/_sources/release/release-0.144.7.md.txt new file mode 100644 index 000000000..2fd4e5fb1 --- /dev/null +++ b/430/_sources/release/release-0.144.7.md.txt @@ -0,0 +1,7 @@ +# Release 0.144.7 + +## General + +- Fail queries with non-equi conjuncts in `OUTER JOIN`s, instead of silently + dropping such conjuncts from the query and producing incorrect results. +- Add {func}`cosine_similarity` function. diff --git a/430/_sources/release/release-0.144.md.txt b/430/_sources/release/release-0.144.md.txt new file mode 100644 index 000000000..9e3ca19a3 --- /dev/null +++ b/430/_sources/release/release-0.144.md.txt @@ -0,0 +1,27 @@ +# Release 0.144 + +:::{warning} +Querying bucketed tables in the Hive connector may produce incorrect results. +This is fixed in {doc}`/release/release-0.144.1`, and {doc}`/release/release-0.145`. +::: + +## General + +- Fix already exists check when adding a column to be case-insensitive. +- Fix correctness issue when complex grouping operations have a partitioned source. +- Fix missing coercion when using `INSERT` with `NULL` literals. +- Fix regression that the queries fail when aggregation functions present in `AT TIME ZONE`. +- Fix potential memory starvation when a query is run with `resource_overcommit=true`. +- Queries run with `resource_overcommit=true` may now be killed before + they reach `query.max-memory` if the cluster is low on memory. +- Discard output stage JSON from completion event when it is very long. + This limit can be configured with `event.max-output-stage-size`. +- Add support for {doc}`/sql/explain-analyze`. +- Change `infoUri` field of `/v1/statement` to point to query HTML page instead of JSON. +- Improve performance when processing results in CLI and JDBC driver. +- Improve performance of `GROUP BY` queries. + +## Hive + +- Fix ORC reader to actually use `hive.orc.stream-buffer-size` configuration property. +- Add support for creating and inserting into bucketed tables. diff --git a/430/_sources/release/release-0.145.md.txt b/430/_sources/release/release-0.145.md.txt new file mode 100644 index 000000000..caa5be422 --- /dev/null +++ b/430/_sources/release/release-0.145.md.txt @@ -0,0 +1,38 @@ +# Release 0.145 + +## General + +- Fix potential memory leak in coordinator query history. +- Fix column resolution issue when qualified name refers to a view. +- Fail arithmetic operations on overflow. +- Fix bugs in planner where coercions were not taken into account when computing + types. +- Fix compiler failure when `TRY` is a sub-expression. +- Fix compiler failure when `TRY` is called on a constant or an input reference. +- Add support for the `integer` type to the Presto engine and the Hive, + Raptor, Redis, Kafka, Cassandra and example-http connectors. +- Add initial support for the `decimal` data type. +- Add `driver.max-page-partitioning-buffer-size` config to control buffer size + used to repartition pages for exchanges. +- Improve performance for distributed JOIN and GROUP BY queries with billions + of groups. +- Improve reliability in highly congested networks by adjusting the default + connection idle timeouts. + +## Verifier + +- Change verifier to only run read-only queries by default. This behavior can be + changed with the `control.query-types` and `test.query-types` config flags. + +## CLI + +- Improve performance of output in batch mode. +- Fix hex rendering in batch mode. +- Abort running queries when CLI is terminated. + +## Hive + +- Fix bug when grouping on a bucketed column which causes incorrect results. +- Add `max_split_size` and `max_initial_split_size` session properties to control + the size of generated splits. +- Add retries to the metastore security calls. diff --git a/430/_sources/release/release-0.146.md.txt b/430/_sources/release/release-0.146.md.txt new file mode 100644 index 000000000..7332e00e3 --- /dev/null +++ b/430/_sources/release/release-0.146.md.txt @@ -0,0 +1,27 @@ +# Release 0.146 + +## General + +- Fix error in {func}`map_concat` when the second map is empty. +- Require at least 4096 file descriptors to run Presto. +- Support casting between map types. +- Add {doc}`/connector/mongodb`. + +## Hive + +- Fix incorrect skipping of data in Parquet during predicate push-down. +- Fix reading of Parquet maps and lists containing nulls. +- Fix reading empty ORC file with `hive.orc.use-column-names` enabled. +- Fix writing to S3 when the staging directory is a symlink to a directory. +- Legacy authorization properties, such as `hive.allow-drop-table`, are now + only enforced when `hive.security=none` is set, which is the default + security system. Specifically, the `sql-standard` authorization system + does not enforce these settings. + +## Black Hole + +- Add support for `varchar(n)`. + +## Cassandra + +- Add support for Cassandra 3.0. diff --git a/430/_sources/release/release-0.147.md.txt b/430/_sources/release/release-0.147.md.txt new file mode 100644 index 000000000..ed35f9d6c --- /dev/null +++ b/430/_sources/release/release-0.147.md.txt @@ -0,0 +1,56 @@ +# Release 0.147 + +## General + +- Fix race condition that can cause queries that process data from non-columnar + data sources to fail. +- Fix incorrect formatting of dates and timestamps before year 1680. +- Fix handling of syntax errors when parsing `EXTRACT`. +- Fix potential scheduling deadlock for connectors that expose node-partitioned data. +- Fix performance regression that increased planning time. +- Fix incorrect results for grouping sets for some queries with filters. +- Add {doc}`/sql/show-create-view` and {doc}`/sql/show-create-table`. +- Add support for column aliases in `WITH` clause. +- Support `LIKE` clause for {doc}`/sql/show-catalogs` and {doc}`/sql/show-schemas`. +- Add support for `INTERSECT`. +- Add support for casting row types. +- Add {func}`sequence` function. +- Add {func}`sign` function. +- Add {func}`flatten` function. +- Add experimental implementation of {doc}`resource groups `. +- Add {doc}`/connector/localfile`. +- Remove experimental intermediate aggregation optimizer. The `optimizer.use-intermediate-aggregations` + config option and `task_intermediate_aggregation` session property are no longer supported. +- Add support for colocated joins for connectors that expose node-partitioned data. +- Improve the performance of {func}`array_intersect`. +- Generalize the intra-node parallel execution system to work with all query stages. + The `task.concurrency` configuration property replaces the old `task.join-concurrency` + and `task.default-concurrency` options. Similarly, the `task_concurrency` session + property replaces the `task_join_concurrency`, `task_hash_build_concurrency`, and + `task_aggregation_concurrency` properties. + +## Hive + +- Fix reading symlinks when the target is in a different HDFS instance. +- Fix `NoClassDefFoundError` for `SubnetUtils` in HDFS client. +- Fix error when reading from Hive tables with inconsistent bucketing metadata. +- Correctly report read bytes when reading Parquet data. +- Include path in unrecoverable S3 exception messages. +- When replacing an existing Presto view, update the view data + in the Hive metastore rather than dropping and recreating it. +- Rename table property `clustered_by` to `bucketed_by`. +- Add support for `varchar(n)`. + +## Kafka + +- Fix `error code 6` when reading data from Kafka. +- Add support for `varchar(n)`. + +## Redis + +- Add support for `varchar(n)`. + +## MySQL and PostgreSQL + +- Cleanup temporary data when a `CREATE TABLE AS` fails. +- Add support for `varchar(n)`. diff --git a/430/_sources/release/release-0.148.md.txt b/430/_sources/release/release-0.148.md.txt new file mode 100644 index 000000000..ebbc95b76 --- /dev/null +++ b/430/_sources/release/release-0.148.md.txt @@ -0,0 +1,107 @@ +# Release 0.148 + +## General + +- Fix issue where auto-commit transaction can be rolled back for a successfully + completed query. +- Fix detection of colocated joins. +- Fix planning bug involving partitioning with constants. +- Fix window functions to correctly handle empty frames between unbounded and + bounded in the same direction. For example, a frame such as + `ROWS BETWEEN UNBOUNDED PRECEDING AND 2 PRECEDING` + would incorrectly use the first row as the window frame for the first two + rows rather than using an empty frame. +- Fix correctness issue when grouping on columns that are also arguments to aggregation functions. +- Fix failure when chaining `AT TIME ZONE`, e.g. + `SELECT TIMESTAMP '2016-01-02 12:34:56' AT TIME ZONE 'America/Los_Angeles' AT TIME ZONE 'UTC'`. +- Fix data duplication when `task.writer-count` configuration mismatches between coordinator and worker. +- Fix bug where `node-scheduler.max-pending-splits-per-node-per-task` config is not always + honored by node scheduler. This bug could stop the cluster from making further progress. +- Fix incorrect results for grouping sets with partitioned source. +- Add `colocated-joins-enabled` to enable colocated joins by default for + connectors that expose node-partitioned data. +- Add support for colocated unions. +- Reduce initial memory usage of {func}`array_agg` function. +- Improve planning of co-partitioned `JOIN` and `UNION`. +- Improve planning of aggregations over partitioned data. +- Improve the performance of the {func}`array_sort` function. +- Improve outer join predicate push down. +- Increase default value for `query.initial-hash-partitions` to `100`. +- Change default value of `query.max-memory-per-node` to `10%` of the Java heap. +- Change default `task.max-worker-threads` to `2` times the number of cores. +- Use HTTPS in JDBC driver when using port 443. +- Warn if Presto server is not using G1 garbage collector. +- Move interval types out of SPI. + +## Interval fixes + +This release fixes several problems with large and negative intervals. + +- Fix parsing of negative interval literals. Previously, the sign of each field was treated + independently instead of applying to the entire interval value. For example, the literal + `INTERVAL '-2-3' YEAR TO MONTH` was interpreted as a negative interval of `21` months + rather than `27` months (positive `3` months was added to negative `24` months). +- Fix handling of `INTERVAL DAY TO SECOND` type in REST API. Previously, intervals greater than + `2,147,483,647` milliseconds (about `24` days) were returned as the wrong value. +- Fix handling of `INTERVAL YEAR TO MONTH` type. Previously, intervals greater than + `2,147,483,647` months were returned as the wrong value from the REST API + and parsed incorrectly when specified as a literal. +- Fix formatting of negative intervals in REST API. Previously, negative intervals + had a negative sign before each component and could not be parsed. +- Fix formatting of negative intervals in JDBC `PrestoInterval` classes. + +:::{note} +Older versions of the JDBC driver will misinterpret most negative +intervals from new servers. Make sure to update the JDBC driver +along with the server. +::: + +## Functions and language features + +- Add {func}`element_at` function for map type. +- Add {func}`split_to_map` function. +- Add {func}`zip` function. +- Add {func}`map_union` aggregation function. +- Add `ROW` syntax for constructing row types. +- Add support for `REVOKE` permission syntax. +- Add support for `SMALLINT` and `TINYINT` types. +- Add support for non-equi outer joins. + +## Verifier + +- Add `skip-cpu-check-regex` config property which can be used to skip the CPU + time comparison for queries that match the given regex. +- Add `check-cpu` config property which can be used to disable CPU time comparison. + +## Hive + +- Fix `NoClassDefFoundError` for `KMSClientProvider` in HDFS client. +- Fix creating tables on S3 in an empty database. +- Implement `REVOKE` permission syntax. +- Add support for `SMALLINT` and `TINYINT` +- Support `DELETE` from unpartitioned tables. +- Add support for Kerberos authentication when talking to Hive/HDFS. +- Push down filters for columns of type `DECIMAL`. +- Improve CPU efficiency when reading ORC files. + +## Cassandra + +- Allow configuring load balancing policy and no host available retry. +- Add support for `varchar(n)`. + +## Kafka + +- Update to Kafka client 0.8.2.2. This enables support for LZ4 data. + +## JMX + +- Add `jmx.history` schema with in-memory periodic samples of values from JMX MBeans. + +## MySQL and PostgreSQL + +- Push down predicates for `VARCHAR`, `DATE`, `TIME` and `TIMESTAMP` types. + +## Other connectors + +- Add support for `varchar(n)` to the Redis, TPCH, MongoDB, Local File + and Example HTTP connectors. diff --git a/430/_sources/release/release-0.149.md.txt b/430/_sources/release/release-0.149.md.txt new file mode 100644 index 000000000..faf6c4ad7 --- /dev/null +++ b/430/_sources/release/release-0.149.md.txt @@ -0,0 +1,39 @@ +# Release 0.149 + +## General + +- Fix runtime failure for queries that use grouping sets over unions. +- Do not ignore null values in {func}`array_agg`. +- Fix failure when casting row values that contain null fields. +- Fix failure when using complex types as map keys. +- Fix potential memory tracking leak when queries are cancelled. +- Fix rejection of queries that do not match any queue/resource group rules. + Previously, a 500 error was returned to the client. +- Fix {func}`trim` and {func}`rtrim` functions to produce more intuitive results + when the argument contains invalid `UTF-8` sequences. +- Add a new web interface with cluster overview, realtime stats, and improved sorting + and filtering of queries. +- Add support for `FLOAT` type. +- Rename `query.max-age` to `query.min-expire-age`. +- `optimizer.columnar-processing` and `optimizer.columnar-processing-dictionary` + properties were merged to `optimizer.processing-optimization` with possible + values `disabled`, `columnar` and `columnar_dictionary` +- `columnar_processing` and `columnar_processing_dictionary` session + properties were merged to `processing_optimization` with possible values + `disabled`, `columnar` and `columnar_dictionary` +- Change `%y` (2-digit year) in {func}`date_parse` to evaluate to a year between + 1970 and 2069 inclusive. +- Add `queued` flag to `StatementStats` in REST API. +- Improve error messages for math operations. +- Improve memory tracking in exchanges to avoid running out of Java heap space. +- Improve performance of subscript operator for the `MAP` type. +- Improve performance of `JOIN` and `GROUP BY` queries. + +## Hive + +- Clean up empty staging directories after inserts. +- Add `hive.dfs.ipc-ping-interval` config for HDFS. +- Change default value of `hive.dfs-timeout` to 60 seconds. +- Fix ORC/DWRF reader to avoid repeatedly fetching the same data when stripes + are skipped. +- Fix force local scheduling for S3 or other non-HDFS file systems. diff --git a/430/_sources/release/release-0.150.md.txt b/430/_sources/release/release-0.150.md.txt new file mode 100644 index 000000000..5c03eedb0 --- /dev/null +++ b/430/_sources/release/release-0.150.md.txt @@ -0,0 +1,20 @@ +# Release 0.150 + +:::{warning} +The Hive bucketing optimizations are broken in this release. You should +disable them by adding `hive.bucket-execution=false` to your +Hive catalog properties. +::: + +## General + +- Fix web UI bug that caused rendering to fail when a stage has no tasks. +- Fix failure due to ambiguity when calling {func}`round` on `tinyint` arguments. +- Fix race in exchange HTTP endpoint, which could cause queries to fail randomly. +- Add support for parsing timestamps with nanosecond precision in {func}`date_parse`. +- Add CPU quotas to resource groups. + +## Hive + +- Add support for writing to bucketed tables. +- Add execution optimizations for bucketed tables. diff --git a/430/_sources/release/release-0.151.md.txt b/430/_sources/release/release-0.151.md.txt new file mode 100644 index 000000000..a2f4a5699 --- /dev/null +++ b/430/_sources/release/release-0.151.md.txt @@ -0,0 +1,30 @@ +# Release 0.151 + +## General + +- Fix issue where aggregations may produce the wrong result when `task.concurrency` is set to `1`. +- Fix query failure when `array`, `map`, or `row` type is used in non-equi `JOIN`. +- Fix performance regression for queries using `OUTER JOIN`. +- Fix query failure when using the {func}`arbitrary` aggregation function on `integer` type. +- Add various math functions that operate directly on `float` type. +- Add flag `deprecated.legacy-array-agg` to restore legacy {func}`array_agg` + behavior (ignore `NULL` input). This flag will be removed in a future release. +- Add support for uncorrelated `EXISTS` clause. +- Add {func}`cosine_similarity` function. +- Allow Tableau web connector to use catalogs other than `hive`. + +## Verifier + +- Add `shadow-writes.enabled` option which can be used to transform `CREATE TABLE AS SELECT` + queries to write to a temporary table (rather than the originally specified table). + +## SPI + +- Remove `getDataSourceName` from `ConnectorSplitSource`. +- Remove `dataSourceName` constructor parameter from `FixedSplitSource`. + +:::{note} +This is a backwards incompatible change with the previous connector SPI. +If you have written a connector, you will need to update your code +before deploying this release. +::: diff --git a/430/_sources/release/release-0.152.1.md.txt b/430/_sources/release/release-0.152.1.md.txt new file mode 100644 index 000000000..d00c48b1e --- /dev/null +++ b/430/_sources/release/release-0.152.1.md.txt @@ -0,0 +1,6 @@ +# Release 0.152.1 + +## General + +- Fix race which could cause failed queries to have no error details. +- Fix race in HTTP layer which could cause queries to fail. diff --git a/430/_sources/release/release-0.152.2.md.txt b/430/_sources/release/release-0.152.2.md.txt new file mode 100644 index 000000000..09aafbefe --- /dev/null +++ b/430/_sources/release/release-0.152.2.md.txt @@ -0,0 +1,5 @@ +# Release 0.152.2 + +## Hive + +- Improve performance of ORC reader when decoding dictionary encoded {ref}`map-type`. diff --git a/430/_sources/release/release-0.152.3.md.txt b/430/_sources/release/release-0.152.3.md.txt new file mode 100644 index 000000000..843b349fd --- /dev/null +++ b/430/_sources/release/release-0.152.3.md.txt @@ -0,0 +1,5 @@ +# Release 0.152.3 + +## General + +- Fix incorrect results for grouping sets when `task.concurrency` is greater than one. diff --git a/430/_sources/release/release-0.152.md.txt b/430/_sources/release/release-0.152.md.txt new file mode 100644 index 000000000..c50811203 --- /dev/null +++ b/430/_sources/release/release-0.152.md.txt @@ -0,0 +1,76 @@ +# Release 0.152 + +## General + +- Add {func}`array_union` function. +- Add {func}`reverse` function for arrays. +- Fix issue that could cause queries with `varchar` literals to fail. +- Fix categorization of errors from {func}`url_decode`, allowing it to be used with `TRY`. +- Fix error reporting for invalid JSON paths provided to JSON functions. +- Fix view creation for queries containing `GROUPING SETS`. +- Fix query failure when referencing a field of a `NULL` row. +- Improve query performance for multiple consecutive window functions. +- Prevent web UI from breaking when query fails without an error code. +- Display port on the task list in the web UI when multiple workers share the same host. +- Add support for `EXCEPT`. +- Rename `FLOAT` type to `REAL` for better compatibility with the SQL standard. +- Fix potential performance regression when transporting rows between nodes. + +## JDBC driver + +- Fix sizes returned from `DatabaseMetaData.getColumns()` for + `COLUMN_SIZE`, `DECIMAL_DIGITS`, `NUM_PREC_RADIX` and `CHAR_OCTET_LENGTH`. + +## Hive + +- Fix resource leak in Parquet reader. +- Rename JMX stat `AllViews` to `GetAllViews` in `ThriftHiveMetastore`. +- Add file based security, which can be configured with the `hive.security` + and `security.config-file` config properties. See {doc}`/connector/hive-security` + for more details. +- Add support for custom S3 credentials providers using the + `presto.s3.credentials-provider` Hadoop configuration property. + +## MySQL + +- Fix reading MySQL `tinyint(1)` columns. Previously, these columns were + incorrectly returned as a boolean rather than an integer. +- Add support for `INSERT`. +- Add support for reading data as `tinyint` and `smallint` types rather than `integer`. + +## PostgreSQL + +- Add support for `INSERT`. +- Add support for reading data as `tinyint` and `smallint` types rather than `integer`. + +## SPI + +- Remove `owner` from `ConnectorTableMetadata`. + +- Replace the generic `getServices()` method in `Plugin` with specific + methods such as `getConnectorFactories()`, `getTypes()`, etc. + Dependencies like `TypeManager` are now provided directly rather + than being injected into `Plugin`. + +- Add first-class support for functions in the SPI. This replaces the old + `FunctionFactory` interface. Plugins can return a list of classes from the + `getFunctions()` method: + + - Scalar functions are methods or classes annotated with `@ScalarFunction`. + - Aggregation functions are methods or classes annotated with `@AggregationFunction`. + - Window functions are an implementation of `WindowFunction`. Most implementations + should be a subclass of `RankingWindowFunction` or `ValueWindowFunction`. + +:::{note} +This is a backwards incompatible change with the previous SPI. +If you have written a plugin, you will need to update your code +before deploying this release. +::: + +## Verifier + +- Fix handling of shadow write queries with a `LIMIT`. + +## Local file + +- Fix file descriptor leak. diff --git a/430/_sources/release/release-0.153.md.txt b/430/_sources/release/release-0.153.md.txt new file mode 100644 index 000000000..6491bf47c --- /dev/null +++ b/430/_sources/release/release-0.153.md.txt @@ -0,0 +1,151 @@ +# Release 0.153 + +## General + +- Fix incorrect results for grouping sets when `task.concurrency` is greater than one. +- Fix silent numeric overflow when casting `INTEGER` to large `DECIMAL` types. +- Fix issue where `GROUP BY ()` would produce no results if the input had no rows. +- Fix null handling in {func}`array_distinct` when applied to the `array(bigint)` type. +- Fix handling of `-2^63` as the element index for {func}`json_array_get`. +- Fix correctness issue when the input to `TRY_CAST` evaluates to null. + For types such as booleans, numbers, dates, timestamps, etc., rather than + returning null, a default value specific to the type such as + `false`, `0` or `1970-01-01` was returned. +- Fix potential thread deadlock in coordinator. +- Fix rare correctness issue with an aggregation on a single threaded right join when + `task.concurrency` is `1`. +- Fix query failure when casting a map with null values. +- Fix failure when view column names contain upper-case letters. +- Fix potential performance regression due to skew issue when + grouping or joining on columns of the following types: `TINYINT`, + `SMALLINT`, `INTEGER`, `BIGINT`, `REAL`, `DOUBLE`, + `COLOR`, `DATE`, `INTERVAL`, `TIME`, `TIMESTAMP`. +- Fix potential memory leak for delete queries. +- Fix query stats to not include queued time in planning time. +- Fix query completion event to log final stats for the query. +- Fix spurious log messages when queries are torn down. +- Remove broken `%w` specifier for {func}`date_format` and {func}`date_parse`. +- Improve performance of {ref}`array-type` when underlying data is dictionary encoded. +- Improve performance of outer joins with non-equality criteria. +- Require task concurrency and task writer count to be a power of two. +- Use nulls-last ordering for {func}`array_sort`. +- Validate that `TRY` is used with exactly one argument. +- Allow running Presto with early-access Java versions. +- Add {doc}`/connector/accumulo`. + +## Functions and language features + +- Allow subqueries in non-equality outer join criteria. +- Add support for {doc}`/sql/create-schema`, {doc}`/sql/drop-schema` + and {doc}`/sql/alter-schema`. +- Add initial support for correlated subqueries. +- Add execution support for prepared statements. +- Add `DOUBLE PRECISION` as an alias for the `DOUBLE` type. +- Add {func}`typeof` for discovering expression types. +- Add decimal support to {func}`avg`, {func}`ceil`, {func}`floor`, {func}`round`, + {func}`truncate`, {func}`abs`, {func}`mod` and {func}`sign`. +- Add {func}`shuffle` function for arrays. + +## Pluggable resource groups + +Resource group management is now pluggable. A `Plugin` can +provide management factories via `getResourceGroupConfigurationManagerFactories()` +and the factory can be enabled via the `etc/resource-groups.properties` +configuration file by setting the `resource-groups.configuration-manager` +property. See the `presto-resource-group-managers` plugin for an example +and {doc}`/admin/resource-groups` for more details. + +## Web UI + +- Fix rendering failures due to null nested data structures. +- Do not include coordinator in active worker count on cluster overview page. +- Replace buffer skew indicators on query details page with scheduled time skew. +- Add stage total buffer, pending tasks and wall time to stage statistics on query details page. +- Add option to filter task lists by status on query details page. +- Add copy button for query text, query ID, and user to query details page. + +## JDBC driver + +- Add support for `real` data type, which corresponds to the Java `float` type. + +## CLI + +- Add support for configuring the HTTPS Truststore. + +## Hive + +- Fix permissions for new tables when using SQL-standard authorization. +- Improve performance of ORC reader when decoding dictionary encoded {ref}`map-type`. +- Allow certain combinations of queries to be executed in a transaction-ish manner, + for example, when dropping a partition and then recreating it. Atomicity is not + guaranteed due to fundamental limitations in the design of Hive. +- Support per-transaction cache for Hive metastore. +- Fail queries that attempt to rename partition columns. +- Add support for ORC bloom filters in predicate push down. + This is can be enabled using the `hive.orc.bloom-filters.enabled` + configuration property or the `orc_bloom_filters_enabled` session property. +- Add new optimized RCFile reader. + This can be enabled using the `hive.rcfile-optimized-reader.enabled` + configuration property or the `rcfile_optimized_reader_enabled` session property. +- Add support for the Presto `real` type, which corresponds to the Hive `float` type. +- Add support for `char(x)` type. +- Add support for creating, dropping and renaming schemas (databases). + The filesystem location can be specified when creating a schema, + which allows, for example, easily creating tables on S3. +- Record Presto query ID for tables or partitions written by Presto + using the `presto_query_id` table or partition property. +- Include path name in error message when listing a directory fails. +- Rename `allow-all` authorization method to `legacy`. This + method is deprecated and will be removed in a future release. +- Do not retry S3 requests that are aborted intentionally. +- Set the user agent suffix for S3 requests to `presto`. +- Allow configuring the user agent prefix for S3 requests + using the `hive.s3.user-agent-prefix` configuration property. +- Add support for S3-compatible storage using the `hive.s3.endpoint` + and `hive.s3.signer-type` configuration properties. +- Add support for using AWS KMS with S3 as an encryption materials provider + using the `hive.s3.kms-key-id` configuration property. +- Allow configuring a custom S3 encryption materials provider using the + `hive.s3.encryption-materials-provider` configuration property. + +## JMX + +- Make name configuration for history tables case-insensitive. + +## MySQL + +- Optimize fetching column names when describing a single table. +- Add support for `char(x)` and `real` data types. + +## PostgreSQL + +- Optimize fetching column names when describing a single table. +- Add support for `char(x)` and `real` data types. +- Add support for querying materialized views. + +## Blackhole + +- Add `page_processing_delay` table property. + +## SPI + +- Add `schemaExists()` method to `ConnectorMetadata`. +- Add transaction to grant/revoke in `ConnectorAccessControl`. +- Add `isCoordinator()` and `getVersion()` methods to `Node`. +- Remove `setOptionalConfig()` method from `Plugin`. +- Remove `ServerInfo` class. +- Make `NodeManager` specific to a connector instance. +- Replace `ConnectorFactoryContext` with `ConnectorContext`. +- Use `@SqlNullable` for functions instead of `@Nullable`. +- Prevent plugins from seeing classes that are not part of the JDK (bootstrap classes) or the SPI. +- Update `presto-maven-plugin`, which provides a Maven packaging and + lifecycle for plugins, to validate that every SPI dependency is marked + as `provided` scope and that only SPI dependencies use `provided` + scope. This helps find potential dependency and class loader issues + at build time rather than at runtime. + +:::{note} +These are backwards incompatible changes with the previous SPI. +If you have written a plugin, you will need to update your code +before deploying this release. +::: diff --git a/430/_sources/release/release-0.154.md.txt b/430/_sources/release/release-0.154.md.txt new file mode 100644 index 000000000..3e4c66dba --- /dev/null +++ b/430/_sources/release/release-0.154.md.txt @@ -0,0 +1,26 @@ +# Release 0.154 + +## General + +- Fix planning issue that could cause `JOIN` queries involving functions + that return null on non-null input to produce incorrect results. +- Fix regression that would cause certain queries involving uncorrelated + subqueries in `IN` predicates to fail during planning. +- Fix potential *"Input symbols do not match output symbols"* + error when writing to bucketed tables. +- Fix potential *"Requested array size exceeds VM limit"* error + that triggers the JVM's `OutOfMemoryError` handling. +- Improve performance of window functions with identical partitioning and + ordering but different frame specifications. +- Add `code-cache-collection-threshold` config which controls when Presto + will attempt to force collection of the JVM code cache and reduce the + default threshold to `40%`. +- Add support for using `LIKE` with {doc}`/sql/create-table`. +- Add support for `DESCRIBE INPUT` to describe the requirements for + the input parameters to a prepared statement. + +## Hive + +- Fix handling of metastore cache TTL. With the introduction of the + per-transaction cache, the cache timeout was reset after each access, + which means cache entries might never expire. diff --git a/430/_sources/release/release-0.155.md.txt b/430/_sources/release/release-0.155.md.txt new file mode 100644 index 000000000..a83d81f9a --- /dev/null +++ b/430/_sources/release/release-0.155.md.txt @@ -0,0 +1,29 @@ +# Release 0.155 + +## General + +- Fix incorrect results when queries contain multiple grouping sets that + resolve to the same set. +- Fix incorrect results when using `map` with `IN` predicates. +- Fix compile failure for outer joins that have a complex join criteria. +- Fix error messages for failures during commit. +- Fix memory accounting for simple aggregation, top N and distinct queries. + These queries may now report higher memory usage than before. +- Reduce unnecessary memory usage of {func}`map_agg`, {func}`multimap_agg` + and {func}`map_union`. +- Make `INCLUDING`, `EXCLUDING` and `PROPERTIES` non-reserved keywords. +- Remove support for the experimental feature to compute approximate queries + based on sampled tables. +- Properly account for time spent creating page source. +- Various optimizations to reduce coordinator CPU usage. + +## Hive + +- Fix schema evolution support in new Parquet reader. +- Fix `NoClassDefFoundError` when using Hadoop KMS. +- Add support for Avro file format. +- Always produce dictionary blocks for DWRF dictionary encoded streams. + +## SPI + +- Remove legacy connector API. diff --git a/430/_sources/release/release-0.156.md.txt b/430/_sources/release/release-0.156.md.txt new file mode 100644 index 000000000..fb72abd39 --- /dev/null +++ b/430/_sources/release/release-0.156.md.txt @@ -0,0 +1,40 @@ +# Release 0.156 + +:::{warning} +Query may incorrectly produce `NULL` when no row qualifies for the aggregation +if the `optimize_mixed_distinct_aggregations` session property or +the `optimizer.optimize-mixed-distinct-aggregations` config option is enabled. +::: + +## General + +- Fix potential correctness issue in queries that contain correlated scalar aggregation subqueries. +- Fix query failure when using `AT TIME ZONE` in `VALUES` list. +- Add support for quantified comparison predicates: `ALL`, `ANY`, and `SOME`. +- Add support for {ref}`array-type` and {ref}`row-type` that contain `NULL` + in {func}`checksum` aggregation. +- Add support for filtered aggregations. Example: `SELECT sum(a) FILTER (WHERE b > 0) FROM ...` +- Add a variant of {func}`from_unixtime` function that takes a timezone argument. +- Improve performance of `GROUP BY` queries that compute a mix of distinct + and non-distinct aggregations. This optimization can be turned on by setting + the `optimizer.optimize-mixed-distinct-aggregations` configuration option or + via the `optimize_mixed_distinct_aggregations` session property. +- Change default task concurrency to 16. + +## Hive + +- Add support for legacy RCFile header version in new RCFile reader. + +## Redis + +- Support `iso8601` data format for the `hash` row decoder. + +## SPI + +- Make `ConnectorPageSink#finish()` asynchronous. + +:::{note} +These are backwards incompatible changes with the previous SPI. +If you have written a plugin, you will need to update your code +before deploying this release. +::: diff --git a/430/_sources/release/release-0.157.1.md.txt b/430/_sources/release/release-0.157.1.md.txt new file mode 100644 index 000000000..ee3055c11 --- /dev/null +++ b/430/_sources/release/release-0.157.1.md.txt @@ -0,0 +1,6 @@ +# Release 0.157.1 + +## General + +- Fix regression that could cause high CPU and heap usage on coordinator, + when processing certain types of long running queries. diff --git a/430/_sources/release/release-0.157.md.txt b/430/_sources/release/release-0.157.md.txt new file mode 100644 index 000000000..4da11201a --- /dev/null +++ b/430/_sources/release/release-0.157.md.txt @@ -0,0 +1,23 @@ +# Release 0.157 + +## General + +- Fix regression that could cause queries containing scalar subqueries to fail + during planning. +- Reduce CPU usage of coordinator in large, heavily loaded clusters. +- Add support for `DESCRIBE OUTPUT`. +- Add {func}`bitwise_and_agg` and {func}`bitwise_or_agg` aggregation functions. +- Add JMX stats for the scheduler. +- Add `query.min-schedule-split-batch-size` config flag to set the minimum number of + splits to consider for scheduling per batch. +- Remove support for scheduling multiple tasks in the same stage on a single worker. +- Rename `node-scheduler.max-pending-splits-per-node-per-stage` to + `node-scheduler.max-pending-splits-per-task`. The old name may still be used, but is + deprecated and will be removed in a future version. + +## Hive + +- Fail attempts to create tables that are bucketed on non-existent columns. +- Improve error message when trying to query tables that are bucketed on non-existent columns. +- Add support for processing partitions whose schema does not match the table schema. +- Add support for creating external Hive tables using the `external_location` table property. diff --git a/430/_sources/release/release-0.158.md.txt b/430/_sources/release/release-0.158.md.txt new file mode 100644 index 000000000..f209f05a0 --- /dev/null +++ b/430/_sources/release/release-0.158.md.txt @@ -0,0 +1,31 @@ +# Release 0.158 + +## General + +- Fix regression that could cause high CPU and heap usage on coordinator + when processing certain types of long running queries. +- Fix incorrect pruning of output columns in `EXPLAIN ANALYZE`. +- Fix ordering of `CHAR` values so that trailing spaces are ordered after control characters. +- Fix query failures for connectors that produce non-remotely accessible splits. +- Fix non-linear performance issue when parsing certain SQL expressions. +- Fix case-sensitivity issues when operating on columns of `ROW` data type. +- Fix failure when creating views for tables names that need quoting. +- Return `NULL` from {func}`element_at` for out-of-range indices instead of failing. +- Remove redundancies in query plans, which can reduce data transfers over the network and reduce CPU requirements. +- Validate resource groups configuration file on startup to ensure that all + selectors reference a configured resource group. +- Add experimental on-disk merge sort for aggregations. This can be enabled with + the `experimental.spill-enabled` configuration flag. +- Push down predicates for `DECIMAL`, `TINYINT`, `SMALLINT` and `REAL` data types. + +## Hive + +- Add hidden `$bucket` column for bucketed tables that + contains the bucket number for the current row. +- Prevent inserting into non-managed (i.e., external) tables. +- Add configurable size limit to Hive metastore cache to avoid using too much + coordinator memory. + +## Cassandra + +- Allow starting the server even if a contact point hostname cannot be resolved. diff --git a/430/_sources/release/release-0.159.md.txt b/430/_sources/release/release-0.159.md.txt new file mode 100644 index 000000000..d251c0014 --- /dev/null +++ b/430/_sources/release/release-0.159.md.txt @@ -0,0 +1,11 @@ +# Release 0.159 + +## General + +- Improve predicate performance for `JOIN` queries. + +## Hive + +- Optimize filtering of partition names to reduce object creation. +- Add limit on the number of partitions that can potentially be read per table scan. + This limit is configured using `hive.max-partitions-per-scan` and defaults to 100,000. diff --git a/430/_sources/release/release-0.160.md.txt b/430/_sources/release/release-0.160.md.txt new file mode 100644 index 000000000..a3433e616 --- /dev/null +++ b/430/_sources/release/release-0.160.md.txt @@ -0,0 +1,18 @@ +# Release 0.160 + +## General + +- Fix planning failure when query has multiple unions with identical underlying columns. +- Fix planning failure when multiple `IN` predicates contain an identical subquery. +- Fix resource waste where coordinator floods rebooted workers if worker + comes back before coordinator times out the query. +- Add {doc}`/functions/lambda`. + +## Hive + +- Fix planning failure when inserting into columns of struct types with uppercase field names. +- Fix resource leak when using Kerberos authentication with impersonation. +- Fix creating external tables so that they are properly recognized by the Hive metastore. + The Hive table property `EXTERNAL` is now set to `TRUE` in addition to the setting + the table type. Any previously created tables need to be modified to have this property. +- Add `bucket_execution_enabled` session property. diff --git a/430/_sources/release/release-0.161.md.txt b/430/_sources/release/release-0.161.md.txt new file mode 100644 index 000000000..eb84aa598 --- /dev/null +++ b/430/_sources/release/release-0.161.md.txt @@ -0,0 +1,32 @@ +# Release 0.161 + +## General + +- Fix correctness issue for queries involving multiple nested EXCEPT clauses. + A query such as `a EXCEPT (b EXCEPT c)` was incorrectly evaluated as + `a EXCEPT b EXCEPT c` and thus could return the wrong result. +- Fix failure when executing prepared statements that contain parameters in the join criteria. +- Fix failure when describing the output of prepared statements that contain aggregations. +- Fix planning failure when a lambda is used in the context of an aggregation or subquery. +- Fix column resolution rules for `ORDER BY` to match the behavior expected + by the SQL standard. This is a change in semantics that breaks + backwards compatibility. To ease migration of existing queries, the legacy + behavior can be restored by the `deprecated.legacy-order-by` config option + or the `legacy_order_by` session property. +- Improve error message when coordinator responds with `403 FORBIDDEN`. +- Improve performance for queries containing expressions in the join criteria + that reference columns on one side of the join. +- Improve performance of {func}`map_concat` when one argument is empty. +- Remove `/v1/execute` resource. +- Add new column to {doc}`/sql/show-columns` (and {doc}`/sql/describe`) + to show extra information from connectors. +- Add {func}`map` to construct an empty {ref}`map-type`. + +## Hive connector + +- Remove `"Partition Key: "` prefix from column comments and + replace it with the new extra information field described above. + +## JMX connector + +- Add support for escaped commas in `jmx.dump-tables` config property. diff --git a/430/_sources/release/release-0.162.md.txt b/430/_sources/release/release-0.162.md.txt new file mode 100644 index 000000000..b4a8be54a --- /dev/null +++ b/430/_sources/release/release-0.162.md.txt @@ -0,0 +1,34 @@ +# Release 0.162 + +:::{warning} +The {func}`xxhash64` function introduced in this release will return a +varbinary instead of a bigint in the next release. +::: + +## General + +- Fix correctness issue when the type of the value in the `IN` predicate does + not match the type of the elements in the subquery. +- Fix correctness issue when the value on the left-hand side of an `IN` + expression or a quantified comparison is `NULL`. +- Fix correctness issue when the subquery of a quantified comparison produces no rows. +- Fix correctness issue due to improper inlining of TRY arguments. +- Fix correctness issue when the right side of a JOIN produces a very large number of rows. +- Fix correctness issue for expressions with multiple nested `AND` and `OR` conditions. +- Improve performance of window functions with similar `PARTITION BY` clauses. +- Improve performance of certain multi-way JOINs by automatically choosing the + best evaluation order. This feature is turned off by default and can be enabled + via the `reorder-joins` config option or `reorder_joins` session property. +- Add {func}`xxhash64` and {func}`to_big_endian_64` functions. +- Add aggregated operator statistics to final query statistics. +- Allow specifying column comments for {doc}`/sql/create-table`. + +## Hive + +- Fix performance regression when querying Hive tables with large numbers of partitions. + +## SPI + +- Connectors can now return optional output metadata for write operations. +- Add ability for event listeners to get connector-specific output metadata. +- Add client-supplied payload field `X-Presto-Client-Info` to `EventListener`. diff --git a/430/_sources/release/release-0.163.md.txt b/430/_sources/release/release-0.163.md.txt new file mode 100644 index 000000000..3a66d5f55 --- /dev/null +++ b/430/_sources/release/release-0.163.md.txt @@ -0,0 +1,36 @@ +# Release 0.163 + +## General + +- Fix data corruption when transporting dictionary-encoded data. +- Fix potential deadlock when resource groups are configured with memory limits. +- Improve performance for `OUTER JOIN` queries. +- Improve exchange performance by reading from buffers in parallel. +- Improve performance when only a subset of the columns resulting from a `JOIN` are referenced. +- Make `ALL`, `SOME` and `ANY` non-reserved keywords. +- Add {func}`from_big_endian_64` function. +- Change {func}`xxhash64` return type from `BIGINT` to `VARBINARY`. +- Change subscript operator for map types to fail if the key is not present in the map. The former + behavior (returning `NULL`) can be restored by setting the `deprecated.legacy-map-subscript` + config option. +- Improve `EXPLAIN ANALYZE` to render stats more accurately and to include input statistics. +- Improve tolerance to communication errors for long running queries. This can be adjusted + with the `query.remote-task.max-error-duration` config option. + +## Accumulo + +- Fix issue that could cause incorrect results for large rows. + +## MongoDB + +- Fix NullPointerException when a field contains a null. + +## Cassandra + +- Add support for `VARBINARY`, `TIMESTAMP` and `REAL` data types. + +## Hive + +- Fix issue that would prevent predicates from being pushed into Parquet reader. +- Fix Hive metastore user permissions caching when tables are dropped or renamed. +- Add experimental file based metastore which stores information in HDFS or S3 instead of a database. diff --git a/430/_sources/release/release-0.164.md.txt b/430/_sources/release/release-0.164.md.txt new file mode 100644 index 000000000..32c3fa6dc --- /dev/null +++ b/430/_sources/release/release-0.164.md.txt @@ -0,0 +1,27 @@ +# Release 0.164 + +## General + +- Fix correctness issue for queries that perform `DISTINCT` and `LIMIT` on the results of a `JOIN`. +- Fix correctness issue when casting between maps where the key or value is the `REAL` type. +- Fix correctness issue in {func}`min_by` and {func}`max_by` when nulls are present in the comparison column. +- Fail queries when `FILTER` clause is specified for scalar functions. +- Fix planning failure for certain correlated subqueries that contain aggregations. +- Fix planning failure when arguments to selective aggregates are derived from other selective aggregates. +- Fix boolean expression optimization bug that can cause long planning times, planning failures and coordinator instability. +- Fix query failure when `TRY` or lambda expression with the exact same body is repeated in an expression. +- Fix split source resource leak in coordinator that can occur when a query fails. +- Improve {func}`array_join` performance. +- Improve error message for map subscript operator when key is not present in the map. +- Improve client error message for invalid session. +- Add `VALIDATE` mode for {doc}`/sql/explain`. + +## Web UI + +- Add resource group to query detail page. + +## Hive + +- Fix handling of ORC files containing extremely large metadata. +- Fix failure when creating views in file based metastore. +- Improve performance for queries that read bucketed tables by optimizing scheduling. diff --git a/430/_sources/release/release-0.165.md.txt b/430/_sources/release/release-0.165.md.txt new file mode 100644 index 000000000..dd4f46cee --- /dev/null +++ b/430/_sources/release/release-0.165.md.txt @@ -0,0 +1,18 @@ +# Release 0.165 + +## General + +- Make `AT` a non-reserved keyword. +- Improve performance of {func}`transform`. +- Improve exchange performance by deserializing in parallel. +- Add support for compressed exchanges. This can be enabled with the `exchange.compression-enabled` + config option. +- Add input and hash collision statistics to {doc}`/sql/explain-analyze` output. + +## Hive + +- Add support for MAP and ARRAY types in optimized Parquet reader. + +## MySQL and PostgreSQL + +- Fix connection leak on workers. diff --git a/430/_sources/release/release-0.166.md.txt b/430/_sources/release/release-0.166.md.txt new file mode 100644 index 000000000..3453e1df7 --- /dev/null +++ b/430/_sources/release/release-0.166.md.txt @@ -0,0 +1,22 @@ +# Release 0.166 + +## General + +- Fix failure due to implicit coercion issue in `IN` expressions for + certain combinations of data types (e.g., `double` and `decimal`). +- Add `query.max-length` config flag to set the maximum length of a query. + The default maximum length is 1MB. +- Improve performance of {func}`approx_percentile`. + +## Hive + +- Include original exception from metastore for `AlreadyExistsException` when adding partitions. +- Add support for the Hive JSON file format (`org.apache.hive.hcatalog.data.JsonSerDe`). + +## Cassandra + +- Add configuration properties for speculative execution. + +## SPI + +- Add peak memory reservation to `SplitStatistics` in split completion events. diff --git a/430/_sources/release/release-0.167.md.txt b/430/_sources/release/release-0.167.md.txt new file mode 100644 index 000000000..479af9f2f --- /dev/null +++ b/430/_sources/release/release-0.167.md.txt @@ -0,0 +1,61 @@ +# Release 0.167 + +## General + +- Fix planning failure when a window function depends on the output of another window function. +- Fix planning failure for certain aggregation with both `DISTINCT` and `GROUP BY`. +- Fix incorrect aggregation of operator summary statistics. +- Fix a join issue that could cause joins that produce and filter many rows + to monopolize worker threads, even after the query has finished. +- Expand plan predicate pushdown capabilities involving implicitly coerced types. +- Short-circuit inner and right join when right side is empty. +- Optimize constant patterns for `LIKE` predicates that use an escape character. +- Validate escape sequences in `LIKE` predicates per the SQL standard. +- Reduce memory usage of {func}`min_by` and {func}`max_by`. +- Add {func}`transform_keys`, {func}`transform_values` and {func}`zip_with` lambda functions. +- Add {func}`levenshtein_distance` function. +- Add JMX stat for the elapsed time of the longest currently active split. +- Add JMX stats for compiler caches. +- Raise required Java version to 8u92. + +## Security + +- The `http.server.authentication.enabled` config option that previously enabled + Kerberos has been replaced with `http-server.authentication.type=KERBEROS`. +- Add support for {doc}`/security/ldap` using username and password. +- Add a read-only {doc}`/develop/system-access-control` named `read-only`. +- Allow access controls to filter the results of listing catalogs, schemas and tables. +- Add access control checks for {doc}`/sql/show-schemas` and {doc}`/sql/show-tables`. + +## Web UI + +- Add operator-level performance analysis. +- Improve visibility of blocked and reserved query states. +- Lots of minor improvements. + +## JDBC driver + +- Allow escaping in `DatabaseMetaData` patterns. + +## Hive + +- Fix write operations for `ViewFileSystem` by using a relative location. +- Remove support for the `hive-cdh4` and `hive-hadoop1` connectors which + support CDH 4 and Hadoop 1.x, respectively. +- Remove the `hive-cdh5` connector as an alias for `hive-hadoop2`. +- Remove support for the legacy S3 block-based file system. +- Add support for KMS-managed keys for S3 server-side encryption. + +## Cassandra + +- Add support for Cassandra 3.x by removing the deprecated Thrift interface used to + connect to Cassandra. The following config options are now defunct and must be removed: + `cassandra.thrift-port`, `cassandra.thrift-connection-factory-class`, + `cassandra.transport-factory-options` and `cassandra.partitioner`. + +## SPI + +- Add methods to `SystemAccessControl` and `ConnectorAccessControl` to + filter the list of catalogs, schemas and tables. +- Add access control checks for {doc}`/sql/show-schemas` and {doc}`/sql/show-tables`. +- Add `beginQuery` and `cleanupQuery` notifications to `ConnectorMetadata`. diff --git a/430/_sources/release/release-0.168.md.txt b/430/_sources/release/release-0.168.md.txt new file mode 100644 index 000000000..8a199730d --- /dev/null +++ b/430/_sources/release/release-0.168.md.txt @@ -0,0 +1,53 @@ +# Release 0.168 + +## General + +- Fix correctness issues for certain `JOIN` queries that require implicit coercions + for terms in the join criteria. +- Fix invalid "No more locations already set" error. +- Fix invalid "No more buffers already set" error. +- Temporarily revert empty join short-circuit optimization due to issue with hanging queries. +- Improve performance of `DECIMAL` type and operators. +- Optimize window frame computation for empty frames. +- {func}`json_extract` and {func}`json_extract_scalar` now support escaping double + quotes or backslashes using a backslash with a JSON path subscript. This changes + the semantics of any invocation using a backslash, as backslashes were previously + treated as normal characters. +- Improve performance of {func}`filter` and {func}`map_filter` lambda functions. +- Add {doc}`/connector/memory`. +- Add {func}`arrays_overlap` and {func}`array_except` functions. +- Allow concatenating more than two arrays with `concat()` or maps with {func}`map_concat`. +- Add a time limit for the iterative optimizer. It can be adjusted via the `iterative_optimizer_timeout` + session property or `experimental.iterative-optimizer-timeout` configuration option. +- `ROW` types are now orderable if all of the field types are orderable. + This allows using them in comparison expressions, `ORDER BY` and + functions that require orderable types (e.g., {func}`max`). + +## JDBC driver + +- Update `DatabaseMetaData` to reflect features that are now supported. +- Update advertised JDBC version to 4.2, which part of Java 8. +- Return correct driver and server versions rather than `1.0`. + +## Hive + +- Fix reading decimals for RCFile text format using non-optimized reader. +- Fix bug which prevented the file based metastore from being used. +- Enable optimized RCFile reader by default. +- Common user errors are now correctly categorized. +- Add new, experimental, RCFile writer optimized for Presto. The new writer can be enabled with the + `rcfile_optimized_writer_enabled` session property or the `hive.rcfile-optimized-writer.enabled` + Hive catalog property. + +## Cassandra + +- Add predicate pushdown for clustering key. + +## MongoDB + +- Allow SSL connections using the `mongodb.ssl.enabled` config flag. + +## SPI + +- ConnectorIndex now returns `ConnectorPageSource` instead of `RecordSet`. Existing connectors + that support index join can use the `RecordPageSource` to adapt to the new API. diff --git a/430/_sources/release/release-0.169.md.txt b/430/_sources/release/release-0.169.md.txt new file mode 100644 index 000000000..03182f8a0 --- /dev/null +++ b/430/_sources/release/release-0.169.md.txt @@ -0,0 +1,21 @@ +# Release 0.169 + +## General + +- Fix regression that could cause queries involving `JOIN` and certain language features + such as `current_date`, `current_time` or `extract` to fail during planning. +- Limit the maximum allowed input size to {func}`levenshtein_distance`. +- Improve performance of {func}`map_agg` and {func}`multimap_agg`. +- Improve memory accounting when grouping on a single `BIGINT` column. + +## JDBC driver + +- Return correct class name for `ARRAY` type from `ResultSetMetaData.getColumnClassName()`. + +## CLI + +- Fix support for non-standard offset time zones (e.g., `GMT+01:00`). + +## Cassandra + +- Add custom error codes. diff --git a/430/_sources/release/release-0.170.md.txt b/430/_sources/release/release-0.170.md.txt new file mode 100644 index 000000000..17d8eeb99 --- /dev/null +++ b/430/_sources/release/release-0.170.md.txt @@ -0,0 +1,38 @@ +# Release 0.170 + +## General + +- Fix race condition that could cause queries to fail with `InterruptedException` in rare cases. +- Fix a performance regression for `GROUP BY` queries over `UNION`. +- Fix a performance regression that occurs when a significant number of exchange + sources produce no data during an exchange (e.g., in a skewed hash join). + +## Web UI + +- Fix broken rendering when catalog properties are set. +- Fix rendering of live plan when query is queued. + +## JDBC driver + +- Add support for `DatabaseMetaData.getTypeInfo()`. + +## Hive + +- Improve decimal support for the Parquet reader. +- Remove misleading "HDFS" string from error messages. + +## Cassandra + +- Fix an intermittent connection issue for Cassandra 2.1. +- Remove support for selecting by partition key when the partition key is only partially specified. + The `cassandra.limit-for-partition-key-select` and `cassandra.fetch-size-for-partition-key-select` + config options are no longer supported. +- Remove partition key cache to improve consistency and reduce load on the Cassandra cluster due to background cache refresh. +- Reduce the number of connections opened to the Cassandra cluster. Now Presto opens a single connection from each node. +- Use exponential backoff for retries when Cassandra hosts are down. The retry timeout can be controlled via the + `cassandra.no-host-available-retry-timeout` config option, which has a default value of `1m`. + The `cassandra.no-host-available-retry-count` config option is no longer supported. + +## Verifier + +- Add support for `INSERT` queries. diff --git a/430/_sources/release/release-0.171.md.txt b/430/_sources/release/release-0.171.md.txt new file mode 100644 index 000000000..36bd123a4 --- /dev/null +++ b/430/_sources/release/release-0.171.md.txt @@ -0,0 +1,24 @@ +# Release 0.171 + +## General + +- Fix planning regression for queries that compute a mix of distinct and non-distinct aggregations. +- Fix casting from certain complex types to `JSON` when source type contains `JSON` or `DECIMAL`. +- Fix issue for data definition queries that prevented firing completion events or purging them from + the coordinator's memory. +- Add support for capture in lambda expressions. +- Add support for `ARRAY` and `ROW` type as the compared value in {func}`min_by` and {func}`max_by`. +- Add support for `CHAR(n)` data type to common string functions. +- Add {func}`codepoint`, {func}`skewness` and {func}`kurtosis` functions. +- Improve validation of resource group configuration. +- Fail queries when casting unsupported types to JSON; see {doc}`/functions/json` for supported types. + +## Web UI + +- Fix the threads UI (`/ui/thread`). + +## Hive + +- Fix issue where some files are not deleted on cancellation of `INSERT` or `CREATE` queries. +- Allow writing to non-managed (external) Hive tables. This is disabled by default but can be + enabled via the `hive.non-managed-table-writes-enabled` configuration option. diff --git a/430/_sources/release/release-0.172.md.txt b/430/_sources/release/release-0.172.md.txt new file mode 100644 index 000000000..af85b1473 --- /dev/null +++ b/430/_sources/release/release-0.172.md.txt @@ -0,0 +1,11 @@ +# Release 0.172 + +## General + +- Fix correctness issue in `ORDER BY` queries due to improper implicit coercions. +- Fix planning failure when `GROUP BY` queries contain lambda expressions. +- Fix planning failure when left side of `IN` expression contains subqueries. +- Fix incorrect permissions check for `SHOW TABLES`. +- Fix planning failure when `JOIN` clause contains lambda expressions that reference columns or variables from the enclosing scope. +- Reduce memory usage of {func}`map_agg` and {func}`map_union`. +- Reduce memory usage of `GROUP BY` queries. diff --git a/430/_sources/release/release-0.173.md.txt b/430/_sources/release/release-0.173.md.txt new file mode 100644 index 000000000..dda87247f --- /dev/null +++ b/430/_sources/release/release-0.173.md.txt @@ -0,0 +1,6 @@ +# Release 0.173 + +## General + +- Fix issue where `FILTER` was ignored for {func}`count` with a constant argument. +- Support table comments for {doc}`/sql/create-table` and {doc}`/sql/create-table-as`. diff --git a/430/_sources/release/release-0.174.md.txt b/430/_sources/release/release-0.174.md.txt new file mode 100644 index 000000000..914b3f1ff --- /dev/null +++ b/430/_sources/release/release-0.174.md.txt @@ -0,0 +1,38 @@ +# Release 0.174 + +## General + +- Fix correctness issue for correlated subqueries containing a `LIMIT` clause. +- Fix query failure when {func}`reduce` function is used with lambda expressions + containing {func}`array_sort`, {func}`shuffle`, {func}`reverse`, {func}`array_intersect`, + {func}`arrays_overlap`, {func}`concat` (for arrays) or {func}`map_concat`. +- Fix a bug that causes underestimation of the amount of memory used by {func}`max_by`, + {func}`min_by`, {func}`max`, {func}`min`, and {func}`arbitrary` aggregations over + varchar/varbinary columns. +- Fix a memory leak in the coordinator that causes long-running queries in highly loaded + clusters to consume unnecessary memory. +- Improve performance of aggregate window functions. +- Improve parallelism of queries involving `GROUPING SETS`, `CUBE` or `ROLLUP`. +- Improve parallelism of `UNION` queries. +- Filter and projection operations are now always processed columnar if possible, and Presto + will automatically take advantage of dictionary encodings where effective. + The `processing_optimization` session property and `optimizer.processing-optimization` + configuration option have been removed. +- Add support for escaped unicode sequences in string literals. +- Add {doc}`/sql/show-grants` and `information_schema.table_privileges` table. + +## Hive + +- Change default value of `hive.metastore-cache-ttl` and `hive.metastore-refresh-interval` to 0 + to disable cross-transaction metadata caching. + +## Web UI + +- Fix ES6 compatibility issue with older browsers. +- Display buffered bytes for every stage in the live plan UI. + +## SPI + +- Add support for retrieving table grants. +- Rename SPI access control check from `checkCanShowTables` to `checkCanShowTablesMetadata`, + which is used for both {doc}`/sql/show-tables` and {doc}`/sql/show-grants`. diff --git a/430/_sources/release/release-0.175.md.txt b/430/_sources/release/release-0.175.md.txt new file mode 100644 index 000000000..c8ea7197a --- /dev/null +++ b/430/_sources/release/release-0.175.md.txt @@ -0,0 +1,31 @@ +# Release 0.175 + +## General + +- Fix *"position is not valid"* query execution failures. +- Fix memory accounting bug that can potentially cause `OutOfMemoryError`. +- Fix regression that could cause certain queries involving `UNION` and + `GROUP BY` or `JOIN` to fail during planning. +- Fix planning failure for `GROUP BY` queries containing correlated + subqueries in the `SELECT` clause. +- Fix execution failure for certain `DELETE` queries. +- Reduce occurrences of *"Method code too large"* errors. +- Reduce memory utilization for certain queries involving `ORDER BY`. +- Improve performance of map subscript from O(n) to O(1) when the map is + produced by an eligible operation, including the map constructor and + Hive readers (except ORC and optimized Parquet). More read and write + operations will take advantage of this in future releases. +- Add `enable_intermediate_aggregations` session property to enable the + use of intermediate aggregations within un-grouped aggregations. +- Add support for `INTERVAL` data type to {func}`avg` and {func}`sum` aggregation functions. +- Add support for `INT` as an alias for the `INTEGER` data type. +- Add resource group information to query events. + +## Hive + +- Make table creation metastore operations idempotent, which allows + recovery when retrying timeouts or other errors. + +## MongoDB + +- Rename `mongodb.connection-per-host` config option to `mongodb.connections-per-host`. diff --git a/430/_sources/release/release-0.176.md.txt b/430/_sources/release/release-0.176.md.txt new file mode 100644 index 000000000..80b8c082a --- /dev/null +++ b/430/_sources/release/release-0.176.md.txt @@ -0,0 +1,23 @@ +# Release 0.176 + +## General + +- Fix an issue where a query (and some of its tasks) continues to + consume CPU/memory on the coordinator and workers after the query fails. +- Fix a regression that cause the GC overhead and pauses to increase significantly when processing maps. +- Fix a memory tracking bug that causes the memory to be overestimated for `GROUP BY` queries on `bigint` columns. +- Improve the performance of the {func}`transform_values` function. +- Add support for casting from `JSON` to `REAL` type. +- Add {func}`parse_duration` function. + +## MySQL + +- Disallow having a database in the `connection-url` config property. + +## Accumulo + +- Decrease planning time by fetching index metrics in parallel. + +## MongoDB + +- Allow predicate pushdown for ObjectID. diff --git a/430/_sources/release/release-0.177.md.txt b/430/_sources/release/release-0.177.md.txt new file mode 100644 index 000000000..2dcfa37c7 --- /dev/null +++ b/430/_sources/release/release-0.177.md.txt @@ -0,0 +1,70 @@ +# Release 0.177 + +:::{warning} +Query may incorrectly produce `NULL` when no row qualifies for the aggregation +if the `optimize_mixed_distinct_aggregations` session property or +the `optimizer.optimize-mixed-distinct-aggregations` config option is enabled. +This optimization was introduced in Presto version 0.156. +::: + +## General + +- Fix correctness issue when performing range comparisons over columns of type `CHAR`. +- Fix correctness issue due to mishandling of nulls and non-deterministic expressions in + inequality joins unless `fast_inequality_join` is disabled. +- Fix excessive GC overhead caused by lambda expressions. There are still known GC issues + with captured lambda expressions. This will be fixed in a future release. +- Check for duplicate columns in `CREATE TABLE` before asking the connector to create + the table. This improves the error message for most connectors and will prevent errors + for connectors that do not perform validation internally. +- Add support for null values on the left-hand side of a semijoin (i.e., `IN` predicate + with subqueries). +- Add `SHOW STATS` to display table and query statistics. +- Improve implicit coercion support for functions involving lambda. Specifically, this makes + it easier to use the {func}`reduce` function. +- Improve plans for queries involving `ORDER BY` and `LIMIT` by avoiding unnecessary + data exchanges. +- Improve performance of queries containing window functions with identical `PARTITION BY` + and `ORDER BY` clauses. +- Improve performance of certain queries involving `OUTER JOIN` and aggregations, or + containing certain forms of correlated subqueries. This optimization is experimental + and can be turned on via the `push_aggregation_through_join` session property or the + `optimizer.push-aggregation-through-join` config option. +- Improve performance of certain queries involving joins and aggregations. This optimization + is experimental and can be turned on via the `push_partial_aggregation_through_join` + session property. +- Improve error message when a lambda expression has a different number of arguments than expected. +- Improve error message when certain invalid `GROUP BY` expressions containing lambda expressions. + +## Hive + +- Fix handling of trailing spaces for the `CHAR` type when reading RCFile. +- Allow inserts into tables that have more partitions than the partitions-per-scan limit. +- Add support for exposing Hive table statistics to the engine. This option is experimental and + can be turned on via the `statistics_enabled` session property. +- Ensure file name is always present for error messages about corrupt ORC files. + +## Cassandra + +- Remove caching of metadata in the Cassandra connector. Metadata caching makes Presto violate + the consistency defined by the Cassandra cluster. It's also unnecessary because the Cassandra + driver internally caches metadata. The `cassandra.max-schema-refresh-threads`, + `cassandra.schema-cache-ttl` and `cassandra.schema-refresh-interval` config options have + been removed. +- Fix intermittent issue in the connection retry mechanism. + +## Web UI + +- Change cluster HUD realtime statistics to be aggregated across all running queries. +- Change parallelism statistic on cluster HUD to be averaged per-worker. +- Fix bug that always showed indeterminate progress bar in query list view. +- Change running drivers statistic to exclude blocked drivers. +- Change unit of CPU and scheduled time rate sparklines to seconds on query details page. +- Change query details page refresh interval to three seconds. +- Add uptime and connected status indicators to every page. + +## CLI + +- Add support for preprocessing commands. When the `PRESTO_PREPROCESSOR` environment + variable is set, all commands are piped through the specified program before being sent to + the Presto server. diff --git a/430/_sources/release/release-0.178.md.txt b/430/_sources/release/release-0.178.md.txt new file mode 100644 index 000000000..24ee0591f --- /dev/null +++ b/430/_sources/release/release-0.178.md.txt @@ -0,0 +1,25 @@ +# Release 0.178 + +## General + +- Fix various memory accounting bugs, which reduces the likelihood of full GCs/OOMs. +- Fix a regression that causes queries that use the keyword "stats" to fail to parse. +- Fix an issue where a query does not get cleaned up on the coordinator after query failure. +- Add ability to cast to `JSON` from `REAL`, `TINYINT` or `SMALLINT`. +- Add support for `GROUPING` operation to {ref}`complex grouping operations`. +- Add support for correlated subqueries in `IN` predicates. +- Add {func}`to_ieee754_32` and {func}`to_ieee754_64` functions. + +## Hive + +- Fix high CPU usage due to schema caching when reading Avro files. +- Preserve decompression error causes when decoding ORC files. + +## Memory connector + +- Fix a bug that prevented creating empty tables. + +## SPI + +- Make environment available to resource group configuration managers. +- Add additional performance statistics to query completion event. diff --git a/430/_sources/release/release-0.179.md.txt b/430/_sources/release/release-0.179.md.txt new file mode 100644 index 000000000..0a22c48e4 --- /dev/null +++ b/430/_sources/release/release-0.179.md.txt @@ -0,0 +1,34 @@ +# Release 0.179 + +## General + +- Fix issue which could cause incorrect results when processing dictionary encoded data. + If the expression can fail on bad input, the results from filtered-out rows containing + bad input may be included in the query output ({issue}`x8262`). +- Fix planning failure when similar expressions appear in the `ORDER BY` clause of a query that + contains `ORDER BY` and `LIMIT`. +- Fix planning failure when `GROUPING()` is used with the `legacy_order_by` session property set to `true`. +- Fix parsing failure when `NFD`, `NFC`, `NFKD` or `NFKC` are used as identifiers. +- Fix a memory leak on the coordinator that manifests itself with canceled queries. +- Fix excessive GC overhead caused by captured lambda expressions. +- Reduce the memory usage of map/array aggregation functions. +- Redact sensitive config property values in the server log. +- Update timezone database to version 2017b. +- Add {func}`repeat` function. +- Add {func}`crc32` function. +- Add file based global security, which can be configured with the `etc/access-control.properties` + and `security.config-file` config properties. See {doc}`/security/built-in-system-access-control` + for more details. +- Add support for configuring query runtime and queueing time limits to resource groups. + +## Hive + +- Fail queries that access encrypted S3 objects that do not have their unencrypted content lengths set in their metadata. + +## JDBC driver + +- Add support for setting query timeout through `Statement.setQueryTimeout()`. + +## SPI + +- Add grantee and revokee to `GRANT` and `REVOKE` security checks. diff --git a/430/_sources/release/release-0.180.md.txt b/430/_sources/release/release-0.180.md.txt new file mode 100644 index 000000000..e12bc0732 --- /dev/null +++ b/430/_sources/release/release-0.180.md.txt @@ -0,0 +1,55 @@ +# Release 0.180 + +## General + +- Fix a rare bug where rows containing only `null` values are not returned + to the client. This only occurs when an entire result page contains only + `null` values. The only known case is a query over an ORC encoded Hive table + that does not perform any transformation of the data. +- Fix incorrect results when performing comparisons between values of approximate + data types (`REAL`, `DOUBLE`) and columns of certain exact numeric types + (`INTEGER`, `BIGINT`, `DECIMAL`). +- Fix memory accounting for {func}`min_by` and {func}`max_by` on complex types. +- Fix query failure due to `NoClassDefFoundError` when scalar functions declared + in plugins are implemented with instance methods. +- Improve performance of map subscript from O(n) to O(1) in all cases. Previously, only maps + produced by certain functions and readers could take advantage of this improvement. +- Skip unknown costs in `EXPLAIN` output. +- Support {doc}`/security/internal-communication` between Presto nodes. +- Add initial support for `CROSS JOIN` against `LATERAL` derived tables. +- Add support for `VARBINARY` concatenation. +- Add {doc}`/connector/thrift` that makes it possible to use Presto with + external systems without the need to implement a custom connector. +- Add experimental `/v1/resourceGroupState` REST endpoint on coordinator. + +## Hive + +- Fix skipping short decimal values in the optimized Parquet reader + when they are backed by the `int32` or `int64` types. +- Ignore partition bucketing if table is not bucketed. This allows dropping + the bucketing from table metadata but leaving it for old partitions. +- Improve error message for Hive partitions dropped during execution. +- The optimized RCFile writer is enabled by default, but can be disabled + with the `hive.rcfile-optimized-writer.enabled` config option. + The writer supports validation which reads back the entire file after + writing. Validation is disabled by default, but can be enabled with the + `hive.rcfile.writer.validate` config option. + +## Cassandra + +- Add support for `INSERT`. +- Add support for pushdown of non-equality predicates on clustering keys. + +## JDBC driver + +- Add support for authenticating using Kerberos. +- Allow configuring SSL/TLS and Kerberos properties on a per-connection basis. +- Add support for executing queries using a SOCKS or HTTP proxy. + +## CLI + +- Add support for executing queries using an HTTP proxy. + +## SPI + +- Add running time limit and queued time limit to `ResourceGroupInfo`. diff --git a/430/_sources/release/release-0.181.md.txt b/430/_sources/release/release-0.181.md.txt new file mode 100644 index 000000000..7928a1dbc --- /dev/null +++ b/430/_sources/release/release-0.181.md.txt @@ -0,0 +1,58 @@ +# Release 0.181 + +## General + +- Fix query failure and memory usage tracking when query contains + {func}`transform_keys` or {func}`transform_values`. +- Prevent `CREATE TABLE IF NOT EXISTS` queries from ever failing with *"Table already exists"*. +- Fix query failure when `ORDER BY` expressions reference columns that are used in + the `GROUP BY` clause by their fully-qualified name. +- Fix excessive GC overhead caused by large arrays and maps containing `VARCHAR` elements. +- Improve error handling when passing too many arguments to various + functions or operators that take a variable number of arguments. +- Improve performance of `count(*)` aggregations over subqueries with known + constant cardinality. +- Add `VERBOSE` option for {doc}`/sql/explain-analyze` that provides additional + low-level details about query performance. +- Add per-task distribution information to the output of `EXPLAIN ANALYZE`. +- Add support for `DROP COLUMN` in {doc}`/sql/alter-table`. +- Change local scheduler to prevent starvation of long running queries + when the cluster is under constant load from short queries. The new + behavior is disabled by default and can be enabled by setting the + config property `task.level-absolute-priority=true`. +- Improve the fairness of the local scheduler such that long-running queries + which spend more time on the CPU per scheduling quanta (e.g., due to + slow connectors) do not get a disproportionate share of CPU. The new + behavior is disabled by default and can be enabled by setting the + config property `task.legacy-scheduling-behavior=false`. +- Add a config option to control the prioritization of queries based on + elapsed scheduled time. The `task.level-time-multiplier` property + controls the target scheduled time of a level relative to the next + level. Higher values for this property increase the fraction of CPU + that will be allocated to shorter queries. This config property only + has an effect when `task.level-absolute-priority=true` and + `task.legacy-scheduling-behavior=false`. + +## Hive + +- Fix potential native memory leak when writing tables using RCFile. +- Correctly categorize certain errors when writing tables using RCFile. +- Decrease the number of file system metadata calls when reading tables. +- Add support for dropping columns. + +## JDBC driver + +- Add support for query cancellation using `Statement.cancel()`. + +## PostgreSQL + +- Add support for operations on external tables. + +## Accumulo + +- Improve query performance by scanning index ranges in parallel. + +## SPI + +- Fix regression that broke serialization for `SchemaTableName`. +- Add access control check for `DROP COLUMN`. diff --git a/430/_sources/release/release-0.182.md.txt b/430/_sources/release/release-0.182.md.txt new file mode 100644 index 000000000..dabc78562 --- /dev/null +++ b/430/_sources/release/release-0.182.md.txt @@ -0,0 +1,37 @@ +# Release 0.182 + +## General + +- Fix correctness issue that causes {func}`corr` to return positive numbers for inverse correlations. +- Fix the {doc}`/sql/explain` query plan for tables that are partitioned + on `TIMESTAMP` or `DATE` columns. +- Fix query failure when using certain window functions that take arrays or maps as arguments (e.g., {func}`approx_percentile`). +- Implement subtraction for all `TIME` and `TIMESTAMP` types. +- Improve planning performance for queries that join multiple tables with + a large number columns. +- Improve the performance of joins with only non-equality conditions by using + a nested loops join instead of a hash join. +- Improve the performance of casting from `JSON` to `ARRAY` or `MAP` types. +- Add a new {ref}`ipaddress-type` type to represent IP addresses. +- Add {func}`to_milliseconds` function to convert intervals (day to second) to milliseconds. +- Add support for column aliases in `CREATE TABLE AS` statements. +- Add a config option to reject queries during cluster initialization. + Queries are rejected if the active worker count is less than the + `query-manager.initialization-required-workers` property while the + coordinator has been running for less than `query-manager.initialization-timeout`. +- Add {doc}`/connector/tpcds`. This connector provides a set of schemas to + support the TPC Benchmark™ DS (TPC-DS). + +## CLI + +- Fix an issue that would sometimes prevent queries from being cancelled when exiting from the pager. + +## Hive + +- Fix reading decimal values in the optimized Parquet reader when they are backed + by the `int32` or `int64` types. +- Add a new experimental ORC writer implementation optimized for Presto. + We have some upcoming improvements, so we recommend waiting a few releases before + using this in production. The new writer can be enabled with the + `hive.orc.optimized-writer.enabled` configuration property or with the + `orc_optimized_writer_enabled` session property. diff --git a/430/_sources/release/release-0.183.md.txt b/430/_sources/release/release-0.183.md.txt new file mode 100644 index 000000000..d58948064 --- /dev/null +++ b/430/_sources/release/release-0.183.md.txt @@ -0,0 +1,53 @@ +# Release 0.183 + +## General + +- Fix planning failure for queries that use `GROUPING` and contain aggregation expressions + that require implicit coercions. +- Fix planning failure for queries that contains a non-equi left join that is semantically + equivalent to an inner join. +- Fix issue where a query may have a reported memory that is higher than actual usage when + an aggregation is followed by other non-trivial work in the same stage. This can lead to failures + due to query memory limit, or lower cluster throughput due to perceived insufficient memory. +- Fix query failure for `CHAR` functions {func}`trim`, {func}`rtrim`, and {func}`substr` when + the return value would have trailing spaces under `VARCHAR` semantics. +- Fix formatting in `EXPLAIN ANALYZE` output. +- Improve error message when a query contains an unsupported form of correlated subquery. +- Improve performance of `CAST(json_parse(...) AS ...)`. +- Add {func}`map_from_entries` and {func}`map_entries` functions. +- Change spilling for aggregations to only occur when the cluster runs out of memory. +- Remove the `experimental.operator-memory-limit-before-spill` config property + and the `operator_memory_limit_before_spill` session property. +- Allow configuring the amount of memory that can be used for merging spilled aggregation data + from disk using the `experimental.aggregation-operator-unspill-memory-limit` config + property or the `aggregation_operator_unspill_memory_limit` session property. + +## Web UI + +- Add output rows, output size, written rows and written size to query detail page. + +## Hive + +- Work around [ORC-222](https://issues.apache.org/jira/browse/ORC-222) which results in + invalid summary statistics in ORC or DWRF files when the input data contains invalid string data. + Previously, this would usually cause the query to fail, but in rare cases it could + cause wrong results by incorrectly skipping data based on the invalid statistics. +- Fix issue where reported memory is lower than actual usage for table columns containing + string values read from ORC or DWRF files. This can lead to high GC overhead or out-of-memory crash. +- Improve error message for small ORC files that are completely corrupt or not actually ORC. +- Add predicate pushdown for the hidden column `"$path"`. + +## TPCH + +- Add column statistics for schemas `tiny` and `sf1`. + +## TPCDS + +- Add column statistics for schemas `tiny` and `sf1`. + +## SPI + +- Map columns or values represented with `ArrayBlock` and `InterleavedBlock` are + no longer supported. They must be represented as `MapBlock` or `SingleMapBlock`. +- Extend column statistics with minimal and maximal value. +- Replace `nullsCount` with `nullsFraction` in column statistics. diff --git a/430/_sources/release/release-0.184.md.txt b/430/_sources/release/release-0.184.md.txt new file mode 100644 index 000000000..1808c876d --- /dev/null +++ b/430/_sources/release/release-0.184.md.txt @@ -0,0 +1,41 @@ +# Release 0.184 + +## General + +- Fix query execution failure for `split_to_map(...)[...]`. +- Fix issue that caused queries containing `CROSS JOIN` to continue using CPU resources + even after they were killed. +- Fix planning failure for some query shapes containing `count(*)` and a non-empty + `GROUP BY` clause. +- Fix communication failures caused by lock contention in the local scheduler. +- Improve performance of {func}`element_at` for maps to be constant time rather than + proportional to the size of the map. +- Improve performance of queries with gathering exchanges. +- Require `coalesce()` to have at least two arguments, as mandated by the SQL standard. +- Add {func}`hamming_distance` function. + +## JDBC driver + +- Always invoke the progress callback with the final stats at query completion. + +## Web UI + +- Add worker status page with information about currently running threads + and resource utilization (CPU, heap, memory pools). This page is accessible + by clicking a hostname on a query task list. + +## Hive + +- Fix partition filtering for keys of `CHAR`, `DECIMAL`, or `DATE` type. +- Reduce system memory usage when reading table columns containing string values + from ORC or DWRF files. This can prevent high GC overhead or out-of-memory crashes. + +## TPCDS + +- Fix display of table statistics when running `SHOW STATS FOR ...`. + +## SPI + +- Row columns or values represented with `ArrayBlock` and `InterleavedBlock` are + no longer supported. They must be represented as `RowBlock` or `SingleRowBlock`. +- Add `source` field to `ConnectorSession`. diff --git a/430/_sources/release/release-0.185.md.txt b/430/_sources/release/release-0.185.md.txt new file mode 100644 index 000000000..b1c28ea3c --- /dev/null +++ b/430/_sources/release/release-0.185.md.txt @@ -0,0 +1,32 @@ +# Release 0.185 + +## General + +- Fix incorrect column names in `QueryCompletedEvent`. +- Fix excessive CPU usage in coordinator for queries that have + large string literals containing non-ASCII characters. +- Fix potential infinite loop during query optimization when constant + expressions fail during evaluation. +- Fix incorrect ordering when the same field appears multiple times + with different ordering specifications in a window function `ORDER BY` + clause. For example: `OVER (ORDER BY x ASC, x DESC)`. +- Do not allow dropping or renaming hidden columns. +- When preparing to drop a column, ignore hidden columns when + checking if the table only has one column. +- Improve performance of joins where the condition is a range over a function. + For example: `a JOIN b ON b.x < f(a.x) AND b.x > g(a.x)` +- Improve performance of certain window functions (e.g., `LAG`) with similar specifications. +- Extend {func}`substr` function to work on `VARBINARY` in addition to `CHAR` and `VARCHAR`. +- Add cast from `JSON` to `ROW`. +- Allow usage of `TRY` within lambda expressions. + +## Hive + +- Improve ORC reader efficiency by only reading small ORC streams when accessed in the query. +- Improve RCFile IO efficiency by increasing the buffer size from 1 to 8 MB. +- Fix native memory leak for optimized RCFile writer. +- Fix potential native memory leak for optimized ORC writer. + +## Memory connector + +- Add support for views. diff --git a/430/_sources/release/release-0.186.md.txt b/430/_sources/release/release-0.186.md.txt new file mode 100644 index 000000000..a3c51abed --- /dev/null +++ b/430/_sources/release/release-0.186.md.txt @@ -0,0 +1,59 @@ +# Release 0.186 + +:::{warning} +This release has a stability issue that may cause query failures in large deployments +due to HTTP requests timing out. +::: + +## General + +- Fix excessive GC overhead caused by map to map cast. +- Fix implicit coercions for `ROW` types, allowing operations between + compatible types such as `ROW(INTEGER)` and `ROW(BIGINT)`. +- Fix issue that may cause queries containing expensive functions, such as regular + expressions, to continue using CPU resources even after they are killed. +- Fix performance issue caused by redundant casts. +- Fix {func}`json_parse` to not ignore trailing characters. Previously, + input such as `[1,2]abc` would successfully parse as `[1,2]`. +- Fix leak in running query counter for failed queries. The counter would + increment but never decrement for queries that failed before starting. +- Reduce coordinator HTTP thread usage for queries that are queued or waiting for data. +- Reduce memory usage when building data of `VARCHAR` or `VARBINARY` types. +- Estimate memory usage for `GROUP BY` more precisely to avoid out of memory errors. +- Add queued time and elapsed time to the client protocol. +- Add `query_max_execution_time` session property and `query.max-execution-time` config + property. Queries will be aborted after they execute for more than the specified duration. +- Add {func}`inverse_normal_cdf` function. +- Add {doc}`/functions/geospatial` including functions for processing Bing tiles. +- Add {doc}`/admin/spill` for joins. +- Add {doc}`/connector/redshift`. + +## Resource groups + +- Query Queues are deprecated in favor of {doc}`/admin/resource-groups` + and will be removed in a future release. +- Rename the `maxRunning` property to `hardConcurrencyLimit`. The old + property name is deprecated and will be removed in a future release. +- Fail on unknown property names when loading the JSON config file. + +## JDBC driver + +- Allow specifying an empty password. +- Add `getQueuedTimeMillis()` and `getElapsedTimeMillis()` to `QueryStats`. + +## Hive + +- Fix `FileSystem closed` errors when using Kerberos authentication. +- Add support for path style access to the S3 file system. This can be enabled + by setting the `hive.s3.path-style-access=true` config property. + +## SPI + +- Add an `ignoreExisting` flag to `ConnectorMetadata::createTable()`. +- Remove the `getTotalBytes()` method from `RecordCursor` and `ConnectorPageSource`. + +:::{note} +These are backwards incompatible changes with the previous SPI. +If you have written a connector, you will need to update your code +before deploying this release. +::: diff --git a/430/_sources/release/release-0.187.md.txt b/430/_sources/release/release-0.187.md.txt new file mode 100644 index 000000000..076531b96 --- /dev/null +++ b/430/_sources/release/release-0.187.md.txt @@ -0,0 +1,6 @@ +# Release 0.187 + +## General + +- Fix a stability issue that may cause query failures due to a large number of HTTP requests timing out. + The issue has been observed in a large deployment under stress. diff --git a/430/_sources/release/release-0.188.md.txt b/430/_sources/release/release-0.188.md.txt new file mode 100644 index 000000000..33a4dcd30 --- /dev/null +++ b/430/_sources/release/release-0.188.md.txt @@ -0,0 +1,39 @@ +# Release 0.188 + +## General + +- Fix handling of negative start indexes in array {func}`slice` function. +- Fix inverted sign for time zones `Etc/GMT-12`, `Etc/GMT-11`, ..., `Etc/GMT-1`, + `Etc/GMT+1`, ... `Etc/GMT+12`. +- Improve performance of server logging and HTTP request logging. +- Reduce GC spikes by compacting join memory over time instead of all at once + when memory is low. This can increase reliability at the cost of additional + CPU. This can be enabled via the `pages-index.eager-compaction-enabled` + config property. +- Improve performance of and reduce GC overhead for compaction of in-memory data structures, + primarily used in joins. +- Mitigate excessive GC and degraded query performance by forcing expiration of + generated classes for functions and expressions one hour after generation. +- Mitigate performance issue caused by JVM when generated code is used + for multiple hours or days. + +## CLI + +- Fix transaction support. Previously, after the first statement in the + transaction, the transaction would be abandoned and the session would + silently revert to auto-commit mode. + +## JDBC driver + +- Support using `Statement.cancel()` for all types of statements. + +## Resource group + +- Add environment support to the `db` resource groups manager. + Previously, configurations for different clusters had to be stored in separate databases. + With this change, different cluster configurations can be stored in the same table and + Presto will use the new `environment` column to differentiate them. + +## SPI + +- Add query plan to the query completed event. diff --git a/430/_sources/release/release-0.189.md.txt b/430/_sources/release/release-0.189.md.txt new file mode 100644 index 000000000..fa5e1e7ad --- /dev/null +++ b/430/_sources/release/release-0.189.md.txt @@ -0,0 +1,73 @@ +# Release 0.189 + +## General + +- Fix query failure while logging the query plan. +- Fix a bug that causes clients to hang when executing `LIMIT` queries when + `optimizer.force-single-node-output` is disabled. +- Fix a bug in the {func}`bing_tile_at` and {func}`bing_tile_polygon` functions + where incorrect results were produced for points close to tile edges. +- Fix variable resolution when lambda argument has the same name as a table column. +- Improve error message when running `SHOW TABLES` on a catalog that does not exist. +- Improve performance for queries with highly selective filters. +- Execute {doc}`/sql/use` on the server rather than in the CLI, allowing it + to be supported by any client. This requires clients to add support for + the protocol changes (otherwise the statement will be silently ignored). +- Allow casting `JSON` to `ROW` even if the `JSON` does not contain every + field in the `ROW`. +- Add support for dereferencing row fields in lambda expressions. + +## Security + +- Support configuring multiple authentication types, which allows supporting + clients that have different authentication requirements or gracefully + migrating between authentication types without needing to update all clients + at once. Specify multiple values for `http-server.authentication.type`, + separated with commas. +- Add support for TLS client certificates as an authentication mechanism by + specifying `CERTIFICATE` for `http-server.authentication.type`. + The distinguished name from the validated certificate will be provided as a + `javax.security.auth.x500.X500Principal`. The certificate authority (CA) + used to sign client certificates will be need to be added to the HTTP server + KeyStore (should technically be a TrustStore but separating them out is not + yet supported). +- Skip sending final leg of SPNEGO authentication when using Kerberos. + +## JDBC driver + +- Per the JDBC specification, close the `ResultSet` when `Statement` is closed. +- Add support for TLS client certificate authentication by configuring the + `SSLKeyStorePath` and `SSLKeyStorePassword` parameters. +- Add support for transactions using SQL statements or the standard JDBC mechanism. +- Allow executing the `USE` statement. Note that this is primarily useful when + running arbitrary SQL on behalf of users. For programmatic use, continuing + to use `setCatalog()` and `setSchema()` on `Connection` is recommended. +- Allow executing `SET SESSION` and `RESET SESSION`. + +## Resource group + +- Add `WEIGHTED_FAIR` resource group scheduling policy. + +## Hive + +- Do not require setting `hive.metastore.uri` when using the file metastore. +- Reduce memory usage when reading string columns from ORC or DWRF files. + +## MySQL, PostgreSQL, Redshift, and SQL Server shanges + +- Change mapping for columns with `DECIMAL(p,s)` data type from Presto `DOUBLE` + type to the corresponding Presto `DECIMAL` type. + +## Kafka + +- Fix documentation for raw decoder. + +## Thrift connector + +- Add support for index joins. + +## SPI + +- Deprecate `SliceArrayBlock`. +- Add `SessionPropertyConfigurationManager` plugin to enable overriding default + session properties dynamically. diff --git a/430/_sources/release/release-0.190.md.txt b/430/_sources/release/release-0.190.md.txt new file mode 100644 index 000000000..23105c784 --- /dev/null +++ b/430/_sources/release/release-0.190.md.txt @@ -0,0 +1,71 @@ +# Release 0.190 + +## General + +- Fix correctness issue for {func}`array_min` and {func}`array_max` when arrays contain `NaN`. +- Fix planning failure for queries involving `GROUPING` that require implicit coercions + in expressions containing aggregate functions. +- Fix potential workload imbalance when using topology-aware scheduling. +- Fix performance regression for queries containing `DISTINCT` aggregates over the same column. +- Fix a memory leak that occurs on workers. +- Improve error handling when a `HAVING` clause contains window functions. +- Avoid unnecessary data redistribution when writing when the target table has + the same partition property as the data being written. +- Ignore case when sorting the output of `SHOW FUNCTIONS`. +- Improve rendering of the `BingTile` type. +- The {func}`approx_distinct` function now supports a standard error + in the range of `[0.0040625, 0.26000]`. +- Add support for `ORDER BY` in aggregation functions. +- Add dictionary processing for joins which can improve join performance up to 50%. + This optimization can be disabled using the `dictionary-processing-joins-enabled` + config property or the `dictionary_processing_join` session property. +- Add support for casting to `INTERVAL` types. +- Add {func}`ST_Buffer` geospatial function. +- Allow treating decimal literals as values of the `DECIMAL` type rather than `DOUBLE`. + This behavior can be enabled by setting the `parse-decimal-literals-as-double` + config property or the `parse_decimal_literals_as_double` session property to `false`. +- Add JMX counter to track the number of submitted queries. + +## Resource groups + +- Add priority column to the DB resource group selectors. +- Add exact match source selector to the DB resource group selectors. + +## CLI + +- Add support for setting client tags. + +## JDBC driver + +- Add `getPeakMemoryBytes()` to `QueryStats`. + +## Accumulo + +- Improve table scan parallelism. + +## Hive + +- Fix query failures for the file-based metastore implementation when partition + column values contain a colon. +- Improve performance for writing to bucketed tables when the data being written + is already partitioned appropriately (e.g., the output is from a bucketed join). +- Add config property `hive.max-outstanding-splits-size` for the maximum + amount of memory used to buffer splits for a single table scan. Additionally, + the default value is substantially higher than the previous hard-coded limit, + which can prevent certain queries from failing. + +## Thrift connector + +- Make Thrift retry configurable. +- Add JMX counters for Thrift requests. + +## SPI + +- Remove the `RecordSink` interface, which was difficult to use + correctly and had no advantages over the `PageSink` interface. + +:::{note} +This is a backwards incompatible change with the previous connector SPI. +If you have written a connector that uses the `RecordSink` interface, +you will need to update your code before deploying this release. +::: diff --git a/430/_sources/release/release-0.191.md.txt b/430/_sources/release/release-0.191.md.txt new file mode 100644 index 000000000..ba78e6515 --- /dev/null +++ b/430/_sources/release/release-0.191.md.txt @@ -0,0 +1,55 @@ +# Release 0.191 + +## General + +- Fix regression that could cause high CPU usage for join queries when dictionary + processing for joins is enabled. +- Fix {func}`bit_count` for bits between 33 and 63. +- The `query.low-memory-killer.enabled` config property has been replaced + with `query.low-memory-killer.policy`. Use `total-reservation` to continue + using the previous policy of killing the largest query. There is also a new + policy, `total-reservation-on-blocked-nodes`, which kills the query that + is using the most memory on nodes that are out of memory (blocked). +- Add support for grouped join execution. When both sides of a join have the + same table partitioning and the partitioning is addressable, partial data + can be loaded into memory at a time, making it possible to execute the join + with less peak memory usage. The colocated join feature must be enabled with + the `colocated-joins-enabled` config property or the `colocated_join` + session property, and the `concurrent_lifespans_per_task` session property + must be specified. +- Allow connectors to report the amount of physical written data. +- Add ability to dynamically scale out the number of writer tasks rather + than allocating a fixed number of tasks. Additional tasks are added when the + the average amount of physical data per writer is above a minimum threshold. + Writer scaling can be enabled with the `scale-writers` config property or + the `scale_writers` session property. The minimum size can be set with the + `writer-min-size` config property or the `writer_min_size` session property. + The tradeoff for writer scaling is that write queries can take longer to run + due to the decreased writer parallelism while the writer count ramps up. + +## Resource groups + +- Add query type to the exact match source selector in the DB resource group selectors. + +## CLI + +- Improve display of values of the Geometry type. + +## Hive + +- Add support for grouped join execution for Hive tables when both + sides of a join have the same bucketing property. +- Report physical written data for the legacy RCFile writer, optimized RCFile + writer, and optimized ORC writer. These writers thus support writer scaling, + which can both reduce the number of written files and create larger files. + This is especially important for tables that have many small partitions, as + small files can take a disproportionately longer time to read. + +## Thrift connector + +- Add page size distribution metrics. + +## MySQL, PostgreSQL, Redshift, and SQL Server + +- Fix querying `information_schema.columns` if there are tables with + no columns or no supported columns. diff --git a/430/_sources/release/release-0.192.md.txt b/430/_sources/release/release-0.192.md.txt new file mode 100644 index 000000000..7b7510c49 --- /dev/null +++ b/430/_sources/release/release-0.192.md.txt @@ -0,0 +1,78 @@ +# Release 0.192 + +## General + +- Fix performance regression in split scheduling introduced in 0.191. If a query + scans a non-trivial number of splits (~1M splits in an hour), the coordinator + CPU utilization can be very high, leading to elevated communication failures. +- Fix correctness issue in the {func}`geometry_to_bing_tiles` function that causes + it to return irrelevant tiles when bottom or right side of the bounding box of the + geometry is aligned with the tile border. +- Fix handling of invalid WKT (well-known text) input in geospatial functions. +- Fix an issue that can cause long-running queries to hang when writer scaling is enabled. +- Fix cast from `REAL` or `DOUBLE` to `DECIMAL` to conform to the SQL standard. + For example, previously `cast (double '100000000000000000000000000000000' as decimal(38))` + would return `100000000000000005366162204393472`. Now it returns `100000000000000000000000000000000`. +- Fix bug in validation of resource groups that prevented use of the `WEIGHTED_FAIR` policy. +- Fail queries properly when the coordinator fails to fetch data from workers. + Previously, it would return an HTTP 500 error to the client. +- Improve memory tracking for queries involving `DISTINCT` or {func}`row_number` that could cause + over-committing memory resources for short time periods. +- Improve performance for queries involving `grouping()`. +- Improve buffer utilization calculation for writer scaling. +- Remove tracking of per-driver peak memory reservation. +- Add `resource-groups.max-refresh-interval` config option to limit the maximum acceptable + staleness of resource group configuration. +- Remove `dictionary-processing-joins-enabled` configuration option and `dictionary_processing_join` + session property. + +## Web UI + +- Fix incorrect reporting of input size and positions in live plan view. + +## CLI + +- Fix update of prompt after `USE` statement. +- Fix correctness issue when rendering arrays of Bing tiles that causes + the first entry to be repeated multiple times. + +## Hive + +- Fix reading partitioned table statistics from newer Hive metastores. +- Do not treat file system errors as corruptions for ORC. +- Prevent reads from tables or partitions with `object_not_readable` attribute set. +- Add support for validating ORC files after they have been written. This behavior can + be turned on via the `hive.orc.writer.validate` configuration property. +- Expose ORC writer statistics via JMX. +- Add configuration options to control ORC writer min/max rows per stripe and row group, + maximum stripe size, and memory limit for dictionaries. +- Allow reading empty ORC files. +- Handle ViewFs when checking file system cache expiration. +- Improve error reporting when the target table of an insert query is dropped. +- Remove retry when creating Hive record reader. This can help queries fail faster. + +## MySQL + +- Remove support for `TIME WITH TIME ZONE` and `TIMESTAMP WITH TIME ZONE` + types due to MySQL types not being able to store timezone information. +- Add support for `REAL` type, which maps to MySQL's `FLOAT` type. + +## PostgreSQL + +- Add support for `VARBINARY` type, which maps to PostgreSQL's `BYTEA` type. + +## MongoDB + +- Fix support for pushing down inequality operators for string types. +- Add support for reading documents as `MAP` values. +- Add support for MongoDB's `Decimal128` type. +- Treat document and array of documents as `JSON` instead of `VARCHAR`. + +## JMX + +- Allow nulls in history table values. + +## SPI + +- Remove `SliceArrayBlock` class. +- Add `offset` and `length` parameters to `Block.getPositions()`. diff --git a/430/_sources/release/release-0.193.md.txt b/430/_sources/release/release-0.193.md.txt new file mode 100644 index 000000000..c6f2d4932 --- /dev/null +++ b/430/_sources/release/release-0.193.md.txt @@ -0,0 +1,60 @@ +# Release 0.193 + +## General + +- Fix an infinite loop during planning for queries containing non-trivial predicates. +- Fix `row_number()` optimization that causes query failure or incorrect results + for queries that constrain the result of `row_number()` to be less than one. +- Fix failure during query planning when lambda expressions are used in `UNNEST` or `VALUES` clauses. +- Fix `Tried to free more revocable memory than is reserved` error for queries that have spilling enabled + and run in the reserved memory pool. +- Improve the performance of the {func}`ST_Contains` function. +- Add {func}`map_zip_with` lambda function. +- Add {func}`normal_cdf` function. +- Add `SET_DIGEST` type and related functions. +- Add query stat that tracks peak total memory. +- Improve performance of queries that filter all data from a table up-front (e.g., due to partition pruning). +- Turn on new local scheduling algorithm by default (see {doc}`release-0.181`). +- Remove the `information_schema.__internal_partitions__` table. + +## Security + +- Apply the authentication methods in the order they are listed in the + `http-server.authentication.type` configuration. + +## CLI + +- Fix rendering of maps of Bing tiles. +- Abort the query when the result pager exits. + +## JDBC driver + +- Use SSL by default for port 443. + +## Hive + +- Allow dropping any column in a table. Previously, dropping columns other + than the last one would fail with `ConcurrentModificationException`. +- Correctly write files for text format tables that use non-default delimiters. + Previously, they were written with the default delimiter. +- Fix reading data from S3 if the data is in a region other than `us-east-1`. + Previously, such queries would fail with + `"The authorization header is malformed; the region 'us-east-1' is wrong; expecting ''"`, + where `` is the S3 region hosting the bucket that is queried. +- Enable `SHOW PARTITIONS FROM
WHERE ` to work for tables + that have more than `hive.max-partitions-per-scan` partitions as long as + the specified `` reduces the number of partitions to below this limit. + +## Blackhole + +- Do not allow creating tables in a nonexistent schema. +- Add support for `CREATE SCHEMA`. + +## Memory connector + +- Allow renaming tables across schemas. Previously, the target schema was ignored. +- Do not allow creating tables in a nonexistent schema. + +## MongoDB + +- Add `INSERT` support. It was previously removed in 0.155. diff --git a/430/_sources/release/release-0.194.md.txt b/430/_sources/release/release-0.194.md.txt new file mode 100644 index 000000000..0a77336ca --- /dev/null +++ b/430/_sources/release/release-0.194.md.txt @@ -0,0 +1,47 @@ +# Release 0.194 + +## General + +- Fix planning performance regression that can affect queries over Hive tables + with many partitions. +- Fix deadlock in memory management logic introduced in the previous release. +- Add {func}`word_stem` function. +- Restrict `n` (number of result elements) to 10,000 or less for + `min(col, n)`, `max(col, n)`, `min_by(col1, col2, n)`, and `max_by(col1, col2, n)`. +- Improve error message when a session property references an invalid catalog. +- Reduce memory usage of {func}`histogram` aggregation function. +- Improve coordinator CPU efficiency when discovering splits. +- Include minimum and maximum values for columns in `SHOW STATS`. + +## Web UI + +- Fix previously empty peak memory display in the query details page. + +## CLI + +- Fix regression in CLI that makes it always print "query aborted by user" when + the result is displayed with a pager, even if the query completes successfully. +- Return a non-zero exit status when an error occurs. +- Add `--client-info` option for specifying client info. +- Add `--ignore-errors` option to continue processing in batch mode when an error occurs. + +## JDBC driver + +- Allow configuring connection network timeout with `setNetworkTimeout()`. +- Allow setting client tags via the `ClientTags` client info property. +- Expose update type via `getUpdateType()` on `PrestoStatement`. + +## Hive + +- Consistently fail queries that attempt to read partitions that are offline. + Previously, the query can have one of the following outcomes: fail as expected, + skip those partitions and finish successfully, or hang indefinitely. +- Allow setting username used to access Hive metastore via the `hive.metastore.username` config property. +- Add `hive_storage_format` and `respect_table_format` session properties, corresponding to + the `hive.storage-format` and `hive.respect-table-format` config properties. +- Reduce ORC file reader memory consumption by allocating buffers lazily. + Buffers are only allocated for columns that are actually accessed. + +## Cassandra + +- Fix failure when querying `information_schema.columns` when there is no equality predicate on `table_name`. diff --git a/430/_sources/release/release-0.195.md.txt b/430/_sources/release/release-0.195.md.txt new file mode 100644 index 000000000..c4a3e2356 --- /dev/null +++ b/430/_sources/release/release-0.195.md.txt @@ -0,0 +1,44 @@ +# Release 0.195 + +## General + +- Fix {func}`histogram` for map type when type coercion is required. +- Fix `nullif` for map type when type coercion is required. +- Fix incorrect termination of queries when the coordinator to worker communication is under high load. +- Fix race condition that causes queries with a right or full outer join to fail. +- Change reference counting for varchar, varbinary, and complex types to be approximate. This + approximation reduces GC activity when computing large aggregations with these types. +- Change communication system to be more resilient to issues such as long GC pauses or networking errors. + The min/max sliding scale of for timeouts has been removed and instead only max time is used. + The `exchange.min-error-duration` and `query.remote-task.min-error-duration` are now ignored and will be + removed in a future release. +- Increase coordinator timeout for cleanup of worker tasks for failed queries. This improves the health of + the system when workers are offline for long periods due to GC or network errors. +- Remove the `compiler.interpreter-enabled` config property. + +## Security + +- Presto now supports generic password authentication using a pluggable {doc}`/develop/password-authenticator`. + Enable password authentication by setting `http-server.authentication.type` to include `PASSWORD` as an + authentication type. +- {doc}`/security/ldap` is now implemented as a password authentication + plugin. You will need to update your configuration if you are using it. + +## CLI and JDBC + +- Provide a better error message when TLS client certificates are expired or not yet valid. + +## MySQL + +- Fix an error that can occur while listing tables if one of the listed tables is dropped. + +## Hive + +- Add support for LZ4 compressed ORC files. +- Add support for reading Zstandard compressed ORC files. +- Validate ORC compression block size when reading ORC files. +- Set timeout of Thrift metastore client. This was accidentally removed in 0.191. + +## MySQL, Redis, Kafka, and MongoDB + +- Fix failure when querying `information_schema.columns` when there is no equality predicate on `table_name`. diff --git a/430/_sources/release/release-0.196.md.txt b/430/_sources/release/release-0.196.md.txt new file mode 100644 index 000000000..933acb0fc --- /dev/null +++ b/430/_sources/release/release-0.196.md.txt @@ -0,0 +1,47 @@ +# Release 0.196 + +## General + +- Fix behavior of `JOIN ... USING` to conform to standard SQL semantics. + The old behavior can be restored by setting the `deprecated.legacy-join-using` + configuration option or the `legacy_join_using` session property. +- Fix memory leak for queries with `ORDER BY`. +- Fix tracking of query peak memory usage. +- Fix skew in dynamic writer scaling by eagerly freeing memory in the source output + buffers. This can be disabled by setting `exchange.acknowledge-pages=false`. +- Fix planning failure for lambda with capture in rare cases. +- Fix decimal precision of `round(x, d)` when `x` is a `DECIMAL`. +- Fix returned value from `round(x, d)` when `x` is a `DECIMAL` with + scale `0` and `d` is a negative integer. Previously, no rounding was done + in this case. +- Improve performance of the {func}`array_join` function. +- Improve performance of the {func}`ST_Envelope` function. +- Optimize {func}`min_by` and {func}`max_by` by avoiding unnecessary object + creation in order to reduce GC overhead. +- Show join partitioning explicitly in `EXPLAIN`. +- Add {func}`is_json_scalar` function. +- Add {func}`regexp_replace` function variant that executes a lambda for + each replacement. + +## Security + +- Add rules to the `file` {doc}`/security/built-in-system-access-control` + to enforce a specific matching between authentication credentials and a + executing username. + +## Hive + +- Fix a correctness issue where non-null values can be treated as null values + when writing dictionary-encoded strings to ORC files with the new ORC writer. +- Fix invalid failure due to string statistics mismatch while validating ORC files + after they have been written with the new ORC writer. This happens when + the written strings contain invalid UTF-8 code points. +- Add support for reading array, map, or row type columns from partitions + where the partition schema is different from the table schema. This can + occur when the table schema was updated after the partition was created. + The changed column types must be compatible. For rows types, trailing fields + may be added or dropped, but the corresponding fields (by ordinal) + must have the same name. +- Add `hive.non-managed-table-creates-enabled` configuration option + that controls whether or not users may create non-managed (external) tables. + The default value is `true`. diff --git a/430/_sources/release/release-0.197.md.txt b/430/_sources/release/release-0.197.md.txt new file mode 100644 index 000000000..df503a657 --- /dev/null +++ b/430/_sources/release/release-0.197.md.txt @@ -0,0 +1,58 @@ +# Release 0.197 + +## General + +- Fix query scheduling hang when the `concurrent_lifespans_per_task` session property is set. +- Fix failure when a query contains a `TIMESTAMP` literal corresponding to a local time that + does not occur in the default time zone of the Presto JVM. For example, if Presto was running + in a CET zone (e.g., `Europe/Brussels`) and the client session was in UTC, an expression + such as `TIMESTAMP '2017-03-26 02:10:00'` would cause a failure. +- Extend predicate inference and pushdown for queries using a ` IN ` predicate. +- Support predicate pushdown for the ` IN ` predicate + where values in the `values list` require casting to match the type of `column`. +- Optimize {func}`min` and {func}`max` to avoid unnecessary object creation in order to reduce GC overhead. +- Optimize the performance of {func}`ST_XMin`, {func}`ST_XMax`, {func}`ST_YMin`, and {func}`ST_YMax`. +- Add `DATE` variant for {func}`sequence` function. +- Add {func}`ST_IsSimple` geospatial function. +- Add support for broadcast spatial joins. + +## Resource groups + +- Change configuration check for weights in resource group policy to validate that + either all of the sub-groups or none of the sub-groups have a scheduling weight configured. +- Add support for named variables in source and user regular expressions that can be + used to parameterize resource group names. +- Add support for optional fields in DB resource group exact match selectors. + +## Hive + +- Fix reading of Hive partition statistics with unset fields. Previously, unset fields + were incorrectly interpreted as having a value of zero. +- Fix integer overflow when writing a single file greater than 2GB with optimized ORC writer. +- Fix system memory accounting to include stripe statistics size and + writer validation size for the optimized ORC writer. +- Dynamically allocate the compression buffer for the optimized ORC writer + to avoid unnecessary memory allocation. Add config property + `hive.orc.writer.max-compression-buffer-size` to limit the maximum size of the buffer. +- Add session property `orc_optimized_writer_max_stripe_size` to tune the + maximum stipe size for the optimized ORC writer. +- Add session property `orc_string_statistics_limit` to drop the string + statistics when writing ORC files if they exceed the limit. +- Use the view owner returned from the metastore at the time of the query rather than + always using the user who created the view. This allows changing the owner of a view. + +## CLI + +- Fix hang when CLI fails to communicate with Presto server. + +## SPI + +- Include connector session properties for the connector metadata calls made + when running `SHOW` statements or querying `information_schema`. +- Add count and time of full GC that occurred while query was running to `QueryCompletedEvent`. +- Change the `ResourceGroupManager` interface to include a `match()` method and + remove the `getSelectors()` method and the `ResourceGroupSelector` interface. +- Rename the existing `SelectionContext` class to be `SelectionCriteria` and + create a new `SelectionContext` class that is returned from the `match()` method + and contains the resource group ID and a manager-defined context field. +- Use the view owner from `ConnectorViewDefinition` when present. diff --git a/430/_sources/release/release-0.198.md.txt b/430/_sources/release/release-0.198.md.txt new file mode 100644 index 000000000..57f17b7f0 --- /dev/null +++ b/430/_sources/release/release-0.198.md.txt @@ -0,0 +1,63 @@ +# Release 0.198 + +## General + +- Perform semantic analysis before enqueuing queries. +- Add support for selective aggregates (`FILTER`) with `DISTINCT` argument qualifiers. +- Support `ESCAPE` for `LIKE` predicate in `SHOW SCHEMAS` and `SHOW TABLES` queries. +- Parse decimal literals (e.g. `42.0`) as `DECIMAL` by default. Previously, they were parsed as + `DOUBLE`. This behavior can be turned off via the `parse-decimal-literals-as-double` config option or + the `parse_decimal_literals_as_double` session property. +- Fix `current_date` failure when the session time zone has a "gap" at `1970-01-01 00:00:00`. + The time zone `America/Bahia_Banderas` is one such example. +- Add variant of {func}`sequence` function for `DATE` with an implicit one-day step increment. +- Increase the maximum number of arguments for the {func}`zip` function from 4 to 5. +- Add {func}`ST_IsValid`, {func}`geometry_invalid_reason`, {func}`simplify_geometry`, and + {func}`great_circle_distance` functions. +- Support {func}`min` and {func}`max` aggregation functions when the input type is unknown at query analysis time. + In particular, this allows using the functions with `NULL` literals. +- Add configuration property `task.max-local-exchange-buffer-size` for setting local exchange buffer size. +- Add trace token support to the scheduler and exchange HTTP clients. Each HTTP request sent + by the scheduler and exchange HTTP clients will have a "trace token" (a unique ID) in their + headers, which will be logged in the HTTP request logs. This information can be used to + correlate the requests and responses during debugging. +- Improve query performance when dynamic writer scaling is enabled. +- Improve performance of {func}`ST_Intersects`. +- Improve query latency when tables are known to be empty during query planning. +- Optimize {func}`array_agg` to avoid excessive object overhead and native memory usage with G1 GC. +- Improve performance for high-cardinality aggregations with `DISTINCT` argument qualifiers. This + is an experimental optimization that can be activated by disabling the `use_mark_distinct` session + property or the `optimizer.use-mark-distinct` config option. +- Improve parallelism of queries that have an empty grouping set. +- Improve performance of join queries involving the {func}`ST_Distance` function. + +## Resource groups + +- Query Queues have been removed. Resource Groups are always enabled. The + config property `experimental.resource-groups-enabled` has been removed. +- Change `WEIGHTED_FAIR` scheduling policy to select oldest eligible sub group + of groups where utilization and share are identical. + +## CLI + +- The `--enable-authentication` option has been removed. Kerberos authentication + is automatically enabled when `--krb5-remote-service-name` is specified. +- Kerberos authentication now requires HTTPS. + +## Hive + +- Add support for using [AWS Glue](https://aws.amazon.com/glue/) as the metastore. + Enable it by setting the `hive.metastore` config property to `glue`. +- Fix a bug in the ORC writer that will write incorrect data of type `VARCHAR` or `VARBINARY` + into files. + +## JMX + +- Add wildcard character `*` which allows querying several MBeans with a single query. + +## SPI + +- Add performance statistics to query plan in `QueryCompletedEvent`. +- Remove `Page.getBlocks()`. This call was rarely used and performed an expensive copy. + Instead, use `Page.getBlock(channel)` or the new helper `Page.appendColumn()`. +- Improve validation of `ArrayBlock`, `MapBlock`, and `RowBlock` during construction. diff --git a/430/_sources/release/release-0.199.md.txt b/430/_sources/release/release-0.199.md.txt new file mode 100644 index 000000000..09e366cce --- /dev/null +++ b/430/_sources/release/release-0.199.md.txt @@ -0,0 +1,78 @@ +# Release 0.199 + +## General + +- Allow users to create views for their own use when they do not have permission + to grant others access to the underlying tables or views. To enable this, + creation permission is now only checked at query time, not at creation time, + and the query time check is skipped if the user is the owner of the view. +- Add support for spatial left join. +- Add {func}`hmac_md5`, {func}`hmac_sha1`, {func}`hmac_sha256`, and {func}`hmac_sha512` functions. +- Add {func}`array_sort` function that takes a lambda as a comparator. +- Add {func}`line_locate_point` geospatial function. +- Add support for `ORDER BY` clause in aggregations for queries that use grouping sets. +- Add support for yielding when unspilling an aggregation. +- Expand grouped execution support to `GROUP BY` and `UNION ALL`, making it possible + to execute aggregations with less peak memory usage. +- Change the signature of `round(x, d)` and `truncate(x, d)` functions so that + `d` is of type `INTEGER`. Previously, `d` could be of type `BIGINT`. + This behavior can be restored with the `deprecated.legacy-round-n-bigint` config option + or the `legacy_round_n_bigint` session property. +- Accessing anonymous row fields via `.field0`, `.field1`, etc., is no longer allowed. + This behavior can be restored with the `deprecated.legacy-row-field-ordinal-access` + config option or the `legacy_row_field_ordinal_access` session property. +- Optimize the {func}`ST_Intersection` function for rectangles aligned with coordinate axes + (e.g., polygons produced by the {func}`ST_Envelope` and {func}`bing_tile_polygon` functions). +- Finish joins early when possible if one side has no rows. This happens for + either side of an inner join, for the left side of a left join, and for the + right side of a right join. +- Improve predicate evaluation performance during predicate pushdown in planning. +- Improve the performance of queries that use `LIKE` predicates on the columns of `information_schema` tables. +- Improve the performance of map-to-map cast. +- Improve the performance of {func}`ST_Touches`, {func}`ST_Within`, {func}`ST_Overlaps`, {func}`ST_Disjoint`, + and {func}`ST_Crosses` functions. +- Improve the serialization performance of geometry values. +- Improve the performance of functions that return maps. +- Improve the performance of joins and aggregations that include map columns. + +## Server RPM + +- Add support for installing on machines with OpenJDK. + +## Security + +- Add support for authentication with JWT access token. + +## JDBC driver + +- Make driver compatible with Java 9+. It previously failed with `IncompatibleClassChangeError`. + +## Hive + +- Fix ORC writer failure when writing `NULL` values into columns of type `ROW`, `MAP`, or `ARRAY`. +- Fix ORC writers incorrectly writing non-null values as `NULL` for all types. +- Support reading Hive partitions that have a different bucket count than the table, + as long as the ratio is a power of two (`1:2^n` or `2^n:1`). +- Add support for the `skip.header.line.count` table property. +- Prevent reading from tables with the `skip.footer.line.count` table property. +- Partitioned tables now have a hidden system table that contains the partition values. + A table named `example` will have a partitions table named `example$partitions`. + This provides the same functionality and data as `SHOW PARTITIONS`. +- Partition name listings, both via the `$partitions` table and using + `SHOW PARTITIONS`, are no longer subject to the limit defined by the + `hive.max-partitions-per-scan` config option. +- Allow marking partitions as offline via the `presto_offline` partition property. + +## Thrift connector + +- Most of the config property names are different due to replacing the + underlying Thrift client implementation. Please see {doc}`/connector/thrift` + for details on the new properties. + +## SPI + +- Allow connectors to provide system tables dynamically. +- Add `resourceGroupId` and `queryType` fields to `SessionConfigurationContext`. +- Simplify the constructor of `RowBlock`. +- `Block.writePositionTo()` now closes the current entry. +- Replace the `writeObject()` method in `BlockBuilder` with `appendStructure()`. diff --git a/430/_sources/release/release-0.200.md.txt b/430/_sources/release/release-0.200.md.txt new file mode 100644 index 000000000..3a0546be8 --- /dev/null +++ b/430/_sources/release/release-0.200.md.txt @@ -0,0 +1,31 @@ +# Release 0.200 + +## General + +- Disable early termination of inner or right joins when the right side + has zero rows. This optimization can cause indefinite query hangs + for queries that join against a small number of rows. + This regression was introduced in 0.199. +- Fix query execution failure for {func}`bing_tile_coordinates`. +- Remove the `log()` function. The arguments to the function were in the + wrong order according to the SQL standard, resulting in incorrect results + when queries were translated to or from other SQL implementations. The + equivalent to `log(x, b)` is `ln(x) / ln(b)`. The function can be + restored with the `deprecated.legacy-log-function` config option. +- Allow including a comment when adding a column to a table with `ALTER TABLE`. +- Add {func}`from_ieee754_32` and {func}`from_ieee754_64` functions. +- Add {func}`ST_GeometryType` geospatial function. + +## Hive + +- Fix reading min/max statistics for columns of `REAL` type in partitioned tables. +- Fix failure when reading Parquet files with optimized Parquet reader + related with the predicate push down for structural types. + Predicates on structural types are now ignored for Parquet files. +- Fix failure when reading ORC files that contain UTF-8 Bloom filter streams. + Such Bloom filters are now ignored. + +## MySQL + +- Avoid reading extra rows from MySQL at query completion. + This typically affects queries with a `LIMIT` clause. diff --git a/430/_sources/release/release-0.201.md.txt b/430/_sources/release/release-0.201.md.txt new file mode 100644 index 000000000..9b1a7e2e6 --- /dev/null +++ b/430/_sources/release/release-0.201.md.txt @@ -0,0 +1,41 @@ +# Release 0.201 + +## General + +- Change grouped aggregations to use `IS NOT DISTINCT FROM` semantics rather than equality + semantics. This fixes incorrect results and degraded performance when grouping on `NaN` + floating point values, and adds support for grouping on structural types that contain nulls. +- Fix planning error when column names are reused in `ORDER BY` query. +- System memory pool is now unused by default and it will eventually be removed completely. + All memory allocations will now be served from the general/user memory pool. The old behavior + can be restored with the `deprecated.legacy-system-pool-enabled` config option. +- Improve performance and memory usage for queries using {func}`row_number` followed by a + filter on the row numbers generated. +- Improve performance and memory usage for queries using `ORDER BY` followed by a `LIMIT`. +- Improve performance of queries that process structural types and contain joins, aggregations, + or table writes. +- Add session property `prefer-partial-aggregation` to allow users to disable partial + aggregations for queries that do not benefit. +- Add support for `current_user` (see {doc}`/functions/session`). + +## Security + +- Change rules in the {doc}`/security/built-in-system-access-control` for enforcing matches + between authentication credentials and a chosen username to allow more fine-grained + control and ability to define superuser-like credentials. + +## Hive + +- Replace ORC writer stripe minimum row configuration `hive.orc.writer.stripe-min-rows` + with stripe minimum data size `hive.orc.writer.stripe-min-size`. +- Change ORC writer validation configuration `hive.orc.writer.validate` to switch to a + sampling percentage `hive.orc.writer.validation-percentage`. +- Fix optimized ORC writer writing incorrect data of type `map` or `array`. +- Fix `SHOW PARTITIONS` and the `$partitions` table for tables that have null partition + values. +- Fix impersonation for the simple HDFS authentication to use login user rather than current + user. + +## SPI + +- Support resource group selection based on resource estimates. diff --git a/430/_sources/release/release-0.202.md.txt b/430/_sources/release/release-0.202.md.txt new file mode 100644 index 000000000..2621b4f22 --- /dev/null +++ b/430/_sources/release/release-0.202.md.txt @@ -0,0 +1,62 @@ +# Release 0.202 + +## General + +- Fix correctness issue for queries involving aggregations over the result of an outer join ({issue}`x10592`). +- Fix {func}`map` to raise an error on duplicate keys rather than silently producing a corrupted map. +- Fix {func}`map_from_entries` to raise an error when input array contains a `null` entry. +- Fix out-of-memory error for bucketed execution by scheduling new splits on the same worker as + the recently finished one. +- Fix query failure when performing a `GROUP BY` on `json` or `ipaddress` types. +- Fix correctness issue in {func}`line_locate_point`, {func}`ST_IsValid`, and {func}`geometry_invalid_reason` + functions to not return values outside of the expected range. +- Fix failure in {func}`geometry_to_bing_tiles` and {func}`ST_NumPoints` functions when + processing geometry collections. +- Fix query failure in aggregation spilling ({issue}`x10587`). +- Remove support for `SHOW PARTITIONS` statement. +- Improve support for correlated subqueries containing equality predicates. +- Improve performance of correlated `EXISTS` subqueries. +- Limit the number of grouping sets in a `GROUP BY` clause. + The default limit is `2048` and can be set via the `analyzer.max-grouping-sets` + configuration property or the `max_grouping_sets` session property. +- Allow coercion between row types regardless of field names. + Previously, a row type is coercible to another only if the field name in the source type + matches the target type, or when target type has anonymous field name. +- Increase default value for `experimental.filter-and-project-min-output-page-size` to `500kB`. +- Improve performance of equals operator on `array(bigint)` and `array(double)` types. +- Respect `X-Forwarded-Proto` header in client protocol responses. +- Add support for column-level access control. + Connectors have not yet been updated to take advantage of this support. +- Add support for correlated subqueries with correlated `OR` predicates. +- Add {func}`multimap_from_entries` function. +- Add {func}`bing_tiles_around`, {func}`ST_NumGeometries`, {func}`ST_GeometryN`, and {func}`ST_ConvexHull` geospatial functions. +- Add {func}`wilson_interval_lower` and {func}`wilson_interval_upper` functions. +- Add `IS DISTINCT FROM` for `json` and `ipaddress` type. + +## Hive + +- Fix optimized ORC writer encoding of `TIMESTAMP` before `1970-01-01`. Previously, the + written value was off by one second. +- Fix query failure when a Hive bucket has no splits. This commonly happens when a + predicate filters some buckets out entirely. +- Remove the `hive.bucket-writing` config property. +- Add support for creating and writing bucketed sorted tables. The list of + sorting columns may be specified using the `sorted_by` table property. + Writing to sorted tables can be disabled using the `hive.sorted-writing` + config property or the `sorted_writing_enabled` session property. The + maximum number of temporary files for can be controlled using the + `hive.max-sort-files-per-bucket` property. +- Collect and store basic table statistics (`rowCount`, `fileCount`, `rawDataSize`, + `totalSize`) when writing. +- Add `hive.orc.tiny-stripe-threshold` config property and `orc_tiny_stripe_threshold` + session property to control the stripe/file size threshold when ORC reader decides to + read multiple consecutive stripes or entire fires at once. Previously, this feature + piggybacks on other properties. + +## CLI + +- Add peak memory usage to `--debug` output. + +## SPI + +- Make `PageSorter` and `PageIndexer` supported interfaces. diff --git a/430/_sources/release/release-0.203.md.txt b/430/_sources/release/release-0.203.md.txt new file mode 100644 index 000000000..209fc57aa --- /dev/null +++ b/430/_sources/release/release-0.203.md.txt @@ -0,0 +1,49 @@ +# Release 0.203 + +## General + +- Fix spurious duplicate key errors from {func}`map`. +- Fix planning failure when a correlated subquery containing a `LIMIT` + clause is used within `EXISTS` ({issue}`x10696`). +- Fix out of memory error caused by missing pushback checks in data exchanges. +- Fix execution failure for queries containing a cross join when using bucketed execution. +- Fix execution failure for queries containing an aggregation function + with `DISTINCT` and a highly selective aggregation filter. + For example: `sum(DISTINCT x) FILTER (WHERE y = 0)` +- Fix quoting in error message for `SHOW PARTITIONS`. +- Eliminate redundant calls to check column access permissions. +- Improve query creation reliability by delaying query start until the client + acknowledges the query ID by fetching the first response link. This eliminates + timeouts during the initial request for queries that take a long time to analyze. +- Remove support for legacy `ORDER BY` semantics. +- Distinguish between inner and left spatial joins in explain plans. + +## Security + +- Fix sending authentication challenge when at least two of the + `KERBEROS`, `PASSWORD`, or `JWT` authentication types are configured. +- Allow using PEM encoded (PKCS #8) keystore and truststore with the HTTP server + and the HTTP client used for internal communication. This was already supported + for the CLI and JDBC driver. + +## Server RPM + +- Declare a dependency on `uuidgen`. The `uuidgen` program is required during + installation of the Presto server RPM package and lack of it resulted in an invalid + config file being generated during installation. + +## Hive connector + +- Fix complex type handling in the optimized Parquet reader. Previously, null values, + optional fields, and Parquet backward compatibility rules were not handled correctly. +- Fix an issue that could cause the optimized ORC writer to fail with a `LazyBlock` error. +- Improve error message for max open writers. + +## Thrift connector + +- Fix retry of requests when the remote Thrift server indicates that the + error is retryable. + +## Local file connector + +- Fix parsing of timestamps when the JVM time zone is UTC ({issue}`x9601`). diff --git a/430/_sources/release/release-0.204.md.txt b/430/_sources/release/release-0.204.md.txt new file mode 100644 index 000000000..3f891b2ba --- /dev/null +++ b/430/_sources/release/release-0.204.md.txt @@ -0,0 +1,45 @@ +# Release 0.204 + +## General + +- Use distributed join if one side is naturally partitioned on join keys. +- Improve performance of correlated subqueries when filters from outer query + can be propagated to the subquery. +- Improve performance for correlated subqueries that contain inequalities. +- Add support for all geometry types in {func}`ST_Area`. +- Add {func}`ST_EnvelopeAsPts` function. +- Add {func}`to_big_endian_32` and {func}`from_big_endian_32` functions. +- Add cast between `VARBINARY` type and `IPADDRESS` type. +- Make {func}`lpad` and {func}`rpad` functions support `VARBINARY` in addition to `VARCHAR`. +- Allow using arrays of mismatched lengths with {func}`zip_with`. + The missing positions are filled with `NULL`. +- Track execution statistics of `AddExchanges` and `PredicatePushdown` optimizer rules. + +## Event listener + +- Add resource estimates to query events. + +## Web UI + +- Fix kill query button. +- Display resource estimates in Web UI query details page. + +## Resource group + +- Fix unnecessary queuing in deployments where no resource group configuration was specified. + +## Hive connector + +- Fix over-estimation of memory usage for scan operators when reading ORC files. +- Fix memory accounting for sort buffer used for writing sorted bucketed tables. +- Disallow creating tables with unsupported partition types. +- Support overwriting partitions for insert queries. This behavior is controlled + by session property `insert_existing_partitions_behavior`. +- Prevent the optimized ORC writer from writing excessively large stripes for + highly compressed, dictionary encoded columns. +- Enable optimized Parquet reader and predicate pushdown by default. + +## Cassandra connector + +- Add support for reading from materialized views. +- Optimize partition list retrieval for Cassandra 2.2+. diff --git a/430/_sources/release/release-0.205.md.txt b/430/_sources/release/release-0.205.md.txt new file mode 100644 index 000000000..0e51aeca5 --- /dev/null +++ b/430/_sources/release/release-0.205.md.txt @@ -0,0 +1,80 @@ +# Release 0.205 + +## General + +- Fix parsing of row types where the field types contain spaces. + Previously, row expressions that included spaces would fail to parse. + For example: `cast(row(timestamp '2018-06-01') AS row(timestamp with time zone))`. +- Fix distributed planning failure for complex queries when using bucketed execution. +- Fix {func}`ST_ExteriorRing` to only accept polygons. + Previously, it erroneously accepted other geometries. +- Add the `task.min-drivers-per-task` and `task.max-drivers-per-task` config options. + The former specifies the guaranteed minimum number of drivers a task will run concurrently + given that it has enough work to do. The latter specifies the maximum number of drivers + a task can run concurrently. +- Add the `concurrent-lifespans-per-task` config property to control the default value + of the `concurrent_lifespans_per_task` session property. +- Add the `query_max_total_memory` session property and the `query.max-total-memory` + config property. Queries will be aborted after their total (user + system) memory + reservation exceeds this threshold. +- Improve stats calculation for outer joins and correlated subqueries. +- Reduce memory usage when a `Block` contains all null or all non-null values. +- Change the internal hash function used in `approx_distinct`. The result of `approx_distinct` + may change in this version compared to the previous version for the same set of values. However, + the standard error of the results should still be within the configured bounds. +- Improve efficiency and reduce memory usage for scalar correlated subqueries with aggregations. +- Remove the legacy local scheduler and associated configuration properties, + `task.legacy-scheduling-behavior` and `task.level-absolute-priority`. +- Do not allow using the `FILTER` clause for the `COALESCE`, `IF`, or `NULLIF` functions. + The syntax was previously allowed but was otherwise ignored. + +## Security + +- Remove unnecessary check for `SELECT` privileges for `DELETE` queries. + Previously, `DELETE` queries could fail if the user only has `DELETE` + privileges but not `SELECT` privileges. + This only affected connectors that implement `checkCanSelectFromColumns()`. +- Add a check that the view owner has permission to create the view when + running `SELECT` queries against a view. This only affected connectors that + implement `checkCanCreateViewWithSelectFromColumns()`. +- Change `DELETE FROM
WHERE ` to check that the user has `SELECT` + privileges on the objects referenced by the `WHERE` condition as is required by the SQL standard. +- Improve the error message when access is denied when selecting from a view due to the + view owner having insufficient permissions to create the view. + +## JDBC driver + +- Add support for prepared statements. +- Add partial query cancellation via `partialCancel()` on `PrestoStatement`. +- Use `VARCHAR` rather than `LONGNVARCHAR` for the Presto `varchar` type. +- Use `VARBINARY` rather than `LONGVARBINARY` for the Presto `varbinary` type. + +## Hive connector + +- Improve the performance of `INSERT` queries when all partition column values are constants. +- Improve stripe size estimation for the optimized ORC writer. + This reduces the number of cases where tiny ORC stripes will be written. +- Respect the `skip.footer.line.count` Hive table property. + +## CLI + +- Prevent the CLI from crashing when running on certain 256 color terminals. + +## SPI + +- Add a context parameter to the `create()` method in `SessionPropertyConfigurationManagerFactory`. +- Disallow non-static methods to be annotated with `@ScalarFunction`. Non-static SQL function + implementations must now be declared in a class annotated with `@ScalarFunction`. +- Disallow having multiple public constructors in `@ScalarFunction` classes. All non-static + implementations of SQL functions will now be associated with a single constructor. + This improves support for providing specialized implementations of SQL functions with generic arguments. +- Deprecate `checkCanSelectFromTable/checkCanSelectFromView` and + `checkCanCreateViewWithSelectFromTable/checkCanCreateViewWithSelectFromView` in `ConnectorAccessControl` + and `SystemAccessControl`. `checkCanSelectFromColumns` and `checkCanCreateViewWithSelectFromColumns` + should be used instead. + +:::{note} +These are backwards incompatible changes with the previous SPI. +If you have written a plugin using these features, you will need +to update your code before deploying this release. +::: diff --git a/430/_sources/release/release-0.206.md.txt b/430/_sources/release/release-0.206.md.txt new file mode 100644 index 000000000..053bc8692 --- /dev/null +++ b/430/_sources/release/release-0.206.md.txt @@ -0,0 +1,46 @@ +# Release 0.206 + +## General + +- Fix execution failure for certain queries containing a join followed by an aggregation + when `dictionary_aggregation` is enabled. +- Fix planning failure when a query contains a `GROUP BY`, but the cardinality of the + grouping columns is one. For example: `SELECT c1, sum(c2) FROM t WHERE c1 = 'foo' GROUP BY c1` +- Fix high memory pressure on the coordinator during the execution of queries + using bucketed execution. +- Add {func}`ST_Union`, {func}`ST_Geometries`, {func}`ST_PointN`, {func}`ST_InteriorRings`, + and {func}`ST_InteriorRingN` geospatial functions. +- Add {func}`split_to_multimap` function. +- Expand the {func}`approx_distinct` function to support the following types: + `INTEGER`, `SMALLINT`, `TINYINT`, `DECIMAL`, `REAL`, `DATE`, + `TIMESTAMP`, `TIMESTAMP WITH TIME ZONE`, `TIME`, `TIME WITH TIME ZONE`, `IPADDRESS`. +- Add a resource group ID column to the `system.runtime.queries` table. +- Add support for executing `ORDER BY` without `LIMIT` in a distributed manner. + This can be disabled with the `distributed-sort` configuration property + or the `distributed_sort` session property. +- Add implicit coercion from `VARCHAR(n)` to `CHAR(n)`, and remove implicit coercion the other way around. + As a result, comparing a `CHAR` with a `VARCHAR` will now follow + trailing space insensitive `CHAR` comparison semantics. +- Improve query cost estimation by only including non-null rows when computing average row size. +- Improve query cost estimation to better account for overhead when estimating data size. +- Add new semantics that conform to the SQL standard for temporal types. + It affects the `TIMESTAMP` (aka `TIMESTAMP WITHOUT TIME ZONE`) type, + `TIME` (aka `TIME WITHOUT TIME ZONE`) type, and `TIME WITH TIME ZONE` type. + The legacy behavior remains default. + At this time, it is not recommended to enable the new semantics. + For any connector that supports temporal types, code changes are required before the connector + can work correctly with the new semantics. No connectors have been updated yet. + In addition, the new semantics are not yet stable as more breaking changes are planned, + particularly around the `TIME WITH TIME ZONE` type. + +## JDBC driver + +- Add `applicationNamePrefix` parameter, which is combined with + the `ApplicationName` property to construct the client source name. + +## Hive connector + +- Reduce ORC reader memory usage by reducing unnecessarily large internal buffers. +- Support reading from tables with `skip.footer.line.count` and `skip.header.line.count` + when using HDFS authentication with Kerberos. +- Add support for case-insensitive column lookup for Parquet readers. diff --git a/430/_sources/release/release-0.207.md.txt b/430/_sources/release/release-0.207.md.txt new file mode 100644 index 000000000..9866266dc --- /dev/null +++ b/430/_sources/release/release-0.207.md.txt @@ -0,0 +1,60 @@ +# Release 0.207 + +## General + +- Fix a planning issue for queries where correlated references were used in `VALUES`. +- Remove support for legacy `JOIN ... USING` behavior. +- Change behavior for unnesting an array of `row` type to produce multiple columns. +- Deprecate the `reorder_joins` session property and the `reorder-joins` + configuration property. They are replaced by the `join_reordering_strategy` + session property and the `optimizer.join-reordering-strategy` configuration + property. `NONE` maintains the order of the joins as written and is equivalent + to `reorder_joins=false`. `ELIMINATE_CROSS_JOINS` will eliminate any + unnecessary cross joins from the plan and is equivalent to `reorder_joins=true`. + `AUTOMATIC` will use the new cost-based optimizer to select the best join order. + To simplify migration, setting the `reorder_joins` session property overrides the + new session and configuration properties. +- Deprecate the `distributed_joins` session property and the + `distributed-joins-enabled` configuration property. They are replaced by the + `join_distribution_type` session property and the `join-distribution-type` + configuration property. `PARTITIONED` turns on hash partitioned joins and + is equivalent to `distributed_joins-enabled=true`. `BROADCAST` changes the + join strategy to broadcast and is equivalent to `distributed_joins-enabled=false`. + `AUTOMATIC` will use the new cost-based optimizer to select the best join + strategy. If no statistics are available, `AUTOMATIC` is the same as + `REPARTITIONED`. To simplify migration, setting the `distributed_joins` + session property overrides the new session and configuration properties. +- Add support for column properties. +- Add `optimizer.max-reordered-joins` configuration property to set the maximum number of joins that + can be reordered at once using cost-based join reordering. +- Add support for `char` type to {func}`approx_distinct`. + +## Security + +- Fail on startup when configuration for file based system access control is invalid. +- Add support for securing communication between cluster nodes with Kerberos authentication. + +## Web UI + +- Add peak total (user + system) memory to query details UI. + +## Hive connector + +- Fix handling of `VARCHAR(length)` type in the optimized Parquet reader. Previously, predicate pushdown + failed with `Mismatched Domain types: varchar(length) vs varchar`. +- Fail on startup when configuration for file based access control is invalid. +- Add support for HDFS wire encryption. +- Allow ORC files to have struct columns with missing fields. This allows the table schema to be changed + without rewriting the ORC files. +- Change collector for columns statistics to only consider a sample of partitions. The sample size can be + changed by setting the `hive.partition-statistics-sample-size` property. + +## Memory connector + +- Add support for dropping schemas. + +## SPI + +- Remove deprecated table/view-level access control methods. +- Change predicate in constraint for accessing table layout to be optional. +- Change schema name in `ConnectorMetadata` to be optional rather than nullable. diff --git a/430/_sources/release/release-0.208.md.txt b/430/_sources/release/release-0.208.md.txt new file mode 100644 index 000000000..dafaf41c4 --- /dev/null +++ b/430/_sources/release/release-0.208.md.txt @@ -0,0 +1,45 @@ +# Release 0.208 + +:::{warning} +This release has the potential for data loss in the Hive connector +when writing bucketed sorted tables. +::: + +## General + +- Fix an issue with memory accounting that would lead to garbage collection pauses + and out of memory exceptions. +- Fix an issue that produces incorrect results when `push_aggregation_through_join` + is enabled ({issue}`x10724`). +- Preserve field names when unnesting columns of type `ROW`. +- Make the cluster out of memory killer more resilient to memory accounting leaks. + Previously, memory accounting leaks on the workers could effectively disable + the out of memory killer. +- Improve planning time for queries over tables with high column count. +- Add a limit on the number of stages in a query. The default is `100` and can + be changed with the `query.max-stage-count` configuration property and the + `query_max_stage_count` session property. +- Add {func}`spooky_hash_v2_32` and {func}`spooky_hash_v2_64` functions. +- Add a cluster memory leak detector that logs queries that have possibly accounted for + memory usage incorrectly on workers. This is a tool to for debugging internal errors. +- Add support for correlated subqueries requiring coercions. +- Add experimental support for running on Linux ppc64le. + +## CLI + +- Fix creation of the history file when it does not exist. +- Add `PRESTO_HISTORY_FILE` environment variable to override location of history file. + +## Hive connector + +- Remove size limit for writing bucketed sorted tables. +- Support writer scaling for Parquet. +- Improve stripe size estimation for the optimized ORC writer. This reduces the + number of cases where tiny ORC stripes will be written. +- Provide the actual size of CHAR, VARCHAR, and VARBINARY columns to the cost based optimizer. +- Collect column level statistics when writing tables. This is disabled by default, + and can be enabled by setting the `hive.collect-column-statistics-on-write` property. + +## Thrift connector + +- Include error message from remote server in query failure message. diff --git a/430/_sources/release/release-0.209.md.txt b/430/_sources/release/release-0.209.md.txt new file mode 100644 index 000000000..6d1c79d95 --- /dev/null +++ b/430/_sources/release/release-0.209.md.txt @@ -0,0 +1,68 @@ +# Release 0.209 + +## General + +- Fix incorrect predicate pushdown when grouping sets contain the empty grouping set ({issue}`x11296`). +- Fix `X-Forwarded-Proto` header handling for requests to the `/` path ({issue}`x11168`). +- Fix a regression that results in execution failure when at least one + of the arguments to {func}`min_by` or {func}`max_by` is a constant `NULL`. +- Fix failure when some buckets are completely filtered out during bucket-by-bucket execution. +- Fix execution failure of queries due to a planning deficiency involving + complex nested joins where a join that is not eligible for bucket-by-bucket + execution feeds into the build side of a join that is eligible. +- Improve numerical stability for {func}`corr`, {func}`covar_samp`, + {func}`regr_intercept`, and {func}`regr_slope`. +- Do not include column aliases when checking column access permissions. +- Eliminate unnecessary data redistribution for scalar correlated subqueries. +- Remove table scan original constraint information from `EXPLAIN` output. +- Introduce distinct error codes for global and per-node memory limit errors. +- Include statistics and cost estimates for `EXPLAIN (TYPE DISTRIBUTED)` and `EXPLAIN ANALYZE`. +- Support equality checks for `ARRAY`, `MAP`, and `ROW` values containing nulls. +- Improve statistics estimation and fix potential negative nulls fraction + estimates for expressions that include `NOT` or `OR`. +- Completely remove the `SHOW PARTITIONS` statement. +- Add {func}`bing_tiles_around` variant that takes a radius. +- Add the {func}`convex_hull_agg` and {func}`geometry_union_agg` geospatial aggregation functions. +- Add `(TYPE IO, FORMAT JSON)` option for {doc}`/sql/explain` that shows + input tables with constraints and the output table in JSON format. +- Add {doc}`/connector/kudu`. +- Raise required Java version to 8u151. This avoids correctness issues for + map to map cast when running under some earlier JVM versions, including 8u92. + +## Web UI + +- Fix the kill query button on the live plan and stage performance pages. + +## CLI + +- Prevent spurious *"No route to host"* errors on macOS when using IPv6. + +## JDBC driver + +- Prevent spurious *"No route to host"* errors on macOS when using IPv6. + +## Hive connector + +- Fix data loss when writing bucketed sorted tables. Partitions would + be missing arbitrary rows if any of the temporary files for a bucket + had the same size. The `numRows` partition property contained the + correct number of rows and can be used to detect if this occurred. +- Fix cleanup of temporary files when writing bucketed sorted tables. +- Allow creating schemas when using `file` based security. +- Reduce the number of cases where tiny ORC stripes will be written when + some columns are highly dictionary compressed. +- Improve memory accounting when reading ORC files. Previously, buffer + memory and object overhead was not tracked for stream readers. +- ORC struct columns are now mapped by name rather than ordinal. + This correctly handles missing or extra struct fields in the ORC file. +- Add procedure `system.create_empty_partition()` for creating empty partitions. + +## Kafka connector + +- Support Avro formatted Kafka messages. +- Support backward compatible Avro schema evolution. + +## SPI + +- Allow using `Object` as a parameter type or return type for SQL + functions when the corresponding SQL type is an unbounded generic. diff --git a/430/_sources/release/release-0.210.md.txt b/430/_sources/release/release-0.210.md.txt new file mode 100644 index 000000000..d93a10d9c --- /dev/null +++ b/430/_sources/release/release-0.210.md.txt @@ -0,0 +1,44 @@ +# Release 0.210 + +## General + +- Fix planning failure when aliasing columns of tables containing hidden + columns ({issue}`x11385`). +- Fix correctness issue when `GROUP BY DISTINCT` terms contain references to + the same column using different syntactic forms ({issue}`x11120`). +- Fix failures when querying `information_schema` tables using capitalized names. +- Improve performance when converting between `ROW` types. +- Remove user CPU time tracking as introduces non-trivial overhead. +- Select join distribution type automatically for queries involving outer joins. + +## Hive connector + +- Fix a security bug introduced in 0.209 when using `hive.security=file`, + which would allow any user to create, drop, or rename schemas. +- Prevent ORC writer from writing stripes larger than the max configured size + when converting a highly dictionary compressed column to direct encoding. +- Support creating Avro tables with a custom schema using the `avro_schema_url` + table property. +- Support backward compatible Avro schema evolution. +- Support cross-realm Kerberos authentication for HDFS and Hive Metastore. + +## JDBC driver + +- Deallocate prepared statement when `PreparedStatement` is closed. Previously, + `Connection` became unusable after many prepared statements were created. +- Remove `getUserTimeMillis()` from `QueryStats` and `StageStats`. + +## SPI + +- `SystemAccessControl.checkCanSetUser()` now takes an `Optional` + rather than a nullable `Principal`. +- Rename `connectorId` to `catalogName` in `ConnectorFactory`, + `QueryInputMetadata`, and `QueryOutputMetadata`. +- Pass `ConnectorTransactionHandle` to `ConnectorAccessControl.checkCanSetCatalogSessionProperty()`. +- Remove `getUserTime()` from `SplitStatistics` (referenced in `SplitCompletedEvent`). + +:::{note} +These are backwards incompatible changes with the previous SPI. +If you have written a plugin, you will need to update your code +before deploying this release. +::: diff --git a/430/_sources/release/release-0.211.md.txt b/430/_sources/release/release-0.211.md.txt new file mode 100644 index 000000000..f0ca6e364 --- /dev/null +++ b/430/_sources/release/release-0.211.md.txt @@ -0,0 +1,50 @@ +# Release 0.211 + +## General + +- Fix missing final query plan in `QueryCompletedEvent`. Statistics and cost estimates + are removed from the plan text because they may not be available during event generation. +- Update the default value of the `http-server.https.excluded-cipher` config + property to exclude cipher suites with a weak hash algorithm or without forward secrecy. + Specifically, this means all ciphers that use the RSA key exchange are excluded by default. + Consequently, TLS 1.0 or TLS 1.1 are no longer supported with the default configuration. + The `http-server.https.excluded-cipher` config property can be set to empty string + to restore the old behavior. +- Add {func}`ST_GeomFromBinary` and {func}`ST_AsBinary` functions that convert + geometries to and from Well-Known Binary format. +- Remove the `verbose_stats` session property, and rename the `task.verbose-stats` + configuration property to `task.per-operator-cpu-timer-enabled`. +- Improve query planning performance for queries containing multiple joins + and a large number of columns ({issue}`x11196`). +- Add built-in {doc}`file based property manager ` + to automate the setting of session properties based on query characteristics. +- Allow running on a JVM from any vendor that meets the functional requirements. + +## Hive connector + +- Fix regression in 0.210 that causes query failure when writing ORC or DWRF files + that occurs for specific patterns of input data. When the writer attempts to give up + using dictionary encoding for a column that is highly compressed, the process of + transitioning to use direct encoding instead can fail. +- Fix coordinator OOM when a query scans many partitions of a Hive table ({issue}`x11322`). +- Improve readability of columns, partitioning, and transactions in explain plains. + +## Thrift connector + +- Fix lack of retry for network errors while sending requests. + +## Resource group + +- Add documentation for new resource group scheduling policies. +- Remove running and queue time limits from resource group configuration. + Legacy behavior can be replicated by using the + {doc}`file based property manager ` + to set session properties. + +## SPI + +- Clarify semantics of `predicate` in `ConnectorTableLayout`. +- Reduce flexibility of `unenforcedConstraint` that a connector can return in `getTableLayouts`. + For each column in the predicate, the connector must enforce the entire domain or none. +- Make the null vector in `ArrayBlock`, `MapBlock`, and `RowBlock` optional. + When it is not present, all entries in the `Block` are non-null. diff --git a/430/_sources/release/release-0.212.md.txt b/430/_sources/release/release-0.212.md.txt new file mode 100644 index 000000000..4766b4098 --- /dev/null +++ b/430/_sources/release/release-0.212.md.txt @@ -0,0 +1,31 @@ +# Release 0.212 + +## General + +- Fix query failures when the {func}`ST_GeomFromBinary` function is run on multiple rows. +- Fix memory accounting for the build side of broadcast joins. +- Fix occasional query failures when running `EXPLAIN ANALYZE`. +- Enhance {func}`ST_ConvexHull` and {func}`convex_hull_agg` functions to support geometry collections. +- Improve performance for some queries using `DISTINCT`. +- Improve performance for some queries that perform filtered global aggregations. +- Remove `round(x, d)` and `truncate(x, d)` functions where `d` is a `BIGINT` ({issue}`x11462`). +- Add {func}`ST_LineString` function to form a `LineString` from an array of points. + +## Hive connector + +- Prevent ORC writer from writing stripes larger than the max configured size for some rare data + patterns ({issue}`x11526`). +- Restrict the maximum line length for text files. The default limit of 100MB can be changed + using the `hive.text.max-line-length` configuration property. +- Add sanity checks that fail queries if statistics read from the metastore are corrupt. Corrupt + statistics can be ignored by setting the `hive.ignore-corrupted-statistics` + configuration property or the `ignore_corrupted_statistics` session property. + +## Thrift connector + +- Fix retry for network errors that occur while sending a Thrift request. +- Remove failed connections from connection pool. + +## Verifier + +- Record the query ID of the test query regardless of query outcome. diff --git a/430/_sources/release/release-0.213.md.txt b/430/_sources/release/release-0.213.md.txt new file mode 100644 index 000000000..b0c701411 --- /dev/null +++ b/430/_sources/release/release-0.213.md.txt @@ -0,0 +1,95 @@ +# Release 0.213 + +## General + +- Fix split scheduling backpressure when plan contains colocated join. Previously, splits + for the second and subsequent scan nodes (in scheduling order) were scheduled continuously + until completion, rather than pausing due to sufficient pending splits. +- Fix query execution failure or indefinite hang during grouped execution when all splits + for any lifespan are completely filtered out. +- Fix grouped execution to respect the configured concurrent lifespans per task. + Previously, it always used a single lifespan per task. +- Fix execution failure when using grouped execution with right or full outer joins + where the right side is not partitioned on the join key. +- Fix a scenario where too many rows are returned to clients in a single response. +- Do not allow setting invalid property values with {doc}`/sql/set-session`. +- Disable stats calculator by default as it can cause a planning failure for + certain complex queries. It can be enabled with the `experimental.enable-stats-calculator` + configuration property or the `enable_stats_calculator` session property. +- Avoid making guesses when estimating filters for joins. Previously, if nothing + was known about the filter, a `0.9` coefficient was applied as a filter factor. + Now, if nothing is known about a filter, the estimate will be unknown. A `0.9` + coefficient will be applied for all additional conjuncts if at least a single + conjunct can be reasonably estimated. +- Improve inference of predicates for inner joins. +- Improve `EXPLAIN ANALYZE` output by adding CPU time and enhancing accuracy of CPU fraction. +- Include stats and cost estimates in textual plans created on query completion. +- Enhance `SHOW STATS` to support `IN` and `BETWEEN` predicates in the + `WHERE` condition of the `SELECT` clause. +- Remove transaction from explain plan for indexes joins. +- Add `max_drivers_per_task` session property, allowing users to limit concurrency by + specifying a number lower than the system configured maximum. This can cause the + query to run slower and consume less resources. +- Add `join-max-broadcast-table-size` configuration property and + `join_max_broadcast_table_size` session property to control the maximum estimated size + of a table that can be broadcast when using `AUTOMATIC` join distribution type ({issue}`x11667`). +- Add experimental config option `experimental.reserved-pool-enabled` to disable the reserved memory pool. +- Add `targetResultSize` query parameter to `/v1/statement` endpoint to control response data size. + +## Geospatial + +- Fix {func}`ST_Distance` function to return `NULL` if any of the inputs is an + empty geometry as required by the SQL/MM specification. +- Add {func}`ST_MultiPoint` function to construct multi-point geometry from an array of points. +- Add {func}`geometry_union` function to efficiently union arrays of geometries. +- Add support for distributed spatial joins ({issue}`x11072`). + +## Server RPM + +- Allow running on a JVM from any vendor. + +## Web UI + +- Remove legacy plan UI. +- Add support for filtering queries by all error categories. +- Add dialog to show errors refreshing data from coordinator. +- Change worker thread list to not show thread stacks by default to improve page peformance. + +## Hive connector + +- Fix LZO and LZOP decompression to work with certain data compressed by Hadoop. +- Fix ORC writer validation percentage so that zero does not result in 100% validation. +- Fix potential out-of-bounds read for ZSTD on corrupted input. +- Stop assuming no distinct values when column null fraction statistic is less than `1.0`. +- Treat `-1` as an absent null count for compatibility with statistics written by + [Impala](https://issues.apache.org/jira/browse/IMPALA-7497). +- Preserve original exception for metastore network errors. +- Preserve exceptions from Avro deserializer +- Categorize text line length exceeded error. +- Remove the old Parquet reader. The `hive.parquet-optimized-reader.enabled` + configuration property and `parquet_optimized_reader_enabled` session property + no longer exist. +- Remove the `hive.parquet-predicate-pushdown.enabled` configuration property + and `parquet_predicate_pushdown_enabled` session property. + Pushdown is always enabled now in the Parquet reader. +- Enable optimized ORC writer by default. It can be disabled using the + `hive.orc.optimized-writer.enabled` configuration property or the + `orc_optimized_writer_enabled` session property. +- Use ORC file format as the default for new tables or partitions. +- Add support for Avro tables where the Avro schema URL is an HDFS location. +- Add `hive.parquet.writer.block-size` and `hive.parquet.writer.page-size` + configuration properties and `parquet_writer_block_size` and + `parquet_writer_page_size` session properties for tuning Parquet writer options. + +## Memory connector + +- Improve table data size accounting. + +## Thrift connector + +- Include constraint in explain plan for index joins. +- Improve readability of columns, tables, layouts, and indexes in explain plans. + +## Verifier + +- Rewrite queries in parallel when shadowing writes. diff --git a/430/_sources/release/release-0.214.md.txt b/430/_sources/release/release-0.214.md.txt new file mode 100644 index 000000000..cba298efe --- /dev/null +++ b/430/_sources/release/release-0.214.md.txt @@ -0,0 +1,56 @@ +# Release 0.214 + +## General + +- Fix history leak in coordinator for failed or canceled queries. +- Fix memory leak related to query tracking in coordinator that was introduced + in {doc}`/release/release-0.213`. +- Fix planning failures when lambdas are used in join filter expression. +- Fix responses to client for certain types of errors that are encountered + during query creation. +- Improve error message when an invalid comparator is provided to the + {func}`array_sort` function. +- Improve performance of lookup operations on map data types. +- Improve planning and query performance for queries with `TINYINT`, + `SMALLINT` and `VARBINARY` literals. +- Fix issue where queries containing distributed `ORDER BY` and aggregation + could sometimes fail to make progress when data was spilled. +- Make top N row number optimization work in some cases when columns are pruned. +- Add session property `optimize-top-n-row-number` and configuration property + `optimizer.optimize-top-n-row-number` to toggle the top N row number + optimization. +- Add {func}`ngrams` function to generate N-grams from an array. +- Add {ref}`qdigest ` type and associated {doc}`/functions/qdigest`. +- Add functionality to delay query execution until a minimum number of workers + nodes are available. The minimum number of workers can be set with the + `query-manager.required-workers` configuration property, and the max wait + time with the `query-manager.required-workers-max-wait` configuration property. +- Remove experimental pre-allocated memory system, and the related configuration + property `experimental.preallocate-memory-threshold`. + +## Security + +- Add functionality to refresh the configuration of file-based access controllers. + The refresh interval can be set using the `security.refresh-period` + configuration property. + +## JDBC driver + +- Clear update count after calling `Statement.getMoreResults()`. + +## Web UI + +- Show query warnings on the query detail page. +- Allow selecting non-default sort orders in query list view. + +## Hive connector + +- Prevent ORC writer from writing stripes larger than the maximum configured size. +- Add `hive.s3.upload-acl-type` configuration property to specify the type of + ACL to use while uploading files to S3. +- Add Hive metastore API recording tool for remote debugging purposes. +- Add support for retrying on metastore connection errors. + +## Verifier + +- Handle SQL execution timeouts while rewriting queries. diff --git a/430/_sources/release/release-0.215.md.txt b/430/_sources/release/release-0.215.md.txt new file mode 100644 index 000000000..9e138c73b --- /dev/null +++ b/430/_sources/release/release-0.215.md.txt @@ -0,0 +1,52 @@ +# Release 0.215 + +## General + +- Fix regression in 0.214 that could cause queries to produce incorrect results for queries + using map types. +- Fix reporting of the processed input data for source stages in `EXPLAIN ANALYZE`. +- Fail queries that use non-leaf resource groups. Previously, they would remain queued forever. +- Improve CPU usage for specific queries ({issue}`x11757`). +- Extend stats and cost model to support {func}`row_number` window function estimates. +- Improve the join type selection and the reordering of join sides for cases where + the join output size cannot be estimated. +- Add dynamic scheduling support to grouped execution. When a stage is executed + with grouped execution and the stage has no remote sources, table partitions can be + scheduled to tasks in a dynamic way, which can help mitigating skew for queries using + grouped execution. This feature can be enabled with the + `dynamic_schedule_for_grouped_execution` session property or the + `dynamic-schedule-for-grouped-execution` config property. +- Add {func}`beta_cdf` and {func}`inverse_beta_cdf` functions. +- Split the reporting of raw input data and processed input data for source operators. +- Remove collection and reporting of raw input data statistics for the `Values`, + `Local Exchange`, and `Local Merge Sort` operators. +- Simplify `EXPLAIN (TYPE IO)` output when there are too many discrete components. + This avoids large output at the cost of reduced granularity. +- Add {func}`parse_presto_data_size` function. +- Add support for `UNION ALL` to optimizer's cost model. +- Add support for estimating the cost of filters by using a default filter factor. + The default value for the filter factor can be configured with the `default_filter_factor_enabled` + session property or the `optimizer.default-filter-factor-enabled`. + +## Geospatial + +- Add input validation checks to {func}`ST_LineString` to conform with the specification. +- Improve spatial join performance. +- Enable spatial joins for join conditions expressed with the {func}`ST_Within` function. + +## Web UI + +- Fix *Capture Snapshot* button for showing current thread stacks. +- Fix dropdown for expanding stage skew component on the query details page. +- Improve the performance of the thread snapshot component on the worker status page. +- Make the reporting of *Cumulative Memory* usage consistent on the query list and query details pages. +- Remove legacy thread UI. + +## Hive + +- Add predicate pushdown support for the `DATE` type to the Parquet reader. This change also fixes + a bug that may cause queries with predicates on `DATE` columns to fail with type mismatch errors. + +## Redis + +- Prevent printing the value of the `redis.password` configuration property to log files. diff --git a/430/_sources/release/release-0.54.md.txt b/430/_sources/release/release-0.54.md.txt new file mode 100644 index 000000000..cf715f7be --- /dev/null +++ b/430/_sources/release/release-0.54.md.txt @@ -0,0 +1,34 @@ +# Release 0.54 + +- Restore binding for the node resource on the coordinator, which provides + the state of all nodes as seen by the coordinator's failure detector. + Access `/v1/node` to see all nodes, or `/v1/node/failed` to see failed nodes. + +- Prevent the {doc}`/client/cli` from hanging when the server goes away. + +- Add Hive connector `hive-hadoop1` for Apache Hadoop 1.x. + +- Add support for Snappy and LZ4 compression codecs for the `hive-cdh4` connector. + +- Add Example HTTP connector `example-http` that reads CSV data via HTTP. + The connector requires a metadata URI that returns a JSON document + describing the table metadata and the CSV files to read. + + Its primary purpose is to serve as an example of how to write a connector, + but it can also be used directly. Create `etc/catalog/example.properties` + with the following contents to mount the `example-http` connector as the + `example` catalog: + + ```text + connector.name=example-http + metadata-uri=http://s3.amazonaws.com/presto-example/v1/example-metadata.json + ``` + +- Show correct error message when a catalog or schema does not exist. + +- Verify JVM requirements on startup. + +- Log an error when the JVM code cache is full. + +- Upgrade the embedded Discovery server to allow using + non-UUID values for the `node.id` property. diff --git a/430/_sources/release/release-0.55.md.txt b/430/_sources/release/release-0.55.md.txt new file mode 100644 index 000000000..94149c00d --- /dev/null +++ b/430/_sources/release/release-0.55.md.txt @@ -0,0 +1,103 @@ +# Release 0.55 + +## RC binary 2-4x gain in CPU efficiency + +Presto uses custom fast-path decoding logic for specific Hive file +formats. In this release we have added a fast path for RCFile when using +the Binary SerDe (`LazyBinaryColumnarSerDe`). In our +micro benchmarks, we see a gain between 2x and 4x in CPU efficiency compared +to the generic (slow) path. Since Hive data decoding accounts for a +significant portion of the CPU time, this should +result in measurable gains for most queries over RC Binary encoded data. +Note that this optimization may not result in a reduction in latency +if your cluster is network or disk I/O bound. + +## Hash distributed aggregations + +`GROUP BY` aggregations are now distributed across a fixed number of machines. +This is controlled by the property `query.initial-hash-partitions` set in +`etc/config.properties` of the coordinator and workers. If the value is +larger than the number of machines available during query scheduling, Presto +will use all available machines. The default value is `8`. + +The maximum memory size of an aggregation is now +`query.initial-hash-partitions` times `task.max-memory`. + +## Simple distinct aggregations + +We have added support for the `DISTINCT` argument qualifier for aggregation +functions. This is currently limited to queries without a `GROUP BY` clause and +where all the aggregation functions have the same input expression. For example: + +``` +SELECT count(DISTINCT country) +FROM users +``` + +Support for complete `DISTINCT` functionality is in our roadmap. + +## Range predicate pushdown + +We've modified the connector API to support range predicates in addition to simple equality predicates. +This lays the ground work for adding connectors to systems that support range +scans (e.g., HBase, Cassandra, JDBC, etc). + +In addition to receiving range predicates, the connector can also communicate +back the ranges of each partition for use in the query optimizer. This can be a +major performance gain for `JOIN` queries where one side of the join has +only a few partitions. For example: + +``` +SELECT * FROM data_1_year JOIN data_1_week USING (ds) +``` + +If `data_1_year` and `data_1_week` are both partitioned on `ds`, the +connector will report back that one table has partitions for 365 days and the +other table has partitions for only 7 days. Then the optimizer will limit +the scan of the `data_1_year` table to only the 7 days that could possible +match. These constraints are combined with other predicates in the +query to further limit the data scanned. + +:::{note} +This is a backwards incompatible change with the previous connector SPI, +so if you have written a connector, you will need to update your code +before deploying this release. +::: + +## json_array_get function + +The {func}`json_array_get` function makes it simple to fetch a single element from a +scalar json array. + +## Non-reserved keywords + +The keywords `DATE`, `TIME`, `TIMESTAMP`, and `INTERVAL` are no longer +reserved keywords in the grammar. This means that you can access a column +named `date` without quoting the identifier. + +## CLI source option + +The Presto CLI now has an option to set the query source. The source +value is shown in the UI and is recorded in events. When using the CLI in +shell scripts it is useful to set the `--source` option to distinguish shell +scripts from normal users. + +## SHOW SCHEMAS FROM + +Although the documentation included the syntax `SHOW SCHEMAS [FROM catalog]`, +it was not implemented. This release now implements this statement correctly. + +## Hive bucketed table fixes + +For queries over Hive bucketed tables, Presto will attempt to limit scans to +the buckets that could possible contain rows that match the WHERE clause. +Unfortunately, the algorithm we were using to select the buckets was not +correct, and sometimes we would either select the wrong files or fail to +select any files. We have aligned +the algorithm with Hive and now the optimization works as expected. + +We have also improved the algorithm for detecting tables that are not properly +bucketed. It is common for tables to declare bucketing in the Hive metadata, but +not actually be bucketed in HDFS. When Presto detects this case, it fallback to a full scan of the +partition. Not only does this change make bucketing safer, but it makes it easier +to migrate a table to use bucketing without rewriting all of the data. diff --git a/430/_sources/release/release-0.56.md.txt b/430/_sources/release/release-0.56.md.txt new file mode 100644 index 000000000..fafb315da --- /dev/null +++ b/430/_sources/release/release-0.56.md.txt @@ -0,0 +1,37 @@ +# Release 0.56 + +## Table creation + +Tables can be created from the result of a query: + +``` +CREATE TABLE orders_by_date AS +SELECT orderdate, sum(totalprice) AS price +FROM orders +GROUP BY orderdate +``` + +Tables are created in Hive without partitions (unpartitioned) and use +RCFile with the Binary SerDe (`LazyBinaryColumnarSerDe`) as this is +currently the best format for Presto. + +:::{note} +This is a backwards incompatible change to `ConnectorMetadata` in the SPI, +so if you have written a connector, you will need to update your code before +deploying this release. We recommend changing your connector to extend from +the new `ReadOnlyConnectorMetadata` abstract base class unless you want to +support table creation. +::: + +## Cross joins + +Cross joins are supported using the standard ANSI SQL syntax: + +``` +SELECT * +FROM a +CROSS JOIN b +``` + +Inner joins that result in a cross join due to the join criteria evaluating +to true at analysis time are also supported. diff --git a/430/_sources/release/release-0.57.md.txt b/430/_sources/release/release-0.57.md.txt new file mode 100644 index 000000000..b18db674b --- /dev/null +++ b/430/_sources/release/release-0.57.md.txt @@ -0,0 +1,47 @@ +# Release 0.57 + +## Distinct aggregations + +The `DISTINCT` argument qualifier for aggregation functions is now +fully supported. For example: + +``` +SELECT country, count(DISTINCT city), count(DISTINCT age) +FROM users +GROUP BY country +``` + +:::{note} +{func}`approx_distinct` should be used in preference to this +whenever an approximate answer is allowable as it is substantially +faster and does not have any limits on the number of distinct items it +can process. `COUNT(DISTINCT ...)` must transfer every item over the +network and keep each distinct item in memory. +::: + +## Hadoop 2.x + +Use the `hive-hadoop2` connector to read Hive data from Hadoop 2.x. +See {doc}`/installation/deployment` for details. + +## Amazon S3 + +All Hive connectors support reading data from +[Amazon S3](http://aws.amazon.com/s3/). +This requires two additional catalog properties for the Hive connector +to specify your AWS Access Key ID and Secret Access Key: + +```text +hive.s3.aws-access-key=AKIAIOSFODNN7EXAMPLE +hive.s3.aws-secret-key=wJalrXUtnFEMI/K7MDENG/bPxRfiCYEXAMPLEKEY +``` + +## Miscellaneous + +- Allow specifying catalog and schema in the {doc}`/client/jdbc` URL. +- Implement more functionality in the JDBC driver. +- Allow certain custom `InputFormat`s to work by propagating + Hive serialization properties to the `RecordReader`. +- Many execution engine performance improvements. +- Fix optimizer performance regression. +- Fix weird `MethodHandle` exception. diff --git a/430/_sources/release/release-0.58.md.txt b/430/_sources/release/release-0.58.md.txt new file mode 100644 index 000000000..d697c858e --- /dev/null +++ b/430/_sources/release/release-0.58.md.txt @@ -0,0 +1,11 @@ +# Release 0.58 + +- Add first version of Cassandra connector. This plugin is still in + development and is not yet bundled with the server. See the `README` + in the plugin source directory for details. +- Support UDFs for internal plugins. This is not yet part of the SPI + and is a stopgap feature intended for advanced users. UDFs must be + implemented using the internal Presto APIs which often change + substantially between releases. +- Fix Hive connector semaphore release bug. +- Fix handling of non-splittable files without blocks. diff --git a/430/_sources/release/release-0.59.md.txt b/430/_sources/release/release-0.59.md.txt new file mode 100644 index 000000000..b3d479cea --- /dev/null +++ b/430/_sources/release/release-0.59.md.txt @@ -0,0 +1,4 @@ +# Release 0.59 + +- Fix hang in `HiveSplitSource`. A query over a large table can hang + in split discovery due to a bug introduced in 0.57. diff --git a/430/_sources/release/release-0.60.md.txt b/430/_sources/release/release-0.60.md.txt new file mode 100644 index 000000000..b6c044f3f --- /dev/null +++ b/430/_sources/release/release-0.60.md.txt @@ -0,0 +1,142 @@ +# Release 0.60 + +## JDBC improvements + +The Presto version of the JDBC `DatabaseMetaData` interface now includes +proper implementations of `getTables`, `getSchemas` and `getCatalogs`. + +The JDBC driver is now always packaged as a standalone jar without any +dependencies. Previously, this artifact was published with the Maven +classifier `standalone`. The new build does not publish this artifact +anymore. + +## USE CATALOG and USE SCHEMA + +The {doc}`/client/cli` now supports `USE CATALOG` and +`USE SCHEMA`. + +## TPCH connector + +We have added a new connector that will generate synthetic data following the +TPC-H specification. This connector makes it easy to generate large datasets for +testing and bug reports. When generating bug reports, we encourage users to use +this catalog since it eases the process of reproducing the issue. The data is +generated dynamically for each query, so no disk space is used by this +connector. To add the `tpch` catalog to your system, create the catalog +property file `etc/catalog/tpch.properties` on both the coordinator and workers +with the following contents: + +```text +connector.name=tpch +``` + +Additionally, update the `datasources` property in the config properties file, +`etc/config.properties`, for the workers to include `tpch`. + +## SPI + +The `Connector` interface now has explicit methods for supplying the services +expected by the query engine. Previously, this was handled by a generic +`getService` method. + +:::{note} +This is a backwards incompatible change to `Connector` in the SPI, +so if you have written a connector, you will need to update your code before +deploying this release. +::: + +Additionally, we have added the `NodeManager` interface to the SPI to allow a +plugin to detect all nodes in the Presto cluster. This is important for some +connectors that can divide a table evenly between all nodes as long as the +connector knows how many nodes exist. To access the node manager, simply add +the following to the `Plugin` class: + +```java +@Inject +public void setNodeManager(NodeManager nodeManager) +{ + this.nodeManager = nodeManager; +} +``` + +## Optimizations + +### DISTINCT LIMIT + +For queries with the following form: + +``` +SELECT DISTINCT ... +FROM T +LIMIT N +``` + +We have added an optimization that stops the query as soon as `N` distinct +rows are found. + +### Range predicates + +When optimizing a join, Presto analyzes the ranges of the partitions on each +side of a join and pushes these ranges to the other side. When tables have a +lot of partitions, this can result in a very large filter with one expression +for each partition. The optimizer now summarizes the predicate ranges to reduce +the complexity of the filters. + +### Compound filters + +Complex expressions involving `AND`, `OR`, or `NOT` are now optimized by +the expression optimizer. + +### Window functions + +Window functions with a `PARTITION BY` clause are now distributed based on the +partition key. + +## Bug fixes + +- Scheduling + + In the changes to schedule splits in batches, we introduced two bugs that + resulted in an unbalanced workload across nodes which increases query latency. + The first problem was not inspecting the queued split count of the nodes while + scheduling the batch, and the second problem was not counting the splits + awaiting creation in the task executor. + +- JSON conversion of complex Hive types + + Presto converts complex Hive types (array, map, struct and union) into JSON. + Previously, numeric keys in maps were converted to numbers, not strings, + which is invalid as JSON only allows strings for object keys. This prevented + the {doc}`/functions/json` from working. + +- Hive hidden files + + Presto will now ignore files in Hive that start with an underscore `_` or + a dot `.`. This matches the behavior of Hadoop MapReduce / Hive. + +- Failures incorrectly reported as no data + + Certain types of failures would result in the query appearing to succeed and + return an incomplete result (often zero rows). There was a race condition + between the error propagation and query teardown. In some cases, the query + would be torn down before the exception made it to the coordinator. This was a + regression introduced during the query teardown optimization work. There are + now tests to catch this type of bug. + +- Exchange client leak + + When a query finished early (e.g., limit or failure) and the exchange operator + was blocked waiting for data from other nodes, the exchange was not be closed + properly. This resulted in continuous failing HTTP requests which leaked + resources and produced large log files. + +- Hash partitioning + + A query with many `GROUP BY` items could fail due to an overflow in the hash + function. + +- Compiled NULL literal + + In some cases queries with a select expression like `CAST(NULL AS varchar)` + would fail due to a bug in the output type detection code in expression + compiler. diff --git a/430/_sources/release/release-0.61.md.txt b/430/_sources/release/release-0.61.md.txt new file mode 100644 index 000000000..ed51fb85d --- /dev/null +++ b/430/_sources/release/release-0.61.md.txt @@ -0,0 +1,65 @@ +# Release 0.61 + +## Add support for table value constructors + +Presto now supports the SQL table value constructor syntax to create inline tables. +The `VALUES` clause can be used anywhere a `SELECT` statement is allowed. +For example, as a top-level query: + +``` +VALUES ('a', 1), ('b', 2); +``` + +```text + _col0 | _col1 +-------+------- + a | 1 + b | 2 +(2 rows) +``` + +Alternatively, in the `FROM` clause: + +``` +SELECT * +FROM ( + VALUES + ('a', 'ape'), + ('b', 'bear') +) AS animal (letter, animal) +JOIN ( + VALUES + ('a', 'apple'), + ('b', 'banana') +) AS fruit (letter, fruit) +USING (letter); +``` + +```text + letter | animal | letter | fruit +--------+--------+--------+--------- + a | ape | a | apple + b | bear | b | banana +(2 rows) +``` + +## Cassandra + +- Add support for upper-case schema, table, and columns names. +- Add support for `DECIMAL` type. + +## Amazon S3 support + +- Completely rewritten Hadoop FileSystem implementation for S3 using the Amazon AWS SDK, + with major performance and reliability improvements. +- Add support for writing data to S3. + +## Miscellaneous + +- General improvements to the JDBC driver, specifically with respect to metadata handling. +- Fix division by zero errors in variance aggregation functions (`VARIANCE`, `STDDEV`, etc.). +- Fix a bug when using `DISTINCT` aggregations in the `HAVING` clause. +- Fix an out of memory issue when writing large tables. +- Fix a bug when using `ORDER BY rand()` in a `JOIN` query. +- Fix handling of timestamps in maps and lists in Hive connector. +- Add instrumentation for Hive metastore and HDFS API calls to track failures and latency. These metrics are exposed via JMX. diff --git a/430/_sources/release/release-0.62.md.txt b/430/_sources/release/release-0.62.md.txt new file mode 100644 index 000000000..1103799ff --- /dev/null +++ b/430/_sources/release/release-0.62.md.txt @@ -0,0 +1,7 @@ +# Release 0.62 + +- Fix an issue with active queries JMX counter reporting incorrect numbers +- Hive binary map keys were not being decoded correctly +- Performance improvements for `APPROX_DISTINCT` +- Fix performance regression when planning queries over a large number of partitions +- Minor improvement to coordinator UI when displaying long SQL queries diff --git a/430/_sources/release/release-0.63.md.txt b/430/_sources/release/release-0.63.md.txt new file mode 100644 index 000000000..cb46be8ad --- /dev/null +++ b/430/_sources/release/release-0.63.md.txt @@ -0,0 +1,5 @@ +# Release 0.63 + +- Minor improvements to coordinator UI +- Minor planner optimization to avoid redundant computation in some cases +- Error handling and classification improvements diff --git a/430/_sources/release/release-0.64.md.txt b/430/_sources/release/release-0.64.md.txt new file mode 100644 index 000000000..96337578f --- /dev/null +++ b/430/_sources/release/release-0.64.md.txt @@ -0,0 +1,8 @@ +# Release 0.64 + +- Fix approximate aggregation error bound calculation +- Error handling and classification improvements +- Fix `GROUP BY` failure when keys are too large +- Add thread visualization UI at `/ui/thread` +- Fix regression in `CREATE TABLE` that can cause column data to be swapped. + This bug was introduced in version 0.57. diff --git a/430/_sources/release/release-0.65.md.txt b/430/_sources/release/release-0.65.md.txt new file mode 100644 index 000000000..f62b791c7 --- /dev/null +++ b/430/_sources/release/release-0.65.md.txt @@ -0,0 +1,4 @@ +# Release 0.65 + +- Fix `NullPointerException` when tearing down queries +- Fix exposed third-party dependencies in JDBC driver JAR diff --git a/430/_sources/release/release-0.66.md.txt b/430/_sources/release/release-0.66.md.txt new file mode 100644 index 000000000..000824471 --- /dev/null +++ b/430/_sources/release/release-0.66.md.txt @@ -0,0 +1,182 @@ +# Release 0.66 + +## Type system + +In this release we have replaced the existing simple fixed type system +with a fully extensible type system and have added several new types. +We have also expanded the function system to support custom +arithmetic, comparison and cast operators. For example, the new date/time +types include an operator for adding an `INTERVAL` to a `TIMESTAMP`. + +Existing functions have been updated to operate on and return the +newly added types. For example, the ANSI color functions now operate +on a `COLOR` type, and the date/time functions operate on standard +SQL date/time types (described below). + +Finally, plugins can now provide custom types and operators in addition +to connectors and functions. This feature is highly experimental, so expect +the interfaces to change over the next few releases. Also, since in SQL +there is only one namespace for types, you should be careful to make names +for custom types unique as we will add other common SQL types to Presto +in the near future. + +## Date/time types + +Presto now supports all standard SQL date/time types: +`DATE`, `TIME`, `TIMESTAMP` and `INTERVAL`. +All of the date/time functions and language constructs now operate on these +types instead of `BIGINT` and perform temporal calculations correctly. +This was previously broken due to, for example, not being able to detect +whether an argument was a `DATE` or a `TIMESTAMP`. +This change comes at the cost of breaking existing queries that perform +arithmetic operations directly on the `BIGINT` value returned from +the date/time functions. + +As part of this work, we have also added the {func}`date_trunc` function +which is convenient for grouping data by a time span. For example, you +can perform an aggregation by hour: + +``` +SELECT date_trunc('hour', timestamp_column), count(*) +FROM ... +GROUP BY 1 +``` + +### Time zones + +This release has full support for time zone rules, which are needed to +perform date/time calculations correctly. Typically, the session time +zone is used for temporal calculations. This is the time zone of the +client computer that submits the query, if available. Otherwise, it is +the time zone of the server running the Presto coordinator. + +Queries that operate with time zones that follow daylight saving can +produce unexpected results. For example, if we run the following query +to add 24 hours using in the `America/Los Angeles` time zone: + +``` +SELECT date_add('hour', 24, TIMESTAMP '2014-03-08 09:00:00'); +-- 2014-03-09 10:00:00.000 +``` + +The timestamp appears to only advance 23 hours. This is because on +March 9th clocks in `America/Los Angeles` are turned forward 1 hour, +so March 9th only has 23 hours. To advance the day part of the timestamp, +use the `day` unit instead: + +``` +SELECT date_add('day', 1, TIMESTAMP '2014-03-08 09:00:00'); +-- 2014-03-09 09:00:00.000 +``` + +This works because the {func}`date_add` function treats the timestamp as +list of fields, adds the value to the specified field and then rolls any +overflow into the next higher field. + +Time zones are also necessary for parsing and printing timestamps. +Queries that use this functionality can also produce unexpected results. +For example, on the same machine: + +``` +SELECT TIMESTAMP '2014-03-09 02:30:00'; +``` + +The above query causes an error because there was no 2:30 AM on March 9th +in `America/Los_Angeles` due to a daylight saving time transition. + +In addition to normal `TIMESTAMP` values, Presto also supports the +`TIMESTAMP WITH TIME ZONE` type, where every value has an explicit time zone. +For example, the following query creates a `TIMESTAMP WITH TIME ZONE`: + +``` +SELECT TIMESTAMP '2014-03-14 09:30:00 Europe/Berlin'; +-- 2014-03-14 09:30:00.000 Europe/Berlin +``` + +You can also change the time zone of an existing timestamp using the +`AT TIME ZONE` clause: + +``` +SELECT TIMESTAMP '2014-03-14 09:30:00 Europe/Berlin' + AT TIME ZONE 'America/Los_Angeles'; +-- 2014-03-14 01:30:00.000 America/Los_Angeles +``` + +Both timestamps represent the same instant in time; +they differ only in the time zone used to print them. + +The time zone of the session can be set on a per-query basis using the +`X-Presto-Time-Zone` HTTP header, or via the +`PrestoConnection.setTimeZoneId(String)` method in the JDBC driver. + +### Localization + +In addition to time zones, the language of the user is important when +parsing and printing date/time types. This release adds localization +support to the Presto engine and functions that require it: +{func}`date_format` and {func}`date_parse`. +For example, if we set the language to Spanish: + +``` +SELECT date_format(TIMESTAMP '2001-01-09 09:04', '%M'); -- enero +``` + +If we set the language to Japanese: + +``` +SELECT date_format(TIMESTAMP '2001-01-09 09:04', '%M'); -- 1月 +``` + +The language of the session can be set on a per-query basis using the +`X-Presto-Language` HTTP header, or via the +`PrestoConnection.setLocale(Locale)` method in the JDBC driver. + +## Optimizations + +- We have upgraded the Hive connector to Hive 0.12 which includes + performance improvements for RCFile. +- `GROUP BY` and `JOIN` operators are now compiled to byte code + and are significantly faster. +- Reduced memory usage of `GROUP BY` and `SELECT DISTINCT`, + which previously required several megabytes of memory + per operator, even when the number of groups was small. +- The planner now optimizes function call arguments. This should improve + the performance of queries that contain complex expressions. +- Fixed a performance regression in the HTTP client. The recent HTTP client + upgrade was using inadvertently GZIP compression and has a bug in the + buffer management resulting in high CPU usage. + +## SPI + +In this release we have made a number of backward incompatible changes to the SPI: + +- Added `Type` and related interfaces +- `ConnectorType` in metadata has been replaced with `Type` +- Renamed `TableHandle` to `ConnectorTableHandle` +- Renamed `ColumnHandle` to `ConnectorColumnHandle` +- Renamed `Partition` to `ConnectorPartition` +- Renamed `PartitionResult` to `ConnectorPartitionResult` +- Renamed `Split` to `ConnectorSplit` +- Renamed `SplitSource` to `ConnectorSplitSource` +- Added a `ConnectorSession` parameter to most `ConnectorMetadata` methods +- Removed most `canHandle` methods + +## General bug fixes + +- Fixed CLI hang after using `USE CATALOG` or `USE SCHEMA` +- Implicit coercions in aggregations now work as expected +- Nulls in expressions work as expected +- Fixed memory leak in compiler +- Fixed accounting bug in task memory usage +- Fixed resource leak caused by abandoned queries +- Fail queries immediately on unrecoverable data transport errors + +## Hive bug fixes + +- Fixed parsing of timestamps in the Hive RCFile Text SerDe (`ColumnarSerDe`) + by adding configuration to set the time zone originally used when writing data + +## Cassandra bug fixes + +- Auto-reconnect if Cassandra session dies +- Format collection types as JSON diff --git a/430/_sources/release/release-0.67.md.txt b/430/_sources/release/release-0.67.md.txt new file mode 100644 index 000000000..ae92c88db --- /dev/null +++ b/430/_sources/release/release-0.67.md.txt @@ -0,0 +1,15 @@ +# Release 0.67 + +- Fix resource leak in Hive connector +- Improve error categorization in event logging +- Fix planning issue with certain queries using window functions + +## SPI + +The `ConnectorSplitSource` interface now extends `Closeable`. + +:::{note} +This is a backwards incompatible change to `ConnectorSplitSource` in the SPI, +so if you have written a connector, you will need to update your code before +deploying this release. +::: diff --git a/430/_sources/release/release-0.68.md.txt b/430/_sources/release/release-0.68.md.txt new file mode 100644 index 000000000..e50004164 --- /dev/null +++ b/430/_sources/release/release-0.68.md.txt @@ -0,0 +1,7 @@ +# Release 0.68 + +- Fix a regression in the handling of Hive tables that are bucketed on a + string column. This caused queries that could take advantage of bucketing + on such tables to choose the wrong bucket and thus would not match any + rows for the table. This regression was introduced in 0.66. +- Fix double counting of bytes and rows when reading records diff --git a/430/_sources/release/release-0.69.md.txt b/430/_sources/release/release-0.69.md.txt new file mode 100644 index 000000000..7aac8f730 --- /dev/null +++ b/430/_sources/release/release-0.69.md.txt @@ -0,0 +1,97 @@ +# Release 0.69 + +:::{warning} +The following config properties must be removed from the +`etc/config.properties` file on both the coordinator and workers: + +- `presto-metastore.db.type` +- `presto-metastore.db.filename` + +Additionally, the `datasources` property is now deprecated and should also be +removed (see [Datasource Configuration](rn-069-datasource-configuration)). +::: + +## Prevent scheduling work on coordinator + +We have a new config property, `node-scheduler.include-coordinator`, +that allows or disallows scheduling work on the coordinator. +Previously, tasks like final aggregations could be scheduled on the +coordinator. For larger clusters, processing work on the coordinator +can impact query performance because the machine's resources are not +available for the critical task of scheduling, managing and monitoring +query execution. + +We recommend setting this property to `false` for the coordinator. +See {ref}`config-properties` for an example. + +(rn-069-datasource-configuration)= +## Datasource configuration + +The `datasources` config property has been deprecated. +Please remove it from your `etc/config.properties` file. +The datasources configuration is now automatically generated based +on the `node-scheduler.include-coordinator` property +(see [Prevent Scheduling Work on Coordinator]). + +## Raptor connector + +Presto has an extremely experimental connector that was previously called +the `native` connector and was intertwined with the main Presto code +(it was written before Presto had connectors). This connector is now +named `raptor` and lives in a separate plugin. + +As part of this refactoring, the `presto-metastore.db.type` and +`presto-metastore.db.filename` config properties no longer exist +and must be removed from `etc/config.properties`. + +The Raptor connector stores data on the Presto machines in a +columnar format using the same layout that Presto uses for in-memory +data. Currently, it has major limitations: lack of replication, +dropping a table does not reclaim the storage, etc. It is only +suitable for experimentation, temporary tables, caching of data from +slower connectors, etc. The metadata and data formats are subject to +change in incompatible ways between releases. + +If you would like to experiment with the connector, create a catalog +properties file such as `etc/catalog/raptor.properties` on both the +coordinator and workers that contains the following: + +```text +connector.name=raptor +metadata.db.type=h2 +metadata.db.filename=var/data/db/MetaStore +``` + +## Machine learning functions + +Presto now has functions to train and use machine learning models +(classifiers and regressors). This is currently only a proof of concept +and is not ready for use in production. Example usage is as follows: + +``` +SELECT evaluate_classifier_predictions(label, classify(features, model)) +FROM ( + SELECT learn_classifier(label, features) AS model + FROM training_data +) +CROSS JOIN validation_data +``` + +In the above example, the column `label` is a `bigint` and the column +`features` is a map of feature identifiers to feature values. The feature +identifiers must be integers (encoded as strings because JSON only supports +strings for map keys) and the feature values are numbers (floating point). + +## Variable length binary type + +Presto now supports the `varbinary` type for variable length binary data. +Currently, the only supported function is {func}`length`. +The Hive connector now maps the Hive `BINARY` type to `varbinary`. + +## General + +- Add missing operator: `timestamp with time zone` - `interval year to month` +- Support explaining sampled queries +- Add JMX stats for abandoned and canceled queries +- Add `javax.inject` to parent-first class list for plugins +- Improve error categorization in event logging diff --git a/430/_sources/release/release-0.70.md.txt b/430/_sources/release/release-0.70.md.txt new file mode 100644 index 000000000..922c22589 --- /dev/null +++ b/430/_sources/release/release-0.70.md.txt @@ -0,0 +1,81 @@ +# Release 0.70 + +:::{warning} +This release contained a packaging error that resulted in an +unusable server tarball. Do not use this release. +::: + +## Views + +We have added support for creating views within Presto. +Views are defined using Presto syntax but are stored (as blobs) +by connectors. Currently, views are supported by the +Raptor and Hive connectors. For the Hive connector, views are +stored within the Hive metastore as Hive views, but they cannot +be queried by Hive, nor can Hive views be queried by Presto. + +See {doc}`/sql/create-view` and {doc}`/sql/drop-view` +for details and examples. + +## DUAL table + +The synthetic `DUAL` table is no longer supported. As an alternative, please +write your queries without a `FROM` clause or use the `VALUES` syntax. + +## Presto Verifier + +There is a new project, Presto Verifier, which can be used to verify a set of +queries against two different clusters. + +## Connector improvements + +- Connectors can now add hidden columns to a table. Hidden columns are not + displayed in `DESCRIBE` or `information_schema`, and are not + considered for `SELECT *`. As an example, we have added a hidden + `row_number` column to the `tpch` connector. +- Presto contains an extensive test suite to verify the correctness. This test + suite has been extracted into the `presto-test` module for use during + connector development. For an example, see `TestRaptorDistributedQueries`. + +## Machine learning functions + +We have added two new machine learning functions, which can be used +by advanced users familiar with LIBSVM. The functions are +`learn_libsvm_classifier` and `learn_libsvm_regressor`. Both take a +parameters string which has the form `key=value,key=value` + +## General + +- New comparison functions: {func}`greatest` and {func}`least` +- New window functions: {func}`first_value`, {func}`last_value`, and {func}`nth_value` +- We have added a config option to disable falling back to the interpreter when + expressions fail to be compiled to bytecode. To set this option, add + `compiler.interpreter-enabled=false` to `etc/config.properties`. + This will force certain queries to fail rather than running slowly. +- `DATE` values are now implicitly coerced to `TIMESTAMP` and `TIMESTAMP WITH TIME ZONE` + by setting the hour/minute/seconds to `0` with respect to the session timezone. +- Minor performance optimization when planning queries over tables with tens of + thousands of partitions or more. +- Fixed a bug when planning `ORDER BY ... LIMIT` queries which could result in + duplicate and un-ordered results under rare conditions. +- Reduce the size of stats collected from tasks, which dramatically reduces + garbage generation and improves coordinator stability. +- Fix compiler cache for expressions. +- Fix processing of empty or commented out statements in the CLI. + +## Hive + +- There are two new configuration options for the Hive connector, + `hive.max-initial-split-size`, which configures the size of the + initial splits, and `hive.max-initial-splits`, which configures + the number of initial splits. This can be useful for speeding up small + queries, which would otherwise have low parallelism. +- The Hive connector will now consider all tables with a non-empty value + for the table property `presto_offline` to be offline. The value of the + property will be used in the error message. +- We have added support for `DROP TABLE` in the hive connector. + By default, this feature is not enabled. To enable it, set + `hive.allow-drop-table=true` in your Hive catalog properties file. +- Ignore subdirectories when generating splits + (this now matches the non-recursive behavior of Hive). +- Fix handling of maps with null keys. diff --git a/430/_sources/release/release-0.71.md.txt b/430/_sources/release/release-0.71.md.txt new file mode 100644 index 000000000..77fa270df --- /dev/null +++ b/430/_sources/release/release-0.71.md.txt @@ -0,0 +1,5 @@ +# Release 0.71 + +- Fix packaging issue that resulted in an unusable server tarball + for the 0.70 release +- Fix logging in Hive connector when using Amazon S3 diff --git a/430/_sources/release/release-0.72.md.txt b/430/_sources/release/release-0.72.md.txt new file mode 100644 index 000000000..e987e7b1b --- /dev/null +++ b/430/_sources/release/release-0.72.md.txt @@ -0,0 +1,4 @@ +# Release 0.72 + +- Fix infinite loop bug in Hive RCFile reader when decoding a Map + with a null key diff --git a/430/_sources/release/release-0.73.md.txt b/430/_sources/release/release-0.73.md.txt new file mode 100644 index 000000000..ce17f46ad --- /dev/null +++ b/430/_sources/release/release-0.73.md.txt @@ -0,0 +1,12 @@ +# Release 0.73 + +## Cassandra plugin + +The Cassandra connector now supports CREATE TABLE and DROP TABLE. Additionally, +the connector now takes into account Cassandra indexes when generating CQL. +This release also includes several bug fixes and performance improvements. + +## General + +- New window functions: {func}`lead`, and {func}`lag` +- New scalar function: {func}`json_size` diff --git a/430/_sources/release/release-0.74.md.txt b/430/_sources/release/release-0.74.md.txt new file mode 100644 index 000000000..1b8ab87ad --- /dev/null +++ b/430/_sources/release/release-0.74.md.txt @@ -0,0 +1,27 @@ +# Release 0.74 + +## Bytecode compiler + +This version includes new infrastructure for bytecode compilation, and lays the groundwork for future improvements. +There should be no impact in performance or correctness with the new code, but we have added a flag to revert to the +old implementation in case of issues. To do so, add `compiler.new-bytecode-generator-enabled=false` to +`etc/config.properties` in the coordinator and workers. + +## Hive storage format + +The storage format to use when writing data to Hive can now be configured via the `hive.storage-format` option +in your Hive catalog properties file. Valid options are `RCBINARY`, `RCTEXT`, `SEQUENCEFILE` and `TEXTFILE`. +The default format if the property is not set is `RCBINARY`. + +## General + +- Show column comments in `DESCRIBE` +- Add {func}`try_cast` which works like {func}`cast` but returns `null` if the cast fails +- `nullif` now correctly returns a value with the type of the first argument +- Fix an issue with {func}`timezone_hour` returning results in milliseconds instead of hours +- Show a proper error message when analyzing queries with non-equijoin clauses +- Improve "too many failures" error message when coordinator can't talk to workers +- Minor optimization of {func}`json_size` function +- Improve feature normalization algorithm for machine learning functions +- Add exponential back-off to the S3 FileSystem retry logic +- Improve CPU efficiency of semi-joins diff --git a/430/_sources/release/release-0.75.md.txt b/430/_sources/release/release-0.75.md.txt new file mode 100644 index 000000000..afb68fd0d --- /dev/null +++ b/430/_sources/release/release-0.75.md.txt @@ -0,0 +1,109 @@ +# Release 0.75 + +## Hive + +- The Hive S3 file system has a new configuration option, + `hive.s3.max-connections`, which sets the maximum number of + connections to S3. The default has been increased from `50` to `500`. +- The Hive connector now supports renaming tables. By default, this feature + is not enabled. To enable it, set `hive.allow-rename-table=true` in + your Hive catalog properties file. + +## General + +- Optimize {func}`count` with a constant to execute as the much faster `count(*)` +- Add support for binary types to the JDBC driver +- The legacy byte code compiler has been removed +- New aggregation framework (~10% faster) +- Added {func}`max_by` aggregation function +- The `approx_avg()` function has been removed. Use {func}`avg` instead. +- Fixed parsing of `UNION` queries that use both `DISTINCT` and `ALL` +- Fixed cross join planning error for certain query shapes +- Added hex and base64 conversion functions for varbinary +- Fix the `LIKE` operator to correctly match against values that contain + multiple lines. Previously, it would stop matching at the first newline. +- Add support for renaming tables using the {doc}`/sql/alter-table` statement. +- Add basic support for inserting data using the {doc}`/sql/insert` statement. + This is currently only supported for the Raptor connector. + +## JSON function + +The {func}`json_extract` and {func}`json_extract_scalar` functions now support +the square bracket syntax: + +``` +SELECT json_extract(json, '$.store[book]'); +SELECT json_extract(json, '$.store["book name"]'); +``` + +As part of this change, the set of characters allowed in a non-bracketed +path segment has been restricted to alphanumeric, underscores and colons. +Additionally, colons cannot be used in a un-quoted bracketed path segment. +Use the new bracket syntax with quotes to match elements that contain +special characters. + +## Scheduler + +The scheduler now assigns splits to a node based on the current load on the node across all queries. +Previously, the scheduler load balanced splits across nodes on a per query level. Every node can have +`node-scheduler.max-splits-per-node` splits scheduled on it. To avoid starvation of small queries, +when the node already has the maximum allowable splits, every task can schedule at most +`node-scheduler.max-pending-splits-per-node-per-task` splits on the node. + +## Row number optimizations + +Queries that use the {func}`row_number` function are substantially faster +and can run on larger result sets for two types of queries. + +Performing a partitioned limit that choses `N` arbitrary rows per +partition is a streaming operation. The following query selects +five arbitrary rows from `orders` for each `orderstatus`: + +``` +SELECT * FROM ( + SELECT row_number() OVER (PARTITION BY orderstatus) AS rn, + custkey, orderdate, orderstatus + FROM orders +) WHERE rn <= 5; +``` + +Performing a partitioned top-N that chooses the maximum or minimum +`N` rows from each partition now uses significantly less memory. +The following query selects the five oldest rows based on `orderdate` +from `orders` for each `orderstatus`: + +``` +SELECT * FROM ( + SELECT row_number() OVER (PARTITION BY orderstatus ORDER BY orderdate) AS rn, + custkey, orderdate, orderstatus + FROM orders +) WHERE rn <= 5; +``` + +Use the {doc}`/sql/explain` statement to see if any of these optimizations +have been applied to your query. + +## SPI + +The core Presto engine no longer automatically adds a column for `count(*)` +queries. Instead, the `RecordCursorProvider` will receive an empty list of +column handles. + +The `Type` and `Block` APIs have gone through a major refactoring in this +release. The main focus of the refactoring was to consolidate all type specific +encoding logic in the type itself, which makes types much easier to implement. +You should consider `Type` and `Block` to be a beta API as we expect +further changes in the near future. + +To simplify the API, `ConnectorOutputHandleResolver` has been merged into +`ConnectorHandleResolver`. Additionally, `ConnectorHandleResolver`, +`ConnectorRecordSinkProvider` and `ConnectorMetadata` were modified to +support inserts. + +:::{note} +This is a backwards incompatible change with the previous connector and +type SPI, so if you have written a connector or type, you will need to update +your code before deploying this release. In particular, make sure your +connector can handle an empty column handles list (this can be verified +by running `SELECT count(*)` on a table from your connector). +::: diff --git a/430/_sources/release/release-0.76.md.txt b/430/_sources/release/release-0.76.md.txt new file mode 100644 index 000000000..5555bd415 --- /dev/null +++ b/430/_sources/release/release-0.76.md.txt @@ -0,0 +1,67 @@ +# Release 0.76 + +## Kafka connector + +This release adds a connector that allows querying of [Apache Kafka] topic data +from Presto. Topics can be live and repeated queries will pick up new data. + +Apache Kafka 0.8+ is supported although Apache Kafka 0.8.1+ is recommended. +There is extensive {doc}`documentation ` about configuring +the connector and a {doc}`tutorial ` to get started. + +## MySQL and PostgreSQL connectors + +This release adds the {doc}`/connector/mysql` and {doc}`/connector/postgresql` +for querying and creating tables in external relational databases. These can +be used to join or copy data between different systems like MySQL and Hive, +or between two different MySQL or PostgreSQL instances, or any combination. + +## Cassandra + +The {doc}`/connector/cassandra` configuration properties +`cassandra.client.read-timeout` and `cassandra.client.connect-timeout` +are now specified using a duration rather than milliseconds (this makes +them consistent with all other such properties in Presto). If you were +previously specifying a value such as `25`, change it to `25ms`. + +The retry policy for the Cassandra client is now configurable via the +`cassandra.retry-policy` property. In particular, the custom `BACKOFF` +retry policy may be useful. + +## Hive + +The new {doc}`/connector/hive` configuration property `hive.s3.socket-timeout` +allows changing the socket timeout for queries that read or write to Amazon S3. +Additionally, the previously added `hive.s3.max-connections` property +was not respected and always used the default of `500`. + +Hive allows the partitions in a table to have a different schema than the +table. In particular, it allows changing the type of a column without +changing the column type of existing partitions. The Hive connector does +not support this and could previously return garbage data for partitions +stored using the RCFile Text format if the column type was converted from +a non-numeric type such as `STRING` to a numeric type such as `BIGINT` +and the actual data in existing partitions was not numeric. The Hive +connector now detects this scenario and fails the query after the +partition metadata has been read. + +The property `hive.storage-format` is broken and has been disabled. It +sets the storage format on the metadata but always writes the table using +`RCBINARY`. This will be implemented in a future release. + +## General + +- Fix hang in verifier when an exception occurs. +- Fix {func}`chr` function to work with Unicode code points instead of ASCII code points. +- The JDBC driver no longer hangs the JVM on shutdown (all threads are daemon threads). +- Fix incorrect parsing of function arguments. +- The bytecode compiler now caches generated code for join and group byqueries, + which should improve performance and CPU efficiency for these types of queries. +- Improve planning performance for certain trivial queries over tables with lots of partitions. +- Avoid creating large output pages. This should mitigate some cases of + *"Remote page is too large"* errors. +- The coordinator/worker communication layer is now fully asynchronous. + Specifically, long-poll requests no longer tie up a thread on the worker. + This makes heavily loaded clusters more efficient. + +[apache kafka]: https://kafka.apache.org/ diff --git a/430/_sources/release/release-0.77.md.txt b/430/_sources/release/release-0.77.md.txt new file mode 100644 index 000000000..ce0ac6021 --- /dev/null +++ b/430/_sources/release/release-0.77.md.txt @@ -0,0 +1,40 @@ +# Release 0.77 + +## Parametric types + +Presto now has a framework for implementing parametric types and functions. +Support for {ref}`array-type` and {ref}`map-type` types has been added, including the element accessor +operator `[]`, and new {doc}`/functions/array`. + +## Streaming index joins + +Index joins will now switch to use a key-by-key streaming join if index +results fail to fit in the allocated index memory space. + +## Distributed joins + +Joins where both tables are distributed are now supported. This allows larger tables to be joined, +and can be enabled with the `distributed-joins-enabled` flag. It may perform worse than the existing +broadcast join implementation because it requires redistributing both tables. +This feature is still experimental, and should be used with caution. + +## Hive + +- Handle spurious `AbortedException` when closing S3 input streams +- Add support for ORC, DWRF and Parquet in Hive +- Add support for `DATE` type in Hive +- Fix performance regression in Hive when reading `VARCHAR` columns + +## Kafka + +- Fix Kafka handling of default port +- Add support for Kafka messages with a null key + +## General + +- Fix race condition in scheduler that could cause queries to hang +- Add ConnectorPageSource which is a more efficient interface for column-oriented sources +- Add support for string partition keys in Cassandra +- Add support for variable arity functions +- Add support for {func}`count` for all types +- Fix bug in HashAggregation that could cause the operator to go in an infinite loop diff --git a/430/_sources/release/release-0.78.md.txt b/430/_sources/release/release-0.78.md.txt new file mode 100644 index 000000000..ebdcfb068 --- /dev/null +++ b/430/_sources/release/release-0.78.md.txt @@ -0,0 +1,54 @@ +# Release 0.78 + +## ARRAY and MAP types in Hive connector + +The Hive connector now returns arrays and maps instead of json encoded strings, +for columns whose underlying type is array or map. Please note that this is a backwards +incompatible change, and the {doc}`/functions/json` will no longer work on these columns, +unless you {func}`cast` them to the `json` type. + +## Session properties + +The Presto session can now contain properties, which can be used by the Presto +engine or connectors to customize the query execution. There is a separate +namespace for the Presto engine and each catalog. A property for a catalog is +simplify prefixed with the catalog name followed by `.` (dot). A connector +can retrieve the properties for the catalog using +`ConnectorSession.getProperties()`. + +Session properties can be set using the `--session` command line argument to +the Presto CLI. For example: + +```text +presto-cli --session color=red --session size=large +``` + +For JDBC, the properties can be set by unwrapping the `Connection` as follows: + +```java +connection.unwrap(PrestoConnection.class).setSessionProperty("name", "value"); +``` + +:::{note} +This feature is a work in progress and will change in a future release. +Specifically, we are planning to require preregistration of properties so +the user can list available session properties and so the engine can verify +property values. Additionally, the Presto grammar will be extended to +allow setting properties via a query. +::: + +## Hive + +- Add `storage_format` session property to override format used for creating tables. +- Add write support for `VARBINARY`, `DATE` and `TIMESTAMP`. +- Add support for partition keys of type `TIMESTAMP`. +- Add support for partition keys with null values (`__HIVE_DEFAULT_PARTITION__`). +- Fix `hive.storage-format` option (see {doc}`release-0.76`). + +## General + +- Fix expression optimizer, so that it runs in linear time instead of exponential time. +- Add {func}`cardinality` for maps. +- Fix race condition in SqlTask creation which can cause queries to hang. +- Fix `node-scheduler.multiple-tasks-per-node-enabled` option. +- Fix an exception when planning a query with a UNION under a JOIN. diff --git a/430/_sources/release/release-0.79.md.txt b/430/_sources/release/release-0.79.md.txt new file mode 100644 index 000000000..4015d18ad --- /dev/null +++ b/430/_sources/release/release-0.79.md.txt @@ -0,0 +1,16 @@ +# Release 0.79 + +## Hive + +- Add configuration option `hive.force-local-scheduling` and session property + `force_local_scheduling` to force local scheduling of splits. +- Add new experimental optimized RCFile reader. The reader can be enabled by + setting the configuration option `hive.optimized-reader.enabled` or session + property `optimized_reader_enabled`. + +## General + +- Add support for {ref}`unnest`, which can be used as a replacement for the `explode()` function in Hive. +- Fix a bug in the scan operator that can cause data to be missed. It currently only affects queries + over `information_schema` or `sys` tables, metadata queries such as `SHOW PARTITIONS` and connectors + that implement the `ConnectorPageSource` interface. diff --git a/430/_sources/release/release-0.80.md.txt b/430/_sources/release/release-0.80.md.txt new file mode 100644 index 000000000..17c885082 --- /dev/null +++ b/430/_sources/release/release-0.80.md.txt @@ -0,0 +1,103 @@ +# Release 0.80 + +## New Hive ORC reader + +We have added a new ORC reader implementation. The new reader supports vectorized +reads, lazy loading, and predicate push down, all of which make the reader more +efficient and typically reduces wall clock time for a query. Although the new +reader has been heavily tested, it is an extensive rewrite of the Apache Hive +ORC reader, and may have some latent issues. If you are seeing issues, you can +disable the new reader on a per-query basis by setting the +`.optimized_reader_enabled` session property, or you can disable +the reader by default by setting the Hive catalog property +`hive.optimized-reader.enabled=false`. + +## Hive + +- The maximum retry time for the Hive S3 file system can be configured + by setting `hive.s3.max-retry-time`. +- Fix Hive partition pruning for null keys (i.e. `__HIVE_DEFAULT_PARTITION__`). + +## Cassandra + +- Update Cassandra driver to 2.1.0. +- Map Cassandra `TIMESTAMP` type to Presto `TIMESTAMP` type. + +## "Big Query" support + +We've added experimental support for "big" queries. This provides a separate +queue controlled by the following properties: + +- `experimental.max-concurrent-big-queries` +- `experimental.max-queued-big-queries` + +There are separate configuration options for queries that are submitted with +the `experimental_big_query` session property: + +- `experimental.big-query-initial-hash-partitions` +- `experimental.big-query-max-task-memory` + +Queries submitted with this property will use hash distribution for all joins. + +## Metadata-only query optimization + +We now support an optimization that rewrites aggregation queries that are insensitive to the +cardinality of the input (e.g., {func}`max`, {func}`min`, `DISTINCT` aggregates) to execute +against table metadata. + +For example, if `key`, `key1` and `key2` are partition keys, the following queries +will benefit: + +``` +SELECT min(key), max(key) FROM t; + +SELECT DISTINCT key FROM t; + +SELECT count(DISTINCT key) FROM t; + +SELECT count(DISTINCT key + 5) FROM t; + +SELECT count(DISTINCT key) FROM (SELECT key FROM t ORDER BY 1 LIMIT 10); + +SELECT key1, count(DISTINCT key2) FROM t GROUP BY 1; +``` + +This optimization is turned off by default. To turn it on, add `optimizer.optimize-metadata-queries=true` +to the coordinator config properties. + +:::{warning} +This optimization will cause queries to produce incorrect results if +the connector allows partitions to contain no data. For example, the +Hive connector will produce incorrect results if your Hive warehouse +contains partitions without data. +::: + +## General + +- Add support implicit joins. The following syntax is now allowed: + + ``` + SELECT * FROM a, b WHERE a.id = b.id; + ``` + +- Add property `task.verbose-stats` to enable verbose statistics collection for + tasks. The default is `false`. + +- Format binary data in the CLI as a hex dump. + +- Add approximate numeric histogram function {func}`numeric_histogram`. + +- Add {func}`array_sort` function. + +- Add {func}`map_keys` and {func}`map_values` functions. + +- Make {func}`row_number` completely streaming. + +- Add property `task.max-partial-aggregation-memory` to configure the memory limit + for the partial step of aggregations. + +- Fix exception when processing queries with an `UNNEST` operation where the output was not used. + +- Only show query progress in UI after the query has been fully scheduled. + +- Add query execution visualization to the coordinator UI. It can be accessed via the query details page. diff --git a/430/_sources/release/release-0.81.md.txt b/430/_sources/release/release-0.81.md.txt new file mode 100644 index 000000000..fd444b435 --- /dev/null +++ b/430/_sources/release/release-0.81.md.txt @@ -0,0 +1,11 @@ +# Release 0.81 + +## Hive + +- Fix ORC predicate pushdown. +- Fix column selection in RCFile. + +## General + +- Fix handling of null and out-of-range offsets for + {func}`lead`, {func}`lag` and {func}`nth_value` functions. diff --git a/430/_sources/release/release-0.82.md.txt b/430/_sources/release/release-0.82.md.txt new file mode 100644 index 000000000..bbbd7ebf7 --- /dev/null +++ b/430/_sources/release/release-0.82.md.txt @@ -0,0 +1,8 @@ +# Release 0.82 + +- Presto now supports the {ref}`row-type` type, and all Hive structs are + converted to ROWs, instead of JSON encoded VARCHARs. +- Add {func}`current_timezone` function. +- Improve planning performance for queries with thousands of columns. +- Fix a regression that was causing excessive memory allocation and GC pressure + in the coordinator. diff --git a/430/_sources/release/release-0.83.md.txt b/430/_sources/release/release-0.83.md.txt new file mode 100644 index 000000000..6048ecc13 --- /dev/null +++ b/430/_sources/release/release-0.83.md.txt @@ -0,0 +1,17 @@ +# Release 0.83 + +## Raptor + +- Raptor now enables specifying the backup storage location. This feature is highly experimental. +- Fix the handling of shards not assigned to any node. + +## General + +- Fix resource leak in query queues. +- Fix NPE when writing null `ARRAY/MAP` to Hive. +- Fix {func}`json_array_get` to handle nested structures. +- Fix `UNNEST` on null collections. +- Fix a regression where queries that fail during parsing or analysis do not expire. +- Make `JSON` type comparable. +- Added an optimization for hash aggregations. This optimization is turned off by default. + To turn it on, add `optimizer.optimize-hash-generation=true` to the coordinator config properties. diff --git a/430/_sources/release/release-0.84.md.txt b/430/_sources/release/release-0.84.md.txt new file mode 100644 index 000000000..4ba55cd62 --- /dev/null +++ b/430/_sources/release/release-0.84.md.txt @@ -0,0 +1,8 @@ +# Release 0.84 + +- Fix handling of `NaN` and infinity in ARRAYs +- Fix approximate queries that use `JOIN` +- Reduce excessive memory allocation and GC pressure in the coordinator +- Fix an issue where setting `node-scheduler.location-aware-scheduling-enabled=false` + would cause queries to fail for connectors whose splits were not remotely accessible +- Fix error when running `COUNT(*)` over tables in `information_schema` and `sys` diff --git a/430/_sources/release/release-0.85.md.txt b/430/_sources/release/release-0.85.md.txt new file mode 100644 index 000000000..c36905181 --- /dev/null +++ b/430/_sources/release/release-0.85.md.txt @@ -0,0 +1,4 @@ +# Release 0.85 + +- Improve query planning performance for tables with large numbers of partitions. +- Fix issue when using `JSON` values in `GROUP BY` expressions. diff --git a/430/_sources/release/release-0.86.md.txt b/430/_sources/release/release-0.86.md.txt new file mode 100644 index 000000000..f7d2d3f80 --- /dev/null +++ b/430/_sources/release/release-0.86.md.txt @@ -0,0 +1,24 @@ +# Release 0.86 + +## General + +- Add support for inequality `INNER JOIN` when each term of the condition refers to only one side of the join. +- Add {func}`ntile` function. +- Add {func}`map` function to create a map from arrays of keys and values. +- Add {func}`min_by` aggregation function. +- Add support for concatenating arrays with the `||` operator. +- Add support for `=` and `!=` to `JSON` type. +- Improve error message when `DISTINCT` is applied to types that are not comparable. +- Perform type validation for `IN` expression where the right-hand side is a subquery expression. +- Improve error message when `ORDER BY ... LIMIT` query exceeds its maximum memory allocation. +- Improve error message when types that are not orderable are used in an `ORDER BY` clause. +- Improve error message when the types of the columns for subqueries of a `UNION` query don't match. +- Fix a regression where queries could be expired too soon on a highly loaded cluster. +- Fix scheduling issue for queries involving tables from information_schema, which could result in + inconsistent metadata. +- Fix an issue with {func}`min_by` and {func}`max_by` that could result in an error when used with + a variable-length type (e.g., `VARCHAR`) in a `GROUP BY` query. +- Fix rendering of array attributes in JMX connector. +- Input rows/bytes are now tracked properly for `JOIN` queries. +- Fix case-sensitivity issue when resolving names of constant table expressions. +- Fix unnesting arrays and maps that contain the `ROW` type. diff --git a/430/_sources/release/release-0.87.md.txt b/430/_sources/release/release-0.87.md.txt new file mode 100644 index 000000000..517efd324 --- /dev/null +++ b/430/_sources/release/release-0.87.md.txt @@ -0,0 +1,6 @@ +# Release 0.87 + +## General + +- Fixed a bug where {ref}`row-type` types could have the wrong field names. +- Changed the minimum JDK version to 1.8. diff --git a/430/_sources/release/release-0.88.md.txt b/430/_sources/release/release-0.88.md.txt new file mode 100644 index 000000000..d52890e78 --- /dev/null +++ b/430/_sources/release/release-0.88.md.txt @@ -0,0 +1,15 @@ +# Release 0.88 + +## General + +- Added {func}`arbitrary` aggregation function. +- Allow using all {doc}`/functions/aggregate` as {doc}`/functions/window`. +- Support specifying window frames and correctly implement frames for all {doc}`/functions/window`. +- Allow {func}`approx_distinct` aggregation function to accept a standard error parameter. +- Implement {func}`least` and {func}`greatest` with variable number of arguments. +- {ref}`array-type` is now comparable and can be used as `GROUP BY` keys or in `ORDER BY` expressions. +- Implement `=` and `<>` operators for {ref}`row-type`. +- Fix excessive garbage creation in the ORC reader. +- Fix an issue that could cause queries using {func}`row_number()` and `LIMIT` to never terminate. +- Fix an issue that could cause queries with {func}`row_number()` and specific filters to produce incorrect results. +- Fixed an issue that caused the Cassandra plugin to fail to load with a SecurityException. diff --git a/430/_sources/release/release-0.89.md.txt b/430/_sources/release/release-0.89.md.txt new file mode 100644 index 000000000..0e4b56c29 --- /dev/null +++ b/430/_sources/release/release-0.89.md.txt @@ -0,0 +1,19 @@ +# Release 0.89 + +## DATE type + +The memory representation of dates is now the number of days since January 1, 1970 +using a 32-bit signed integer. + +:::{note} +This is a backwards incompatible change with the previous date +representation, so if you have written a connector, you will need to update +your code before deploying this release. +::: + +## General + +- `USE CATALOG` and `USE SCHEMA` have been replaced with {doc}`/sql/use`. +- Fix issue where `SELECT NULL` incorrectly returns 0 rows. +- Fix rare condition where `JOIN` queries could produce incorrect results. +- Fix issue where `UNION` queries involving complex types would fail during planning. diff --git a/430/_sources/release/release-0.90.md.txt b/430/_sources/release/release-0.90.md.txt new file mode 100644 index 000000000..0dbd23760 --- /dev/null +++ b/430/_sources/release/release-0.90.md.txt @@ -0,0 +1,60 @@ +# Release 0.90 + +:::{warning} +This release has a memory leak and should not be used. +::: + +## General + +- Initial support for partition and placement awareness in the query planner. This can + result in better plans for queries involving `JOIN` and `GROUP BY` over the same + key columns. +- Improve planning of UNION queries. +- Add presto version to query creation and completion events. +- Add property `task.writer-count` to configure the number of writers per task. +- Fix a bug when optimizing constant expressions involving binary types. +- Fix bug where a table writer commits partial results while cleaning up a failed query. +- Fix a bug when unnesting an array of doubles containing NaN or Infinity. +- Fix failure when accessing elements in an empty array. +- Fix *"Remote page is too large"* errors. +- Improve error message when attempting to cast a value to `UNKNOWN`. +- Update the {func}`approx_distinct` documentation with correct standard error bounds. +- Disable falling back to the interpreter when expressions fail to be compiled + to bytecode. To enable this option, add `compiler.interpreter-enabled=true` + to the coordinator and worker config properties. Enabling this option will + allow certain queries to run slowly rather than failing. +- Improve {doc}`/client/jdbc` conformance. In particular, all unimplemented + methods now throw `SQLException` rather than `UnsupportedOperationException`. + +## Functions and language features + +- Add {func}`bool_and` and {func}`bool_or` aggregation functions. +- Add standard SQL function {func}`every` as an alias for {func}`bool_and`. +- Add {func}`year_of_week` function. +- Add {func}`regexp_extract_all` function. +- Add {func}`map_agg` aggregation function. +- Add support for casting `JSON` to `ARRAY` or `MAP` types. +- Add support for unparenthesized expressions in `VALUES` clause. +- Added {doc}`/sql/set-session`, {doc}`/sql/reset-session` and {doc}`/sql/show-session`. +- Improve formatting of `EXPLAIN (TYPE DISTRIBUTED)` output and include additional + information such as output layout, task placement policy and partitioning functions. + +## Hive + +- Disable optimized metastore partition fetching for non-string partition keys. + This fixes an issue were Presto might silently ignore data with non-canonical + partition values. To enable this option, add `hive.assume-canonical-partition-keys=true` + to the coordinator and worker config properties. +- Don't retry operations against S3 that fail due to lack of permissions. + +## SPI + +- Add `getColumnTypes` to `RecordSink`. +- Use `Slice` for table writer fragments. +- Add `ConnectorPageSink` which is a more efficient interface for column-oriented sources. + +:::{note} +This is a backwards incompatible change with the previous connector SPI. +If you have written a connector, you will need to update your code +before deploying this release. +::: diff --git a/430/_sources/release/release-0.91.md.txt b/430/_sources/release/release-0.91.md.txt new file mode 100644 index 000000000..46e88c33d --- /dev/null +++ b/430/_sources/release/release-0.91.md.txt @@ -0,0 +1,9 @@ +# Release 0.91 + +:::{warning} +This release has a memory leak and should not be used. +::: + +## General + +- Clear `LazyBlockLoader` reference after load to free memory earlier. diff --git a/430/_sources/release/release-0.92.md.txt b/430/_sources/release/release-0.92.md.txt new file mode 100644 index 000000000..db0baf572 --- /dev/null +++ b/430/_sources/release/release-0.92.md.txt @@ -0,0 +1,5 @@ +# Release 0.92 + +## General + +- Fix buffer leak when a query fails. diff --git a/430/_sources/release/release-0.93.md.txt b/430/_sources/release/release-0.93.md.txt new file mode 100644 index 000000000..bdc560490 --- /dev/null +++ b/430/_sources/release/release-0.93.md.txt @@ -0,0 +1,35 @@ +# Release 0.93 + +## ORC memory usage + +This release changes the Presto ORC reader to favor small buffers when reading +varchar and varbinary data. Some ORC files contain columns of data that are +hundreds of megabytes when decompressed. In the previous Presto ORC reader, we +would allocate a single large shared buffer for all values in the column. This +would cause heap fragmentation in CMS and G1, and it would cause OOMs since +each value of the column retains a reference to the shared buffer. In this +release the ORC reader uses a separate buffer for each value in the column. +This reduces heap fragmentation and excessive memory retention at the expense +of object creation. + +## Verifier + +- Add support for setting username and password per query + +If you're upgrading from 0.92, you need to alter your verifier_queries table + +```sql +ALTER TABLE verifier_queries add test_username VARCHAR(256) NOT NULL default 'verifier-test'; +ALTER TABLE verifier_queries add test_password VARCHAR(256); +ALTER TABLE verifier_queries add control_username VARCHAR(256) NOT NULL default 'verifier-test'; +ALTER TABLE verifier_queries add control_password VARCHAR(256); +``` + +## General + +- Add optimizer for `LIMIT 0` +- Fix incorrect check to disable string statistics in ORC +- Ignore hidden columns in `INSERT` and `CREATE TABLE AS` queries +- Add SOCKS support to CLI +- Improve CLI output for update queries +- Disable pushdown for non-deterministic predicates diff --git a/430/_sources/release/release-0.94.md.txt b/430/_sources/release/release-0.94.md.txt new file mode 100644 index 000000000..18402f4a1 --- /dev/null +++ b/430/_sources/release/release-0.94.md.txt @@ -0,0 +1,21 @@ +# Release 0.94 + +## ORC memory usage + +This release contains additional changes to the Presto ORC reader to favor +small buffers when reading varchar and varbinary data. Some ORC files contain +columns of data that are hundreds of megabytes compressed. When reading these +columns, Presto would allocate a single buffer for the compressed column data, +and this would cause heap fragmentation in CMS and G1 and eventually OOMs. +In this release, the `hive.orc.max-buffer-size` sets the maximum size for a +single ORC buffer, and for larger columns we instead stream the data. This +reduces heap fragmentation and excessive buffers in ORC at the expense of +HDFS IOPS. The default value is `8MB`. + +## General + +- Update Hive CDH 4 connector to CDH 4.7.1 +- Fix `ORDER BY` with `LIMIT 0` +- Fix compilation of `try_cast` +- Group threads into Java thread groups to ease debugging +- Add `task.min-drivers` config to help limit number of concurrent readers diff --git a/430/_sources/release/release-0.95.md.txt b/430/_sources/release/release-0.95.md.txt new file mode 100644 index 000000000..3b11e73ec --- /dev/null +++ b/430/_sources/release/release-0.95.md.txt @@ -0,0 +1,5 @@ +# Release 0.95 + +## General + +- Fix task and stage leak, caused when a stage finishes before its substages. diff --git a/430/_sources/release/release-0.96.md.txt b/430/_sources/release/release-0.96.md.txt new file mode 100644 index 000000000..c244837f6 --- /dev/null +++ b/430/_sources/release/release-0.96.md.txt @@ -0,0 +1,18 @@ +# Release 0.96 + +## General + +- Fix {func}`try_cast` for `TIMESTAMP` and other types that + need access to session information. +- Fix planner bug that could result in incorrect results for + tables containing columns with the same prefix, underscores and numbers. +- `MAP` type is now comparable. +- Fix output buffer leak in `StatementResource.Query`. +- Fix leak in `SqlTasks` caused by invalid heartbeats . +- Fix double logging of queries submitted while the queue is full. +- Fixed "running queries" JMX stat. +- Add `distributed_join` session property to enable/disable distributed joins. + +## Hive + +- Add support for tables partitioned by `DATE`. diff --git a/430/_sources/release/release-0.97.md.txt b/430/_sources/release/release-0.97.md.txt new file mode 100644 index 000000000..69ee5c179 --- /dev/null +++ b/430/_sources/release/release-0.97.md.txt @@ -0,0 +1,16 @@ +# Release 0.97 + +## General + +- The queueing policy in Presto may now be injected. +- Speed up detection of ASCII strings in implementation of `LIKE` operator. +- Fix NullPointerException when metadata-based query optimization is enabled. +- Fix possible infinite loop when decompressing ORC data. +- Fix an issue where `NOT` clause was being ignored in `NOT BETWEEN` predicates. +- Fix a planning issue in queries that use `SELECT *`, window functions and implicit coercions. +- Fix scheduler deadlock for queries with a `UNION` between `VALUES` and `SELECT`. + +## Hive + +- Fix decoding of `STRUCT` type from Parquet files. +- Speed up decoding of ORC files with very small stripes. diff --git a/430/_sources/release/release-0.98.md.txt b/430/_sources/release/release-0.98.md.txt new file mode 100644 index 000000000..cfef01f7c --- /dev/null +++ b/430/_sources/release/release-0.98.md.txt @@ -0,0 +1,31 @@ +# Release 0.98 + +## Array, map, and row types + +The memory representation of these types is now `VariableWidthBlockEncoding` +instead of `JSON`. + +:::{note} +This is a backwards incompatible change with the previous representation, +so if you have written a connector or function, you will need to update +your code before deploying this release. +::: + +## Hive + +- Fix handling of ORC files with corrupt checkpoints. + +## SPI + +- Rename `Index` to `ConnectorIndex`. + +:::{note} +This is a backwards incompatible change, so if you have written a connector +that uses `Index`, you will need to update your code before deploying this release. +::: + +## General + +- Fix bug in `UNNEST` when output is unreferenced or partially referenced. +- Make {func}`max` and {func}`min` functions work on all orderable types. +- Optimize memory allocation in {func}`max_by` and other places that `Block` is used. diff --git a/430/_sources/release/release-0.99.md.txt b/430/_sources/release/release-0.99.md.txt new file mode 100644 index 000000000..84cb8c63d --- /dev/null +++ b/430/_sources/release/release-0.99.md.txt @@ -0,0 +1,8 @@ +# Release 0.99 + +## General + +- Reduce lock contention in `TaskExecutor`. +- Fix reading maps with null keys from ORC. +- Fix precomputed hash optimization for nulls values. +- Make {func}`contains()` work for all comparable types. diff --git a/430/_sources/release/release-300.md.txt b/430/_sources/release/release-300.md.txt new file mode 100644 index 000000000..ecf4917ca --- /dev/null +++ b/430/_sources/release/release-300.md.txt @@ -0,0 +1,97 @@ +# Release 300 (22 Jan 2019) + +## General + +- Fix {func}`array_intersect` and {func}`array_distinct` + skipping zeros when input also contains nulls. +- Fix `count(*)` aggregation returning null on empty relation + when `optimize_mixed_distinct_aggregation` is enabled. +- Improve table scan performance for structured types. +- Improve performance for {func}`array_intersect`. +- Improve performance of window functions by filtering partitions early. +- Add {func}`reduce_agg` aggregate function. +- Add {func}`millisecond` function. +- Remove `ON` keyword from {doc}`/sql/show-stats` (use `FOR` instead). +- Restrict `WHERE` clause in {doc}`/sql/show-stats` + to filters that can be pushed down to connectors. +- Return final results to clients immediately for failed queries. + +## JMX MBean naming + +- The base domain name for server MBeans is now `presto`. The old names can be + used by setting the configuration property `jmx.base-name` to `com.facebook.presto`. +- The base domain name for the Hive, Raptor, and Thrift connectors is `presto.plugin`. + The old names can be used by setting the catalog configuration property + `jmx.base-name` to `com.facebook.presto.hive`, `com.facebook.presto.raptor`, + or `com.facebook.presto.thrift`, respectively. + +## Web UI + +- Fix rendering of live plan view for queries involving index joins. + +## JDBC driver + +- Change driver class name to `io.prestosql.jdbc.PrestoDriver`. + +## System connector + +- Remove `node_id` column from `system.runtime.queries` table. + +## Hive connector + +- Fix accounting of time spent reading Parquet data. +- Fix corner case where the ORC writer fails with integer overflow when writing + highly compressible data using dictionary encoding ({issue}`x11930`). +- Fail queries reading Parquet files if statistics in those files are corrupt + (e.g., min > max). To disable this behavior, set the configuration + property `hive.parquet.fail-on-corrupted-statistics` + or session property `parquet_fail_with_corrupted_statistics` to false. +- Add support for S3 Select pushdown, which enables pushing down + column selection and range filters into S3 for text files. + +## Kudu connector + +- Add `number_of_replicas` table property to `SHOW CREATE TABLE` output. + +## Cassandra connector + +- Add `cassandra.splits-per-node` and `cassandra.protocol-version` configuration + properties to allow connecting to Cassandra servers older than 2.1.5. + +## MySQL connector + +- Add support for predicate pushdown for columns of `char(x)` type. + +## PostgreSQL connector + +- Add support for predicate pushdown for columns of `char(x)` type. + +## Redshift connector + +- Add support for predicate pushdown for columns of `char(x)` type. + +## SQL Server connector + +- Add support for predicate pushdown for columns of `char(x)` type. + +## Raptor Legacy connector + +- Change name of connector to `raptor-legacy`. + +## Verifier + +- Add `run-teardown-on-result-mismatch` configuration property to facilitate debugging. + When set to false, temporary tables will not be dropped after checksum failures. + +## SPI + +- Change base package to `io.prestosql.spi`. +- Move connector related classes to package `io.prestosql.spi.connector`. +- Make `ConnectorBucketNodeMap` a top level class. +- Use list instead of map for bucket-to-node mapping. + +:::{note} +These are backwards incompatible changes with the previous SPI. +If you have written a plugin, you will need to update your code +before deploying this release. +::: diff --git a/430/_sources/release/release-301.md.txt b/430/_sources/release/release-301.md.txt new file mode 100644 index 000000000..f7400d34c --- /dev/null +++ b/430/_sources/release/release-301.md.txt @@ -0,0 +1,52 @@ +# Release 301 (31 Jan 2019) + +## General + +- Fix reporting of aggregate input data size stats. ({issue}`100`) +- Add support for role management (see {doc}`/sql/create-role`). Note, using {doc}`/sql/set-role` + requires an up-to-date client library. ({issue}`90`) +- Add `INVOKER` security mode for {doc}`/sql/create-view`. ({issue}`30`) +- Add `ANALYZE` SQL statement for collecting table statistics. ({issue}`99`) +- Add {func}`log` function with arbitrary base. ({issue}`36`) +- Remove the `deprecated.legacy-log-function` configuration option. The legacy behavior + (reverse argument order) for the {func}`log` function is no longer available. ({issue}`36`) +- Remove the `deprecated.legacy-array-agg` configuration option. The legacy behavior + (ignoring nulls) for {func}`array_agg` is no longer available. ({issue}`77`) +- Improve performance of `COALESCE` expressions. ({issue}`35`) +- Improve error message for unsupported {func}`reduce_agg` state type. ({issue}`55`) +- Improve performance of queries involving `SYSTEM` table sampling and computations over the + columns of the sampled table. ({issue}`29`) + +## Server RPM + +- Do not allow uninstalling RPM while server is still running. ({issue}`67`) + +## Security + +- Support LDAP with anonymous bind disabled. ({issue}`97`) + +## Hive connector + +- Add procedure for dumping metastore recording to a file. ({issue}`54`) +- Add Metastore recorder support for Glue. ({issue}`61`) +- Add `hive.temporary-staging-directory-enabled` configuration property and + `temporary_staging_directory_enabled` session property to control whether a temporary staging + directory should be used for write operations. ({issue}`70`) +- Add `hive.temporary-staging-directory-path` configuration property and + `temporary_staging_directory_path` session property to control the location of temporary + staging directory that is used for write operations. The `${USER}` placeholder can be used to + use a different location for each user (e.g., `/tmp/${USER}`). ({issue}`70`) + +## Kafka connector + +- The minimum supported Kafka broker version is now 0.10.0. ({issue}`53`) + +## Base-JDBC connector library + +- Add support for defining procedures. ({issue}`73`) +- Add support for providing table statistics. ({issue}`72`) + +## SPI + +- Include session trace token in `QueryCreatedEvent` and `QueryCompletedEvent`. ({issue}`24`) +- Fix regression in `NodeManager` where node list was not being refreshed on workers. ({issue}`27`) diff --git a/430/_sources/release/release-302.md.txt b/430/_sources/release/release-302.md.txt new file mode 100644 index 000000000..764a49fad --- /dev/null +++ b/430/_sources/release/release-302.md.txt @@ -0,0 +1,54 @@ +# Release 302 (6 Feb 2019) + +## General + +- Fix cluster starvation when wait for minimum number of workers is enabled. ({issue}`155`) +- Fix backup of queries blocked waiting for minimum number of workers. ({issue}`155`) +- Fix failure when preparing statements that contain a quoted reserved word as a table name. ({issue}`80`) +- Fix query failure when spilling is triggered during certain phases of query execution. ({issue}`164`) +- Fix `SHOW CREATE VIEW` output to preserve table name quoting. ({issue}`80`) +- Add {doc}`/connector/elasticsearch`. ({issue}`118`) +- Add support for `boolean` type to {func}`approx_distinct`. ({issue}`82`) +- Add support for boolean columns to `EXPLAIN` with type `IO`. ({issue}`157`) +- Add `SphericalGeography` type and related {doc}`geospatial functions `. ({issue}`166`) +- Remove deprecated system memory pool. ({issue}`168`) +- Improve query performance for certain queries involving `ROLLUP`. ({issue}`105`) + +## CLI + +- Add `--trace-token` option to set the trace token. ({issue}`117`) +- Display spilled data size as part of debug information. ({issue}`161`) + +## Web UI + +- Add spilled data size to query details page. ({issue}`161`) + +## Security + +- Add `http.server.authentication.krb5.principal-hostname` configuration option to set the hostname + for the Kerberos service principal. ({issue}`146`, {issue}`153`) +- Add support for client-provided extra credentials that can be utilized by connectors. ({issue}`124`) + +## Hive connector + +- Fix Parquet predicate pushdown for `smallint`, `tinyint` types. ({issue}`131`) +- Add support for Google Cloud Storage (GCS). Credentials can be provided globally using the + `hive.gcs.json-key-file-path` configuration property, or as a client-provided extra credential + named `hive.gcs.oauth` if the `hive.gcs.use-access-token` configuration property is enabled. ({issue}`124`) +- Allow creating tables with the `external_location` property pointing to an empty S3 directory. ({issue}`75`) +- Reduce GC pressure from Parquet reader by constraining the maximum column read size. ({issue}`58`) +- Reduce network utilization and latency for S3 when reading ORC or Parquet. ({issue}`142`) + +## Kafka connector + +- Fix query failure when reading `information_schema.columns` without an equality condition on `table_name`. ({issue}`120`) + +## Redis connector + +- Fix query failure when reading `information_schema.columns` without an equality condition on `table_name`. ({issue}`120`) + +## SPI + +- Include query peak task user memory in `QueryCreatedEvent` and `QueryCompletedEvent`. ({issue}`163`) +- Include plan node cost and statistics estimates in `QueryCompletedEvent`. ({issue}`134`) +- Include physical and internal network input data size in `QueryCompletedEvent`. ({issue}`133`) diff --git a/430/_sources/release/release-303.md.txt b/430/_sources/release/release-303.md.txt new file mode 100644 index 000000000..c2460b32e --- /dev/null +++ b/430/_sources/release/release-303.md.txt @@ -0,0 +1,43 @@ +# Release 303 (13 Feb 2019) + +## General + +- Fix incorrect padding for `CHAR` values containing Unicode supplementary characters. + Previously, such values would be incorrectly padded with too few spaces. ({issue}`195`) +- Fix an issue where a union of a table with a `VALUES` statement would execute on a + single node, which could lead to out of memory errors. ({issue}`207`) +- Fix `/v1/info` to report started status after all plugins have been registered and initialized. ({issue}`213`) +- Improve performance of window functions by avoiding unnecessary data exchanges over the network. ({issue}`177`) +- Choose the distribution type for semi joins based on cost when the + `join_distribution_type` session property is set to `AUTOMATIC`. ({issue}`160`) +- Expand grouped execution support to window functions, making it possible + to execute them with less peak memory usage. ({issue}`169`) + +## Web UI + +- Add additional details to and improve rendering of live plan. ({issue}`182`) + +## CLI + +- Add `--progress` option to show query progress in batch mode. ({issue}`34`) + +## Hive connector + +- Fix query failure when reading Parquet data with no columns selected. + This affects queries such as `SELECT count(*)`. ({issue}`203`) + +## Mongo connector + +- Fix failure for queries involving joins or aggregations on `ObjectId` type. ({issue}`215`) + +## Base-JDBC connector library + +- Allow customizing how query predicates are pushed down to the underlying database. ({issue}`109`) +- Allow customizing how values are written to the underlying database. ({issue}`109`) + +## SPI + +- Remove deprecated methods `getSchemaName` and `getTableName` from the `SchemaTablePrefix` + class. These were replaced by the `getSchema` and `getTable` methods. ({issue}`89`) +- Remove deprecated variants of methods `listTables` and `listViews` + from the `ConnectorMetadata` class. ({issue}`89`) diff --git a/430/_sources/release/release-304.md.txt b/430/_sources/release/release-304.md.txt new file mode 100644 index 000000000..d0c661980 --- /dev/null +++ b/430/_sources/release/release-304.md.txt @@ -0,0 +1,54 @@ +# Release 304 (27 Feb 2019) + +## General + +- Fix wrong results for queries involving `FULL OUTER JOIN` and `coalesce` expressions + over the join keys. ({issue}`288`) +- Fix failure when a column is referenced using its fully qualified form. ({issue}`250`) +- Correctly report physical and internal network position count for operators. ({issue}`271`) +- Improve plan stability for repeated executions of the same query. ({issue}`226`) +- Remove deprecated `datasources` configuration property. ({issue}`306`) +- Improve error message when a query contains zero-length delimited identifiers. ({issue}`249`) +- Avoid opening an unnecessary HTTP listener on an arbitrary port. ({issue}`239`) +- Add experimental support for spilling for queries involving `ORDER BY` or window functions. ({issue}`228`) + +## Server RPM + +- Preserve modified configuration files when the RPM is uninstalled. ({issue}`267`) + +## Web UI + +- Fix broken timeline view. ({issue}`283`) +- Show data size and position count reported by connectors and by worker-to-worker data transfers + in detailed query view. ({issue}`271`) + +## Hive connector + +- Fix authorization failure when using SQL Standard Based Authorization mode with user identifiers + that contain capital letters. ({issue}`289`) +- Fix wrong results when filtering on the hidden `$bucket` column for tables containing + partitions with different bucket counts. Instead, queries will now fail in this case. ({issue}`286`) +- Record the configured Hive time zone when writing ORC files. ({issue}`212`) +- Use the time zone recorded in ORC files when reading timestamps. + The configured Hive time zone, which was previously always used, is now + used only as a default when the writer did not record the time zone. ({issue}`212`) +- Support Parquet files written with Parquet 1.9+ that use `DELTA_BINARY_PACKED` + encoding with the Parquet `INT64` type. ({issue}`334`) +- Allow setting the retry policy for the Thrift metastore client using the + `hive.metastore.thrift.client.*` configuration properties. ({issue}`240`) +- Reduce file system read operations when reading Parquet file footers. ({issue}`296`) +- Allow ignoring Glacier objects in S3 rather than failing the query. This is + disabled by default, as it may skip data that is expected to exist, but it can + be enabled using the `hive.s3.skip-glacier-objects` configuration property. ({issue}`305`) +- Add procedure `system.sync_partition_metadata()` to synchronize the partitions + in the metastore with the partitions that are physically on the file system. ({issue}`223`) +- Improve performance of ORC reader for columns that only contain nulls. ({issue}`229`) + +## PostgreSQL connector + +- Map PostgreSQL `json` and `jsonb` types to Presto `json` type. ({issue}`81`) + +## Cassandra connector + +- Support queries over tables containing partitioning columns of any type. ({issue}`252`) +- Support `smallint`, `tinyint` and `date` Cassandra types. ({issue}`141`) diff --git a/430/_sources/release/release-305.md.txt b/430/_sources/release/release-305.md.txt new file mode 100644 index 000000000..a78e7c7b7 --- /dev/null +++ b/430/_sources/release/release-305.md.txt @@ -0,0 +1,46 @@ +# Release 305 (7 Mar 2019) + +## General + +- Fix failure of {doc}`/functions/regexp` for certain patterns and inputs + when using the default `JONI` library. ({issue}`350`) +- Fix a rare `ClassLoader` related problem for plugins providing an `EventListenerFactory`. ({issue}`299`) +- Expose `join_max_broadcast_table_size` session property, which was previously hidden. ({issue}`346`) +- Improve performance of queries when spill is enabled but not triggered. ({issue}`315`) +- Consider estimated query peak memory when making cost based decisions. ({issue}`247`) +- Include revocable memory in total memory stats. ({issue}`273`) +- Add peak revocable memory to operator stats. ({issue}`273`) +- Add {func}`ST_Points` function to access vertices of a linestring. ({issue}`316`) +- Add a system table `system.metadata.analyze_properties` + to list all {doc}`/sql/analyze` properties. ({issue}`376`) + +## Resource groups + +- Fix resource group selection when selector uses regular expression variables. ({issue}`373`) + +## Web UI + +- Display peak revocable memory, current total memory, + and peak total memory in detailed query view. ({issue}`273`) + +## CLI + +- Add option to output CSV without quotes. ({issue}`319`) + +## Hive connector + +- Fix handling of updated credentials for Google Cloud Storage (GCS). ({issue}`398`) +- Fix calculation of bucket number for timestamps that contain a non-zero + milliseconds value. Previously, data would be written into the wrong bucket, + or could be incorrectly skipped on read. ({issue}`366`) +- Allow writing ORC files compatible with Hive 2.0.0 to 2.2.0 by identifying + the writer as an old version of Hive (rather than Presto) in the files. + This can be enabled using the `hive.orc.writer.use-legacy-version-number` + configuration property. ({issue}`353`) +- Support dictionary filtering for Parquet v2 files using `RLE_DICTIONARY` encoding. ({issue}`251`) +- Remove legacy writers for ORC and RCFile. ({issue}`353`) +- Remove support for the DWRF file format. ({issue}`353`) + +## Base-JDBC connector library + +- Allow access to extra credentials when opening a JDBC connection. ({issue}`281`) diff --git a/430/_sources/release/release-306.md.txt b/430/_sources/release/release-306.md.txt new file mode 100644 index 000000000..3a3d57875 --- /dev/null +++ b/430/_sources/release/release-306.md.txt @@ -0,0 +1,67 @@ +# Release 306 (16 Mar 2019) + +## General + +- Fix planning failure for queries containing a `LIMIT` after a global + aggregation. ({issue}`437`) +- Fix missing column types in `EXPLAIN` output. ({issue}`328`) +- Fix accounting of peak revocable memory reservation. ({issue}`413`) +- Fix double memory accounting for aggregations when spilling is active. ({issue}`413`) +- Fix excessive CPU usage that can occur when spilling for window functions. ({issue}`468`) +- Fix incorrect view name displayed by `SHOW CREATE VIEW`. ({issue}`433`) +- Allow specifying `NOT NULL` when creating tables or adding columns. ({issue}`418`) +- Add a config option (`query.stage-count-warning-threshold`) to specify a + per-query threshold for the number of stages. When this threshold is exceeded, + a `TOO_MANY_STAGES` warning is raised. ({issue}`330`) +- Support session property values with special characters (e.g., comma or equals sign). ({issue}`407`) +- Remove the `deprecated.legacy-unnest-array-rows` configuration option. + The legacy behavior for `UNNEST` of arrays containing `ROW` values is no + longer supported. ({issue}`430`) +- Remove the `deprecated.legacy-row-field-ordinal-access` configuration option. + The legacy mechanism for accessing fields of anonymous `ROW` types is no longer + supported. ({issue}`428`) +- Remove the `deprecated.group-by-uses-equal` configuration option. The legacy equality + semantics for `GROUP BY` are not longer supported. ({issue}`432`) +- Remove the `deprecated.legacy-map-subscript`. The legacy behavior for the map subscript + operator on missing keys is no longer supported. ({issue}`429`) +- Remove the `deprecated.legacy-char-to-varchar-coercion` configuration option. The + legacy coercion rules between `CHAR` and `VARCHAR` types are no longer + supported. ({issue}`431`) +- Remove deprecated `distributed_join` system property. Use `join_distribution_type` + instead. ({issue}`452`) + +## Hive connector + +- Fix calling procedures immediately after startup, before any other queries are run. + Previously, the procedure call would fail and also cause all subsequent Hive queries + to fail. ({issue}`414`) +- Improve ORC reader performance for decoding `REAL` and `DOUBLE` types. ({issue}`465`) + +## MySQL connector + +- Allow creating or renaming tables, and adding, renaming, or dropping columns. ({issue}`418`) + +## PostgreSQL connector + +- Fix predicate pushdown for PostgreSQL `ENUM` type. ({issue}`408`) +- Allow creating or renaming tables, and adding, renaming, or dropping columns. ({issue}`418`) + +## Redshift connector + +- Allow creating or renaming tables, and adding, renaming, or dropping columns. ({issue}`418`) + +## SQL Server connector + +- Allow creating or renaming tables, and adding, renaming, or dropping columns. ({issue}`418`) + +## Base-JDBC connector library + +- Allow mapping column type to Presto type based on `Block`. ({issue}`454`) + +## SPI + +- Deprecate Table Layout APIs. Connectors can opt out of the legacy behavior by implementing + `ConnectorMetadata.usesLegacyTableLayouts()`. ({issue}`420`) +- Add support for limit pushdown into connectors via the `ConnectorMetadata.applyLimit()` + method. ({issue}`421`) +- Add time spent waiting for resources to `QueryCompletedEvent`. ({issue}`461`) diff --git a/430/_sources/release/release-307.md.txt b/430/_sources/release/release-307.md.txt new file mode 100644 index 000000000..789b688c3 --- /dev/null +++ b/430/_sources/release/release-307.md.txt @@ -0,0 +1,64 @@ +# Release 307 (3 Apr 2019) + +## General + +- Fix cleanup of spill files for queries using window functions or `ORDER BY`. ({issue}`543`) +- Optimize queries containing `ORDER BY` together with `LIMIT` over an `OUTER JOIN` + by pushing `ORDER BY` and `LIMIT` to the outer side of the join. ({issue}`419`) +- Improve performance of table scans for data sources that produce tiny pages. ({issue}`467`) +- Improve performance of `IN` subquery expressions that contain a `DISTINCT` clause. ({issue}`551`) +- Expand support of types handled in `EXPLAIN (TYPE IO)`. ({issue}`509`) +- Add support for outer joins involving lateral derived tables (i.e., `LATERAL`). ({issue}`390`) +- Add support for setting table comments via the {doc}`/sql/comment` syntax. ({issue}`200`) + +## Web UI + +- Allow UI to work when opened as `/ui` (no trailing slash). ({issue}`500`) + +## Security + +- Make query result and cancellation URIs secure. Previously, an authenticated + user could potentially steal the result data of any running query. ({issue}`561`) + +## Server RPM + +- Prevent JVM from allocating large amounts of native memory. The new configuration is applied + automatically when Presto is installed from RPM. When Presto is installed another way, or when + you provide your own `jvm.config`, we recommend adding `-Djdk.nio.maxCachedBufferSize=2000000` + to your `jvm.config`. See {doc}`/installation/deployment` for details. ({issue}`542`) + +## CLI + +- Always abort query in batch mode when CLI is killed. ({issue}`508`, {issue}`580`) + +## JDBC driver + +- Abort query synchronously when the `ResultSet` is closed or when the + `Statement` is cancelled. Previously, the abort was sent in the background, + allowing the JVM to exit before the abort was received by the server. ({issue}`580`) + +## Hive connector + +- Add safety checks for Hive bucketing version. Hive 3.0 introduced a new + bucketing version that uses an incompatible hash function. The Hive connector + will treat such tables as not bucketed when reading and disallows writing. ({issue}`512`) +- Add support for setting table comments via the {doc}`/sql/comment` syntax. ({issue}`200`) + +## Other connectors + +These changes apply to the MySQL, PostgreSQL, Redshift, and SQL Server connectors. + +- Fix reading and writing of `timestamp` values. Previously, an incorrect value + could be read, depending on the Presto JVM time zone. ({issue}`495`) +- Add support for using a client-provided username and password. The credential + names can be configured using the `user-credential-name` and `password-credential-name` + configuration properties. ({issue}`482`) + +## SPI + +- `LongDecimalType` and `IpAddressType` now use `Int128ArrayBlock` instead + of `FixedWithBlock`. Any code that creates blocks directly, rather than using + the `BlockBuilder` returned from the `Type`, will need to be updated. ({issue}`492`) +- Remove `FixedWidthBlock`. Use one of the `*ArrayBlock` classes instead. ({issue}`492`) +- Add support for simple constraint pushdown into connectors via the + `ConnectorMetadata.applyFilter()` method. ({issue}`541`) diff --git a/430/_sources/release/release-308.md.txt b/430/_sources/release/release-308.md.txt new file mode 100644 index 000000000..b2e60d237 --- /dev/null +++ b/430/_sources/release/release-308.md.txt @@ -0,0 +1,56 @@ +# Release 308 (11 Apr 2019) + +## General + +- Fix a regression that prevented the server from starting on Java 9+. ({issue}`610`) +- Fix correctness issue for queries involving `FULL OUTER JOIN` and `coalesce`. ({issue}`622`) + +## Security + +- Add authorization for listing table columns. ({issue}`507`) + +## CLI + +- Add option for specifying Kerberos service principal pattern. ({issue}`597`) + +## JDBC driver + +- Correctly report precision and column display size in `ResultSetMetaData` + for `char` and `varchar` columns. ({issue}`615`) +- Add option for specifying Kerberos service principal pattern. ({issue}`597`) + +## Hive connector + +- Fix regression that could cause queries to fail with `Query can potentially + read more than X partitions` error. ({issue}`619`) +- Improve ORC read performance significantly. For TPC-DS, this saves about 9.5% of + total CPU when running over gzip-compressed data. ({issue}`555`) +- Require access to a table (any privilege) in order to list the columns. ({issue}`507`) +- Add directory listing cache for specific tables. The list of tables is specified + using the `hive.file-status-cache-tables` configuration property. ({issue}`343`) + +## MySQL connector + +- Fix `ALTER TABLE ... RENAME TO ...` statement. ({issue}`586`) +- Push simple `LIMIT` queries into the external database. ({issue}`589`) + +## PostgreSQL connector + +- Push simple `LIMIT` queries into the external database. ({issue}`589`) + +## Redshift connector + +- Push simple `LIMIT` queries into the external database. ({issue}`589`) + +## SQL Server connector + +- Fix writing `varchar` values with non-Latin characters in `CREATE TABLE AS`. ({issue}`573`) +- Support writing `varchar` and `char` values with length longer than 4000 + characters in `CREATE TABLE AS`. ({issue}`573`) +- Support writing `boolean` values in `CREATE TABLE AS`. ({issue}`573`) +- Push simple `LIMIT` queries into the external database. ({issue}`589`) + +## Elasticsearch connector + +- Add support for Search Guard in Elasticsearch connector. Please refer to {doc}`/connector/elasticsearch` + for the relevant configuration properties. ({issue}`438`) diff --git a/430/_sources/release/release-309.md.txt b/430/_sources/release/release-309.md.txt new file mode 100644 index 000000000..62244a0c2 --- /dev/null +++ b/430/_sources/release/release-309.md.txt @@ -0,0 +1,63 @@ +# Release 309 (25 Apr 2019) + +## General + +- Fix incorrect match result for {doc}`/functions/regexp` when pattern ends + with a word boundary matcher. This only affects the default `JONI` library. + ({issue}`661`) +- Fix failures for queries involving spatial joins. ({issue}`652`) +- Add support for `SphericalGeography` to {func}`ST_Area()`. ({issue}`383`) + +## Security + +- Add option for specifying the Kerberos GSS name type. ({issue}`645`) + +## Server RPM + +- Update default JVM configuration to recommended settings (see {doc}`/installation/deployment`). + ({issue}`642`) + +## Hive connector + +- Fix rare failure when reading `DECIMAL` values from ORC files. ({issue}`664`) +- Add a hidden `$properties` table for each table that describes its Hive table + properties. For example, a table named `example` will have an associated + properties table named `example$properties`. ({issue}`268`) + +## MySQL connector + +- Match schema and table names case insensitively. This behavior can be enabled by setting + the `case-insensitive-name-matching` catalog configuration option to true. ({issue}`614`) + +## PostgreSQL connector + +- Add support for `ARRAY` type. ({issue}`317`) +- Add support writing `TINYINT` values. ({issue}`317`) +- Match schema and table names case insensitively. This behavior can be enabled by setting + the `case-insensitive-name-matching` catalog configuration option to true. ({issue}`614`) + +## Redshift connector + +- Match schema and table names case insensitively. This behavior can be enabled by setting + the `case-insensitive-name-matching` catalog configuration option to true. ({issue}`614`) + +## SQL Server connector + +- Match schema and table names case insensitively. This behavior can be enabled by setting + the `case-insensitive-name-matching` catalog configuration option to true. ({issue}`614`) + +## Cassandra connector + +- Allow reading from tables which have Cassandra column types that are not supported by Presto. + These columns will not be visible in Presto. ({issue}`592`) + +## SPI + +- Add session parameter to the `applyFilter()` and `applyLimit()` methods in + `ConnectorMetadata`. ({issue}`636`) + +:::{note} +This is a backwards incompatible changes with the previous SPI. +If you have written a connector that implements these methods, +you will need to update your code before deploying this release. +::: diff --git a/430/_sources/release/release-310.md.txt b/430/_sources/release/release-310.md.txt new file mode 100644 index 000000000..63ab934e9 --- /dev/null +++ b/430/_sources/release/release-310.md.txt @@ -0,0 +1,37 @@ +# Release 310 (3 May 2019) + +## General + +- Reduce compilation failures for expressions over types containing an extremely + large number of nested types. ({issue}`537`) +- Fix error reporting when query fails with due to running out of memory. ({issue}`696`) +- Improve performance of `JOIN` queries involving join keys of different types. + ({issue}`665`) +- Add initial and experimental support for late materialization. + This feature can be enabled via `experimental.work-processor-pipelines` + feature config or via `work_processor_pipelines` session config. + Simple select queries of type `SELECT ... FROM table ORDER BY cols LIMIT n` can + experience significant CPU and performance improvement. ({issue}`602`) +- Add support for `FETCH FIRST` syntax. ({issue}`666`) + +## CLI + +- Make the final query time consistent with query stats. ({issue}`692`) + +## Hive connector + +- Ignore boolean column statistics when the count is `-1`. ({issue}`241`) +- Prevent failures for `information_schema` queries when a table has an invalid + storage format. ({issue}`568`) +- Add support for assuming AWS role when accessing S3 or Glue. ({issue}`698`) +- Add support for coercions between `DECIMAL`, `DOUBLE`, and `REAL` for + partition and table schema mismatch. ({issue}`352`) +- Fix typo in Metastore recorder duration property name. ({issue}`711`) + +## PostgreSQL connector + +- Support for the `ARRAY` type has been disabled by default. ({issue}`687`) + +## Blackhole connector + +- Support having tables with same name in different Blackhole schemas. ({issue}`550`) diff --git a/430/_sources/release/release-311.md.txt b/430/_sources/release/release-311.md.txt new file mode 100644 index 000000000..14add699e --- /dev/null +++ b/430/_sources/release/release-311.md.txt @@ -0,0 +1,31 @@ +# Release 311 (14 May 2019) + +## General + +- Fix incorrect results for aggregation query that contains a `HAVING` clause but no + `GROUP BY` clause. ({issue}`733`) +- Fix rare error when moving already completed query to a new memory pool. ({issue}`725`) +- Fix leak in operator peak memory computations ({issue}`764`) +- Improve consistency of reported query statistics. ({issue}`773`) +- Add support for `OFFSET` syntax. ({issue}`732`) +- Print cost metrics using appropriate units in the output of `EXPLAIN`. ({issue}`68`) +- Add {func}`combinations` function. ({issue}`714`) + +## Hive connector + +- Add support for static AWS credentials for the Glue metastore. ({issue}`748`) + +## Cassandra connector + +- Support collections nested in other collections. ({issue}`657`) +- Automatically discover the Cassandra protocol version when the previously required + `cassandra.protocol-version` configuration property is not set. ({issue}`596`) + +## Black Hole connector + +- Fix rendering of tables and columns in plans. ({issue}`728`) +- Add table and column statistics. ({issue}`728`) + +## System connector + +- Add `system.metadata.table_comments` table that contains table comments. ({issue}`531`) diff --git a/430/_sources/release/release-312.md.txt b/430/_sources/release/release-312.md.txt new file mode 100644 index 000000000..006c0d4b8 --- /dev/null +++ b/430/_sources/release/release-312.md.txt @@ -0,0 +1,65 @@ +# Release 312 (29 May 2019) + +## General + +- Fix incorrect results for queries using `IS [NOT] DISTINCT FROM`. ({issue}`795`) +- Fix `array_distinct`, `array_intersect` semantics with respect to indeterminate + values (i.e., `NULL` or structural types containing `NULL`). ({issue}`559`) +- Fix failure when the largest negative `BIGINT` value (`-9223372036854775808`) is used + as a constant in a query. ({issue}`805`) +- Improve reliability for network errors when using Kerberos with + {doc}`/security/internal-communication`. ({issue}`838`) +- Improve performance of `JOIN` queries involving inline tables (`VALUES`). ({issue}`743`) +- Improve performance of queries containing duplicate expressions. ({issue}`730`) +- Improve performance of queries involving comparisons between values of different types. ({issue}`731`) +- Improve performance of queries containing redundant `ORDER BY` clauses in subqueries. This may + affect the semantics of queries that incorrectly rely on implementation-specific behavior. The + old behavior can be restored via the `skip_redundant_sort` session property or the + `optimizer.skip-redundant-sort` configuration property. ({issue}`818`) +- Improve performance of `IN` predicates that contain subqueries. ({issue}`767`) +- Improve support for correlated subqueries containing redundant `LIMIT` clauses. ({issue}`441`) +- Add a new {ref}`uuid-type` type to represent UUIDs. ({issue}`755`) +- Add {func}`uuid` function to generate random UUIDs. ({issue}`786`) +- Add {doc}`/connector/phoenix`. ({issue}`672`) +- Make semantic error name available in client protocol. ({issue}`790`) +- Report operator statistics when `experimental.work-processor-pipelines` + is enabled. ({issue}`788`) + +## Server + +- Raise required Java version to 8u161. This version allows unlimited strength crypto. ({issue}`779`) +- Show JVM configuration hint when JMX agent fails to start on Java 9+. ({issue}`838`) +- Skip starting JMX agent on Java 9+ if it is already configured via JVM properties. ({issue}`838`) +- Support configuring TrustStore for {doc}`/security/internal-communication` using the + `internal-communication.https.truststore.path` and `internal-communication.https.truststore.key` + configuration properties. The path can point at a Java KeyStore or a PEM file. ({issue}`785`) +- Remove deprecated check for minimum number of workers before starting a coordinator. Use the + `query-manager.required-workers` and `query-manager.required-workers-max-wait` configuration + properties instead. ({issue}`95`) + +## Hive connector + +- Fix `SHOW GRANTS` failure when metastore contains few tables. ({issue}`791`) +- Fix failure reading from `information_schema.table_privileges` table when metastore + contains few tables. ({issue}`791`) +- Use Hive naming convention for file names when writing to bucketed tables. ({issue}`822`) +- Support new Hive bucketing conventions by allowing any number of files per bucket. + This allows reading from partitions that were inserted into multiple times by Hive, + or were written to by Hive on Tez (which does not create files for empty buckets). +- Allow disabling the creation of files for empty buckets when writing data. + This behavior is enabled by default for compatibility with previous versions of Presto, + but can be disabled using the `hive.create-empty-bucket-files` configuration property + or the `create_empty_bucket_files` session property. ({issue}`822`) + +## MySQL connector + +- Map MySQL `json` type to Presto `json` type. ({issue}`824`) + +## PostgreSQL connector + +- Add support for PostgreSQL's `TIMESTAMP WITH TIME ZONE` data type. ({issue}`640`) + +## SPI + +- Add support for pushing `TABLESAMPLE` into connectors via the + `ConnectorMetadata.applySample()` method. ({issue}`753`) diff --git a/430/_sources/release/release-313.md.txt b/430/_sources/release/release-313.md.txt new file mode 100644 index 000000000..a191d7629 --- /dev/null +++ b/430/_sources/release/release-313.md.txt @@ -0,0 +1,20 @@ +# Release 313 (31 May 2019) + +## General + +- Fix leak in operator peak memory computations. ({issue}`843`) +- Fix incorrect results for queries involving `GROUPING SETS` and `LIMIT`. ({issue}`864`) +- Add compression and encryption support for {doc}`/admin/spill`. ({issue}`778`) + +## CLI + +- Fix failure when selecting a value of type {ref}`uuid-type`. ({issue}`854`) + +## JDBC driver + +- Fix failure when selecting a value of type {ref}`uuid-type`. ({issue}`854`) + +## Phoenix connector + +- Allow matching schema and table names case insensitively. This can be enabled by setting + the `case-insensitive-name-matching` configuration property to true. ({issue}`872`) diff --git a/430/_sources/release/release-314.md.txt b/430/_sources/release/release-314.md.txt new file mode 100644 index 000000000..1298f9b29 --- /dev/null +++ b/430/_sources/release/release-314.md.txt @@ -0,0 +1,64 @@ +# Release 314 (7 Jun 2019) + +## General + +- Fix incorrect results for `BETWEEN` involving `NULL` values. ({issue}`877`) +- Fix query history leak in coordinator. ({issue}`939`, {issue}`944`) +- Fix idle client timeout handling. ({issue}`947`) +- Improve performance of {func}`json_parse` function. ({issue}`904`) +- Visualize plan structure in `EXPLAIN` output. ({issue}`888`) +- Add support for positional access to `ROW` fields via the subscript + operator. ({issue}`860`) + +## CLI + +- Add JSON output format. ({issue}`878`) + +## Web UI + +- Fix queued queries counter in UI. ({issue}`894`) + +## Server RPM + +- Change default location of the `http-request.log` to `/var/log/presto`. Previously, + the log would be located in `/var/lib/presto/data/var/log` by default. ({issue}`919`) + +## Hive connector + +- Fix listing tables and views from Hive 2.3+ Metastore on certain databases, + including Derby and Oracle. This fixes `SHOW TABLES`, `SHOW VIEWS` and + reading from `information_schema.tables` table. ({issue}`833`) +- Fix handling of Avro tables with `avro.schema.url` defined in Hive + `SERDEPROPERTIES`. ({issue}`898`) +- Fix regression that caused ORC bloom filters to be ignored. ({issue}`921`) +- Add support for reading LZ4 and ZSTD compressed Parquet data. ({issue}`910`) +- Add support for writing ZSTD compressed ORC data. ({issue}`910`) +- Add support for configuring ZSTD and LZ4 as default compression methods via the + `hive.compression-codec` configuration option. ({issue}`910`) +- Do not allow inserting into text format tables that have a header or footer. ({issue}`891`) +- Add `textfile_skip_header_line_count` and `textfile_skip_footer_line_count` table properties + for text format tables that specify the number of header and footer lines. ({issue}`845`) +- Add `hive.max-splits-per-second` configuration property to allow throttling + the split discovery rate, which can reduce load on the file system. ({issue}`534`) +- Support overwriting unpartitioned tables for insert queries. ({issue}`924`) + +## PostgreSQL connector + +- Support PostgreSQL arrays declared using internal type + name, for example `_int4` (rather than `int[]`). ({issue}`659`) + +## Elasticsearch connector + +- Add support for mixed-case field names. ({issue}`887`) + +## Base-JDBC connector library + +- Allow connectors to customize how they store `NULL` values. ({issue}`918`) + +## SPI + +- Expose the SQL text of the executed prepared statement to `EventListener`. ({issue}`908`) +- Deprecate table layouts for `ConnectorMetadata.makeCompatiblePartitioning()`. ({issue}`689`) +- Add support for delete pushdown into connectors via the `ConnectorMetadata.applyDelete()` + and `ConnectorMetadata.executeDelete()` methods. ({issue}`689`) +- Allow connectors without distributed tables. ({issue}`893`) diff --git a/430/_sources/release/release-315.md.txt b/430/_sources/release/release-315.md.txt new file mode 100644 index 000000000..b1a9ed1a1 --- /dev/null +++ b/430/_sources/release/release-315.md.txt @@ -0,0 +1,50 @@ +# Release 315 (14 Jun 2019) + +## General + +- Fix incorrect results when dividing certain decimal numbers. ({issue}`958`) +- Add support for `FETCH FIRST ... WITH TIES` syntax. ({issue}`832`) +- Add locality awareness to default split scheduler. ({issue}`680`) +- Add {func}`format` function. ({issue}`548`) + +## Server RPM + +- Require JDK version 8u161+ during installation, which is the version the server requires. ({issue}`983`) + +## CLI + +- Fix alignment of nulls for numeric columns in aligned output format. ({issue}`871`) + +## Hive connector + +- Fix regression in partition pruning for certain query shapes. ({issue}`984`) +- Correctly identify EMRFS as S3 when deciding to use a temporary location for writes. ({issue}`935`) +- Allow creating external tables on S3 even if the location does not exist. ({issue}`935`) +- Add support for UTF-8 ORC bloom filters. ({issue}`914`) +- Add support for `DATE`, `TIMESTAMP` and `REAL` in ORC bloom filters. ({issue}`967`) +- Disable usage of old, non UTF-8, ORC bloom filters for `VARCHAR` and `CHAR`. ({issue}`914`) +- Allow logging all calls to Hive Thrift metastore service. This can be enabled + by turning on `DEBUG` logging for + `io.prestosql.plugin.hive.metastore.thrift.ThriftHiveMetastoreClient`. ({issue}`946`) + +## MongoDB connector + +- Fix query failure when `ROW` with an `ObjectId` field is used as a join key. ({issue}`933`) +- Add cast from `ObjectId` to `VARCHAR`. ({issue}`933`) + +## SPI + +- Allow connectors to provide view definitions. `ConnectorViewDefinition` now contains + the real view definition rather than an opaque blob. Connectors that support view storage + can use the JSON representation of that class as a stable storage format. The JSON + representation is the same as the previous opaque blob, thus all existing view + definitions will continue to work. ({issue}`976`) +- Add `getView()` method to `ConnectorMetadata` as a replacement for `getViews()`. + The `getViews()` method now exists only as an optional method for connectors that + can efficiently support bulk retrieval of views and has a different signature. ({issue}`976`) + +:::{note} +These are backwards incompatible changes with the previous SPI. +If you have written a connector that supports views, you will +need to update your code before deploying this release. +::: diff --git a/430/_sources/release/release-316.md.txt b/430/_sources/release/release-316.md.txt new file mode 100644 index 000000000..3d6bddbf8 --- /dev/null +++ b/430/_sources/release/release-316.md.txt @@ -0,0 +1,54 @@ +# Release 316 (8 Jul 2019) + +## General + +- Fix `date_format` function failure when format string contains non-ASCII + characters. ({issue}`1056`) +- Improve performance of queries using `UNNEST`. ({issue}`901`) +- Improve error message when statement parsing fails. ({issue}`1042`) + +## CLI + +- Fix refresh of completion cache when catalog or schema is changed. ({issue}`1016`) +- Allow reading password from console when stdout is a pipe. ({issue}`982`) + +## Hive connector + +- Acquire S3 credentials from the default AWS locations if not configured explicitly. ({issue}`741`) +- Only allow using roles and grants with SQL standard based authorization. ({issue}`972`) +- Add support for `CSV` file format. ({issue}`920`) +- Support reading from and writing to Hadoop encryption zones (Hadoop KMS). ({issue}`997`) +- Collect column statistics on write by default. This can be disabled using the + `hive.collect-column-statistics-on-write` configuration property or the + `collect_column_statistics_on_write` session property. ({issue}`981`) +- Eliminate unused idle threads when using the metastore cache. ({issue}`1061`) + +## PostgreSQL connector + +- Add support for columns of type `UUID`. ({issue}`1011`) +- Export JMX statistics for various JDBC and connector operations. ({issue}`906`). + +## MySQL connector + +- Export JMX statistics for various JDBC and connector operations. ({issue}`906`). + +## Redshift connector + +- Export JMX statistics for various JDBC and connector operations. ({issue}`906`). + +## SQL Server connector + +- Export JMX statistics for various JDBC and connector operations. ({issue}`906`). + +## TPC-H connector + +- Fix `SHOW TABLES` failure when used with a hidden schema. ({issue}`1005`) + +## TPC-DS connector + +- Fix `SHOW TABLES` failure when used with a hidden schema. ({issue}`1005`) + +## SPI + +- Add support for pushing simple column and row field reference expressions into + connectors via the `ConnectorMetadata.applyProjection()` method. ({issue}`676`) diff --git a/430/_sources/release/release-317.md.txt b/430/_sources/release/release-317.md.txt new file mode 100644 index 000000000..03b29012d --- /dev/null +++ b/430/_sources/release/release-317.md.txt @@ -0,0 +1,61 @@ +# Release 317 (1 Aug 2019) + +## General + +- Fix {func}`url_extract_parameter` when the query string contains an encoded `&` or `=` character. +- Export MBeans from the `db` resource group configuration manager. ({issue}`1151`) +- Add {func}`all_match`, {func}`any_match`, and {func}`none_match` functions. ({issue}`1045`) +- Add support for fractional weights in {func}`approx_percentile`. ({issue}`1168`) +- Add support for node dynamic filtering for semi-joins and filters when the experimental + WorkProcessor pipelines feature is enabled. ({issue}`1075`, {issue}`1155`, {issue}`1119`) +- Allow overriding session time zone for clients via the + `sql.forced-session-time-zone` configuration property. ({issue}`1164`) + +## Web UI + +- Fix tooltip visibility on stage performance details page. ({issue}`1113`) +- Add planning time to query details page. ({issue}`1115`) + +## Security + +- Allow schema owner to create, drop, and rename schema when using file-based + connector access control. ({issue}`1139`) +- Allow respecting the `X-Forwarded-For` header when retrieving the IP address + of the client submitting the query. This information is available in the + `remoteClientAddress` field of the `QueryContext` class for query events. + The behavior can be controlled via the `dispatcher.forwarded-header` + configuration property, as the header should only be used when the Presto + coordinator is behind a proxy. ({issue}`1033`) + +## JDBC driver + +- Fix `DatabaseMetaData.getURL()` to include the `jdbc:` prefix. ({issue}`1211`) + +## Elasticsearch connector + +- Add support for nested fields. ({issue}`1001`) + +## Hive connector + +- Fix bucketing version safety check to correctly disallow writes + to tables that use an unsupported bucketing version. ({issue}`1199`) +- Fix metastore error handling when metastore debug logging is enabled. ({issue}`1152`) +- Improve performance of file listings in `system.sync_partition_metadata` procedure, + especially for S3. ({issue}`1093`) + +## Kudu connector + +- Update Kudu client library version to `1.10.0`. ({issue}`1086`) + +## MongoDB connector + +- Allow passwords to contain the `:` or `@` characters. ({issue}`1094`) + +## PostgreSQL connector + +- Add support for reading `hstore` data type. ({issue}`1101`) + +## SPI + +- Allow delete to be implemented for non-legacy connectors. ({issue}`1015`) +- Remove deprecated method from `ConnectorPageSourceProvider`. ({issue}`1095`) diff --git a/430/_sources/release/release-318.md.txt b/430/_sources/release/release-318.md.txt new file mode 100644 index 000000000..5c3025bbb --- /dev/null +++ b/430/_sources/release/release-318.md.txt @@ -0,0 +1,97 @@ +# Release 318 (26 Aug 2019) + +## General + +- Fix query failure when using `DISTINCT FROM` with the `UUID` or + `IPADDRESS` types. ({issue}`1180`) +- Improve query performance when `optimize_hash_generation` is enabled. ({issue}`1071`) +- Improve performance of information schema tables. ({issue}`999`, {issue}`1306`) +- Rename `http.server.authentication.*` configuration options to `http-server.authentication.*`. ({issue}`1270`) +- Change query CPU tracking for resource groups to update periodically while + the query is running. Previously, CPU usage would only update at query + completion. This improves resource management fairness when using + CPU-limited resource groups. ({issue}`1128`) +- Remove `distributed_planning_time_ms` column from `system.runtime.queries`. ({issue}`1084`) +- Add support for `Asia/Qostanay` time zone. ({issue}`1221`) +- Add session properties that allow overriding the query per-node memory limits: + `query_max_memory_per_node` and `query_max_total_memory_per_node`. These properties + can be used to decrease limits for a query, but not to increase them. ({issue}`1212`) +- Add {doc}`/connector/googlesheets`. ({issue}`1030`) +- Add `planning_time_ms` column to the `system.runtime.queries` table that shows + the time spent on query planning. This is the same value that used to be in the + `analysis_time_ms` column, which was a misnomer. ({issue}`1084`) +- Add {func}`last_day_of_month` function. ({issue}`1295`) +- Add support for cancelling queries via the `system.runtime.kill_query` procedure when + they are in the queue or in the semantic analysis stage. ({issue}`1079`) +- Add queries that are in the queue or in the semantic analysis stage to the + `system.runtime.queries` table. ({issue}`1079`) + +## Web UI + +- Display information about queries that are in the queue or in the semantic analysis + stage. ({issue}`1079`) +- Add support for cancelling queries that are in the queue or in the semantic analysis + stage. ({issue}`1079`) + +## Hive connector + +- Fix query failure due to missing credentials while writing empty bucket files. ({issue}`1298`) +- Fix bucketing of `NaN` values of `real` type. Previously `NaN` values + could be assigned a wrong bucket. ({issue}`1336`) +- Fix reading `RCFile` collection delimiter set by Hive version earlier than 3.0. ({issue}`1321`) +- Return proper error when selecting `"$bucket"` column from a table using + Hive bucketing v2. ({issue}`1336`) +- Improve performance of S3 object listing. ({issue}`1232`) +- Improve performance when reading data from GCS. ({issue}`1200`) +- Add support for reading data from S3 Requester Pays buckets. This can be enabled + using the `hive.s3.requester-pays.enabled` configuration property. ({issue}`1241`) +- Allow inserting into bucketed, unpartitioned tables. ({issue}`1127`) +- Allow inserting into existing partitions of bucketed, partitioned tables. ({issue}`1347`) + +## PostgreSQL connector + +- Add support for providing JDBC credential in a separate file. This can be enabled by + setting the `credential-provider.type=FILE` and `connection-credential-file` + config options in the catalog properties file. ({issue}`1124`) +- Allow logging all calls to `JdbcClient`. This can be enabled by turning + on `DEBUG` logging for `io.prestosql.plugin.jdbc.JdbcClient`. ({issue}`1274`) +- Add possibility to force mapping of certain types to `varchar`. This can be enabled + by setting `jdbc-types-mapped-to-varchar` to comma-separated list of type names. ({issue}`186`) +- Add support for PostgreSQL `timestamp[]` type. ({issue}`1023`, {issue}`1262`, {issue}`1328`) + +## MySQL connector + +- Add support for providing JDBC credential in a separate file. This can be enabled by + setting the `credential-provider.type=FILE` and `connection-credential-file` + config options in the catalog properties file. ({issue}`1124`) +- Allow logging all calls to `JdbcClient`. This can be enabled by turning + on `DEBUG` logging for `io.prestosql.plugin.jdbc.JdbcClient`. ({issue}`1274`) +- Add possibility to force mapping of certain types to `varchar`. This can be enabled + by setting `jdbc-types-mapped-to-varchar` to comma-separated list of type names. ({issue}`186`) + +## Redshift connector + +- Add support for providing JDBC credential in a separate file. This can be enabled by + setting the `credential-provider.type=FILE` and `connection-credential-file` + config options in the catalog properties file. ({issue}`1124`) +- Allow logging all calls to `JdbcClient`. This can be enabled by turning + on `DEBUG` logging for `io.prestosql.plugin.jdbc.JdbcClient`. ({issue}`1274`) +- Add possibility to force mapping of certain types to `varchar`. This can be enabled + by setting `jdbc-types-mapped-to-varchar` to comma-separated list of type names. ({issue}`186`) + +## SQL Server connector + +- Add support for providing JDBC credential in a separate file. This can be enabled by + setting the `credential-provider.type=FILE` and `connection-credential-file` + config options in the catalog properties file. ({issue}`1124`) +- Allow logging all calls to `JdbcClient`. This can be enabled by turning + on `DEBUG` logging for `io.prestosql.plugin.jdbc.JdbcClient`. ({issue}`1274`) +- Add possibility to force mapping of certain types to `varchar`. This can be enabled + by setting `jdbc-types-mapped-to-varchar` to comma-separated list of type names. ({issue}`186`) + +## SPI + +- Add `Block.isLoaded()` method. ({issue}`1216`) +- Update security APIs to accept the new `ConnectorSecurityContext` + and `SystemSecurityContext` classes. ({issue}`171`) +- Allow connectors to override minimal schedule split batch size. ({issue}`1251`) diff --git a/430/_sources/release/release-319.md.txt b/430/_sources/release/release-319.md.txt new file mode 100644 index 000000000..fde505d29 --- /dev/null +++ b/430/_sources/release/release-319.md.txt @@ -0,0 +1,77 @@ +# Release 319 (22 Sep 2019) + +## General + +- Fix planning failure for queries involving `UNION` and `DISTINCT` aggregates. ({issue}`1510`) +- Fix excessive runtime when parsing expressions involving `CASE`. ({issue}`1407`) +- Fix fragment output size in `EXPLAIN ANALYZE` output. ({issue}`1345`) +- Fix a rare failure when running `EXPLAIN ANALYZE` on a query containing + window functions. ({issue}`1401`) +- Fix failure when querying `/v1/resourceGroupState` endpoint for non-existing resource + group. ({issue}`1368`) +- Fix incorrect results when reading `information_schema.table_privileges` with + an equality predicate on `table_name` but without a predicate on `table_schema`. + ({issue}`1534`) +- Fix planning failure due to coercion handling for correlated subqueries. ({issue}`1453`) +- Improve performance of queries against `information_schema` tables. ({issue}`1329`) +- Reduce metadata querying during planning. ({issue}`1308`, {issue}`1455`) +- Improve performance of certain queries involving coercions and complex expressions in `JOIN` + conditions. ({issue}`1390`) +- Include cost estimates in output of `EXPLAIN (TYPE IO)`. ({issue}`806`) +- Improve support for correlated subqueries involving `ORDER BY` or `LIMIT`. ({issue}`1415`) +- Improve performance of certain `JOIN` queries when automatic join ordering is enabled. ({issue}`1431`) +- Allow setting the default session catalog and schema via the `sql.default-catalog` + and `sql.default-schema` configuration properties. ({issue}`1524`) +- Add support for `IGNORE NULLS` for window functions. ({issue}`1244`) +- Add support for `INNER` and `OUTER` joins involving `UNNEST`. ({issue}`1522`) +- Rename `legacy` and `flat` {doc}`scheduler policies ` to + `uniform` and `topology` respectively. These can be configured via the `node-scheduler.policy` + configuration property. ({issue}`10491`) +- Add `file` {doc}`network topology provider ` which can be configured + via the `node-scheduler.network-topology.type` configuration property. ({issue}`1500`) +- Add support for `SphericalGeography` to {func}`ST_Length`. ({issue}`1551`) + +## Security + +- Allow configuring read-only access in {doc}`/security/built-in-system-access-control`. ({issue}`1153`) +- Add missing checks for schema create, rename, and drop in file-based `SystemAccessControl`. ({issue}`1153`) +- Allow authentication over HTTP for forwarded requests containing the + `X-Forwarded-Proto` header. This is disabled by default, but can be enabled using the + `http-server.authentication.allow-forwarded-https` configuration property. ({issue}`1442`) + +## Web UI + +- Fix rendering bug in Query Timeline resulting in inconsistency of presented information after + query finishes. ({issue}`1371`) +- Show total memory in Query Timeline instead of user memory. ({issue}`1371`) + +## CLI + +- Add `--insecure` option to skip validation of server certificates for debugging. ({issue}`1484`) + +## Hive connector + +- Fix reading from `information_schema`, as well as `SHOW SCHEMAS`, `SHOW TABLES`, and + `SHOW COLUMNS` when connecting to a Hive 3.x metastore that contains an `information_schema` + schema. ({issue}`1192`) +- Improve performance when reading data from GCS. ({issue}`1443`) +- Allow accessing tables in Glue metastore that do not have a table type. ({issue}`1343`) +- Add support for Azure Data Lake (`adl`) file system. ({issue}`1499`) +- Allow using custom S3 file systems by relying on the default Hadoop configuration by specifying + `HADOOP_DEFAULT` for the `hive.s3-file-system-type` configuration property. ({issue}`1397`) +- Add support for instance credentials for the Glue metastore via the + `hive.metastore.glue.use-instance-credentials` configuration property. ({issue}`1363`) +- Add support for custom credentials providers for the Glue metastore via the + `hive.metastore.glue.aws-credentials-provider` configuration property. ({issue}`1363`) +- Do not require setting the `hive.metastore-refresh-interval` configuration property + when enabling metastore caching. ({issue}`1473`) +- Add `textfile_field_separator` and `textfile_field_separator_escape` table properties + to support custom field separators for `TEXTFILE` format tables. ({issue}`1439`) +- Add `$file_size` and `$file_modified_time` hidden columns. ({issue}`1428`) +- The `hive.metastore-timeout` configuration property is now accepted only when using the + Thrift metastore. Previously, it was accepted for other metastore type, but was + ignored. ({issue}`1346`) +- Disallow reads from transactional tables. Previously, reads would appear to work, + but would not return any data. ({issue}`1218`) +- Disallow writes to transactional tables. Previously, writes would appear to work, + but the data would be written incorrectly. ({issue}`1218`) diff --git a/430/_sources/release/release-320.md.txt b/430/_sources/release/release-320.md.txt new file mode 100644 index 000000000..c57887a46 --- /dev/null +++ b/430/_sources/release/release-320.md.txt @@ -0,0 +1,58 @@ +# Release 320 (10 Oct 2019) + +## General + +- Fix incorrect parameter binding order for prepared statement execution when + parameters appear inside a `WITH` clause. ({issue}`1191`) +- Fix planning failure for certain queries involving a mix of outer and + cross joins. ({issue}`1589`) +- Improve performance of queries containing complex predicates. ({issue}`1515`) +- Avoid unnecessary evaluation of redundant filters. ({issue}`1516`) +- Improve performance of certain window functions when using bounded window + frames (e.g., `ROWS BETWEEN ... PRECEDING AND ... FOLLOWING`). ({issue}`464`) +- Add {doc}`/connector/kinesis`. ({issue}`476`) +- Add {func}`geometry_from_hadoop_shape`. ({issue}`1593`) +- Add {func}`at_timezone`. ({issue}`1612`) +- Add {func}`with_timezone`. ({issue}`1612`) + +## JDBC driver + +- Only report warnings on `Statement`, not `ResultSet`, as warnings + are not associated with reads of the `ResultSet`. ({issue}`1640`) + +## CLI + +- Add multi-line editing and syntax highlighting. ({issue}`1380`) + +## Hive connector + +- Add impersonation support for calls to the Hive metastore. This can be enabled using the + `hive.metastore.thrift.impersonation.enabled` configuration property. ({issue}`43`) +- Add caching support for Glue metastore. ({issue}`1625`) +- Add separate configuration property `hive.hdfs.socks-proxy` for accessing HDFS via a + SOCKS proxy. Previously, it was controlled with the `hive.metastore.thrift.client.socks-proxy` + configuration property. ({issue}`1469`) + +## MySQL connector + +- Add `mysql.jdbc.use-information-schema` configuration property to control whether + the MySQL JDBC driver should use the MySQL `information_schema` to answer metadata + queries. This may be helpful when diagnosing problems. ({issue}`1598`) + +## PostgreSQL connector + +- Add support for reading PostgreSQL system tables, e.g., `pg_catalog` relations. + The functionality is disabled by default and can be enabled using the + `postgresql.include-system-tables` configuration property. ({issue}`1527`) + +## Elasticsearch connector + +- Add support for `VARBINARY`, `TIMESTAMP`, `TINYINT`, `SMALLINT`, + and `REAL` data types. ({issue}`1639`) +- Discover available tables and their schema dynamically. ({issue}`1639`) +- Add support for special `_id`, `_score` and `_source` columns. ({issue}`1639`) +- Add support for {ref}`full text queries `. ({issue}`1662`) + +## SPI + +- Introduce a builder for `Identity` and deprecate its public constructors. ({issue}`1624`) diff --git a/430/_sources/release/release-321.md.txt b/430/_sources/release/release-321.md.txt new file mode 100644 index 000000000..6e2b5039e --- /dev/null +++ b/430/_sources/release/release-321.md.txt @@ -0,0 +1,57 @@ +# Release 321 (15 Oct 2019) + +:::{warning} +The server RPM is broken in this release. +::: + +## General + +- Fix incorrect result of {func}`round` when applied to a `tinyint`, `smallint`, + `integer`, or `bigint` type with negative decimal places. ({issue}`42`) +- Improve performance of queries with `LIMIT` over `information_schema` tables. ({issue}`1543`) +- Improve performance for broadcast joins by using dynamic filtering. This can be enabled + via the `experimental.enable-dynamic-filtering` configuration option or the + `enable_dynamic_filtering` session property. ({issue}`1686`) + +## Security + +- Improve the security of query results with one-time tokens. ({issue}`1654`) + +## Hive connector + +- Fix reading `TEXT` file collection delimiter set by Hive versions earlier + than 3.0. ({issue}`1714`) +- Fix a regression that prevented Presto from using the AWS Glue metastore. ({issue}`1698`) +- Allow skipping header or footer lines for `CSV` format tables via the + `skip_header_line_count` and `skip_footer_line_count` table properties. ({issue}`1090`) +- Rename table property `textfile_skip_header_line_count` to `skip_header_line_count` + and `textfile_skip_footer_line_count` to `skip_footer_line_count`. ({issue}`1090`) +- Add support for LZOP compressed (`.lzo`) files. Previously, queries accessing LZOP compressed + files would fail, unless all files were small. ({issue}`1701`) +- Add support for bucket-aware read of tables using bucketing version 2. ({issue}`538`) +- Add support for writing to tables using bucketing version 2. ({issue}`538`) +- Allow caching directory listings for all tables or schemas. ({issue}`1668`) +- Add support for dynamic filtering for broadcast joins. ({issue}`1686`) + +## PostgreSQL connector + +- Support reading PostgreSQL arrays as the `JSON` data type. This can be enabled by + setting the `postgresql.experimental.array-mapping` configuration property or the + `array_mapping` catalog session property to `AS_JSON`. ({issue}`682`) + +## Elasticsearch connector + +- Add support for Amazon Elasticsearch Service. ({issue}`1693`) + +## Cassandra connector + +- Add TLS support. ({issue}`1680`) + +## JMX connector + +- Add support for wildcards in configuration of history tables. ({issue}`1572`) + +## SPI + +- Fix `QueryStatistics.getWallTime()` to report elapsed time rather than total + scheduled time. ({issue}`1719`) diff --git a/430/_sources/release/release-322.md.txt b/430/_sources/release/release-322.md.txt new file mode 100644 index 000000000..4a097a870 --- /dev/null +++ b/430/_sources/release/release-322.md.txt @@ -0,0 +1,21 @@ +# Release 322 (16 Oct 2019) + +## General + +- Improve performance of certain join queries by reducing the amount of data + that needs to be scanned. ({issue}`1673`) + +## Server RPM + +- Fix a regression that caused zero-length files in the RPM. ({issue}`1767`) + +## Other connectors + +These changes apply to MySQL, PostgreSQL, Redshift, and SQL Server. + +- Add support for providing credentials using a keystore file. This can be enabled + by setting the `credential-provider.type` configuration property to `KEYSTORE` + and by setting the `keystore-file-path`, `keystore-type`, `keystore-password`, + `keystore-user-credential-password`, `keystore-password-credential-password`, + `keystore-user-credential-name`, and `keystore-password-credential-name` + configuration properties. ({issue}`1521`) diff --git a/430/_sources/release/release-323.md.txt b/430/_sources/release/release-323.md.txt new file mode 100644 index 000000000..640e57208 --- /dev/null +++ b/430/_sources/release/release-323.md.txt @@ -0,0 +1,49 @@ +# Release 323 (23 Oct 2019) + +## General + +- Fix query failure when referencing columns from a table that contains + hidden columns. ({issue}`1796`) +- Fix a rare issue in which the server produces an extra row containing + the boolean value `true` as the last row in the result set. For most queries, + this will result in a client error, since this row does not match the result + schema, but is a correctness issue when the result schema is a single boolean + column. ({issue}`1732`) +- Allow using `.*` on expressions of type `ROW` in the `SELECT` clause to + convert the fields of a row into multiple columns. ({issue}`1017`) + +## JDBC driver + +- Fix a compatibility issue when connecting to pre-321 servers. ({issue}`1785`) +- Fix reporting of views in `DatabaseMetaData.getTables()`. ({issue}`1488`) + +## CLI + +- Fix a compatibility issue when connecting to pre-321 servers. ({issue}`1785`) + +## Hive + +- Fix the ORC writer to correctly write the file footers. Previously written files were + sometimes unreadable in Hive 3.1 when querying the table for a second (or subsequent) + time. ({issue}`456`) +- Prevent writing to materialized views. ({issue}`1725`) +- Reduce metastore load when inserting data or analyzing tables. ({issue}`1783`, {issue}`1793`, {issue}`1794`) +- Allow using multiple Hive catalogs that use different Kerberos or other authentication + configurations. ({issue}`760`, {issue}`978`, {issue}`1820`) + +## PostgreSQL + +- Support for PostgreSQL arrays is no longer considered experimental, therefore + the configuration property `postgresql.experimental.array-mapping` is now named + to `postgresql.array-mapping`. ({issue}`1740`) + +## SPI + +- Add support for unnesting dictionary blocks duration compaction. ({issue}`1761`) +- Change `LazyBlockLoader` to directly return the loaded block. ({issue}`1744`) + +:::{note} +This is a backwards incompatible changes with the previous SPI. +If you have written a plugin that instantiates `LazyBlock`, +you will need to update your code before deploying this release. +::: diff --git a/430/_sources/release/release-324.md.txt b/430/_sources/release/release-324.md.txt new file mode 100644 index 000000000..778bf3e2e --- /dev/null +++ b/430/_sources/release/release-324.md.txt @@ -0,0 +1,38 @@ +# Release 324 (1 Nov 2019) + +## General + +- Fix query failure when `CASE` operands have different types. ({issue}`1825`) +- Add support for `ESCAPE` clause in `SHOW CATALOGS LIKE ...`. ({issue}`1691`) +- Add {func}`line_interpolate_point` and {func}`line_interpolate_points`. ({issue}`1888`) +- Allow references to tables in the enclosing query when using `.*`. ({issue}`1867`) +- Configuration properties for optimizer and spill support no longer + have `experimental.` prefix. ({issue}`1875`) +- Configuration property `experimental.reserved-pool-enabled` was renamed to + `experimental.reserved-pool-disabled` (with meaning reversed). ({issue}`1916`) + +## Security + +- Perform access control checks when displaying table or view definitions + with `SHOW CREATE`. ({issue}`1517`) + +## Hive + +- Allow using `SHOW GRANTS` on a Hive view when using the `sql-standard` + security mode. ({issue}`1842`) +- Improve performance when filtering dictionary-encoded Parquet columns. ({issue}`1846`) + +## PostgreSQL + +- Add support for inserting `MAP(VARCHAR, VARCHAR)` values into columns of + `hstore` type. ({issue}`1894`) + +## Elasticsearch + +- Fix failure when reading datetime columns in Elasticsearch 5.x. ({issue}`1844`) +- Add support for mixed-case field names. ({issue}`1914`) + +## SPI + +- Introduce a builder for `ColumnMetadata`. The various overloaded constructors + are now deprecated. ({issue}`1891`) diff --git a/430/_sources/release/release-325.md.txt b/430/_sources/release/release-325.md.txt new file mode 100644 index 000000000..68d0d8304 --- /dev/null +++ b/430/_sources/release/release-325.md.txt @@ -0,0 +1,43 @@ +# Release 325 (14 Nov 2019) + +:::{warning} +There is a performance regression in this release. +::: + +## General + +- Fix incorrect results for certain queries involving `FULL` or `RIGHT` joins and + `LATERAL`. ({issue}`1952`) +- Fix incorrect results when using `IS DISTINCT FROM` on columns of `DECIMAL` type + with precision larger than 18. ({issue}`1985`) +- Fix query failure when row types contain a field named after a reserved SQL keyword. ({issue}`1963`) +- Add support for `LIKE` predicate to `SHOW SESSION` and `SHOW FUNCTIONS`. ({issue}`1688`, {issue}`1692`) +- Add support for late materialization to join operations. ({issue}`1256`) +- Reduce number of metadata queries during planning. + This change disables stats collection for non-`EXPLAIN` queries. If you + want to have access to such stats and cost in query completion events, you + need to re-enable stats collection using the `collect-plan-statistics-for-all-queries` + configuration property. ({issue}`1866`) +- Add variant of {func}`strpos` that returns the Nth occurrence of a substring. ({issue}`1811`) +- Add {func}`to_encoded_polyline` and {func}`from_encoded_polyline` geospatial functions. ({issue}`1827`) + +## Web UI + +- Show actual query for an `EXECUTE` statement. ({issue}`1980`) + +## Hive + +- Fix incorrect behavior of `CREATE TABLE` when Hive metastore is configured + with `metastore.create.as.acid` set to `true`. ({issue}`1958`) +- Fix query failure when reading Parquet files that contain character data without statistics. ({issue}`1955`) +- Allow analyzing a subset of table columns (rather than all columns). ({issue}`1907`) +- Support overwriting unpartitioned tables for insert queries when using AWS Glue. ({issue}`1243`) +- Add support for reading Parquet files where the declared precision of decimal columns does not match + the precision in the table or partition schema. ({issue}`1949`) +- Improve performance when reading Parquet files with small row groups. ({issue}`1925`) + +## Other connectors + +These changes apply to the MySQL, PostgreSQL, Redshift, and SQL Server connectors. + +- Fix incorrect insertion of data when the target table has an unsupported type. ({issue}`1930`) diff --git a/430/_sources/release/release-326.md.txt b/430/_sources/release/release-326.md.txt new file mode 100644 index 000000000..84c84610c --- /dev/null +++ b/430/_sources/release/release-326.md.txt @@ -0,0 +1,39 @@ +# Release 326 (27 Nov 2019) + +## General + +- Fix incorrect query results when query contains `LEFT JOIN` over `UNNEST`. ({issue}`2097`) +- Fix performance regression in queries involving `JOIN`. ({issue}`2047`) +- Fix accounting of semantic analysis time when queued queries are cancelled. ({issue}`2055`) +- Add {doc}`/connector/singlestore`. ({issue}`1906`) +- Improve performance of `INSERT` and `CREATE TABLE ... AS` queries containing redundant + `ORDER BY` clauses. ({issue}`2044`) +- Improve performance when processing columns of `map` type. ({issue}`2015`) + +## Server RPM + +- Allow running Presto with {ref}`Java 11 or above `. ({issue}`2057`) + +## Security + +- Deprecate Kerberos in favor of JWT for {doc}`/security/internal-communication`. ({issue}`2032`) + +## Hive + +- Fix table creation error for tables with S3 location when using `file` metastore. ({issue}`1664`) +- Fix a compatibility issue with the CDH 5.x metastore which results in stats + not being recorded for {doc}`/sql/analyze`. ({issue}`973`) +- Improve performance for Glue metastore by fetching partitions in parallel. ({issue}`1465`) +- Improve performance of `sql-standard` security. ({issue}`1922`, {issue}`1929`) + +## Phoenix connector + +- Collect statistics on the count and duration of each call to Phoenix. ({issue}`2024`) + +## Other connectors + +These changes apply to the MySQL, PostgreSQL, Redshift, and SQL Server connectors. + +- Collect statistics on the count and duration of operations to create + and destroy `JDBC` connections. ({issue}`2024`) +- Add support for showing column comments. ({issue}`1840`) diff --git a/430/_sources/release/release-327.md.txt b/430/_sources/release/release-327.md.txt new file mode 100644 index 000000000..d95744e5b --- /dev/null +++ b/430/_sources/release/release-327.md.txt @@ -0,0 +1,70 @@ +# Release 327 (20 Dec 2019) + +## General + +- Fix join query failure when late materialization is enabled. ({issue}`2144`) +- Fix failure of {func}`word_stem` for certain inputs. ({issue}`2145`) +- Fix query failure when using `transform_values()` inside `try()` and the transformation fails + for one of the rows. ({issue}`2315`) +- Fix potential incorrect results for aggregations involving `FILTER (WHERE ...)` + when the condition is a reference to a table column. ({issue}`2267`) +- Allow renaming views with {doc}`/sql/alter-view`. ({issue}`1060`) +- Add `error_type` and `error_code` columns to `system.runtime.queries`. ({issue}`2249`) +- Rename `experimental.work-processor-pipelines` configuration property to `experimental.late-materialization.enabled` + and rename `work_processor_pipelines` session property to `late_materialization`. ({issue}`2275`) + +## Security + +- Allow using multiple system access controls. ({issue}`2178`) +- Add {doc}`/security/password-file`. ({issue}`797`) + +## Hive connector + +- Fix incorrect query results when reading `timestamp` values from ORC files written by + Hive 3.1 or later. ({issue}`2099`) +- Fix a CDH 5.x metastore compatibility issue resulting in failure when analyzing or inserting + into a table with `date` columns. ({issue}`556`) +- Reduce number of metastore calls when fetching partitions. ({issue}`1921`) +- Support reading from insert-only transactional tables. ({issue}`576`) +- Deprecate `parquet.fail-on-corrupted-statistics` (previously known as `hive.parquet.fail-on-corrupted-statistics`). + Setting this configuration property to `false` may hide correctness issues, leading to incorrect query results. + Session property `parquet_fail_with_corrupted_statistics` is deprecated as well. + Both configuration and session properties will be removed in a future version. ({issue}`2129`) +- Improve concurrency when updating table or partition statistics. ({issue}`2154`) +- Add support for renaming views. ({issue}`2189`) +- Allow configuring the `hive.orc.use-column-names` config property on a per-session + basis using the `orc_use_column_names` session property. ({issue}`2248`) + +## Kudu connector + +- Support predicate pushdown for the `decimal` type. ({issue}`2131`) +- Fix column position swap for delete operations that may result in deletion of the wrong records. ({issue}`2252`) +- Improve predicate pushdown for queries that match a column against + multiple values (typically using the `IN` operator). ({issue}`2253`) + +## MongoDB connector + +- Add support for reading from views. ({issue}`2156`) + +## PostgreSQL connector + +- Allow converting unsupported types to `VARCHAR` by setting the session property + `unsupported_type_handling` or configuration property `unsupported-type-handling` + to `CONVERT_TO_VARCHAR`. ({issue}`1182`) + +## MySQL connector + +- Fix `INSERT` query failure when `GTID` mode is enabled. ({issue}`2251`) + +## Elasticsearch connector + +- Improve performance for queries involving equality and range filters + over table columns. ({issue}`2310`) + +## Google Sheets connector + +- Fix incorrect results when listing tables in `information_schema`. ({issue}`2118`) + +## SPI + +- Add `executionTime` to `QueryStatistics` for event listeners. ({issue}`2247`) diff --git a/430/_sources/release/release-328.md.txt b/430/_sources/release/release-328.md.txt new file mode 100644 index 000000000..b295c9b33 --- /dev/null +++ b/430/_sources/release/release-328.md.txt @@ -0,0 +1,72 @@ +# Release 328 (10 Jan 2020) + +## General + +- Fix correctness issue for certain correlated join queries when the correlated subquery on + the right produces no rows. ({issue}`1969`) +- Fix incorrect handling of multi-byte characters for {doc}`/functions/regexp` when + the pattern is empty. ({issue}`2313`) +- Fix failure when join criteria contains columns of different types. ({issue}`2320`) +- Fix failure for complex outer join queries when dynamic filtering is enabled. ({issue}`2363`) +- Improve support for correlated queries. ({issue}`1969`) +- Allow inserting values of a larger type into as smaller type when the values fit. For example, + `BIGINT` into `SMALLINT`, or `VARCHAR(10)` into `VARCHAR(3)`. Values that don't fit will + cause an error at runtime. ({issue}`2061`) +- Add {func}`regexp_count` and {func}`regexp_position` functions. ({issue}`2136`) +- Add support for interpolating {doc}`/security/secrets` in server and catalog configuration + files. ({issue}`2370`) + +## Security + +- Fix a security issue allowing users to gain unauthorized access to Presto cluster + when using password authenticator with LDAP. ({issue}`2356`) +- Add support for LDAP referrals in LDAP password authenticator. ({issue}`2354`) + +## JDBC driver + +- Fix behavior of `java.sql.Connection#commit()` and `java.sql.Connection#rollback()` + methods when no statements performed in a transaction. Previously, these methods + would fail. ({issue}`2339`) +- Fix failure when restoring autocommit mode with + `java.sql.Connection#setAutocommit()` ({issue}`2338`) + +## Hive connector + +- Reduce query latency and Hive metastore load when using the + `AUTOMATIC` join reordering strategy. ({issue}`2184`) +- Allow configuring `hive.max-outstanding-splits-size` to values larger than 2GB. ({issue}`2395`) +- Avoid redundant file system stat call when writing Parquet files. ({issue}`1746`) +- Avoid retrying permanent errors for S3-related services such as STS. ({issue}`2331`) + +## Kafka connector + +- Remove internal columns: `_segment_start`, `_segment_end` and + `_segment_count`. ({issue}`2303`) +- Add new configuration property `kafka.messages-per-split` to control how many Kafka + messages will be processed by a single Presto split. ({issue}`2303`) + +## Elasticsearch connector + +- Fix query failure when an object in an Elasticsearch document + does not have any fields. ({issue}`2217`) +- Add support for querying index aliases. ({issue}`2324`) + +## Phoenix connector + +- Add support for mapping unsupported data types to `VARCHAR`. This can be enabled by setting + the `unsupported-type-handling` configuration property or the `unsupported_type_handling` session + property to `CONVERT_TO_VARCHAR`. ({issue}`2427`) + +## Other connectors + +These changes apply to the MySQL, PostgreSQL, Redshift and SQL Server connectors: + +- Add support for creating schemas. ({issue}`1874`) +- Add support for caching metadata. The configuration property `metadata.cache-ttl` + controls how long to cache data (it defaults to `0ms` which disables caching), + and `metadata.cache-missing` controls whether or not missing tables are cached. ({issue}`2290`) + +This change applies to the MySQL and PostgreSQL connectors: + +- Add support for mapping `DECIMAL` types with precision larger than 38 + to Presto `DECIMAL`. ({issue}`2088`) diff --git a/430/_sources/release/release-329.md.txt b/430/_sources/release/release-329.md.txt new file mode 100644 index 000000000..3adb47a1a --- /dev/null +++ b/430/_sources/release/release-329.md.txt @@ -0,0 +1,65 @@ +# Release 329 (23 Jan 2020) + +## General + +- Fix incorrect result for {func}`last_day_of_month` function for first day of month. ({issue}`2452`) +- Fix incorrect results when handling `DOUBLE` or `REAL` types with `NaN` values. ({issue}`2582`) +- Fix query failure when coordinator hostname contains underscores. ({issue}`2571`) +- Fix `SHOW CREATE TABLE` failure when row types contain a field named after a + reserved SQL keyword. ({issue}`2130`) +- Handle common disk failures during spill. When one disk fails but multiple + spill locations are configured, the healthy disks will be used for future queries. + ({issue}`2444`) +- Improve performance and reduce load on external systems when + querying `information_schema`. ({issue}`2488`) +- Improve performance of queries containing redundant scalar subqueries. ({issue}`2456`) +- Limit broadcasted table size to `100MB` by default when using the `AUTOMATIC` + join type selection strategy. This avoids query failures or excessive memory usage when joining two or + more very large tables. ({issue}`2527`) +- Enable {doc}`cost based ` join reordering and join type selection + optimizations by default. The previous behavior can be restored by + setting `optimizer.join-reordering-strategy` configuration property to `ELIMINATE_CROSS_JOINS` + and `join-distribution-type` to `PARTITIONED`. ({issue}`2528`) +- Hide non-standard columns `comment` and `extra_info` in the standard + `information_schema.columns` table. These columns can still be selected, + but will no longer appear when describing the table. ({issue}`2306`) + +## Security + +- Add `ldap.bind-dn` and `ldap.bind-password` LDAP properties to allow LDAP authentication + access LDAP server using service account. ({issue}`1917`) + +## Hive connector + +- Fix incorrect data returned when using S3 Select on uncompressed files. In our testing, S3 Select + was apparently returning incorrect results when reading uncompressed files, so S3 Select is disabled + for uncompressed files. ({issue}`2399`) +- Fix incorrect data returned when using S3 Select on a table with `skip.header.line.count` or + `skip.footer.line.count` property. S3 Select API does not support skipping footers or more than one + line of a header. In our testing, S3 Select was apparently sometimes returning incorrect results when + reading a compressed file with header skipping, so S3 Select is disabled when any of these table + properties is set to non-zero value. ({issue}`2399`) +- Fix query failure for writes when one of the inserted `REAL` or `DOUBLE` values + is infinite or `NaN`. ({issue}`2471`) +- Fix performance degradation reading from S3 when the Kinesis connector is installed. ({issue}`2496`) +- Allow reading data from Parquet files when the column type is declared as `INTEGER` + in the table or partition, but is a `DECIMAL` type in the file. ({issue}`2451`) +- Validate the scale of decimal types when reading Parquet files. This prevents + incorrect results when the decimal scale in the file does not match the declared + type for the table or partition. ({issue}`2451`) +- Delete storage location when dropping an empty schema. ({issue}`2463`) +- Improve performance when deleting multiple partitions by executing these actions concurrently. ({issue}`1812`) +- Improve performance for queries containing `IN` predicates over bucketing columns. ({issue}`2277`) +- Add procedure `system.drop_stats()` to remove the column statistics + for a table or selected partitions. ({issue}`2538`) + +## Elasticsearch connector + +- Add support for {ref}`elasticsearch-array-types`. ({issue}`2441`) +- Reduce load on Elasticsearch cluster and improve query performance. ({issue}`2561`) + +## PostgreSQL connector + +- Fix mapping between PostgreSQL's `TIME` and Presto's `TIME` data types. + Previously the mapping was incorrect, shifting it by the relative offset between the session + time zone and the Presto server's JVM time zone. ({issue}`2549`) diff --git a/430/_sources/release/release-330.md.txt b/430/_sources/release/release-330.md.txt new file mode 100644 index 000000000..7e9486d8d --- /dev/null +++ b/430/_sources/release/release-330.md.txt @@ -0,0 +1,121 @@ +# Release 330 (18 Feb 2020) + +## General + +- Fix incorrect behavior of {func}`format` for `char` values. Previously, the function + did not preserve trailing whitespace of the value being formatted. ({issue}`2629`) +- Fix query failure in some cases when aggregation uses inputs from both sides of a join. ({issue}`2560`) +- Fix query failure when dynamic filtering is enabled and the query contains complex + multi-level joins. ({issue}`2659`) +- Fix query failure for certain co-located joins when dynamic filtering is enabled. ({issue}`2685`) +- Fix failure of `SHOW` statements or queries that access `information_schema` schema tables + with an empty value used in a predicate. ({issue}`2575`) +- Fix query failure when {doc}`/sql/execute` is used with an expression containing a function call. ({issue}`2675`) +- Fix failure in `SHOW CATALOGS` when the user does not have permissions to see any catalogs. ({issue}`2593`) +- Improve query performance for some join queries when {doc}`/optimizer/cost-based-optimizations` + are enabled. ({issue}`2722`) +- Prevent uneven distribution of data that can occur when writing data with redistribution or writer + scaling enabled. ({issue}`2788`) +- Add support for `CREATE VIEW` with comment ({issue}`2557`) +- Add support for all major geometry types to {func}`ST_Points`. ({issue}`2535`) +- Add `required_workers_count` and `required_workers_max_wait_time` session properties + to control the number of workers that must be present in the cluster before query + processing starts. ({issue}`2484`) +- Add `physical_input_bytes` column to `system.runtime.tasks` table. ({issue}`2803`) +- Verify that the target schema exists for the {doc}`/sql/use` statement. ({issue}`2764`) +- Verify that the session catalog exists when executing {doc}`/sql/set-role`. ({issue}`2768`) + +## Server + +- Require running on {ref}`Java 11 or above `. This requirement may be temporarily relaxed by adding + `-Dpresto-temporarily-allow-java8=true` to the Presto {ref}`jvm-config`. + This fallback will be removed in future versions of Presto after March 2020. ({issue}`2751`) +- Add experimental support for running on Linux aarch64 (ARM64). ({issue}`2809`) + +## Security + +- {ref}`system-file-auth-principal-rules` are deprecated and will be removed in a future release. + These rules have been replaced with {doc}`/security/user-mapping`, which + specifies how a complex authentication user name is mapped to a simple + user name for Presto, and {ref}`system-file-auth-impersonation-rules` which + control the ability of a user to impersonate another user. ({issue}`2215`) +- A shared secret is now required when using {doc}`/security/internal-communication`. ({issue}`2202`) +- Kerberos for {doc}`/security/internal-communication` has been replaced with the new shared secret mechanism. + The `internal-communication.kerberos.enabled` and `internal-communication.kerberos.use-canonical-hostname` + configuration properties must be removed. ({issue}`2202`) +- When authentication is disabled, the Presto user may now be set using standard + HTTP basic authentication with an empty password. ({issue}`2653`) + +## Web UI + +- Display physical read time in detailed query view. ({issue}`2805`) + +## JDBC driver + +- Fix a performance issue on JDK 11+ when connecting using HTTP/2. ({issue}`2633`) +- Implement `PreparedStatement.setTimestamp()` variant that takes a `Calendar`. ({issue}`2732`) +- Add `roles` property for catalog authorization roles. ({issue}`2780`) +- Add `sessionProperties` property for setting system and catalog session properties. ({issue}`2780`) +- Add `clientTags` property to set client tags for selecting resource groups. ({issue}`2468`) +- Allow using the `:` character within an extra credential value specified via the + `extraCredentials` property. ({issue}`2780`) + +## CLI + +- Fix a performance issue on JDK 11+ when connecting using HTTP/2. ({issue}`2633`) + +## Cassandra connector + +- Fix query failure when identifiers should be quoted. ({issue}`2455`) + +## Hive connector + +- Fix reading symlinks from HDFS when using Kerberos. ({issue}`2720`) +- Reduce Hive metastore load when updating partition statistics. ({issue}`2734`) +- Allow redistributing writes for un-bucketed partitioned tables on the + partition keys, which results in a single writer per partition. This reduces + memory usage, results in a single file per partition, and allows writing a + large number of partitions (without hitting the open writer limit). However, + writing large partitions with a single writer can take substantially longer, so + this feature should only be enabled when required. To enable this feature, set the + `use-preferred-write-partitioning` system configuration property or the + `use_preferred_write_partitioning` system session property to `true`. ({issue}`2358`) +- Remove extra file status call after writing text-based, SequenceFile, or Avro file types. ({issue}`1748`) +- Allow using writer scaling with all file formats. Previously, it was not supported for + text-based, SequenceFile, or Avro formats. ({issue}`2657`) +- Add support for symlink-based tables with Avro files. ({issue}`2720`) +- Add support for ignoring partitions with a non-existent data directory. This can be configured + using the `hive.ignore-absent-partitions=true` configuration property or the + `ignore_absent_partitions` session property. ({issue}`2555`) +- Allow creation of external tables with data via `CREATE TABLE AS` when + both `hive.non-managed-table-creates-enabled` and `hive.non-managed-table-writes-enabled` + are set to `true`. Previously this required executing `CREATE TABLE` and `INSERT` + as separate statement ({issue}`2669`) +- Add support for Azure WASB, ADLS Gen1 (ADL) and ADLS Gen2 (ABFS) file systems. ({issue}`2494`) +- Add experimental support for executing basic Hive views. To enable this feature, the + `hive.views-execution.enabled` configuration property must be set to `true`. ({issue}`2715`) +- Add {ref}`register_partition ` and {ref}`unregister_partition ` + procedures for adding partitions to and removing partitions from a partitioned table. ({issue}`2692`) +- Allow running {doc}`/sql/analyze` collecting only basic table statistics. ({issue}`2762`) + +## Elasticsearch connector + +- Improve performance of queries containing a `LIMIT` clause. ({issue}`2781`) +- Add support for `nested` data type. ({issue}`754`) + +## PostgreSQL connector + +- Add read support for PostgreSQL `money` data type. The type is mapped to `varchar` in Presto. + ({issue}`2601`) + +## Other connectors + +These changes apply to the MySQL, PostgreSQL, Redshift, Phoenix and SQL Server connectors. + +- Respect `DEFAULT` column clause when writing to a table. ({issue}`1185`) + +## SPI + +- Allow procedures to have optional arguments with default values. ({issue}`2706`) +- `SystemAccessControl.checkCanSetUser()` is deprecated and has been replaced + with {doc}`/security/user-mapping` and `SystemAccessControl.checkCanImpersonateUser()`. ({issue}`2215`) diff --git a/430/_sources/release/release-331.md.txt b/430/_sources/release/release-331.md.txt new file mode 100644 index 000000000..5e03d925c --- /dev/null +++ b/430/_sources/release/release-331.md.txt @@ -0,0 +1,87 @@ +# Release 331 (16 Mar 2020) + +## General + +- Prevent query failures when worker is shut down gracefully. ({issue}`2648`) +- Fix join failures for queries involving `OR` predicate with non-comparable functions. ({issue}`2861`) +- Ensure query completed event is fired when there is an error during analysis or planning. ({issue}`2842`) +- Fix memory accounting for `ORDER BY` queries. ({issue}`2612`) +- Fix {func}`last_day_of_month` for `timestamp with time zone` values. ({issue}`2851`) +- Fix excessive runtime when parsing deeply nested expressions with unmatched parenthesis. ({issue}`2968`) +- Correctly reject `date` literals that cannot be represented in Presto. ({issue}`2888`) +- Improve query performance by removing redundant data reshuffling. ({issue}`2853`) +- Improve performance of inequality joins involving `BETWEEN`. ({issue}`2859`) +- Improve join performance for dictionary encoded data. ({issue}`2862`) +- Enable dynamic filtering by default. ({issue}`2793`) +- Show reorder join cost in `EXPLAIN ANALYZE VERBOSE` ({issue}`2725`) +- Allow configuring resource groups selection based on user's groups. ({issue}`3023`) +- Add `SET AUTHORIZATION` action to {doc}`/sql/alter-schema`. ({issue}`2673`) +- Add {doc}`/connector/bigquery`. ({issue}`2532`) +- Add support for large prepared statements. ({issue}`2719`) + +## Security + +- Remove unused `internal-communication.jwt.enabled` configuration property. ({issue}`2709`) +- Rename JWT configuration properties from `http.authentication.jwt.*` to `http-server.authentication.jwt.*`. ({issue}`2712`) +- Add access control checks for query execution, view query, and kill query. This can be + configured using {ref}`query-rules` in {doc}`/security/file-system-access-control`. ({issue}`2213`) +- Hide columns of tables for which the user has no privileges in {doc}`/security/file-system-access-control`. ({issue}`2925`) + +## JDBC driver + +- Implement `PreparedStatement.getMetaData()`. ({issue}`2770`) + +## Web UI + +- Fix copying worker address to clipboard. ({issue}`2865`) +- Fix copying query ID to clipboard. ({issue}`2872`) +- Fix display of data size values. ({issue}`2810`) +- Fix redirect from `/` to `/ui/` when Presto is behind a proxy. ({issue}`2908`) +- Fix display of prepared queries. ({issue}`2784`) +- Display physical input read rate. ({issue}`2873`) +- Add simple form based authentication that utilizes the configured password authenticator. ({issue}`2755`) +- Allow disabling the UI via the `web-ui.enabled` configuration property. ({issue}`2755`) + +## CLI + +- Fix formatting of `varbinary` in nested data types. ({issue}`2858`) +- Add `--timezone` parameter. ({issue}`2961`) + +## Hive connector + +- Fix incorrect results for reads from `information_schema` tables and + metadata queries when using a Hive 3.x metastore. ({issue}`3008`) +- Fix query failure when using Glue metastore and the table storage descriptor has no properties. ({issue}`2905`) +- Fix deadlock when Hive caching is enabled and has a refresh interval configured. ({issue}`2984`) +- Respect `bucketing_version` table property when using Glue metastore. ({issue}`2905`) +- Improve performance of partition fetching from Glue. ({issue}`3024`) +- Add support for bucket sort order in Glue when creating or updating a table or partition. ({issue}`1870`) +- Add support for Hive full ACID tables. ({issue}`2068`, {issue}`1591`, {issue}`2790`) +- Allow data conversion when reading decimal data from Parquet files and precision or scale in the file schema + is different from the precision or scale in partition schema. ({issue}`2823`) +- Add option to enforce that a filter on a partition key be present in the query. This can be enabled by setting the + `hive.query-partition-filter-required` configuration property or the `query_partition_filter_required` session property + to `true`. ({issue}`2334`) +- Allow selecting the `Intelligent-Tiering` S3 storage class when writing data to S3. This can be enabled by + setting the `hive.s3.storage-class` configuration property to `INTELLIGENT_TIERING`. ({issue}`3032`) +- Hide the Hive system schema `sys` for security reasons. ({issue}`3008`) +- Add support for changing the owner of a schema. ({issue}`2673`) + +## MongoDB connector + +- Fix incorrect results when queries contain filters on certain data types, such + as `real` or `decimal`. ({issue}`1781`) + +## Other connectors + +These changes apply to the MemSQL, MySQL, PostgreSQL, Redshift, Phoenix, and SQL Server connectors. + +- Add support for dropping schemas. ({issue}`2956`) + +## SPI + +- Remove deprecated `Identity` constructors. ({issue}`2877`) +- Introduce a builder for `ConnectorIdentity` and deprecate its public constructors. ({issue}`2877`) +- Add support for row filtering and column masking via the `getRowFilter()` and `getColumnMask()` APIs in + `SystemAccessControl` and `ConnectorAccessControl`. ({issue}`1480`) +- Add access control check for executing procedures. ({issue}`2924`) diff --git a/430/_sources/release/release-332.md.txt b/430/_sources/release/release-332.md.txt new file mode 100644 index 000000000..9377faa25 --- /dev/null +++ b/430/_sources/release/release-332.md.txt @@ -0,0 +1,99 @@ +# Release 332 (08 Apr 2020) + +## General + +- Fix query failure during planning phase for certain queries involving multiple joins. ({issue}`3149`) +- Fix execution failure for queries involving large `IN` predicates on decimal values with precision larger than 18. ({issue}`3191`) +- Fix prepared statements or view creation for queries containing certain nested aliases or `TABLESAMPLE` clauses. ({issue}`3250`) +- Fix rare query failure. ({issue}`2981`) +- Ignore trailing whitespace when loading configuration files such as + `etc/event-listener.properties` or `etc/group-provider.properties`. + Trailing whitespace in `etc/config.properties` and catalog properties + files was already ignored. ({issue}`3231`) +- Reduce overhead for internal communication requests. ({issue}`3215`) +- Include filters over all table columns in output of `EXPLAIN (TYPE IO)`. ({issue}`2743`) +- Support configuring multiple event listeners. The properties files for all the event listeners + can be specified using the `event-listener.config-files` configuration property. ({issue}`3128`) +- Add `CREATE SCHEMA ... AUTHORIZATION` syntax to create a schema with specified owner. ({issue}`3066`). +- Add `optimizer.push-partial-aggregation-through-join` configuration property to control + pushing partial aggregations through inner joins. Previously, this was only available + via the `push_partial_aggregation_through_join` session property. ({issue}`3205`) +- Rename configuration property `optimizer.push-aggregation-through-join` + to `optimizer.push-aggregation-through-outer-join`. ({issue}`3205`) +- Add operator statistics for the number of splits processed with a dynamic filter applied. ({issue}`3217`) + +## Security + +- Fix LDAP authentication when user belongs to multiple groups. ({issue}`3206`) +- Verify access to table columns when running `SHOW STATS`. ({issue}`2665`) +- Only return views accessible to the user from `information_schema.views`. ({issue}`3290`) + +## JDBC driver + +- Add `clientInfo` property to set extra information about the client. ({issue}`3188`) +- Add `traceToken` property to set a trace token for correlating requests across systems. ({issue}`3188`) + +## BigQuery connector + +- Extract parent project ID from service account before looking at the environment. ({issue}`3131`) + +## Elasticsearch connector + +- Add support for `ip` type. ({issue}`3347`) +- Add support for `keyword` fields with numeric values. ({issue}`3381`) +- Remove unnecessary `elasticsearch.aws.use-instance-credentials` configuration property. ({issue}`3265`) + +## Hive connector + +- Fix failure reading certain Parquet files larger than 2GB. ({issue}`2730`) +- Improve performance when reading gzip-compressed Parquet data. ({issue}`3175`) +- Explicitly disallow reading from Delta Lake tables. Previously, reading + from partitioned tables would return zero rows, and reading from + unpartitioned tables would fail with a cryptic error. ({issue}`3366`) +- Add `hive.fs.new-directory-permissions` configuration property for setting the permissions of new directories + created by Presto. Default value is `0777`, which corresponds to previous behavior. ({issue}`3126`) +- Add `hive.partition-use-column-names` configuration property and matching `partition_use_column_names` catalog + session property that allows to match columns between table and partition schemas by names. By default they are mapped + by index. ({issue}`2933`) +- Add support for `CREATE SCHEMA ... AUTHORIZATION` to create a schema with specified owner. ({issue}`3066`). +- Allow specifying the Glue metastore endpoint URL using the + `hive.metastore.glue.endpoint-url` configuration property. ({issue}`3239`) +- Add experimental file system caching. This can be enabled with the `hive.cache.enabled` configuration property. ({issue}`2679`) +- Support reading files compressed with newer versions of LZO. ({issue}`3209`) +- Add support for Alluxio Catalog Service. ({issue}`2116`) +- Remove unnecessary `hive.metastore.glue.use-instance-credentials` configuration property. ({issue}`3265`) +- Remove unnecessary `hive.s3.use-instance-credentials` configuration property. ({issue}`3265`) +- Add flexible {ref}`hive-s3-security-mapping`, allowing for separate credentials + or IAM roles for specific users or buckets/paths. ({issue}`3265`) +- Add support for specifying an External ID for an IAM role trust policy using + the `hive.metastore.glue.external-id` configuration property ({issue}`3144`) +- Allow using configured S3 credentials with IAM role. Previously, + the configured IAM role was silently ignored. ({issue}`3351`) + +## Kudu connector + +- Fix incorrect column mapping in Kudu connector. ({issue}`3170`, {issue}`2963`) +- Fix incorrect query result for certain queries involving `IS NULL` predicates with `OR`. ({issue}`3274`) + +## Memory connector + +- Include views in the list of tables returned to the JDBC driver. ({issue}`3208`) + +## MongoDB connector + +- Add `objectid_timestamp` for extracting the timestamp from `ObjectId`. ({issue}`3089`) +- Delete document from `_schema` collection when `DROP TABLE` + is executed for a table that exists only in `_schema`. ({issue}`3234`) + +## SQL Server connector + +- Disallow renaming tables between schemas. Previously, such renames were allowed + but the schema name was ignored when performing the rename. ({issue}`3284`) + +## SPI + +- Expose row filters and column masks in `QueryCompletedEvent`. ({issue}`3183`) +- Expose referenced functions and procedures in `QueryCompletedEvent`. ({issue}`3246`) +- Allow `Connector` to provide `EventListener` instances. ({issue}`3166`) +- Deprecate the `ConnectorPageSourceProvider.createPageSource()` variant without the + `dynamicFilter` parameter. The method will be removed in a future release. ({issue}`3255`) diff --git a/430/_sources/release/release-333.md.txt b/430/_sources/release/release-333.md.txt new file mode 100644 index 000000000..50093534a --- /dev/null +++ b/430/_sources/release/release-333.md.txt @@ -0,0 +1,83 @@ +# Release 333 (04 May 2020) + +## General + +- Fix planning failure when lambda expressions are repeated in a query. ({issue}`3218`) +- Fix failure when input to `TRY` is a constant `NULL`. ({issue}`3408`) +- Fix failure for {doc}`/sql/show-create-table` for tables with + row types that contain special characters. ({issue}`3380`) +- Fix failure when using {func}`max_by` or {func}`min_by` + where the second argument is of type `varchar`. ({issue}`3424`) +- Fix rare failure due to an invalid size estimation for T-Digests. ({issue}`3625`) +- Do not require coordinator to have spill paths setup when spill is enabled. ({issue}`3407`) +- Improve performance when dynamic filtering is enabled. ({issue}`3413`) +- Improve performance of queries involving constant scalar subqueries ({issue}`3432`) +- Allow overriding the count of available workers used for query cost + estimation via the `cost_estimation_worker_count` session property. ({issue}`2705`) +- Add data integrity verification for Presto internal communication. This can be configured + with the `exchange.data-integrity-verification` configuration property. ({issue}`3438`) +- Add support for `LIKE` predicate to {doc}`/sql/show-columns`. ({issue}`2997`) +- Add {doc}`/sql/show-create-schema`. ({issue}`3099`) +- Add {func}`starts_with` function. ({issue}`3392`) + +## Server + +- Require running on {ref}`Java 11 or above `. ({issue}`2799`) + +## Server RPM + +- Reduce size of RPM and disk usage after installation. ({issue}`3595`) + +## Security + +- Allow configuring trust certificate for LDAP password authenticator. ({issue}`3523`) + +## JDBC driver + +- Fix hangs on JDK 8u252 when using secure connections. ({issue}`3444`) + +## BigQuery connector + +- Improve performance for queries that contain filters on table columns. ({issue}`3376`) +- Add support for partitioned tables. ({issue}`3376`) + +## Cassandra connector + +- Allow {doc}`/sql/insert` statement for table having hidden `id` column. ({issue}`3499`) +- Add support for {doc}`/sql/create-table` statement. ({issue}`3478`) + +## Elasticsearch connector + +- Fix failure when querying Elasticsearch 7.x clusters. ({issue}`3447`) + +## Hive connector + +- Fix incorrect query results when reading Parquet data with a `varchar` column predicate + which is a comparison with a value containing non-ASCII characters. ({issue}`3517`) +- Ensure cleanup of resources (file descriptors, sockets, temporary files, etc.) + when an error occurs while writing an ORC file. ({issue}`3390`) +- Generate multiple splits for files in bucketed tables. ({issue}`3455`) +- Make file system caching honor Hadoop properties from `hive.config.resources`. ({issue}`3557`) +- Disallow enabling file system caching together with S3 security mapping or GCS access tokens. ({issue}`3571`) +- Disable file system caching parallel warmup by default. + It is currently broken and should not be enabled. ({issue}`3591`) +- Include metrics from S3 Select in the S3 JMX metrics. ({issue}`3429`) +- Report timings for request retries in S3 JMX metrics. + Previously, only the first request was reported. ({issue}`3429`) +- Add S3 JMX metric for client retry pause time (how long the thread was asleep + between request retries in the client itself). ({issue}`3429`) +- Add support for {doc}`/sql/show-create-schema`. ({issue}`3099`) +- Add `hive.projection-pushdown-enabled` configuration property and + `projection_pushdown_enabled` session property. ({issue}`3490`) +- Add support for connecting to the Thrift metastore using TLS. ({issue}`3440`) + +## MongoDB connector + +- Skip unknown types in nested BSON object. ({issue}`2935`) +- Fix query failure when the user does not have access privileges for `system.views`. ({issue}`3355`) + +## Other connectors + +These changes apply to the MemSQL, MySQL, PostgreSQL, Redshift, and SQL Server connectors. + +- Export JMX statistics for various connector operations. ({issue}`3479`). diff --git a/430/_sources/release/release-334.md.txt b/430/_sources/release/release-334.md.txt new file mode 100644 index 000000000..bfe6e77de --- /dev/null +++ b/430/_sources/release/release-334.md.txt @@ -0,0 +1,83 @@ +# Release 334 (29 May 2020) + +## General + +- Fix incorrect query results for certain queries involving comparisons of `real` and `double` types + when values include negative zero. ({issue}`3745`) +- Fix failure when querying an empty table with late materialization enabled. ({issue}`3577`) +- Fix failure when the inputs to `UNNEST` are repeated. ({issue}`3587`) +- Fix failure when an aggregation is used in the arguments to {func}`format`. ({issue}`3829`) +- Fix {func}`localtime` and {func}`current_time` for session zones with DST or with historical offset changes + in legacy (default) timestamp semantics. ({issue}`3846`, {issue}`3850`) +- Fix dynamic filter failures in complex spatial join queries. ({issue}`3694`) +- Improve performance of queries involving {func}`row_number`. ({issue}`3614`) +- Improve performance of queries containing `LIKE` predicate. ({issue}`3618`) +- Improve query performance when dynamic filtering is enabled. ({issue}`3632`) +- Improve performance for queries that read fields from nested structures. ({issue}`2672`) +- Add variant of {func}`random` function that produces a number in the provided range. ({issue}`1848`) +- Show distributed plan by default in {doc}`/sql/explain`. ({issue}`3724`) +- Add {doc}`/connector/oracle`. ({issue}`1959`) +- Add {doc}`/connector/pinot`. ({issue}`2028`) +- Add {doc}`/connector/prometheus`. ({issue}`2321`) +- Add support for standards compliant ({rfc}`7239`) HTTP forwarded headers. Processing of HTTP forwarded headers is now controlled by the + `http-server.process-forwarded` configuration property, and the old `http-server.authentication.allow-forwarded-https` and + `dispatcher.forwarded-header` configuration properties are no longer supported. ({issue}`3714`) +- Add pluggable {doc}`/develop/certificate-authenticator`. ({issue}`3804`) + +## JDBC driver + +- Implement `toString()` for `java.sql.Array` results. ({issue}`3803`) + +## CLI + +- Improve rendering of elapsed time for short queries. ({issue}`3311`) + +## Web UI + +- Add `fixed`, `certificate`, `JWT`, and `Kerberos` to UI authentication. ({issue}`3433`) +- Show join distribution type in Live Plan. ({issue}`1323`) + +## JDBC driver + +- Improve performance of `DatabaseMetaData.getColumns()` when the + parameters contain unescaped `%` or `_`. ({issue}`1620`) + +## Elasticsearch connector + +- Fix failure when executing `SHOW CREATE TABLE`. ({issue}`3718`) +- Improve performance for `count(*)` queries. ({issue}`3512`) +- Add support for raw Elasticsearch queries. ({issue}`3735`) + +## Hive connector + +- Fix matching bucket filenames without leading zeros. ({issue}`3702`) +- Fix creation of external tables using `CREATE TABLE AS`. Previously, the + tables were created as managed and with the default location. ({issue}`3755`) +- Fix incorrect table statistics for newly created external tables. ({issue}`3819`) +- Prevent Presto from starting when cache fails to initialize. ({issue}`3749`) +- Fix race condition that could cause caching to be permanently disabled. ({issue}`3729`, {issue}`3810`) +- Fix malformed reads when asynchronous read mode for caching is enabled. ({issue}`3772`) +- Fix eviction of cached data while still under size eviction threshold. ({issue}`3772`) +- Improve performance when creating unpartitioned external tables over large data sets. ({issue}`3624`) +- Leverage Parquet file statistics when reading decimal columns. ({issue}`3581`) +- Change type of `$file_modified_time` hidden column from `bigint` to `timestamp with timezone type`. ({issue}`3611`) +- Add caching support for HDFS and Azure file systems. ({issue}`3772`) +- Fix S3 connection pool depletion when asynchronous read mode for caching is enabled. ({issue}`3772`) +- Disable caching on coordinator by default. ({issue}`3820`) +- Use asynchronous read mode for caching by default. ({issue}`3799`) +- Cache delegation token for Hive thrift metastore. This can be configured with + the `hive.metastore.thrift.delegation-token.cache-ttl` and `hive.metastore.thrift.delegation-token.cache-maximum-size` + configuration properties. ({issue}`3771`) + +## MemSQL connector + +- Include {doc}`/connector/singlestore` in the server tarball and RPM. ({issue}`3743`) + +## MongoDB connector + +- Support case insensitive database and collection names. This can be enabled with the + `mongodb.case-insensitive-name-matching` configuration property. ({issue}`3453`) + +## SPI + +- Allow a `SystemAccessControl` to provide an `EventListener`. ({issue}`3629`). diff --git a/430/_sources/release/release-335.md.txt b/430/_sources/release/release-335.md.txt new file mode 100644 index 000000000..10de82323 --- /dev/null +++ b/430/_sources/release/release-335.md.txt @@ -0,0 +1,55 @@ +# Release 335 (14 Jun 2020) + +## General + +- Fix failure when {func}`reduce_agg` is used as a window function. ({issue}`3883`) +- Fix incorrect cast from `TIMESTAMP` (without time zone) to `TIME` type. ({issue}`3848`) +- Fix incorrect query results when converting very large `TIMESTAMP` values into + `TIMESTAMP WITH TIME ZONE`, or when parsing very large + `TIMESTAMP WITH TIME ZONE` values. ({issue}`3956`) +- Return `VARCHAR` type when {func}`substr` argument is `CHAR` type. ({issue}`3599`, {issue}`3456`) +- Improve optimized local scheduling with regard to non-uniform data distribution. ({issue}`3922`) +- Add support for variable-precision `TIMESTAMP` (without time zone) type. ({issue}`3783`) +- Add a variant of {func}`substring` that takes a `CHAR` argument. ({issue}`3949`) +- Add `information_schema.role_authorization_descriptors` table that returns information about the roles + granted to principals. ({issue}`3535`) + +## Security + +- Add schema access rules to {doc}`/security/file-system-access-control`. ({issue}`3766`) + +## Web UI + +- Fix the value displayed in the worker memory pools bar. ({issue}`3920`) + +## Accumulo connector + +- The server-side iterators are now in a JAR file named `presto-accumulo-iterators`. ({issue}`3673`) + +## Hive connector + +- Collect column statistics for inserts into empty tables. ({issue}`2469`) +- Add support for `information_schema.role_authorization_descriptors` table when using the `sql-standard` + security mode. ({issue}`3535`) +- Allow non-lowercase column names in {ref}`system.sync_partition_metadata` procedure. This can be enabled + by passing `case_sensitive=false` when invoking the procedure. ({issue}`3431`) +- Support caching with secured coordinator. ({issue}`3874`) +- Prevent caching from becoming disabled due to intermittent network failures. ({issue}`3874`) +- Ensure HDFS impersonation is not enabled when caching is enabled. ({issue}`3913`) +- Add `hive.cache.ttl` and `hive.cache.disk-usage-percentage` cache properties. ({issue}`3840`) +- Improve query performance when caching is enabled by scheduling work on nodes with cached data. ({issue}`3922`) +- Add support for `UNIONTYPE`. This is mapped to `ROW` containing a `tag` field and a field for each data type in the union. For + example, `UNIONTYPE` is mapped to `ROW(tag INTEGER, field0 INTEGER, field1 DOUBLE)`. ({issue}`3483`) +- Make `partition_values` argument to `drop_stats` procedure optional. ({issue}`3937`) +- Add support for dynamic partition pruning to improve performance of complex queries + over partitioned data. ({issue}`1072`) + +## Phoenix connector + +- Allow configuring whether `DROP TABLE` is allowed. This is controlled by the new `allow-drop-table` + catalog configuration property and defaults to `true`, compatible with the previous behavior. ({issue}`3953`) + +## SPI + +- Add support for aggregation pushdown into connectors via the + `ConnectorMetadata.applyAggregation()` method. ({issue}`3697`) diff --git a/430/_sources/release/release-336.md.txt b/430/_sources/release/release-336.md.txt new file mode 100644 index 000000000..be9cd9805 --- /dev/null +++ b/430/_sources/release/release-336.md.txt @@ -0,0 +1,17 @@ +# Release 336 (16 Jun 2020) + +## General + +- Fix failure when querying timestamp columns from older clients. ({issue}`4036`) +- Improve reporting of configuration errors. ({issue}`4050`) +- Fix rare failure when recording server stats in T-Digests. ({issue}`3965`) + +## Security + +- Add table access rules to {doc}`/security/file-system-access-control`. ({issue}`3951`) +- Add new `default` system access control that allows all operations except user impersonation. ({issue}`4040`) + +## Hive connector + +- Fix incorrect query results when reading Parquet files with predicates + when `hive.parquet.use-column-names` is set to `false` (the default). ({issue}`3574`) diff --git a/430/_sources/release/release-337.md.txt b/430/_sources/release/release-337.md.txt new file mode 100644 index 000000000..2257c0fbf --- /dev/null +++ b/430/_sources/release/release-337.md.txt @@ -0,0 +1,66 @@ +# Release 337 (25 Jun 2020) + +:::{Note} +This release fixes a potential security vulnerability when secure internal communication is enabled in a cluster. A malicious +attacker can take advantage of this vulnerability to escalate privileges to internal APIs. We encourage everyone to upgrade as soon +as possible. +::: + +## General + +- Fix incorrect results for inequality join involving `NaN`. ({issue}`4120`) +- Fix peak non-revocable memory metric in event listener. ({issue}`4096`) +- Fix queued query JMX stats. ({issue}`4129`) +- Fix rendering of types in the output of `DESCRIBE INPUT`. ({issue}`4023`) +- Improve performance of queries involving comparisons between `DOUBLE` or `REAL` values and integer values. ({issue}`3533`) +- Reduce idle CPU consumption in coordinator. ({issue}`3990`) +- Add peak non-revocable memory metric to query stats. ({issue}`4096`) +- Add support for variable-precision `TIMESTAMP WITH TIME ZONE` type ({issue}`3947`) +- Add support for `IN` predicate with subqueries in outer join condition. ({issue}`4151`) +- Add support for quantified comparisons (e.g., `> ALL (...)`) in aggregation queries. ({issue}`4128`) +- Add {doc}`/connector/druid`. ({issue}`3522`) +- Add {func}`translate` function. ({issue}`4080`) +- Reduce worker graceful shutdown duration. ({issue}`4192`) + +## Security + +- Disable insecure authentication over HTTP by default when HTTPS with authentication is enabled. This + can be overridden via the `http-server.authentication.allow-insecure-over-http` configuration property. ({issue}`4199`) +- Add support for insecure authentication over HTTPS to the Web UI. ({issue}`4199`) +- Add {ref}`system-file-auth-system-information` which control the ability of a + user to access to read and write system management information. + ({issue}`4199`) +- Disable user impersonation in default system security. ({issue}`4082`) + +## Elasticsearch connector + +- Add support for password authentication. ({issue}`4165`) + +## Hive connector + +- Fix reading CSV tables with `separatorChar`, `quoteChar` or `escapeChar` table property + containing more than one character. For compatibility with Hive, only first character is considered + and remaining are ignored. ({issue}`3891`) +- Improve performance of `INSERT` queries writing to bucketed tables when some buckets do not contain any data. ({issue}`1375`) +- Improve performance of queries reading Parquet data with predicates on `timestamp` columns. ({issue}`4104`) +- Improve performance for join queries over partitioned tables. ({issue}`4156`) +- Add support for `null_format` table property for tables using TextFile storage format ({issue}`4056`) +- Add support for `null_format` table property for tables using RCText and SequenceFile + storage formats ({issue}`4143`) +- Add optimized Parquet writer. The new writer is disabled by default, and can be enabled with the + `parquet_optimized_writer_enabled` session property or the `hive.parquet.optimized-writer.enabled` configuration + property. ({issue}`3400`) +- Add support caching data in Azure Data Lake and AliyunOSS storage. ({issue}`4213`) +- Fix failures when caching data from Google Cloud Storage. ({issue}`4213`) +- Support ACID data files naming used when direct inserts are enabled in Hive (HIVE-21164). + Direct inserts is an upcoming feature in Hive 4. ({issue}`4049`) + +## PostgreSQL connector + +- Improve performance of aggregation queries by computing aggregations within PostgreSQL database. + Currently, the following aggregate functions are eligible for pushdown: + `count`, `min`, `max`, `sum` and `avg`. ({issue}`3881`) + +## Base-JDBC connector library + +- Implement framework for aggregation pushdown. ({issue}`3881`) diff --git a/430/_sources/release/release-338.md.txt b/430/_sources/release/release-338.md.txt new file mode 100644 index 000000000..b55462cfd --- /dev/null +++ b/430/_sources/release/release-338.md.txt @@ -0,0 +1,58 @@ +# Release 338 (07 Jul 2020) + +## General + +* Fix incorrect results when joining tables on a masked column. ({issue}`4251`) +* Fix planning failure when multiple columns have a mask. ({issue}`4322`) +* Fix incorrect comparison for `TIMESTAMP WITH TIME ZONE` values with precision larger than 3. ({issue}`4305`) +* Fix incorrect rounding for timestamps before 1970-01-01. ({issue}`4370`) +* Fix query failure when using `VALUES` with a floating point `NaN` value. ({issue}`4119`) +* Fix query failure when joining tables on a `real` or `double` column and one of the joined tables + contains `NaN` value. ({issue}`4272`) +* Fix unauthorized error for internal requests to management endpoints. ({issue}`4304`) +* Fix memory leak while using dynamic filtering. ({issue}`4228`) +* Improve dynamic partition pruning for broadcast joins. ({issue}`4262`) +* Add support for setting column comments via the `COMMENT ON COLUMN` syntax. ({issue}`2516`) +* Add compatibility mode for legacy clients when rendering datetime type names with default precision + in `information_schema` tables. This can be enabled via the `deprecated.omit-datetime-type-precision` + configuration property or `omit_datetime_type_precision` session property. ({issue}`4349`, {issue}`4377`) +* Enforce `NOT NULL` column declarations when writing data. ({issue}`4144`) + +## JDBC driver + +* Fix excessive CPU usage when reading query results. ({issue}`3928`) +* Implement `DatabaseMetaData.getClientInfoProperties()`. ({issue}`4318`) + +## Elasticsearch connector + +* Add support for reading numeric values encoded as strings. ({issue}`4341`) + +## Hive connector + +* Fix incorrect query results when Parquet file has no min/max statistics for an integral column. ({issue}`4200`) +* Fix query failure when reading from a table partitioned on a `real` or `double` column containing + a `NaN` value. ({issue}`4266`) +* Fix sporadic failure when writing to bucketed sorted tables on S3. ({issue}`2296`) +* Fix handling of strings when translating Hive views. ({issue}`3266`) +* Do not require cache directories to be configured on coordinator. ({issue}`3987`, {issue}`4280`) +* Fix Azure ADL caching support. ({issue}`4240`) +* Add support for setting column comments. ({issue}`2516`) +* Add hidden `$partition` column for partitioned tables that contains the partition name. ({issue}`3582`) + +## Kafka connector + +* Fix query failure when a column is projected and also referenced in a query predicate + when reading from Kafka topic using `RAW` decoder. ({issue}`4183`) + +## MySQL connector + +* Fix type mapping for unsigned integer types. ({issue}`4187`) + +## Oracle connector + +* Exclude internal schemas (e.g., sys) from schema listings. ({issue}`3784`) +* Add support for connection pooling. ({issue}`3770`) + +## Base-JDBC connector library + +* Exclude the underlying database's `information_schema` from schema listings. ({issue}`3834`) diff --git a/430/_sources/release/release-339.md.txt b/430/_sources/release/release-339.md.txt new file mode 100644 index 000000000..986241ae6 --- /dev/null +++ b/430/_sources/release/release-339.md.txt @@ -0,0 +1,70 @@ +# Release 339 (21 Jul 2020) + +## General + +* Add {func}`approx_most_frequent`. ({issue}`3425`) +* Physical bytes scan limit for queries can be configured via `query.max-scan-physical-bytes` configuration property + and `query_max_scan_physical_bytes` session property. ({issue}`4075`) +* Remove support for addition and subtraction between `TIME` and `INTERVAL YEAR TO MONTH` types. ({issue}`4308`) +* Fix planning failure when join criteria contains subqueries. ({issue}`4380`) +* Fix failure when subquery appear in window function arguments. ({issue}`4127`) +* Fix failure when subquery in `WITH` clause contains hidden columns. ({issue}`4423`) +* Fix failure when referring to type names with different case in a `GROUP BY` clause. ({issue}`2960`) +* Fix failure for queries involving `DISTINCT` when expressions in `ORDER BY` clause differ by case from expressions in `SELECT` clause. ({issue}`4233`) +* Fix incorrect type reporting for `TIMESTAMP` and `TIMESTAMP WITH TIME ZONE` for legacy clients. ({issue}`4408`) +* Fix failure when querying nested `TIMESTAMP` or `TIMESTAMP WITH TIME ZONE` for legacy clients. ({issue}`4475`, {issue}`4425`) +* Fix failure when parsing timestamps with time zone with an offset of the form `+NNNN`. ({issue}`4490`) + +## JDBC driver + +* Fix reading `TIMESTAMP` and `TIMESTAMP WITH TIME ZONE` values with a negative year + or a year higher than 9999. ({issue}`4364`) +* Fix incorrect column size metadata for `TIMESTAMP` and `TIMESTAMP WITH TIME ZONE` types. ({issue}`4411`) +* Return correct value from `ResultSet.getDate()`, `ResultSet.getTime()` and `ResultSet.getTimestamp()` methods + when session zone is set to a different zone than the default zone of the JVM the JDBC is run in. + The previous behavior can temporarily be restored using `useSessionTimeZone` JDBC connection + parameter. ({issue}`4017`) + +## Druid connector + +* Fix handling of table and column names containing non-ASCII characters. ({issue}`4312`) + +## Hive connector + +* Make `location` parameter optional for the `system.register_partition` procedure. ({issue}`4443`) +* Avoid creating tiny splits at the end of block boundaries. ({issue}`4485`) +* Remove requirement to configure `metastore.storage.schema.reader.impl` in Hive 3.x metastore + to let Presto access CSV tables. ({issue}`4457`) +* Fail query if there are bucket files outside of the bucket range. + Previously, these extra files were skipped. ({issue}`4378`) +* Fix a query failure when reading from Parquet file containing `real` or `double` `NaN` values, + if the file was written by a non-conforming writer. ({issue}`4267`) + +## Kafka connector + +* Add insert support for Avro. ({issue}`4418`) +* Add insert support for CSV. ({issue}`4287`) + +## Kudu connector + +* Add support for grouped execution. It can be enabled with the `kudu.grouped-execution.enabled` + configuration property or the `grouped_execution` session property. ({issue}`3715`) + +## MongoDB connector + +* Allow querying Azure Cosmos DB. ({issue}`4415`) + +## Oracle connector + +* Allow providing credentials via the `connection-user` and `connection-password` + configuration properties. These properties were previously ignored if connection pooling + was enabled. ({issue}`4430`) + +## Phoenix connector + +* Fix handling of row key definition with white space. ({issue}`3251`) + +## SPI + +* Allow connectors to wait for dynamic filters before splits are generated via the new + `DynamicFilter` object passed to `ConnectorSplitManager.getSplits()`. ({issue}`4224`) diff --git a/430/_sources/release/release-340.md.txt b/430/_sources/release/release-340.md.txt new file mode 100644 index 000000000..83438fc6b --- /dev/null +++ b/430/_sources/release/release-340.md.txt @@ -0,0 +1,72 @@ +# Release 340 (8 Aug 2020) + +## General + +* Add support for query parameters in `LIMIT`, `OFFSET` and `FETCH FIRST` clauses. ({issue}`4529`, {issue}`4601`) +* Add experimental support for recursive queries. ({issue}`4250`) +* Add {func}`bitwise_left_shift`, {func}`bitwise_right_shift` and {func}`bitwise_right_shift_arithmetic`. ({issue}`740`) +* Add {func}`luhn_check`. ({issue}`4011`) +* Add `IF EXISTS `and `IF NOT EXISTS` syntax to `ALTER TABLE`. ({issue}`4651`) +* Include remote host in error info for page transport errors. ({issue}`4511`) +* Improve minimum latency for dynamic partition pruning. ({issue}`4388`) +* Reduce cluster load by cancelling query stages from which data is no longer required. ({issue}`4290`) +* Reduce query memory usage by improving retained size estimation for `VARCHAR` and `CHAR` types. ({issue}`4123`) +* Improve query performance for queries containing {func}`starts_with`. ({issue}`4669`) +* Improve performance of queries that use `DECIMAL` data type. ({issue}`4730`) +* Fix failure when `GROUP BY` clause contains duplicate expressions. ({issue}`4609`) +* Fix potential hang during query planning ({issue}`4635`). + +## Security + +* Fix unprivileged access to table's schema via `CREATE TABLE LIKE`. ({issue}`4472`) + +## JDBC driver + +* Fix handling of dates before 1582-10-15. ({issue}`4563`) +* Fix handling of timestamps before 1900-01-01. ({issue}`4563`) + +## Elasticsearch connector + +* Fix failure when index mapping is missing. ({issue}`4535`) + +## Hive connector + +* Allow creating a table with `external_location` when schema's location is not valid. ({issue}`4069`) +* Add read support for tables that were created as non-transactional and converted to be + transactional later. ({issue}`2293`) +* Allow creation of transactional tables. Note that writing to transactional tables + is not yet supported. ({issue}`4516`) +* Add `hive.metastore.glue.max-error-retries` configuration property for the + number of retries performed when accessing the Glue metastore. ({issue}`4611`) +* Support using Java KeyStore files for Thrift metastore TLS configuration. ({issue}`4432`) +* Expose hit rate statistics for Hive metastore cache via JMX. ({issue}`4458`) +* Improve performance when querying a table with large files and with `skip.header.line.count` property set to 1. ({issue}`4513`) +* Improve performance of reading JSON tables. ({issue}`4705`) +* Fix query failure when S3 data location contains a `_$folder$` marker object. ({issue}`4552`) +* Fix failure when referencing nested fields of a `ROW` type when table and partition metadata differs. ({issue}`3967`) + +## Kafka connector + +* Add insert support for Raw data format. ({issue}`4417`) +* Add insert support for JSON. ({issue}`4477`) +* Remove unused `kafka.connect-timeout` configuration properties. ({issue}`4664`) + +## MongoDB connector + +* Add `mongodb.max-connection-idle-time` properties to limit the maximum idle time of a pooled connection. ({issue}`4483`) + +## Phoenix connector + +* Add table level property to specify data block encoding when creating tables. ({issue}`4617`) +* Fix query failure when listing schemas. ({issue}`4560`) + +## PostgreSQL connector + +* Push down {func}`count` aggregations over constant expressions. + For example, `SELECT count(1)`. ({issue}`4362`) + +## SPI + +* Expose information about query type in query Event Listener. ({issue}`4592`) +* Add support for TopN pushdown via the `ConnectorMetadata.applyLimit()` method. ({issue}`4249`) +* Deprecate the older variants of `ConnectorSplitManager.getSplits()`. ({issue}`4508`) diff --git a/430/_sources/release/release-341.md.txt b/430/_sources/release/release-341.md.txt new file mode 100644 index 000000000..456368a67 --- /dev/null +++ b/430/_sources/release/release-341.md.txt @@ -0,0 +1,168 @@ +# Release 341 (8 Sep 2020) + +## General + +* Add support for variable-precision `TIME` type. ({issue}`4381`) +* Add support for variable precision `TIME WITH TIME ZONE` type. ({issue}`4905`) +* Add {doc}`/connector/iceberg`. +* Add {func}`human_readable_seconds` function. ({issue}`4344`) +* Add [`reverse()`](function-reverse-varbinary) function for `VARBINARY`. ({issue}`4741`) +* Add support for {func}`extract` for `timestamp(p) with time zone` with values of `p` other than 3. ({issue}`4867`) +* Add support for correlated subqueries in recursive queries. ({issue}`4877`) +* Add [](optimizer-rule-stats) system table. ({issue}`4659`) +* Report dynamic filters statistics. ({issue}`4440`) +* Improve query scalability when new nodes are added to cluster. ({issue}`4294`) +* Improve error message when JSON parsing fails. ({issue}`4616`) +* Reduce latency when dynamic filtering is in use. ({issue}`4924`) +* Remove support for political time zones in `TIME WITH TIME ZONE` type. ({issue}`191`) +* Remove deprecated `reorder_joins` session property. ({issue}`5027`) +* Remove the `deprecated.legacy-timestamp` configuration property and the `legacy_timestamp` session property. ({issue}`4799`) +* Change timestamp operations to match the SQL specification. The value of a `TIMESTAMP` + type is not affected by the session time zone. ({issue}`37`) +* Preserve precision when applying `AT TIME ZONE` to values of type `TIMESTAMP`. ({issue}`4866`) +* Fix serialization of `NULL` values in `ROW`, `MAP` and `ARRAY` types for old Presto clients. ({issue}`4778`) +* Fix failure when aggregation query contains duplicate expressions. ({issue}`4872`) +* Fix compiler failure when querying timestamps with a precision greater than 6. ({issue}`4824`) +* Fix parsing failure of timestamps due to daylight saving changes. ({issue}`37`) +* Fix failure when calling {func}`extract` with `TIMEZONE_HOUR` and `TIMEZONE_MINUTE` for + `TIMESTAMP WITH TIME ZONE` type. ({issue}`4867`) +* Fix query deadlock for connectors that wait for dynamic filters. ({issue}`4946`) +* Fix failure when `TIME` or `TIMESTAMP` subtraction returns a negative value. ({issue}`4847`) +* Fix failure when duplicate expressions appear in `DISTINCT` clause. ({issue}`4787`) +* Fix failure for certain join queries during spilling or when available memory is low. ({issue}`4994`) +* Fix issue where the `query_max_scan_physical_bytes` session property was ignored if + the `query.max-scan-physical-bytes` configuration property was not defined. ({issue}`5009`) +* Correctly compute sample ratio when `TABLESAMPLE` is used with a fractional percentage. ({issue}`5074`) +* Fail queries with a proper error message when `TABLESAMPLE` is used with a non-numeric sample ratio. ({issue}`5074`) +* Fail with an explicit error rather than `OutOfMemoryError` for certain operations. ({issue}`4890`) + +## Security + +* Add [Salesforce password authentication](/security/salesforce). ({issue}`4372`) +* Add support for interpolating [secrets](/security/secrets) into `access-control.properties`. ({issue}`4854`) +* Only request HTTPS client certificate when certificate authentication is enabled. ({issue}`4804`) +* Add {doc}`/security/user-mapping` support for uppercasing or lowercasing usernames. ({issue}`4736`) + +## Web UI + +* Fix display of physical input read time in detailed query view. ({issue}`4962`) + +## JDBC driver + +* Implement `ResultSet.getStatement()`. ({issue}`4957`) + +## BigQuery connector + +* Add support for hourly partitioned tables. ({issue}`4968`) +* Redact the value of `bigquery.credentials-key` in the server log. ({issue}`4968`) + +## Cassandra connector + +* Map Cassandra `TIMESTAMP` type to Presto `TIMESTAMP(3) WITH TIME ZONE` type. ({issue}`2269`) + +## Hive connector + +* Skip stripes and row groups based on timestamp statistics for ORC files. ({issue}`1147`) +* Skip S3 objects with the `DeepArchive` storage class (in addition to the `Glacier` + storage class) when `hive.s3.skip-glacier-objects` is enabled. ({issue}`5002`) +* Use a temporary staging directory for temporary files when writing to sorted bucketed tables. + This allows using a more efficient file system for temporary files. ({issue}`3434`) +* Fix metastore cache invalidation for `GRANT` and `REVOKE`. ({issue}`4768`) +* Add Parquet and RCBinary [configuration properties](hive-configuration-properties) `hive.parquet.time-zone` + and `hive.rcfile.time-zone` to adjust binary timestamp values to a specific time zone. For Hive 3.1+, this + should be set to UTC. The default value is the JVM default time zone, for backwards compatibility with + earlier versions of Hive. ({issue}`4799`) +* Add ORC [configuration property](hive-configuration-properties) `hive.orc.time-zone` to set the default + time zone for legacy ORC files that did not declare a time zone. ({issue}`4799`) +* Replace the `hive.time-zone` configuration property with format specific properties: `hive.orc.time-zone`, + `hive.parquet.time-zone`, `hive.rcfile.time-zone`. ({issue}`4799`) +* Allow using the cluster default role with S3 security mapping. ({issue}`4931`) +* Remove support for bucketing on timestamp. The definition of the hash function for timestamp + incorrectly depends on the storage time zone and can result in incorrect results. ({issue}`4759`) +* Decrease the number of requests to the Glue metastore when fetching partitions. + This helps avoid hitting rate limits and decreases service costs. ({issue}`4938`) +* Match the existing user and group of the table or partition when creating new files on HDFS. ({issue}`4414`) +* Fix invalid timestamp values for nested data in Text, Avro, SequenceFile, JSON and CSV formats. ({issue}`4799`) +* Fix query failure when reading an ORC ACID table with a filter after the table + underwent a minor table compaction. ({issue}`4622`) +* Fix incorrect query results when reading an ORC ACID table that has deleted rows + and underwent a minor compaction. ({issue}`4623`) +* Fix query failure when storage caching is enabled and cached data is evicted during query execution. ({issue}`3580`) + +## JMX connector + +* Change `timestamp` column type in history tables to `TIMESTAMP WITH TIME ZONE`. ({issue}`4753`) + +## Kafka connector + +* Preserve time zone when parsing `TIMESTAMP WITH TIME ZONE` values. ({issue}`4799`) + +## Kinesis connector + +* Preserve time zone when parsing `TIMESTAMP WITH TIME ZONE` values. ({issue}`4799`) + +## Kudu connector + +* Fix delete when applied on table having primary key of decimal type. ({issue}`4683`) + +## Local File connector + +* Change `timestamp` column type to `TIMESTAMP WITH TIME ZONE`. ({issue}`4752`) + +## MySQL connector + +* Improve performance of aggregation queries by pushing the aggregation computation into the MySQL database. + Currently, the following aggregate functions are eligible for pushdown: `count`, `min`, `max`, + `sum` and `avg`. ({issue}`4138`) + +## Oracle connector + +* Add `oracle.connection-pool.inactive-timeout` configuration property to specify how long + pooled connection can be inactive before it is closed. It defaults to 20 minutes. ({issue}`4779`) +* Add support for database internationalization. ({issue}`4775`) +* Add resilience to momentary connection authentication issues. ({issue}`4947`) +* Allowing forcing the mapping of certain types to `VARCHAR`. This can be enabled by + setting the `jdbc-types-mapped-to-varchar` configuration property to a comma-separated + list of type names. ({issue}`4955`) +* Prevent query failure for pushdown of predicates involving a large number of conjuncts. ({issue}`4918`) + +## Phoenix connector + +* Fix overwriting of former value when insert is applied without specifying that column. ({issue}`4670`) + +## Pinot connector + +* Add support for `REAL` and `INTEGER` types. ({issue}`4725`) +* Add support for functions in pass-through queries. ({issue}`4801`) +* Enforce a limit on the number of rows fetched from Pinot. This can be configured via the + `pinot.max-rows-per-split-for-segment-queries` configuration property. ({issue}`4723`) +* Fix incorrect results for `count(*)` queries. ({issue}`4802`) +* Fix incorrect results for queries involving {func}`avg` over columns of type `long`, `int`, or `float`. ({issue}`4802`) +* Fix incorrect results when columns in pass-through query do not match selected columns. ({issue}`4802`) + +## Prometheus connector + +* Change the type of the `timestamp` column to `TIMESTAMP(3) WITH TIME ZONE` type. ({issue}`4799`) + +## PostgreSQL connector + +* Improve performance of aggregation queries with predicates by pushing the computation + of both the filtering and aggregations into the PostgreSQL server where possible. ({issue}`4111`) +* Fix handling of PostgreSQL arrays when `unsupported-type-handling` is set to `CONVERT_TO_VARCHAR`. ({issue}`4981`) + +## Raptor connector + +* Remove the `storage.shard-day-boundary-time-zone` configuration property, which was used to work + around legacy timestamp semantics in Presto. ({issue}`4799`) + +## Redis connector + +* Preserve time zone when parsing `TIMESTAMP WITH TIME ZONE` values. ({issue}`4799`) + +## SPI + +* The `TIMESTAMP` type is encoded as a number of fractional seconds from `1970-01-01 00:00:00` in the proleptic + Gregorian calendar. This value is no longer adjusted to the session time zone. Timestamps with precision less + than or equal to 3 are now represented in microseconds. ({issue}`4799`) +* Remove `isLegacyTimestamp()` from `ConnectorSession`. ({issue}`4799`) +* Enable connectors to wait for dynamic filters before producing data on worker nodes. ({issue}`3414`) diff --git a/430/_sources/release/release-342.md.txt b/430/_sources/release/release-342.md.txt new file mode 100644 index 000000000..00cdf6de5 --- /dev/null +++ b/430/_sources/release/release-342.md.txt @@ -0,0 +1,76 @@ +# Release 342 (24 Sep 2020) + +## General + +* Add {func}`from_iso8601_timestamp_nanos` function. ({issue}`5048`) +* Improve performance of queries that use the `DECIMAL` type. ({issue}`4886`) +* Improve performance of queries involving `IN` with subqueries by extending support for dynamic filtering. ({issue}`5017`) +* Improve performance and latency of queries leveraging dynamic filters. ({issue}`4988`) +* Improve performance of queries joining tables with missing or incomplete column statistics when + cost based optimization is enabled (which is the default). ({issue}`5141`) +* Reduce latency for queries that perform a broadcast join of a large table. ({issue}`5237`) +* Allow collection of dynamic filters for joins with large build side using the + `enable-large-dynamic-filters` configuration property or the `enable_large_dynamic_filters` + session property. ({issue}`5262`) +* Fix query failure when lambda expression references a table column containing a dot. ({issue}`5087`) + +## Atop connector + +* Fix incorrect query results when query contains predicates on `start_time` or `end_time` column. ({issue}`5125`) + +## Elasticsearch connector + +* Allow reading boolean values stored as strings. ({issue}`5269`) + +## Hive connector + +* Add support for S3 encrypted files. ({issue}`2536`) +* Add support for ABFS OAuth authentication. ({issue}`5052`) +* Support reading timestamp with microsecond or nanosecond precision. This can be enabled with the + `hive.timestamp-precision` configuration property. ({issue}`4953`) +* Allow overwrite on insert by default using the `hive.insert-existing-partitions-behavior` configuration property. ({issue}`4999`) +* Allow delaying table scans until dynamic filtering can be performed more efficiently. This can be enabled + using the `hive.dynamic-filtering-probe-blocking-timeout` configuration property or the + `dynamic_filtering_probe_blocking_timeout` session property. ({issue}`4991`) +* Disable matching the existing user and group of the table or partition when creating new files on HDFS. + The functionality was added in 341 and is now disabled by default. It can be enabled using the + `hive.fs.new-file-inherit-ownership` configuration property. ({issue}`5187`) +* Improve performance when reading small files in `RCTEXT` or `RCBINARY` format. ({issue}`2536`) +* Improve planning time for queries with non-equality filters on partition columns when using the Glue metastore. ({issue}`5060`) +* Improve performance when reading `JSON` and `CSV` file formats. ({issue}`5142`) + +## Iceberg connector + +* Fix partition transforms for temporal columns for dates before 1970. ({issue}`5273`) + +## Kafka connector + +* Expose message headers as a `_headers` column of `MAP(VARCHAR, ARRAY(VARBINARY))` type. ({issue}`4462`) +* Add write support for `TIME`, `TIME WITH TIME ZONE`, `TIMESTAMP` and `TIMESTAMP WITH TIME ZONE` + for Kafka connector when using the JSON encoder. ({issue}`4743`) +* Remove JSON decoder support for nonsensical combinations of input-format-type / data-type. The following + combinations are no longer supported: ({issue}`4743`) + - `rfc2822`: `DATE`, `TIME`, `TIME WITH TIME ZONE` + - `milliseconds-since-epoch`: `TIME WITH TIME ZONE`, `TIMESTAMP WITH TIME ZONE` + - `seconds-since-epoch`: `TIME WITH TIME ZONE`, `TIMESTAMP WITH TIME ZONE` + +## MySQL connector + +* Improve performance of `INSERT` queries when GTID mode is disabled in MySQL. ({issue}`4995`) + +## PostgreSQL connector + +* Add support for variable-precision TIMESTAMP and TIMESTAMP WITH TIME ZONE types. ({issue}`5124`, {issue}`5105`) + +## SQL Server connector + +* Fix failure when inserting `NULL` into a `VARBINARY` column. ({issue}`4846`) +* Improve performance of aggregation queries by computing aggregations within SQL Server database. + Currently, the following aggregate functions are eligible for pushdown: + `count`, `min`, `max`, `sum` and `avg`. ({issue}`4139`) + +## SPI + +* Add `DynamicFilter.isAwaitable()` method that returns whether or not the dynamic filter is complete + and can be awaited for using the `isBlocked()` method. ({issue}`5043`) +* Enable connectors to wait for dynamic filters derived from replicated joins before generating splits. ({issue}`4685`) diff --git a/430/_sources/release/release-343.md.txt b/430/_sources/release/release-343.md.txt new file mode 100644 index 000000000..614f97b32 --- /dev/null +++ b/430/_sources/release/release-343.md.txt @@ -0,0 +1,14 @@ +# Release 343 (25 Sep 2020) + +## BigQuery connector + +* Add support for yearly partitioned tables. ({issue}`5298`) + +## Hive connector + +* Fix query failure when read from or writing to a bucketed table containing a column of `timestamp` type. ({issue}`5295`) + +## SQL Server connector + +* Improve performance of aggregation queries with `stddev`, `stddev_samp`, `stddev_pop`, `variance`, `var_samp`, `var_pop` + aggregate functions by computing aggregations within SQL Server database. ({issue}`5299`) diff --git a/430/_sources/release/release-344.md.txt b/430/_sources/release/release-344.md.txt new file mode 100644 index 000000000..476748dba --- /dev/null +++ b/430/_sources/release/release-344.md.txt @@ -0,0 +1,77 @@ +# Release 344 (9 Oct 2020) + +## General + +* Add {func}`murmur3` function. ({issue}`5054`) +* Add {func}`from_unixtime_nanos` function. ({issue}`5046`) +* Add [T-Digest](/functions/tdigest) type and functions. ({issue}`5158`) +* Improve performance and latency of queries leveraging dynamic filters. ({issue}`5081`, {issue}`5340`) +* Add `dynamic-filtering.service-thread-count` config property to specify number of + threads used for processing dynamic filters on coordinator. ({issue}`5341`) +* Extend {doc}`/security/secrets` environment variable substitution to allow + multiple replacements in a single configuration property. ({issue}`4345`) +* Remove the `fast-inequality-joins` configuration property. This feature is always enabled. ({issue}`5375`) +* Use `timestamp(3) with time zone` rather than `timestamp(3)` for the `queries`, + `transactions`, and `tasks` tables in `system.runtime`. ({issue}`5464`) +* Improve performance and accuracy of {func}`approx_percentile`. ({issue}`5158`) +* Improve performance of certain cross join queries. ({issue}`5276`) +* Prevent potential query deadlock when query runs out of memory. ({issue}`5289`) +* Fix failure due to rounding error when casting between two `timestamp` types with precision + higher than 6. ({issue}`5310`) +* Fix failure due to rounding error when casting between two `timestamp with time zone` types + with precision higher than 3. ({issue}`5371`) +* Fix column pruning for `EXPLAIN ANALYZE`. ({issue}`4760`) +* Fix incorrect timestamp values returned by the `queries`, `transactions`, + and `tasks` tables in `system.runtime`. ({issue}`5462`) + +## Security + +```{warning} +The file-based system and catalog access controls have changed in ways that reduce or increase permissions. +Please, read these release notes carefully. +``` + +* Change file-based catalog access control from deny to allow when table, schema, or session + property rules are not defined. ({issue}`5039`) +* Add missing table rule checks for table and view DDL in file-based system access control. ({issue}`5039`) +* Add missing schema rule checks for create schema in file-based system access control. ({issue}`5039`) +* Add session property rules to file-based system access control. ({issue}`5039`) +* Add catalog regex to table and schema rules in file-based system access control. ({issue}`5039`) +* Change create, rename, alter, and drop table in file-based system controls to only check for + table ownership. ({issue}`5039`) +* Change file-based system access control to support files without catalog rules defined. In this case, + all access to catalogs is allowed. ({issue}`5039`) +* Change file-based system and catalog access controls to only show catalogs, schemas, and tables a user + has permissions on. ({issue}`5039`) +* Change file-based catalog access control to deny permissions inspection and manipulation. ({issue}`5039`) +* Add [file-based group provider](/security/group-file). ({issue}`5028`) + +## Hive connector + +* Add support for `hive.security=allow-all`, which allows to skip all authorization checks. ({issue}`5416`) +* Support Kerberos authentication for Hudi tables. ({issue}`5472`) +* Allow hiding Delta Lake tables from table listings such as `SHOW TABLES` or `information_schema.tables`, + as these tables cannot be queried by the Hive connector. This be enabled using the `hive.hide-delta-lake-tables` + configuration property. ({issue}`5430`) +* Improve query concurrency by listing data files more efficiently. ({issue}`5260`) +* Fix Parquet encoding for timestamps before 1970-01-01. ({issue}`5364`) + +## Kafka connector + +* Expose message timestamp via `_timestamp` internal column. ({issue}`4805`) +* Add predicate pushdown for `_timestamp`, `_partition_offset` and `_partition_id` columns. ({issue}`4805`) + +## Phoenix connector + +* Fix query failure when a column name in `CREATE TABLE` requires quoting. ({issue}`3601`) + +## PostgreSQL connector + +* Add support for setting a column comment. ({issue}`5307`) +* Add support for variable-precision `time` type. ({issue}`5342`) +* Allow `CREATE TABLE` and `CREATE TABLE AS` with `timestamp` and `timestamp with time zone` with precision higher than 6. + The resulting column will be declared with precision of 6, maximal supported by PostgreSQL. ({issue}`5342`) + +## SQL Server connector + +* Improve performance of queries with aggregations and `WHERE` clause. ({issue}`5327`) diff --git a/430/_sources/release/release-345.md.txt b/430/_sources/release/release-345.md.txt new file mode 100644 index 000000000..251978546 --- /dev/null +++ b/430/_sources/release/release-345.md.txt @@ -0,0 +1,66 @@ +# Release 345 (23 Oct 2020) + +## General + +* Add {func}`concat_ws` function. ({issue}`4680`) +* Add support for {func}`extract` for `time with time zone` values with precision other than 3. ({issue}`5539`) +* Add dynamic filtering support for right joins. ({issue}`5555`) +* Add support for catalog session properties to the file and database backed session property managers. ({issue}`5330`) +* Improve query scalability by increasing the default broadcast join buffer size to 200MB. + Broadcast join buffer size can be set using the `sink.max-broadcast-buffer-size` configuration property. ({issue}`5551`) +* Improve query performance by allowing larger dynamic filters. ({issue}`5529`) +* Improve performance for join queries where join keys are not of the same type. ({issue}`5461`) +* Improve performance of encrypted spilling. ({issue}`5557`) +* Improve performance of queries that use the `decimal` type. ({issue}`5181`) + +## Security + +* Add support for JSON Web Key (JWK) to the existing JSON Web Token (JWT) authenticator. This is enabled by + setting the `jwt.key-file` configuration property to a `http` or `https` url. ({issue}`5419`) +* Add column security, column mask and row filter to file-based access controls. ({issue}`5460`) +* Enforce access control for column references in `USING` clause. ({issue}`5620`) + +## JDBC driver + +* Add `source` parameter for directly setting the source name for a query. ({issue}`4739`) + +## Hive connector + +* Add support for `INSERT` and `DELETE` for ACID tables. ({issue}`5402`) +* Apply `hive.domain-compaction-threshold` to dynamic filters. ({issue}`5365`) +* Add support for reading Parquet timestamps encoded as microseconds. ({issue}`5483`) +* Improve translation of Hive views. ({issue}`4661`) +* Improve storage caching by better distributing files across workers. ({issue}`5621`) +* Fix disk space accounting for storage caching. ({issue}`5621`) +* Fix failure when reading Parquet `timestamp` columns encoded as `int64`. ({issue}`5443`) + +## MongoDB connector + +* Add support for adding columns. ({issue}`5512`) +* Fix incorrect result for `IS NULL` predicates on fields that do not exist in the document. ({issue}`5615`) + +## MemSQL connector + +* Fix representation for many MemSQL types. ({issue}`5495`) +* Prevent a query failure when table column name contains a semicolon by explicitly forbidding such names. ({issue}`5495`) +* Add support for case-insensitive table name matching. ({issue}`5495`) + +## MySQL connector + +* Improve performance of queries with aggregations and `LIMIT` clause (but without `ORDER BY`). ({issue}`5261`) + +## PostgreSQL connector + +* Improve performance of queries with aggregations and `LIMIT` clause (but without `ORDER BY`). ({issue}`5261`) + +## Redshift connector + +* Add support for setting column comments. ({issue}`5397`) + +## SQL Server connector + +* Improve performance of queries with aggregations and `LIMIT` clause (but without `ORDER BY`). ({issue}`5261`) + +## Thrift connector + +* Fix handling of timestamp values. ({issue}`5596`) diff --git a/430/_sources/release/release-346.md.txt b/430/_sources/release/release-346.md.txt new file mode 100644 index 000000000..c520149f9 --- /dev/null +++ b/430/_sources/release/release-346.md.txt @@ -0,0 +1,118 @@ +# Release 346 (10 Nov 2020) + +## General + +* Add support for `RANGE BETWEEN PRECEDING AND FOLLOWING` window frames. ({issue}`609`) +* Add support for window frames based on `GROUPS`. ({issue}`5713`) +* Add support for {func}`extract` with `TIMEZONE_HOUR` and `TIMEZONE_MINUTE` for `time with time zone` values. ({issue}`5668`) +* Add SQL syntax for `GRANT` and `REVOKE` on schema. This is not yet used by any connector. ({issue}`4396`) +* Add `ALTER TABLE ... SET AUTHORIZATION` syntax to allow changing the table owner. ({issue}`5717`) +* Make `EXPLAIN` more readable for queries containing `timestamp` or `timestamp with time zone` constants. ({issue}`5683`) +* Improve performance for queries with inequality conditions. ({issue}`2674`) +* Improve performance of queries with uncorrelated `IN` clauses. ({issue}`5582`) +* Use consistent NaN behavior for {func}`least`, {func}`greatest`, + {func}`array_min`, {func}`array_max`, {func}`min`, {func}`max`, + {func}`min_by`, and {func}`max_by`. + NaN is only returned when it is the only value (except for null + which are ignored for aggregation functions). ({issue}`5851`) +* Restore previous null handling for {func}`least` and {func}`greatest`. ({issue}`5787`) +* Restore previous null handling for {func}`array_min` and {func}`array_max`. ({issue}`5787`) +* Remove configuration properties `arrayagg.implementation`, + `multimapagg.implementation`, and `histogram.implementation`. ({issue}`4581`) +* Fix incorrect handling of negative offsets for the `time with time zone` type. ({issue}`5696`) +* Fix incorrect result when casting `time(p)` to `timestamp(p)` for precisions higher than 6. ({issue}`5736`) +* Fix incorrect query results when comparing a `timestamp` column with a `timestamp with time zone` constant. ({issue}`5685`) +* Fix improper table alias visibility for queries that select all fields. ({issue}`5660`) +* Fix failure when query parameter appears in a lambda expression. ({issue}`5640`) +* Fix failure for queries containing `DISTINCT *` and fully-qualified column names in the `ORDER BY` clause. ({issue}`5647`) +* Fix planning failure for certain queries involving `INNER JOIN`, `GROUP BY` and correlated subqueries. ({issue}`5846`) +* Fix recording of query completion event when query is aborted early. ({issue}`5815`) +* Fix exported JMX name for `QueryManager`. ({issue}`5702`) +* Fix failure when {func}`approx_distinct` is used with high precision `timestamp(p)`/`timestamp(p) with time zone`/`time(p) with time zone` + data types. ({issue}`5392`) + +## Web UI + +* Fix "Capture Snapshot" button on the Worker page. ({issue}`5759`) + +## JDBC driver + +* Support number accessor methods like `ResultSet.getLong()` or `ResultSet.getDouble()` + on `decimal` values, as well as `char` or `varchar` values that can be unambiguously interpreted as numbers. ({issue}`5509`) +* Add `SSLVerification` JDBC connection parameter that allows configuring SSL verification. ({issue}`5610`) +* Remove legacy `useSessionTimeZone` JDBC connection parameter. ({issue}`4521`) +* Implement `ResultSet.getRow()`. ({issue}`5769`) + +## Server RPM + +* Remove leftover empty directories after RPM uninstall. ({issue}`5782`) + +## BigQuery connector + +* Fix issue when query could return invalid results if some column references were pruned out during query optimization. ({issue}`5618`) + +## Cassandra connector + +* Improve performance of `INSERT` queries with batch statement. The batch size can be configured via the `cassandra.batch-size` + configuration property. ({issue}`5047`) + +## Elasticsearch connector + +* Fix failure when index mappings do not contain a `properties` section. ({issue}`5807`) + +## Hive connector + +* Add support for `ALTER TABLE ... SET AUTHORIZATION` SQL syntax to change the table owner. ({issue}`5717`) +* Add support for writing timestamps with microsecond or nanosecond precision, in addition to milliseconds. ({issue}`5283`) +* Export JMX statistics for Glue metastore client request metrics. ({issue}`5693`) +* Collect column statistics during `ANALYZE` and when data is inserted to table for columns of `timestamp(p)` + when precision is greater than 3. ({issue}`5392`) +* Improve query performance by adding support for dynamic bucket pruning. ({issue}`5634`) +* Remove deprecated `parquet.fail-on-corrupted-statistics` (previously known as `hive.parquet.fail-on-corrupted-statistics`). + A new configuration property, `parquet.ignore-statistics`, can be used to deal with Parquet files with incorrect metadata. ({issue}`3077`) +* Do not write min/max statistics for `timestamp` columns. ({issue}`5858`) +* If multiple metastore URIs are defined via `hive.metastore.uri`, prefer connecting to one which was seen operational most recently. + This prevents query failures when one or more metastores are misbehaving. ({issue}`5795`) +* Fix Hive view access when catalog name is other than `hive`. ({issue}`5785`) +* Fix failure when the declared length of a `varchar(n)` column in the partition schema differs from the table schema. ({issue}`5484`) +* Fix Glue metastore pushdown for complex expressions. ({issue}`5698`) + +## Iceberg connector + +* Add support for materialized views. ({issue}`4832`) +* Remove deprecated `parquet.fail-on-corrupted-statistics` (previously known as `hive.parquet.fail-on-corrupted-statistics`). + A new configuration property, `parquet.ignore-statistics`, can be used to deal with Parquet files with incorrect metadata. ({issue}`3077`) + +## Kafka connector + +* Fix incorrect column comment. ({issue}`5751`) + +## Kudu connector + +* Improve performance of queries having only `LIMIT` clause. ({issue}`3691`) + +## MySQL connector + +* Improve performance for queries containing a predicate on a `varbinary` column. ({issue}`5672`) + +## Oracle connector + +* Add support for setting column comments. ({issue}`5399`) +* Allow enabling remarks reporting via `oracle.remarks-reporting.enabled` configuration property. ({issue}`5720`) + +## PostgreSQL connector + +* Improve performance of queries comparing a `timestamp` column with a `timestamp with time zone` constants + for `timestamp with time zone` precision higher than 3. ({issue}`5543`) + +## Other connectors + +* Improve performance of queries with `DISTINCT` or `LIMIT`, or with `GROUP BY` and no aggregate functions and `LIMIT`, + when the computation can be pushed down to the underlying database for the PostgreSQL, MySQL, Oracle, Redshift and + SQL Server connectors. ({issue}`5522`) + +## SPI + +* Fix propagation of connector session properties to `ConnectorNodePartitioningProvider`. ({issue}`5690`) +* Add user groups to query events. ({issue}`5643`) +* Add planning time to query completed event. ({issue}`5643`) diff --git a/430/_sources/release/release-347.md.txt b/430/_sources/release/release-347.md.txt new file mode 100644 index 000000000..2657c6fa1 --- /dev/null +++ b/430/_sources/release/release-347.md.txt @@ -0,0 +1,66 @@ +# Release 347 (25 Nov 2020) + +## General + +* Add `ALTER VIEW ... SET AUTHORIZATION` syntax for changing owner of the view. ({issue}`5789`) +* Add support for `INTERSECT ALL` and `EXCEPT ALL`. ({issue}`2152`) +* Add {func}`contains_sequence` function. ({issue}`5593`) +* Support defining cluster topology (used for query scheduling) using network subnets. ({issue}`4862`) +* Improve query performance by reducing worker to worker communication overhead. ({issue}`5905`, {issue}`5949`) +* Allow disabling client HTTP response compression, which can improve throughput over fast network links. + Compression can be disabled globally via the `query-results.compression-enabled` config property, for CLI via + the `--disable-compression` flag, and for the JDBC driver via the `disableCompression` driver property. ({issue}`5818`) +* Rename ``rewrite-filtering-semi-join-to-inner-join`` session property to ``rewrite_filtering_semi_join_to_inner_join``. ({issue}`5954`) +* Throw a user error when session property value cannot be decoded. ({issue}`5731`) +* Fix query failure when expressions that produce values of type `row` are used in a `VALUES` clause. ({issue}`3398`) + +## Server + +* A minimum Java version of 11.0.7 is now required for Presto to start. This is to mitigate JDK-8206955. ({issue}`5957`) + +## Security + +* Add support for multiple LDAP bind patterns. ({issue}`5874`) +* Include groups for view owner when checking permissions for views. ({issue}`5945`) + +## JDBC driver + +* Implement `addBatch()`, `clearBatch()` and `executeBatch()` methods in `PreparedStatement`. ({issue}`5507`) + +## CLI + +* Add support for providing queries to presto-cli via shell redirection. ({issue}`5881`) + +## Docker image + +* Update Presto docker image to use CentOS 8 as the base image. ({issue}`5920`) + +## Hive connector + +* Add support for `ALTER VIEW ... SET AUTHORIZATION` SQL syntax to change the view owner. This supports Presto and Hive views. ({issue}`5789`) +* Allow configuring HDFS replication factor via the `hive.dfs.replication` config property. ({issue}`1829`) +* Add access checks for tables in Hive Procedures. ({issue}`1489`) +* Decrease latency of `INSERT` and `CREATE TABLE AS ...` queries by updating table and column statistics in parallel. ({issue}`3638`) +* Fix leaking S3 connections when querying Avro tables. ({issue}`5562`) + +## Kudu connector + +* Add dynamic filtering support. It can be enabled by setting a non-zero duration value for ``kudu.dynamic-filtering.wait-timeout`` config property + or ``dynamic_filtering_wait_timeout`` session property. ({issue}`5594`) + +## MongoDB connector + +* Improve performance of queries containing a `LIMIT` clause. ({issue}`5870`) + +## Other connectors + +* Improve query performance by compacting large pushed down predicates for the PostgreSQL, MySQL, Oracle, + Redshift and SQL Server connectors. Compaction threshold can be changed using the ``domain-compaction-threshold`` + config property or ``domain_compaction_threshold`` session property. ({issue}`6057`) +* Improve performance for the PostgreSQL, MySQL, SQL Server connectors for certain complex queries involving + aggregation and predicates by pushing the aggregation and predicates computation into the remote database. ({issue}`4112`) + +## SPI + +* Add support for connectors to redirect table scan operations to another connector. ({issue}`5792`) +* Add physical input bytes and rows for table scan operation to query completion event. ({issue}`5872`) diff --git a/430/_sources/release/release-348.md.txt b/430/_sources/release/release-348.md.txt new file mode 100644 index 000000000..c0eecfdea --- /dev/null +++ b/430/_sources/release/release-348.md.txt @@ -0,0 +1,86 @@ +# Release 348 (14 Dec 2020) + +## General + +* Add support for `DISTINCT` clause in aggregations within correlated subqueries. ({issue}`5904`) +* Support `SHOW STATS` for arbitrary queries. ({issue}`3109`) +* Improve query performance by reducing worker to worker communication overhead. ({issue}`6126`) +* Improve performance of `ORDER BY ... LIMIT` queries. ({issue}`6072`) +* Reduce memory pressure and improve performance of queries involving joins. ({issue}`6176`) +* Fix ``EXPLAIN ANALYZE`` for certain queries that contain broadcast join. ({issue}`6115`) +* Fix planning failures for queries that contain outer joins and aggregations using `FILTER (WHERE )` syntax. ({issue}`6141`) +* Fix incorrect results when correlated subquery in join contains aggregation functions such as `array_agg` or `checksum`. ({issue}`6145`) +* Fix incorrect query results when using `timestamp with time zone` constants with precision higher than 3 + describing same point in time but in different zones. ({issue}`6318`) +* Fix duplicate query completion events if query fails early. ({issue}`6103`) +* Fix query failure when views are accessed and current session does not + specify default schema and catalog. ({issue}`6294`) + +## Web UI + +* Add support for OAuth2 authorization. ({issue}`5355`) +* Fix invalid operator stats in Stage Performance view. ({issue}`6114`) + +## JDBC driver + +* Allow reading `timestamp with time zone` value as `ZonedDateTime` using `ResultSet.getObject(int column, Class type)` method. ({issue}`307`) +* Accept `java.time.LocalDate` in `PreparedStatement.setObject(int, Object)`. ({issue}`6301`) +* Extend `PreparedStatement.setObject(int, Object, int)` to allow setting `time` and `timestamp` values with precision higher than nanoseconds. ({issue}`6300`) + This can be done via providing a `String` value representing a valid SQL literal. +* Change representation of a `row` value. `ResultSet.getObject` now returns an instance of `io.prestosql.jdbc.Row` class, which better represents + the returned value. Previously a `row` value was represented as a `Map` instance, with unnamed fields being named like `field0`, `field1`, etc. + You can access the previous behavior by invoking `getObject(column, Map.class)` on the `ResultSet` object. ({issue}`4588`) +* Represent `varbinary` value using hex string representation in `ResultSet.getString`. Previously the return value was useless, similar to `"B@2de82bf8"`. ({issue}`6247`) +* Report precision of the `time(p)`, `time(p) with time zone`, `timestamp(p)` and `timestamp(p) with time zone` in the `DECIMAL_DIGITS` column + in the result set returned from `DatabaseMetaData#getColumns`. ({issue}`6307`) +* Fix the value of the `DATA_TYPE` column for `time(p)` and `time(p) with time zone` in the result set returned from `DatabaseMetaData#getColumns`. ({issue}`6307`) +* Fix failure when reading a `timestamp` or `timestamp with time zone` value with seconds fraction greater than or equal to 999999999500 picoseconds. ({issue}`6147`) +* Fix failure when reading a `time` value with seconds fraction greater than or equal to 999999999500 picoseconds. ({issue}`6204`) +* Fix element representation in arrays returned from `ResultSet.getArray`, making it consistent with `ResultSet.getObject`. + Previously the elements were represented using internal client representation (e.g. `String`). ({issue}`6048`) +* Fix `ResultSetMetaData.getColumnType` for `timestamp with time zone`. Previously the type was miscategorized as `java.sql.Types.TIMESTAMP`. ({issue}`6251`) +* Fix `ResultSetMetaData.getColumnType` for `time with time zone`. Previously the type was miscategorized as `java.sql.Types.TIME`. ({issue}`6251`) +* Fix failure when an instance of `SphericalGeography` geospatial type is returned in the `ResultSet`. ({issue}`6240`) + +## CLI + +* Fix rendering of `row` values with unnamed fields. Previously they were printed using fake field names like `field0`, `field1`, etc. ({issue}`4587`) +* Fix query progress reporting. ({issue}`6119`) +* Fix failure when an instance of `SphericalGeography` geospatial type is returned to the client. ({issue}`6238`) + +## Hive connector + +* Allow configuring S3 endpoint in security mapping. ({issue}`3869`) +* Add support for S3 streaming uploads. Data is uploaded to S3 as it is written, rather + than staged to a local temporary file. This feature is disabled by default, and can be enabled + using the `hive.s3.streaming.enabled` configuration property. ({issue}`3712`, {issue}`6201`) +* Reduce load on metastore when background cache refresh is enabled. ({issue}`6101`, {issue}`6156`) +* Verify that data is in the correct bucket file when reading bucketed tables. + This is enabled by default, as incorrect bucketing can cause incorrect query results, + but can be disabled using the `hive.validate-bucketing` configuration property + or the `validate_bucketing` session property. ({issue}`6012`) +* Allow fallback to legacy Hive view translation logic via `hive.legacy-hive-view-translation` config property or + `legacy_hive_view_translation` session property. ({issue}`6195 `) +* Add deserializer class name to split information exposed to the event listener. ({issue}`6006`) +* Improve performance when querying tables that contain symlinks. ({issue}`6158`, {issue}`6213`) + +## Iceberg connector + +* Improve performance of queries containing filters on non-partition columns. Such filters are now used + for optimizing split generation and table scan. ({issue}`4932`) +* Add support for Google Cloud Storage and Azure Storage. ({issue}`6186`) + +## Kafka connector + +* Allow writing `timestamp with time zone` values into columns using `milliseconds-since-epoch` or + `seconds-since-epoch` JSON encoders. ({issue}`6074`) + +## Other connectors + +* Fix ineffective table metadata caching for PostgreSQL, MySQL, SQL Server, Redshift, MemSQL and Phoenix connectors. ({issue}`6081`, {issue}`6167`) + +## SPI + +* Change `SystemAccessControl#filterColumns` and `ConnectorAccessControl#filterColumns` methods to accept a set of + column names, and return a set of visible column names. ({issue}`6084`) +* Expose catalog names corresponding to the splits through the split completion event of the event listener. ({issue}`6006`) diff --git a/430/_sources/release/release-349.md.txt b/430/_sources/release/release-349.md.txt new file mode 100644 index 000000000..aaf5e0dde --- /dev/null +++ b/430/_sources/release/release-349.md.txt @@ -0,0 +1,5 @@ +# Release 349 (28 Dec 2020) + +```{warning} +This release is broken and should not be used. +``` diff --git a/430/_sources/release/release-350.md.txt b/430/_sources/release/release-350.md.txt new file mode 100644 index 000000000..69267c768 --- /dev/null +++ b/430/_sources/release/release-350.md.txt @@ -0,0 +1,53 @@ +# Release 350 (28 Dec 2020) + +## General + +* Add HTTP client JMX metrics. ({issue}`6453`) +* Improve query performance by reducing worker to worker communication overhead. ({issue}`6283`, {issue}`6349`) +* Improve performance of queries that contain `IS NOT DISTINCT FROM` join predicates. ({issue}`6404`) +* Fix failure when restricted columns have column masks. ({issue}`6017`) +* Fix failure when `try` expressions reference columns that contain `@` or `:` in their names. ({issue}`6380`) +* Fix memory management config handling to use `query.max-total-memory-per-node` + rather than only using `query.max-memory-per-node` for both values. ({issue}`6349`) + +## Web UI + +* Fix truncation of query text in cluster overview page. ({issue}`6216`) + +## JDBC driver + +* Accept `java.time.OffsetTime` in `PreparedStatement.setObject(int, Object)`. ({issue}`6352`) +* Extend `PreparedStatement.setObject(int, Object, int)` to allow setting `time with time zone` and `timestamp with time zone` + values with precision higher than nanoseconds. This can be done via providing a `String` value representing a valid SQL literal. ({issue}`6352`) + +## BigQuery connector + +* Fix incorrect results for `count(*)` queries with views. ({issue}`5635`) + +## Cassandra connector + +* Support `DELETE` statement with primary key or partition key. ({issue}`4059`) + +## Elasticsearch connector + +* Improve query analysis performance when Elasticsearch contains many index mappings. ({issue}`6368`) + +## Kafka connector + +* Support Kafka Schema Registry for Avro topics. ({issue}`6137`) + +## SQL Server connector + +* Add `data_compression` table property to control the target compression in SQL Server. + The allowed values are `NONE`, `ROW` or `PAGE`. ({issue}`4693`) + +## Other connectors + +This change applies to the MySQL, Oracle, PostgreSQL, Redshift, and SQL Server connectors. + +* Send shorter and potentially more performant queries to remote database when a Presto query has a `NOT IN` + predicate eligible for pushdown into the connector. ({issue}`6075`) + +## SPI + +* Rename `LongTimeWithTimeZone.getPicoSeconds()` to `LongTimeWithTimeZone.getPicoseconds()`. ({issue}`6354`) diff --git a/430/_sources/release/release-351.md.txt b/430/_sources/release/release-351.md.txt new file mode 100644 index 000000000..260f5c58b --- /dev/null +++ b/430/_sources/release/release-351.md.txt @@ -0,0 +1,69 @@ +# Release 351 (3 Jan 2021) + +## General + +* Rename client protocol headers to start with `X-Trino-`. + Legacy clients can be supported by setting the configuration property + `protocol.v1.alternate-header-name` to `Presto`. This configuration + property is deprecated and will be removed in a future release. + +## JMX MBean naming + +* Rename base domain name for server MBeans to `trino`. The name can + be changed using the configuration property `jmx.base-name`. +* Rename base domain name for the Elasticsearch, Hive, Iceberg, Raptor, + and Thrift connectors to `trino.plugin`. The name can be changed + using the catalog configuration property `jmx.base-name`. + +## Server RPM + +* Rename installation directories from `presto` to `trino`. + +## Docker image + +* Publish image as [`trinodb/trino`](https://hub.docker.com/r/trinodb/trino). +* Change base image to `azul/zulu-openjdk-centos`. +* Change configuration directory to `/etc/trino`. +* Rename CLI in image to `trino`. + +## CLI + +* Use new client protocol header names. The CLI is not compatible with older servers. + +## JDBC driver + +* Use new client protocol header names. The driver is not compatible with older servers. +* Change driver URL prefix to `jdbc:trino:`. + The old prefix is deprecated and will be removed in a future release. +* Change driver class to `io.trino.jdbc.TrinoDriver`. + The old class name is deprecated and will be removed in a future release. +* Rename Java package for all driver classes to `io.trino.jdbc` and rename + various driver classes such as `TrinoConnection` to start with `Trino`. + +## Hive connector + +* Rename JMX name for `PrestoS3FileSystem` to `TrinoS3FileSystem`. +* Change configuration properties + `hive.hdfs.presto.principal` to `hive.hdfs.trino.principal` and + `hive.hdfs.presto.keytab` to `hive.hdfs.trino.keytab`. + The old names are deprecated and will be removed in a future release. + +## Local file connector + +* Change configuration properties + `presto-logs.http-request-log.location` to `trino-logs.http-request-log.location` and + `presto-logs.http-request-log.pattern` to `trino-logs.http-request-log.pattern`. + The old names are deprecated and will be removed in a future release. + +## Thrift connector + +* Rename Thrift service method names starting with `presto` to `trino`. +* Rename all classes in the Thrift IDL starting with `Presto` to `Trino`. +* Rename configuration properties starting with `presto` to `trino`. + +## SPI + +* Rename Java package to `io.trino.spi`. +* Rename `PrestoException` to `TrinoException`. +* Rename `PrestoPrincipal` to `TrinoPrincipal`. +* Rename `PrestoWarning` to `TrinoWarning`. diff --git a/430/_sources/release/release-352.md.txt b/430/_sources/release/release-352.md.txt new file mode 100644 index 000000000..3c1f7b8a8 --- /dev/null +++ b/430/_sources/release/release-352.md.txt @@ -0,0 +1,121 @@ +# Release 352 (9 Feb 2021) + +## General + +* Add support for [`WINDOW` clause](window-clause). ({issue}`651`) +* Add support for {doc}`/sql/update`. ({issue}`5861`) +* Add {func}`version` function. ({issue}`4627`) +* Allow prepared statement parameters for `SHOW STATS`. ({issue}`6582`) +* Update tzdata version to 2020d. As a result, queries can no longer reference the + `US/Pacific-New` zone, as it has been removed. ({issue}`6660`) +* Add `plan-with-table-node-partitioning` feature config that corresponds to + existing `plan_with_table_node_partitioning` session property. ({issue}`6811`) +* Improve performance of queries using {func}`rank()` window function. ({issue}`6333`) +* Improve performance of {func}`sum` and {func}`avg` for `decimal` types. ({issue}`6951`) +* Improve join performance. ({issue}`5981`) +* Improve query planning time for queries using range predicates or large `IN` lists. ({issue}`6544`) +* Fix window and streaming aggregation semantics regarding peer rows. Now peer rows are + grouped using `IS NOT DISTINCT FROM` instead of the `=` operator. ({issue}`6472`) +* Fix query failure when using an element of `array(timestamp(p))` in a complex expression + for `p` greater than 6. ({issue}`6350`) +* Fix failure when using geospatial functions in a join clause and `spatial_partitioning_table_name` is set. ({issue}`6587`) +* Fix `CREATE TABLE AS` failure when source table has hidden columns. ({issue}`6835`) + +## Security + +* Allow configuring HTTP client used for OAuth2 authentication. ({issue}`6600`) +* Add token polling client API for OAuth2 authentication. ({issue}`6625`) +* Support JWK with certificate chain for OAuth2 authorization. ({issue}`6428`) +* Add scopes to OAuth2 configuration. ({issue}`6580`) +* Optionally verify JWT audience (`aud`) field for OAuth2 authentication. ({issue}`6501`) +* Guard against replay attacks in OAuth2 by using `nonce` cookie when `openid` scope is requested. ({issue}`6580`) + +## JDBC driver + +* Add OAuth2 authentication. ({issue}`6576`) +* Support user impersonation when using password-based authentication + using the new `sessionUser` parameter. ({issue}`6549`) + +## Docker image + +* Remove support for configuration directory `/usr/lib/trino/etc`. The configuration + should be provided in `/etc/trino`. ({issue}`6497`) + +## CLI + +* Support user impersonation when using password-based authentication using the + `--session-user` command line option. ({issue}`6567`) + +## BigQuery connector + +* Add a `view_definition` system table which exposes BigQuery view definitions. ({issue}`3687`) +* Fix query failure when calculating `count(*)` aggregation on a view more than once, + without any filter. ({issue}`6706`). + +## Hive connector + +* Add `UPDATE` support for ACID tables. ({issue}`5861`) +* Match columns by index rather than by name by default for ORC ACID tables. ({issue}`6479`) +* Match columns by name rather than by index by default for Parquet files. + This can be changed using `hive.parquet.use-column-names` configuration property and `parquet_use_column_names` + session property. ({issue}`6479`) +* Remove the `hive.partition-use-column-names` configuration property and the + `partition_use_column_names ` session property. This is now determined automatically. ({issue}`6479`) +* Support timestamps with microsecond or nanosecond precision (as configured with + `hive.timestamp-precision` property) nested within `array`, `map` or `struct` data types. ({issue}`5195`) +* Support reading from table in Sequencefile format that uses LZO compression. ({issue}`6452`) +* Expose AWS HTTP Client stats via JMX. ({issue}`6503`) +* Allow specifying S3 KMS Key ID used for client side encryption via security mapping + config and extra credentials. ({issue}`6802`) +* Fix writing incorrect `timestamp` values within `row`, `array` or `map` when using Parquet file format. ({issue}`6760`) +* Fix possible S3 connection leak on query failure. ({issue}`6849`) + +## Iceberg connector + +* Add `iceberg.max-partitions-per-writer` config property to allow configuring the limit on partitions per writer. ({issue}`6650`) +* Optimize cardinality-insensitive aggregations ({func}`max`, {func}`min`, {func}`distinct`, {func}`approx_distinct`) + over identity partition columns with `optimizer.optimize-metadata-queries` config property + or `optimize_metadata_queries` session property. ({issue}`5199`) +* Provide `use_file_size_from_metadata` catalog session property and `iceberg.use-file-size-from-metadata` + config property to fix query failures on tables with wrong file sizes stored in the metadata. ({issue}`6369`) +* Fix the mapping of nested fields between table metadata and ORC file metadata. This + enables evolution of `row` typed columns for Iceberg tables stored in ORC. ({issue}`6520`) + +## Kinesis connector + +* Support GZIP message compression. ({issue}`6442`) + +## MySQL connector + +* Improve performance for certain complex queries involving aggregation and predicates (e.g. `HAVING` clause) + by pushing the aggregation and predicates computation into the remote database. ({issue}`6667`) +* Improve performance for certain queries using `stddev_pop`, `stddev_samp`, `var_pop`, `var_samp` aggregation + functions by pushing the aggregation and predicates computation into the remote database. ({issue}`6673`) + +## PostgreSQL connector + +* Improve performance for certain complex queries involving aggregation and predicates (e.g. `HAVING` clause) + by pushing the aggregation and predicates computation into the remote database. ({issue}`6667`) +* Improve performance for certain queries using `stddev_pop`, `stddev_samp`, `var_pop`, `var_samp`, + `covar_pop`, `covar_samp`, `corr`, `regr_intercept`, `regr_slope` aggregation functions + by pushing the aggregation and predicates computation into the remote database. ({issue}`6731`) + +## Redshift connector + +* Use the Redshift JDBC driver to access Redshift. As a result, `connection-url` in catalog + configuration files needs to be updated from `jdbc:postgresql:...` to `jdbc:redshift:...`. ({issue}`6465`) + +## SQL Server connector + +* Avoid query failures due to transaction deadlocks in SQL Server by using transaction snapshot isolation. ({issue}`6274`) +* Honor precision of SQL Server's `datetime2` type . ({issue}`6654`) +* Add support for Trino `timestamp` type in `CREATE TABLE` statement, by mapping it to SQL Server's `datetime2` type. + Previously, it was incorrectly mapped to SQL Server's `timestamp` type. ({issue}`6654`) +* Add support for the `time` type. ({issue}`6654`) +* Improve performance for certain complex queries involving aggregation and predicates (e.g. `HAVING` clause) + by pushing the aggregation and predicates computation into the remote database. ({issue}`6667`) +* Fix failure when querying tables having indexes and constraints. ({issue}`6464`) + +## SPI + +* Add support for join pushdown via the `ConnectorMetadata.applyJoin()` method. ({issue}`6752`) diff --git a/430/_sources/release/release-353.md.txt b/430/_sources/release/release-353.md.txt new file mode 100644 index 000000000..aa9ebbe8e --- /dev/null +++ b/430/_sources/release/release-353.md.txt @@ -0,0 +1,119 @@ +# Release 353 (5 Mar 2021) + +## General + +* Add {doc}`/connector/clickhouse`. ({issue}`4500`) +* Extend support for correlated subqueries including `UNNEST`. ({issue}`6326`, {issue}`6925`, {issue}`6951`) +* Add {func}`to_geojson_geometry` and {func}`from_geojson_geometry` functions. ({issue}`6355`) +* Add support for values of any integral type (`tinyint`, `smallint`, `integer`, `bigint`, `decimal(p, 0)`) + in window frame bound specification. ({issue}`6897`) +* Improve query planning time for queries containing `IN` predicates with many elements. ({issue}`7015`) +* Fix potential incorrect results when columns from `WITH` clause are exposed with aliases. ({issue}`6839`) +* Fix potential incorrect results for queries containing multiple `<` predicates. ({issue}`6896`) +* Always show `SECURITY` clause in `SHOW CREATE VIEW`. ({issue}`6913`) +* Fix reporting of column references for aliased tables in `QueryCompletionEvent`. ({issue}`6972`) +* Fix potential compiler failure when constructing an array with more than 128 elements. ({issue}`7014`) +* Fail `SHOW COLUMNS` when column metadata cannot be retrieved. ({issue}`6958`) +* Fix rendering of function references in `EXPLAIN` output. ({issue}`6703`) +* Fix planning failure when `WITH` clause contains hidden columns. ({issue}`6838`) +* Prevent client hangs when OAuth2 authentication fails. ({issue}`6659`) + +## Server RPM + +* Allow configuring process environment variables through `/etc/trino/env.sh`. ({issue}`6635`) + +## BigQuery connector + +* Add support for `CREATE TABLE` and `DROP TABLE` statements. ({issue}`3767`) +* Allow for case-insensitive identifiers matching via `bigquery.case-insensitive-name-matching` config property. ({issue}`6748`) + +## Hive connector + +* Add support for `current_user()` in Hive defined views. ({issue}`6720`) +* Add support for reading and writing column statistics from Glue metastore. ({issue}`6178`) +* Improve parallelism of bucketed tables inserts. Inserts into bucketed tables can now be parallelized + within task using `task.writer-count` feature config. ({issue}`6924`, {issue}`6866`) +* Fix a failure when `INSERT` writes to a partition created by an earlier `INSERT` statement. ({issue}`6853`) +* Fix handling of folders created using the AWS S3 Console. ({issue}`6992`) +* Fix query failures on ``information_schema.views`` table when there are failures + translating hive view definitions. ({issue}`6370`) + +## Iceberg connector + +* Fix handling of folders created using the AWS S3 Console. ({issue}`6992`) +* Fix query failure when reading nested columns with field names that may + contain upper case characters. ({issue}`7180`) + +## Kafka connector + +* Fix failure when querying Schema Registry tables. ({issue}`6902`) +* Fix querying of Schema Registry tables with References in their schema. ({issue}`6907`) +* Fix listing of schema registry tables having ambiguous subject name in lower case. ({issue}`7048`) + +## MySQL connector + +* Fix failure when reading a `timestamp` or `datetime` value with more than 3 decimal digits + in the fractional seconds part. ({issue}`6852`) +* Fix incorrect predicate pushdown for `char` and `varchar` column with operators + like `<>`, `<`, `<=`, `>` and `>=` due different case sensitivity between Trino + and MySQL. ({issue}`6746`, {issue}`6671`) + +## MemSQL connector + +* Fix failure when reading a `timestamp` or `datetime` value with more than 3 decimal digits + of the second fraction. ({issue}`6852`) +* Fix incorrect predicate pushdown for `char` and `varchar` column with operators + like `<>`, `<`, `<=`, `>` and `>=` due different case sensitivity between Trino + and MemSQL. ({issue}`6746`, {issue}`6671`) + +## Phoenix connector + +* Add support for Phoenix 5.1. This can be used by setting `connector.name=phoenix5` in catalog + configuration properties. ({issue}`6865`) +* Fix failure when query contains a `LIMIT` exceeding 2147483647. ({issue}`7169`) + +## PostgreSQL connector + +* Improve performance of queries with `ORDER BY ... LIMIT` clause, when the computation + can be pushed down to the underlying database. This can be enabled by setting `topn-pushdown.enabled`. + Enabling this feature can currently result in incorrect query results when sorting + on `char` or `varchar` columns. ({issue}`6847`) +* Fix incorrect predicate pushdown for `char` and `varchar` column with operators + like `<>`, `<`, `<=`, `>` and `>=` due different case collation between Trino + and PostgreSQL. ({issue}`3645`) + +## Redshift connector + +* Fix failure when reading a `timestamp` value with more than 3 decimal digits of + the second fraction. ({issue}`6893`) + +## SQL Server connector + +* Abort queries on the SQL Server side when the Trino query is finished. ({issue}`6637`) +* Fix incorrect predicate pushdown for `char` and `varchar` column with operators + like `<>`, `<`, `<=`, `>` and `>=` due different case sensitivity between Trino + and SQL Server. ({issue}`6753`) + +## Other connectors + +* Reduce number of opened JDBC connections during planning for ClickHouse, Druid, MemSQL, MySQL, + Oracle, Phoenix, Redshift, and SQL Server connectors. ({issue}`7069`) +* Add experimental support for join pushdown in PostgreSQL, MySQL, MemSQL, Oracle, and SQL Server connectors. + It can be enabled with the `experimental.join-pushdown.enabled=true` catalog configuration property. ({issue}`6874`) + +## SPI + +* Fix lazy blocks to call listeners that are registered after the top level block is already loaded. + Previously, such registered listeners were not called when the nested blocks were later loaded. ({issue}`6783`) +* Fix case where LazyBlock.getFullyLoadedBlock() would not load nested blocks + when the top level block was already loaded. ({issue}`6783`) +* Do not include coordinator node in the result of `ConnectorAwareNodeManager.getWorkerNodes()` + when `node-scheduler.include-coordinator` is false. ({issue}`7007`) +* The function name passed to `ConnectorMetadata.applyAggregation()` + is now the canonical function name. Previously, if query used function alias, the alias + name was passed. ({issue}`6189`) +* Add support for redirecting table scans to multiple tables that are unioned together. ({issue}`6679`) +* Change return type of `Range.intersect(Range)`. The method now + returns `Optional.empty()` instead of throwing when ranges do not overlap. ({issue}`6976`) +* Change signature of `ConnectorMetadata.applyJoin()` to have an additional `JoinStatistics` argument. ({issue}`7000`) +* Deprecate `io.trino.spi.predicate.Marker`. diff --git a/430/_sources/release/release-354.md.txt b/430/_sources/release/release-354.md.txt new file mode 100644 index 000000000..9a1d80754 --- /dev/null +++ b/430/_sources/release/release-354.md.txt @@ -0,0 +1,88 @@ +# Release 354 (19 Mar 2021) + +## General + +* Improve performance of certain queries involving `LEFT`, `RIGHT` and `FULL JOIN` + when one side of the join is known to produce a single row. ({issue}`7090`) +* Preferred write partitioning is now triggered automatically when the estimated number + of written partitions exceeds or is equal to `preferred-write-partitioning-min-number-of-partitions`. + This estimation requires that the input tables of the query have statistics. To enable + preferred partitioning unconditionally, set `preferred-write-partitioning-min-number-of-partitions` to `1`. + To disable preferred write partitioning, set `use-preferred-write-partitioning` to `false`. + ({issue}`6920`) +* Fix incorrect results when multiple similar subqueries contain `UNION` clauses that differ + only in the `ALL` vs `DISTINCT` qualifier. ({issue}`7345`) +* Fix `DELETE` and `UPDATE` for connectors that subsume filters. ({issue}`7302`) +* Fix failure when subqueries contain `EXCEPT` or `INTERSECT`. ({issue}`7342`) +* Fix failure of queries that contain `RIGHT JOIN` when late materialization is enabled. ({issue}`6784`) + +## Security + +* Fix retries for OAuth 2.0 authentication in case of token expiry. ({issue}`7172`) + +## CLI + +* Support OAuth 2.0 authentication. ({issue}`7054`) + +## ClickHouse connector + +* Use correct case for name of the schema in `CREATE SCHEMA`. ({issue}`7239`) + +## Elasticsearch connector + +* Fix failure when reading single-valued fields for array types. ({issue}`7012`) + +## Hive connector + +* Respect partition filter for `DELETE` and `UPDATE` of ACID tables. Previously, the partition + filter was ignored, resulting in the deletion or update of too many rows. ({issue}`7302`) +* Fix allocation of statement IDs for ACID tables, which could result in query failure or + data loss due to creating multiple delta files with the same name. ({issue}`7302`) +* Fix incorrect query results when reading from an incorrectly bucketed table created and registered + with the metastore by Spark. ({issue}`6848`) +* Avoid leaking file system connections or other resources when using the Avro file format. ({issue}`7178`) +* Fix query failure when columns of a CSV table are declared as a type other than `varchar` (`string`) in Glue + metastore. Columns are now interpreted as `varchar` values, instead. ({issue}`7059`) +* Rename `hive.parallel-partitioned-bucketed-inserts` configuration property to `hive.parallel-partitioned-bucketed-writes`. ({issue}`7259`) + +## Iceberg connector + +* Fix queries on empty tables without a snapshot ID that were created by Spark. ({issue}`7233`) +* Update to Iceberg 0.11.0 behavior for transforms of dates and timestamps + before 1970. Data written by older versions of Trino and Iceberg will be + read correctly. New data will be written correctly, but may be read + incorrectly by older versions of Trino and Iceberg. ({issue}`7049`) + +## MemSQL connector + +* Add support for MemSQL 3.2. ({issue}`7179`) +* Use correct case for name of the schema in `CREATE SCHEMA`. ({issue}`7239`) +* Improve performance of queries with `ORDER BY ... LIMIT` clause when the computation + can be pushed down to the underlying database. ({issue}`7326`) + +## MySQL connector + +* Use proper column type (`datetime(3)`) in MySQL when creating a table with `timestamp(3)` column. + Previously, the second fraction was being truncated. ({issue}`6909`) +* Use correct case for name of the schema in `CREATE SCHEMA`. ({issue}`7239`) +* Improve performance of queries with `ORDER BY ... LIMIT` clause when the computation + can be pushed down to the underlying database. ({issue}`7326`) + +## PostgreSQL connector + +* Fix incorrect query results for `ORDER BY ... LIMIT` clause when sorting on `char` or `varchar` columns + and `topn-pushdown.enabled` configuration property is enabled. The optimization is now enabled by default. + ({issue}`7170`, {issue}`7314`) +* Use correct case for name of the schema in `CREATE SCHEMA`. ({issue}`7239`) + +## Redshift connector + +* Fix failure when query contains a `LIMIT` exceeding 2147483647. ({issue}`7236`) +* Use correct case for name of the schema in `CREATE SCHEMA`. ({issue}`7239`) + +## SQL Server connector + +* Add support for parametric `time` type. ({issue}`7122`) +* Use correct case for name of the schema in `CREATE SCHEMA`. ({issue}`7239`) +* Improve performance of queries with `ORDER BY ... LIMIT` clause when the computation + can be pushed down to the underlying database. ({issue}`7324`) diff --git a/430/_sources/release/release-355.md.txt b/430/_sources/release/release-355.md.txt new file mode 100644 index 000000000..26d3f939b --- /dev/null +++ b/430/_sources/release/release-355.md.txt @@ -0,0 +1,52 @@ +# Release 355 (8 Apr 2021) + +## General + +* Report tables that are directly referenced by a query in `QueryCompletedEvent`. ({issue}`7330`) +* Report columns that are the target of `INSERT` or `UPDATE` queries in `QueryCompletedEvent`. This includes + information about which input columns they are derived from. ({issue}`7425`, {issue}`7465`) +* Rename `optimizer.plan-with-table-node-partitioning` config property to `optimizer.use-table-scan-node-partitioning`. ({issue}`7257`) +* Improve query parallelism when table bucket count is small compared to number of nodes. + This optimization is now triggered automatically when the ratio between table buckets and + possible table scan tasks exceeds or is equal to `optimizer.table-scan-node-partitioning-min-bucket-to-task-ratio`. ({issue}`7257`) +* Include information about {doc}`/admin/spill` in {doc}`/sql/explain-analyze`. ({issue}`7427`) +* Disallow inserting data into tables that have row filters. ({issue}`7346`) +* Improve performance of queries that can benefit from both {doc}`/optimizer/cost-based-optimizations` and join pushdown + by giving precedence to cost-based optimizations. ({issue}`7331`) +* Fix inconsistent behavior for {func}`to_unixtime` with values of type `timestamp(p)`. ({issue}`7450`) +* Change return type of {func}`from_unixtime` and {func}`from_unixtime_nanos` to `timestamp(p) with time zone`. ({issue}`7460`) + +## Security + +* Add support for configuring multiple password authentication plugins. ({issue}`7151`) + +## JDBC driver + +* Add `assumeLiteralNamesInMetadataCallsForNonConformingClients` parameter for use as a workaround when + applications do not properly escape schema or table names in calls to `DatabaseMetaData` methods. ({issue}`7438`) + +## ClickHouse connector + +* Support creating tables with MergeTree storage engine. ({issue}`7135`) + +## Hive connector + +* Support Hive views containing `LATERAL VIEW json_tuple(...) AS ...` syntax. ({issue}`7242`) +* Fix incorrect results when reading from a Hive view that uses array subscript operators. ({issue}`7271`) +* Fix incorrect results when querying the `$file_modified_time` hidden column. ({issue}`7511`) + +## Phoenix connector + +* Improve performance when fetching table metadata during query analysis. ({issue}`6975`) +* Improve performance of queries with `ORDER BY ... LIMIT` clause when the computation + can be pushed down to the underlying database. ({issue}`7490`) + +## SQL Server connector + +* Improve performance when fetching table metadata during query analysis. ({issue}`6975`) + +## SPI + +* Engine now uses `ConnectorMaterializedViewDefinition#storageTable` + to determine materialized view storage table. ({issue}`7319`) + diff --git a/430/_sources/release/release-356.md.txt b/430/_sources/release/release-356.md.txt new file mode 100644 index 000000000..b83d6654b --- /dev/null +++ b/430/_sources/release/release-356.md.txt @@ -0,0 +1,111 @@ +# Release 356 (30 Apr 2021) + +## General + +* Add support for {doc}`/sql/match-recognize`. ({issue}`6111`) +* Add {func}`soundex` function. ({issue}`4022`) +* Introduce `system.metadata.materialized_view_properties` table for listing available materialized view properties. ({issue}`7615`) +* Add support for limiting the maximum planning time via the `query.max-planning-time` configuration property. ({issue}`7213`) +* Allow redirecting clients to an alternative location to fetch query information. This can be configured + via the `query.info-url-template` configuration property. ({issue}`7678`) +* Allow cancellation of queries during planning phase. ({issue}`7213`) +* Improve performance of `ORDER BY ... LIMIT` queries over a `LEFT JOIN`. ({issue}`7028`) +* Improve performance of queries with predicates on boolean columns. ({issue}`7263`) +* Improve planning time for queries with large `IN` predicates. ({issue}`7556`) +* Improve performance of queries that contain joins on `varchar` keys of different length. ({issue}`7644`) +* Improve performance of queries when late materialization is enabled. ({issue}`7695`) +* Reduce coordinator network overhead when scheduling queries. ({issue}`7351`) +* Fix possible deadlock for `JOIN` queries when spilling is enabled. ({issue}`7455`) +* Fix incorrect results for queries containing full outer join with an input that is known to produce one row. ({issue}`7629`) +* Fix failure when quantified comparison expressions contain scalar subqueries. ({issue}`7792`) + +## Security + +* Materialized views require `UPDATE` privilege to be refreshed. ({issue}`7707`) +* Add dedicated access control for creating and dropping materialized views. ({issue}`7645`) +* Add dedicated access control for refreshing materialized views. Insert privilege on storage table is no + longer required. ({issue}`7707`) +* Fix authentication failure when providing multiple scope values for `http-server.authentication.oauth2.scopes`. ({issue}`7706`) + +## JDBC driver + +* Add support for caching OAuth2 credentials in memory to avoid unnecessary authentication flows. ({issue}`7309`) + +## BigQuery connector + +* Add support for `CREATE SCHEMA` and `DROP SCHEMA` statements. ({issue}`7543`) +* Improve table listing performance when case insensitive matching is enabled. ({issue}`7628`) + +## Cassandra connector + +* Fix `NullPointerException` when reading an empty timestamp value. ({issue}`7433`) + +## Hive connector + +* Improve performance when reading dictionary-encoded Parquet files. ({issue}`7754`) +* Fix incorrect results when referencing nested fields with non-lowercase names from ORC files. ({issue}`7350`) +* Always use row-by-row deletes for ACID tables rather than dropping partitions. ({issue}`7621`) +* Allow reading from ORC ACID transactional tables when `_orc_acid_version` metadata files are missing. ({issue}`7579`) +* Add `UPDATE` support for ACID tables that were originally created as non-transactional. ({issue}`7622`) +* Add support for connection proxying for Azure ADLS endpoints. ({issue}`7509`) + +## Iceberg connector + +* Show Iceberg tables created by other engines in `SHOW TABLES` output. ({issue}`1592`) +* Improve performance when reading dictionary-encoded Parquet files. ({issue}`7754`) +* Improve query planning through table metadata caching. ({issue}`7336`) +* Fix failure querying materialized views that were created using the session catalog and schema. ({issue}`7711`) +* Fix listing of materialized views when using `SHOW TABLES` query. ({issue}`7790`) + +## Kafka connector + +* Add support for TLS security protocol. ({issue}`6929`) + +## MemSQL connector + +* Improve metadata caching hit rate. ({issue}`7039`) + +## MongoDB connector + +* Fix handling of non-lowercase MongoDB views. ({issue}`7491`) + +## MySQL connector + +* Improve metadata caching hit rate. ({issue}`7039`) +* Exclude an internal `sys` schema from schema listings. ({issue}`6337`) + +## Oracle connector + +* Improve metadata caching hit rate. ({issue}`7039`) + +## Phoenix connector + +* Improve metadata caching hit rate. ({issue}`7039`) + +## PostgreSQL connector + +* Improve metadata caching hit rate. ({issue}`7039`) +* Cancel query on PostgreSQL when the Trino query is cancelled. ({issue}`7306`) +* Discontinue support for PostgreSQL 9.5, which has reached end of life. ({issue}`7676`) + +## Redshift connector + +* Improve metadata caching hit rate. ({issue}`7039`) + +## SQL Server connector + +* Improve metadata caching hit rate. ({issue}`7039`) +* Fix query failure when snapshot isolation is disabled in target SQL Server database, + but `READ_COMMITTED_SNAPSHOT` is still enabled. ({issue}`7548`) +* Fix reading `date` values before 1583-10-14. ({issue}`7634`) + +## SPI + +* Require that `ConnectorMaterializedViewDefinition` provides a view owner. ({issue}`7489`) +* Add `Connector#getMaterializedViewPropertyManager` for specifying materialized view properties. ({issue}`7615`) +* Add `ConnectorAccessControl.checkCanCreateMaterializedView()` and + `ConnectorAccessControl.checkCanDropMaterializedView()` for authorizing + creation and removal of materialized views. ({issue}`7645`) +* Allow a materialized view to return a storage table in a different catalog or schema. ({issue}`7638`) +* Add `ConnectorAccessControl.checkCanRefreshMaterializedView()` for authorizing + refresh of materialized views. ({issue}`7707`) diff --git a/430/_sources/release/release-357.md.txt b/430/_sources/release/release-357.md.txt new file mode 100644 index 000000000..3ed9da896 --- /dev/null +++ b/430/_sources/release/release-357.md.txt @@ -0,0 +1,86 @@ +# Release 357 (21 May 2021) + +## General + +* Add support for subquery expressions that return multiple columns. + Example: `SELECT x = (VALUES (1, 'a'))` ({issue}`7773`, {issue}`7863`) +* Allow aggregation pushdown when `COUNT(1)` is used with `GROUP BY`. ({issue}`7251`) +* Add support for `CURRENT_CATALOG` and `CURRENT_SCHEMA`. ({issue}`7824`) +* Add {func}`format_number` function. ({issue}`1878`) +* Change `row` to `json` cast to produce JSON objects instead of JSON arrays. This behavior can be restored + with the `deprecated.legacy-row-to-json-cast` configuration option. ({issue}`3536`) +* Print dynamic filters summary in `EXPLAIN ANALYZE`. ({issue}`7874`) +* Improve performance for queries using `IN` predicate with a short list of constants. ({issue}`7840`) +* Release memory immediately when queries involving window functions fail. ({issue}`7947`) +* Fix incorrect handling of row expressions for `IN` predicates, quantified comparisons and scalar subqueries. Previously, + the queries would succeed where they should have failed with a type mismatch error. ({issue}`7797`) +* Fix failure when using `PREPARE` with a `GRANT` statement that contains quoted SQL keywords. ({issue}`7941`) +* Fix cluster instability after executing certain large `EXPLAIN` queries. ({issue}`8017`) + +## Security + +* Enforce materialized view creator security policies when view is fresh. ({issue}`7618`) +* Use system truststore for OAuth2 and JWK for JWT authentication. Previously, the truststore + configured for internal communication was used. This means that globally trusted certificates + will work by default. ({issue}`7936`) +* Fix handling of SNI for multiple TLS certificates. ({issue}`8007`) + +## Web UI + +* Make the UI aware of principal-field (configured with `http-server.authentication.oauth2.principal-field`) when + `web-ui.authentication.type` is set to `oauth2`. ({issue}`7526`) + +## JDBC driver + +* Cancel Trino query execution when JDBC statement is closed. ({issue}` 7819`) +* Close statement when connection is closed. ({issue}` 7819`) + +## CLI + +* Add `clear` command to clear the screen. ({issue}`7632`) + +## BigQuery connector + +* Fix failures for queries accessing `information_schema.columns` when `case-insensitive-name-matching` is disabled. ({issue}`7830`) +* Fix query failure when a predicate on a BigQuery `string` column contains a value with a single quote (`'`). ({issue}`7784`) + +## ClickHouse connector + +* Improve performance of aggregation queries by computing aggregations within ClickHouse. Currently, the following aggregate functions + are eligible for pushdown: `count`, `min`, `max`, `sum` and `avg`. ({issue}`7434`) +* Map ClickHouse `UUID` columns as `UUID` type in Trino instead of `VARCHAR`. ({issue}`7097`) + +## Elasticsearch connector + +* Support decoding `timestamp` columns encoded as strings containing milliseconds since epoch values. ({issue}`7838`) +* Retry requests with backoff when Elasticsearch is overloaded. ({issue}`7423`) + +## Kinesis connector + +* Add `kinesis.table-description-refresh-interval` configuration property to set the + refresh interval for fetching table descriptions from S3. ({issue}`1609`) + +## Kudu connector + +* Fix query failures for grouped execution on range partitioned tables. ({issue}`7738`) + +## MongoDB connector + +* Redact the value of `mongodb.credentials` in the server log. ({issue}`7862`) +* Add support for dropping columns. ({issue}`7853`) + +## Pinot connector + +* Add support for complex filter expressions in passthrough queries. ({issue}`7161`) + +## Other connectors + +This change applies to the Druid, MemSQL, MySQL, Oracle, Phoenix, PosgreSQL, Redshift, and SQL Server connectors. +* Add rule support for identifier mapping. The rules can be configured via the +`case-insensitive-name-matching.config-file` configuration property. ({issue}`7841`) + +## SPI + +* Make `ConnectorMaterializedViewDefinition` non-serializable. It is the responsibility of the connector to serialize + and store the materialized view definitions in an appropriate format. ({issue}`7762`) +* Deprecate `TupleDomain.transform`. ({issue}`7980`) diff --git a/430/_sources/release/release-358.md.txt b/430/_sources/release/release-358.md.txt new file mode 100644 index 000000000..fa5d7ef88 --- /dev/null +++ b/430/_sources/release/release-358.md.txt @@ -0,0 +1,43 @@ +# Release 358 (1 Jun 2021) + +## General + +* Support arbitrary queries in {doc}`/sql/show-stats`. ({issue}`8026`) +* Improve performance of complex queries involving joins and `TABLESAMPLE`. ({issue}`8094`) +* Improve performance of `ORDER BY ... LIMIT` queries on sorted data. ({issue}`6634`) +* Reduce graceful shutdown time for worker nodes. ({issue}`8149`) +* Fix query failure columns of non-orderable types (e.g. `HyperLogLog`, `tdigest`, etc.), are involved in a join. ({issue}`7723`) +* Fix failure for queries containing repeated ordinals in a `GROUP BY` clause. + Example: `SELECT x FROM t GROUP BY 1, 1`. ({issue}`8023`) +* Fix failure for queries containing repeated expressions in the `ORDER BY` clause of an aggregate function. + Example: `SELECT array_agg(x ORDER BY y, y) FROM (VALUES ('a', 2)) t(x, y)`. ({issue}`8080`) + +## JDBC Driver + +* Remove legacy JDBC URL prefix `jdbc:presto:`. ({issue}`8042`) +* Remove legacy driver classes `io.prestosql.jdbc.PrestoDriver` + and `com.facebook.presto.jdbc.PrestoDriver`. ({issue}`8042`) + +## Hive connector + +* Add support for reading from Hive views that use `LATERAL VIEW EXPLODE` + or `LATERAL VIEW OUTER EXPLODE` over array of `STRUCT`. ({issue}`8120`) +* Improve performance of `ORDER BY ... LIMIT` queries on sorted data. ({issue}`6634`) + +## Iceberg connector + +* Fix failure when listing materialized views in `information_schema.tables` or via the + `java.sql.DatabaseMetaData.getTables()` JDBC API. ({issue}`8151`) + +## Memory connector + +* Improve performance of certain complex queries involving joins. ({issue}`8095`) + +## SPI + +* Remove deprecated `ConnectorPageSourceProvider.createPageSource()` method overrides. ({issue}`8077`) +* Add support for casting the columns of a redirected table scan when source column types don't match. ({issue}`6066`) +* Add `ConnectorMetadata.redirectTable()` to allow connectors to redirect table reads and metadata listings. ({issue}`7606`) +* Add `ConnectorMetadata.streamTableColumns()` for streaming column metadata in a redirection-aware manner. The + alternate method for column listing `ConnectorMetadata.listTableColumns()` is now deprecated. ({issue}`7606`) + diff --git a/430/_sources/release/release-359.md.txt b/430/_sources/release/release-359.md.txt new file mode 100644 index 000000000..2f2b4b82d --- /dev/null +++ b/430/_sources/release/release-359.md.txt @@ -0,0 +1,82 @@ +# Release 359 (1 Jul 2021) + +## General + +* Raise minimum required Java version for running Trino server to 11.0.11. ({issue}`8103`) +* Add support for row pattern recognition in window specification. ({issue}`8141`) +* Add support for {doc}`/sql/set-time-zone`. ({issue}`8112`) +* Add {func}`geometry_nearest_points`. ({issue}`8280`) +* Add {func}`current_groups`. ({issue}`8446`) +* Add support for `varchar`, `varbinary` and `date` types to {func}`make_set_digest`. ({issue}`8295`) +* Add support for granting `UPDATE` privileges. ({issue}`8279`) +* List materialized view columns in the `information_schema.columns` table. ({issue}`8113`) +* Expose comments in views and materialized views in `system.metadata.table_comments` correctly. ({issue}`8327`) +* Fix query failure for certain queries with `ORDER BY ... LIMIT` on sorted data. ({issue}`8184`) +* Fix incorrect query results for certain queries using `LIKE` with pattern against + `char` columns in the `WHERE` clause. ({issue}`8311`) +* Fix planning failure when using {func}`hash_counts`. ({issue}`8248`) +* Fix error message when grouping expressions in `GROUP BY` queries contain aggregations, window functions or grouping operations. ({issue}`8247`) + +## Security + +* Fix spurious impersonation check when applying user mapping for password authentication. ({issue}`7027`) +* Fix handling of multiple LDAP user bind patterns. ({issue}`8134`) + +## Web UI + +* Show session timezone in query details page. ({issue}`4196`) + +## Docker image + +* Add support for ARM64. ({issue}`8397`) + +## CLI + +* Add support for logging of network traffic via the `--network-logging` command line option. ({issue}`8329`) + +## BigQuery connector + +* Add `bigquery.views-cache-ttl` config property to allow configuring the cache expiration for BigQuery views. ({issue}`8236`) +* Fix incorrect results when accessing BigQuery records with wrong index. ({issue}`8183`) + +## Elasticsearch connector + +* Fix potential incorrect results when queries contain an `IS NULL` predicate. ({issue}`3605`) +* Fix failure when multiple indexes share the same alias. ({issue}`8158`) + +## Hive connector + +* Rename `hive-hadoop2` connector to `hive`. ({issue}`8166`) +* Add support for Hive views which use `GROUP BY` over a subquery that also uses `GROUP BY` on matching columns. ({issue}`7635`) +* Add support for granting `UPDATE` privileges when `hive.security=sql-standard` is used. ({issue}`8279`) +* Add support for inserting data into CSV and TEXT tables with `skip_header_line_count` table property set to 1. + The same applies to creating tables with data using `CREATE TABLE ... AS SELECT` syntax. ({issue}`8390`) +* Disallow creating CSV and TEXT tables with data if `skip_header_line_count` is set to a value + greater than 0. ({issue}`8373`) +* Fix query failure when reading from a non-ORC insert-only transactional table. ({issue}`8259`) +* Fix incorrect results when reading ORC ACID tables containing deleted rows. ({issue}`8208`) +* Respect `hive.metastore.glue.get-partition-threads` configuration property. ({issue}`8320`) + +## Iceberg connector + +* Do not include Hive views in `SHOW TABLES` query results. ({issue}`8153`) + +## MongoDB connector + +* Skip creating an index for the `_schema` collection if it already exists. ({issue}`8264`) + +## MySQL connector + +* Support reading and writing `timestamp` values with precision higher than 3. ({issue}`6910`) +* Support predicate pushdown on `timestamp` columns. ({issue}`7413`) +* Handle `timestamp` values during forward offset changes ('gaps' in DST) correctly. ({issue}`5449`) + +## SPI + +* Introduce `ConnectorMetadata#listMaterializedViews` for listing materialized view names. ({issue}`8113`) +* Introduce `ConnectorMetadata#getMaterializedViews` for getting materialized view definitions. ({issue}`8113`) +* Enable connector to delegate materialized view refresh to itself. ({issue}`7960`) +* Allow computing HyperLogLog based approximate set summary as a column statistic during `ConnectorMetadata` + driven statistics collection flow. ({issue}`8355`) +* Report output column types through `EventListener`. ({issue}`8405`) +* Report input column information for queries involving set operations (`UNION`, `INTERSECT` and `EXCEPT`). ({issue}`8371`) diff --git a/430/_sources/release/release-360.md.txt b/430/_sources/release/release-360.md.txt new file mode 100644 index 000000000..120f8f03a --- /dev/null +++ b/430/_sources/release/release-360.md.txt @@ -0,0 +1,132 @@ +# Release 360 (30 Jul 2021) + +## General + +* Improve support for correlated subqueries with `GROUP BY` or `LIMIT` and + complex correlated filter conditions. ({issue}`8554`) +* Report cumulative query system memory usage. ({issue}`8615`) +* Fix `DROP SCHEMA` and `DROP SCHEMA RESTRICT` not to drop then schema if it is not empty. ({issue}`8660`) +* Fix query failure when there is a constant predicate on some + column `col` (e.g `col=1`), followed by `ORDER BY col` and `LIMIT`. ({issue}`8535`) +* Fix `SHOW CREATE SCHEMA` failure. ({issue}`8598`) +* Fix query failure when running `SHOW CREATE SCHEMA information_schema`. ({issue}`8600`) +* Improve performance of `WHERE` clause evaluation. ({issue}`8624`) +* Reduce coordinator network load. ({issue}`8460`) +* Improve query performance by sending collected dynamic filters from coordinator to workers. ({issue}`5183`) +* Improve performance of inequality joins where join condition sides have different type. ({issue}`8500`) +* Improve performance of `IN ()` expressions. ({issue}`8639`) + +## Security + +* Add support for automatic configuration of TLS for {doc}`/security/internal-communication`. This removes + the need to provision per-worker TLS certificates. ({issue}`7954`) + +## CLI + +* Fix auto completion when pressing the tab button. ({issue}`8529`) + +## ClickHouse connector + +* Add `write.batch-size` connector configuration property to control JDBC batch size used during writes. + It can also be controlled using the `write_batch_size` session property. ({issue}`8434`) +* Add new configuration property `insert.non-transactional-insert.enabled` to allow INSERT queries to write directly + to the target table. This can improve performance in some cases by sacrificing transactional behaviour. It can also + be controlled using `non_transactional_insert` session property. ({issue}`8496`) + +## Elasticsearch connector + +* Add support for assuming an IAM role. ({issue}`8714`) + +## Hive connector + +* Fix data corruption when performing `UPDATE` or `INSERT` on ORC ACID transactional table. ({issue}`8268`, {issue}`8452`) + +## Iceberg connector + +* Add support for Trino views. ({issue}`8540`) +* Prevent incorrect query results by failing a query when Iceberg table has row-level deletes. ({issue}`8450`) +* Fix query failure when joining with a bucketed Iceberg table. ({issue}`7502`) +* Fix query failure when showing stats for a bucketed Iceberg table. ({issue}`8616`) +* Fix query failure when joining with a partitioned table that has structural columns (`array`, `map` or `row`). ({issue}`8647`) +* Fix failures for queries that write tables in Parquet format. ({issue}`5201`) +* Improve query planning time by reducing calls to the metastore. ({issue}`8676`, {issue}`8689`) + +## MemSQL connector + +* Add `write.batch-size` connector configuration property to control JDBC batch size used during writes. + It can also be controlled using the `write_batch_size` session property. ({issue}`8434`) +* Add new configuration property `insert.non-transactional-insert.enabled` to allow INSERT queries to write directly + to the target table. This can improve performance in some cases by sacrificing transactional behaviour. It can also + be controlled using `non_transactional_insert` session property. ({issue}`8496`) +* Partial support for `DELETE` statement where predicate can be fully pushed down to the remote datasource. ({issue}`6287`) +* Fix performance regression of writes. ({issue}`8559`) + +## MongoDB connector + +* Add support for `json` type. ({issue}`8352`) +* Support reading MongoDB `DBRef` type. ({issue}`3134`) + +## MySQL connector + +* Add `write.batch-size` connector configuration property to control JDBC batch size used during writes. + It can also be controlled using the `write_batch_size` session property. ({issue}`8434`) +* Add new configuration property `insert.non-transactional-insert.enabled` to allow INSERT queries to write directly + to the target table. This can improve performance in some cases by sacrificing transactional behaviour. It can also + be controlled using `non_transactional_insert` session property. ({issue}`8496`) +* Partial support for `DELETE` statement where predicate can be fully pushed down to the remote datasource. ({issue}`6287`) +* Fix performance regression of writes. ({issue}`8559`) + +## Oracle connector + +* Add `write.batch-size` connector configuration property to control JDBC batch size used during writes. + It can also be controlled using the `write_batch_size` session property. ({issue}`8434`) +* Add new configuration property `insert.non-transactional-insert.enabled` to allow INSERT queries to write directly + to the target table. This can improve performance in some cases by sacrificing transactional behaviour. It can also + be controlled using `non_transactional_insert` session property. ({issue}`8496`) +* Partial support for `DELETE` statement where predicate can be fully pushed down to the remote datasource. ({issue}`6287`) + +## Phoenix connector + +* Add `write.batch-size` connector configuration property to control JDBC batch size used during writes. + It can also be controlled using the `write_batch_size` session property. ({issue}`8434`) +* Add new configuration property `insert.non-transactional-insert.enabled` to allow INSERT queries to write directly + to the target table. This can improve performance in some cases by sacrificing transactional behaviour. It can also + be controlled using `non_transactional_insert` session property. ({issue}`8496`) +* Improve performance of `ORDER BY ... LIMIT` queries on sorted data for Phoenix 5. ({issue}`8171`) +* Partial support for `DELETE` statement where predicate can be fully pushed down to the remote datasource. ({issue}`6287`) +* Fix performance regression of writes. ({issue}`8559`) + +## PostgreSQL connector + +* Add `write.batch-size` connector configuration property to control JDBC batch size used during writes. + It can also be controlled using the `write_batch_size` session property. ({issue}`8434`) +* Add new configuration property `insert.non-transactional-insert.enabled` to allow INSERT queries to write directly + to the target table. This can improve performance in some cases by sacrificing transactional behaviour. It can also + be controlled using `non_transactional_insert` session property. ({issue}`8496`) +* Partial support for `DELETE` statement where predicate can be fully pushed down to the remote datasource. ({issue}`6287`) + +## Redshift connector + +* Add `write.batch-size` connector configuration property to control JDBC batch size used during writes. + It can also be controlled using the `write_batch_size` session property. ({issue}`8434`) +* Add new configuration property `insert.non-transactional-insert.enabled` to allow INSERT queries to write directly + to the target table. This can improve performance in some cases by sacrificing transactional behaviour. It can also + be controlled using `non_transactional_insert` session property. ({issue}`8496`) +* Partial support for `DELETE` statement where predicate can be fully pushed down to the remote datasource. ({issue}`6287`) + +## SQL Server connector + +* Add `write.batch-size` connector configuration property to control JDBC batch size used during writes. + It can also be controlled using the `write_batch_size` session property. ({issue}`8434`) +* Add new configuration property `insert.non-transactional-insert.enabled` to allow INSERT queries to write directly + to the target table. This can improve performance in some cases by sacrificing transactional behaviour. It can also + be controlled using `non_transactional_insert` session property. ({issue}`8496`) +* Partial support for `DELETE` statement where predicate can be fully pushed down to the remote datasource. ({issue}`6287`) +* Fix performance regression of writes. ({issue}`8559`) + +## SPI + +* Cast materialized view storage table columns to match view column types. ({issue}`8408`) +* Remove deprecated `ConnectorSplitManager#getSplits` method overrides. ({issue}`8569`) +* Introduce `ConnectorPageSource#getCompletedPositions` for tracing physically read positions. ({issue}`8524`) +* Remove deprecated `TupleDomain.transform`. ({issue}`8056`) diff --git a/430/_sources/release/release-361.md.txt b/430/_sources/release/release-361.md.txt new file mode 100644 index 000000000..5eca09ae4 --- /dev/null +++ b/430/_sources/release/release-361.md.txt @@ -0,0 +1,123 @@ +# Release 361 (27 Aug 2021) + +## General + +* Add support for subqueries in `MATCH_RECOGNIZE` and `WINDOW` clause. ({issue}`8736`) +* Add `system.metadata.materialized_views` table that contains detailed information about materialized views. ({issue}`8796`) +* Support table redirection for `INSERT`, `UPDATE` and `DELETE` operations. ({issue}`8683`) +* Improve performance of {func}`sum` and {func}`avg` aggregations on `decimal` values. ({issue}`8878`) +* Improve performance for queries using `IN` predicate with moderate to large number of constants. ({issue}`8833`) +* Fix failures of specific queries accessing `row` columns with + with field names that would require quoting when used as an identifier. ({issue}`8845`) +* Fix incorrect results for queries with a comparison between a `varchar` column and a `char` constant. ({issue}`8984`) +* Fix invalid result when two decimals are added together. This happened in certain + queries where decimals had different precision. ({issue}`8973`) +* Prevent dropping or renaming objects with an incompatible SQL command. For example, `DROP TABLE` no longer allows dropping a view. ({issue}`8869`) + +## Security + +* Add support for OAuth2/OIDC opaque access tokens. The property + `http-server.authentication.oauth2.audience` has been removed in favor of + using `http-server.authentication.oauth2.client-id`, as expected by OIDC. + The new property `http-server.authentication.oauth2.additional-audiences` + supports audiences which are not the `client-id`. Additionally, the new + property `http-server.authentication.oauth2.issuer` is now required; + tokens which are not issued by this URL will be rejected. ({issue}`8641`) + +## JDBC driver + +* Implement the `PreparedStatement.getParameterMetaData()` method. ({issue}`2978`) +* Fix listing columns where table or schema name pattern contains an upper case value. + Note that this fix is on the server, not in the JDBC driver. ({issue}`8978`) + +## BigQuery connector + +* Fix incorrect result when using BigQuery `time` type. ({issue}`8999`) + +## Cassandra connector + +* Add support for predicate pushdown of `smallint`, `tinyint` and `date` types on partition columns. ({issue}`3763`) +* Fix incorrect results for queries containing inequality predicates on a clustering key in the `WHERE` clause. ({issue}`401`) + +## ClickHouse connector + +* Allow limiting the size of the metadata cache via the `metadata.cache-maximum-size` configuration property. ({issue}`8652`) +* Fix incorrect results for aggregation functions applied to columns of type `varchar` and `char`. ({issue}`7320`) + +## Druid connector + +* Allow limiting the size of the metadata cache via the `metadata.cache-maximum-size` configuration property. ({issue}`8652`) + +## Elasticsearch connector + +* Add support for reading fields as `json` values. ({issue}`7308`) + +## Hive connector + +* Expose `$properties` system table for Trino and Hive views. ({issue}`8805`) +* Add support for translating Hive views which contain common table expressions. ({issue}`5977`) +* Add support for translating Hive views which contain outer parentheses. ({issue}`8789`) +* Add support for translating Hive views which use the `from_utc_timestamp` function. ({issue}`8502`) +* Add support for translating Hive views which use the `date` function. ({issue}`8789`) +* Add support for translating Hive views which use the `pmod` function. ({issue}`8935`) +* Prevent creating of tables that have column names containing commas, or leading or trailing spaces. ({issue}`8954`) +* Improve performance of updating Glue table statistics for partitioned tables. ({issue}`8839`) +* Change default Glue statistics read/write parallelism from 1 to 5. ({issue}`8839`) +* Improve performance of querying Parquet data for files containing column indexes. ({issue}`7349`) +* Fix query failure when inserting data into a Hive ACID table which is not explicitly bucketed. ({issue}`8899`) + +## Iceberg connector + +* Fix reading or writing Iceberg tables that previously contained a + partition field that was later dropped. ({issue}`8730`) +* Allow reading from Iceberg tables which specify the Iceberg + `write.object-storage.path` table property. ({issue}`8573`) +* Allow using randomized location when creating a table, so that future table + renames or drops do not interfere with new tables created with the same name. + This can be enabled using the `iceberg.unique-table-location` configuration + property. ({issue}`6063`) +* Return proper query results for queries accessing multiple snapshots of single Iceberg table. ({issue}`8868`) + +## MemSQL connector + +* Allow limiting the size of the metadata cache via the `metadata.cache-maximum-size` configuration property. ({issue}`8652`) + +## MongoDB connector + +* Add {func}`timestamp_objectid` function. ({issue}`8824`) +* Enable `mongodb.socket-keep-alive` config property by default. ({issue}`8832`) + +## MySQL connector + +* Allow limiting the size of the metadata cache via the `metadata.cache-maximum-size` configuration property. ({issue}`8652`) +* Fix incorrect results for aggregation functions applied to columns of type `varchar` and `char`. ({issue}`7320`) + +## Oracle connector + +* Allow limiting the size of the metadata cache via the `metadata.cache-maximum-size` configuration property. ({issue}`8652`) + +## Phoenix connector + +* Allow limiting the size of the metadata cache via the `metadata.cache-maximum-size` configuration property. ({issue}`8652`) + +## Pinot connector + +* Implement aggregation pushdown for `count`, `avg`, `min`, `max`, `sum`, `count(DISTINCT)` and `approx_distinct`. + It is enabled by default and can be disabled using the configuration property `pinot.aggregation-pushdown.enabled` + or the catalog session property `aggregation_pushdown_enabled`. ({issue}`4140`) +* Allow `https` URLs in `pinot.controller-urls`. ({issue}`8617`) +* Fix failures when querying `information_schema.columns` with a filter on the table name. ({issue}`8307`) + +## PostgreSQL connector + +* Allow limiting the size of the metadata cache via the `metadata.cache-maximum-size` configuration property. ({issue}`8652`) +* Fix incorrect results for aggregation functions applied to columns of type `varchar` and `char`. ({issue}`7320`) + +## Redshift connector + +* Allow limiting the size of the metadata cache via the `metadata.cache-maximum-size` configuration property. ({issue}`8652`) + +## SQL Server connector + +* Allow limiting the size of the metadata cache via the `metadata.cache-maximum-size` configuration property. ({issue}`8652`) +* Fix incorrect results for aggregation functions applied to columns of type `varchar` and `char`. ({issue}`7320`) diff --git a/430/_sources/release/release-362.md.txt b/430/_sources/release/release-362.md.txt new file mode 100644 index 000000000..fe6ca261a --- /dev/null +++ b/430/_sources/release/release-362.md.txt @@ -0,0 +1,104 @@ +# Release 362 (20 Sep 2021) + +## General + +* Add support for {func}`listagg`. ({issue}`4835`) +* Improve join performance. ({issue}`8974`) +* Improve performance of queries that contain a `DISTINCT` clause. ({issue}`8967`, {issue}`9194`) +* Improve query performance by reducing CPU overhead of repartitioning data across worker nodes. ({issue}`9102`) +* Split metrics that are reported in operator stats into `connectorMetrics` and `metrics`. ({issue}`9156`) +* Prevent failure when querying `system.materialized_views` and a materialized view is dropped concurrently. ({issue}`9050`) +* Fix incorrect results for aggregations that contain `DISTINCT` and `FILTER` clauses. ({issue}`9265`) +* Fix incorrect query results when row pattern contains back references. ({issue}`9109`) +* Fix `ALTER SCHEMA ... SET AUTHORIZATION` to resolve roles using the catalog of the schema instead of the session catalog. ({issue}`9066`) +* Fix query failure when query contains a cast from `varchar` to a shorter `char`. ({issue}`9036`) +* Fix planning failure of `INSERT` statement when source table has hidden columns. ({issue}`9150`) +* Fix planning of recursive queries when the recursion, the base plan, or the recursion step plan produce duplicate outputs. ({issue}`9153`) +* Fix failure when querying the [optimizer_rule_stats](optimizer-rule-stats) system table. ({issue}`8700`) +* Fix failure for queries that push projections into connectors. ({issue}`6200`) +* Fix planning timeout for queries containing `IS NULL`, `AND`, and `OR` predicates in the `WHERE` clause. ({issue}`9250`) +* Fix failure for queries containing `ORDER BY ... LIMIT` when columns in the subquery are known to be constant. ({issue}`9171`) + +## Security + +* Add `IN catalog` clause to `CREATE ROLE`, `DROP ROLE`, `GRANT ROLE`, `REVOKE ROLE`, and `SET ROLE` to specify + the target catalog of the statement instead of using the current session catalog. This change is necessary to + support system roles in the future. The old behavior of these statements can be restored by setting the + `deprecated.legacy-catalog-roles` config property. ({issue}`9087`) + +## Web UI + +* Add query error info to cluster overview page. ({issue}`8762`) + +## JDBC driver + +* Improve performance of listing table columns via `java.sql.DatabaseMetaData` API when filtering on schema name. ({issue}`9214`) + +## Server RPM + +* Change RPM architecture to `noarch` to allow installing on any machine type. ({issue}`9187`) + +## BigQuery connector + +* Support case insensitive name matching for BigQuery views. ({issue}`9164`) +* Change type mapping of BigQuery `datetime` from `timestamp(3)` to `timestamp(6)` in Trino. ({issue}`9052`) +* Change type mapping of BigQuery `time` from `time with time zone` to `time(6)` in Trino. ({issue}`9052`) +* Change type mapping of BigQuery `timestamp` from `timestamp(3) with time zone` to `timestamp(6) with time zone` in Trino. ({issue}`9052`) +* Fix failure for queries where predicate on `datetime` column is pushed down to BigQuery. ({issue}`9005`) +* Fix incorrect results when using parameterized `numeric` type with non-default precision and scale. ({issue}`9060`) +* Fix query failure when accessing tables with unsupported data type columns. ({issue}`9086`) +* Fix failure for queries where predicate on `float64` column involving `+infinity` or + `-infinity` values is pushed down to BigQuery. ({issue}`9122`) + +## Cassandra connector + +* Change minimum number of speculative executions from 2 to 1. ({issue}`9096`) + +## Hive connector + +* Support reading Parquet timestamp stored with millisecond or microsecond precision as `INT64` with + logical type annotations when Hive timestamp precision is `NANOSECONDS`. ({issue}`9139`) +* Support reading Parquet timestamp stored as `INT64` with nanosecond precision. ({issue}`9188`) +* Support writing Parquet timestamp stored as `INT64` with nanosecond precision when experimental Parquet writer is enabled. + To use this, the Hive timestamp precision should be set to `NANOSECONDS`. ({issue}`9188`) +* Support loading of S3 mappings via HTTP(S) url. The `hive.s3.security-mapping.config-file property` can now + either point to a local file or a URL. ({issue}`6210`) +* Allow reading from tables bucketed on a column that uses type for which bucketing is not natively + supported by Trino. Writing to such tables is still not allowed. ({issue}`8986`) +* Extend set of statistics reported by JMX to include metrics for calls made to the Glue statistics API. ({issue}`9100`) +* Limit maximum file size generated by write operations to 1 GB by default. The limit is not exact and is applied on a best-effort basis. + The limit can be set with the `hive.target-max-file-size` configuration property or the `target_max_file_size` session property. ({issue}`7991`) +* Allow specifying the list of schemas for which Trino will enforce that queries use a filter on partition keys for source tables. + The list can be specified using the `hive.query-partition-filter-required-schemas`, or the `query_partition_filter_required_schemas` session property. + The list is taken into consideration only if the `hive.query-partition-filter-required` configuration property or the `query_partition_filter_required` + session property is set to `true`. ({issue}`9106`) +* Fix failure when writing Parquet files with Snappy compression on ARM64. ({issue}`9148`) +* Fix performance regression when reading Parquet files that contain dictionaries. ({issue}`9161`) +* Fix incorrect table locking in Hive metastore when querying unpartitioned non-transactional tables. ({issue}`9070`) +* Fix `ArrayIndexOutOfBoundsException` when using the experimental Parquet writer. ({issue}`5518`) +* Fix reading Avro tables written with older Avro versions that had certain semi-invalid schemas. ({issue}`9114`) +* Fix possible `INSERT`/`UPDATE`/`ANALYZE` query failure when Glue metastore is in use and table statistics collection is enabled. ({issue}`9297`) + +## Iceberg connector + +* Add support for Iceberg `uuid` type. ({issue}`6663`) +* Fix the mapping of nested fields between table metadata and Parquet file metadata. This + enables evolution of `row` typed columns for Iceberg tables stored in Parquet. ({issue}`9124`) +* Fix failure for queries filtering on columns with array, map, or row type. ({issue}`8822`) +* Fix `ArrayIndexOutOfBoundsException` when using the experimental Parquet writer. ({issue}`5518`) +* Fix query failures caused by errors reading certain Avro metadata files. ({issue}`9114`) + +## Pinot connector + +* Support pushdown of filters on `varbinary` columns to Pinot. ({issue}`9180`) +* Fix incorrect results for queries that contain aggregations and `IN` and `NOT IN` filters over `varchar` columns. ({issue}`9133`) +* Fix failure for queries with filters on `real` or `double` columns having `+Infinity` or `-Infinity` values. ({issue}`9180`) + +## TPC-H connector + +* Add support for switching the mapping of floating point values between SQL `double` and `decimal` types. The mapping + can be set via the `tpch.double-type-mapping` configuration property. ({issue}`7990`) + +## SPI + +* Change `Connector.isSingleStatementWritesOnly()` to return `true` by default. ({issue}`8872`) diff --git a/430/_sources/release/release-363.md.txt b/430/_sources/release/release-363.md.txt new file mode 100644 index 000000000..8f5e5d632 --- /dev/null +++ b/430/_sources/release/release-363.md.txt @@ -0,0 +1,71 @@ +# Release 363 (6 Oct 2021) + +## General + +* Add {doc}`/admin/event-listeners-http` implementation which sends JSON serialized events to a remote HTTP endpoint. ({issue}`8821`) +* Improve performance of queries that group by `bigint` columns. ({issue}`9510`) +* Improve performance of queries that process row or array data. ({issue}`9402`) +* Improve query planning performance. ({issue}`9462`) +* Reduce query memory usage when spilling occurs. ({issue}`9270`, {issue}`9275`) +* Reduce memory usage for processing `JOIN` clauses. ({issue}`9327`) +* Fix potential data loss in query results when clients retry requests to the coordinator. ({issue}`9453`) +* Fix incorrect result for comparisons between zero-valued decimals. ({issue}`8985`) +* Fix `SHOW ROLES` failure when there are no roles to display. ({issue}`9319`) +* Fix `EXPLAIN ANALYZE` to show estimates. ({issue}`9396`) +* Fix incorrect result for {func}`round` with precision set to 0. ({issue}`9371`) +* Respect `deprecated.legacy-catalog-roles=true` configuration property in `SHOW ROLES`, + `SHOW CURRENT ROLES` and `SHOW ROLE GRANTS` statements. ({issue}`9496`) + +## Python client + +* Fix column type reported in `cursor.description` for `time with time zone` column. ({issue}`9460`) + +## BigQuery connector + +* Fix failure for queries where predicates on `geography`, `array` or `struct` column are pushed down to BigQuery. ({issue}`9391`) + +## Cassandra connector + +* Add support for Cassandra `tuple` type. ({issue}`8570`) + +## Elasticsearch connector + +* Add support for `scaled_float` type. ({issue}`9358`) + +## Hive connector + +* Support usage of `avro_schema_url` table property in partitioned tables. ({issue}`9370`} +* Add support for insert overwrite operations on S3-backed tables. ({issue}`9234`) +* Improve query performance when reading Parquet data with predicate on a `decimal` column. ({issue}`9338`) +* Fix `Failed reading parquet data: Socket is closed by peer` query failure when reading from Parquet table with a predicate. ({issue}`9097`) +* Fix query failure when updating or deleting from an ORC ACID transactional table that has some rows deleted since the last major compaction. ({issue}`9354`) +* Fix failure when reading large Parquet files. ({issue}`9469`) +* Fix failures for some `UPDATE` queries, such as those where the `SET` clause contains the same constant more than once. ({issue}`9295`) +* Fix incorrect results when filtering on Parquet columns containing a dot in their name. ({issue}`9516`) + +## Iceberg connector + +* Improve query performance when reading Parquet data with predicate on a `decimal` column. ({issue}`9338`) +* Fix support for comments when adding a new column. Previously, they were silently ignored. ({issue}`9123`) +* Fix page and block sizing when writing Parquet data. ({issue}`9326`) +* Fix failure when reading large Parquet files. ({issue}`9469`) + +## MySQL connector + +* Add support for variable precision `time` type. ({issue}`9339`) +* Support `CREATE TABLE` and `CREATE TABLE AS` statements for `time` type. ({issue}`9339`) + +## Phoenix connector + +* Allowing forcing the mapping of certain types to `varchar`. This can be enabled by + setting the `jdbc-types-mapped-to-varchar` configuration property to a comma-separated + list of type names. ({issue}`2084`) + +## Pinot connector + +* Fix failure when a column name is a reserved keyword. ({issue}`9373`) + +## SQL Server connector + +* Add support for SQL Server `datetimeoffset` type. ({issue}`9329`) +* Fix failure for queries where predicates on `text` or `ntext` typed columns are pushed down to SQL Server. ({issue}`9387`) diff --git a/430/_sources/release/release-364.md.txt b/430/_sources/release/release-364.md.txt new file mode 100644 index 000000000..5d70ee5d4 --- /dev/null +++ b/430/_sources/release/release-364.md.txt @@ -0,0 +1,74 @@ +# Release 364 (1 Nov 2021) + +## General + +* Add support for [`ALTER MATERIALIZED VIEW ... RENAME TO ...`](/sql/alter-materialized-view). ({issue}`9492`) +* Improve performance of `GROUP BY` with single grouping column. ({issue}`9514`) +* Improve performance of decimal aggregations. ({issue}`9640`) +* Improve performance when evaluating the `WHERE` and `SELECT` clause. ({issue}`9610`) +* Improve performance when computing the product of `decimal` values with precision larger than 19. ({issue}`9744`) +* Improve CPU coordinator utilization. ({issue}`8650`) +* Remove support for the `unwrap_casts` session property and `optimizer.unwrap-casts` configuration option. ({issue}`9550`) +* Fix incorrect results for queries with nested joins and ``IS NOT DISTINCT FROM`` join clauses. ({issue}`9805`) +* Fix displaying character type dynamic filter values in ``EXPLAIN ANALYZE``. ({issue}`9673`) +* Fix query failure for update operation if it has a correlated subquery. ({issue}`8286`) +* Fix decimal division when result is between `-1` and `0`. ({issue}`9696`) +* Fix [`SHOW STATS`](/sql/show-stats) failure for a query projecting a boolean column. ({issue}`9710`) + +## Web UI + +* Improve responsiveness of Web UI when query history contains queries with long query text. ({issue}`8892`) + +## JDBC driver + +* Allow using token from existing Kerberos context. This allows the client to perform Kerberos authentication without + passing the Keytab or credential cache to the driver. ({issue}`4826`) + +## Cassandra connector + +* Map Cassandra `uuid` type to Trino `uuid`. ({issue}`5231`) + +## Elasticsearch connector + +* Fix failure when documents contain fields of unsupported types. ({issue}`9552`) + +## Hive connector + +* Allow to skip setting permissions on new directories by setting `hive.fs.new-directory-permissions=skip` in connector properties file. ({issue}`9539`) +* Allow translating Hive views which cast `timestamp` columns to `decimal`. ({issue}`9530`) +* Add `optimize` table procedure for merging small files in non-transactional Hive table. + Procedure can be executed using `ALTER TABLE
EXECUTE optimize(file_size_threshold => ...)` syntax. ({issue}`9665`) +* Restrict partition overwrite on insert to auto-commit context only. ({issue}`9559`) +* Reject execution of `CREATE TABLE` when bucketing is requested on columns with unsupported column types. + Previously `CREATE` was allowed but it was not possible to insert data to such a table. ({issue}`9793`) +* Improve performance of querying Parquet data for files containing column indexes. ({issue}`9633`) +* Fix Hive 1 and Hive 3 incompatibility with Parquet files containing `char` or `varchar` data produced by the experimental Parquet writer. + Hive 2 or newer should now read such files correctly, while Hive 1.x is still known not to read them. ({issue}`9515`, ({issue}`6377`)) +* Fix `ArrayIndexOutOfBoundsException` when inserting into a partitioned table with `hive.target-max-file-size` set. ({issue}`9557`) +* Fix reading Avro schema written by Avro 1.8.2 with non-spec-compliant default values. ({issue}`9243`) +* Fix failure when querying nested Parquet data if column indexes are enabled. ({issue}`9587`) +* Fix incorrect results when querying Parquet data. ({issue}`9587`) +* Fix query failure when writing to a partitioned table with target max file size set. ({issue}`9557`) + +## Iceberg connector + +* Add support for renaming materialized views. ({issue}`9492`) +* Create Parquet files that can be read more efficiently. ({issue}`9569`) +* Improve query performance when dynamic filtering can be leveraged. ({issue}`4115`) +* Return value with UTC zone for table partitioned on `timestamp with time zone`. ({issue}`9704`) +* Fix data loss in case of concurrent inserts to a table. ({issue}`9583`) +* Fix query failure when reading from `$partitions` table for a table partitioned on `timestamp with time zone` or `uuid` ({issue}`9703`, {issue}`9757`) +* Fix query failure when reading Iceberg table statistics. ({issue}`9714`) + +## MemSQL connector + +* Support reading and writing MemSQL `datetime(6)` and `timestamp(6)` types as Trino `timestamp(6)`. ({issue}`9725`) + +## SQL Server connector + +* Fix query failure when `count(*)` aggregation is pushed down to SQL Server database and the table has more than 2147483647 rows. ({issue}`9549`) + +## SPI + +* Expose which columns are covered by dynamic filters. ({issue}`9644`) +* Add SPI for table procedures that can process table data in a distributed manner. Table procedures can be run via `ALTER TABLE ... EXECUTE ...` syntax. ({issue}`9665`) diff --git a/430/_sources/release/release-365.md.txt b/430/_sources/release/release-365.md.txt new file mode 100644 index 000000000..6fbb9d190 --- /dev/null +++ b/430/_sources/release/release-365.md.txt @@ -0,0 +1,149 @@ +# Release 365 (3 Dec 2021) + +## General + +* Add support for [`TRUNCATE TABLE`](/sql/truncate). ({issue}`8921`) +* Add support for aggregate functions in row pattern recognition context. ({issue}`8738`) +* Add support for time travel queries. ({issue}`8773`) +* Add support for spilling aggregations containing `ORDER BY` or `DISTINCT` clauses. ({issue}`9723`) +* Add [`contains`](ip-address-contains) function to check whether a CIDR contains an IP address. ({issue}`9654`) +* Report connector metrics in `EXPLAIN ANALYZE VERBOSE`. ({issue}`9858`) +* Report operator input row count distribution in `EXPLAIN ANALYZE VERBOSE`. ({issue}`10133`) +* Allow executing `INSERT` or `DELETE` statements on tables restricted with a row filter. ({issue}`8856`) +* Remove `owner` column from the `system.metadata.materialized_views` table. ({issue}`9961`) +* Remove the `optimizer.iterative-rule-based-column-pruning` config property. The legacy column pruning optimizer is no longer available. ({issue}`9564`) +* Improve performance of inequality joins. ({issue}`9307`) +* Improve performance of joins involving a small table on one side. ({issue}`9851`) +* Improve CPU utilization by adjusting `task.concurrency` automatically based on the number of physical cores. ({issue}`10088`) +* Make query final query statistics more accurate. ({issue}`9888`, {issue}`9913`) +* Improve query planning performance for queries containing large `IN` predicates. ({issue}`9874`) +* Reduce peak memory usage for queries involving the `rank`, `dense_rank`, or `row_number` window functions. ({issue}`10056`) +* Fix incorrect results when casting `bigint` values to `varchar(n)` type. ({issue}`552`) +* Fix query failure when the `PREPARE` statement is used with `DROP` or `INSERT` + and the table or schema name contains special characters. ({issue}`9822`) +* Fix minor memory leak when queries are abandoned during the initial query submission phase. ({issue}`9962`) +* Collect connector metrics after `ConnectorPageSource` is closed. ({issue}`9615`) + +## Security + +* Allow configuring HTTP proxy for OAuth2 authentication. ({issue}`9920`, {issue}`10069`) +* Add group-based and owner-based query access rules to file based system access control. ({issue}`9811`) +* Use internal names for discovery client when automatic TLS is enabled for internal communications. This allows `discovery.uri` + to be configured using a normal DNS name like `https://coordinator.trino` and still use automatic TLS certificates. ({issue}`9821`) +* Use Kerberos operating system ticket cache if keytab file is not provided to JDBC and CLI for Kerberos authentication. ({issue}`8987`) +* Fix internal communication automatic TLS on Java 17. ({issue}`9821`) + +## CLI + +* Automatically use HTTPS when port is set to 443. ({issue}`8798`) + +## BigQuery connector + +* Support reading `bignumeric` type whose precision is less than or equal to 38. ({issue}`9882`) +* Fix failure when a schema is dropped while listing tables. ({issue}`9954`) + +## Cassandra connector + +* Support reading user defined types in Cassandra. ({issue}`147`) + +## ClickHouse connector + +* Add support for [truncating tables](/sql/truncate). ({issue}`8921`) +* Fix incorrect query results when query contains predicates on `real` type columns. ({issue}`9998`) + +## Druid connector + +* Add support for [truncating tables](/sql/truncate). ({issue}`8921`) + +## Elasticsearch connector + +* Add support for additional Elastic Cloud node roles. ({issue}`9892`) +* Fix failure when empty values exist in numeric fields. ({issue}`9939`) + +## Hive connector + +* Allow reading empty files of type Parquet, RCFile, SequenceFile. ({issue}`9929`) +* Enable `hive.s3.streaming` by default. ({issue}`9715`) +* Improve performance by not generating splits for empty files. ({issue}`9929`) +* Improve performance of decimal `avg` aggregation. ({issue}`9738`) +* Improve performance when reading Parquet files with timestamps encoded using `int64` representation. ({issue}`9414`) +* Improve dynamic partition pruning efficiency. ({issue}`9866`, {issue}`9869`) +* Improve query performance on partitioned tables or tables with small files + by increasing `hive.split-loader-concurrency` from `4` to `64`. ({issue}`9979`) +* Fix reporting of number of read bytes for tables using `ORC` file format. ({issue}`10048`) +* Account for memory used for deleted row information when reading from ACID tables. ({issue}`9914`, {issue}`10070`) +* Fix `REVOKE GRANT OPTION` to revoke only the grant option instead of revoking the entire privilege. ({issue}`10094`) +* Fix bug where incorrect rows were deleted when deleting from a transactional table that has original files (before the first major compaction). ({issue}`10095`) +* Fix delete and update failure when changing a table after a major compaction. ({issue}`10120`) +* Fix incorrect results when decoding decimal values in Parquet reader. ({issue}`9971`) +* Fix `hive.dynamic-filtering.wait-timeout` not having any effect. ({issue}`10106`) +* Fix failure when reading Parquet data if column indexes are enabled. ({issue}`9890`, {issue}`10076`) + +## Iceberg connector + +* Add support for storing and reading UUID nested in `row`, `array` or `map` type. ({issue}`9918`) +* Use Iceberg's `schema.name-mapping.default` table property for scanning files + with missing Iceberg IDs. This aligns Trino behavior on migrated files with the + Iceberg spec. ({issue}`9959`) +* Use ZSTD compression by default. ({issue}`10058`) +* Add read-only security option which can be enabled by setting the configuration `iceberg.security=read-only`. ({issue}`9974`) +* Change schema of `$partitions` system table to avoid conflicts when table name contains a column named `row_count`, `file_count` + or `total_size`, or when a column is used for partitioning for part of table data, and it not used for partitioning in some other + part of the table data. ({issue}`9519`, {issue}`8729`). +* Improve performance when reading timestamps from Parquet files. ({issue}`9414`) +* Improve query performance for certain queries with complex predicates. ({issue}`9309`) +* Reduce resource consumption and create bigger files when writing to an Iceberg table with partitioning. Bigger files are more efficient to query later. ({issue}`9826`) +* Improve performance for queries on nested data through dereference pushdown. ({issue}`8129`) +* Write correct `file_size_in_bytes` in manifest when creating new ORC files. ({issue}`9810`) +* Fix query failures that could appear when reading Parquet files which contained ROW columns that were subject to schema evolution. ({issue}`9264`) +* Fix failure caused by stale metadata in the `rollback_to_snapshot` procedure. ({issue}`9921`) + +## Kudu connector + +* Avoid scanner time-out issues when reading Kudu tables. ({issue}`7250`) + +## MemSQL connector + +* Add support for [truncating tables](/sql/truncate). ({issue}`8921`) +* Fix incorrect query results when query contains predicates on `real` type columns. ({issue}`9998`) + +## MongoDB connector + +* Support connecting to MongoDB clusters via `mongodb.connection-url` config property. `mongodb.seeds` and `mongodb.credentials` properties are now deprecated. ({issue}`9819`) + +## MySQL connector + +* Add support for [truncating tables](/sql/truncate). ({issue}`8921`) +* Fix incorrect query results when query contains predicates on `real` type columns. ({issue}`9998`) + +## Oracle connector + +* Add support for [truncating tables](/sql/truncate). ({issue}`8921`) + +## Phoenix connector + +* Support reading `decimal` columns from Phoenix with unspecified precision or scale. ({issue}`9795`) +* Fix query failures when reading Phoenix tables. ({issue}`9151`) + +## Pinot connector + +* Update Pinot connector to be compatible with versions >= 0.8.0 and drop support for older versions. ({issue}`9098`) + +## PostgreSQL connector + +* Add support for [truncating tables](/sql/truncate). ({issue}`8921`) +* Add experimental support for range predicate pushdown on string columns. It can be enabled + by setting the `postgresql.experimental.enable-string-pushdown-with-collate` catalog configuration + property or the corresponding `enable_string_pushdown_with_collate` session property to `true`. ({issue}`9746`) + +## Redshift connector + +* Add support for [truncating tables](/sql/truncate). ({issue}`8921`) + +## SQL Server connector + +* Add support for [truncating tables](/sql/truncate). ({issue}`8921`) + +## SPI + +* Allow split manager to filter splits based on a predicate not expressible as a `TupleDomain`. ({issue}`7608`) diff --git a/430/_sources/release/release-366.md.txt b/430/_sources/release/release-366.md.txt new file mode 100644 index 000000000..8281d94bf --- /dev/null +++ b/430/_sources/release/release-366.md.txt @@ -0,0 +1,52 @@ +# Release 366 (14 Dec 2021) + +## General + +* Add support for automatic query retries. This feature can be turned on by setting the `retry-policy` + config property or the `retry_policy` session property to `retry`. ({issue}`9361`) +* Add CREATE privilege kind to SQL grammar. Note that this permission is not used by any + existing security systems, but is available for plugins. ({issue}`10206`) +* Add support for `DENY` statement in the engine. Note that this statement is not supported by any + existing security systems, but is available for plugins. ({issue}`10205`) +* Reduce lock contention during query execution. ({issue}`10246`, {issue}`10239`) +* Improve query performance through optimizations to in-memory representations. ({issue}`10225`) +* Reduce query latency for contended clusters or complex queries with multiple stages. ({issue}`10249`) +* Fix incorrect results or failure when casting numeric values to `varchar(n)` type. ({issue}`552`) +* Remove support for spilling aggregations containing `ORDER BY` or `DISTINCT` clauses and associated + configuration properties `spill-distincting-aggregations-enabled`, `spill-ordering-aggregations-enabled`. + ({issue}`10183`) + +## Elasticsearch connector + +* Read extended metadata from the `_meta.trino` index mapping attribute. `_meta.presto` is still + supported for backward compatibility. ({issue}`8383`) + +## Hive connector + +* Add support for redirects from Hive to Iceberg. This can be configured with `hive.iceberg-catalog-name` + catalog configuration property. ({issue}`10173`) +* Improve performance of uploading data into tables that use S3 filesystem. ({issue}`10180`) + +## Iceberg connector + +* Fix incorrect query results for tables partitioned on columns of type `binary`. ({issue}`9755`) + +## MemSQL connector + +* Fix incorrect result when a `date` value is older than or equal to `1582-10-14`. ({issue}`10054`) + +## MySQL connector + +* Fix incorrect result when a `date` value is older than or equal to `1582-10-14`. ({issue}`10054`) + +## Phoenix connector + +* Avoid running into out of memory errors with certain types of queries. ({issue}`10143`) + +## Prometheus connector + +* Support configuring a read timeout via the `prometheus.read-timeout` config property. ({issue}`10101`) + +## PostgreSQL connector + +* Fix incorrect result when a `date` value is older than or equal to `1582-10-14`. ({issue}`10054`) diff --git a/430/_sources/release/release-367.md.txt b/430/_sources/release/release-367.md.txt new file mode 100644 index 000000000..14af5ec6e --- /dev/null +++ b/430/_sources/release/release-367.md.txt @@ -0,0 +1,38 @@ +# Release 367 (22 Dec 2021) + +## General + +* Capture lineage details for columns from `WITH` clauses and subqueries. ({issue}`10272`) +* Improve `CREATE VIEW` error message when table or materialized view already exists. ({issue}`10186`) +* Disallow query retries when connectors cannot perform them safely. ({issue}`10064`) +* Improve performance when query retries are enabled by adding support for dynamic filtering. ({issue}`10274`) +* Fix failure in `min_by` and `max_by` aggregation execution. ({issue}`10347`) +* Fix planning failure for queries that access fields of `row` types by index (`ROW(...)[n]`) or that + select all row fields (`ROW(..).*`). ({issue}`10321`) +* Fix bug where certain queries which use broadcast joins could hang and never complete. ({issue}`10344`) +* Fix failure when row or array in `VALUES` clause contains nulls. ({issue}`10141`) + +## Security + +* Hide inaccessible columns from `SELECT *` statement when + the `hide-inaccessible-columns` configuration property is set to true. ({issue}`9991`) +* Disable `SET AUTHORIZATION` when `VIEW` runs as `DEFINER`. ({issue}`10351`) + +## Web UI + +* Improve user experience by introducing a new landing page for logout flow when + Oauth2 authentication is used. ({issue}`10299`) + +## Hive connector + +* Add procedure `system.flush_metadata_cache()` to flush metadata caches. ({issue}`10251`) +* Prevent data loss during `DROP SCHEMA` when schema location contains files but not tables. ({issue}`10146`) +* Ensure no duplicate rows are created if query which writes data to Hive table is retried. ({issue}`10252`, {issue}`10064`) + +## Iceberg connector + +* Prevent data loss during `DROP SCHEMA` when schema location contains files but not tables. ({issue}`9767`) + +## SPI + +* Fix `ClassNotFoundException` when using aggregation with a custom state type. ({issue}`10341`) diff --git a/430/_sources/release/release-368.md.txt b/430/_sources/release/release-368.md.txt new file mode 100644 index 000000000..f09c16371 --- /dev/null +++ b/430/_sources/release/release-368.md.txt @@ -0,0 +1,69 @@ +# Release 368 (11 Jan 2022) + +## General + +* Allow setting per task memory limits via `query.max-total-memory-per-task` + config property or via `query_max_total_memory_per_task` session property. + ({issue}`10308`) +* Improve wall time for query processing with the `phased` scheduling policy. + The previous behavior can be restored by setting the `query.execution-policy` + configuration property to `legacy-phased`. ({issue}`10350`) +* Enable `phased` scheduling policy by default. The previous behavior can be + restored by setting the `query.execution-policy` configuration property to + `all-at-once`. ({issue}`10455`) +* Improve performance of arithmetic operations involving decimals with precision + larger than 18. ({issue}`10051`) +* Reduce risk of out-of-memory failure on congested clusters with high memory + usage. ({issue}`10475`) +* Fix queries not being unblocked when placed in reserved memory pool. + ({issue}`10475`) +* Prevent execution of `REFRESH MATERIALIZED VIEW` from getting stuck. + ({issue}`10360`) +* Fix double reporting of scheduled time for scan operators in + `EXPLAIN ANALYZE`. ({issue}`10472`) +* Fix issue where the length of log file names grow indefinitely upon log + rotation. ({issue}`10394`) + +## Hive connector + +* Improve performance of decoding decimal values with precision larger than 18 + in ORC, Parquet and RCFile data. ({issue}`10051`) +* Disallow querying the properties system table for Delta Lake tables, since + Delta Lake tables are not supported. This fixes the previous behavior of + silently returning incorrect values. ({issue}`10447`) +* Reduce risk of worker out-of-memory exception when scanning ORC files. + ({issue}`9949`) + +## Iceberg connector + +* Fix Iceberg table creation with location when schema location inaccessible. + ({issue}`9732`) +* Support file based access control. ({issue}`10493`) +* Display the Iceberg table location in `SHOW CREATE TABLE` output. + ({issue}`10459`) + +## SingleStore (MemSQL) connector + +* Add support for `time` type. ({issue}`10332`) + +## Oracle connector + +* Fix incorrect result when a `date` value is older than or equal to + `1582-10-14`. ({issue}`10380`) + +## Phoenix connector + +* Add support for reading `binary` type. ({issue}`10539`) + +## PostgreSQL connector + +* Add support for accessing tables created with declarative partitioning in + PostgreSQL. ({issue}`10400`) + +## SPI + +* Encode long decimal values using two's complement representation and change + their carrier type to `io.trino.type.Int128` instead of + `io.airlift.slice.Slice`. ({issue}`10051`) +* Fix `ClassNotFoundException` when using aggregation with a custom state type. + ({issue}`10408`) diff --git a/430/_sources/release/release-369.md.txt b/430/_sources/release/release-369.md.txt new file mode 100644 index 000000000..f99b8ed8d --- /dev/null +++ b/430/_sources/release/release-369.md.txt @@ -0,0 +1,165 @@ +# Release 369 (24 Jan 2022) + +## General + +* Add support for `Pacific/Kanton` time zone. ({issue}`10679`) +* Display `Physical input read time` using most succinct time unit in `EXPLAIN + ANALYZE VERBOSE`. ({issue}`10576`) +* Fine tune request retry mechanism in HTTP event listener. ({issue}`10566`) +* Add support for using PostgreSQL and Oracle as backend database for resource + groups. ({issue}`9812`) +* Remove unnecessary spilling configuration properties `spill-order-by` and + `spill-window-operator`. ({issue}`10591`) +* Remove distinction between system and user memory to simplify + cluster configuration. The configuration property + `query.max-total-memory-per-node` is removed. Use `query.max-memory-per-node` + instead. ({issue}`10574`) +* Use formatting specified in the SQL standard when casting `double` and `real` + values to `varchar` type. ({issue}`552`) +* Add support for `ALTER MATERIALIZED VIEW ... SET PROPERTIES`. ({issue}`9613`) +* Add experimental implementation of task level retries. This can be enabled by + setting the `retry-policy` configuration property or the `retry_policy` + session property to `task`. ({issue}`9818`) +* Improve query wall time by splitting workload between nodes in a more balanced + way. Previous workload balancing policy can be restored via + `node-scheduler.splits-balancing-policy=node`. ({issue}`10660`) +* Prevent hanging query execution on failures with `phased` execution policy. + ({issue}`10656`) +* Catch overflow in decimal multiplication. ({issue}`10732`) +* Fix `UnsupportedOperationException` in `max_by` and `min_by` aggregation. + ({issue}`10599`) +* Fix incorrect results or failure when casting date to `varchar(n)` type. + ({issue}`552`) +* Fix issue where the length of log file names grow indefinitely upon log + rotation. ({issue}`10738`) + +## Security + +* Allow extracting groups from OAuth2 claims from + ``http-server.authentication.oauth2.groups-field``. ({issue}`10262`) + +## JDBC driver + +* Fix memory leak when using `DatabaseMetaData`. ({issue}`10584`, + {issue}`10632`) + +## BigQuery connector + +* Remove ``bigquery.case-insensitive-name-matching.cache-ttl`` configuration + option. It was previously ignored. ({issue}`10697`) +* Fix query failure when reading columns with `numeric` or `bignumeric` type. + ({issue}`10564`) + +## ClickHouse connector + +* Upgrade minimum required version to 21.3. ({issue}`10703`) +* Add support for [renaming schemas](/sql/alter-schema). ({issue}`10558`) +* Add support for setting [column comments](/sql/comment). ({issue}`10641`) +* Map ClickHouse `ipv4` and `ipv6` types to Trino `ipaddress` type. + ({issue}`7098`) +* Allow mapping ClickHouse `fixedstring` or `string` as Trino `varchar` via the + `map_string_as_varchar` session property. ({issue}`10601`) +* Disable `avg` pushdown on `decimal` types to avoid incorrect results. + ({issue}`10650`) +* Fix spurious errors when metadata cache is enabled. ({issue}`10544`, + {issue}`10512`) + +## Druid connector + +* Fix spurious errors when metadata cache is enabled. ({issue}`10544`, + {issue}`10512`) + +## Hive connector + +* Add support for writing Bloom filters in ORC files. ({issue}`3939`) +* Allow flushing the metadata cache for specific schemas, tables, or partitions + with the [flush_metadata_cache](hive-flush-metadata-cache) system procedure. + ({issue}`10385`) +* Add support for long lived AWS Security Token Service (STS) credentials for + authentication with Glue catalog. ({issue}`10735`) +* Ensure transaction locks in the Hive Metastore are released in case of query + failure when querying Hive ACID tables. ({issue}`10401`) +* Disallow reading from Iceberg tables when redirects from Hive to Iceberg are + not enabled. ({issue}`8693`, {issue}`10441`) +* Improve performance of queries using range predicates when reading ORC files + with Bloom filters. ({issue}`4108`) +* Support writing Parquet files greater than 2GB. ({issue}`10722`) +* Fix spurious errors when metadata cache is enabled. ({issue}`10646`, + {issue}`10512`) +* Prevent data loss during `DROP SCHEMA` when the schema location contains files + that are not part of existing tables. ({issue}`10485`) +* Fix inserting into transactional table when `task_writer_count` > 1. + ({issue}`9149`) +* Fix possible data corruption when writing data to S3 with streaming enabled. + ({issue}`10710 `) + +## Iceberg connector + +* Add `$properties` system table which can be queried to inspect Iceberg table + properties. ({issue}`10480`) +* Add support for `ALTER TABLE .. EXECUTE OPTIMIZE` statement. ({issue}`10497`) +* Respect Iceberg column metrics mode when writing. ({issue}`9938`) +* Add support for long lived AWS Security Token Service (STS) credentials for + authentication with Glue catalog. ({issue}`10735`) +* Improve performance of queries using range predicates when reading ORC files + with Bloom filters. ({issue}`4108`) +* Improve select query planning performance after write operations from Trino. + ({issue}`9340`) +* Ensure table statistics are accumulated in a deterministic way from Iceberg + column metrics. ({issue}`9716`) +* Prevent data loss during `DROP SCHEMA` when the schema location contains files + that are not part of existing tables. ({issue}`10485`) +* Support writing Parquet files greater than 2GB. ({issue}`10722`) +* Fix materialized view refresh when view a query references the same table + multiple times. ({issue}`10570`) + * Fix possible data corruption when writing data to S3 with streaming enabled. + ({issue}`10710 `) + +## MySQL connector + +* Fix spurious errors when metadata cache is enabled. ({issue}`10544`, + {issue}`10512`) + +## Oracle connector + +* Map Oracle `date` to Trino `timestamp(0)` type. ({issue}`10626`) +* Fix performance regression of predicate pushdown on indexed `date` columns. + ({issue}`10626`) +* Fix spurious errors when metadata cache is enabled. ({issue}`10544`, + {issue}`10512`) + +## Phoenix connector + +* Fix spurious errors when metadata cache is enabled. ({issue}`10544`, + {issue}`10512`) + +## Pinot connector + +* Add support for basic authentication. ({issue}`9531`) + +## PostgreSQL connector + +* Add support for [renaming schemas](/sql/alter-schema). ({issue}`8939`) +* Fix spurious errors when metadata cache is enabled. ({issue}`10544`, + {issue}`10512`) + +## Redshift connector + +* Add support for [renaming schemas](/sql/alter-schema). ({issue}`8939`) +* Fix spurious errors when metadata cache is enabled. ({issue}`10544`, + {issue}`10512`) + +## SingleStore (MemSQL) connector + +* Fix spurious errors when metadata cache is enabled. ({issue}`10544`, + {issue}`10512`) + +## SQL Server connector + +* Fix spurious errors when metadata cache is enabled. ({issue}`10544`, + {issue}`10512`) + +## SPI + +* Remove support for the `ConnectorMetadata.getTableLayout()` API. + ({issue}`781`) diff --git a/430/_sources/release/release-370.md.txt b/430/_sources/release/release-370.md.txt new file mode 100644 index 000000000..f82f1804e --- /dev/null +++ b/430/_sources/release/release-370.md.txt @@ -0,0 +1,139 @@ +# Release 370 (3 Feb 2022) + +## General + +* Add support for `DEFAULT` keyword in `ALTER TABLE...SET PROPERTIES...`. + ({issue}`10331`) +* Improve performance of map and row types. ({issue}`10469`) +* Improve performance when evaluating expressions in `WHERE` and `SELECT` + clauses. ({issue}`10322`) +* Prevent queries deadlock when using `phased` execution policy with dynamic + filters in multi-join queries. ({issue}`10868`) +* Fix query scheduling regression introduced in Trino 360 that caused + coordinator slowness in assigning splits to workers. ({issue}`10839`) +* Fix `information_schema` query failure when an `IS NOT NULL` predicate is + used. ({issue}`10861`) +* Fix failure when nested subquery contains a `TABLESAMPLE` clause. + ({issue}`10764`) + +## Security + +* Reduced the latency of successful OAuth 2.0 authentication. ({issue}`10929`) +* Fix server start failure when using JWT and OAuth 2.0 authentication together + (`http-server.authentication.type=jwt,oauth2`). ({issue}`10811`) + +## CLI + +* Add support for ARM64 processors. ({issue}`10177`) +* Allow to choose the way how external authentication is handled with the + `--external-authentication-redirect-handler` parameter. ({issue}`10248`) + +## RPM package + +* Fix failure when operating system open file count is set too low. + ({issue}`8819`) + +## Docker image + +* Change base image to `registry.access.redhat.com/ubi8/ubi`, since CentOS 8 has + reached end-of-life. ({issue}`10866`) + +## Cassandra connector + +* Fix query failure when pushing predicates on `uuid` partitioned columns. + ({issue}`10799`) + +## ClickHouse connector + +* Support creating tables with Trino `timestamp(0)` type columns. +* Drop support for ClickHouse servers older than version 20.7 to avoid using a + deprecated driver. You can continue to use the deprecated driver with the + `clickhouse.legacy-driver` flag when connecting to old servers. + ({issue}`10541`) +* Remove the legacy `allow-drop-table` configuration property. This defaulted to + `false`, which disallowed dropping tables, but other modifications were still + allowed. Use {doc}`/security/built-in-system-access-control` instead, if + desired. ({issue}`588`) + +## Druid connector + +* Remove the legacy `allow-drop-table` configuration property. This defaulted to + `false`, which disallowed dropping tables, but other modifications were still + allowed. Use {doc}`/security/built-in-system-access-control` instead, if + desired. ({issue}`588`) + +## Hive connector + +* Improve query performance when reading ORC data. ({issue}`10575`) +* Add configuration property `hive.single-statement-writes` to require + auto-commit for writes. This can be used to disallow multi-statement write + transactions. ({issue}`10820`) +* Fix sporadic query failure `Partition no longer exists` when working with wide + tables using a AWS Glue catalog as metastore. ({issue}`10696`) +* Fix `SHOW TABLES` failure when `hive.hide-delta-lake-tables` is enabled, and + Glue metastore references the table with no properties. ({issue}`10864`) + +## Iceberg connector + +* Fix query failure when reading from a table that underwent partitioning + evolution. ({issue}`10770`) +* Fix data corruption when writing Parquet files. ({issue}`9749`) + +## MySQL connector + +* Remove the legacy `allow-drop-table` configuration property. This defaulted to + `false`, which disallowed dropping tables, but other modifications were still + allowed. Use {doc}`/security/built-in-system-access-control` instead, if + desired. ({issue}`588`) + +## Oracle connector + +* Remove the legacy `allow-drop-table` configuration property. This defaulted to + `false`, which disallowed dropping tables, but other modifications were still + allowed. Use {doc}`/security/built-in-system-access-control` instead, if + desired. ({issue}`588`) + +## Phoenix connector + +* Fix incorrect result when a `date` value is older than or equal to + `1899-12-31`. ({issue}`10749`) + +## PostgreSQL connector + +* Remove the legacy `allow-drop-table` configuration property. This defaulted to + `false`, which disallowed dropping tables, but other modifications were still + allowed. Use {doc}`/security/built-in-system-access-control` instead, if + desired. ({issue}`588`) + +## Redshift connector + +* Remove the legacy `allow-drop-table` configuration property. This defaulted to + `false`, which disallowed dropping tables, but other modifications were still + allowed. Use {doc}`/security/built-in-system-access-control` instead, if + desired. ({issue}`588`) + +## SingleStore (MemSQL) connector + +* Remove the legacy `allow-drop-table` configuration property. This defaulted to + `false`, which disallowed dropping tables, but other modifications were still + allowed. Use {doc}`/security/built-in-system-access-control` instead, if + desired. ({issue}`588`) + +## SQL Server connector + +* Remove the legacy `allow-drop-table` configuration property. This defaulted to + `false`, which disallowed dropping tables, but other modifications were still + allowed. Use {doc}`/security/built-in-system-access-control` instead, if + desired. ({issue}`588`) + +## SPI + +* Allow null property names in `ConnetorMetadata#setTableProperties`. + ({issue}`10331`) +* Rename `ConnectorNewTableLayout` to `ConnectorTableLayout`. ({issue}`10587`) +* Connectors no longer need to explicitly declare handle classes. The + `ConnectorFactory.getHandleResolver` and `Connector.getHandleResolver` methods + are removed. ({issue}`10858`, {issue}`10872`) +* Remove unnecessary `Block.writePositionTo` and `BlockBuilder.appendStructure` + methods. Use of these methods can be replaced with the existing + `Type.appendTo` or `writeObject` methods. ({issue}`10602`) diff --git a/430/_sources/release/release-371.md.txt b/430/_sources/release/release-371.md.txt new file mode 100644 index 000000000..84db2fc9f --- /dev/null +++ b/430/_sources/release/release-371.md.txt @@ -0,0 +1,67 @@ +# Release 371 (16 Feb 2022) + +## General + +* Add support for using secrets in database backed resource group manager + configuration. ({issue}`10996`) +* Add support for the user group selector rule in database backed resource group + manager. ({issue}`10914`) +* Remove `deprecated.disable-set-properties-security-check-for-create-ddl` + configuration toggle. ({issue}`10923`) +* Prevent infinite planning loop by removing redundant predicates above table + scan. ({issue}`10532`) +* Prevent time outs for planning of certain queries. ({issue}`10985`) +* Fix performance regression in internal communication authentication + processing. ({issue}`10954`) +* Fix failure when casting values to `decimal(38, 38)`. ({issue}`10946`) +* Enforce timeout for idle transactions. ({issue}`10923`) +* Ensure removal of all catalog session properties when using session property + defaults with transactions. ({issue}`10923`) + +## Security + +* Invoke correct authorization check when table is created via `CREATE TABLE + AS`. ({issue}`10939`) + +## ClickHouse connector + +* Remove support for ClickHouse connector in Altinity distribution 20.3. + ({issue}`10975`) +* Add missing output of table properties for `SHOW CREATE TABLE` statements. + ({issue}`11027`) + +## Hive connector + +* Allow specifying AWS role session name via S3 security mapping config. + ({issue}`10714`) +* Disallow writes to bucketed tables recognized as created by Spark to prevent + data corruption. Spark uses a custom bucketing hash function that is not + compatible with Hive and Trino. ({issue}`10815`) +* Fix failure when reading Hive tables that contain symlinks that are text + files. ({issue}`10910`) +* Fix metastore impersonation for Avro tables. ({issue}`11035`) + +## Iceberg connector + +* Allow running queries performing DML on Iceberg tables with fault-tolerant + execution. ({issue}`10622`) +* Create files of up to approximately 1GB of size when writing. This can be + configured using `hive.target-max-file-size` catalog property or + `target_max_file_size` session property. ({issue}`10957`) + +## Kudu connector + +* Drop support for Kudu versions older than 1.13.0. ({issue}`10940`) + +## SQL Server connector + +* Fix incorrect results when negative dates are specified in predicates. + ({issue}`10263`) +* Fix incorrect results when writing negative dates. ({issue}`10263`) + +## SPI + +* Add `ConnectorSession` to the `Connector` `getMetadata` method. The former + signature is deprecated and should be updated. ({issue}`9482`) +* Remove deprecated `checkCanCreateTable` and `checkCanCreateMaterializedView` + methods not taking parameters. ({issue}`10939`) diff --git a/430/_sources/release/release-372.md.txt b/430/_sources/release/release-372.md.txt new file mode 100644 index 000000000..9fce1e7d5 --- /dev/null +++ b/430/_sources/release/release-372.md.txt @@ -0,0 +1,140 @@ +# Release 372 (2 Mar 2022) + +## General + +* Add {func}`trim_array` function. ({issue}`11238`) +* Improve handling of prepared statements with long query text by compressing + them within HTTP headers. This can be configured or disabled using the + ``protocol.v1.prepared-statement-compression.length-threshold`` configuration + property. ({issue}`11098`) +* Improve performance of specific queries which compare table columns of type + `timestamp` with `date` literals. ({issue}`11170`) +* Add redirection awareness for `ADD COLUMN`, `DROP TABLE`, `COMMENT` tasks. ({issue}`11072`) +* Remove support for reserved memory pool. Configuration property + `experimental.reserved-pool-disabled` can no longer be used. ({issue}`6677`) +* Ensure memory is released completely after query completion. ({issue}`11030`) +* Fix certain queries failing due to dictionary compacting error. ({issue}`11080`) +* Fix `SET SESSION` and `RESET SESSION` not working for catalogs which include + special characters in their name. ({issue}`11171`) +* Fix bug where queries were not transitioned to `RUNNING` state when task-level + retries were enabled. ({issue}`11198`) + +## Security + +* Allow configuration of connect and read timeouts for LDAP authentication. ({issue}`10925`) + +## Docker image + +* Add a health check to the Docker container image. ({issue}`10413`) + +## JDBC driver + +* Fix `DatabaseMetaData#getTables` and `DatabaseMetaData#getColumns` to include + views for Iceberg, Raptor, Accumulo and Blackhole connectors. ({issue}`11063`, {issue}`11060`) + +## Base-JDBC connector library + +* Fix spurious query failures when metadata cache is not enabled and data + access depends on the session state. ({issue}`11068`) + +## Accumulo connector + +* Fix incorrect results when querying `date` type columns. ({issue}`11055`) + +## Cassandra connector + +* Fix incorrect results when filtering partition keys without projections. + ({issue}`11001`) + +## ClickHouse connector + +* Fix spurious query failures when metadata cache is not enabled and extra + credentials with `user-credential-name` or `password-credential-name` are used + to access data. ({issue}`11068`) + +## Druid connector + +* Fix spurious query failures when metadata cache is not enabled and extra + credentials with `user-credential-name` or `password-credential-name` are used + to access data. ({issue}`11068`) + +## Hive connector + +* Allow redirects of Iceberg or Delta tables which have no ``StorageDescriptor`` + in the Glue metastore. ({issue}`11092`) +* Stop logging spurious failures when [storage caching](/connector/hive-caching) + is used. ({issue}`11101`) +* Allow reading Zstandard compressed Avro files. ({issue}`11090`) +* Fix incorrect query results after writes to a table when directory caching is + enabled enabled with the `hive.file-status-cache-tables` configuration + property. ({issue}`10621`) +* Fix potential query failures for queries writing data to tables backed by S3. + ({issue}`11089`) + +## Iceberg connector + +* Add support for ``COMMENT ON COLUMN`` statement. ({issue}`11143`) +* Improve query performance after table schema evolved, by fixing the connector + to support table stats in such case. ({issue}`11091`) +* Fix potential query failures for queries writing data to tables backed by S3. ({issue}`11089`) +* Prevent query failure from dereference pushdown when a column has a comment. ({issue}`11104`) + +## Kudu connector + +* Add support for Kerberos authentication. ({issue}`10953`) + +## MongoDB connector + +* Map MongoDB `bindata` type to Trino `varbinary` type if explicit schema does + not exist. ({issue}`11122`) + +## MySQL connector + +* Fix spurious query failures when metadata cache is not enabled and extra + credentials with `user-credential-name` or `password-credential-name` are used + to access data. ({issue}`11068`) + +## Oracle connector + +* Fix spurious query failures when metadata cache is not enabled and extra + credentials with `user-credential-name` or `password-credential-name` are used + to access data. ({issue}`11068`) + +## Phoenix connector + +* Fix query failures when applying predicates on `array(char)` type columns. ({issue}`10451`) +* Fix metadata listing failure in case of concurrent table deletion. ({issue}`10904`) + +## PostgreSQL connector + +* Add support for pushing down joins on character string type columns. ({issue}`10059`) +* Fix spurious query failures when metadata cache is not enabled and extra + credentials with `user-credential-name` or `password-credential-name` are used + to access data. ({issue}`11068`) + +## Redshift connector + +* Fix spurious query failures when metadata cache is not enabled and extra + credentials with `user-credential-name` or `password-credential-name` are used + to access data. ({issue}`11068`) + +## SingleStore (MemSQL) connector + +* Fix spurious query failures when metadata cache is not enabled, and extra + credentials with `user-credential-name` or `password-credential-name` are used + to access data. ({issue}`11068`) + +## SQL Server connector + +* Update JDBC driver to 10.2.0. The new version automatically enables TLS and + certificate verification. Update the [TLS configuration](sqlserver-tls) to + keep the old behavior. ({issue}`10898`) +* Fix spurious query failures when metadata cache is not enabled and extra + credentials with `user-credential-name` or `password-credential-name` are used + to access data. ({issue}`11068`) + +## SPI + +* Pass more information about predicates in `ConnectorMetadata#applyFilter` + invocation. The predicates that cannot be represented with a `TupleDomain` are + available via `Constraint.getExpression()`. ({issue}`7994`) diff --git a/430/_sources/release/release-373.md.txt b/430/_sources/release/release-373.md.txt new file mode 100644 index 000000000..5a0cbf6e7 --- /dev/null +++ b/430/_sources/release/release-373.md.txt @@ -0,0 +1,77 @@ +# Release 373 (9 Mar 2022) + +## General + +* Add {doc}`/connector/delta-lake`. ({issue}`11296`, {issue}`10897`) +* Improve query performance by reducing overhead of cluster internal + communication. ({issue}`11146`) +* Handle `varchar` to `timestamp` conversion errors in {func}`try`. ({issue}`11259`) +* Add redirection awareness for `DROP COLUMN` task. ({issue}`11304`) +* Add redirection awareness for `RENAME COLUMN` task. ({issue}`11226`) +* Disallow table redirections in `SHOW GRANTS` statement. ({issue}`11270`) +* Allow low memory killer to abort individual tasks when `retry-mode` is set to + `TASK`. This requires `query.low-memory-killer.policy` set to + `total-reservation-on-blocked-nodes`. ({issue}`11129`) +* Fix incorrect results when distinct or ordered aggregation are used and + spilling is enabled. ({issue}`11353`) + +## Web UI + +* Add CPU time, scheduled time, and cumulative memory statistics regarding + failed tasks in a query. ({issue}`10754`) + +## BigQuery connector + +* Allow configuring view expiration time via the `bigquery.view-expire-duration` + config property. ({issue}`11272`) + +## Elasticsearch connector + +* Improve performance of queries involving `LIKE` by pushing predicate + computation to the Elasticsearch cluster. ({issue}`7994`, {issue}`11308`) + +## Hive connector + +* Support access to S3 via a HTTP proxy. ({issue}`11255`) +* Improve query performance by better estimating partitioned tables statistics. ({issue}`11333`) +* Prevent failure for queries with the final number of partitions + below `HIVE_EXCEEDED_PARTITION_LIMIT`. ({issue}`10215`) +* Fix issue where duplicate rows could be inserted into a partition when + `insert_existing_partitions_behavior` was set to `OVERWRITE` and + `retry-policy` was `TASK`. ({issue}`11196`) +* Fix failure when querying Hive views containing column aliases that differ in + case only. ({issue}`11159`) + +## Iceberg connector + +* Support access to S3 via a HTTP proxy. ({issue}`11255`) +* Delete table data when dropping table. ({issue}`11062`) +* Fix `SHOW TABLES` failure when a materialized view is removed during query + execution. ({issue}`10976`) +* Fix query failure when reading from `information_schema.tables` or + `information_schema.columns` and a materialized view is removed during + query execution. ({issue}`10976`) + +## Oracle connector + +* Fix query failure when performing concurrent write operations. ({issue}`11318`) + +## Phoenix connector + +* Prevent writing incorrect results when arrays contain `null` values. ({issue}`11351`) + +## PostgreSQL connector + +* Improve performance of queries involving `LIKE` by pushing predicate + computation to the underlying database. ({issue}`11045`) + +## SQL Server connector + +* Fix incorrect results when querying SQL Server `tinyint` columns by mapping + them to Trino `smallint`. ({issue}`11209`) + +## SPI + +* Add CPU time, scheduled time, and cumulative memory statistics regarding + failed tasks in a query to query-completion events. ({issue}`10734`) + diff --git a/430/_sources/release/release-374.md.txt b/430/_sources/release/release-374.md.txt new file mode 100644 index 000000000..aabd0c877 --- /dev/null +++ b/430/_sources/release/release-374.md.txt @@ -0,0 +1,79 @@ +# Release 374 (17 Mar 2022) + +## General + +* Add support for query parameters in `CREATE SCHEMA`. ({issue}`11485`) +* Improve performance when reading from S3-based spool for + [fault-tolerant execution](/admin/fault-tolerant-execution). ({issue}`11050`) +* Improve performance of queries with `GROUP BY` clauses. ({issue}`11392`) +* Improve performance of `GROUP BY` with a large number of groups. ({issue}`11011`) +* Improve handling of queries where individual tasks require lots of memory when + `retry-policy` is set to `TASK`. ({issue}`10432`) +* Produce better query plans by improving cost-based-optimizer estimates in the + presence of correlated columns. ({issue}`11324`) +* Fix memory accounting and improve performance for queries involving certain + variable-width data types such as `varchar` or `varbinary`. ({issue}`11315`) +* Fix performance regression for `GROUP BY` queries. ({issue}`11234`) +* Fix `trim`, `ltrim` and `rtim` function results when the argument is `char` + type. Previously, it returned padded results as `char` type. It returns + `varchar` type without padding now. ({issue}`11440`) + +## JDBC driver + +* Add support for `DatabaseMetaData.getImportedKeys`. ({issue}`8708`) +* Fix `Driver.getPropertyInfo()`, and validate allowed properties. ({issue}`10624`) + +## CLI + +* Add support for selecting Vim or Emacs editing modes with the `--editing-mode` + command line argument. ({issue}`3377`) + +## Cassandra connector + +* Add support for [TRUNCATE TABLE](/sql/truncate). ({issue}`11425`) +* Fix incorrect query results for certain complex queries. ({issue}`11083`) + +## ClickHouse connector + +* Add support for `uint8`, `uint16`, `uint32` and `uint64` types. ({issue}`11490`) + +## Delta Lake connector + +* Allow specifying STS endpoint to be used when connecting to S3. ({issue}`10169`) +* Fix query failures due to exhausted file system resources after `DELETE` or + `UPDATE`. ({issue}`11418`) + +## Hive connector + +* Allow specifying STS endpoint to be used when connecting to S3. ({issue}`10169`) +* Fix shared metadata caching with Hive ACID tables. ({issue}`11443`) + +## Iceberg connector + +* Allow specifying STS endpoint to be used when connecting to S3. ({issue}`10169`) +* Add support for using Glue metastore as Iceberg catalog. ({issue}`10845`) + +## MongoDB connector + +* Add support for [`CREATE SCHEMA`](/sql/create-schema) and + [`DROP SCHEMA`](/sql/drop-schema). ({issue}`11409`) +* Add support for [`COMMENT ON TABLE`](/sql/comment). ({issue}`11424`) +* Add support for [`COMMENT ON COLUMN`](/sql/comment). ({issue}`11457`) +* Support storing a comment when adding new columns. ({issue}`11487`) + +## PostgreSQL connector + +* Improve performance of queries involving `OR` with simple comparisons and + `LIKE` predicates by pushing predicate computation to the PostgreSQL database. + ({issue}`11086`) +* Improve performance of aggregation queries with certain complex predicates by + computing predicates and aggregations within PostgreSQL. ({issue}`11083`) +* Fix possible connection leak when connecting to PostgreSQL failed. ({issue}`11449`) + +## SingleStore (MemSQL) connector + +* The connector now uses the official Single Store JDBC Driver. As a result, + `connection-url` in catalog configuration files needs to be updated from + `jdbc:mariadb:...` to `jdbc:singlestore:...`. ({issue}`10669`) +* Deprecate `memsql` as the connector name. We recommend using `singlestore` in + the `connector.name` configuration property. ({issue}`11459`) diff --git a/430/_sources/release/release-375.md.txt b/430/_sources/release/release-375.md.txt new file mode 100644 index 000000000..a9b14f8e0 --- /dev/null +++ b/430/_sources/release/release-375.md.txt @@ -0,0 +1,83 @@ +# Release 375 (28 Mar 2022) + +## General + +* Change behavior of `ALTER TABLE qualified_name TO unqualified_name` to leave + the table in the schema. This is backwards-incompatible behavioral change. ({issue}`11282`) +* Disallow table redirections for the `GRANT`, `REVOKE`, and + `SET TABLE AUTHORIZATION` table tasks. ({issue}`11302`) +* Improve performance of queries that contain filter predicates on aggregation + results. ({issue}`11469`) +* Improve performance of queries that contain `array_distinct(array_sort(…))` + expressions. ({issue}`8777`) +* Fix `SHOW CREATE TABLE` to show actual table name in case of table + redirections, so that the statement can be used to recreate the table. ({issue}`11604`) +* Fix scheduling for non-remotely accessible splits in fault-tolerant execution. ({issue}`11581`) +* Fix incorrect `is_nullable` value in `information_schema.columns` table. ({issue}`11613`) + +## JDBC driver + +* Allow `PreparedStatement.close()` to be called multiple times. ({issue}`11620`) +* Fix incorrect `is_nullable` value in `DatabaseMetaData.getColumns()` method. ({issue}`11613`) + +## Cassandra connector + +* Return `0` instead of row count in completed bytes. ({issue}`11644`) + +## Delta Lake connector + +* Add access control to `drop_extended_stats` and `vacuum` procedures. ({issue}`11633`) +* Fix incorrect query results when query executes concurrent with `DROP TABLE`. ({issue}`11562`) + +## Hive connector + +* Fix infinite loop in the query optimizer when query contains predicates on a + struct field. ({issue}`11559`) +* Fix query failure when reading a partitioned table with a predicate on a + partition column with a specific name such as `table`, `key`, `order`, and + others. ({issue}`11512`) + +## Iceberg connector + +* Fix failure when query contains predicates on a struct field. ({issue}`11560`) +* Fix query failure when reading from `$files` system table after a table column + has been dropped. ({issue}`11576`) + +## Kudu connector + +* Improve write performance by flushing operations in batches. ({issue}`11264`) +* Fix failure when multiple queries run concurrently and schema emulation is enabled. ({issue}`11264`) + +## MongoDB connector + +* Support predicate pushdown on `boolean` columns. ({issue}`11536`) +* Return `0` instead of row count in completed bytes. ({issue}`11679`) + +## MySQL connector + +* Add support for table comments. ({issue}`11211`) + +## Pinot connector + +* Fix handling of passthrough queries that contain aggregation functions. ({issue}`9137`) +* Fix incorrect results when aggregation functions on columns having + non-lowercase names are pushed down to Pinot. ({issue}`9137`, {issue}`10148`) +* Fix possible incorrect results when grouping on columns of array types. ({issue}`9781`) + +## PostgreSQL connector + +* Improve performance of queries involving `OR` with `IS NULL`, `IS NOT NULL` + predicates, or involving `NOT` expression by pushing predicate computation to + the PostgreSQL database. ({issue}`11514`) +* Improve performance of queries with predicates involving `nullif` function by + pushing predicate computation to the PostgreSQL database. ({issue}`11532`) +* Improve performance of queries involving joins by pushing computation to the + PostgreSQL database. ({issue}`11635`) +* Improve performance of queries involving predicates with arithmetic + expressions by pushing predicate computation to the PostgreSQL database. ({issue}`11510`) +* Fix deletion of too much data when delete query involves a `LIKE` predicate. ({issue}`11615`) + +## SPI + +* Add processed input bytes and rows to query events in event listener. ({issue}`11623`) +* Remove deprecated constructors from `ColumnMetadata`. ({issue}`11621`) diff --git a/430/_sources/release/release-376.md.txt b/430/_sources/release/release-376.md.txt new file mode 100644 index 000000000..382d45397 --- /dev/null +++ b/430/_sources/release/release-376.md.txt @@ -0,0 +1,97 @@ +# Release 376 (7 Apr 2022) + +## General + +* Add table redirection awareness for `RENAME table` operations. ({issue}`11277`) +* Deny adding column with comment if the connector does not support this feature. ({issue}`11486`) +* Improve performance for queries that contain inequality expressions. ({issue}`11518`) +* Consider null values as identical values in `array_except`, `array_union`, + `map_concat`, `map_from_entries`, `multimap_from_entries`, and `multimap_agg` + functions. ({issue}`560`) +* Fix failure of `DISTINCT .. LIMIT` operator when input data is dictionary + encoded. ({issue}`11776`) +* Fix returning of invalid results for distinct aggregation when input data is + dictionary encoded. ({issue}`11776`) +* Fix query failure when performing joins with connectors that support index lookups. ({issue}`11758`) +* Fix incorrect stage memory statistics reporting for queries running with + `retry-policy` set to `TASK`. ({issue}`11801`) + +## Security + +* Add support to use two-way TLS/SSL certificate validation with LDAP authentication. + Additionally `ldap.ssl-trust-certificate` config is replaced by `ldap.ssl.truststore.path`. ({issue}`11070`). +* Fix failures in information schema role tables for catalogs using system roles. ({issue}`11694`) + +## Web UI + +* Add new page to display the runtime information of all workers in the cluster. ({issue}`11653`) + +## JDBC driver + +* Add support for using the system truststore with the `SSLUseSystemTrustStore` parameter. ({issue}`10482`) +* Add support for `ResultSet.getAsciiStream()` and `ResultSet.getBinaryStream()`. ({issue}`11753`) +* Remove `user` property requirement. ({issue}`11350`) + +## CLI + +* Add support for using the system truststore with the `--use-system-truststore` option. ({issue}`10482`) + +## Accumulo connector + +* Add support for adding and dropping schemas. ({issue}`11808`) +* Disallow creating tables in a schema that doesn't exist. ({issue}`11808`) + +## ClickHouse connector + +* Add support for column comments when creating new tables. ({issue}`11606`) +* Add support for column comments when adding new columns. ({issue}`11606`) + +## Delta Lake connector + +* Add support for `INSERT`, `UPDATE`, and `DELETE` queries on Delta Lake tables + with fault-tolerant execution. ({issue}`11591`) +* Allow setting duration for completion of [dynamic filtering](/admin/dynamic-filtering) + with the `delta.dynamic-filtering.wait-timeout` configuration property. ({issue}`11600`) +* Improve query planning time after `ALTER TABLE ... EXECUTE optimize` by always + creating a transaction log checkpoint. ({issue}`11721`) +* Add support for reading Delta Lake tables in with auto-commit mode disabled. ({issue}`11792`) + +## Hive connector + +* Store file min/max ORC statistics for string columns even when actual min or + max value exceeds 64 bytes. This improves query performance when filtering on + such column. ({issue}`11652`) +* Improve performance when reading Parquet data. ({issue}`11675`) +* Improve query performance when the same table is referenced multiple times + within a query. ({issue}`11650`) + +## Iceberg connector + +* Add support for views when using Iceberg Glue catalog. ({issue}`11499`) +* Add support for reading Iceberg v2 tables containing deletion files. ({issue}`11642`) +* Add support for table redirections to the Hive connector. ({issue}`11356`) +* Include non-Iceberg tables when listing tables from Hive catalogs. ({issue}`11617`) +* Expose `nan_count` in the `$partitions` metadata table. ({issue}`10709`) +* Store file min/max ORC statistics for string columns even when actual min or + max value exceeds 64 bytes. This improves query performance when filtering on + such column. ({issue}`11652`) +* Improve performance when reading Parquet data. ({issue}`11675`) +* Fix NPE when an Iceberg data file is missing null count statistics. ({issue}`11832`) + +## Kudu connector + +* Add support for adding columns with comment. ({issue}`11486`) + +## MySQL connector + +* Improve performance of queries involving joins by pushing computation to the + MySQL database. ({issue}`11638`) + +## Oracle connector + +* Improve query performance of queries involving aggregation by pushing + aggregation computation to the Oracle database. ({issue}`11657`) + +## SPI + +* Add support for table procedures that execute on the coordinator only. ({issue}`11750`) diff --git a/430/_sources/release/release-377.md.txt b/430/_sources/release/release-377.md.txt new file mode 100644 index 000000000..00cdb47d1 --- /dev/null +++ b/430/_sources/release/release-377.md.txt @@ -0,0 +1,29 @@ +# Release 377 (13 Apr 2022) + +## General + +* Add support for standard SQL `trim` syntax. ({issue}`11236`) +* Fix incorrect results when queries execute in fault-tolerant mode. ({issue}`11870`) + +## Hive connector + +* Add support for `date` type partition names with timestamp formatting. ({issue}`11873`) +* Improve performance of queries that use Glue metadata. ({issue}`11869`) +* Fix failure of the `sync_partition_metadata` procedure when partition names + differ from partition paths on the file system. ({issue}`11864`) + +## Iceberg connector + +* Support setting Glue metastore catalog identifier with the + `hive.metastore.glue.catalogid` catalog configuration property. ({issue}`11520`) +* Add support for materialized views when using Glue metastore. ({issue}`11780`) + +## Kafka connector + +* Add support for additional Kafka client properties specified with the + `kafka.config.resources` catalog configuration property. ({issue}`8743`) + +## SQL Server connector + +* Improve performance of queries involving joins by pushing computation to the + SQL Server database. ({issue}`11637`) diff --git a/430/_sources/release/release-378.md.txt b/430/_sources/release/release-378.md.txt new file mode 100644 index 000000000..755783ba4 --- /dev/null +++ b/430/_sources/release/release-378.md.txt @@ -0,0 +1,70 @@ +# Release 378 (21 Apr 2022) + +## General + +* Add {func}`to_base32` and {func}`from_base32` functions. ({issue}`11439`) +* Improve planning performance of queries with large `IN` lists. + ({issue}`11902`, {issue}`11918`, {issue}`11956`) +* Improve performance of queries involving correlated `IN` or `EXISTS` + predicates. ({issue}`12047`) +* Fix reporting of total spilled bytes in JMX metrics. ({issue}`11983`) + +## Security + +* Require value for [the shared secret configuration for internal + communication](/security/internal-communication) when any authentication is + enabled. ({issue}`11944`) + +## CLI + +* Allow disabling progress reporting during query executing in the CLI client by + specifying `--no-progress` ({issue}`11894`) +* Reduce latency for very short queries. ({issue}`11768`) + +## Delta Lake connector + +* Improve query planning performance. ({issue}`11858`) +* Fix failure when reading from `information_schema.columns` when metastore + contains views. ({issue}`11946`) +* Add support for dropping tables with invalid metadata. ({issue}`11924`) +* Fix query failure when partition column has a `null` value and query has a + complex predicate on that partition column. ({issue}`12056`) + +## Hive connector + +* Improve query planning performance. ({issue}`11858`) + +## Iceberg connector + +* Add support for hidden `$path` columns. ({issue}`8769`) +* Add support for creating tables with either Iceberg format version 1, or 2. ({issue}`11880`) +* Add the `expire_snapshots` table procedure. ({issue}`10810`) +* Add the `delete_orphan_files` table procedure. ({issue}`10810`) +* Allow reading Iceberg tables written by Glue that have locations containing + double slashes. ({issue}`11964`) +* Improve query planning performance. ({issue}`11858`) +* Fix query failure with a dynamic filter prunes a split on a worker node. ({issue}`11976`) +* Include missing `format_version` property in `SHOW CREATE TABLE` output. ({issue}`11980`) + +## MySQL connector + +* Improve query planning performance. ({issue}`11858`) + +## Pinot connector + +* Support querying tables having non-lowercase names in Pinot. ({issue}`6789`) +* Fix handling of hybrid tables in Pinot and stop returning duplicate data. ({issue}`10125`) + +## PostgreSQL connector + +* Improve query planning performance. ({issue}`11858`) + +## SQL Server connector + +* Improve query planning performance. ({issue}`11858`) + +## SPI + +* Deprecate passing constraints to `ConnectorMetadata.getTableStatistics()`. + Constraints can be associated with the table handle in + `ConnectorMetadata.applyFilter()`. ({issue}`11877`) diff --git a/430/_sources/release/release-379.md.txt b/430/_sources/release/release-379.md.txt new file mode 100644 index 000000000..afc1ccd19 --- /dev/null +++ b/430/_sources/release/release-379.md.txt @@ -0,0 +1,48 @@ +# Release 379 (28 Apr 2022) + +## General + +* Add {doc}`/connector/mariadb`. ({issue}`10046`) +* Improve performance of queries that contain `JOIN` and `UNION` clauses. ({issue}`11935`) +* Improve performance of queries that contain `GROUP BY` clauses. ({issue}`12095`) +* Fail `DROP TABLE IF EXISTS` when deleted entity is not a table. Previously the + statement did not delete anything. ({issue}`11555`) +* Fail `DROP VIEW IF EXISTS` when deleted entity is not a view. Previously the + statement did not delete anything. ({issue}`11555`) +* Fail `DROP MATERIALIZED VIEW IF EXISTS` when deleted entity is not a + materialized view. Previously the statement did not delete anything. + ({issue}`11555`) + +## Web UI + +* Group information about tasks by stage. ({issue}`12099`) +* Show aggregated statistics for failed tasks of queries that are executed with + `retry-policy` set to `TASK`. ({issue}`12099`) +* Fix reporting of `physical input read time`. ({issue}`12135`) + +## Delta Lake connector + +* Add support for Google Cloud Storage. ({issue}`12144`) +* Fix failure when reading from `information_schema.columns` when non-Delta + tables are present in the metastore. ({issue}`12122`) + +## Iceberg connector + +* Add support for {doc}`/sql/delete` with arbitrary predicates. ({issue}`11886`) +* Improve compatibility when Glue storage properties are used. ({issue}`12164`) +* Prevent data loss when queries modify a table concurrently when Glue catalog + is used. ({issue}`11713`) +* Enable commit retries when conflicts occur writing a transaction to a Hive Metastore. ({issue}`12419`) +* Always return the number of deleted rows for {doc}`/sql/delete` statements. ({issue}`12055`) + +## Pinot connector + +* Add support for Pinot 0.10. ({issue}`11475`) + +## Redis connector + +* Improve performance when reading data from Redis. ({issue}`12108`) + +## SQL Server connector + +* Properly apply snapshot isolation to all connections when it is enabled. ({issue}`11662`) diff --git a/430/_sources/release/release-380.md.txt b/430/_sources/release/release-380.md.txt new file mode 100644 index 000000000..1b098ba52 --- /dev/null +++ b/430/_sources/release/release-380.md.txt @@ -0,0 +1,45 @@ +# Release 380 (6 May 2022) + +## General + +* Enable automatic [writer scaling](/admin/properties-writer-scaling) by default. ({issue}`10614`) +* Improve performance of joins involving comparisons with the `<`,`<=`, `>`,`>=` operators. ({issue}`12236`) + +## Cassandra connector + +* Add support for the v5 and v6 protocols. ({issue}`7729`) +* Removes support for v2 protocol. ({issue}`7729`) +* Make the `cassandra.load-policy.use-dc-aware` and `cassandra.load-policy.dc-aware.local-dc` + catalog configuration properties mandatory. ({issue}`7729`) + +## Hive connector + +* Support table redirections from Hive to Delta Lake. ({issue}`11550`) +* Allow configuring a default value for the `auto_purge` table property with the + `hive.auto-purge` catalog property. ({issue}`11749`) +* Allow configuration of the Hive views translation security semantics with the + `hive.hive-views.run-as-invoker` catalog configuration property. ({issue}`9227`) +* Rename catalog configuration property `hive.translate-hive-views` to + `hive.hive-views.enabled`. The former name is still accepted. ({issue}`12238`) +* Rename catalog configuration property `hive.legacy-hive-view-translation` + to `hive.hive-views.legacy-translation`. The former name is still accepted. ({issue}`12238`) +* Rename session property `legacy_hive_view_translation` to + `hive_views_legacy_translation`. ({issue}`12238`) + +## Iceberg connector + +* Allow updating tables from the Iceberg v1 table format to v2 with + `ALTER TABLE ... SET PROPERTIES`. ({issue}`12161`) +* Allow changing the default [file format](iceberg-table-properties) for a table + with `ALTER TABLE ... SET PROPERTIES`. ({issue}`12161`) +* Prevent potential corruption when a table change is interrupted by networking + or timeout failures. ({issue}`10462`) + +## MongoDB connector + +* Add support for [`ALTER TABLE ... RENAME TO ...`](/sql/alter-table). ({issue}`11423`) +* Fix failure when reading decimal values with precision larger than 18. ({issue}`12205`) + +## SQL Server connector + +* Add support for bulk data insertion. ({issue}`12176`) diff --git a/430/_sources/release/release-381.md.txt b/430/_sources/release/release-381.md.txt new file mode 100644 index 000000000..b07753698 --- /dev/null +++ b/430/_sources/release/release-381.md.txt @@ -0,0 +1,115 @@ +# Release 381 (16 May 2022) + +## General + +* Add support for fault-tolerant execution with exchange spooling on Azure Blob Storage. ({issue}`12211`) +* Add experimental support for {doc}`/functions/table`. ({issue}`1839`) +* Increase the default number of stages allowed for a query from 100 to 150, + specified with `query.max-stage-count`. ({issue}`12292`) +* Allow configuring the number of partitions for distributed joins and + aggregations when task-based fault-tolerant execution is enabled. This can be + set with the `fault-tolerant-execution-partition-count` configuration property + or the `fault_tolerant_execution_partition_count` session property. ({issue}`12263`) +* Introduce the `least-waste` low memory task killer policy. This policy avoids + killing tasks that are already executing for a long time, so the amount of + wasted work is minimized. It can be enabled with the + `task.low-memory-killer.policy` configuration property. ({issue}`12393`) +* Fix potential planning failure of queries with multiple subqueries. ({issue}`12199`) + +## Security + +* Add support for automatic discovery of OpenID Connect metadata with OAuth 2.0 + authentication. ({issue}`9788`) +* Re-introduce `ldap.ssl-trust-certificate` as legacy configuration to avoid + failures when updating Trino version. ({issue}`12187`) +* Fix potential query failure when a table has multiple column masks defined. ({issue}`12262`) +* Fix incorrect masking of columns when multiple rules in file-based system and + connector access controls match. ({issue}`12203`) +* Fix authentication failure when using the LDAP password authenticator with + ActiveDirectory. ({issue}`12321`) + +## Web UI + +* Ensure consistent sort order in the list of workers. ({issue}`12290`) + +## Docker image + +* Improve Advanced Encryption Standard (AES) processing performance on ARM64 + processors. This is used for operations such as accessing object storage + systems via TLS/SSL. ({issue}`12251`) + +## CLI + +* Add automatic suggestions from command history. This can be disabled with the + `--disable-auto-suggestion` option. ({issue}`11671`) + +## BigQuery connector + +* Support reading materialized views. ({issue}`12352`) +* Allow skipping view materialization via `bigquery.skip-view-materialization` + configuration property. ({issue}`12210`) +* Support reading snapshot tables. ({issue}`12380`) + +## ClickHouse connector + +* Add support for [`COMMENT ON TABLE`](/sql/comment). ({issue}`11216`) +* Prevent data loss when non-transactional insert fails. ({issue}`12229`) + +## Druid connector + +* Prevent data loss when non-transactional insert fails. ({issue}`12229`) + +## Elasticsearch connector + +* Improve query performance by simplifying filters sent to Elasticsearch. ({issue}`10717`) +* Fix failure when reading nested timestamp values that are not ISO 8601 formatted. ({issue}`12250`) + +## Hive connector + +* Fix query failure when the table and partition bucket counts do not match. ({issue}`11885`) + +## Iceberg connector + +* Add support for {doc}`/sql/update`. ({issue}`12026`) +* Fix potential query failure or incorrect results when reading data from an + Iceberg table that contains + [equality delete files](https://iceberg.apache.org/spec/#equality-delete-files). ({issue}`12026`) + +## MariaDB connector + +* Prevent data loss when non-transactional insert fails. ({issue}`12229`) + +## MySQL connector + +* Prevent data loss when non-transactional insert fails. ({issue}`12229`) + +## Oracle connector + +* Prevent data loss when non-transactional insert fails. ({issue}`12229`) + +## PostgreSQL connector + +* Prevent data loss when non-transactional insert fails. ({issue}`12225`) + +## Redis connector + +* Allow specifying the refresh interval for fetching the table description with + the `redis.table-description-cache-ttl` configuration property. ({issue}`12240`) +* Support setting username for the connection with the `redis.user` + configuration property. ({issue}`12279`) + +## Redshift connector + +* Prevent data loss when non-transactional insert fails. ({issue}`12229`) + +## SingleStore (MemSQL) connector + +* Prevent data loss when non-transactional insert fails. ({issue}`12229`) + +## SQL Server connector + +* Prevent data loss when non-transactional insert fails. ({issue}`12229`) + +## SPI + +* Remove deprecated `ConnectorMetadata` methods without the retry mode parameter. ({issue}`12342`) diff --git a/430/_sources/release/release-382.md.txt b/430/_sources/release/release-382.md.txt new file mode 100644 index 000000000..03fdd3846 --- /dev/null +++ b/430/_sources/release/release-382.md.txt @@ -0,0 +1,60 @@ +# Release 382 (25 May 2022) + +## General + +* Add support for fault-tolerant execution with [exchange spooling on Google Cloud Storage](fte-exchange-gcs). ({issue}`12360`) +* Drop support for exchange spooling on S3 with for the legacy schemes `s3n://` and `s3a://`. ({issue}`12360`) +* Improve join performance when one side of the join is small. ({issue}`12257`) +* Fix potential query failures due to `EXCEEDED_TASK_DESCRIPTOR_STORAGE_CAPACITY` + errors with task-based fault-tolerant execution. ({issue}`12478`) + +## BigQuery connector + +* Add support for [using BigQuery's cached query results](https://cloud.google.com/bigquery/docs/cached-results). + This can be enabled using the `bigquery.query-results-cache.enabled` configuration property. ({issue}`12408`) +* Support reading wildcard tables. ({issue}`4124`) + +## Delta Lake connector + +* Improve performance of queries that include filters on columns of `timestamp with time zone` type. ({issue}`12007`) +* Add support for adding columns with `ALTER TABLE`. ({issue}`12371`) + +## Hive connector + +* Add support for disabling partition caching in the Hive metastore with the + `hive.metastore-cache.cache-partitions` catalog configuration property. ({issue}`12343`) +* Fix potential query failure when metastore caching is enabled. ({issue}`12513`) +* Fix query failure when a transactional table contains a column named + `operation`, `originalTransaction`, `bucket`, `rowId`, `row`, or + `currentTransaction`. ({issue}`12401`) +* Fix `sync_partition_metadata` procedure failure when table has a large number of partitions. ({issue}`12525`) + +## Iceberg connector + +* Support updating Iceberg table partitioning using `ALTER TABLE ... SET PROPERTIES`. ({issue}`12174`) +* Improves the performance of queries using equality and `IN` predicates when + reading ORC data that contains Bloom filters. ({issue}`11732`) +* Rename the `delete_orphan_files` table procedure to `remove_orphan_files`. ({issue}`12468`) +* Improve query performance of reads after `DELETE` removes all rows from a file. ({issue}`12197`) + +## MySQL connector + +* Improve `INSERT` performance. ({issue}`12411`) + +## Oracle connector + +* Improve `INSERT` performance when data includes `NULL` values. ({issue}`12400`) + +## PostgreSQL connector + +* Improve `INSERT` performance. ({issue}`12417`) + +## Prometheus connector + +* Add support for Basic authentication. ({issue}`12302`) + +## SPI + +* Change `ConnectorTableFunction` into an interface and add + `AbstractConnectorTableFunction` class as the base implementation of table + functions. ({issue}`12531`) diff --git a/430/_sources/release/release-383.md.txt b/430/_sources/release/release-383.md.txt new file mode 100644 index 000000000..a07899298 --- /dev/null +++ b/430/_sources/release/release-383.md.txt @@ -0,0 +1,123 @@ +# Release 383 (1 Jun 2022) + +```{warning} +This release has a regression that may cause queries to fail. +``` + +## General + +* Introduce `json_exists`, `json_query`, and `json_value` [JSON functions](/functions/json). ({issue}`9081`) +* Add AWS IAM role support for exchange spooling on S3. ({issue}`12444`) +* Improve query performance by reducing worker-to-worker communication overhead. ({issue}`11289`) +* Improve performance and reduce memory usage of queries that contain aggregations. ({issue}`12336`) +* Improve performance of correlated queries involving distinct aggregations. ({issue}`12564`) + +## Web UI + +* Clarify format of cumulative user memory on query details page. ({issue}`12596`) + +## Accumulo connector + +* Fail creating a new table if a table comment is specified. Previously, the + comment was ignored. ({issue}`12452`) + +## BigQuery connector + +* Fail creating a new table if a table comment or a column comment is specified. + Previously, the comment was ignored. ({issue}`12452`, {issue}`12574`) + +## Cassandra connector + +* Fail creating a new table if a table comment or a column comment is specified. + Previously, the comment was ignored. ({issue}`12452`, {issue}`12574`) + +## ClickHouse connector + +* Fix incorrect results for certain aggregation queries when aggregations are + pushed down to the underlying database. ({issue}`12598`) + +## Delta Lake connector + +* Add support for table comments during table creation.. ({issue}`12452`) +* Fix incorrect `table already exists` error caused by a client timeout when + creating a new table. ({issue}`12300`) +* Fail creating a new table if a column comment is specified. Previously, the + comment was ignored. ({issue}`12574`) + +## Iceberg connector + +* Add support for v2 tables for the `optimize` table procedure. ({issue}`12351`) +* Rename `hive.target-max-file-size` to `iceberg.target-max-file-size` and + `hive.delete-schema-locations-fallback` to `iceberg.delete-schema-locations-fallback`. ({issue}`12330`) + +## Kudu connector + +* Fail creating a new table if a table comment or a column comment is specified. + Previously, the comment was ignored. ({issue}`12452`, {issue}`12574`) + +## MariaDB connector + +* Fix incorrect results for certain queries involving aggregations that are + pushed down to the underlying database. ({issue}`12598`) +* Fail creating a new table if a column comment is specified. Previously, the + comment was ignored. ({issue}`12574`) + +## Memory connector + +* Fail creating a new table if a table comment or a column comment is specified. + Previously, the comment was ignored. ({issue}`12452`, {issue}`12574`) + +## MySQL connector + +* Fix incorrect results for certain aggregation queries when aggregations are + pushed down to the underlying database. ({issue}`12598`) +* Fail creating a new table if a column comment is specified. Previously, the + comment was ignored. ({issue}`12574`) + +## Oracle connector + +* Fail creating a new table if a table comment or a column comment is specified. + Previously, the comment was ignored. ({issue}`12452`, {issue}`12574`) +* Fix incorrect results for certain aggregation queries when aggregations are + pushed down to the underlying database. ({issue}`12598`) + +## Phoenix connector + +* Fail creating a new table if a table comment or a column comment is specified. + Previously, the comment was ignored. ({issue}`12452`, {issue}`12574`) + +## PostgreSQL connector + +* Fail creating a new table if a table comment or a column comment is specified. + Previously, the comment was ignored. ({issue}`12452`, {issue}`12574`) +* Fix incorrect results for certain aggregation queries when aggregations are + pushed down to the underlying database. ({issue}`12598`) + +## Raptor connector + +* Fail creating a new table if a table comment or a column comment is specified. + Previously, the comment was ignored. ({issue}`12452`, {issue}`12574`) + +## Redshift connector + +* Fail creating a new table if a table comment or a column comment is specified. + Previously, the comment was ignored. ({issue}`12452`, {issue}`12574`) + +## SingleStore (MemSQL) connector + +* Fail creating a new table if a table comment or a column comment is specified. + Previously, the comment was ignored. ({issue}`12452`, {issue}`12574`) +* Fix incorrect results for certain aggregation queries when aggregations are + pushed down to the underlying database. ({issue}`12598`) + +## SQL Server connector + +* Fail creating a new table if a table comment or a column comment is specified. + Previously, the comment was ignored. ({issue}`12452`, {issue}`12574`) +* Fix incorrect results for certain aggregation queries when aggregations are + pushed down to the underlying database. ({issue}`12598`) + +## SPI + +* Allow limiting access to functions based on whether they are scalar, + aggregation, window, or table functions. ({issue}`12544`) diff --git a/430/_sources/release/release-384.md.txt b/430/_sources/release/release-384.md.txt new file mode 100644 index 000000000..cb5f40653 --- /dev/null +++ b/430/_sources/release/release-384.md.txt @@ -0,0 +1,38 @@ +# Release 384 (3 Jun 2022) + +## General + +* Fix potential query failure due to a regression introduced in version 383. ({issue}`12667`) +* Fix potential failure or incorrect result of the `json_query` [JSON function](/functions/json). ({issue}`12681`) + +## Druid connector + +* Add `query` table function for full query pass-through to the connector. ({issue}`9163`) + +## MariaDB connector + +* Add `query` table function for full query pass-through to the connector. ({issue}`9163`) + +## MySQL connector + +* Add `query` table function for full query pass-through to the connector. ({issue}`9163`) + +## Oracle connector + +* Add `query` table function for full query pass-through to the connector. ({issue}`9163`) + +## PostgreSQL connector + +* Add `query` table function for full query pass-through to the connector. ({issue}`9163`) + +## Redshift connector + +* Add `query` table function for full query pass-through to the connector. ({issue}`9163`) + +## SingleStore (MemSQL) connector + +* Add `query` table function for full query pass-through to the connector. ({issue}`9163`) + +## SQL Server connector + +* Add `query` table function for full query pass-through to the connector. ({issue}`9163`) diff --git a/430/_sources/release/release-385.md.txt b/430/_sources/release/release-385.md.txt new file mode 100644 index 000000000..75a940bcb --- /dev/null +++ b/430/_sources/release/release-385.md.txt @@ -0,0 +1,38 @@ +# Release 385 (8 Jun 2022) + +## General + +* Add the `json_array` and `json_object` [JSON functions](/functions/json). ({issue}`9081`) +* Support all types that can be cast to `varchar` as parameters for the [JSON + path](json-path-language). ({issue}`12682`) +* Allow `CREATE TABLE LIKE` clause on a table from a different catalog if + explicitly excluding table properties. ({issue}`3171`) +* Reduce `Exceeded limit of N open writers for partitions` errors when + fault-tolerant execution is enabled. ({issue}`12721`) + +## Delta Lake connector + +* Add support for the [appendOnly field](https://docs.delta.io/latest/delta-batch.html#-table-properties). ({issue}`12635`) +* Add support for column comments when creating a table or a column. ({issue}`12455`, {issue}`12715`) + +## Hive connector + +* Allow cancelling a query on a transactional table if it is waiting for a lock. ({issue}`11798`) +* Add support for selecting a compression scheme when writing Avro files via the + `hive.compression-codec` config property or the `compression_codec` session + property. ({issue}`12639`) + +## Iceberg connector + +* Improve query performance when a table consists of many small files. ({issue}`12579`) +* Improve query performance when performing a delete or update. ({issue}`12671`) +* Add support for the `[VERSION | TIMESTAMP] AS OF` clause. ({issue}`10258`) +* Show Iceberg location and `format_version` in `SHOW CREATE MATERIALIZED VIEW`. ({issue}`12504`) + +## MariaDB connector + +* Add support for `timestamp(p)` type. ({issue}`12200`) + +## TPC-H connector + +* Fix query failure when reading the `dbgen_version` table. ({issue}`12673`) diff --git a/430/_sources/release/release-386.md.txt b/430/_sources/release/release-386.md.txt new file mode 100644 index 000000000..50645ff16 --- /dev/null +++ b/430/_sources/release/release-386.md.txt @@ -0,0 +1,66 @@ +# Release 386 (15 Jun 2022) + +## General + +* Improve out-of-the-box performance of queries when `task` retry policy is + enabled. ({issue}`12646`) +* Improve query latency when `task` retry policy is enabled. ({issue}`12615`) + +## JDBC driver + +* Add configuration property + `assumeLiteralUnderscoreInMetadataCallsForNonConformingClients` for situations + where applications do not properly escape schema or table names in calls to + `DatabaseMetaData`. ({issue}`12672`) + +## Accumulo connector + +* Disallow creating a view in a non-existent schema. ({issue}`12475`) + +## Delta Lake connector + +* Improve query performance on tables with many small files. ({issue}`12755`) +* Disallow reading tables if `delta.columnMapping.mode` table property is + specified. ({issue}`12621`) +* Set a target maximum file size during table writes. The default is 1 GB and + can be configured with the `target_max_file_size` session property or the + `target-max-file-size` configuration property. ({issue}`12820`) + +## Hive connector + +* Fix incompatibility with Apache Hive when writing decimal values with + precision of 18 or less with the experimental Parquet writer. ({issue}`12658`) +* Fix potential query failure when using schema evolution with union-typed + columns. ({issue}`12520`) +* Fix potential query failure when reading `timestamp(6) with time zone` values. ({issue}`12804`) + +## Iceberg connector + +* Disallow creating a table with a pre-existing destination location. ({issue}`12573`) +* Fix `NoClassDefFoundError` query failure when using Google Cloud Storage. ({issue}`12674`) +* Fix `ClassNotFoundException: Class io.trino.plugin.hive.s3.TrinoS3FileSystem` + error when querying `information_schema.columns`. ({issue}`12676`) +* Avoid creating a table snapshot when a write statement does not change the + table state. ({issue}`12319`) +* Fix incorrect query results when filtering on `$path` synthetic column and on + at least one other column. ({issue}`12790`) +* Fix potential query failure when reading `timestamp(6) with time zone` values. ({issue}`12804`) +* Fix query failure when using the `[VERSION | TIMESTAMP] AS OF` clause on a + table with redirection. ({issue}`12542`) +* Fix query failure when reading a `timestamp(p) with time zone` value before + 1970 from a Parquet file. ({issue}`12852`) + +## Kafka connector + +* Fix failure when decoding a `float` value to `real` type. ({issue}`12784`) + +## Phoenix connector + +* Remove support for Phoenix 4. ({issue}`12772`) + +## SPI + +* Add new version of `getStatisticsCollectionMetadata()` to `ConnectorMetadata` + which returns `ConnectorAnalyzeMetadata`. Deprecate the existing method and + `getTableHandleForStatisticsCollection()`. ({issue}`12388`) +* Remove deprecated `ConnectorMetadata.getTableStatistics` method. ({issue}`12489`) diff --git a/430/_sources/release/release-387.md.txt b/430/_sources/release/release-387.md.txt new file mode 100644 index 000000000..2d34d27b5 --- /dev/null +++ b/430/_sources/release/release-387.md.txt @@ -0,0 +1,56 @@ +# Release 387 (22 Jun 2022) + +## General + +* Add support for query parameters in table function arguments. ({issue}`12910`) +* Update minimum required Java version to 11.0.15. ({issue}`12841`) +* Fix incorrect result for {func}`to_iso8601` when the timestamp is in the + daylight savings transition region. ({issue}`11619`) + +## CLI + +* Fix query history not being stored when a query starts with whitespace. ({issue}`12847`) + +## Delta Lake connector + +* Record table size when analyzing a table. ({issue}`12814`) +* Enable the optimized Parquet writer by default. This can be disabled via the + `parquet.experimental-optimized-writer.enabled` configuration property. ({issue}`12757`) +* Disallow adding a new column to a table that has been written with an + unsupported writer. ({issue}`12883`) + +## Hive connector + +* Add support for ORC bloom filters on `varchar` columns. ({issue}`11757`) + +## Iceberg connector + +* Allow `OPTIMIZE` on a table partitioned on a `timestamp with time zone` column + when using `CAST(timestamp_col AS date) >= DATE '...'` syntax. ({issue}`12362`) +* Allow `OPTIMIZE` with a predicate on a table that does not have identity + partitioning. ({issue}`12795`) +* Improve performance of `DELETE` when deleting whole partitions from a table + that does not have identity partitioning. ({issue}`7905`) +* Fix incorrect results when a query contains a filter on a `UUID` column. ({issue}`12834`) +* Fail queries that attempt to modify old snapshots. ({issue}`12860`) +* Deprecate using synthetic `@`-based syntax for Iceberg snapshot access in + favor of the `AS OF` syntax. The old behavior can be restored by setting the + `allow_legacy_snapshot_syntax` session property or + `iceberg.allow-legacy-snapshot-syntax` configuration property. ({issue}`10768`) + +## Kudu connector + +* Fix failure when inserting into a table with a `row_uuid` column. ({issue}`12915`) + +## Pinot connector + +* Add support for querying Pinot via the gRPC endpoint. ({issue}`9296 `) + +## Redis connector + +* Add support for predicate pushdown on columns of type `string`. ({issue}`12218`) + +## SPI + +* Add information about query retry policy to `QueryCompletedEvent` and + `QueryCreatedEvent`. ({issue}`12898`) diff --git a/430/_sources/release/release-388.md.txt b/430/_sources/release/release-388.md.txt new file mode 100644 index 000000000..04c82f40b --- /dev/null +++ b/430/_sources/release/release-388.md.txt @@ -0,0 +1,49 @@ +# Release 388 (29 Jun 2022) + +## General + +* Add support for `EXPLAIN (TYPE LOGICAL, FORMAT JSON)`. ({issue}`12694`) +* Add `use_exact_partitioning` session property to re-partition data when the + upstream stage's partitioning does not exactly match what the downstream stage + expects. ({issue}`12495`) +* Improve read performance for `row` data types. ({issue}`12926`) +* Remove the grouped execution mechanism, including the + `grouped-execution-enabled`, `dynamic-schedule-for-grouped-execution`, + and `concurrent-lifespans-per-task` configuration properties and the + `grouped_execution`, `dynamic_schedule_for_grouped_execution`, and + `concurrent_lifespans_per_task` session properties. ({issue}`12916`) + +## Security + +* Add [refresh token](https://oauth.net/2/refresh-tokens/) support in OAuth 2.0. ({issue}`12664`) + +## Delta Lake connector + +* Add support for setting table and column comments with the `COMMENT` + statement. ({issue}`12971`) +* Support reading tables with the property `delta.columnMapping.mode=name`. ({issue}`12675`) +* Allow renaming tables with an explicitly set location. ({issue}`11400`) + +## Elasticsearch connector + +* Remove support for Elasticsearch versions below 6.6.0. ({issue}`11263`) + +## Hive connector + +* Improve performance of listing files and generating splits when recursive + directory listings are enabled and tables are stored in S3. ({issue}`12443`) +* Fix incompatibility that prevents Apache Hive 3 and older from reading + timestamp columns in files produced by Trino's optimized Parquet + writer. ({issue}`12857 `) +* Prevent reading from a table that was modified within the same Trino + transaction. Previously, this returned incorrect query results. ({issue}`11769`) + +## Iceberg connector + +* Add support for reading `tinyint` columns from ORC files. ({issue}`8919`) +* Add the ability to configure the schema for materialized view storage tables. ({issue}`12591`) +* Remove old deletion-tracking files when running `optimize`. ({issue}`12617`) +* Fix failure when invoking the `rollback_to_snapshot` procedure. ({issue}`12887`) +* Fix query failure when reading the `$partitions` table after table + partitioning changed. ({issue}`12874`) + diff --git a/430/_sources/release/release-389.md.txt b/430/_sources/release/release-389.md.txt new file mode 100644 index 000000000..59418811d --- /dev/null +++ b/430/_sources/release/release-389.md.txt @@ -0,0 +1,53 @@ +# Release 389 (7 Jul 2022) + +## General + +* Improve performance of queries involving `row` type or certain aggregations + such as `sum`, `avg`, etc. ({issue}`12762`) +* Improve performance when spilling to disk is disabled. ({issue}`12618`) +* Fix potential incorrect results for queries involving non-equality filters on + top of an outer join. ({issue}`13109`) +* Fix query failure when no arguments are passed to a table function. ({issue}`12951`) +* Fix potential failure when using `EXPLAIN` with queries involving table + functions. ({issue}`13106`) +* Fix potential resource leaks when calling long-running regular expression + functions. ({issue}`13064`) + +## Delta Lake connector + +* Improve optimized Parquet writer performance for + [non-structural data types](structural-data-types). ({issue}`13030`) +* Prevent failure when starting the server if the internal table snapshots cache + is disabled. ({issue}`13086`) + +## Elasticsearch connector + +* Add `raw_query` table function for full query pass-through to the connector. ({issue}`12324`) + +## Hive connector + +* Improve optimized Parquet writer performance for + [non-structural data types](structural-data-types). ({issue}`13030`) + +## Iceberg connector + +* Improve performance when writing Parquet files with + [non-structural data types](structural-data-types). ({issue}`13030`) + +## MongoDB connector + +* Create a collection when creating a new table. Previously, it was created when + the data was written to the table for the first time. ({issue}`12892`) + +## Phoenix connector + +* Add support for Java 17. ({issue}`13108`) + +## PostgreSQL connector + +* Prevent creating a new table with a name longer than the max length. + Previously, the name was truncated to the max length. ({issue}`12892`) + +## SPI + +* Remove deprecated version of `ConnectorRecordSetProvider#getRecordSet`. ({issue}`13084`) diff --git a/430/_sources/release/release-390.md.txt b/430/_sources/release/release-390.md.txt new file mode 100644 index 000000000..7460a5b50 --- /dev/null +++ b/430/_sources/release/release-390.md.txt @@ -0,0 +1,54 @@ +# Release 390 (13 Jul 2022) + +## General + +* Update minimum required Java version to 17.0.3. ({issue}`13014`) +* Add support for [setting comments on views](/sql/comment). ({issue}`8349`) +* Improve performance of queries with an `UNNEST` clause. ({issue}`10506`) +* Fix potential query failure when spilling to disk is enabled by the + `force-spilling-join-operator` configuration property or the + `force_spilling_join` session property. ({issue}`13123`) +* Fix incorrect results for certain join queries containing filters involving + explicit or implicit casts. ({issue}`13145 `) + +## Cassandra connector + +* Change mapping for Cassandra `inet` type to Trino `ipaddress` type. + Previously, `inet` was mapped to `varchar`. ({issue}`851`) +* Remove support for the + `cassandra.load-policy.use-token-aware`, + `cassandra.load-policy.shuffle-replicas`, and + `cassandra.load-policy.allowed-addresses` configuration properties. ({issue}`12223`) + +## Delta Lake connector + +* Add support for filtering splits based on `$path` column predicates. ({issue}`13169`) +* Add support for Databricks runtime 10.4 LTS. ({issue}`13081`) +* Expose AWS Glue metastore statistics via JMX. ({issue}`13087`) +* Fix failure when using the Glue metastore and queries contain `IS NULL` or + `IS NOT NULL` filters on numeric partition columns. ({issue}`13124`) + +## Hive connector + +* Expose AWS Glue metastore statistics via JMX. ({issue}`13087`) +* Add support for [setting comments on views](/sql/comment). ({issue}`13147`) +* Fix failure when using the Glue metastore and queries contain `IS NULL` or + `IS NOT NULL` filters on numeric partition columns. ({issue}`13124`) +* Fix and re-enable usage of Amazon S3 Select for uncompressed files. ({issue}`12633`) + +## Iceberg connector + +* Add `added_rows_count`, `existing_rows_count`, and `deleted_rows_count` + columns to the `$manifests` table. ({issue}`10809`) +* Add support for [setting comments on views](/sql/comment). ({issue}`13147`) +* Expose AWS Glue metastore statistics via JMX. ({issue}`13087`) +* Fix failure when using the Glue metastore and queries contain `IS NULL` or + `IS NOT NULL` filters on numeric partition columns. ({issue}`13124`) + +## Memory connector + +* Add support for [setting comments on views](/sql/comment). ({issue}`8349`) + +## Prometheus connector + +* Fix failure when reading a table without specifying a `labels` column. ({issue}`12510`) diff --git a/430/_sources/release/release-391.md.txt b/430/_sources/release/release-391.md.txt new file mode 100644 index 000000000..7c5fa7756 --- /dev/null +++ b/430/_sources/release/release-391.md.txt @@ -0,0 +1,82 @@ +# Release 391 (22 Jul 2022) + +## General + +* Improve performance of repeated aggregations with ``CASE`` expressions. ({issue}`12548`) +* Improve query latency when there is high concurrency. ({issue}`13213`) +* Improve planning performance for join queries when tables have statistics. ({issue}`13047`) +* Fail queries that get stuck in long-running regular expression functions. ({issue}`12392`) +* Fix potential query failure when the ``UUID`` type is used. ({issue}`13265`) +* Set the default value of the ``optimizer.force-single-node-output`` + configuration property to false. ({issue}`13217`) + +## BigQuery connector + +* Add support for reading external tables. ({issue}`13164`) +* Add support for specifying table and column comments when creating a table. ({issue}`13105`) + +## Delta Lake connector + +* Improve optimized Parquet writer performance. ({issue}`13203`, {issue}`13208`) +* Store query ID when creating a new schema. ({issue}`13242`) +* Fix incorrect `schema already exists` error caused by a client timeout when + creating a new schema. ({issue}`13242`) +* Fix incorrect query results when reading a table with an outdated cached + representation of its active data files. ({issue}`13181`) + +## Druid connector + +* Fix potential query failure when using the ``query`` table function with + metadata caching and the underlying table schema is changed via Trino. ({issue}`12526`) + +## Hive connector + +* Add support for [AWS Athena partition projection](partition-projection). ({issue}`11305`) +* Improve optimized Parquet writer performance. ({issue}`13203`, {issue}`13208`) +* Fix potential failure when creating empty ORC bucket files while using ZSTD + compression. ({issue}`9775`) +* Fix query failure or potentially incorrect statistics when running concurrent + `CREATE TABLE AS` queries with the `IF NOT EXISTS` clause for the same + non-existent table. ({issue}`12895`) +* Fix incorrect results when using the Glue metastore with queries that contain + `IS NULL` and additional filters. ({issue}`13122`) + +## Iceberg connector + +* Improve performance when writing Parquet writer data. ({issue}`13203`, {issue}`13208`) +* Fix query failure when reading an Iceberg table with deletion-tracking files. ({issue}`13035`) + +## MariaDB connector + +* Fix potential query failure when using the ``query`` table function with + metadata caching and the underlying table schema is changed via Trino. ({issue}`12526`) + +## MySQL connector + +* Fix potential query failure when using the ``query`` table function with + metadata caching and the underlying table schema is changed via Trino. ({issue}`12526`) + +## Oracle connector + +* Fix potential query failure when using the ``query`` table function with + metadata caching and the underlying table schema is changed via Trino. ({issue}`12526`) + +## PostgreSQL connector + +* Fix potential query failure when using the ``query`` table function with + metadata caching and the underlying table schema is changed via Trino. ({issue}`12526`) + +## Redshift connector + +* Fix potential query failure when using the ``query`` table function with + metadata caching and the underlying table schema is changed via Trino. ({issue}`12526`) + +## SQL Server connector + +* Fix potential query failure when using the ``query`` table function with + metadata caching and the underlying table schema is changed via Trino. ({issue}`12526`) + +## SPI + +* Removed deprecated methods and classes related to the grouped execution + feature. ({issue}`13125`) diff --git a/430/_sources/release/release-392.md.txt b/430/_sources/release/release-392.md.txt new file mode 100644 index 000000000..48ef3c7f8 --- /dev/null +++ b/430/_sources/release/release-392.md.txt @@ -0,0 +1,105 @@ +# Release 392 (3 Aug 2022) + +## General + +* Add support for dynamic filtering when task-based fault-tolerant execution is enabled. ({issue}`9935`) +* Add support for correlated sub-queries in `DELETE` queries. ({issue}`9447`) +* Fix potential query failure in certain complex queries with multiple joins and + aggregations. ({issue}`13315`) + +## JDBC driver + +* Add the `assumeLiteralUnderscoreInMetadataCallsForNonConformingClients` + configuration property as a replacement for + `assumeLiteralNamesInMetadataCallsForNonConformingClients`, which is + deprecated and planned to be removed in a future release. ({issue}`12761`) + +## ClickHouse connector + +* Report the total time spent reading data from the data source. ({issue}`13132`) + +## Delta Lake connector + +* Add support for using a randomized location when creating a table, so that + future table renames or drops do not interfere with new tables created with + the same name. This can be disabled by setting the + `delta.unique-table-location` configuration property to false. ({issue}`12980`) +* Add `delta.metadata.live-files.cache-ttl` configuration property for the + caching duration of active data files. ({issue}`13316`) +* Retain metadata properties and column metadata after schema changes. ({issue}`13368`, {issue}`13418`) +* Prevent writing to a table with `NOT NULL` or + [column invariants](https://github.com/delta-io/delta/blob/master/PROTOCOL.md#column-invariants) + columns. ({issue}`13353`) +* Fix incorrect min and max column statistics when writing `NULL` values. ({issue}`13389`) + +## Druid connector + +* Add support for `timestamp(p)` predicate pushdown. ({issue}`8404`) +* Report the total time spent reading data from the data source. ({issue}`13132`) +* Change mapping for the Druid `float` type to the Trino `real` type instead of + the `double` type. ({issue}`13412`) + +## Hive connector + +* Add support for short timezone IDs when translating Hive views. For example, + `JST` now works as an alias for `Asia/Tokyo`. ({issue}`13179`) +* Add support for Amazon S3 Select pushdown for JSON files. ({issue}`13354`) + +## Iceberg connector + +* Add support for hidden `$file_modified_time` columns. ({issue}`13082`) +* Add support for the Avro file format. ({issue}`12125`) +* Add support for filtering splits based on `$path` column predicates. ({issue}`12785`) +* Improve query performance for tables with updated or deleted rows. ({issue}`13092`) +* Improve performance of the `expire_snapshots` command for tables with many + snapshots. ({issue}`13399`) +* Use unique table locations by default. This can be disabled by setting the + `iceberg.unique-table-location` configuration property to false. ({issue}`12941`) +* Use the correct table schema when reading a past version of a table. ({issue}`12786`) +* Return the `$path` column without encoding when the path contains double + slashes on S3. ({issue}`13012`) +* Fix failure when inserting into a Parquet table with columns that have + quotation marks in their names. ({issue}`13074`) + +## MariaDB connector + +* Report the total time spent reading data from the data source. ({issue}`13132`) + +## MySQL connector + +* Report the total time spent reading data from the data source. ({issue}`13132`) +* Change mapping for the MySQL `enum` type to the Trino `varchar` type instead + of the `char` type. ({issue}`13303`) +* Fix failure when reading table statistics while the + `information_schema.column_statistics` table doesn't exist. ({issue}`13323`) + +## Oracle connector + +* Report the total time spent reading data from the data source. ({issue}`13132`) + +## Phoenix connector + +* Report the total time spent reading data from the data source. ({issue}`13132`) + +## Pinot connector + +* Redact the values of `pinot.grpc.tls.keystore-password` and + `pinot.grpc.tls.truststore-password` in the server log. ({issue}`13422`) + +## PostgreSQL connector + +* Report the total time spent reading data from the data source. ({issue}`13132`) +* Improve performance of queries with an `IN` expression within a complex + expression. ({issue}`13136`) + +## Redshift connector + +* Report the total time spent reading data from the data source. ({issue}`13132`) + +## SingleStore (MemSQL) connector + +* Report the total time spent reading data from the data source. ({issue}`13132`) + +## SQL Server connector + +* Report the total time spent reading data from the data source. ({issue}`13132`) diff --git a/430/_sources/release/release-393.md.txt b/430/_sources/release/release-393.md.txt new file mode 100644 index 000000000..033df6588 --- /dev/null +++ b/430/_sources/release/release-393.md.txt @@ -0,0 +1,139 @@ +# Release 393 (17 Aug 2022) + +## General + +* Add support for {doc}`/sql/merge`. ({issue}`7933`) +* Add query state and error code to the timeline log message. ({issue}`13698`) +* Improve performance of highly selective `LIMIT` queries by making them finish + as soon as the required number of rows is produced. ({issue}`13616`) +* Disallow inserting `NULL` into non-nullable columns.. ({issue}`13462`) +* Fix queries over bucketed tables never completing. ({issue}`13655`) +* Fix potential query failure for `GROUP BY` queries involving aggregations with + `ORDER BY` and `FILTER` clauses. ({issue}`13627`) + +## Security + +* Fix potential incorrect rejection of OAuth 2.0 refresh tokens. ({issue}`13168`) +* Fix incorrectly showing outdated OAuth 2.0 refresh tokens in the web UI. ({issue}`13168`) + +## Docker image + +* Add experimental image for `ppc64le`. ({issue}`13522`) + +## ClickHouse connector + +* Add support for the `unsupported-type-handling` catalog configuration + property. ({issue}`13542`) +* Improve performance for queries with selective joins. ({issue}`13334`) + +## Delta Lake connector + +* Add support for {doc}`/sql/merge`. ({issue}`7933`) +* Add support for the `NOT NULL` column constraint. ({issue}`13436`) +* Fix writing incorrect results when the order of partition columns is different + from the order in the table definition. ({issue}`13505`) +* Fix failure when reading a table which has partition columns renamed by + another engine. ({issue}`13521`) + +## Druid connector + +* Improve performance for queries with selective joins. ({issue}`13334`) + +## Hive connector + +* Add support for {doc}`/sql/merge`. ({issue}`7933`) +* Add support for bucket filtering on bucketed columns of `float`, `double`, + `date`, `list`, `map` and `bounded varchar` data types. ({issue}`13553`) +* Add `exchange.azure.max-error-retries` configuration property for the number + of retries performed when accessing Azure blob storage. ({issue}`13663`) +* Improve performance of queries with S3 Select pushdown by not utilizing + pushdown when it is unnecessary. ({issue}`13477`) +* Reduce Thrift metastore communication overhead when impersonation is enabled. ({issue}`13606`) +* Improve performance when retrieving table statistics from the metastore. ({issue}`13488`) +* Fix error when writing to a table with only `date` columns while using the + Hive metastore. ({issue}`13502`) +* Fix error when reading a Hive view which has a column names with a reserved + keyword. ({issue}`13450`) + +## Iceberg connector + +* Add support for {doc}`/sql/merge`. ({issue}`7933`) +* Improve performance when filtering on `$file_modified_time` column. ({issue}`13504`) +* Improve performance of read queries on Iceberg v2 tables with + deletion-tracking files. ({issue}`13395`) +* Allow partitioning over columns which use whitespace in their names. ({issue}`12226`) +* Disallow specifying a `NOT NULL` constraint when adding a new column. + Previously, the option was ignored. ({issue}`13673`) +* Fix error when querying tables which are empty and contain no table history. ({issue}`13576`) +* Prevent truncation of the table history in the `$snapshots` system table by + certain `DELETE` queries. ({issue}`12843`) +* Prevent errors when optimizing an Iceberg table which is empty and contains + no table history. ({issue}`13582`) +* Fix incorrect query results when reading from a materialized view that was + created on a table which was empty and contained no history. ({issue}`13574`) + +## Kafka connector + +* Fix query failure when applying a negative timestamp predicate on the + `_timestamp` column. ({issue}`13167`) + +## Kudu connector + +* Add support for {doc}`/sql/merge`. ({issue}`7933`) + +## MariaDB connector + +* Improve performance for queries with selective joins. ({issue}`13334`) + +## MongoDB connector + +* Prevent renaming a table with a name longer than the max length supported by + MongoDB. Previously, the name was truncated to the max length. ({issue}`13073`) + +## MySQL connector + +* Improve performance for queries with selective joins. ({issue}`13334`) + +## Oracle connector + +* Improve performance for queries with selective joins. ({issue}`13334`) + +## Phoenix connector + +* Improve performance for queries with selective joins. ({issue}`13334`) + +## Pinot connector + +* Add support for the Pinot `bytes` type. ({issue}`13427`) +* Add support for the `json` type. ({issue}`13428`) + +## PostgreSQL connector + +* Improve performance for queries with selective joins. ({issue}`13334`) +* Prevent using schema names or renaming a table with a name which is longer + than the max length supported by PostgreSQL. Previously, long names were + truncated to the max length. ({issue}`13307`, {issue}`13073`) + +## Raptor connector + +* Add support for {doc}`/sql/merge`. ({issue}`7933`) + +## Redshift connector + +* Improve performance for queries with selective joins. ({issue}`13334`) + +## SingleStore (MemSQL) connector + +* Improve performance for queries with selective joins. ({issue}`13334`) + +## SQL Server connector + +* Improve performance for queries with selective joins. ({issue}`13334`) +* Prevent renaming a table with a name longer than the max length supported by + SQL Server. Previously, the name was truncated to the max length. ({issue}`13073`) + +## SPI + +* Add `@Experimental` annotation to designate SPIs that are still under active + development. ({issue}`13302`) +* Deprecate `io.trino.spi.block.MethodHandleUtil`. ({issue}`13245`) diff --git a/430/_sources/release/release-394.md.txt b/430/_sources/release/release-394.md.txt new file mode 100644 index 000000000..80dddbe3f --- /dev/null +++ b/430/_sources/release/release-394.md.txt @@ -0,0 +1,88 @@ +# Release 394 (29 Aug 2022) + +## General + +* Add support for JSON as an output format of `EXPLAIN`. ({issue}`12968`) +* Improve performance of queries involving joins on a single `bigint` column. ({issue}`13432`) +* Improve performance of `LIKE` expressions. ({issue}`13479`) +* Ensure `UPDATE` queries cannot insert null values into columns with a + `NOT NULL` constraint. ({issue}`13435`) +* Fix failure when an `UPDATE` query contains a `WHERE` clause which always + evaluates to false. ({issue}`12422`) +* Fix potential failure for queries involving joins and implicit or explicit + casts of `null` to a concrete type. ({issue}`13565`) + +## Docker image + +* Ensure Trino stops running with insufficient resources to avoid partial + failures. ({issue}`13736`) + +## BigQuery connector + +* Add `query` table function for full query pass-through to the connector. ({issue}`12502`) +* Add support for the `INSERT` statement. ({issue}`6868`) +* Add support for the `CREATE TABLE ... AS SELECT ...` statement. ({issue}`6869`) + +## Delta Lake connector + +* Disallow adding a column with a `NOT NULL` constraint to a table which is not + empty. ({issue}`13785`) +* Fix failure when reading Parquet data that contains only null values. ({issue}`9424`) +* Fix potential failure of unrelated queries after dropping a schema. ({issue}`13810`) + +## Druid connector + +* Improve performance of queries that perform filtering on `varchar` columns that + contain temporal data with the format `YYYY-MM-DD`. ({issue}`12925`) + +## Elasticsearch connector + +* Add support for multiple hosts in the `elasticsearch.host` configuration + property. ({issue}`12530`) + +## Hive connector + +* Add support for a Kerberos credential cache. ({issue}`13482`) +* Fix failure when reading Parquet data that contains only null values. ({issue}`9424`) +* Fix failure when the metastore returns duplicated column statistics. ({issue}`13787`) +* Fix potential failure of unrelated queries after dropping a schema. ({issue}`13810`) + +## Iceberg connector + +* Improve query planning performance when a `varchar` partitioning column + contains date values in the `YYYY-MM-DD` format. ({issue}`12925`) +* Fix query failure when using the `[VERSION | TIMESTAMP] AS OF` clause on a + table created with Iceberg versions older than 0.12. ({issue}`13613`) +* Fix failure when reading Parquet data that contains only null values. ({issue}`9424`) + +## Oracle connector + +* Improve performance of queries that perform filtering on `varchar` columns that + contain temporal data with the format `YYYY-MM-DD`. ({issue}`12925`) + +## Phoenix connector + +* Improve performance of queries that perform filtering on `varchar` columns that + contain temporal data with the format `YYYY-MM-DD`. ({issue}`12925`) + +## Pinot connector + +* Add support for TLS when connecting to the Pinot controllers and brokers. ({issue}`13410`) +* Fix query failure when using the `HAVING` clause. ({issue}`13429`) + +## PostgreSQL connector + +* Improve performance of queries that perform filtering on `varchar` columns + that contain temporal data with the format `YYYY-MM-DD`. ({issue}`12925`) +* Prevent using a column name which is longer than the maximum length supported + by PostgreSQL. Previously, long names were truncated. ({issue}`13742`) + +## SQL Server connector + +* Prevent renaming a column to a name which is longer than the maximum length + supported by SQL Server. Previously, long names were truncated. ({issue}`13742`) + +## SPI + +* Add the query plan in JSON format to `QueryCompletedEvent`, and allow + connectors to request anonymized query plans in the `QueryCompletedEvent`. ({issue}`12968`) diff --git a/430/_sources/release/release-395.md.txt b/430/_sources/release/release-395.md.txt new file mode 100644 index 000000000..a34414f2a --- /dev/null +++ b/430/_sources/release/release-395.md.txt @@ -0,0 +1,102 @@ +# Release 395 (7 Sep 2022) + +## General + +* Reduce memory consumption when fault-tolerant execution is enabled. ({issue}`13855`) +* Reduce memory consumption of aggregations. ({issue}`12512`) +* Improve performance of aggregations with decimals. ({issue}`13573`) +* Improve concurrency for large clusters. ({issue}`13934`, `13986`) +* Remove `information_schema.role_authorization_descriptors` table. ({issue}`11341`) +* Fix `SHOW CREATE TABLE` or `SHOW COLUMNS` showing an invalid type for columns + that use a reserved keyword as column name. ({issue}`13483`) + +## ClickHouse connector + +* Fix query failure when renaming or dropping a column with a name which matches + a reserved keyword or has special characters which require it to be quoted. ({issue}`13839`) + +## Delta Lake connector + +* Add support for the `ALTER TABLE ... RENAME TO` statement with a Glue + metastore. ({issue}`12985`) +* Improve performance of inserts by automatically scaling the number of writers + within a worker node. ({issue}`13111`) +* Enforce `delta.checkpoint.writeStatsAsJson` and + `delta.checkpoint.writeStatsAsStruct` table properties to ensure table + statistics are written in the correct format. ({issue}`12031`) + +## Hive connector + +* Improve performance of inserts by automatically scaling the number of writers + within a worker node. ({issue}`13111`) +* Improve performance of S3 Select when using CSV files as an input. ({issue}`13754`) +* Fix error where the S3 KMS key is not searched in the proper AWS region when + S3 client-side encryption is used. ({issue}`13715`) + +## Iceberg connector + +* Improve performance of inserts by automatically scaling the number of writers + within a worker node. ({issue}`13111`) +* Fix creating metadata and manifest files with a URL-encoded name on S3 when + the metadata location has trailing slashes. ({issue}`13759`) + +## MariaDB connector + +* Fix query failure when renaming or dropping a column with a name which matches + a reserved keyword or has special characters which require it to be quoted. ({issue}`13839`) + +## Memory connector + +* Add support for table and column comments. ({issue}`13936`) + +## MongoDB connector + +* Fix query failure when filtering on columns of `json` type. ({issue}`13536`) + +## MySQL connector + +* Fix query failure when renaming or dropping a column with a name which matches + a reserved keyword or has special characters which require it to be quoted. ({issue}`13839`) + +## Oracle connector + +* Fix query failure when renaming or dropping a column with a name which matches + a reserved keyword or has special characters which require it to be quoted. ({issue}`13839`) + +## Phoenix connector + +* Fix query failure when adding, renaming, or dropping a column with a name + which matches a reserved keyword or has special characters which require it to + be quoted. ({issue}`13839`) + +## PostgreSQL connector + +* Fix query failure when renaming or dropping a column with a name which matches + a reserved keyword or has special characters which require it to be quoted. ({issue}`13839`) + +## Prometheus connector + +* Add support for case-insensitive table name matching with the + `prometheus.case-insensitive-name-matching` configuration property. ({issue}`8740`) + +## Redshift connector + +* Fix query failure when renaming or dropping a column with a name which matches + a reserved keyword or has special characters which require it to be quoted. ({issue}`13839`) + +## SingleStore (MemSQL) connector + +* Fix query failure when renaming or dropping a column with a name which matches + a reserved keyword or has special characters which require it to be quoted. ({issue}`13839`) + +## SQL Server connector + +* Fix query failure when renaming or dropping a column with a name which matches + a reserved keyword or has special characters which require it to be quoted. ({issue}`13839`) + +## SPI + +* Add support for dynamic function resolution. ({issue}`8`) +* Rename `LIKE_PATTERN_FUNCTION_NAME` to `LIKE_FUNCTION_NAME` in + `StandardFunctions`. ({issue}`13965`) +* Remove the `listAllRoleGrants` method from `ConnectorMetadata`. ({issue}`11341`) diff --git a/430/_sources/release/release-396.md.txt b/430/_sources/release/release-396.md.txt new file mode 100644 index 000000000..551c86c8b --- /dev/null +++ b/430/_sources/release/release-396.md.txt @@ -0,0 +1,89 @@ +# Release 396 (15 Sep 2022) + +## General + +* Improve performance of queries that process string data. ({issue}`12798`) +* Fix failure when querying views that use table functions. ({issue}`13944`) + +## BigQuery connector + +* Add support for [column and table comments](/sql/comment). ({issue}`13882`) + +## ClickHouse connector + +* Improve performance when pushing down large lists of predicates by raising + the default threshold before the predicate is compacted. ({issue}`14029`) +* Fix mapping to the ClickHouse `Date` and `DateTime` types to include the full + range of possible values. ({issue}`11116`) +* Fix failure when specifying a table or column comment that contains special + characters. ({issue}`14058`) + +## Delta Lake connector + +* Add support for writing to tables using [version 3 of the + writer](https://docs.delta.io/latest/versioning.html#features-by-protocol-version). + This does not yet include support for `CHECK` constraints. ({issue}`14068`) +* Add support for reading tables with the table property + `delta.columnMapping.mode` set to `id`. ({issue}`13629`) +* Improve performance when writing + [structural data types](structural-data-types) to Parquet files. ({issue}`13714`) +* Attempt to undo the operation when writing a checkpoint file fails. ({issue}`14108`) +* Fix performance regression when reading Parquet data. ({issue}`14094`) + +## Hive connector + +* Add verification for files written by the optimized Parquet writer. This can + be configured with the [`parquet.optimized-writer.validation-percentage` + configuration property](hive-parquet-configuration) or the + `parquet_optimized_writer_validation_percentage` session property. ({issue}`13246`) +* Improve optimized Parquet writer performance for + [structural data types](structural-data-types). ({issue}`13714`) +* Fix performance regression in reading Parquet files. ({issue}`14094`) + +## Iceberg connector + +* Improve performance when writing + [structural data types](structural-data-types) to Parquet files. ({issue}`13714`) +* Improve performance of queries that contain predicates involving `date_trunc` + on `date`, `timestamp` or `timestamp with time zone` partition columns. ({issue}`14011`) +* Fix incorrect results from using the `[VERSION | TIMESTAMP] AS OF` clause when + the snapshot's schema differs from the current schema of the table. ({issue}`14064`) +* Prevent `No bucket node map` failures when inserting data. ({issue}`13960`) +* Fix performance regression when reading Parquet data introduced in + [Trino version 394](release-394.md). ({issue}`14094`) + +## MariaDB connector + +* Fix failure when using special characters in a table or column comment when + creating a table. ({issue}`14058`) + +## MySQL connector + +* Fix failure when using special characters in a table or column comment when + creating a table. ({issue}`14058`) + +## Oracle connector + +* Fix failure when setting a column comment with special characters. ({issue}`14058`) + +## Phoenix connector + +* Improve performance when pushing down large lists of predicates by raising + the default threshold before the predicate is compacted. ({issue}`14029`) + +## PostgreSQL connector + +* Fix failure when setting a column comment with special characters. ({issue}`14058`) + +## Redshift connector + +* Fix failure when setting a column comment with special characters. ({issue}`14058`) + +## SPI + +* Add the `SystemAccessControl.checkCanGrantExecuteFunctionPrivilege` overload, + which needs to be implemented to allow views that use table functions. ({issue}`13944`) +* Add the `ConnectorMetadata.applyJoin` overload. It provides the connector + with a join condition that is as complete as possible to represent using + `ConnectorExpression`. Deprecate the previous version of + `ConnectorMetadata.applyJoin`. ({issue}`13943`) diff --git a/430/_sources/release/release-397.md.txt b/430/_sources/release/release-397.md.txt new file mode 100644 index 000000000..3e6d22b09 --- /dev/null +++ b/430/_sources/release/release-397.md.txt @@ -0,0 +1,57 @@ +# Release 397 (21 Sep 2022) + +## General + +* Fix incorrect parsing of invalid values in cast from `varchar` to `timestamp`. ({issue}`14164`) +* Fix potential incorrect results for queries with a partitioned output which + doesn't depend on any column data. ({issue}`14168`) +* Fix `EXPLAIN (TYPE IO)` query failure for certain queries on empty tables. ({issue}`10398`) + +## Security + +* Add support for multiple recipients with JWT authentication. ({issue}`13442 `) +* Fix OAuth 2.0 token refresh causing JWT authentication failure. ({issue}`13575`) + +## JDBC driver + +* Fix potential memory leak when cancelling statements. ({issue}`14176`) + +## Delta Lake connector + +* Rename the `parquet.experimental-optimized-writer.enabled` configuration + property and `experimental_parquet_optimized_writer_enabled` session property + to `parquet.optimized-writer.enabled` and `parquet_optimized_writer_enabled`, + respectively. ({issue}`14137`) + +## Hive connector + +* Rename the `parquet.experimental-optimized-writer.enabled` configuration + property and `experimental_parquet_optimized_writer_enabled` session property + to `parquet.optimized-writer.enabled` and `parquet_optimized_writer_enabled`, + respectively. ({issue}`14137`) +* Improve performance when querying JSON data and Hive S3 Select pushdown is + enabled. ({issue}`14040`) +* Improve planning performance when querying tables in the Glue catalog that + contain a large number of columns. ({issue}`14206`) +* Allow reading from a partitioned table after a column's data type was changed + from `decimal` to `varchar` or `string`. ({issue}`2817`) +* Fix query failure when reading from a Hive view and + `hive.hive-views.run-as-invoker` and `hive.hive-views.legacy-translation` are + both enabled. ({issue}`14077`) + +## Iceberg connector + +* Improve performance of queries that contain predicates involving `date_trunc` + with an `hour` unit on `date`, `timestamp`, or `timestamp with time zone` + partition columns. ({issue}`14161`) +* Improve performance of reads after a `DELETE` removes all rows from a file. ({issue}`14198`) +* Reduce query latency when using a Glue catalog for metadata. ({issue}`13875`) +* Fix materialized views temporarily appearing empty when a refresh is about to + complete. ({issue}`14145`) +* Fix potential table corruption when changing a table before it is known if + committing to the Hive metastore has failed or succeeded. ({issue}`14174`) + +## SPI + +* Replace `DictionaryBlock` constructors with a factory method. ({issue}`14092`) +* Replace `RunLengthEncodedBlock` constructors with a factory method. ({issue}`14092`) diff --git a/430/_sources/release/release-398.md.txt b/430/_sources/release/release-398.md.txt new file mode 100644 index 000000000..f71d7d899 --- /dev/null +++ b/430/_sources/release/release-398.md.txt @@ -0,0 +1,63 @@ +# Release 398 (28 Sep 2022) + +## General + +* Add Hudi connector. ({issue}`10228`) +* Add metrics for the execution time of filters and projections to `EXPLAIN + ANALYZE VERBOSE`. ({issue}`14135`) +* Show local cost estimates when using `EXPLAIN`. ({issue}`14268`) +* Fix timeouts happening too early because of improper handling of the + `node-scheduler.allowed-no-matching-node-period` configuration property. ({issue}`14256`) +* Fix query failure for `MERGE` queries when `task_writer_count` is greater + than one. ({issue}`14306`) + +## Accumulo connector + +* Add support for column comments when creating a new table. ({issue}`14114`) +* Move column mapping and index information into the output of `DESCRIBE` + instead of a comment. ({issue}`14095`) + +## BigQuery connector + +* Fix improper escaping of backslash and newline characters. ({issue}`14254`) +* Fix query failure when the predicate involves a `varchar` value with a + backslash. ({issue}`14254`) + +## ClickHouse connector + +* Upgrade minimum required Clickhouse version to 21.8. ({issue}`14112`) + +## Delta Lake connector + +* Improve performance when reading Parquet files for queries with predicates. ({issue}`14247`) + +## Elasticsearch connector + +* Deprecate support for query pass-through using the special + `$query:` dynamic tables in favor of the `raw_query` table + function. Legacy behavior can be re-enabled with the + `elasticsearch.legacy-pass-through-query.enabled` configuration property. ({issue}`14015`) + +## Hive connector + +* Add support for partitioned views when legacy mode for view translation is + enabled. ({issue}`14028`) +* Extend the `flush_metadata_cache` procedure to be able to flush table-related + caches instead of only partition-related caches. ({issue}`14219`) +* Improve performance when reading Parquet files for queries with predicates. ({issue}`14247`) + +## Iceberg connector + +* Improve performance when reading Parquet files for queries with predicates. ({issue}`14247`) +* Fix potential table corruption when changing a table before it is known if + committing to the Glue metastore has failed or succeeded. ({issue}`14174`) + +## Pinot connector + +* Add support for the `timestamp` type. ({issue}`10199`) + +## SPI + +* Extend `ConnectorMetadata.getStatisticsCollectionMetadata` to allow the + connector to request the computation of any aggregation function during stats + collection. ({issue}`14233`) diff --git a/430/_sources/release/release-399.md.txt b/430/_sources/release/release-399.md.txt new file mode 100644 index 000000000..633c6346d --- /dev/null +++ b/430/_sources/release/release-399.md.txt @@ -0,0 +1,75 @@ +# Release 399 (6 Oct 2022) + +## General + +* Add operator CPU and wall time distribution to `EXPLAIN ANALYZE VERBOSE`. ({issue}`14370`) +* Improve performance of joins. ({issue}`13352`) +* Remove support for the deprecated `row` to `json` cast behavior, and remove the + `deprecated.legacy-row-to-json-cast` configuration property. ({issue}`14388`) +* Fix error when using `PREPARE` with `DROP VIEW` when the view name is quoted. ({issue}`14196`) +* Fix potential planning failure for queries involving `UNION`. ({issue}`14472`) +* Fix error when using aggregations in window expressions when the function + loaded from a plugin. ({issue}`14486`) + +## Accumulo connector + +* Change the default value of the `accumulo.zookeeper.metadata.root` + configuration property to `/trino-accumulo` from `/presto-accumulo`. ({issue}`14326`) + +## BigQuery connector + +* Add support for writing `array`, `row`, and `timestamp` columns. ({issue}`14418`, {issue}`14473`) + +## ClickHouse connector + +* Fix bug where the intended default value of the `domain-compaction-threshold` + configuration property was incorrectly used as a maximum limit. ({issue}`14350`) + +## Delta Lake connector + +* Improve performance of reading decimal columns from Parquet files. ({issue}`14260`) +* Allow setting the AWS Security Token Service endpoint and region when using a + Glue metastore. ({issue}`14412`) + +## Hive connector + +* Add `max-partition-drops-per-query` configuration property to limit the number + of partition drops. ({issue}`12386`) +* Add `hive.s3.region` configuration property to force S3 to connect to a + specific region. ({issue}`14398`) +* Improve performance of reading decimal columns from Parquet files. ({issue}`14260`) +* Reduce memory usage on the coordinator. ({issue}`14408`) +* Reduce query memory usage during inserts to S3. ({issue}`14212`) +* Change the name of the `partition_column` and `partition_value` arguments for + the `flush_metadata_cache` procedure to `partition_columns` and + `partition_values`, respectively, for parity with other procedures. ({issue}`13566`) +* Change field name matching to be case insensitive. ({issue}`13423`) +* Allow setting the AWS STS endpoint and region when using a Glue metastore. ({issue}`14412`) + +## Hudi connector + +* Fix failure when reading hidden columns. ({issue}`14341`) + +## Iceberg connector + +* Improve performance of reading decimal columns from Parquet files. ({issue}`14260`) +* Reduce planning time for complex queries. ({issue}`14443`) +* Store metastore `table_type` property value in uppercase for compatibility + with other Iceberg catalog implementations. ({issue}`14384`) +* Allow setting the AWS STS endpoint and region when using a Glue metastore. ({issue}`14412`) + +## Phoenix connector + +* Fix bug where the intended default value of the `domain-compaction-threshold` + configuration property was incorrectly used as a maximum limit. ({issue}`14350`) + +## SQL Server connector + +* Fix error when querying or listing tables with names that contain special + characters. ({issue}`14286`) + +## SPI + +* Add stage output buffer distribution to `EventListener`. ({issue}`14400`) +* Remove deprecated `TimeType.TIME`, `TimestampType.TIMESTAMP` and + `TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE` constants. ({issue}`14414`) diff --git a/430/_sources/release/release-400.md.txt b/430/_sources/release/release-400.md.txt new file mode 100644 index 000000000..e35477878 --- /dev/null +++ b/430/_sources/release/release-400.md.txt @@ -0,0 +1,60 @@ +# Release 400 (13 Oct 2022) + +## General + +* Add output buffer utilization to `EXPLAIN ANALYZE VERBOSE`. ({issue}`14396`) +* Increase concurrency for large clusters. ({issue}`14395`) +* Fix JSON serialization failure for `QueryCompletedEvent` in event listener. + ({issue}`14604`) +* Fix occasional `maximum pending connection acquisitions exceeded` failure + when fault-tolerant execution is enabled. ({issue}`14580`) +* Fix incorrect results when calling the `round` function on large `real` and + `double` values. ({issue}`14613`) +* Fix query failure when using the `merge(qdigest)` function. ({issue}`14616`) + +## BigQuery connector + +* Add support for [truncating tables](/sql/truncate). ({issue}`14494`) + +## Delta Lake connector + +* Prevent coordinator out-of-memory failure when querying a large number of + tables in a short period of time. ({issue}`14571`) + +## Hive connector + +* Reduce memory usage when scanning a large number of partitions, and add the + `hive.max-partitions-for-eager-load` configuration property to manage the + number of partitions that can be loaded into memory. ({issue}`14225`) +* Increase the default value of the `hive.max-partitions-per-scan` + configuration property to `1000000` from `100000`. ({issue}`14225`) +* Utilize the `hive.metastore.thrift.delete-files-on-drop` configuration + property when dropping partitions and tables. Previously, it was only used + when dropping tables. ({issue}`13545`) + +## Hudi connector + +* Hide Hive system schemas. ({issue}`14510`) + +## Iceberg connector + +* Reduce query latency when querying tables with a large number of files. ({issue}`14504`) +* Prevent table corruption when changing a table fails due to an inability to + release the table lock from the Hive metastore. ({issue}`14386`) +* Fix query failure when reading from a table with a leading double slash in the + metadata location. ({issue}`14299`) + +## Pinot connector + +* Add support for the Pinot proxy for controller/broker and server gRPC + requests. ({issue}`13015`) +* Update minimum required version to 0.10.0. ({issue}`14090`) + +## SQL Server connector + +* Allow renaming column names containing special characters. ({issue}`14272`) + +## SPI + +* Add `ConnectorAccessControl.checkCanGrantExecuteFunctionPrivilege` overload + which must be implemented to allow views that use table functions. ({issue}`13944`) diff --git a/430/_sources/release/release-401.md.txt b/430/_sources/release/release-401.md.txt new file mode 100644 index 000000000..dacba5819 --- /dev/null +++ b/430/_sources/release/release-401.md.txt @@ -0,0 +1,81 @@ +# Release 401 (26 Oct 2022) + +## General + +* Add support for using path-style access for all requests to S3 when using + fault-tolerant execution with exchange spooling. This can be enabled with the + `exchange.s3.path-style-access` configuration property. ({issue}`14655`) +* Add support for table functions in file-based access control. ({issue}`13713`) +* Add output buffer utilization distribution to `EXPLAIN ANALYZE VERBOSE`. ({issue}`14596`) +* Add operator blocked time distribution to `EXPLAIN ANALYZE VERBOSE`. ({issue}`14640`) +* Improve performance and reliability of `INSERT` and `MERGE`. ({issue}`14553`) +* Fix query failure caused by a + `com.google.common.base.VerifyException: cannot unset noMoreSplits` error. ({issue}`14668`) +* Fix underestimation of CPU usage and scheduled time statistics for joins in + `EXPLAIN ANALYZE`. ({issue}`14572`) + +## Cassandra connector + +* Upgrade minimum required Cassandra version to 3.0. ({issue}`14562`) + +## Delta Lake connector + +* Add support for writing to tables with [Delta Lake writer protocol version 4](https://docs.delta.io/latest/versioning.html#features-by-protocol-version). + This does not yet include support for [change data feeds](https://docs.delta.io/2.0.0/delta-change-data-feed.html) + or generated columns. ({issue}`14573 `) +* Add support for writes on Google Cloud Storage. ({issue}`12264`) +* Avoid overwriting the reader and writer versions when executing a `COMMENT` or + `ALTER TABLE ... ADD COLUMN` statement. ({issue}`14611`) +* Fix failure when listing tables from the Glue metastore and one of the tables + has no properties. ({issue}`14577`) + +## Hive connector + +* Add support for [IBM Cloud Object Storage](/connector/hive-cos). ({issue}`14625`) +* Allow creating tables with an Avro schema literal using the new table property + `avro_schema_literal`. ({issue}`14426`) +* Fix potential query failure or incorrect results when reading from a table + with the `avro.schema.literal` Hive table property set. ({issue}`14426`) +* Fix failure when listing tables from the Glue metastore and one of the tables + has no properties. ({issue}`14577`) + +## Iceberg connector + +* Improve performance of the `remove_orphan_files` table procedure. ({issue}`13691`) +* Fix query failure when analyzing a table that contains a column with a + non-lowercase name. ({issue}`14583`) +* Fix failure when listing tables from the Glue metastore and one of the tables + has no properties. ({issue}`14577`) + +## Kafka connector + +* Add support for configuring the prefix for internal column names with the + `kafka.internal-column-prefix` catalog configuration property. The default + value is `_` to maintain current behavior. ({issue}`14224`) + +## MongoDB connector + +* Add `query` table function for query pass-through to the connector. ({issue}`14535`) + +## MySQL connector + +* Add support for writes when [fault-tolerant + execution](/admin/fault-tolerant-execution) is enabled. ({issue}`14445`) + +## Pinot connector + +* Fix failure when executing `SHOW CREATE TABLE`. ({issue}`14071`) + +## PostgreSQL connector + +* Add support for writes when [fault-tolerant + execution](/admin/fault-tolerant-execution) is enabled. ({issue}`14445`) + +## SQL Server connector + +* Add support for writes when [fault-tolerant + execution](/admin/fault-tolerant-execution) is enabled. ({issue}`14730`) + +## SPI + +* Add stage output buffer distribution to `EventListener`. ({issue}`14638`) diff --git a/430/_sources/release/release-402.md.txt b/430/_sources/release/release-402.md.txt new file mode 100644 index 000000000..709f6b733 --- /dev/null +++ b/430/_sources/release/release-402.md.txt @@ -0,0 +1,108 @@ +# Release 402 (2 Nov 2022) + +## General + +* Fix query processing when [fault-tolerant execution](/admin/fault-tolerant-execution) + is enabled and a [stage](trino-concept-stage) of the query produces no data. ({issue}`14794`) + +## Blackhole connector + +* Add support for column comments on view columns. ({issue}`10705`) + +## Clickhouse connector + +* Reuse JDBC connections for metadata queries. This can be disabled with the + `query.reuse-connection` configuration property. ({issue}`14653`) + +## Delta Lake connector + +* Remove the deprecated `hive.parquet.fail-on-corrupted-statistics` and + `parquet.fail-on-corrupted-statistics` configuration properties. The + `parquet.ignore-statistics` property can be used to allow querying Parquet + files with corrupted or incorrect statistics. ({issue}`14777`) +* Fix memory leak and improve memory tracking during large `INSERT` queries. ({issue}`14823`) + +## Druid connector + +* Reuse JDBC connections for metadata queries. This can be disabled with the + `query.reuse-connection` configuration property. ({issue}`14653`) + +## Hive connector + +* Add support for column comments on view columns. ({issue}`10705`) +* Remove the deprecated `hive.parquet.fail-on-corrupted-statistics` and + `parquet.fail-on-corrupted-statistics` configuration properties. The + `parquet.ignore-statistics` property can be used to allow querying Parquet + files with corrupted or incorrect statistics. ({issue}`14777`) +* Fix memory leak and improve memory tracking during large `INSERT` queries. ({issue}`14823`) + +## Hudi connector + +* Remove the deprecated `hive.parquet.fail-on-corrupted-statistics` and + `parquet.fail-on-corrupted-statistics` configuration properties. The + `parquet.ignore-statistics` property can be used to allow querying Parquet + files with corrupted or incorrect statistics. ({issue}`14777`) + +## Iceberg connector + +* Add support to skip archiving when committing to a table in the Glue + metastore and the `iceberg.glue.skip-archive` configuration property is set + to true. ({issue}`13413`) +* Add support for column comments on view columns. ({issue}`10705`) +* Remove the deprecated `hive.parquet.fail-on-corrupted-statistics` and + `parquet.fail-on-corrupted-statistics` configuration properties. The + `parquet.ignore-statistics` property can be used to allow querying Parquet + files with corrupted or incorrect statistics. ({issue}`14777`) +* Fix incorrect results when the column order in the equality delete filter is + different from the table definition. ({issue}`14693`) +* Fix memory leak and improve memory tracking during large `INSERT` queries. ({issue}`14823`) + +## MariaDB connector + +* Reuse JDBC connections for metadata queries. This can be disabled with the + `query.reuse-connection` configuration property. ({issue}`14653`) + +## Memory connector + +* Add support for column comments on view columns. ({issue}`10705`) + +## MySQL connector + +* Reuse JDBC connections for metadata queries. This can be disabled with the + `query.reuse-connection` configuration property. ({issue}`14653`) + +## MongoDB connector + +* Support predicate pushdown on `date`, `time(3)`, `timestamp(3)` and + `timestamp(3) with time zone` columns. ({issue}`14795`) + +## Oracle connector + +* Reuse JDBC connections for metadata queries. This can be disabled with the + `query.reuse-connection` configuration property. ({issue}`14653`) + +## Phoenix connector + +* Reuse JDBC connections for metadata queries. This can be disabled with the + `query.reuse-connection` configuration property. ({issue}`14653`) + +## PostgreSQL connector + +* Reuse JDBC connections for metadata queries. This can be disabled with the + `query.reuse-connection` configuration property. ({issue}`14653`) + +## Redshift connector + +* Reuse JDBC connections for metadata queries. This can be disabled with the + `query.reuse-connection` configuration property. ({issue}`14653`) + +## SingleStore (MemSQL) connector + +* Reuse JDBC connections for metadata queries. This can be disabled with the + `query.reuse-connection` configuration property. ({issue}`14653`) + +## SQL Server connector + +* Improve performance of certain queries which use the `OR` operator. ({issue}`14570`) +* Improve performance of queries with predicates involving the `nullif` function + or arithmetic expressions. ({issue}`14570`) diff --git a/430/_sources/release/release-403.md.txt b/430/_sources/release/release-403.md.txt new file mode 100644 index 000000000..55bc9f564 --- /dev/null +++ b/430/_sources/release/release-403.md.txt @@ -0,0 +1,71 @@ +# Release 403 (15 Nov 2022) + +## General + +* Include the amount of data read from external sources in the output of + `EXPLAIN ANALYZE`. ({issue}`14907`) +* Improve performance of worker-to-worker data transfer encryption when + fault-tolerant execution is enabled. ({issue}`14941`) +* Improve performance of aggregations when input data does not contain nulls. ({issue}`14567`) +* Fix potential failure when clients do not support variable precision temporal + types. ({issue}`14950`) +* Fix query deadlock in multi-join queries where broadcast join size is + underestimated. ({issue}`14948`) +* Fix incorrect results when `min(x, n)` or `max(x, n)` is used as a window + function. ({issue}`14886`) +* Fix failure for certain queries involving joins over partitioned tables. ({issue}`14317`) +* Fix incorrect order of parameters in `DESCRIBE INPUT` when they appear in a + `WITH` clause. ({issue}`14738`) +* Fix failure for queries involving `BETWEEN` predicates over `varchar` columns + that contain temporal data. ({issue}`14954`) + +## Security + +* Allow access token passthrough when using OAuth 2.0 authentication with + refresh tokens enabled. ({issue}`14949`) + +## BigQuery connector + +* Improve performance of `SHOW SCHEMAS` by adding a metadata cache. This can be + configured with the `bigquery.metadata.cache-ttl` catalog property, which is + disabled by default. ({issue}`14729`) +* Fix failure when a [row access policy](https://cloud.google.com/bigquery/docs/row-level-security-intro) + returns an empty result. ({issue}`14760`) + +## ClickHouse connector + +* Add mapping for the ClickHouse `DateTime(timezone)` type to the Trino + `timestamp(0) with time zone` type for read-only operations. ({issue}`13541`) + +## Delta Lake connector + +* Fix statistics for `DATE` columns. ({issue}`15005`) + +## Hive connector + +* Avoid showing the unsupported `AUTHORIZATION ROLE` property in the result of + `SHOW CREATE SCHEMA` when the access control doesn't support roles. ({issue}`8817`) + +## Iceberg connector + +* Improve performance and storage requirements when running the + `expire_snapshots` table procedure on S3-compatible storage. ({issue}`14434`) +* Allow registering existing table files in the metastore with the new + [`register_table` procedure](iceberg-register-table). ({issue}`13552`) + +## MongoDB connector + +* Add support for {doc}`/sql/delete`. ({issue}`14864`) +* Fix incorrect results when predicates over `varchar` and `char` columns are + pushed into the connector and MongoDB collections have a collation specified. ({issue}`14900`) + +## SQL Server connector + +* Fix incorrect results when non-transactional `INSERT` is disabled and bulk + `INSERT` is enabled. ({issue}`14856`) + +## SPI + +* Enhance `ConnectorTableLayout` to allow the connector to specify that multiple + writers per partition are allowed. ({issue}`14956`) +* Remove deprecated methods from `ConnectorPageSinkProvider`. ({issue}`14959`) diff --git a/430/_sources/release/release-404.md.txt b/430/_sources/release/release-404.md.txt new file mode 100644 index 000000000..393a328d9 --- /dev/null +++ b/430/_sources/release/release-404.md.txt @@ -0,0 +1,6 @@ +# Release 404 (???) + +```{note} +Where's release 404? Not found. (We skipped it on purpose because we think we're +funny.) +``` diff --git a/430/_sources/release/release-405.md.txt b/430/_sources/release/release-405.md.txt new file mode 100644 index 000000000..e64c63c9d --- /dev/null +++ b/430/_sources/release/release-405.md.txt @@ -0,0 +1,243 @@ +# Release 405 (28 Dec 2022) + +## General + +* Add Trino version to the output of `EXPLAIN`. ({issue}`15317`) +* Add task input/output size distribution to the output of + `EXPLAIN ANALYZE VERBOSE`. ({issue}`15286`) +* Add stage skewness warnings to the output of `EXPLAIN ANALYZE`. ({issue}`15286`) +* Add support for `ALTER COLUMN ... SET DATA TYPE` statement. ({issue}`11608`) +* Allow configuring a refresh interval for the database resource group manager + with the `resource-groups.refresh-interval` configuration property. ({issue}`14514`) +* Improve performance of queries that compare `date` columns with + `timestamp(n) with time zone` literals. ({issue}`5798`) +* Improve performance and resource utilization when inserting into tables. ({issue}`14718`, {issue}`14874`) +* Improve performance for `INSERT` queries when fault-tolerant execution is + enabled. ({issue}`14735`) +* Improve planning performance for queries with many `GROUP BY` clauses. ({issue}`15292`) +* Improve query performance for large clusters and skewed queries. ({issue}`15369`) +* Rename the `node-scheduler.max-pending-splits-per-task` configuration property + to `node-scheduler.min-pending-splits-per-task`. ({issue}`15168`) +* Ensure that the configured number of task retries is not larger than 126. ({issue}`14459`) +* Fix incorrect rounding of `time(n)` and `time(n) with time zone` values near + the top of the range of allowed values. ({issue}`15138`) +* Fix incorrect results for queries involving window functions without a + `PARTITION BY` clause followed by the evaluation of window functions with a + `PARTITION BY` and `ORDER BY` clause. ({issue}`15203`) +* Fix incorrect results when adding or subtracting an `interval` from a + `timestamp with time zone`. ({issue}`15103`) +* Fix potential incorrect results when joining tables on indexed and non-indexed + columns at the same time. ({issue}`15334`) +* Fix potential failure of queries involving `MATCH_RECOGNIZE`. ({issue}`15343`) +* Fix incorrect reporting of `Projection CPU time` in the output of `EXPLAIN + ANALYZE VERBOSE`. ({issue}`15364`) +* Fix `SET TIME ZONE LOCAL` to correctly reset to the initial time zone of the + client session. ({issue}`15314`) + +## Security + +* Add support for string replacement as part of + [impersonation rules](system-file-auth-impersonation-rules). ({issue}`14962`) +* Add support for fetching access control rules via HTTPS. ({issue}`14008`) +* Fix some `system.metadata` tables improperly showing the names of catalogs + which the user cannot access. ({issue}`14000`) +* Fix `USE` statement improperly disclosing the names of catalogs and schemas + which the user cannot access. ({issue}`14208`) +* Fix improper HTTP redirect after OAuth 2.0 token refresh. ({issue}`15336`) + +## Web UI + +* Display operator CPU time in the "Stage Performance" tab. ({issue}`15339`) + +## JDBC driver + +* Return correct values in `NULLABLE` columns of the + `DatabaseMetaData.getColumns` result. ({issue}`15214`) + +## BigQuery connector + +* Improve read performance with experimental support for [Apache Arrow](https://arrow.apache.org/docs/) + serialization when reading from BigQuery. This can be enabled with the + `bigquery.experimental.arrow-serialization.enabled` catalog configuration + property. ({issue}`14972`) +* Fix queries incorrectly executing with the project ID specified in the + credentials instead of the project ID specified in the `bigquery.project-id` + catalog property. ({issue}`14083`) + +## Delta Lake connector + +* Add support for views. ({issue}`11609`) +* Add support for configuring batch size for reads on Parquet files using the + `parquet.max-read-block-row-count` configuration property or the + `parquet_max_read_block_row_count` session property. ({issue}`15474`) +* Improve performance and reduce storage requirements when running the `vacuum` + procedure on S3-compatible storage. ({issue}`15072`) +* Improve memory accounting for `INSERT`, `MERGE`, and + `CREATE TABLE ... AS SELECT` queries. ({issue}`14407`) +* Improve performance of reading Parquet files for `boolean`, `tinyint`, + `short`, `int`, `long`, `float`, `double`, `short decimal`, `UUID`, `time`, + `decimal`, `varchar`, and `char` data types. This optimization can be disabled + with the `parquet.optimized-reader.enabled` catalog configuration property. ({issue}`14423`, {issue}`14667`) +* Improve query performance when the `nulls fraction` statistic is not available + for some columns. ({issue}`15132`) +* Improve performance when reading Parquet files. ({issue}`15257`, {issue}`15474`) +* Improve performance of reading Parquet files for queries with filters. ({issue}`15268`) +* Improve `DROP TABLE` performance for tables stored on AWS S3. ({issue}`13974`) +* Improve performance of reading Parquet files for `timestamp` and + `timestamp with timezone` data types. ({issue}`15204`) +* Improve performance of queries that read a small number of columns and queries + that process tables with large Parquet row groups or ORC stripes. ({issue}`15168`) +* Improve stability and reduce peak memory requirements when reading from + Parquet files. ({issue}`15374`) +* Allow registering existing table files in the metastore with the new + [`register_table` procedure](delta-lake-register-table). ({issue}`13568`) +* Deprecate creating a new table with existing table content. This can be + re-enabled using the `delta.legacy-create-table-with-existing-location.enabled` + configuration property or the + `legacy_create_table_with_existing_location_enabled` session property. ({issue}`13568`) +* Fix query failure when reading Parquet files with large row groups. ({issue}`5729`) +* Fix `DROP TABLE` leaving files behind when using managed tables stored on S3 + and created by the Databricks runtime. ({issue}`13017`) +* Fix query failure when the path contains special characters. ({issue}`15183`) +* Fix potential `INSERT` failure for tables stored on S3. ({issue}`15476`) + +## Google Sheets connector + +* Add support for setting a read timeout with the `gsheets.read-timeout` + configuration property. ({issue}`15322`) +* Add support for `base64`-encoded credentials using the + `gsheets.credentials-key` configuration property. ({issue}`15477`) +* Rename the `credentials-path` configuration property to + `gsheets.credentials-path`, `metadata-sheet-id` to + `gsheets.metadata-sheet-id`, `sheets-data-max-cache-size` to + `gsheets.max-data-cache-size`, and `sheets-data-expire-after-write` to + `gsheets.data-cache-ttl`. ({issue}`15042`) + +## Hive connector + +* Add support for referencing nested fields in columns with the `UNIONTYPE` Hive + type. ({issue}`15278`) +* Add support for configuring batch size for reads on Parquet files using the + `parquet.max-read-block-row-count` configuration property or the + `parquet_max_read_block_row_count` session property. ({issue}`15474`) +* Improve memory accounting for `INSERT`, `MERGE`, and `CREATE TABLE AS SELECT` + queries. ({issue}`14407`) +* Improve performance of reading Parquet files for `boolean`, `tinyint`, + `short`, `int`, `long`, `float`, `double`, `short decimal`, `UUID`, `time`, + `decimal`, `varchar`, and `char` data types. This optimization can be disabled + with the `parquet.optimized-reader.enabled` catalog configuration property. ({issue}`14423`, {issue}`14667`) +* Improve performance for queries which write data into multiple partitions. ({issue}`15241`, {issue}`15066`) +* Improve performance when reading Parquet files. ({issue}`15257`, {issue}`15474`) +* Improve performance of reading Parquet files for queries with filters. ({issue}`15268`) +* Improve `DROP TABLE` performance for tables stored on AWS S3. ({issue}`13974`) +* Improve performance of reading Parquet files for `timestamp` and + `timestamp with timezone` data types. ({issue}`15204`) +* Improve performance of queries that read a small number of columns and queries + that process tables with large Parquet row groups or ORC stripes. ({issue}`15168`) +* Improve stability and reduce peak memory requirements when reading from + Parquet files. ({issue}`15374`) +* Disallow creating transactional tables when not using the Hive metastore. ({issue}`14673`) +* Fix query failure when reading Parquet files with large row groups. ({issue}`5729`) +* Fix incorrect `schema already exists` error caused by a client timeout when + creating a new schema. ({issue}`15174`) +* Fix failure when an access denied exception happens while listing tables or + views in a Glue metastore. ({issue}`14746`) +* Fix `INSERT` failure on ORC ACID tables when Apache Hive 3.1.2 is used as a + metastore. ({issue}`7310`) +* Fix failure when reading Hive views with `char` types. ({issue}`15470`) +* Fix potential `INSERT` failure for tables stored on S3. ({issue}`15476`) + +## Hudi connector + +* Improve performance of reading Parquet files for `boolean`, `tinyint`, + `short`, `int`, `long`, `float`, `double`, `short decimal`, `UUID`, `time`, + `decimal`, `varchar`, and `char` data types. This optimization can be disabled + with the `parquet.optimized-reader.enabled` catalog configuration property. ({issue}`14423`, {issue}`14667`) +* Improve performance of reading Parquet files for queries with filters. ({issue}`15268`) +* Improve performance of reading Parquet files for `timestamp` and + `timestamp with timezone` data types. ({issue}`15204`) +* Improve performance of queries that read a small number of columns and queries + that process tables with large Parquet row groups or ORC stripes. ({issue}`15168`) +* Improve stability and reduce peak memory requirements when reading from + Parquet files. ({issue}`15374`) +* Fix query failure when reading Parquet files with large row groups. ({issue}`5729`) + +## Iceberg connector + +* Add support for configuring batch size for reads on Parquet files using the + `parquet.max-read-block-row-count` configuration property or the + `parquet_max_read_block_row_count` session property. ({issue}`15474`) +* Add support for the Iceberg REST catalog. ({issue}`13294`) +* Improve memory accounting for `INSERT`, `MERGE`, and `CREATE TABLE AS SELECT` + queries. ({issue}`14407`) +* Improve performance of reading Parquet files for `boolean`, `tinyint`, + `short`, `int`, `long`, `float`, `double`, `short decimal`, `UUID`, `time`, + `decimal`, `varchar`, and `char` data types. This optimization can be disabled + with the `parquet.optimized-reader.enabled` catalog configuration property. ({issue}`14423`, {issue}`14667`) +* Improve performance when reading Parquet files. ({issue}`15257`, {issue}`15474`) +* Improve performance of reading Parquet files for queries with filters. ({issue}`15268`) +* Improve `DROP TABLE` performance for tables stored on AWS S3. ({issue}`13974`) +* Improve performance of reading Parquet files for `timestamp` and + `timestamp with timezone` data types. ({issue}`15204`) +* Improve performance of queries that read a small number of columns and queries + that process tables with large Parquet row groups or ORC stripes. ({issue}`15168`) +* Improve stability and reduce peak memory requirements when reading from + Parquet files. ({issue}`15374`) +* Fix incorrect results when predicates over `row` columns on Parquet files are + pushed into the connector. ({issue}`15408`) +* Fix query failure when reading Parquet files with large row groups. ({issue}`5729`) +* Fix `REFRESH MATERIALIZED VIEW` failure when the materialized view is based on + non-Iceberg tables. ({issue}`13131`) +* Fix failure when an access denied exception happens while listing tables or + views in a Glue metastore. ({issue}`14971`) +* Fix potential `INSERT` failure for tables stored on S3. ({issue}`15476`) + +## Kafka connector + +* Add support for [Protobuf encoding](kafka-protobuf-encoding). ({issue}`14734`) + +## MongoDB connector + +* Add support for [fault-tolerant execution](/admin/fault-tolerant-execution). ({issue}`15062`) +* Add support for setting a file path and password for the truststore and + keystore. ({issue}`15240`) +* Add support for case-insensitive name-matching in the `query` table function. ({issue}`15329`) +* Rename the `mongodb.ssl.enabled` configuration property to + `mongodb.tls.enabled`. ({issue}`15240`) +* Upgrade minimum required MongoDB version to + [4.2](https://www.mongodb.com/docs/manual/release-notes/4.2/). ({issue}`15062`) +* Delete a MongoDB field from collections when dropping a column. + Previously, the connector deleted only metadata. ({issue}`15226`) +* Remove deprecated `mongodb.seeds` and `mongodb.credentials` configuration + properties. ({issue}`15263`) +* Fix failure when an unauthorized exception happens while listing schemas or + tables. ({issue}`1398`) +* Fix `NullPointerException` when a column name contains uppercase characters in + the `query` table function. ({issue}`15294`) +* Fix potential incorrect results when the `objectid` function is used more than + once within a single query. ({issue}`15426`) + +## MySQL connector + +* Fix failure when the `query` table function contains a `WITH` clause. ({issue}`15332`) + +## PostgreSQL connector + +* Fix query failure when a `FULL JOIN` is pushed down. ({issue}`14841`) + +## Redshift connector + +* Add support for aggregation, join, and `ORDER BY ... LIMIT` pushdown. ({issue}`15365`) +* Add support for `DELETE`. ({issue}`15365`) +* Add schema, table, and column name length checks. ({issue}`15365`) +* Add full type mapping for Redshift types. The previous behavior can be + restored via the `redshift.use-legacy-type-mapping` configuration property. ({issue}`15365`) + +## SPI + +* Remove deprecated `ConnectorNodePartitioningProvider.getBucketNodeMap()` + method. ({issue}`14067`) +* Use the `MERGE` APIs in the engine to execute `DELETE` and `UPDATE`. + Require connectors to implement `beginMerge()` and related APIs. + Deprecate `beginDelete()`, `beginUpdate()` and `UpdatablePageSource`, which + are unused and do not need to be implemented. ({issue}`13926`) diff --git a/430/_sources/release/release-406.md.txt b/430/_sources/release/release-406.md.txt new file mode 100644 index 000000000..2c313f160 --- /dev/null +++ b/430/_sources/release/release-406.md.txt @@ -0,0 +1,117 @@ +# Release 406 (25 Jan 2023) + +## General + +* Add support for [exchange spooling on HDFS](fte-exchange-hdfs) when + fault-tolerant execution is enabled. ({issue}`15160`) +* Add support for `CHECK` constraints in an `INSERT` statement. ({issue}`14964`) +* Improve planner estimates for queries containing outer joins over a subquery + involving `ORDER BY` and `LIMIT`. ({issue}`15428`) +* Improve accuracy of memory usage reporting for table scans. ({issue}`15711`) +* Improve performance of queries parsing date values in ISO 8601 format. ({issue}`15548`) +* Improve performance of queries with selective joins. ({issue}`15569`) +* Remove `legacy-phased` execution scheduler as an option for the + `query.execution-policy` configuration property. ({issue}`15657`) +* Fix failure when `WHERE` or `JOIN` clauses contain a `LIKE` expression with a + non-constant pattern or escape. ({issue}`15629`) +* Fix inaccurate planner estimates for queries with filters on columns without + statistics. ({issue}`15642`) +* Fix queries with outer joins failing when fault-tolerant execution is + enabled. ({issue}`15608`) +* Fix potential query failure when using `MATCH_RECOGNIZE`. ({issue}`15461`) +* Fix query failure when using group-based access control with column masks or + row filters. ({issue}`15583`) +* Fix potential hang during shutdown. ({issue}`15675`) +* Fix incorrect results when referencing a field resulting from the application + of a column mask expression that produces a `row` type. ({issue}`15659`) +* Fix incorrect application of column masks when a mask expression references a + different column in the underlying table. ({issue}`15680`) + +## BigQuery connector + +* Add support for [fault-tolerant execution](/admin/fault-tolerant-execution). ({issue}`15620`) +* Fix possible incorrect results for certain queries like `count(*)` when a + table has recently been written to. ({issue}`14981`) + +## Cassandra connector + +* Fix incorrect results when the Cassandra `list`, `map`, or `set` types contain + user-defined types. ({issue}`15771`) + +## Delta Lake connector + +* Reduce latency for `INSERT` queries on unpartitioned tables. ({issue}`15708`) +* Improve performance of reading Parquet files. ({issue}`15498`) +* Improve memory accounting of the Parquet reader. ({issue}`15554`) +* Improve performance of queries with filters or projections on low-cardinality + string columns stored in Parquet files. ({issue}`15269`) +* Fix reading more data than necessary from Parquet files for queries with + filters. ({issue}`15552`) +* Fix potential query failure when writing to Parquet from a table with an + `INTEGER` range on a `BIGINT` column. ({issue}`15496`) +* Fix query failure due to missing null counts in Parquet column indexes. ({issue}`15706`) + +## Hive connector + +* Add support for table redirections to catalogs using the Hudi connector. ({issue}`14750`) +* Reduce latency for `INSERT` queries on unpartitioned tables. ({issue}`15708`) +* Improve performance of caching. ({issue}`13243 `) +* Improve performance of reading Parquet files. ({issue}`15498`) +* Improve memory accounting of the Parquet reader. ({issue}`15554`) +* Improve performance of queries with filters or projections on low-cardinality + string columns stored in Parquet files. ({issue}`15269`) +* Improve performance of queries with filters when Bloom filter indexes are + present in Parquet files. Use of Bloom filters from Parquet files can be + disabled with the `parquet.use-bloom-filter` configuration property or the + `parquet_use_bloom_filter` session property. ({issue}`14428`) +* Allow coercion between Hive `UNIONTYPE` and Hive `STRUCT`-typed columns. ({issue}`15017`) +* Fix reading more data than necessary from Parquet files for queries with + filters. ({issue}`15552`) +* Fix query failure due to missing null counts in Parquet column indexes. ({issue}`15706`) +* Fix incorrect `schema already exists` error caused by a client timeout when + creating a new schema. ({issue}`15174`) + +## Hudi connector + +* Improve performance of reading Parquet files. ({issue}`15498`) +* Improve memory accounting of the Parquet reader. ({issue}`15554`) +* Improve performance of queries with filters or projections on low-cardinality + string columns stored in Parquet files. ({issue}`15269`) +* Fix reading more data than necessary from Parquet files for queries with + filters. ({issue}`15552`) +* Fix query failure due to missing null counts in Parquet column indexes. ({issue}`15706`) + +## Iceberg connector + +* Add support for changing column types. ({issue}`15515`) +* Add [support for the JDBC catalog](iceberg-jdbc-catalog). ({issue}`9968`) +* Reduce latency for `INSERT` queries on unpartitioned tables. ({issue}`15708`) +* Improve performance of reading Parquet files. ({issue}`15498`) +* Improve memory accounting of the Parquet reader. ({issue}`15554`) +* Improve performance of queries with filters or projections on low-cardinality + string columns stored in Parquet files. ({issue}`15269`) +* Fix reading more data than necessary from Parquet files for queries with + filters. ({issue}`15552`) +* Fix query failure due to missing null counts in Parquet column indexes. ({issue}`15706`) +* Fix query failure when a subquery contains [time travel](iceberg-time-travel). ({issue}`15607`) +* Fix failure when reading columns that had their type changed from `float` to + `double` by other query engines. ({issue}`15650`) +* Fix incorrect results when reading or writing `NaN` with `real` or `double` + types on partitioned columns. ({issue}`15723`) + +## MongoDB connector + +* Fix schemas not being dropped when trying to drop schemas with the + `mongodb.case-insensitive-name-matching` configuration property enabled. ({issue}`15716`) + +## PostgreSQL connector + +* Add support for changing column types. ({issue}`15515`) + +## SPI + +* Remove the `getDeleteRowIdColumnHandle()`, `beginDelete()`, `finishDelete()`, + `getUpdateRowIdColumnHandle()`, `beginUpdate()`, and `finishUpdate()` methods + from `ConnectorMetadata`. ({issue}`15161`) +* Remove the `UpdatablePageSource` interface. ({issue}`15161`) +* Remove support for multiple masks on a single column. ({issue}`15680`) diff --git a/430/_sources/release/release-407.md.txt b/430/_sources/release/release-407.md.txt new file mode 100644 index 000000000..e1593daef --- /dev/null +++ b/430/_sources/release/release-407.md.txt @@ -0,0 +1,119 @@ +# Release 407 (16 Feb 2023) + +## General + +* Add support for correlated queries involving a `VALUES` clause with a single + row. ({issue}`15989`) +* Reduce memory usage for large schemas. This behavior can be enabled with the + `query.remote-task.enable-adaptive-request-size` configuration property and + configured with the `query.remote-task.max-request-size`, + `query.remote-task.request-size-headroom`, and + `query.remote-task.guaranteed-splits-per-task` configuration properties or + their respective session properties. ({issue}`15721`) +* Improve concurrency when small, concurrent queries are run on a large cluster + by automatically determining how many nodes to use for distributed joins + and aggregations. This can be managed with the `query.max-hash-partition-count` + (renamed from `query.hash-partition-count`) and `query.min-hash-partition-count` + configuration properties. ({issue}`15489`) +* Improve query memory tracking. ({issue}`15983`) +* Improve memory usage accounting for queries with dynamic filters. ({issue}`16110`) +* Improve query performance when a predicate evaluates to a null value. ({issue}`15744`) +* Improve performance of queries with joins on the output of global + aggregations. ({issue}`15858`) +* Improve performance of selective queries, queries that read a small number of + columns, and queries that process tables with large Parquet row groups or ORC + stripes. ({issue}`15579`) +* Improve performance of queries with window functions. ({issue}`15994`) +* Return an exit code of `100` when Trino crashes during startup. ({issue}`16113`) +* Fix precision loss when converting `time` values with a precision higher than + three and `time with time zone` values with lower precision. ({issue}`15861`) +* Fix potential incorrect results due to a query reporting normal completion + instead of failing. ({issue}`15917`) +* Fix connection errors caused by a reusable connection being closed. ({issue}`16121`) +* Fix incorrect results for queries involving an equality predicate in a `WHERE` + clause that is equal to a term of a `SELECT` clause in one of the branches of + a `JOIN`. ({issue}`16101`) + +## Cassandra connector + +* Add `query` table function for full query pass-through to the connector. ({issue}`15973`) + +## Delta Lake connector + +* Add support for the `unregister_table` procedure. ({issue}`15784`) +* Add support for inserting into tables that have `CHECK` constraints. ({issue}`15396`) +* Add support for writing to the [change data feed](https://docs.delta.io/2.0.0/delta-change-data-feed.html). + This can be enabled with the `delta.enableChangeDataFeed` table property. ({issue}`15453`) +* Add a `$history` system table which can be queried to inspect Delta Lake table + history. ({issue}`15683`) +* Improve performance of reading decimal types from Parquet files. ({issue}`15713`) +* Improve performance of reading numeric types from Parquet files. ({issue}`15850`) +* Improve performance of reading string types from Parquet files. ({issue}`15897`, {issue}`15923`) +* Improve performance of reading timestamp and boolean types from Parquet files. ({issue}`15954`) +* Improve query performance on tables created by Trino with `CREATE TABLE AS`. ({issue}`15878`) +* Remove support for the legacy Parquet writer. ({issue}`15436`) +* Fix query failure when reading Parquet files written by Apache Impala. ({issue}`15942`) +* Fix listing relations failure when a Glue table has no table type set. ({issue}`15909`) + +## Hive connector + +* Reduce query latency. ({issue}`15811`) +* Improve performance of reading decimal types from Parquet files. ({issue}`15713`) +* Improve performance of reading numeric types from Parquet files. ({issue}`15850`) +* Improve performance of reading string types from Parquet files. ({issue}`15897`, {issue}`15923`) +* Improve performance of reading timestamp and boolean types from Parquet files. ({issue}`15954`) +* Improve performance of predicate pushdown to partitioned columns in tables + with a high number of partitions. ({issue}`16113`) +* Reduce server errors in high-load scenarios. This can be enabled with the + `hive.s3.connect-ttl` configuration property. ({issue}`16005`) +* Allow setting the `hive.max-partitions-per-scan` configuration property to a + value lower than the value set in `hive.max-partitions-for-eager-load`. ({issue}`16111`) +* Fix query failure when reading Parquet files written by Apache Impala. ({issue}`15942`) +* Fix listing relations failure when a Glue table has no table type set. ({issue}`15909`) + +## Hudi connector + +* Improve performance of reading decimal types from Parquet files. ({issue}`15713`) +* Improve performance of reading numeric types from Parquet files. ({issue}`15850`) +* Improve performance of reading string types from Parquet files. ({issue}`15897`, {issue}`15923`) +* Improve performance of reading timestamp and boolean types from Parquet files. ({issue}`15954`) +* Fix query failure when reading Parquet files written by Apache Impala. ({issue}`15942`) + +## Iceberg connector + +* Add support for the `unregister_table` procedure. ({issue}`15784`) +* Add support for `register_table` procedures in the JDBC catalog. ({issue}`15853`) +* Add support for specifying a user and password when connecting to the JDBC + catalog via the `iceberg.jdbc-catalog.connection-user` and + `iceberg.jdbc-catalog.connection-password` configuration properties. ({issue}`16040`) +* Add support for compacting manifests asynchronously, which can be enabled by + setting the `iceberg.merge_manifests_on_write` session property to `false`. ({issue}`14822`) +* Improve performance of `DROP TABLE`. ({issue}`15981`) +* Improve performance of reading [position delete files](https://iceberg.apache.org/spec/#position-delete-files) + with ORC data ({issue}`15969`). +* Improve performance of reading decimal columns from Parquet files. ({issue}`15713`) +* Improve performance of reading numeric types from Parquet files. ({issue}`15850`) +* Improve performance of reading string types from Parquet files. ({issue}`15897`, {issue}`15923`) +* Improve performance of reading timestamp and boolean types from Parquet files. ({issue}`15954`) +* Prevent creating a table when the specified schema does not exist. ({issue}`15779`) +* Fix query failure when reading Parquet files written by Apache Impala. ({issue}`15942`) +* Fix listing relations failure when a Glue table has no table type set. ({issue}`15909`) +* Fix failure when encountering access denied exceptions while listing + materialized views in the Glue metastore. ({issue}`15893`) + +## Kudu connector + +* Fix authentication failure when Kerberos tickets expire. ({issue}`14372`) + +## Memory connector + +* Fix potential failure when reading table column metadata with concurrent + `CREATE TABLE` or `DROP TABLE` operations. ({issue}`16062`) + +## MongoDB connector + +* Add support for changing column types. ({issue}`15515`) + +## MySQL connector + +* Fix potential failure when `zeroDateTimeBehavior` is set to `convertToNull`. ({issue}`16027`) diff --git a/430/_sources/release/release-408.md.txt b/430/_sources/release/release-408.md.txt new file mode 100644 index 000000000..bb9559273 --- /dev/null +++ b/430/_sources/release/release-408.md.txt @@ -0,0 +1,80 @@ +# Release 408 (23 Feb 2023) + +## General + +* Add physical input read time to query statistics and the output of `EXPLAIN + ANALYZE`. ({issue}`16190`) +* Fix query failure for queries involving joins or aggregations with a + [structural type](structural-data-types) that contains `NULL` elements. ({issue}`16140`) + +## Security + +* Deprecate using groups with OAuth 2.0 authentication, and rename the + `http-server.authentication.oauth2.groups-field` configuration property to + `deprecated.http-server.authentication.oauth2.groups-field`. ({issue}`15669`) + +## CLI + +* Add `AUTO` output format which switches from `ALIGNED` to `VERTICAL` if + the output doesn't fit the current terminal. ({issue}`12208`) +* Add `--pager` and `--history-file` options to match the existing `TRINO_PAGER` + and `TRINO_HISTORY_FILE` environmental variables. Also allow setting these + options in a configuration file. ({issue}`16151`) + +## BigQuery connector + +* Add support for writing `decimal` types to BigQuery. ({issue}`16145`) + +## Delta Lake connector + +* Rename the connector to `delta_lake`. The old name `delta-lake` is now + deprecated and will be removed in a future release. ({issue}`13931`) +* Add support for creating tables with the Trino `change_data_feed_enabled` + table property. ({issue}`16129`) +* Improve query performance on tables that Trino has written to with `INSERT`. ({issue}`16026`) +* Improve performance of reading [structural types](structural-data-types) from + Parquet files. This optimization can be disabled with the + `parquet_optimized_nested_reader_enabled` catalog session property or the + `parquet.optimized-nested-reader.enabled` catalog configuration property. ({issue}`16177`) +* Retry dropping Delta tables registered in the Glue catalog to avoid failures + due to concurrent modifications. ({issue}`13199`) +* Allow updating the `reader_version` and `writer_version` table properties. ({issue}`15932`) +* Fix inaccurate change data feed entries for `MERGE` queries. ({issue}`16127`) +* Fix performance regression when writing to partitioned tables if table + statistics are absent. ({issue}`16152`) + +## Hive connector + +* Remove support for the deprecated `hive-hadoop2` connector name, requiring the + `connector.name` property to be set to `hive`. ({issue}`16166`) +* Retry dropping Delta tables registered in the Glue catalog to avoid failures + due to concurrent modifications. ({issue}`13199`) +* Fix performance regression when writing to partitioned tables if table + statistics are absent. ({issue}`16152`) + +## Iceberg connector + +* Reduce memory usage when reading `$files` system tables. ({issue}`15991`) +* Require the `iceberg.jdbc-catalog.driver-class` configuration property to be + set to prevent a "driver not found" error after initialization. ({issue}`16196`) +* Fix performance regression when writing to partitioned tables if table + statistics are absent. ({issue}`16152`) + +## Ignite connector + +* Add [Ignite connector](/connector/ignite). ({issue}`8098`) + +## SingleStore connector + +* Remove support for the deprecated `memsql` connector name, requiring the + `connector.name` property to be set to `singlestore`. ({issue}`16180`) + +## SQL Server connector + +* Add support for pushing down `=`, `<>` and `IN` predicates over text columns + if the column uses a case-sensitive collation within SQL Server. ({issue}`15714`) + +## Thrift connector + +* Rename the connector to `trino_thrift`. The old name `trino-thrift` is now + deprecated and will be removed in a future release. ({issue}`13931`) diff --git a/430/_sources/release/release-409.md.txt b/430/_sources/release/release-409.md.txt new file mode 100644 index 000000000..387676768 --- /dev/null +++ b/430/_sources/release/release-409.md.txt @@ -0,0 +1,96 @@ +# Release 409 (3 Mar 2023) + +## General + +* Add support for dropping nested fields with a `DROP COLUMN` statement. ({issue}`15975`) +* Add queue, analysis, planning, and execution time to the output of + `EXPLAIN ANALYZE`. ({issue}`16329`) +* Add support for executing table functions with any number of table arguments. ({issue}`1839`) +* Improve output of `EXPLAIN` queries to show statistics when a query contains + aggregations. ({issue}`16201`) +* Improve performance of queries with aggregations containing a `DISTINCT` + clause using table statistics. This can be configured with the + `optimizer.mark-distinct-strategy`configuration property, and the + `optimizer.use-mark-distinct` configuration property is now deprecated in + favor of it. ({issue}`15927`) +* Improve performance of LIKE expressions with patterns constructed dynamically. ({issue}`15999`) +* Remove the `distributed-index-joins-enabled` configuration property and + related functionality. ({issue}`15375`) +* Fix failure when using non-comparable and non-sortable values as part of a + `VALUES` expression. ({issue}`16242`) + +## BigQuery connector + +* Add support for using default values when inserting data. ({issue}`16327`) +* Fix failure when non-lowercase column names exist in the `query` table + function. ({issue}`16075`) + +## Cassandra connector + +* Add support for `TIME` type. ({issue}`13063`) + +## ClickHouse connector + +* Remove support for the `ru.yandex.clickhouse.ClickHouseDriver` legacy JDBC + driver in the `clickhouse.legacy-driver` configuration property. ({issue}`16188`) +* Remove support for specifying expressions in the `sample_by` table property to + prevent SQL injection. ({issue}`16261`) + +## Delta Lake connector + +* Avoid query failure by inferring required Delta Lake version when creating new + tables or configuring table features. ({issue}`16310`) +* Fix query failure when reading Parquet files generated by Kafka Connect. ({issue}`16264`) + +## Hive connector + +* Add support for the Hadoop `DefaultCodec` to Hive formats. ({issue}`16250`) +* Add a native CSV file format reader and writer. These can be disabled with the + `csv_native_reader_enabled` and `csv_native_writer_enabled` session properties + or the `csv.native-reader.enabled` and `csv.native-writer.enabled` + configuration properties. ({issue}`15918`) +* Add a native JSON file format reader and writer. These can be disabled with + the `json_native_reader_enabled` and `json_native_writer_enabled` session + properties or the `json.native-reader.enabled` and + `json.native-writer.enabled` configuration properties. ({issue}`15918`) +* Add a native text file format reader and writer. These can be disabled with + the `text_file_native_reader_enabled` and `text_file_native_writer_enabled` + session properties or the `text-file.native-reader.enabled` and + `text-file.native-writer.enabled` configuration properties. ({issue}`15918`) +* Add a native sequence file format reader and writer. These can be disabled + with the `sequence_file_native_reader_enabled` and + `sequence_file_native_writer_enabled` session properties or the + `sequence-file.native-reader.enabled` and + `sequence-file.native-writer.enabled` configuration properties. ({issue}`15918`) +* Add a native regex file format reader. The reader can be disabled with the + `regex_native_reader_enabled` session property or the + `regex.native-reader.enabled` configuration property. ({issue}`15918`) +* Add `regex` and `regex_case_insensitive` table properties for the `REGEX` + format. ({issue}`16271`) +* Improve performance of queries which read from partitioned Hive tables and + write to partitioned tables when statistics are not available for the source + tables. ({issue}`16229`) +* Improve query performance when only table statistics generated by Apache Spark + are available. This can be disabled via the + `hive.metastore.thrift.use-spark-table-statistics-fallback` configuration + property. ({issue}`16120`) +* Fix incorrectly ignoring computed table statistics in `ANALYZE`. ({issue}`15995`) +* Fix query failure when reading Parquet files generated by Kafka Connect. ({issue}`16264`) + +## Hudi connector + +* Fix query failure when reading Parquet files generated by Kafka Connect. ({issue}`16264`) + +## Iceberg connector + +* Add support for dropping nested fields with a `DROP COLUMN` statement. ({issue}`15975`) +* Add support for Iceberg table sort orders. Tables can have a list of + `sorted_by` columns which are used to order files written to the table. ({issue}`14891`) +* Fix query failure when reading nested columns on a table with + [equality delete files](https://iceberg.apache.org/spec/#equality-delete-files). ({issue}`14836`) +* Fix query failure when reading Parquet files generated by Kafka Connect. ({issue}`16264`) + +## SQL Server connector + +* Add support for pushing down joins using `=` and `!=` predicates over text + columns if the column uses a case-sensitive collation within SQL Server. ({issue}`16185`) diff --git a/430/_sources/release/release-410.md.txt b/430/_sources/release/release-410.md.txt new file mode 100644 index 000000000..92bad2e87 --- /dev/null +++ b/430/_sources/release/release-410.md.txt @@ -0,0 +1,40 @@ +# Release 410 (8 Mar 2023) + +## General + +* Improve performance of certain queries with unions. ({issue}`16019`) +* Fix potential query failure when using an `ORDER BY` clause while spilling to + disk is enabled. ({issue}`16406`) +* Fix potential query failure when performing a `JOIN` with a comparison between + two different types. ({issue}`16444`) + +## Delta Lake connector + +* Fix query failure when reading Parquet files generated by Airbyte. ({issue}`16362`) + +## Google Sheets connector + +* Add support for the [`sheet` table function](google-sheets-sheet-function). ({issue}`12502`) + +## Hive connector + +* Add support for AWS Security Token Support with S3 Select. ({issue}`5947`) +* Fix incorrectly ignoring the `hive.timestamp-precision` configuration property + in Hive views. ({issue}`6295`) +* Fix query failure when reading Parquet files generated by Airbyte. ({issue}`16362`) +* Fix incorrect results when using a value with uppercase characters for the + `partition_projection_location_template` table property. ({issue}`16349`) + +## Hudi connector + +* Fix query failure when reading Parquet files generated by Airbyte. ({issue}`16362`) + +## Iceberg connector + +* Improve file pruning when generating Iceberg table statistics. ({issue}`16244`) +* Fix query failure when reading Parquet files generated by Airbyte. ({issue}`16362`) + +## Ignite connector + +* Add support for [aggregation pushdown](aggregation-pushdown) of the + `count(distinct)` function. ({issue}`16342`) diff --git a/430/_sources/release/release-411.md.txt b/430/_sources/release/release-411.md.txt new file mode 100644 index 000000000..44b7671d7 --- /dev/null +++ b/430/_sources/release/release-411.md.txt @@ -0,0 +1,145 @@ +# Release 411 (29 Mar 2023) + +## General + +* Add spilled data size to query statistics. ({issue}`16442`) +* Add {func}`sinh` function. ({issue}`16494`) +* Add {func}`quantile_at_value` function. ({issue}`16736`) +* Add support for a `GRACE PERIOD` clause in the `CREATE MATERIALIZED VIEW` + task. For backwards compatibility, the existing materialized views are + interpreted as having a `GRACE PERIOD` of zero, however, new materialized + views have an unlimited grace period by default. This is a backwards + incompatible change, and the previous behavior can be restored with the + `legacy.materialized-view-grace-period` configuration property or the + `legacy_materialized_view_grace_period` session property. ({issue}`15842`) +* Fix potential incorrect query stats when tasks are waiting on running drivers + to fully terminate. ({issue}`15478`) +* Add support for specifying the number of nodes that will write data during + `INSERT`, `CREATE TABLE ... AS SELECT`, or `EXECUTE` queries with the + `query.max-writer-tasks-count` configuration property. ({issue}`16238`) +* Improve performance of queries that contain predicates involving the `year` + function. ({issue}`14078`) +* Improve performance of queries that contain a `sum` aggregation. ({issue}`16624`) +* Improve performance of `filter` function on arrays. ({issue}`16681`) +* Reduce coordinator memory usage. ({issue}`16668`, {issue}`16669`) +* Reduce redundant data exchanges for queries with multiple aggregations. ({issue}`16328`) +* Fix incorrect query results when using `keyvalue()` methods in the + [JSON path](json-path-language). ({issue}`16482`) +* Fix potential incorrect results in queries involving joins and a + non-deterministic value. ({issue}`16512`) +* Fix potential query failure when exchange compression is enabled. ({issue}`16541`) +* Fix query failure when calling a function with a large number of parameters. ({issue}`15979`) + +## BigQuery connector + +* Fix failure of aggregation queries when executed against a materialized view, + external table, or snapshot table. ({issue}`15546`) + +## Delta Lake connector + +* Add support for inserting into tables that have + [simple invariants](https://github.com/delta-io/delta/blob/master/PROTOCOL.md#column-invariants). ({issue}`16136`) +* Add [generated column expressions](https://docs.delta.io/latest/delta-batch.html#use-generated-columns) + to the `Extra` column in the results of `DESCRIBE` and `SHOW COLUMNS`. ({issue}`16631`) +* Expand the `flush_metadata_cache` table procedure to also flush the internal + caches of table snapshots and active data files. ({issue}`16466`) +* Collect statistics for newly-created columns. ({issue}`16109`) +* Remove the `$data` system table. ({issue}`16650`) +* Fix query failure when evaluating a `WHERE` clause on a partition column. ({issue}`16388`) + +## Druid connector + +* Fix failure when the query passed to the `query` table function contains a + column alias. ({issue}`16225`) + +## Elasticsearch connector + +* Remove the deprecated pass-through query, which has been replaced with the + `raw_query` table function. ({issue}`13050`) + +## Hive connector + +* Add a native OpenX JSON file format reader and writer. These can be disabled + with the `openx_json_native_reader_enabled` and + `openx_json_native_writer_enabled` session properties or the + `openx-json.native-reader.enabled` and `openx-json.native-writer.enabled` + configuration properties. ({issue}`16073`) +* Add support for implicit coercions between `char` types of different lengths. ({issue}`16402`) +* Improve performance of queries with joins where both sides of a join have keys + with the same table bucketing definition. ({issue}`16381`) +* Improve query planning performance for queries scanning tables with a large + number of columns. ({issue}`16203`) +* Improve scan performance for `COUNT(*)` queries on row-oriented formats. ({issue}`16595`) +* Ensure the value of the `hive.metastore-stats-cache-ttl` configuration + property always is greater than or equal to the value specified in the + `hive.metastore-cache-ttl` configuration property. ({issue}`16625`) +* Skip listing Glue metastore tables with invalid column types. ({issue}`16677`) +* Fix query failure when a file that is using a text file format with a + single header row that is large enough to be split into multiple files. ({issue}`16492`) +* Fix potential query failure when Kerberos is enabled and the query execution + takes longer than a Kerberos ticket's lifetime. ({issue}`16680`) + +## Hudi connector + +* Add a `$timeline` system table which can be queried to inspect the Hudi table + timeline. ({issue}`16149`) + +## Iceberg connector + +* Add a `migrate` procedure that converts a Hive table to an Iceberg table. ({issue}`13196`) +* Add support for materialized views with a freshness grace period. ({issue}`15842`) +* Add a `$refs` system table which can be queried to inspect snapshot + references. ({issue}`15649`) +* Add support for creation of materialized views partitioned with a temporal + partitioning function on a `timestamp with time zone` column. ({issue}`16637`) +* Improve performance of queries run after data was written by Trino. ({issue}`15441`) +* Remove the `$data` system table. ({issue}`16650`) +* Fix failure when the `$files` system table contains non-null values in the + `key_metadata`, `split_offsets`, and `equality_ids` columns. ({issue}`16473`) +* Fix failure when partitioned column names contain uppercase characters. ({issue}`16622`) + +## Ignite connector + +* Add support for predicate pushdown with a `LIKE` clause. ({issue}`16396`) +* Add support for pushdown of joins. ({issue}`16428`) +* Add support for {doc}`/sql/delete`. ({issue}`16720`) + +## MariaDB connector + +* Fix failure when the query passed to the `query` table function contains a + column alias. ({issue}`16225`) + +## MongoDB connector + +* Fix incorrect results when the query passed to the MongoDB `query` table + function contains helper functions such as `ISODate`. ({issue}`16626`) + +## MySQL connector + +* Fix failure when the query passed to the `query` table function contains a + column alias. ({issue}`16225`) + +## Oracle connector + +* Improve performance of queries when the network latency between Trino and + Oracle is high, or when selecting a small number of columns. ({issue}`16644`) + +## PostgreSQL connector + +* Improve performance of queries when the network latency between Trino and + PostgreSQL is high, or when selecting a small number of columns. ({issue}`16644`) + +## Redshift connector + +* Improve performance of queries when the network latency between Trino and + Redshift is high, or when selecting a small number of columns. ({issue}`16644`) + +## SingleStore connector + +* Fix failure when the query passed to the `query` table function contains a + column alias. ({issue}`16225`) + +## SQL Server connector + +* Add support for executing stored procedures using the `procedure` table + function. ({issue}`16696`) diff --git a/430/_sources/release/release-412.md.txt b/430/_sources/release/release-412.md.txt new file mode 100644 index 000000000..4c9030d8c --- /dev/null +++ b/430/_sources/release/release-412.md.txt @@ -0,0 +1,85 @@ +# Release 412 (5 Apr 2023) + +## General + +* Add support for aggregate functions and parameters as arguments for the + [`json_object()`](json-object) and [`json_array()`](json-array) + functions. ({issue}`16489`, {issue}`16523`, {issue}`16525`) +* Expose optimizer rule execution statistics in query statistics. The number of + rules for which statistics are collected can be limited with the + `query.reported-rule-stats-limit` configuration property. ({issue}`2578`) +* Add the {func}`exclude_columns` table function. ({issue}`16584`) +* Allow disabling the use of the cost-based optimizer to determine partitioning + of a stage with the `optimizer.use-cost-based-partitioning`configuration + property or the `use_cost_based_partitioning` session property. ({issue}`16781`) +* Improve performance of queries involving table functions with table arguments. ({issue}`16012`) +* Improve latency for small queries when fault-tolerant execution is enabled. ({issue}`16103`) +* Fix failure when querying a nested field of a `row` type in queries involving + `ORDER BY ... LIMIT`. ({issue}`16768`) + +## JDBC driver + +* Allow configuring a custom DNS resolver. ({issue}`16647`) + +## ClickHouse connector + +* Improve performance of queries involving `sum(DISTINCT ...)` by pushing + computation down to ClickHouse. ({issue}`16452`) + +## Delta Lake connector + +* Add support for arithmetic binary expressions in table check constraints. ({issue}`16721`) +* Improve performance of queries that only read partition columns. ({issue}`16788`) + +## Hive connector + +* Fix query failure when bucketing or sorting column names are registered in + a metastore in uppercase. ({issue}`16796`) +* Fix query failure when reading transactional tables with locations containing + hidden directories. ({issue}`16773`) + +## Iceberg connector + +* Fix incorrect results for the `migrate` procedure when the table location + contains a hidden directory. ({issue}`16779`) + +## Ignite connector + +* Add support for `ALTER TABLE ... ADD COLUMN`. ({issue}`16755`) +* Improve performance of queries involving `sum(DISTINCT ...)` by pushing + computation to Ignite. ({issue}`16452`) + +## MariaDB connector + +* Improve performance of queries involving `sum(DISTINCT ...)` by pushing + computation down to MariaDB. ({issue}`16452`) + +## MySQL connector + +* Improve performance of queries involving `sum(DISTINCT ...)` by pushing + computation down to MySQL. ({issue}`16452`) + +## Oracle connector + +* Improve performance of queries involving `sum(DISTINCT ...)` by pushing + computation down to Oracle. ({issue}`16452`) + +## PostgreSQL connector + +* Add support for [table comments](/sql/comment). ({issue}`16135`) +* Improve performance of queries involving `sum(DISTINCT ...)` by pushing + computation down to PostgreSQL. ({issue}`16452`) + +## Redshift connector + +* Improve performance of queries involving `sum(DISTINCT ...)` by pushing + computation down to Redshift. ({issue}`16452`) + +## SQL Server connector + +* Improve performance of queries involving `sum(DISTINCT ...)` by pushing + computation down to SQL Server. ({issue}`16452`) + +## SPI + +* Allow table functions to return anonymous columns. ({issue}`16584`) diff --git a/430/_sources/release/release-413.md.txt b/430/_sources/release/release-413.md.txt new file mode 100644 index 000000000..f32718276 --- /dev/null +++ b/430/_sources/release/release-413.md.txt @@ -0,0 +1,55 @@ +# Release 413 (12 Apr 2023) + +## General + +* Improve performance of queries involving window operations or + [row pattern recognition](/sql/pattern-recognition-in-window) on small + partitions. ({issue}`16748`) +* Improve performance of queries with the {func}`row_number` and {func}`rank` + window functions. ({issue}`16753`) +* Fix potential failure when cancelling a query. ({issue}`16960`) + +## Delta Lake connector + +* Add support for nested `timestamp with time zone` values in + [structural data types](structural-data-types). ({issue}`16826`) +* Disallow using `_change_type`, `_commit_version`, and `_commit_timestamp` as + column names when creating a table or adding a column with + [change data feed](https://docs.delta.io/2.0.0/delta-change-data-feed.html). ({issue}`16913`) +* Disallow enabling change data feed when the table contains + `_change_type`, `_commit_version` and `_commit_timestamp` columns. ({issue}`16913`) +* Fix incorrect results when reading `INT32` values without a decimal logical + annotation in Parquet files. ({issue}`16938`) + +## Hive connector + +* Fix incorrect results when reading `INT32` values without a decimal logical + annotation in Parquet files. ({issue}`16938`) +* Fix incorrect results when the file path contains hidden characters. ({issue}`16386`) + +## Hudi connector + +* Fix incorrect results when reading `INT32` values without a decimal logical + annotation in Parquet files. ({issue}`16938`) + +## Iceberg connector + +* Fix incorrect results when reading `INT32` values without a decimal logical + annotation in Parquet files. ({issue}`16938`) +* Fix failure when creating a schema with a username containing uppercase + characters in the Iceberg Glue catalog. ({issue}`16116`) + +## Oracle connector + +* Add support for [table comments](/sql/comment) and creating tables with + comments. ({issue}`16898`) + +## Phoenix connector + +* Add support for {doc}`/sql/merge`. ({issue}`16661`) + +## SPI + +* Deprecate the `getSchemaProperties()` and `getSchemaOwner()` methods in + `ConnectorMetadata` in favor of versions that accept a `String` for the schema + name rather than `CatalogSchemaName`. ({issue}`16862`) diff --git a/430/_sources/release/release-414.md.txt b/430/_sources/release/release-414.md.txt new file mode 100644 index 000000000..c6bb51b85 --- /dev/null +++ b/430/_sources/release/release-414.md.txt @@ -0,0 +1,61 @@ +# Release 414 (19 Apr 2023) + +## General + +* Add [recursive member access](json-descendant-member-accessor) to the + [JSON path language](json-path-language). ({issue}`16854`) +* Add the [`sequence()`](built-in-table-functions) table function. ({issue}`16716`) +* Add support for progress estimates when + [fault-tolerant execution](/admin/fault-tolerant-execution) is enabled. ({issue}`13072`) +* Add support for `CUBE` and `ROLLUP` with composite sets. ({issue}`16981`) +* Add experimental support for tracing using [OpenTelemetry](https://opentelemetry.io/). + This can be enabled by setting the `tracing.enabled` configuration property to + `true` and optionally configuring the + [OLTP/gRPC endpoint](https://opentelemetry.io/docs/reference/specification/protocol/otlp/) + by setting the `tracing.exporter.endpoint` configuration property. ({issue}`16950`) +* Improve performance for certain queries that produce no values. ({issue}`15555`, {issue}`16515`) +* Fix query failure for recursive queries involving lambda expressions. ({issue}`16989`) +* Fix incorrect results when using the {func}`sequence` function with values + greater than 231 (about 2.1 billion). ({issue}`16742`) + +## Security + +* Disallow [graceful shutdown](/admin/graceful-shutdown) with the `default` + [system access control](/security/built-in-system-access-control). Shutdowns + can be re-enabled by using the `allow-all` system access control, or by + configuring [system information rules](system-file-auth-system-information) + with the `file` system access control. ({issue}`17105`) + +## Delta Lake connector + +* Add support for `INSERT`, `UPDATE`, and `DELETE` operations on + tables with a `name` column mapping. ({issue}`12638`) +* Add support for [Databricks 12.2 LTS](https://docs.databricks.com/release-notes/runtime/12.2.html). ({issue}`16905`) +* Disallow reading tables with [deletion vectors](https://github.com/delta-io/delta/blob/master/PROTOCOL.md#deletion-vectors). + Previously, this returned incorrect results. ({issue}`16884`) + +## Iceberg connector + +* Add support for Hive external tables in the `migrate` table procedure. ({issue}`16704`) + +## Kafka connector + +* Fix query failure when a Kafka topic contains tombstones (messages with a + ``NULL`` value). ({issue}`16962`) + +## Kudu connector + +* Fix query failure when merging two tables that were created by + `CREATE TABLE ... AS SELECT ...`. ({issue}`16848`) + +## Pinot connector + +* Fix incorrect results due to incorrect pushdown of aggregations. ({issue}`12655`) + +## PostgreSQL connector + +* Fix failure when fetching table statistics for PostgreSQL 14.0 and later. ({issue}`17061`) + +## Redshift connector + +* Add support for [fault-tolerant execution](/admin/fault-tolerant-execution). ({issue}`16860`) diff --git a/430/_sources/release/release-415.md.txt b/430/_sources/release/release-415.md.txt new file mode 100644 index 000000000..66cfe3167 --- /dev/null +++ b/430/_sources/release/release-415.md.txt @@ -0,0 +1,36 @@ +# Release 415 (28 Apr 2023) + +## General + +* Improve performance of aggregations with variable file sizes. ({issue}`11361`) +* Perform missing permission checks for table arguments to table functions. ({issue}`17279`) + +## Web UI + +* Add CPU planning time to the query details page. ({issue}`15318`) + +## Delta Lake connector + +* Add support for commenting on tables and columns with an `id` and `name` + column mapping mode. ({issue}`17139`) +* Add support for `BETWEEN` predicates in table check constraints. ({issue}`17120`) + +## Hive connector + +* Improve performance of queries with selective filters on primitive fields in + `row` columns. ({issue}`15163`) + +## Iceberg connector + +* Improve performance of queries with filters when Bloom filter indexes are + present in Parquet files. ({issue}`17192`) +* Fix failure when trying to use `DROP TABLE` on a corrupted table. ({issue}`12318`) + +## Kafka connector + +* Add support for Protobuf `oneof` types when using the Confluent table + description provider. ({issue}`16836`) + +## SPI + +* Expose ``planningCpuTime`` in ``QueryStatistics``. ({issue}`15318`) diff --git a/430/_sources/release/release-416.md.txt b/430/_sources/release/release-416.md.txt new file mode 100644 index 000000000..224bced24 --- /dev/null +++ b/430/_sources/release/release-416.md.txt @@ -0,0 +1,14 @@ +# Release 416 (3 May 2023) + +## General + +* Improve performance of partitioned `INSERT`, `CREATE TABLE AS .. SELECT`, and + `EXECUTE` statements when the source table statistics are missing or + inaccurate. ({issue}`16802`) +* Improve performance of `LIKE` expressions that contain `%`. ({issue}`16167`) +* Remove the deprecated `preferred-write-partitioning-min-number-of-partitions` + configuration property. ({issue}`16802`) + +## Hive connector + +* Reduce coordinator memory usage when file metadata caching is enabled. ({issue}`17270`) diff --git a/430/_sources/release/release-417.md.txt b/430/_sources/release/release-417.md.txt new file mode 100644 index 000000000..a8cc21344 --- /dev/null +++ b/430/_sources/release/release-417.md.txt @@ -0,0 +1,33 @@ +# Release 417 (10 May 2023) + +## General + +* Improve performance of `UNION ALL` queries. ({issue}`17265`) + +## Delta Lake connector + +* Add support for [`COMMENT ON VIEW`](/sql/comment). ({issue}`17089`) +* Improve performance when reading Parquet data written by Trino. ({issue}`17373`, {issue}`17404`) +* Improve read performance for tables with `row` columns when only a subset of + fields is needed for a query. ({issue}`17085`) + +## Hive connector + +* Add support for specifying arbitrary table properties via the + `extra_properties` table property. ({issue}`954`) +* Improve performance when reading Parquet data written by Trino. ({issue}`17373`, {issue}`17404`) +* Improve performance when reading text files that contain more columns in the + file than are mapped in the schema. ({issue}`17364`) +* Limit file listing cache based on in-memory size instead of number of entries. + This is configured via the `hive.file-status-cache.max-retained-size` and + `hive.per-transaction-file-status-cache.max-retained-size` configuration + properties. The `hive.per-transaction-file-status-cache-maximum-size` and + `hive.file-status-cache-size` configuration properties are deprecated. ({issue}`17285`) + +## Hudi connector + +* Improve performance when reading Parquet data written by Trino. ({issue}`17373`, {issue}`17404`) + +## Iceberg connector + +* Improve performance when reading Parquet data written by Trino. ({issue}`17373`, {issue}`17404`) diff --git a/430/_sources/release/release-418.md.txt b/430/_sources/release/release-418.md.txt new file mode 100644 index 000000000..a41eb6237 --- /dev/null +++ b/430/_sources/release/release-418.md.txt @@ -0,0 +1,62 @@ +# Release 418 (17 May 2023) + +## General + +* Add support for [EXECUTE IMMEDIATE](/sql/execute-immediate). ({issue}`17341`) +* Fix failure when invoking `current_timestamp`. ({issue}`17455`) + +## BigQuery connector + +* Add support for adding labels to BigQuery jobs started by Trino as part of + query processing. The name and value of the label can be configured via the + `bigquery.job.label-name` and `bigquery.job.label-format` catalog + configuration properties, respectively. ({issue}`16187`) + +## Delta Lake connector + +* Add support for `INSERT`, `UPDATE`, `DELETE`, and `MERGE` statements for + tables with an `id` column mapping. ({issue}`16600`) +* Add the `table_changes` table function. ({issue}`16205`) +* Improve performance of joins on partition columns. ({issue}`14493`) + +## Hive connector + +* Improve performance of querying `information_schema.tables` when using the + Hive metastore. ({issue}`17127`) +* Improve performance of joins on partition columns. ({issue}`14493`) +* Improve performance of writing Parquet files by enabling the optimized Parquet + writer by default. ({issue}`17393`) +* Remove the `temporary_staging_directory_enabled` and + `temporary_staging_directory_path` session properties. ({issue}`17390`) +* Fix failure when querying text files in S3 if the native reader is enabled. ({issue}`16546`) + +## Hudi connector + +* Improve performance of joins on partition columns. ({issue}`14493`) + +## Iceberg connector + +* Improve planning time for `SELECT` queries. ({issue}`17347`) +* Improve performance of joins on partition columns. ({issue}`14493`) +* Fix incorrect results when querying the `$history` table if the REST catalog + is used. ({issue}`17470`) + +## Kafka connector + +* Fix query failure when a Kafka key or message cannot be de-serialized, and + instead correctly set the `_key_corrupt` and `_message_corrupt` columns. ({issue}`17479`) + +## Kinesis connector + +* Fix query failure when a Kinesis message cannot be de-serialized, and + instead correctly set the `_message_valid` column. ({issue}`17479`) + +## Oracle connector + +* Add support for writes when [fault-tolerant + execution](/admin/fault-tolerant-execution) is enabled. ({issue}`17200`) + +## Redis connector + +* Fix query failure when a Redis key or value cannot be de-serialized, and + instead correctly set the `_key_corrupt` and `_value_corrupt` columns. ({issue}`17479`) diff --git a/430/_sources/release/release-419.md.txt b/430/_sources/release/release-419.md.txt new file mode 100644 index 000000000..6235d55b0 --- /dev/null +++ b/430/_sources/release/release-419.md.txt @@ -0,0 +1,64 @@ +# Release 419 (5 Jun 2023) + +## General + +* Add the {func}`array_histogram` function to find the number of occurrences of + the unique elements in an array. ({issue}`14725 `) +* Improve planning performance for queries involving joins. ({issue}`17458`) +* Fix query failure when the server JSON response exceeds the 5MB limit for + string values. ({issue}`17557`) + +## Web UI + +* Allow uppercase or mixed case values for the `web-ui.authentication.type` + configuration property. ({issue}`17334`) + +## BigQuery connector + +* Add support for proxying BigQuery APIs via an HTTP(S) proxy. ({issue}`17508`) +* Improve performance of retrieving metadata from BigQuery. ({issue}`16064`) + +## Delta Lake connector + +* Support the `id` and `name` mapping modes when adding new columns. ({issue}`17236`) +* Improve performance of reading Parquet files. ({issue}`17612`) +* Improve performance when writing Parquet files with + [structural data types](structural-data-types). ({issue}`17665`) +* Properly display the schema, table name, and location of tables being inserted + into in the output of `EXPLAIN` queries. ({issue}`17590`) +* Fix query failure when writing to a file location with a trailing `/` in its + name. ({issue}`17552`) + +## Hive connector + +* Add support for reading ORC files with shorthand timezone ids in the Stripe + footer metadata. You can set the `hive.orc.read-legacy-short-zone-id` + configuration property to `true` to enable this behavior. ({issue}`12303`) +* Improve performance of reading ORC files with Bloom filter indexes. ({issue}`17530`) +* Improve performance of reading Parquet files. ({issue}`17612`) +* Improve optimized Parquet writer performance for + [structural data types](structural-data-types). ({issue}`17665`) +* Fix query failure for tables with file paths that contain non-alphanumeric + characters. ({issue}`17621`) + +## Hudi connector + +* Improve performance of reading Parquet files. ({issue}`17612`) +* Improve performance when writing Parquet files with + [structural data types](structural-data-types). ({issue}`17665`) + +## Iceberg connector + +* Add support for the [Nessie catalog](iceberg-nessie-catalog). ({issue}`11701`) +* Disallow use of the `migrate` table procedure on Hive tables with `array`, + `map` and `row` types. Previously, this returned incorrect results after the + migration. ({issue}`17587`) +* Improve performance of reading ORC files with Bloom filter indexes. ({issue}`17530`) +* Improve performance of reading Parquet files. ({issue}`17612`) +* Improve performance when writing Parquet files with + [structural data types](structural-data-types). ({issue}`17665`) +* Improve performance of reading table statistics. ({issue}`16745`) + +## SPI + +* Remove unused `NullAdaptationPolicy` from `ScalarFunctionAdapter`. ({issue}`17706`) diff --git a/430/_sources/release/release-420.md.txt b/430/_sources/release/release-420.md.txt new file mode 100644 index 000000000..0abd8f51f --- /dev/null +++ b/430/_sources/release/release-420.md.txt @@ -0,0 +1,80 @@ +# Release 420 (22 Jun 2023) + +## General + +* Add support for the {func}`any_value` aggregation function. ({issue}`17777`) +* Add support for underscores in numeric literals. ({issue}`17776`) +* Add support for hexadecimal, binary, and octal numeric literals. ({issue}`17776`) +* Deprecate the `dynamic-filtering.small-broadcast.*` and + `dynamic-filtering.large-broadcast.*` configuration properties in favor of + `dynamic-filtering.small.*` and `dynamic-filtering.large.*`. ({issue}`17831`) + +## Security + +* Add support for configuring authorization rules for + `ALTER ... SET AUTHORIZATION...` statements in file-based access control. ({issue}`16691`) +* Remove the deprecated `legacy.allow-set-view-authorization` configuration + property. ({issue}`16691`) + +## BigQuery connector + +* Fix direct download of access tokens, and correctly use the proxy when it + is enabled with the `bigquery.rpc-proxy.enabled` configuration property. ({issue}`17783`) + +## Delta Lake connector + +* Add support for [comments](/sql/comment) on view columns. ({issue}`17773`) +* Add support for recalculating all statistics with an `ANALYZE` statement. ({issue}`15968`) +* Disallow using the root directory of a bucket (`scheme://authority`) as a + table location without a trailing slash in the location name. ({issue}`17921`) +* Fix Parquet writer incompatibility with Apache Spark and Databricks Runtime. ({issue}`17978`) + +## Druid connector + +* Add support for tables with uppercase characters in their names. ({issue}`7197`) + +## Hive connector + +* Add a native Avro file format reader. This can be disabled with the + `avro.native-reader.enabled` configuration property or the + `avro_native_reader_enabled` session property. ({issue}`17221`) +* Require admin role privileges to perform `ALTER ... SET AUTHORIZATION...` + statements when the `hive-security` configuration property is set to + `sql-standard`. ({issue}`16691`) +* Improve query performance on partitioned Hive tables when table statistics are + not available. ({issue}`17677`) +* Disallow using the root directory of a bucket (`scheme://authority`) as a + table location without a trailing slash in the location name. ({issue}`17921`) +* Fix Parquet writer incompatibility with Apache Spark and Databricks Runtime. ({issue}`17978`) +* Fix reading from a Hive table when its location is the root directory of an S3 + bucket. ({issue}`17848`) + +## Hudi connector + +* Disallow using the root directory of a bucket (`scheme://authority`) as a + table location without a trailing slash in the location name. ({issue}`17921`) +* Fix Parquet writer incompatibility with Apache Spark and Databricks Runtime. ({issue}`17978`) +* Fix failure when fetching table metadata for views. ({issue}`17901`) + +## Iceberg connector + +* Disallow using the root directory of a bucket (`scheme://authority`) as a + table location without a trailing slash in the location name. ({issue}`17921`) +* Fix Parquet writer incompatibility with Apache Spark and Databricks Runtime. ({issue}`17978`) +* Fix scheduling failure when dynamic filtering is enabled. ({issue}`17871`) + +## Kafka connector + +* Fix server startup failure when a Kafka catalog is present. ({issue}`17299`) + +## MongoDB connector + +* Add support for `ALTER TABLE ... RENAME COLUMN`. ({issue}`17874`) +* Fix incorrect results when the order of the + [dbref type](https://www.mongodb.com/docs/manual/reference/database-references/#dbrefs) + fields is different from `databaseName`, `collectionName`, and `id`. ({issue}`17883`) + +## SPI + +* Move table function infrastructure to the `io.trino.spi.function.table` + package. ({issue}`17774`) diff --git a/430/_sources/release/release-421.md.txt b/430/_sources/release/release-421.md.txt new file mode 100644 index 000000000..ffc6aebe1 --- /dev/null +++ b/430/_sources/release/release-421.md.txt @@ -0,0 +1,68 @@ +# Release 421 (6 Jul 2023) + +## General + +* Add support for check constraints in an `UPDATE` statement. ({issue}`17195`) +* Improve performance for queries involving a `year` function within an `IN` + predicate. ({issue}`18092`) +* Fix failure when cancelling a query with a window function. ({issue}`18061`) +* Fix failure for queries involving the `concat_ws` function on arrays with more + than 254 values. ({issue}`17816`) +* Fix query failure or incorrect results when coercing a + [structural data type](structural-data-types) that contains a timestamp. ({issue}`17900`) + +## JDBC driver + +* Add support for using an alternative hostname with the `hostnameInCertificate` + property when SSL verification is set to `FULL`. ({issue}`17939`) + +## Delta Lake connector + +* Add support for check constraints and column invariants in `UPDATE` + statements. ({issue}`17195`) +* Add support for creating tables with the `column` mapping mode. ({issue}`12638`) +* Add support for using the `OPTIMIZE` procedure on column mapping tables. ({issue}`17527`) +* Add support for `DROP COLUMN`. ({issue}`15792`) + +## Google Sheets connector + +* Add support for {doc}`/sql/insert` statements. ({issue}`3866`) + +## Hive connector + +* Add Hive partition projection column properties to the output of + `SHOW CREATE TABLE`. ({issue}`18076`) +* Fix incorrect query results when using S3 Select with `IS NULL` or + `IS NOT NULL` predicates. ({issue}`17563`) +* Fix incorrect query results when using S3 Select and a table's `null_format` + field is set. ({issue}`17563`) + +## Iceberg connector + +* Add support for migrating a bucketed Hive table into a non-bucketed Iceberg + table. ({issue}`18103`) + +## Kafka connector + +* Add support for reading Protobuf messages containing the `Any` Protobuf type. + This is disabled by default and can be enabled by setting the + `kafka.protobuf-any-support-enabled` configuration property to `true`. ({issue}`17394`) + +## MongoDB connector + +* Improve query performance on tables with `row` columns when only a subset of + fields is needed for the query. ({issue}`17710`) + +## Redshift connector + +* Add support for [table comments](/sql/comment). ({issue}`16900`) + +## SPI + +* Add the `BLOCK_AND_POSITION_NOT_NULL` argument convention. ({issue}`18035`) +* Add the `BLOCK_BUILDER` return convention that writes function results + directly to a `BlockBuilder`. ({issue}`18094`) +* Add the `READ_VALUE` operator that can read a value from any argument + convention to any return convention. ({issue}`18094`) +* Remove write methods from the BlockBuilder interface. ({issue}`17342`) +* Change array, map, and row build to use a single `writeEntry`. ({issue}`17342`) diff --git a/430/_sources/release/release-422.md.txt b/430/_sources/release/release-422.md.txt new file mode 100644 index 000000000..bf4456600 --- /dev/null +++ b/430/_sources/release/release-422.md.txt @@ -0,0 +1,63 @@ +# Release 422 (13 Jul 2023) + +## General + +* Add support for adding nested fields with an `ADD COLUMN` statement. ({issue}`16248`) +* Improve performance of `INSERT` and `CREATE TABLE AS ... SELECT` queries. ({issue}`18005`) +* Prevent queries from hanging when worker nodes fail and the + `task.retry-policy` configuration property is set to `TASK`. ({issue}`18175 `) + +## Security + +* Add support for validating JWT types with OAuth 2.0 authentication. ({issue}`17640`) +* Fix error when the `http-server.authentication.type` configuration property + is set to `oauth2` or `jwt` and the `principal-field` property's value + differs. ({issue}`18210`) + +## BigQuery connector + +* Add support for writing to columns with a `timestamp(p) with time zone` type. ({issue}`17793`) + +## Delta Lake connector + +* Add support for renaming columns. ({issue}`15821`) +* Improve performance of reading from tables with a large number of + [checkpoints](https://docs.delta.io/latest/delta-batch.html#-data-retention). ({issue}`17405`) +* Disallow using the `vacuum` procedure when the max + [writer version](https://docs.delta.io/latest/versioning.html#features-by-protocol-version) + is above 5. ({issue}`18095`) + +## Hive connector + +* Add support for reading the `timestamp with local time zone` Hive type. ({issue}`1240`) +* Add a native Avro file format writer. This can be disabled with the + `avro.native-writer.enabled` configuration property or the + `avro_native_writer_enabled` session property. ({issue}`18064`) +* Fix query failure when the `hive.recursive-directories` configuration property + is set to true and partition names contain non-alphanumeric characters. ({issue}`18167`) +* Fix incorrect results when reading text and `RCTEXT` files with a value that + contains the character that separates fields. ({issue}`18215`) +* Fix incorrect results when reading concatenated `GZIP` compressed text files. ({issue}`18223`) +* Fix incorrect results when reading large text and sequence files with a single + header row. ({issue}`18255`) +* Fix incorrect reporting of bytes read for compressed text files. ({issue}`1828`) + +## Iceberg connector + +* Add support for adding nested fields with an `ADD COLUMN` statement. ({issue}`16248`) +* Add support for the `register_table` procedure to register Hadoop tables. ({issue}`16363`) +* Change the default file format to Parquet. The `iceberg.file-format` + catalog configuration property can be used to specify a different default file + format. ({issue}`18170`) +* Improve performance of reading `row` types from Parquet files. ({issue}`17387`) +* Fix failure when writing to tables sorted on `UUID` or `TIME` types. ({issue}`18136`) + +## Kudu connector + +* Add support for table comments when creating tables. ({issue}`17945`) + +## Redshift connector + +* Prevent returning incorrect results by throwing an error when encountering + unsupported types. Previously, the query would fall back to the legacy type + mapping. ({issue}`18209`) diff --git a/430/_sources/release/release-423.md.txt b/430/_sources/release/release-423.md.txt new file mode 100644 index 000000000..6930d3a7b --- /dev/null +++ b/430/_sources/release/release-423.md.txt @@ -0,0 +1,160 @@ +# Release 423 (10 Aug 2023) + +## General + +* Add support for renaming nested fields in a column via `RENAME COLUMN`. ({issue}`16757`) +* Add support for setting the type of a nested field in a column via `SET DATA TYPE`. ({issue}`16959`) +* Add support for comments on materialized view columns. ({issue}`18016`) +* Add support for displaying all Unicode characters in string literals. ({issue}`5061`) +* Improve performance of `INSERT` and `CREATE TABLE AS ... SELECT` queries. ({issue}`18212`) +* Improve performance when planning queries involving multiple window functions. ({issue}`18491`) +* Improve performance of queries involving `BETWEEN` clauses. ({issue}`18501`) +* Improve performance of queries containing redundant `ORDER BY` clauses in + views or `WITH` clauses. This may affect the semantics of queries that + incorrectly rely on implementation-specific behavior. The old behavior can be + restored via the `skip_redundant_sort` session property or the + `optimizer.skip-redundant-sort` configuration property. ({issue}`18159`) +* Reduce default values for the `task.partitioned-writer-count` and + `task.scale-writers.max-writer-count` configuration properties to reduce the + memory requirements of queries that write data. ({issue}`18488`) +* Remove the deprecated `optimizer.use-mark-distinct` configuration property, + which has been replaced with `optimizer.mark-distinct-strategy`. ({issue}`18540`) +* Fix query planning failure due to dynamic filters in + [fault tolerant execution mode](/admin/fault-tolerant-execution). ({issue}`18383`) +* Fix `EXPLAIN` failure when a query contains `WHERE ... IN (NULL)`. ({issue}`18328`) + +## JDBC driver + +* Add support for + [constrained delegation](https://web.mit.edu/kerberos/krb5-latest/doc/appdev/gssapi.html#constrained-delegation-s4u) + with Kerberos. ({issue}`17853`) + +## CLI + +* Add support for accepting a single Trino JDBC URL with parameters as an + alternative to passing command line arguments. ({issue}`12587`) + +## ClickHouse connector + +* Add support for `CASCADE` option in `DROP SCHEMA` statements. ({issue}`18305`) + +## Blackhole connector + +* Add support for the `COMMENT ON VIEW` statement. ({issue}`18516`) + +## Delta Lake connector + +* Add `$properties` system table which can be queried to inspect Delta Lake + table properties. ({issue}`17294`) +* Add support for reading the `timestamp_ntz` type. ({issue}`17502`) +* Add support for writing the `timestamp with time zone` type on partitioned + columns. ({issue}`16822`) +* Add option to enforce that a filter on a partition key is present for + query processing. This can be enabled by setting the + ``delta.query-partition-filter-required`` configuration property or the + ``query_partition_filter_required`` session property to ``true``. + ({issue}`18345`) +* Improve performance of the `$history` system table. ({issue}`18427`) +* Improve memory accounting of the Parquet writer. ({issue}`18564`) +* Allow metadata changes on Delta Lake tables with + [identity columns](https://github.com/delta-io/delta/blob/master/PROTOCOL.md#identity-columns). ({issue}`18200`) +* Fix incorrectly creating files smaller than the configured + `file_size_threshold` as part of `OPTIMIZE`. ({issue}`18388`) +* Fix query failure when a table has a file with a location ending with + whitespace. ({issue}`18206`) + +## Hive connector + +* Add support for `varchar` to `timestamp` coercion in Hive tables. ({issue}`18014`) +* Improve memory accounting of the Parquet writer. ({issue}`18564`) +* Remove the legacy Parquet writer, along with the + `parquet.optimized-writer.enabled` configuration property and the + `parquet_optimized_writer_enabled ` session property. Replace the + `parquet.optimized-writer.validation-percentage` configuration property with + `parquet.writer.validation-percentage`. ({issue}`18420`) +* Disallow coercing Hive `timestamp` types to `varchar` for dates before 1900. ({issue}`18004`) +* Fix loss of data precision when coercing Hive `timestamp` values. ({issue}`18003`) +* Fix incorrectly creating files smaller than the configured + `file_size_threshold` as part of `OPTIMIZE`. ({issue}`18388`) +* Fix query failure when a table has a file with a location ending with + whitespace. ({issue}`18206`) +* Fix incorrect results when using S3 Select and a query predicate includes a + quote character (`"`) or a decimal column. ({issue}`17775`) +* Add the `hive.s3select-pushdown.experimental-textfile-pushdown-enabled` + configuration property to enable S3 Select pushdown for `TEXTFILE` tables. ({issue}`17775`) + +## Hudi connector + +* Fix query failure when a table has a file with a location ending with + whitespace. ({issue}`18206`) + +## Iceberg connector + +* Add support for renaming nested fields in a column via `RENAME COLUMN`. ({issue}`16757`) +* Add support for setting the type of a nested field in a column via + `SET DATA TYPE`. ({issue}`16959`) +* Add support for comments on materialized view columns. ({issue}`18016`) +* Add support for `tinyint` and `smallint` types in the `migrate` procedure. ({issue}`17946`) +* Add support for reading Parquet files with time stored in millisecond precision. ({issue}`18535`) +* Improve performance of `information_schema.columns` queries for tables managed + by Trino with AWS Glue as metastore. ({issue}`18315`) +* Improve performance of `system.metadata.table_comments` when querying Iceberg + tables backed by AWS Glue as metastore. ({issue}`18517`) +* Improve performance of `information_schema.columns` when using the Glue + catalog. ({issue}`18586`) +* Improve memory accounting of the Parquet writer. ({issue}`18564`) +* Fix incorrectly creating files smaller than the configured + `file_size_threshold` as part of `OPTIMIZE`. ({issue}`18388`) +* Fix query failure when a table has a file with a location ending with + whitespace. ({issue}`18206`) +* Fix failure when creating a materialized view on a table which has been + rolled back. ({issue}`18205`) +* Fix query failure when reading ORC files with nullable `time` columns. ({issue}`15606`) +* Fix failure to calculate query statistics when referring to `$path` as part of + a `WHERE` clause. ({issue}`18330`) +* Fix write conflict detection for `UPDATE`, `DELETE`, and `MERGE` operations. + In rare situations this issue may have resulted in duplicate rows when + multiple operations were run at the same time, or at the same time as an + `optimize` procedure. ({issue}`18533`) + +## Kafka connector + +* Rename the `ADD_DUMMY` value for the `kafka.empty-field-strategy` + configuration property and the `empty_field_strategy` session property to + `MARK` ({issue}`18485`). + +## Kudu connector + +* Add support for optimized local scheduling of splits. ({issue}`18121`) + +## MariaDB connector + +* Add support for `CASCADE` option in `DROP SCHEMA` statements. ({issue}`18305`) + +## MongoDB connector + +* Add support for predicate pushdown on `char` and `decimal` type. ({issue}`18382`) + +## MySQL connector + +* Add support for predicate pushdown for `=`, `<>`, `IN`, `NOT IN`, and `LIKE` + operators on case-sensitive `varchar` and `nvarchar` columns. ({issue}`18140`, {issue}`18441`) +* Add support for `CASCADE` option in `DROP SCHEMA` statements. ({issue}`18305`) + +## Oracle connector + +* Add support for Oracle `timestamp` types with non-millisecond precision. ({issue}`17934`) +* Add support for `CASCADE` option in `DROP SCHEMA` statements. ({issue}`18305`) + +## SingleStore connector + +* Add support for `CASCADE` option in `DROP SCHEMA` statements. ({issue}`18305`) + +## SPI + +* Deprecate the `ConnectorMetadata.getTableHandle(ConnectorSession, SchemaTableName)` + method signature. Connectors should implement + `ConnectorMetadata.getTableHandle(ConnectorSession, SchemaTableName, Optional, Optional)` + instead. ({issue}`18596`) +* Remove the deprecated `supportsReportingWrittenBytes` method from + ConnectorMetadata. ({issue}`18617`) diff --git a/430/_sources/release/release-424.md.txt b/430/_sources/release/release-424.md.txt new file mode 100644 index 000000000..9f00c41d6 --- /dev/null +++ b/430/_sources/release/release-424.md.txt @@ -0,0 +1,58 @@ +# Release 424 (17 Aug 2023) + +## General + +* Reduce coordinator overhead on large clusters. ({issue}`18542`) +* Require the JVM default charset to be UTF-8. This can be set with the JVM + command line option `-Dfile.encoding=UTF-8`. ({issue}`18657`) + +## JDBC driver + +* Add the number of bytes that have been written to the query results response. ({issue}`18651`) + +## Delta Lake connector + +* Remove the legacy Parquet reader, along with the + `parquet.optimized-reader.enabled` and + `parquet.optimized-nested-reader.enabled` configuration properties. ({issue}`18639`) + +## Hive connector + +* Improve performance for line-oriented Hive formats. ({issue}`18703`) +* Improve performance of reading JSON files. ({issue}`18709`) +* Remove the legacy Parquet reader, along with the + `parquet.optimized-reader.enabled` and + `parquet.optimized-nested-reader.enabled` configuration properties. ({issue}`18639`) +* Fix incorrect reporting of written bytes for uncompressed text files, which + prevented the `target_max_file_size` session property from working. ({issue}`18701`) + +## Hudi connector + +* Remove the legacy Parquet reader, along with the + `parquet.optimized-reader.enabled` and + `parquet.optimized-nested-reader.enabled` configuration properties. ({issue}`18639`) + +## Iceberg connector + +* Add support for `CASCADE` option in `DROP SCHEMA` statements. ({issue}`18689`) +* Remove the legacy Parquet reader, along with the + `parquet.optimized-reader.enabled` and + `parquet.optimized-nested-reader.enabled` configuration properties. ({issue}`18639`) +* Fix potential incorrect query results when a query involves a predicate on a + `timestamp with time zone` column. ({issue}`18588`) + +## Memory connector + +* Add support for `CASCADE` option in `DROP SCHEMA` statements. ({issue}`18668`) + +## PostgreSQL connector + +* Add support for `CASCADE` option in `DROP SCHEMA` statements. ({issue}`18663`) +* Remove support for Postgres versions older than + [version 11](https://www.postgresql.org/support/versioning/). ({issue}`18696`) + +## SPI + +* Introduce the `getNewTableWriterScalingOptions` and + `getInsertWriterScalingOptions` methods to `ConnectorMetadata`, which enable + connectors to limit writer scaling. ({issue}`18561`) diff --git a/430/_sources/release/release-425.md.txt b/430/_sources/release/release-425.md.txt new file mode 100644 index 000000000..8883d5a12 --- /dev/null +++ b/430/_sources/release/release-425.md.txt @@ -0,0 +1,40 @@ +# Release 425 (24 Aug 2023) + +## General + +* Improve performance of `GROUP BY`. ({issue}`18106`) +* Fix incorrect reporting of cumulative memory usage. ({issue}`18714`) + +## BlackHole connector + +* Remove support for materialized views. ({issue}`18628`) + +## Delta Lake connector + +* Add support for check constraints in `MERGE` statements. ({issue}`15411`) +* Improve performance when statistics are missing from the transaction log. ({issue}`16743`) +* Improve memory usage accounting of the Parquet writer. ({issue}`18756`) +* Improve performance of `DELETE` statements when they delete the whole table or + when the filters only apply to partition columns. ({issue}`18332 `) + +## Hive connector + +* Add support for `CASCADE` option in `DROP SCHEMA` statements. ({issue}`18320`) +* Create a new directory if the specified external location for a new table does + not exist. ({issue}`17920`) +* Improve memory usage accounting of the Parquet writer. ({issue}`18756`) +* Improve performance of writing to JSON files. ({issue}`18683`) + +## Iceberg connector + +* Improve memory usage accounting of the Parquet writer. ({issue}`18756`) + +## Kudu connector + +* Add support for `CASCADE` option in `DROP SCHEMA` statements. ({issue}`18629`) + +## MongoDB connector + +* Add support for the `Decimal128` MongoDB type. ({issue}`18722`) +* Add support for `CASCADE` option in `DROP SCHEMA` statements. ({issue}`18629`) +* Fix query failure when reading the value of `-0` as a `decimal` type. ({issue}`18777`) diff --git a/430/_sources/release/release-426.md.txt b/430/_sources/release/release-426.md.txt new file mode 100644 index 000000000..e5631a6eb --- /dev/null +++ b/430/_sources/release/release-426.md.txt @@ -0,0 +1,49 @@ +# Release 426 (5 Sep 2023) + +## General + +* Add support for `SET SESSION AUTHORIZATION` and `RESET SESSION AUTHORIZATION`. ({issue}`16067`) +* Add support for automatic type coercion when creating tables. ({issue}`13994`) +* Improve performance of aggregations over decimal values. ({issue}`18868`) +* Fix event listener incorrectly reporting output columns for `UPDATE` + statements with subqueries. ({issue}`18815`) +* Fix failure when performing an outer join involving geospatial functions in + the join clause. ({issue}`18860`) +* Fix failure when querying partitioned tables with a `WHERE` clause that + contains lambda expressions. ({issue}`18865`) +* Fix failure for `GROUP BY` queries over `map` and `array` types. ({issue}`18863`) + +## Security + +* Fix authentication failure with OAuth 2.0 when authentication tokens are + larger than 4 KB. ({issue}`18836`) + +## Delta Lake connector + +* Add support for the `TRUNCATE TABLE` statement. ({issue}`18786`) +* Add support for the `CASCADE` option in `DROP SCHEMA` statements. ({issue}`18333`) +* Add support for + [Databricks 13.3 LTS](https://docs.databricks.com/en/release-notes/runtime/13.3lts.html). ({issue}`18888`) +* Fix writing an incorrect transaction log for partitioned tables with an `id` + or `name` column mapping mode. ({issue}`18661`) + +## Hive connector + +* Add the `hive.metastore.thrift.batch-fetch.enabled` configuration property, + which can be set to `false` to disable batch metadata fetching from the Hive + metastore. ({issue}`18111`) +* Fix `ANALYZE` failure when row count stats are missing. ({issue}`18798`) +* Fix the `hive.target-max-file-size` configuration property being ignored + when writing to sorted tables. ({issue}`18653`) +* Fix query failure when reading large SequenceFile, RCFile, or Avro files. ({issue}`18837`) + +## Iceberg connector + +* Fix the `iceberg.target-max-file-size` configuration property being ignored + when writing to sorted tables. ({issue}`18653`) + +## SPI + +* Remove the deprecated + `ConnectorMetadata#dropSchema(ConnectorSession session, String schemaName)` + method. ({issue}`18839`) diff --git a/430/_sources/release/release-427.md.txt b/430/_sources/release/release-427.md.txt new file mode 100644 index 000000000..e8d34d6d9 --- /dev/null +++ b/430/_sources/release/release-427.md.txt @@ -0,0 +1,107 @@ +# Release 427 (26 Sep 2023) + +## General + +* Add support for comparing IPv4 and IPv6 addresses and CIDRs with [contains](ip-address-contains). ({issue}`18497`) +* Improve performance of `GROUP BY` and `DISTINCT`. ({issue}`19059`) +* Reduce coordinator memory footprint when scannning tables. ({issue}`19009`) +* Fix failure due to exceeding node memory limits with `INSERT` statements. ({issue}`18771`) +* Fix query hang for certain `LIKE` patterns involving a mix of `%` and `_`. ({issue}`19146`) + +## Security + +* Ensure authorization is checked when accessing table comments with table redirections. ({issue}`18514`) + +## Delta Lake connector + +* Add support for reading tables with + [Deletion Vectors](https://docs.delta.io/latest/delta-deletion-vectors.html). ({issue}`16903`) +* Add support for Delta Lake writer + [version 7](https://docs.delta.io/latest/versioning.html#features-by-protocol-version). ({issue}`15873`) +* Add support for writing columns with the `timestamp(p)` type. ({issue}`16927`) +* Reduce data read from Parquet files for queries with filters. ({issue}`19032`) +* Improve performance of writing to Parquet files. ({issue}`19122`) +* Fix error reading Delta Lake table history when the initial transaction logs + have been removed. ({issue}`18845`) + +## Elasticsearch connector + +* Fix query failure when a `LIKE` clause contains multi-byte characters. ({issue}`18966`) + +## Hive connector + +* Add support for changing column comments when using the Glue catalog. ({issue}`19076`) +* Reduce data read from Parquet files for queries with filters. ({issue}`19032`) +* Improve performance of reading text files. ({issue}`18959`) +* Allow changing a column's type from `double` to `varchar` in Hive tables. ({issue}`18930`) +* Remove legacy Hive readers and writers. The `*_native_reader_enabled` and + `*_native_writer_enabled` session properties and `*.native-reader.enabled` and + `*.native-writer.enabled` configuration properties are removed. ({issue}`18241`) +* Remove support for S3 Select. The `s3_select_pushdown_enabled` session + property and the `hive.s3select*` configuration properties are removed. ({issue}`18241`) +* Remove support for disabling optimized symlink listing. The + `optimize_symlink_listing` session property and + `hive.optimize-symlink-listing` configuration property are removed. ({issue}`18241`) +* Fix incompatibility with Hive OpenCSV deserialization. As a result, when the + escape character is explicitly set to `"`, a `\` (backslash) must be used + instead. ({issue}`18918`) +* Fix performance regression when reading CSV files on AWS S3. ({issue}`18976`) +* Fix failure when creating a table with a `varchar(0)` column. ({issue}`18811`) + +## Hudi connector + +* Fix query failure when reading from Hudi tables with + [`instants`](https://hudi.apache.org/docs/concepts/#timeline) that have been + replaced. ({issue}`18213`) + +## Iceberg connector + +* Add support for usage of `date` and `timestamp` arguments in `FOR TIMESTAMP AS + OF` expressions. ({issue}`14214`) +* Add support for using tags with `AS OF VERSION` queries. ({issue}`19111`) +* Reduce data read from Parquet files for queries with filters. ({issue}`19032`) +* Improve performance of writing to Parquet files. ({issue}`19090`) +* Improve performance of reading tables with many equality delete files. ({issue}`17114`) + +## Ignite connector + +* Add support for `UPDATE`. ({issue}`16445`) + +## MariaDB connector + +* Add support for `UPDATE`. ({issue}`16445`) + +## MongoDB connector + +* Fix query failure when mapping MongoDB `Decimal128` values with leading zeros. ({issue}`19068`) + +## MySQL connector + +* Add support for `UPDATE`. ({issue}`16445`) +* Change mapping for MySQL `TIMESTAMP` types from `timestamp(n)` to + `timestamp(n) with time zone`. ({issue}`18470`) + +## Oracle connector + +* Add support for `UPDATE`. ({issue}`16445`) +* Fix potential query failure when joins are pushed down to Oracle. ({issue}`18924`) + +## PostgreSQL connector + +* Add support for `UPDATE`. ({issue}`16445`) + +## Redshift connector + +* Add support for `UPDATE`. ({issue}`16445`) + +## SingleStore connector + +* Add support for `UPDATE`. ({issue}`16445`) + +## SQL Server connector + +* Add support for `UPDATE`. ({issue}`16445`) + +## SPI + +* Change `BlockBuilder` to no longer extend `Block`. ({issue}`18738`) diff --git a/430/_sources/release/release-428.md.txt b/430/_sources/release/release-428.md.txt new file mode 100644 index 000000000..7bc201687 --- /dev/null +++ b/430/_sources/release/release-428.md.txt @@ -0,0 +1,59 @@ +# Release 428 (4 Oct 2023) + +## General + +* Reduce memory usage for queries involving `GROUP BY` clauses. ({issue}`19187`) +* Simplify writer count configuration. Add the new `task.min-writer-count` + and `task.max-writer-count` configuration properties along with the + `task_min_writer_count` and `task_max_writer_count` session properties, which + control the number of writers depending on scenario. Deprecate the + `task.writer-count`, `task.scale-writers.max-writer-count`, and + `task.partitioned-writer-count` configuration properties, which will be + removed in the future. Remove the `task_writer_count`, + `task_partitioned_writer_count`, and `task_scale_writers_max_writer_count` + session properties. ({issue}`19135`) +* Remove support for the `parse-decimal-literals-as-double` legacy configuration + property. ({issue}`19166`) +* Fix out of memory error when running queries with `GROUP BY` clauses. ({issue}`19119`) + +## Delta Lake connector + +* Reduce the number of read requests for scanning small Parquet files. Add the + `parquet.small-file-threshold` configuration property and the + `parquet_small_file_threshold` session property to change the default size of + `3MB`, below which, files will be read in their entirety. Setting this + configuration to `0B` disables the feature. ({issue}`19127`) +* Fix potential data duplication when running `OPTIMIZE` coincides with + updates to a table. ({issue}`19128`) +* Fix error when deleting rows in tables that have partitions with certain + non-alphanumeric characters in their names. ({issue}`18922`) + +## Hive connector + +* Reduce the number of read requests for scanning small Parquet files. Add the + `parquet.small-file-threshold` configuration property and the + `parquet_small_file_threshold` session property to change the default size of + `3MB`, below which, files will be read in their entirety. Setting this + configuration to `0B` disables the feature. ({issue}`19127`) + +## Hudi connector + +* Reduce the number of read requests for scanning small Parquet files. Add the + `parquet.small-file-threshold` configuration property and the + `parquet_small_file_threshold` session property to change the default size of + `3MB`, below which, files will be read in their entirety. Setting this + configuration to `0B` disables the feature. ({issue}`19127`) + +## Iceberg connector + +* Reduce the number of read requests for scanning small Parquet files. Add the + `parquet.small-file-threshold` configuration property and the + `parquet_small_file_threshold` session property to change the default size of + `3MB`, below which, files will be read in their entirety. Setting this + configuration to `0B` disables the feature. ({issue}`19127`) +* Fix incorrect column statistics for the Parquet file format in manifest files. ({issue}`19052`) + +## Pinot connector + +* Add support for [query options](https://docs.pinot.apache.org/users/user-guide-query/query-options) + in dynamic tables. ({issue}`19078`) diff --git a/430/_sources/release/release-429.md.txt b/430/_sources/release/release-429.md.txt new file mode 100644 index 000000000..2671df563 --- /dev/null +++ b/430/_sources/release/release-429.md.txt @@ -0,0 +1,52 @@ +# Release 429 (11 Oct 2023) + +## General + +* Allow {doc}`/sql/show-functions` for a specific schema. ({issue}`19243`) +* Add security for function listing. ({issue}`19243`) + +## Security + +* Stop performing security checks for functions in the `system.builtin` schema. ({issue}`19160`) +* Remove support for using function kind as a rule in file-based access control. ({issue}`19160`) + +## Web UI + +* Log out from a Trino OAuth session when logging out from the Web UI. ({issue}`13060`) + +## Delta Lake connector + +* Allow using the `#` and `?` characters in S3 location paths or URLs. ({issue}`19296`) + +## Hive connector + +* Add support for changing a column's type from `varchar` to `date`. ({issue}`19201`) +* Add support for changing a column's type from `decimal` to `tinyint`, + `smallint`, `integer`, or `bigint` in partitioned Hive tables. ({issue}`19201`) +* Improve performance of reading ORC files. ({issue}`19295`) +* Allow using the `#` and `?` characters in S3 location paths or URLs. ({issue}`19296`) +* Fix error reading Avro files when a schema has uppercase characters in its + name. ({issue}`19249`) + +## Hudi connector + +* Allow using the `#` and `?` characters in S3 location paths or URLs. ({issue}`19296`) + +## Iceberg connector + +* Add support for specifying timestamp precision as part of + `CREATE TABLE AS .. SELECT` statements. ({issue}`13981`) +* Improve performance of reading ORC files. ({issue}`19295`) +* Allow using the `#` and `?` characters in S3 location paths or URLs. ({issue}`19296`) + +## MongoDB connector + +* Fix mixed case schema names being inaccessible when using custom roles and + the `case-insensitive-name-matching` configuration property is enabled. ({issue}`19218`) + +## SPI + +* Change function security checks to return a boolean instead of throwing an + exception. ({issue}`19160`) +* Add SQL path field to `ConnectorViewDefinition`, + `ConnectorMaterializedViewDefinition`, and `ViewExpression`. ({issue}`19160`) diff --git a/430/_sources/release/release-430.md.txt b/430/_sources/release/release-430.md.txt new file mode 100644 index 000000000..774a25f64 --- /dev/null +++ b/430/_sources/release/release-430.md.txt @@ -0,0 +1,47 @@ +# Release 430 (20 Oct 2023) + +## General + +* Improve performance of queries with `GROUP BY`. ({issue}`19302`) +* Fix incorrect results for queries involving `ORDER BY` and window functions + with ordered frames. ({issue}`19399`) +* Fix incorrect results for query involving an aggregation in a correlated + subquery. ({issue}`19002`) + +## Security + +* Enforce authorization capability of client when receiving commands `RESET` and + `SET` for `SESSION AUTHORIZATION`. ({issue}`19217`) + +## JDBC driver + +* Add support for a `timezone` parameter to set the session timezone. ({issue}`19102`) + +## Iceberg connector + +* Add an option to require filters on partition columns. This can be enabled by + setting the ``iceberg.query-partition-filter-required`` configuration property + or the ``query_partition_filter_required`` session property. ({issue}`17263`) +* Improve performance when reading partition columns. ({issue}`19303`) + +## Ignite connector + +* Fix failure when a query contains `LIKE` with `ESCAPE`. ({issue}`19464`) + +## MariaDB connector + +* Add support for table statistics. ({issue}`19408`) + +## MongoDB connector + +* Fix incorrect results when a query contains several `<>` or `NOT IN` + predicates. ({issue}`19404`) + +## Oracle connector + +* Improve reliability of connecting to the source database. ({issue}`19191`) + +## SPI + +* Change the Java stack type for a `map` value to `SqlMap` and a `row` value to + `SqlRow`, which do not implement `Block`. ({issue}`18948`) diff --git a/430/_sources/security.md.txt b/430/_sources/security.md.txt new file mode 100644 index 000000000..5b1ecf62f --- /dev/null +++ b/430/_sources/security.md.txt @@ -0,0 +1,63 @@ +# Security + +## Introduction + +```{toctree} +:maxdepth: 1 + +security/overview +``` + +## Cluster access security + +```{toctree} +:maxdepth: 1 + +security/tls +security/inspect-pem +security/inspect-jks +``` + +(security-authentication)= + +## Authentication + +```{toctree} +:maxdepth: 1 + +security/authentication-types +security/password-file +security/ldap +security/salesforce +security/oauth2 +security/kerberos +security/certificate +security/jwt +``` + +## User name management + +```{toctree} +:maxdepth: 1 + +security/user-mapping +security/group-file +``` + +## Access control + +```{toctree} +:maxdepth: 1 + +security/built-in-system-access-control +security/file-system-access-control +``` + +## Security inside the cluster + +```{toctree} +:maxdepth: 1 + +security/internal-communication +security/secrets +``` diff --git a/430/_sources/security/authentication-types.md.txt b/430/_sources/security/authentication-types.md.txt new file mode 100644 index 000000000..2116687d5 --- /dev/null +++ b/430/_sources/security/authentication-types.md.txt @@ -0,0 +1,84 @@ +# Authentication types + +Trino supports multiple authentication types to ensure all users of the system +are authenticated. Different authenticators allow user management in one or more +systems. Using {doc}`TLS ` and {doc}`a configured shared secret +` are required for all authentications types. + +You can configure one or more authentication types with the +`http-server.authentication.type` property. The following authentication types +and authenticators are available: + +- `PASSWORD` for + + - {doc}`password-file` + - {doc}`ldap` + - {doc}`salesforce` + +- `OAUTH2` for {doc}`oauth2` + +- `KERBEROS` for {doc}`kerberos` + +- `CERTIFICATE` for {doc}`certificate` + +- `JWT` for {doc}`jwt` + +- `HEADER` for {doc}`/develop/header-authenticator` + +Get started with a basic password authentication configuration backed by a +{doc}`password file `: + +```properties +http-server.authentication.type=PASSWORD +``` + +## Multiple authentication types + +You can use multiple authentication types, separated with commas in the +configuration: + +```properties +http-server.authentication.type=PASSWORD,CERTIFICATE +``` + +Authentication is performed in order of the entries, and first successful +authentication results in access, using the {doc}`mapped user ` +from that authentication method. + +## Multiple password authenticators + +You can use multiple password authenticator types by referencing multiple +configuration files: + +```properties +http-server.authentication.type=PASSWORD +password-authenticator.config-files=etc/ldap1.properties,etc/ldap2.properties,etc/password.properties +``` + +In the preceding example, the configuration files `ldap1.properties` and +`ldap2.properties` are regular {doc}`LDAP authenticator configuration files +`. The `password.properties` is a {doc}`password file authenticator +configuration file `. + +Relative paths to the installation directory or absolute paths can be used. + +User authentication credentials are first validated against the LDAP server from +`ldap1`, then the separate server from `ldap2`, and finally the password +file. First successful authentication results in access, and no further +authenticators are called. + +## Multiple header authenticators + +You can use multiple header authenticator types by referencing multiple +configuration files: + +```properties +http-server.authentication.type=HEADER +header-authenticator.config-files=etc/xfcc.properties,etc/azureAD.properties +``` + +Relative paths to the installation directory or absolute paths can be used. + +The pre-configured headers are first validated against the `xfcc` authenticator, +then the `azureAD` authenticator. First successful authentication results in access, +and no further authenticators are called. diff --git a/430/_sources/security/built-in-system-access-control.md.txt b/430/_sources/security/built-in-system-access-control.md.txt new file mode 100644 index 000000000..cca8b6c77 --- /dev/null +++ b/430/_sources/security/built-in-system-access-control.md.txt @@ -0,0 +1,62 @@ +# System access control + +A system access control enforces authorization at a global level, +before any connector level authorization. You can use one of the built-in +implementations in Trino, or provide your own by following the guidelines in +{doc}`/develop/system-access-control`. + +To use a system access control, add an `etc/access-control.properties` file +with the following content and the desired system access control name on all +cluster nodes: + +```text +access-control.name=allow-all +``` + +Multiple system access control implementations may be configured at once +using the `access-control.config-files` configuration property. It should +contain a comma separated list of the access control property files to use +(rather than the default `etc/access-control.properties`). + +Trino offers the following built-in system access control implementations: + +:::{list-table} +:widths: 20, 80 +:header-rows: 1 + +* - Name + - Description +* - `default` + - All operations are permitted, except for user impersonation and triggering + [](/admin/graceful-shutdown). + + This is the default access control if none are configured. +* - `allow-all` + - All operations are permitted. +* - `read-only` + - Operations that read data or metadata are permitted, but none of the + operations that write data or metadata are allowed. +* - `file` + - Authorization rules are specified in a config file. See + [](/security/file-system-access-control). +::: + +If you want to limit access on a system level in any other way than the ones +listed above, you must implement a custom {doc}`/develop/system-access-control`. + +Access control must be configured on the coordinator. Authorization for +operations on specific worker nodes, such a triggering +{doc}`/admin/graceful-shutdown`, must also be configured on all workers. + +## Read only system access control + +This access control allows any operation that reads data or +metadata, such as `SELECT` or `SHOW`. Setting system level or catalog level +session properties is also permitted. However, any operation that writes data or +metadata, such as `CREATE`, `INSERT` or `DELETE`, is prohibited. +To use this access control, add an `etc/access-control.properties` +file with the following contents: + +```text +access-control.name=read-only +``` diff --git a/430/_sources/security/certificate.md.txt b/430/_sources/security/certificate.md.txt new file mode 100644 index 000000000..dfabad8d2 --- /dev/null +++ b/430/_sources/security/certificate.md.txt @@ -0,0 +1,106 @@ +# Certificate authentication + +You can configure Trino to support client-provided certificates validated by the +Trino server on initial connection. + +:::{important} +This authentication method is only provided to support sites that have an +absolute requirement for client authentication *and already have* client +certificates for each client. Sites in this category have an existing PKI +infrastructure, possibly including an onsite Certificate Authority (CA). + +This feature is not appropriate for sites that need to generate a set of +client certificates in order to use this authentication type. Consider +instead using another {ref}`authentication type `. +::: + +Using {doc}`TLS ` and {doc}`a configured shared secret +` is required for certificate authentication. + +## Using certificate authentication + +All clients connecting with TLS/HTTPS go through the following initial steps: + +1. The client attempts to contact the coordinator. +2. The coordinator returns its certificate to the client. +3. The client validates the server's certificate using the client's trust store. + +A cluster with certificate authentication enabled goes through the following +additional steps: + +4. The coordinator asks the client for its certificate. +5. The client responds with its certificate. +6. The coordinator verifies the client's certificate, using the coordinator's + trust store. + +Several rules emerge from these steps: + +- Trust stores used by clients must include the certificate of the signer of + the coordinator's certificate. +- Trust stores used by coordinators must include the certificate of the signer + of client certificates. +- The trust stores used by the coordinator and clients do not need to be the + same. +- The certificate that verifies the coordinator does not need to be the same as + the certificate verifying clients. + +Trino validates certificates based on the distinguished name (DN) from the +X.509 `Subject` field. You can use {doc}`user mapping +` to map the subject DN to a Trino user name. + +There are three levels of client certificate support possible. From the point of +view of the server: + +- The server does not require a certificate from clients. +- The server asks for a certificate from clients, but allows connection without one. +- The server must have a certificate from clients to allow connection. + +Trino's client certificate support is the middle type. It asks for a certificate +but allows connection if another authentication method passes. + +## Certificate authentication configuration + +Enable certificate authentication by setting the {doc}`Certificate +authentication type ` in {ref}`etc/config.properties +`: + +```properties +http-server.authentication.type=CERTIFICATE +``` + +You can specify certificate authentication along with another authenticaton +method, such as `PASSWORD`. In this case, authentication is performed in the +order of entries, and the first successful authentication results in access. +For example, the following setting shows the use of two authentication types: + +```properties +http-server.authentication.type=CERTIFICATE,PASSWORD +``` + +The following configuration properties are also available: + +:::{list-table} Configuration properties +:widths: 50 50 +:header-rows: 1 + +* - Property name + - Description +* - `http-server.authentication.certificate.user-mapping.pattern` + - A regular expression pattern to [map all user + names](/security/user-mapping) for this authentication type to the format + expected by Trino. +* - `http-server.authentication.certificate.user-mapping.file` + - The path to a JSON file that contains a set of [user mapping + rules](/security/user-mapping) for this authentication type. +::: + +## Use certificate authentication with clients + +When using the Trino {doc}`CLI `, specify the +`--keystore-path` and `--keystore-password` options as described +in {ref}`cli-certificate-auth`. + +When using the Trino {doc}`JDBC driver ` to connect to a +cluster with certificate authentication enabled, use the `SSLKeyStoreType` and +`SSLKeyStorePassword` {ref}`parameters ` to specify +the path to the client's certificate and its password, if any. diff --git a/430/_sources/security/file-system-access-control.md.txt b/430/_sources/security/file-system-access-control.md.txt new file mode 100644 index 000000000..64ef7a615 --- /dev/null +++ b/430/_sources/security/file-system-access-control.md.txt @@ -0,0 +1,912 @@ +# File-based access control + +To secure access to data in your cluster, you can implement file-based access +control where access to data and operations is defined by rules declared in +manually-configured JSON files. + +There are two types of file-based access control: + +- **System-level access control** uses the access control plugin with a single + JSON file that specifies authorization rules for the whole cluster. +- **Catalog-level access control** uses individual JSON files for each catalog + for granular control over the data in that catalog, including column-level + authorization. + +(system-file-based-access-control)= + +## System-level access control files + +The access control plugin allows you to specify authorization rules for the +cluster in a single JSON file. + +### Configuration + +To use the access control plugin, add an `etc/access-control.properties` file +containing two required properties: `access-control.name`, which must be set +to `file`, and `security.config-file`, which must be set to the location +of the config file. The configuration file location can either point to the local +disc or to a http endpoint. For example, if a config file named `rules.json` resides +in `etc`, add an `etc/access-control.properties` with the following +contents: + +```text +access-control.name=file +security.config-file=etc/rules.json +``` + +If the config should be loaded via the http endpoint `http://trino-test/config` and +is wrapped into a JSON object and available via the `data` key `etc/access-control.properties` +should look like this: + +```text +access-control.name=file +security.config-file=http://trino-test/config +security.json-pointer=/data +``` + +The config file is specified in JSON format. It contains rules that define which +users have access to which resources. The rules are read from top to bottom and +the first matching rule is applied. If no rule matches, access is denied. A JSON +pointer (RFC 6901) can be specified using the `security.json-pointer` property +to specify a nested object inside the JSON content containing the rules. Per default, +the file is assumed to contain a single object defining the rules rendering +the specification of `security.json-pointer` unnecessary in that case. + +### Refresh + +By default, when a change is made to the JSON rules file, Trino must be +restarted to load the changes. There is an optional property to refresh the +properties without requiring a Trino restart. The refresh period is specified in +the `etc/access-control.properties`: + +```text +security.refresh-period=1s +``` + +### Catalog, schema, and table access + +Access to catalogs, schemas, tables, and views is controlled by the catalog, +schema, and table rules. The catalog rules are coarse-grained rules used to +restrict all access or write access to catalogs. They do not explicitly grant +any specific schema or table permissions. The table and schema rules are used to +specify who can create, drop, alter, select, insert, delete, etc. for schemas +and tables. + +:::{note} +These rules do not apply to system-defined tables in the +`information_schema` schema. +::: + +For each rule set, permission is based on the first matching rule read from top +to bottom. If no rule matches, access is denied. If no rules are provided at +all, then access is granted. + +The following table summarizes the permissions required for each SQL command: + +| SQL command | Catalog | Schema | Table | Note | +| ---------------------------------- | --------- | ------- | -------------------- | ----------------------------------------------------------------- | +| SHOW CATALOGS | | | | Always allowed | +| SHOW SCHEMAS | read-only | any\* | any\* | Allowed if catalog is {ref}`visible` | +| SHOW TABLES | read-only | any\* | any\* | Allowed if schema {ref}`visible` | +| CREATE SCHEMA | read-only | owner | | | +| DROP SCHEMA | all | owner | | | +| SHOW CREATE SCHEMA | all | owner | | | +| ALTER SCHEMA ... RENAME TO | all | owner\* | | Ownership is required on both old and new schemas | +| ALTER SCHEMA ... SET AUTHORIZATION | all | owner | | | +| CREATE TABLE | all | | owner | | +| DROP TABLE | all | | owner | | +| ALTER TABLE ... RENAME TO | all | | owner\* | Ownership is required on both old and new tables | +| ALTER TABLE ... SET PROPERTIES | all | | owner | | +| CREATE VIEW | all | | owner | | +| DROP VIEW | all | | owner | | +| ALTER VIEW ... RENAME TO | all | | owner\* | Ownership is required on both old and new views | +| REFRESH MATERIALIZED VIEW | all | | update | | +| COMMENT ON TABLE | all | | owner | | +| COMMENT ON COLUMN | all | | owner | | +| ALTER TABLE ... ADD COLUMN | all | | owner | | +| ALTER TABLE ... DROP COLUMN | all | | owner | | +| ALTER TABLE ... RENAME COLUMN | all | | owner | | +| SHOW COLUMNS | all | | any | | +| SELECT FROM table | read-only | | select | | +| SELECT FROM view | read-only | | select, grant_select | | +| INSERT INTO | all | | insert | | +| DELETE FROM | all | | delete | | +| UPDATE | all | | update | | + +Permissions required for executing functions: + +:::{list-table} +:widths: 30, 10, 20, 40 +:header-rows: 1 + +* - SQL command + - Catalog + - Function permission + - Note +* - `SELECT function()` + - + - `execute`, `grant_execute*` + - `grant_execute` is required when the function is used in a `SECURITY DEFINER` + view. +* - `SELECT FROM TABLE(table_function())` + - `all` + - `execute`, `grant_execute*` + - `grant_execute` is required when the function is used in a `SECURITY DEFINER` + view. +::: + +(system-file-auth-visibility)= + +#### Visibility + +For a catalog, schema, or table to be visible in a `SHOW` command, the user +must have at least one permission on the item or any nested item. The nested +items do not need to already exist as any potential permission makes the item +visible. Specifically: + +- `catalog`: Visible if user is the owner of any nested schema, has + permissions on any nested table or function, or has permissions to + set session properties in the catalog. +- `schema`: Visible if the user is the owner of the schema, or has permissions + on any nested table or function. +- `table`: Visible if the user has any permissions on the table. + +#### Catalog rules + +Each catalog rule is composed of the following fields: + +- `user` (optional): regex to match against user name. Defaults to `.*`. +- `role` (optional): regex to match against role names. Defaults to `.*`. +- `group` (optional): regex to match against group names. Defaults to `.*`. +- `catalog` (optional): regex to match against catalog name. Defaults to + `.*`. +- `allow` (required): string indicating whether a user has access to the + catalog. This value can be `all`, `read-only` or `none`, and defaults to + `none`. Setting this value to `read-only` has the same behavior as the + `read-only` system access control plugin. + +In order for a rule to apply the user name must match the regular expression +specified in `user` attribute. + +For role names, a rule can be applied if at least one of the currently enabled +roles matches the `role` regular expression. + +For group names, a rule can be applied if at least one group name of this user +matches the `group` regular expression. + +The `all` value for `allow` means these rules do not restrict access in any +way, but the schema and table rules can restrict access. + +:::{note} +By default, all users have access to the `system` catalog. You can +override this behavior by adding a rule. + +Boolean `true` and `false` are also supported as legacy values for +`allow`, to support backwards compatibility. `true` maps to `all`, +and `false` maps to `none`. +::: + +For example, if you want to allow only the role `admin` to access the +`mysql` and the `system` catalog, allow users from the `finance` and +`human_resources` groups access to `postgres` catalog, allow all users to +access the `hive` catalog, and deny all other access, you can use the +following rules: + +```json +{ + "catalogs": [ + { + "role": "admin", + "catalog": "(mysql|system)", + "allow": "all" + }, + { + "group": "finance|human_resources", + "catalog": "postgres", + "allow": true + }, + { + "catalog": "hive", + "allow": "all" + }, + { + "user": "alice", + "catalog": "postgresql", + "allow": "read-only" + }, + { + "catalog": "system", + "allow": "none" + } + ] +} +``` + +For group-based rules to match, users need to be assigned to groups by a +{doc}`/develop/group-provider`. + +#### Schema rules + +Each schema rule is composed of the following fields: + +- `user` (optional): regex to match against user name. Defaults to `.*`. +- `role` (optional): regex to match against role names. Defaults to `.*`. +- `group` (optional): regex to match against group names. Defaults to `.*`. +- `catalog` (optional): regex to match against catalog name. Defaults to + `.*`. +- `schema` (optional): regex to match against schema name. Defaults to + `.*`. +- `owner` (required): boolean indicating whether the user is to be considered + an owner of the schema. Defaults to `false`. + +For example, to provide ownership of all schemas to role `admin`, treat all +users as owners of the `default.default` schema and prevent user `guest` +from ownership of any schema, you can use the following rules: + +```json +{ + "schemas": [ + { + "role": "admin", + "schema": ".*", + "owner": true + }, + { + "user": "guest", + "owner": false + }, + { + "catalog": "default", + "schema": "default", + "owner": true + } + ] +} +``` + +#### Table rules + +Each table rule is composed of the following fields: + +- `user` (optional): regex to match against user name. Defaults to `.*`. +- `role` (optional): regex to match against role names. Defaults to `.*`. +- `group` (optional): regex to match against group names. Defaults to `.*`. +- `catalog` (optional): regex to match against catalog name. Defaults to + `.*`. +- `schema` (optional): regex to match against schema name. Defaults to `.*`. +- `table` (optional): regex to match against table names. Defaults to `.*`. +- `privileges` (required): zero or more of `SELECT`, `INSERT`, + `DELETE`, `UPDATE`, `OWNERSHIP`, `GRANT_SELECT` +- `columns` (optional): list of column constraints. +- `filter` (optional): boolean filter expression for the table. +- `filter_environment` (optional): environment use during filter evaluation. + +#### Column constraint + +These constraints can be used to restrict access to column data. + +- `name`: name of the column. +- `allow` (optional): if false, column can not be accessed. +- `mask` (optional): mask expression applied to column. +- `mask_environment` (optional): environment use during mask evaluation. + +#### Filter and mask environment + +- `user` (optional): username for checking permission of subqueries in mask. + +:::{note} +These rules do not apply to `information_schema`. + +`mask` can contain conditional expressions such as `IF` or `CASE`, which achieves conditional masking. +::: + +The example below defines the following table access policy: + +- Role `admin` has all privileges across all tables and schemas +- User `banned_user` has no privileges +- All users have `SELECT` privileges on `default.hr.employees`, but the + table is filtered to only the row for the current user. +- All users have `SELECT` privileges on all tables in the `default.default` + schema, except for the `address` column which is blocked, and `ssn` which + is masked. + +```json +{ + "tables": [ + { + "role": "admin", + "privileges": ["SELECT", "INSERT", "DELETE", "UPDATE", "OWNERSHIP"] + }, + { + "user": "banned_user", + "privileges": [] + }, + { + "catalog": "default", + "schema": "hr", + "table": "employee", + "privileges": ["SELECT"], + "filter": "user = current_user", + "filter_environment": { + "user": "system_user" + } + }, + { + "catalog": "default", + "schema": "default", + "table": ".*", + "privileges": ["SELECT"], + "columns" : [ + { + "name": "address", + "allow": false + }, + { + "name": "SSN", + "mask": "'XXX-XX-' + substring(credit_card, -4)", + "mask_environment": { + "user": "system_user" + } + } + ] + } + ] +} +``` + +(system-file-function-rules)= + +#### Function rules + +These rules control the user's ability to execute functions. + +:::{note} +Users always have access to functions in the `system.builtin` schema, and +you cannot override this behavior by adding a rule. +::: + +Each function rule is composed of the following fields: + +- `user` (optional): regular expression to match against user name. + Defaults to `.*`. +- `role` (optional): regular expression to match against role names. + Defaults to `.*`. +- `group` (optional): regular expression to match against group names. + Defaults to `.*`. +- `catalog` (optional): regular expression to match against catalog name. + Defaults to `.*`. +- `schema` (optional): regular expression to match against schema name. + Defaults to `.*`. +- `function` (optional): regular expression to match against function names. + Defaults to `.*`. +- `privileges` (required): zero or more of `EXECUTE`, `GRANT_EXECUTE`. + +To explicitly allow the system builtin functions in queries (and SECURITY +DEFINER views), you can use the following rule: + +```json +{ + "functions": [ + { + "catalog": "system", + "schema": "builtin", + "privileges": [ + "EXECUTE", + "GRANT_EXECUTE" + ] + } + ] +} +``` + +Care should be taken when granting permission to the `system` schema of any +catalog, as this is the schema Trino uses for table function such as `query`. +These table functions can be used to access or modify the underlying data of +the catalog. + +The following example allows the `admin` user to execute `query` table +function from any catalog: + +```json +{ + "functions": [ + { + "catalog": "system", + "schema": "builtin", + "privileges": [ + "EXECUTE", + "GRANT_EXECUTE" + ] + }, + { + "user": "admin", + "schema": "system", + "function": "query", + "privileges": [ + "EXECUTE" + ] + } + ] +} +``` + +(verify-rules)= + +#### Verify configuration + +To verify the system-access control file is configured properly, set the +rules to completely block access to all users of the system: + +```json +{ + "catalogs": [ + { + "catalog": "system", + "allow": "none" + } + ] +} +``` + +Restart your cluster to activate the rules for your cluster. With the +Trino {doc}`CLI ` run a query to test authorization: + +```text +trino> SELECT * FROM system.runtime.nodes; +Query 20200824_183358_00000_c62aw failed: Access Denied: Cannot access catalog system +``` + +Remove these rules and restart the Trino cluster. + +(system-file-auth-session-property)= + +### Session property rules + +These rules control the ability of a user to set system and catalog session +properties. The user is granted or denied access, based on the first matching +rule, read from top to bottom. If no rules are specified, all users are allowed +set any session property. If no rule matches, setting the session property is +denied. System session property rules are composed of the following fields: + +- `user` (optional): regex to match against user name. Defaults to `.*`. +- `role` (optional): regex to match against role names. Defaults to `.*`. +- `group` (optional): regex to match against group names. Defaults to `.*`. +- `property` (optional): regex to match against the property name. Defaults to + `.*`. +- `allow` (required): boolean indicating if the setting the session + property should be allowed. + +The catalog session property rules have the additional field: + +- `catalog` (optional): regex to match against catalog name. Defaults to + `.*`. + +The example below defines the following table access policy: + +- Role `admin` can set all session property +- User `banned_user` can not set any session properties +- All users can set the `resource_overcommit` system session property, and the + `bucket_execution_enabled` session property in the `hive` catalog. + +```{literalinclude} session-property-access.json +:language: json +``` + +(query-rules)= + +### Query rules + +These rules control the ability of a user to execute, view, or kill a query. The +user is granted or denied access, based on the first matching rule read from top +to bottom. If no rules are specified, all users are allowed to execute queries, +and to view or kill queries owned by any user. If no rule matches, query +management is denied. Each rule is composed of the following fields: + +- `user` (optional): regex to match against user name. Defaults to `.*`. +- `role` (optional): regex to match against role names. Defaults to `.*`. +- `group` (optional): regex to match against group names. Defaults to `.*`. +- `queryOwner` (optional): regex to match against the query owner name. + Defaults to `.*`. +- `allow` (required): set of query permissions granted to user. Values: + `execute`, `view`, `kill` + +:::{note} +Users always have permission to view or kill their own queries. + +A rule that includes `queryOwner` may not include the `execute` access mode. +Queries are only owned by a user once their execution has begun. +::: + +For example, if you want to allow the role `admin` full query access, allow +the user `alice` to execute and kill queries, allow members of the group +`contractors` to view queries owned by users `alice` or `dave`, allow any +user to execute queries, and deny all other access, you can use the following +rules: + +```{literalinclude} query-access.json +:language: json +``` + +(system-file-auth-impersonation-rules)= + +### Impersonation rules + +These rules control the ability of a user to impersonate another user. In +some environments it is desirable for an administrator (or managed system) to +run queries on behalf of other users. In these cases, the administrator +authenticates using their credentials, and then submits a query as a different +user. When the user context is changed, Trino verifies that the administrator +is authorized to run queries as the target user. + +When these rules are present, the authorization is based on the first matching +rule, processed from top to bottom. If no rules match, the authorization is +denied. If impersonation rules are not present but the legacy principal rules +are specified, it is assumed impersonation access control is being handled by +the principal rules, so impersonation is allowed. If neither impersonation nor +principal rules are defined, impersonation is not allowed. + +Each impersonation rule is composed of the following fields: + +- `original_user` (optional): regex to match against the user requesting the + impersonation. Defaults to `.*`. +- `original_role` (optional): regex to match against role names of the + requesting impersonation. Defaults to `.*`. +- `new_user` (required): regex to match against the user to impersonate. Can + contain references to subsequences captured during the match against + *original_user*, and each reference is replaced by the result of evaluating + the corresponding group respectively. +- `allow` (optional): boolean indicating if the authentication should be + allowed. Defaults to `true`. + +The impersonation rules are a bit different than the other rules: The attribute +`new_user` is required to not accidentally prevent more access than intended. +Doing so it was possible to make the attribute `allow` optional. + +The following example allows the `admin` role, to impersonate any user, except +for `bob`. It also allows any user to impersonate the `test` user. It also +allows a user in the form `team_backend` to impersonate the +`team_backend_sandbox` user, but not arbitrary users: + +```{literalinclude} user-impersonation.json +:language: json +``` + +(system-file-auth-principal-rules)= + +### Principal rules + +:::{warning} +Principal rules are deprecated. Instead, use {doc}`/security/user-mapping` +which specifies how a complex authentication user name is mapped to a simple +user name for Trino, and impersonation rules defined above. +::: + +These rules serve to enforce a specific matching between a principal and a +specified user name. The principal is granted authorization as a user, based +on the first matching rule read from top to bottom. If no rules are specified, +no checks are performed. If no rule matches, user authorization is denied. +Each rule is composed of the following fields: + +- `principal` (required): regex to match and group against principal. +- `user` (optional): regex to match against user name. If matched, it + grants or denies the authorization based on the value of `allow`. +- `principal_to_user` (optional): replacement string to substitute against + principal. If the result of the substitution is same as the user name, it + grants or denies the authorization based on the value of `allow`. +- `allow` (required): boolean indicating whether a principal can be authorized + as a user. + +:::{note} +You would at least specify one criterion in a principal rule. If you specify +both criteria in a principal rule, it returns the desired conclusion when +either of criteria is satisfied. +::: + +The following implements an exact matching of the full principal name for LDAP +and Kerberos authentication: + +```json +{ + "principals": [ + { + "principal": "(.*)", + "principal_to_user": "$1", + "allow": true + }, + { + "principal": "([^/]+)(/.*)?@.*", + "principal_to_user": "$1", + "allow": true + } + ] +} +``` + +If you want to allow users to use the exact same name as their Kerberos +principal name, and allow `alice` and `bob` to use a group principal named +as `group@example.net`, you can use the following rules. + +```json +{ + "principals": [ + { + "principal": "([^/]+)/?.*@example.net", + "principal_to_user": "$1", + "allow": true + }, + { + "principal": "group@example.net", + "user": "alice|bob", + "allow": true + } + ] +} +``` + +(system-file-auth-system-information)= + +### System information rules + +These rules specify which users can access the system information management +interface. System information access includes the following aspects: + +- Read access to details such as Trino version, uptime of the node, and others + from the `/v1/info` and `/v1/status` REST endpoints. +- Read access with the {doc}`system information functions `. +- Read access with the {doc}`/connector/system`. +- Write access to trigger {doc}`/admin/graceful-shutdown`. + +The user is granted or denied access based on the first matching +rule read from top to bottom. If no rules are specified, all access to system +information is denied. If no rule matches, system access is denied. Each rule is +composed of the following fields: + +- `role` (optional): regex to match against role. If matched, it + grants or denies the authorization based on the value of `allow`. +- `user` (optional): regex to match against user name. If matched, it + grants or denies the authorization based on the value of `allow`. +- `allow` (required): set of access permissions granted to user. Values: + `read`, `write` + +The following configuration provides and example: + +```{literalinclude} system-information-access.json +:language: json +``` + +- All users with the `admin` role have read and write access to system + information. This includes the ability to trigger + {doc}`/admin/graceful-shutdown`. +- The user `alice` can read system information. +- All other users and roles are denied access to system information. + +A fixed user can be set for management interfaces using the `management.user` +configuration property. When this is configured, system information rules must +still be set to authorize this user to read or write to management information. +The fixed management user only applies to HTTP by default. To enable the fixed +user over HTTPS, set the `management.user.https-enabled` configuration +property. + +(system-file-auth-authorization)= + +### Authorization rules + +These rules control the ability of how owner of schema, table or view can +be altered. These rules are applicable to commands like: + +> ALTER SCHEMA name SET AUTHORIZATION ( user | USER user | ROLE role ) +> ALTER TABLE name SET AUTHORIZATION ( user | USER user | ROLE role ) +> ALTER VIEW name SET AUTHORIZATION ( user | USER user | ROLE role ) + +When these rules are present, the authorization is based on the first matching +rule, processed from top to bottom. If no rules match, the authorization is +denied. + +Notice that in order to execute `ALTER` command on schema, table or view user requires `OWNERSHIP` +privilege. + +Each authorization rule is composed of the following fields: + +- `original_user` (optional): regex to match against the user requesting the + authorization. Defaults to `.*`. +- `original_group` (optional): regex to match against group names of the + requesting authorization. Defaults to `.*`. +- `original_role` (optional): regex to match against role names of the + requesting authorization. Defaults to `.*`. +- `new_user` (optional): regex to match against the new owner user of the schema, table or view. + By default it does not match. +- `new_role` (optional): regex to match against the new owner role of the schema, table or view. + By default it does not match. +- `allow` (optional): boolean indicating if the authentication should be + allowed. Defaults to `true`. + +Notice that `new_user` and `new_role` are optional, however it is required to provide at least one of them. + +The following example allows the `admin` role, to change owner of any schema, table or view +to any user, except to\`\`bob\`\`. + +```{literalinclude} authorization.json +:language: json +``` + +(system-file-auth-system-information-1)= + +(catalog-file-based-access-control)= + +## Catalog-level access control files + +You can create JSON files for individual catalogs that define authorization +rules specific to that catalog. To enable catalog-level access control files, +add a connector-specific catalog configuration property that sets the +authorization type to `FILE` and the `security.config-file` catalog +configuration property that specifies the JSON rules file. + +For example, the following Iceberg catalog configuration properties use the +`rules.json` file for catalog-level access control: + +```properties +iceberg.security=FILE +security.config-file=etc/catalog/rules.json +``` + +Catalog-level access control files are supported on a per-connector basis, refer +to the connector documentation for more information. + +:::{note} +These rules do not apply to system-defined tables in the +`information_schema` schema. +::: + +### Configure a catalog rules file + +The configuration file is specified in JSON format. This file is composed of +the following sections, each of which is a list of rules that are processed in +order from top to bottom: + +1. `schemas` +2. `tables` +3. `session_properties` + +The user is granted the privileges from the first matching rule. All regexes +default to `.*` if not specified. + +#### Schema rules + +These rules govern who is considered an owner of a schema. + +- `user` (optional): regex to match against user name. +- `group` (optional): regex to match against every user group the user belongs + to. +- `schema` (optional): regex to match against schema name. +- `owner` (required): boolean indicating ownership. + +#### Table rules + +These rules govern the privileges granted on specific tables. + +- `user` (optional): regex to match against user name. +- `group` (optional): regex to match against every user group the user belongs + to. +- `schema` (optional): regex to match against schema name. +- `table` (optional): regex to match against table name. +- `privileges` (required): zero or more of `SELECT`, `INSERT`, + `DELETE`, `UPDATE`, `OWNERSHIP`, `GRANT_SELECT`. +- `columns` (optional): list of column constraints. +- `filter` (optional): boolean filter expression for the table. +- `filter_environment` (optional): environment used during filter evaluation. + +##### Column constraints + +These constraints can be used to restrict access to column data. + +- `name`: name of the column. +- `allow` (optional): if false, column can not be accessed. +- `mask` (optional): mask expression applied to column. +- `mask_environment` (optional): environment use during mask evaluation. + +##### Filter environment and mask environment + +These rules apply to `filter_environment` and `mask_environment`. + +- `user` (optional): username for checking permission of subqueries in a mask. + +:::{note} +`mask` can contain conditional expressions such as `IF` or `CASE`, which achieves conditional masking. +::: + +#### Function rules + +Each function rule is composed of the following fields: + +- `user` (optional): regular expression to match against user name. + Defaults to `.*`. +- `group` (optional): regular expression to match against group names. + Defaults to `.*`. +- `schema` (optional): regular expression to match against schema name. + Defaults to `.*`. +- `function` (optional): regular expression to match against function names. + Defaults to `.*`. +- `privileges` (required): zero or more of `EXECUTE`, `GRANT_EXECUTE`. + +#### Session property rules + +These rules govern who may set session properties. + +- `user` (optional): regex to match against user name. +- `group` (optional): regex to match against every user group the user belongs + to. +- `property` (optional): regex to match against session property name. +- `allow` (required): boolean indicating whether this session property may be + set. + +### Example + +```json +{ + "schemas": [ + { + "user": "admin", + "schema": ".*", + "owner": true + }, + { + "group": "finance|human_resources", + "schema": "employees", + "owner": true + }, + { + "user": "guest", + "owner": false + }, + { + "schema": "default", + "owner": true + } + ], + "tables": [ + { + "user": "admin", + "privileges": ["SELECT", "INSERT", "DELETE", "UPDATE", "OWNERSHIP"] + }, + { + "user": "banned_user", + "privileges": [] + }, + { + "schema": "hr", + "table": "employee", + "privileges": ["SELECT"], + "filter": "user = current_user" + }, + { + "schema": "default", + "table": ".*", + "privileges": ["SELECT"], + "columns" : [ + { + "name": "address", + "allow": false + }, + { + "name": "ssn", + "mask": "'XXX-XX-' + substring(credit_card, -4)", + "mask_environment": { + "user": "admin" + } + } + ] + } + ], + "session_properties": [ + { + "property": "force_local_scheduling", + "allow": true + }, + { + "user": "admin", + "property": "max_split_size", + "allow": true + } + ] +} +``` diff --git a/430/_sources/security/group-file.md.txt b/430/_sources/security/group-file.md.txt new file mode 100644 index 000000000..17abc9a7f --- /dev/null +++ b/430/_sources/security/group-file.md.txt @@ -0,0 +1,33 @@ +# File group provider + +Trino can map user names onto groups for easier access control and +resource group management. Group file resolves group membership using +a file on the coordinator. + +## Group file configuration + +Enable group file by creating an `etc/group-provider.properties` +file on the coordinator: + +```text +group-provider.name=file +file.group-file=/path/to/group.txt +``` + +The following configuration properties are available: + +| Property | Description | +| --------------------- | ----------------------------------------------------- | +| `file.group-file` | Path of the group file. | +| `file.refresh-period` | How often to reload the group file. Defaults to `5s`. | + +## Group files + +### File format + +The group file contains a list of groups and members, one per line, +separated by a colon. Users are separated by a comma. + +```text +group_name:user_1,user_2,user_3 +``` diff --git a/430/_sources/security/inspect-jks.md.txt b/430/_sources/security/inspect-jks.md.txt new file mode 100644 index 000000000..2b092f292 --- /dev/null +++ b/430/_sources/security/inspect-jks.md.txt @@ -0,0 +1,129 @@ +# JKS files + +This topic describes how to validate a {ref}`Java keystore (JKS) ` +file used to configure {doc}`/security/tls`. + +The Java KeyStore (JKS) system is provided as part of your Java installation. +Private keys and certificates for your server are stored in a *keystore* file. +The JKS system supports both PKCS #12 `.p12` files as well as legacy +keystore `.jks` files. + +The keystore file itself is always password-protected. The keystore file can +have more than one key in the the same file, each addressed by its **alias** +name. + +If you receive a keystore file from your site's network admin group, verify that +it shows the correct information for your Trino cluster, as described next. + +(troubleshooting-keystore)= + +## Inspect and validate keystore + +Inspect the keystore file to make sure it contains the correct information for +your Trino server. Use the `keytool` command, which is installed as part of +your Java installation, to retrieve information from your keystore file: + +```text +keytool -list -v -keystore yourKeystore.jks +``` + +Keystores always require a password. If not provided on the `keytool` command +line, `keytool` prompts for the password. + +Independent of the keystore's password, it is possible that an individual key +has its own password. It is easiest to make sure these passwords are the same. +If the JKS key inside the keystore has a different password, you are prompted +twice. + +In the output of the `keytool -list` command, look for: + +- The keystore may contain either a private key (`Entry type: + PrivateKeyEntry`) or certificate (`Entry type: trustedCertEntry`) or both. + +- Modern browsers now enforce 398 days as the maximum validity period for a + certificate. Look for the `Valid from ... until` entry, and make sure the + time span does not exceed 398 days. + +- Modern browsers and clients require the **SubjectAlternativeName** (SAN) + field. Make sure this shows the DNS name of your server, such as + `DNS:cluster.example.com`. Certificates without SANs are not + supported. + + Example: + +```text +SubjectAlternativeName [ + DNSName: cluster.example.com +] +``` + +If your keystore shows valid information for your cluster, proceed to configure +the Trino server, as described in {ref}`cert-placement` and +{ref}`configure-https`. + +The rest of this page describes additional steps that may apply in certain +circumstances. + +(import-to-keystore)= + +## Extra: add PEM to keystore + +Your site may have standardized on using JKS semantics for all servers. If a +vendor sends you a PEM-encoded certificate file for your Trino server, you can +import it into a keystore with a command like the following. Consult `keytool` +references for different options. + +```shell +keytool -trustcacerts -import -alias cluster -file localhost.pem -keystore localkeys.jks +``` + +If the specified keystore file exists, `keytool` prompts for its password. If +you are creating a new keystore, `keytool` prompts for a new password, then +prompts you to confirm the same password. `keytool` shows you the +contents of the key being added, similar to the `keytool -list` format, then +prompts: + +```text +Trust this certificate? [no]: +``` + +Type `yes` to add the PEM certificate to the keystore. + +The `alias` name is an arbitrary string used as a handle for the certificate +you are adding. A keystore can contain multiple keys and certs, so `keytool` +uses the alias to address individual entries. + +(cli-java-truststore)= + +## Extra: Java truststores + +:::{note} +Remember that there may be no need to identify a local truststore when +directly using a signed PEM-encoded certificate, independent of a keystore. +PEM certs can contain the server's private key and the certificate chain all +the way back to a recognzied CA. +::: + +Truststore files contain a list of {ref}`Certificate Authorities ` +trusted by Java to validate the private keys of servers, plus a list of the +certificates of trusted TLS servers. The standard Java-provided truststore file, +`cacerts`, is part of your Java installation in a standard location. + +Keystores normally rely on the default location of the system truststore, which +therefore does not need to be configured. + +However, there are cases in which you need to use an alternate truststore. For +example, if your site relies on the JKS system, your network managers may have +appended site-specific, local CAs to the standard list, to validate locally +signed keys. + +If your server must use a custom truststore, identify its location in the +server's config properties file. For example: + +```text +http-server.https.truststore.path=/mnt/shared/certs/localcacerts +http-server.https.truststore.key= +``` + +If connecting clients such as browsers or the Trino CLI must be separately +configured, contact your site's network administrators for assistance. diff --git a/430/_sources/security/inspect-pem.md.txt b/430/_sources/security/inspect-pem.md.txt new file mode 100644 index 000000000..a5834a823 --- /dev/null +++ b/430/_sources/security/inspect-pem.md.txt @@ -0,0 +1,124 @@ +# PEM files + +PEM (Privacy Enhanced Mail) is a standard for public key and certificate +information, and an encoding standard used to transmit keys and certificates. + +Trino supports PEM files. If you want to use other supported formats, see: + +- {doc}`JKS keystores ` +- {ref}`PKCS 12 ` stores. (Look up alternate commands for these in + `openssl` references.) + +A single PEM file can contain either certificate or key pair information, or +both in the same file. Certified keys can contain a chain of certificates from +successive certificate authorities. + +Follow the steps in this topic to inspect and validate key and certificate in +PEM files. See {ref}`troubleshooting-keystore` to validate JKS keystores. + +(inspect-pems)= + +## Inspect PEM file + +The file name extensions shown on this page are examples only; there is no +extension naming standard. + +You may receive a single file that includes a private key and its certificate, +or separate files. If you received separate files, concatenate them into one, +typically in order from key to certificate. For example: + +```shell +cat clustercoord.key clustercoord.cert > clustercoord.pem +``` + +Next, use the `cat` command to view this plain text file. For example: + +```shell +cat clustercoord.pem | less +``` + +Make sure the PEM file shows at least one `KEY` and one `CERTIFICATE` +section. A key section looks something like the following: + +```text +-----BEGIN PRIVATE KEY----- +MIIEowIBAAKCAQEAwJL8CLeDFAHhZe3QOOF1vWt4Vuk9vyO38Y1y9SgBfB02b2jW +.... +-----END PRIVATE KEY----- +``` + +If your key section reports `BEGIN ENCRYPTED PRIVATE KEY` instead, this means +the key is encrypted and you must use the password to open or inspect the key. +You may have specified the password when requesting the key, or the password +could be assigned by your site's network managers. Note that password protected +PEM files are not supported by Trino. + +If your key section reports `BEGIN EC PRIVATE KEY` or `BEGIN DSA PRIVATE +KEY`, this designates a key using Elliptical Curve or DSA alternatives to RSA. + +The certificate section looks like the following example: + +```text +-----BEGIN CERTIFICATE----- +MIIDujCCAqICAQEwDQYJKoZIhvcNAQEFBQAwgaIxCzAJBgNVBAYTAlVTMRYwFAYD +.... +-----END CERTIFICATE----- +-----BEGIN CERTIFICATE----- +MIIDwjCCAqoCCQCxyqwZ9GK50jANBgkqhkiG9w0BAQsFADCBojELMAkGA1UEBhMC +.... +-----END CERTIFICATE----- +``` + +The file can show a single certificate section, or more than one to express a +chain of authorities, each certifying the previous. + +(validate-pems)= + +## Validate PEM key section + +This page presumes your system provides the `openssl` command from OpenSSL 1.1 +or later. + +Test an RSA private key's validity with the following command: + +```text +openssl rsa -in clustercoord.pem -check -noout +``` + +Look for the following confirmation message: + +```text +RSA key ok +``` + +:::{note} +Consult `openssl` references for the appropriate versions of the +verification commands for EC or DSA keys. +::: + +## Validate PEM certificate section + +Analyze the certificate section of your PEM file with the following `openssl` +command: + +```text +openssl x509 -in clustercoord.pem -text -noout +``` + +If your certificate was generated with a password, `openssl` prompts for it. +Note that password protected PEM files are not supported by Trino. + +In the output of the `openssl` command, look for the following +characteristics: + +- Modern browsers now enforce 398 days as the maximum validity period for a + certificate. Look for `Not Before` and `Not After` dates in the + `Validity` section of the output, and make sure the time span does not + exceed 398 days. +- Modern browsers and clients require the **Subject Alternative Name** (SAN) + field. Make sure this shows the DNS name of your server, such as + `DNS:clustercoord.example.com`. Certificates without SANs are not + supported. + +If your PEM file shows valid information for your cluster, proceed to configure +the server, as described in {ref}`cert-placement` and {ref}`configure-https`. diff --git a/430/_sources/security/internal-communication.md.txt b/430/_sources/security/internal-communication.md.txt new file mode 100644 index 000000000..ed6914306 --- /dev/null +++ b/430/_sources/security/internal-communication.md.txt @@ -0,0 +1,150 @@ +# Secure internal communication + +The Trino cluster can be configured to use secured communication with internal +authentication of the nodes in the cluster, and to optionally use added security +with {ref}`TLS `. + +## Configure shared secret + +Configure a shared secret to authenticate all communication between nodes of the +cluster. Use this configuration under the following conditions: + +- When opting to configure [internal TLS encryption](internal-tls) + between nodes of the cluster +- When using any {doc}`external authentication ` method + between clients and the coordinator + +Set the shared secret to the same value in {ref}`config.properties +` on all nodes of the cluster: + +```text +internal-communication.shared-secret= +``` + +A large random key is recommended, and can be generated with the following Linux +command: + +```text +openssl rand 512 | base64 +``` + +(verify-secrets)= + +### Verify configuration + +To verify shared secret configuration: + +1. Start your Trino cluster with two or more nodes configured with a shared + secret. +2. Connect to the {doc}`Web UI `. +3. Confirm the number of `ACTIVE WORKERS` equals the number of nodes + configured with your shared secret. +4. Change the value of the shared secret on one worker, and restart the worker. +5. Log in to the Web UI and confirm the number of `ACTIVE WORKERS` is one + less. The worker with the invalid secret is not authenticated, and therefore + not registered with the coordinator. +6. Stop your Trino cluster, revert the value change on the worker, and restart + your cluster. +7. Confirm the number of `ACTIVE WORKERS` equals the number of nodes + configured with your shared secret. + +(internal-tls)= + +## Configure internal TLS + +You can optionally add an extra layer of security by configuring the cluster to +encrypt communication between nodes with {ref}`TLS `. + +You can configure the coordinator and all workers to encrypt all communication +with each other using TLS. Every node in the cluster must be configured. Nodes +that have not been configured, or are configured incorrectly, are not able to +communicate with other nodes in the cluster. + +In typical deployments, you should enable {ref}`TLS directly on the coordinator +` for fully encrypted access to the cluster by client +tools. + +Enable TLS for internal communication with the following +configuration identical on all cluster nodes. + +1. Configure a shared secret for internal communication as described in + the preceding section. + +2. Enable automatic certificate creation and trust setup in + `etc/config.properties`: + + ```properties + internal-communication.https.required=true + ``` + +3. Change the URI for the discovery service to use HTTPS and point to the IP + address of the coordinator in `etc/config.properties`: + + ```properties + discovery.uri=https://: + ``` + + Note that using hostnames or fully qualified domain names for the URI is + not supported. The automatic certificate creation for internal TLS only + supports IP addresses. + +4. Enable the HTTPS endpoint on all workers. + + ```properties + http-server.https.enabled=true + http-server.https.port= + ``` + +5. Restart all nodes. + +Certificates are automatically created and used to ensure all communication +inside the cluster is secured with TLS. + +:::{warning} +Older versions of Trino required you to manually manage all the certificates +on the nodes. If you upgrade from this setup, you must remove the following +configuration properties: + +- `internal-communication.https.keystore.path` +- `internal-communication.https.truststore.path` +- `node.internal-address-source` +::: + +### Performance with SSL/TLS enabled + +Enabling encryption impacts performance. The performance degradation can vary +based on the environment, queries, and concurrency. + +For queries that do not require transferring too much data between the Trino +nodes e.g. `SELECT count(*) FROM table`, the performance impact is negligible. + +However, for CPU intensive queries which require a considerable amount of data +to be transferred between the nodes (for example, distributed joins, aggregations and +window functions, which require repartitioning), the performance impact can be +considerable. The slowdown may vary from 10% to even 100%+, depending on the network +traffic and the CPU utilization. + +### Advanced performance tuning + +In some cases, changing the source of random numbers improves performance +significantly. + +By default, TLS encryption uses the `/dev/urandom` system device as a source of entropy. +This device has limited throughput, so on environments with high network bandwidth +(e.g. InfiniBand), it may become a bottleneck. In such situations, it is recommended to try +to switch the random number generator algorithm to `SHA1PRNG`, by setting it via +`http-server.https.secure-random-algorithm` property in `config.properties` on the coordinator +and all of the workers: + +```text +http-server.https.secure-random-algorithm=SHA1PRNG +``` + +Be aware that this algorithm takes the initial seed from +the blocking `/dev/random` device. For environments that do not have enough entropy to seed +the `SHAPRNG` algorithm, the source can be changed to `/dev/urandom` +by adding the `java.security.egd` property to `jvm.config`: + +```text +-Djava.security.egd=file:/dev/urandom +``` diff --git a/430/_sources/security/jwt.md.txt b/430/_sources/security/jwt.md.txt new file mode 100644 index 000000000..32858227f --- /dev/null +++ b/430/_sources/security/jwt.md.txt @@ -0,0 +1,141 @@ +# JWT authentication + +Trino can be configured to authenticate client access using [JSON web tokens](https://wikipedia.org/wiki/JSON_Web_Token). A JWT is a small, web-safe +JSON file that contains cryptographic information similar to a certificate, +including: + +- Subject +- Valid time period +- Signature + +A JWT is designed to be passed between servers as proof of prior authentication +in a workflow like the following: + +1. An end user logs into a client application and requests access to a server. + +2. The server sends the user's credentials to a separate authentication service + that: + + - validates the user + - generates a JWT as proof of validation + - returns the JWT to the requesting server + +3. The same JWT can then be forwarded to other services to maintain the user's + validation without further credentials. + +:::{important} +If you are trying to configure OAuth2 or OIDC, there is a dedicated system +for that in Trino, as described in {doc}`/security/oauth2`. When using +OAuth2 authentication, you do not need to configure JWT authentication, +because JWTs are handled automatically by the OAuth2 code. + +A typical use for JWT authentication is to support administrators at large +sites who are writing their own single sign-on or proxy system to stand +between users and the Trino coordinator, where their new system submits +queries on behalf of users. +::: + +Using {doc}`TLS ` and {doc}`a configured shared secret +` is required for JWT authentication. + +## Using JWT authentication + +Trino supports Base64 encoded JWTs, but not encrypted JWTs. + +There are two ways to get the encryption key necessary to validate the JWT +signature: + +- Load the key from a JSON web key set (JWKS) endpoint service (the + typical case) +- Load the key from the local file system on the Trino coordinator + +A JWKS endpoint is a read-only service that contains public key information in +[JWK](https://datatracker.ietf.org/doc/html/rfc7517) format. These public +keys are the counterpart of the private keys that sign JSON web tokens. + +## JWT authentication configuration + +Enable JWT authentication by setting the {doc}`JWT authentication type +` in {ref}`etc/config.properties `, and +specifying a URL or path to a key file: + +```properties +http-server.authentication.type=JWT +http-server.authentication.jwt.key-file=https://cluster.example.net/.well-known/jwks.json +``` + +JWT authentication is typically used in addition to other authentication +methods: + +```properties +http-server.authentication.type=PASSWORD,JWT +http-server.authentication.jwt.key-file=https://cluster.example.net/.well-known/jwks.json +``` + +The following configuration properties are available: + +:::{list-table} Configuration properties for JWT authentication +:widths: 50 50 +:header-rows: 1 + +* - Property + - Description +* - `http-server.authentication.jwt.key-file` + - Required. Specifies either the URL to a JWKS service or the path to a PEM or + HMAC file, as described below this table. +* - `http-server.authentication.jwt.required-issuer` + - Specifies a string that must match the value of the JWT's issuer (`iss`) + field in order to consider this JWT valid. The `iss` field in the JWT + identifies the principal that issued the JWT. +* - `http-server.authentication.jwt.required-audience` + - Specifies a string that must match the value of the JWT's Audience (`aud`) + field in order to consider this JWT valid. The `aud` field in the JWT + identifies the recipients that the JWT is intended for. +* - `http-server.authentication.jwt.principal-field` + - String to identify the field in the JWT that identifies the subject of the + JWT. The default value is `sub`. This field is used to create the Trino + principal. +* - `http-server.authentication.jwt.user-mapping.pattern` + - A regular expression pattern to [map all user names](/security/user-mapping) + for this authentication system to the format expected by the Trino server. +* - `http-server.authentication.jwt.user-mapping.file` + - The path to a JSON file that contains a set of [user mapping + rules](/security/user-mapping) for this authentication system. +::: + +Use the `http-server.authentication.jwt.key-file` property to specify +either: + +- The URL to a JWKS endpoint service, where the URL begins with `https://`. + The JWKS service must be reachable from the coordinator. If the coordinator + is running in a secured or firewalled network, the administrator *may* have + to open access to the JWKS server host. + + :::{caution} + The Trino server also accepts JWKS URLs that begin with `http://`, but + using this protocol results in a severe security risk. Only use this + protocol for short-term testing during development of your cluster. + ::: + +- The path to a local file in {doc}`PEM ` or [HMAC](https://wikipedia.org/wiki/HMAC) format that contains a single key. + If the file path contains `$KEYID`, then Trino interpolates the `keyid` + from the JWT into the file path before loading this key. This enables support + for setups with multiple keys. + +## Using JWTs with clients + +When using the Trino {doc}`CLI `, specify a JWT as described +in {ref}`cli-jwt-auth`. + +When using the Trino JDBC driver, specify a JWT with the `accessToken` +{ref}`parameter `. + +## Resources + +The following resources may prove useful in your work with JWTs and JWKs. + +- [jwt.io](https://jwt.io) helps you decode and verify a JWT. +- [An article on using RS256](https://auth0.com/blog/navigating-rs256-and-jwks/) + to sign and verify your JWTs. +- An [online JSON web key](https://mkjwk.org) generator. +- A [command line JSON web key](https://connect2id.com/products/nimbus-jose-jwt/generator) generator. diff --git a/430/_sources/security/kerberos.md.txt b/430/_sources/security/kerberos.md.txt new file mode 100644 index 000000000..3d9a3c804 --- /dev/null +++ b/430/_sources/security/kerberos.md.txt @@ -0,0 +1,229 @@ +# Kerberos authentication + +Trino can be configured to enable Kerberos authentication over HTTPS for +clients, such as the {doc}`Trino CLI `, or the JDBC and ODBC +drivers. + +To enable Kerberos authentication for Trino, Kerberos-related configuration +changes are made on the Trino coordinator. + +Using {doc}`TLS ` and {doc}`a configured shared secret +` is required for Kerberos authentication. + +## Environment configuration + +(server-kerberos-services)= + +### Kerberos services + +You will need a Kerberos {abbr}`KDC (Key Distribution Center)` running on a +node that the Trino coordinator can reach over the network. The KDC is +responsible for authenticating principals and issuing session keys that can be +used with Kerberos-enabled services. KDCs typically run on port 88, which is +the IANA-assigned port for Kerberos. + +(server-kerberos-configuration)= + +### MIT Kerberos configuration + +Kerberos needs to be configured on the Trino coordinator. At a minimum, there needs +to be a `kdc` entry in the `[realms]` section of the `/etc/krb5.conf` +file. You may also want to include an `admin_server` entry and ensure that +the Trino coordinator can reach the Kerberos admin server on port 749. + +```text +[realms] + TRINO.EXAMPLE.COM = { + kdc = kdc.example.com + admin_server = kdc.example.com + } + +[domain_realm] + .trino.example.com = TRINO.EXAMPLE.COM + trino.example.com = TRINO.EXAMPLE.COM +``` + +The complete [documentation](http://web.mit.edu/kerberos/krb5-latest/doc/admin/conf_files/kdc_conf.html) +for `krb5.conf` is hosted by the MIT Kerberos Project. If you are using a +different implementation of the Kerberos protocol, you will need to adapt the +configuration to your environment. + +(server-kerberos-principals)= + +### Kerberos principals and keytab files + +The Trino coordinator needs a Kerberos principal, as do users who are going to +connect to the Trino coordinator. You need to create these users in Kerberos +using [kadmin](http://web.mit.edu/kerberos/krb5-latest/doc/admin/admin_commands/kadmin_local.html). + +In addition, the Trino coordinator needs a [keytab file](http://web.mit.edu/kerberos/krb5-devel/doc/basic/keytab_def.html). After you +create the principal, you can create the keytab file using {command}`kadmin` + +```text +kadmin +> addprinc -randkey trino@EXAMPLE.COM +> addprinc -randkey trino/trino-coordinator.example.com@EXAMPLE.COM +> ktadd -k /etc/trino/trino.keytab trino@EXAMPLE.COM +> ktadd -k /etc/trino/trino.keytab trino/trino-coordinator.example.com@EXAMPLE.COM +``` + +:::{note} +Running {command}`ktadd` randomizes the principal's keys. If you have just +created the principal, this does not matter. If the principal already exists, +and if existing users or services rely on being able to authenticate using a +password or a keytab, use the `-norandkey` option to {command}`ktadd`. +::: + +### Configuration for TLS + +When using Kerberos authentication, access to the Trino coordinator must be +through {doc}`TLS and HTTPS `. + +## System access control plugin + +A Trino coordinator with Kerberos enabled probably needs a +{doc}`/develop/system-access-control` plugin to achieve the desired level of +security. + +## Trino coordinator node configuration + +You must make the above changes to the environment prior to configuring the +Trino coordinator to use Kerberos authentication and HTTPS. After making the +following environment changes, you can make the changes to the Trino +configuration files. + +- {doc}`/security/tls` +- {ref}`server-kerberos-services` +- {ref}`server-kerberos-configuration` +- {ref}`server-kerberos-principals` +- {doc}`System Access Control Plugin ` + +### config.properties + +Kerberos authentication is configured in the coordinator node's +{file}`config.properties` file. The entries that need to be added are listed +below. + +```text +http-server.authentication.type=KERBEROS + +http-server.authentication.krb5.service-name=trino +http-server.authentication.krb5.principal-hostname=trino.example.com +http-server.authentication.krb5.keytab=/etc/trino/trino.keytab +http.authentication.krb5.config=/etc/krb5.conf + +http-server.https.enabled=true +http-server.https.port=7778 + +http-server.https.keystore.path=/etc/trino/keystore.jks +http-server.https.keystore.key=keystore_password + +node.internal-address-source=FQDN +``` + +| Property | Description | +| ------------------------------------------------------ | -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| `http-server.authentication.type` | Authentication type for the Trino coordinator. Must be set to `KERBEROS`. | +| `http-server.authentication.krb5.service-name` | The Kerberos service name for the Trino coordinator. Must match the Kerberos principal. | +| `http-server.authentication.krb5.principal-hostname` | The Kerberos hostname for the Trino coordinator. Must match the Kerberos principal. This parameter is optional. If included, Trino uses this value in the host part of the Kerberos principal instead of the machine's hostname. | +| `http-server.authentication.krb5.keytab` | The location of the keytab that can be used to authenticate the Kerberos principal. | +| `http.authentication.krb5.config` | The location of the Kerberos configuration file. | +| `http-server.https.enabled` | Enables HTTPS access for the Trino coordinator. Should be set to `true`. | +| `http-server.https.port` | HTTPS server port. | +| `http-server.https.keystore.path` | The location of the Java Keystore file that is used to secure TLS. | +| `http-server.https.keystore.key` | The password for the keystore. This must match the password you specified when creating the keystore. | +| `http-server.authentication.krb5.user-mapping.pattern` | Regex to match against user. If matched, user will be replaced with first regex group. If not matched, authentication is denied. Default is `(.*)`. | +| `http-server.authentication.krb5.user-mapping.file` | File containing rules for mapping user. See {doc}`/security/user-mapping` for more information. | +| `node.internal-address-source` | Kerberos is typically sensitive to DNS names. Setting this property to use `FQDN` ensures correct operation and usage of valid DNS host names. | + +See {ref}`Standards supported ` for a discussion of the +supported TLS versions and cipher suites. + +### access-control.properties + +At a minimum, an {file}`access-control.properties` file must contain an +`access-control.name` property. All other configuration is specific for the +implementation being configured. See {doc}`/develop/system-access-control` for +details. + +(coordinator-troubleshooting)= + +## User mapping + +After authenticating with Kerberos, the Trino server receives the user's +principal which is typically similar to an email address. For example, when +`alice` logs in Trino might receive `alice@example.com`. By default, Trino +uses the full Kerberos principal name, but this can be mapped to a shorter +name using a user-mapping pattern. For simple mapping rules, the +`http-server.authentication.krb5.user-mapping.pattern` configuration property +can be set to a Java regular expression, and Trino uses the value of the +first matcher group. If the regular expression does not match, the +authentication is denied. For more complex user-mapping rules, see +{doc}`/security/user-mapping`. + +## Troubleshooting + +Getting Kerberos authentication working can be challenging. You can +independently verify some of the configuration outside of Trino to help narrow +your focus when trying to solve a problem. + +### Kerberos verification + +Ensure that you can connect to the KDC from the Trino coordinator using +{command}`telnet`: + +```text +$ telnet kdc.example.com 88 +``` + +Verify that the keytab file can be used to successfully obtain a ticket using +[kinit](http://web.mit.edu/kerberos/krb5-1.12/doc/user/user_commands/kinit.html) and +[klist](http://web.mit.edu/kerberos/krb5-1.12/doc/user/user_commands/klist.html) + +```text +$ kinit -kt /etc/trino/trino.keytab trino@EXAMPLE.COM +$ klist +``` + +### Java keystore file verification + +Verify the password for a keystore file and view its contents using +{ref}`troubleshooting-keystore`. + +(kerberos-debug)= + +### Additional Kerberos debugging information + +You can enable additional Kerberos debugging information for the Trino +coordinator process by adding the following lines to the Trino `jvm.config` +file: + +```text +-Dsun.security.krb5.debug=true +-Dlog.enable-console=true +``` + +`-Dsun.security.krb5.debug=true` enables Kerberos debugging output from the +JRE Kerberos libraries. The debugging output goes to `stdout`, which Trino +redirects to the logging system. `-Dlog.enable-console=true` enables output +to `stdout` to appear in the logs. + +The amount and usefulness of the information the Kerberos debugging output +sends to the logs varies depending on where the authentication is failing. +Exception messages and stack traces can provide useful clues about the +nature of the problem. + +See [Troubleshooting Security](https://docs.oracle.com/en/java/javase/11/security/troubleshooting-security.html) +in the Java documentation for more details about the `-Djava.security.debug` +flag, and [Troubleshooting](https://docs.oracle.com/en/java/javase/11/security/troubleshooting.html) for +more details about the Java GSS-API and Kerberos issues. + +(server-additional-resources)= + +### Additional resources + +[Common Kerberos Error Messages (A-M)](http://docs.oracle.com/cd/E19253-01/816-4557/trouble-6/index.html) + +[Common Kerberos Error Messages (N-Z)](http://docs.oracle.com/cd/E19253-01/816-4557/trouble-27/index.html) + +[MIT Kerberos Documentation: Troubleshooting](http://web.mit.edu/kerberos/krb5-latest/doc/admin/troubleshoot.html) diff --git a/430/_sources/security/ldap.md.txt b/430/_sources/security/ldap.md.txt new file mode 100644 index 000000000..dd3cc7399 --- /dev/null +++ b/430/_sources/security/ldap.md.txt @@ -0,0 +1,274 @@ +# LDAP authentication + +Trino can be configured to enable frontend LDAP authentication over +HTTPS for clients, such as the {ref}`cli-ldap`, or the JDBC and ODBC +drivers. At present, only simple LDAP authentication mechanism involving +username and password is supported. The Trino client sends a username +and password to the coordinator, and the coordinator validates these +credentials using an external LDAP service. + +To enable LDAP authentication for Trino, LDAP-related configuration changes are +made on the Trino coordinator. + +Using {doc}`TLS ` and {doc}`a configured shared secret +` is required for LDAP authentication. + +## Trino server configuration + +### Trino coordinator node configuration + +Access to the Trino coordinator should be through HTTPS, configured as described +on {doc}`TLS and HTTPS `. + +You also need to make changes to the Trino configuration files. +LDAP authentication is configured on the coordinator in two parts. +The first part is to enable HTTPS support and password authentication +in the coordinator's `config.properties` file. The second part is +to configure LDAP as the password authenticator plugin. + +#### Server config properties + +The following is an example of the required properties that need to be added +to the coordinator's `config.properties` file: + +```text +http-server.authentication.type=PASSWORD + +http-server.https.enabled=true +http-server.https.port=8443 + +http-server.https.keystore.path=/etc/trino/keystore.jks +http-server.https.keystore.key=keystore_password +``` + +| Property | Description | +| ---------------------------------------------------------- | ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| `http-server.authentication.type` | Enable the password {doc}`authentication type ` for the Trino coordinator. Must be set to `PASSWORD`. | +| `http-server.https.enabled` | Enables HTTPS access for the Trino coordinator. Should be set to `true`. Default value is `false`. | +| `http-server.https.port` | HTTPS server port. | +| `http-server.https.keystore.path` | The location of the PEM or Java keystore file is used to enable TLS. | +| `http-server.https.keystore.key` | The password for the PEM or Java keystore. This must match the password you specified when creating the PEM or keystore. | +| `http-server.process-forwarded` | Enable treating forwarded HTTPS requests over HTTP as secure. Requires the `X-Forwarded-Proto` header to be set to `https` on forwarded requests. Default value is `false`. | +| `http-server.authentication.password.user-mapping.pattern` | Regex to match against user. If matched, user will be replaced with first regex group. If not matched, authentication is denied. Default is `(.*)`. | +| `http-server.authentication.password.user-mapping.file` | File containing rules for mapping user. See {doc}`/security/user-mapping` for more information. | + +#### Password authenticator configuration + +Password authentication must be configured to use LDAP. Create an +`etc/password-authenticator.properties` file on the coordinator. Example: + +```text +password-authenticator.name=ldap +ldap.url=ldaps://ldap-server:636 +ldap.ssl.truststore.path=/path/to/ldap_server.pem +ldap.user-bind-pattern= +``` + +| Property | Description | +| ------------------------------ | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ | +| `ldap.url` | The URL to the LDAP server. The URL scheme must be `ldap://` or `ldaps://`. Connecting to the LDAP server without TLS enabled requires `ldap.allow-insecure=true`. | +| `ldap.allow-insecure` | Allow using an LDAP connection that is not secured with TLS. | +| `ldap.ssl.keystore.path` | The path to the {doc}`PEM ` or {doc}`JKS ` keystore file. | +| `ldap.ssl.keystore.password` | Password for the key store. | +| `ldap.ssl.truststore.path` | The path to the {doc}`PEM ` or {doc}`JKS ` truststore file. | +| `ldap.ssl.truststore.password` | Password for the truststore. | +| `ldap.user-bind-pattern` | This property can be used to specify the LDAP user bind string for password authentication. This property must contain the pattern `${USER}`, which is replaced by the actual username during the password authentication.The property can contain multiple patterns separated by a colon. Each pattern will be checked in order until a login succeeds or all logins fail. Example: `${USER}@corp.example.com:${USER}@corp.example.co.uk` | +| `ldap.ignore-referrals` | Ignore referrals to other LDAP servers while performing search queries. Defaults to `false`. | +| `ldap.cache-ttl` | LDAP cache duration. Defaults to `1h`. | +| `ldap.timeout.connect` | Timeout for establishing an LDAP connection. | +| `ldap.timeout.read` | Timeout for reading data from an LDAP connection. | + +Based on the LDAP server implementation type, the property +`ldap.user-bind-pattern` can be used as described below. + +##### Active Directory + +```text +ldap.user-bind-pattern=${USER}@ +``` + +Example: + +```text +ldap.user-bind-pattern=${USER}@corp.example.com +``` + +##### OpenLDAP + +```text +ldap.user-bind-pattern=uid=${USER}, +``` + +Example: + +```text +ldap.user-bind-pattern=uid=${USER},OU=America,DC=corp,DC=example,DC=com +``` + +#### Authorization based on LDAP group membership + +You can further restrict the set of users allowed to connect to the Trino +coordinator, based on their group membership, by setting the optional +`ldap.group-auth-pattern` and `ldap.user-base-dn` properties, in addition +to the basic LDAP authentication properties. + +| Property | Description | +| ------------------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| `ldap.user-base-dn` | The base LDAP distinguished name for the user who tries to connect to the server. Example: `OU=America,DC=corp,DC=example,DC=com` | +| `ldap.group-auth-pattern` | This property is used to specify the LDAP query for the LDAP group membership authorization. This query is executed against the LDAP server and if successful, the user is authorized. This property must contain a pattern `${USER}`, which is replaced by the actual username in the group authorization search query. See samples below. | + +Based on the LDAP server implementation type, the property +`ldap.group-auth-pattern` can be used as described below. + +#### Authorization using Trino LDAP service user + +Trino server can use dedicated LDAP service user for doing user group membership queries. +In such case Trino will first issue a group membership query for a Trino user that needs +to be authenticated. A user distinguished name will be extracted from a group membership +query result. Trino will then validate user password by creating LDAP context with +user distinguished name and user password. In order to use this mechanism `ldap.bind-dn`, +`ldap.bind-password` and `ldap.group-auth-pattern` properties need to be defined. + +| Property | Description | +| ------------------------- | ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| `ldap.bind-dn` | Bind distinguished name used by Trino when issuing group membership queries. Example: `CN=admin,OU=CITY_OU,OU=STATE_OU,DC=domain` | +| `ldap.bind-password` | Bind password used by Trino when issuing group membership queries. Example: `password1234` | +| `ldap.group-auth-pattern` | This property is used to specify the LDAP query for the LDAP group membership authorization. This query will be executed against the LDAP server and if successful, a user distinguished name will be extracted from a query result. Trino will then validate user password by creating LDAP context with user distinguished name and user password. | + +##### Active Directory + +```text +ldap.group-auth-pattern=(&(objectClass=)(sAMAccountName=${USER})(memberof=)) +``` + +Example: + +```text +ldap.group-auth-pattern=(&(objectClass=person)(sAMAccountName=${USER})(memberof=CN=AuthorizedGroup,OU=Asia,DC=corp,DC=example,DC=com)) +``` + +##### OpenLDAP + +```text +ldap.group-auth-pattern=(&(objectClass=)(uid=${USER})(memberof=)) +``` + +Example: + +```text +ldap.group-auth-pattern=(&(objectClass=inetOrgPerson)(uid=${USER})(memberof=CN=AuthorizedGroup,OU=Asia,DC=corp,DC=example,DC=com)) +``` + +For OpenLDAP, for this query to work, make sure you enable the +`memberOf` [overlay](http://www.openldap.org/doc/admin24/overlays.html). + +You can use this property for scenarios where you want to authorize a user +based on complex group authorization search queries. For example, if you want to +authorize a user belonging to any one of multiple groups (in OpenLDAP), this +property may be set as follows: + +```text +ldap.group-auth-pattern=(&(|(memberOf=CN=normal_group,DC=corp,DC=com)(memberOf=CN=another_group,DC=com))(objectClass=inetOrgPerson)(uid=${USER})) +``` + +(cli-ldap)= + +## Trino CLI + +### Environment configuration + +#### TLS configuration + +When using LDAP authentication, access to the Trino coordinator must be through +{doc}`TLS/HTTPS `. + +### Trino CLI execution + +In addition to the options that are required when connecting to a Trino +coordinator that does not require LDAP authentication, invoking the CLI +with LDAP support enabled requires a number of additional command line +options. You can either use `--keystore-*` or `--truststore-*` properties +to secure TLS connection. The simplest way to invoke the CLI is with a +wrapper script. + +```text +#!/bin/bash + +./trino \ +--server https://trino-coordinator.example.com:8443 \ +--keystore-path /tmp/trino.jks \ +--keystore-password password \ +--truststore-path /tmp/trino_truststore.jks \ +--truststore-password password \ +--catalog \ +--schema \ +--user \ +--password +``` + +Find details on the options used in {ref}`cli-tls` and +{ref}`cli-username-password-auth`. + +## Troubleshooting + +### Java keystore file verification + +Verify the password for a keystore file and view its contents using +{ref}`troubleshooting-keystore`. + +### Debug Trino to LDAP server issues + +If you need to debug issues with Trino communicating with the LDAP server, +you can change the {ref}`log level ` for the LDAP authenticator: + +```none +io.trino.plugin.password=DEBUG +``` + +### TLS debugging for Trino CLI + +If you encounter any TLS related errors when running the Trino CLI, you can run +the CLI using the `-Djavax.net.debug=ssl` parameter for debugging. Use the +Trino CLI executable JAR to enable this. For example: + +```text +java -Djavax.net.debug=ssl \ +-jar \ +trino-cli--executable.jar \ +--server https://coordinator:8443 \ + +``` + +#### Common TLS/SSL errors + +##### java.security.cert.CertificateException: No subject alternative names present + +This error is seen when the Trino coordinator’s certificate is invalid, and does not have the IP you provide +in the `--server` argument of the CLI. You have to regenerate the coordinator's TLS certificate +with the appropriate {abbr}`SAN (Subject Alternative Name)` added. + +Adding a SAN to this certificate is required in cases where `https://` uses IP address in the URL, rather +than the domain contained in the coordinator's certificate, and the certificate does not contain the +{abbr}`SAN (Subject Alternative Name)` parameter with the matching IP address as an alternative attribute. + +#### Authentication or TLS errors with JDK upgrade + +Starting with the JDK 8u181 release, to improve the robustness of LDAPS +(secure LDAP over TLS) connections, endpoint identification algorithms were +enabled by default. See release notes +[from Oracle](https://www.oracle.com/technetwork/java/javase/8u181-relnotes-4479407.html#JDK-8200666.). +The same LDAP server certificate on the Trino coordinator, running on JDK +version >= 8u181, that was previously able to successfully connect to an +LDAPS server, may now fail with the following error: + +```text +javax.naming.CommunicationException: simple bind failed: ldapserver:636 +[Root exception is javax.net.ssl.SSLHandshakeException: java.security.cert.CertificateException: No subject alternative DNS name matching ldapserver found.] +``` + +If you want to temporarily disable endpoint identification, you can add the +property `-Dcom.sun.jndi.ldap.object.disableEndpointIdentification=true` +to Trino's `jvm.config` file. However, in a production environment, we +suggest fixing the issue by regenerating the LDAP server certificate so that +the certificate {abbr}`SAN (Subject Alternative Name)` or certificate subject +name matches the LDAP server. diff --git a/430/_sources/security/oauth2.md.txt b/430/_sources/security/oauth2.md.txt new file mode 100644 index 000000000..914340697 --- /dev/null +++ b/430/_sources/security/oauth2.md.txt @@ -0,0 +1,276 @@ +# OAuth 2.0 authentication + +Trino can be configured to enable OAuth 2.0 authentication over HTTPS for the +Web UI and the JDBC driver. Trino uses the [Authorization Code](https://tools.ietf.org/html/rfc6749#section-1.3.1) flow which exchanges an +Authorization Code for a token. At a high level, the flow includes the following +steps: + +1. the Trino coordinator redirects a user's browser to the Authorization Server +2. the user authenticates with the Authorization Server, and it approves the Trino's permissions request +3. the user's browser is redirected back to the Trino coordinator with an authorization code +4. the Trino coordinator exchanges the authorization code for a token + +To enable OAuth 2.0 authentication for Trino, configuration changes are made on +the Trino coordinator. No changes are required to the worker configuration; +only the communication from the clients to the coordinator is authenticated. + +Set the callback/redirect URL to `https:///oauth2/callback`, +when configuring an OAuth 2.0 authorization server like an OpenID Connect (OIDC) +provider. + +If Web UI is enabled, set the post-logout callback URL to +`https:///ui/logout/logout.html` when configuring +an OAuth 2.0 authentication server like an OpenID Connect (OIDC) provider. + +Using {doc}`TLS ` and {doc}`a configured shared secret +` is required for OAuth 2.0 authentication. + +## OpenID Connect Discovery + +Trino supports reading Authorization Server configuration from [OIDC provider +configuration metadata document](https://openid.net/specs/openid-connect-discovery-1_0.html#ProviderMetadata). +During startup of the coordinator Trino retrieves the document and uses provided +values to set corresponding OAuth2 authentication configuration properties: + +- `authorization_endpoint` -> `http-server.authentication.oauth2.auth-url` +- `token_endpoint` -> `http-server.authentication.oauth2.token-url` +- `jwks_uri` -> `http-server.authentication.oauth2.jwks-url` +- `userinfo_endpoint` -> `http-server.authentication.oauth2.userinfo-url` +- `access_token_issuer` -> `http-server.authentication.oauth2.access-token-issuer` +- `end_session_endpoint` -> `http-server.authentication.oauth2.end-session-url` + +:::{warning} +If the authorization server is issuing JSON Web Tokens (JWTs) and the +metadata document contains `userinfo_endpoint`, Trino uses this endpoint to +check the validity of OAuth2 access tokens. Since JWTs can be inspected +locally, using them against `userinfo_endpoint` may result in authentication +failure. In this case, set the +`http-server.authentication.oauth2.oidc.use-userinfo-endpoint` configuration +property to `false` +(`http-server.authentication.oauth2.oidc.use-userinfo-endpoint=false`). This +instructs Trino to ignore `userinfo_endpoint` and inspect tokens locally. +::: + +This functionality is enabled by default but can be turned off with: +`http-server.authentication.oauth2.oidc.discovery=false`. + +(trino-server-configuration-oauth2)= + +## Trino server configuration + +Using the OAuth2 authentication requires the Trino coordinator to be secured +with TLS. + +The following is an example of the required properties that need to be added +to the coordinator's `config.properties` file: + +```properties +http-server.authentication.type=oauth2 + +http-server.https.port=8443 +http-server.https.enabled=true + +http-server.authentication.oauth2.issuer=https://authorization-server.com +http-server.authentication.oauth2.client-id=CLIENT_ID +http-server.authentication.oauth2.client-secret=CLIENT_SECRET +``` + +To enable OAuth 2.0 authentication for the Web UI, the following +property must be be added: + +```properties +web-ui.authentication.type=oauth2 +``` + +The following configuration properties are available: + +:::{list-table} OAuth2 configuration properties +:widths: 40 60 +:header-rows: 1 + +* - Property + - Description +* - `http-server.authentication.type` + - The type of authentication to use. Must be set to `oauth2` to enable OAuth2 + authentication for the Trino coordinator. +* - `http-server.authentication.oauth2.issuer` + - The issuer URL of the IdP. All issued tokens must have this in the `iss` + field. +* - `http-server.authentication.oauth2.access-token-issuer` + - The issuer URL of the IdP for access tokens, if different. All issued access + tokens must have this in the `iss` field. Providing this value while OIDC + discovery is enabled overrides the value from the OpenID provider metadata + document. Defaults to the value of + `http-server.authentication.oauth2.issuer`. +* - `http-server.authentication.oauth2.auth-url` + - The authorization URL. The URL a user's browser will be redirected to in + order to begin the OAuth 2.0 authorization process. Providing this value + while OIDC discovery is enabled overrides the value from the OpenID provider + metadata document. +* - `http-server.authentication.oauth2.token-url` + - The URL of the endpoint on the authorization server which Trino uses to + obtain an access token. Providing this value while OIDC discovery is enabled + overrides the value from the OpenID provider metadata document. +* - `http-server.authentication.oauth2.jwks-url` + - The URL of the JSON Web Key Set (JWKS) endpoint on the authorization server. + It provides Trino the set of keys containing the public key to verify any + JSON Web Token (JWT) from the authorization server. Providing this value + while OIDC discovery is enabled overrides the value from the OpenID provider + metadata document. +* - `http-server.authentication.oauth2.userinfo-url` + - The URL of the IdPs `/userinfo` endpoint. If supplied then this URL is used + to validate the OAuth access token and retrieve any associated claims. This + is required if the IdP issues opaque tokens. Providing this value while OIDC + discovery is enabled overrides the value from the OpenID provider metadata + document. +* - `http-server.authentication.oauth2.client-id` + - The public identifier of the Trino client. +* - `http-server.authentication.oauth2.client-secret` + - The secret used to authorize Trino client with the authorization server. +* - `http-server.authentication.oauth2.additional-audiences` + - Additional audiences to trust in addition to the client ID which is + always a trusted audience. +* - `http-server.authentication.oauth2.scopes` + - Scopes requested by the server during the authorization challenge. See: + https://tools.ietf.org/html/rfc6749#section-3.3 +* - `http-server.authentication.oauth2.challenge-timeout` + - Maximum [duration](prop-type-duration) of the authorization challenge. + Default is `15m`. +* - `http-server.authentication.oauth2.state-key` + - A secret key used by the SHA-256 [HMAC](https://tools.ietf.org/html/rfc2104) + algorithm to sign the state parameter in order to ensure that the + authorization request was not forged. Default is a random string generated + during the coordinator start. +* - `http-server.authentication.oauth2.user-mapping.pattern` + - Regex to match against user. If matched, the user name is replaced with + first regex group. If not matched, authentication is denied. Default is + `(.*)` which allows any user name. +* - `http-server.authentication.oauth2.user-mapping.file` + - File containing rules for mapping user. See [](/security/user-mapping) for + more information. +* - `http-server.authentication.oauth2.principal-field` + - The field of the access token used for the Trino user principal. Defaults to + `sub`. Other commonly used fields include `sAMAccountName`, `name`, + `upn`, and `email`. +* - `http-server.authentication.oauth2.oidc.discovery` + - Enable reading the [OIDC provider metadata](https://openid.net/specs/openid-connect-discovery-1_0.html#ProviderMetadata). + Default is `true`. +* - `http-server.authentication.oauth2.oidc.discovery.timeout` + - The timeout when reading OpenID provider metadata. Default is `30s`. +* - `http-server.authentication.oauth2.oidc.use-userinfo-endpoint` + - Use the value of `userinfo_endpoint` in the [provider + metadata](https://openid.net/specs/openid-connect-discovery-1_0.html#ProviderMetadata). + When a `userinfo_endpoint` value is supplied this URL is used to validate + the OAuth 2.0 access token, and retrieve any associated claims. This flag + allows ignoring the value provided in the metadata document. Default is + `true`. +* - `http-server.authentication.oauth2.end-session-url` + - The URL of the endpoint on the authentication server to which the user's + browser is redirected to so that End-User is logged out from the + authentication server when logging out from Trino. +::: + +(trino-oauth2-refresh-tokens)= + +### Refresh tokens + +*Refresh tokens* allow you to securely control the length of user sessions +within applications. The refresh token has a longer lifespan (TTL) and is used +to refresh the *access token* that has a shorter lifespan. When refresh tokens +are used in conjunction with access tokens, users can remain logged in for an +extended duration without interruption by another login request. + +In a refresh token flow, there are three tokens with different expiration times: + +- access token +- refresh token +- Trino-encrypted token that is a combination of the access and refresh tokens. + The encrypted token manages the session lifetime with the timeout value that + is set with the + `http-server.authentication.oauth2.refresh-tokens.issued-token.timeout` + property. + +In the following scenario, the lifespan of the tokens issued by an IdP are: + +- access token 5m +- refresh token 24h + +Because the access token lifespan is only five minutes, Trino uses the longer +lifespan refresh token to request another access token every five minutes on +behalf of a user. In this case, the maximum +`http-server.authentication.oauth2.refresh-tokens.issued-token.timeout` is +twenty-four hours. + +To use refresh token flows, the following property must be +enabled in the coordinator configuration. + +```properties +http-server.authentication.oauth2.refresh-tokens=true +``` + +Additional scopes for offline access might be required, depending on +IdP configuration. + +```properties +http-server.authentication.oauth2.scopes=openid,offline_access [or offline] +``` + +The following configuration properties are available: + +:::{list-table} OAuth2 configuration properties for refresh flow +:widths: 40 60 +:header-rows: 1 + +* - Property + - Description +* - `http-server.authentication.oauth2.refresh-tokens.issued-token.timeout` + - Expiration time for an issued token, which is the Trino-encrypted token that + contains an access token and a refresh token. The timeout value must be less + than or equal to the [duration](prop-type-duration) of the refresh token + expiration issued by the IdP. Defaults to `1h`. The timeout value is the + maximum session time for an OAuth2-authenticated client with refresh tokens + enabled. For more details, see [](trino-oauth2-troubleshooting). +* - `http-server.authentication.oauth2.refresh-tokens.issued-token.issuer` + - Issuer representing the coordinator instance, that is referenced in the + issued token, defaults to `Trino_coordinator`. The current Trino version is + appended to the value. This is mainly used for debugging purposes. +* - `http-server.authentication.oauth2.refresh-tokens.issued-token.audience` + - Audience representing this coordinator instance, that is used in the + issued token. Defaults to `Trino_coordinator`. +* - `http-server.authentication.oauth2.refresh-tokens.secret-key` + - Base64-encoded secret key used to encrypt the generated token. By default + it's generated during startup. +::: + +(trino-oauth2-troubleshooting)= + +## Troubleshooting + +To debug issues, change the {ref}`log level ` for the OAuth 2.0 +authenticator: + +```none +io.trino.server.security.oauth2=DEBUG +``` + +To debug issues with OAuth 2.0 authentication use with the web UI, set the +following configuration property: + +```none +io.trino.server.ui.OAuth2WebUiAuthenticationFilter=DEBUG +``` + +This assumes the OAuth 2.0 authentication for the Web UI is enabled as described +in {ref}`trino-server-configuration-oauth2`. + +The logged debug error for a lapsed refresh token is `Tokens refresh challenge +has failed`. + +:::{warning} +If a refresh token lapses, the user session is interrupted and the user must +reauthenticate by logging in again. Ensure you set the +`http-server.authentication.oauth2.refresh-tokens.issued-token.timeout` +value to less than or equal to the duration of the refresh token expiration +issued by your IdP. Optimally, the timeout should be slightly less than the +refresh token lifespan of your IdP to ensure that sessions end gracefully. +::: diff --git a/430/_sources/security/overview.md.txt b/430/_sources/security/overview.md.txt new file mode 100644 index 000000000..1d82479a8 --- /dev/null +++ b/430/_sources/security/overview.md.txt @@ -0,0 +1,161 @@ +# Security overview + +After the initial {doc}`installation ` of your cluster, security +is the next major concern for successfully operating Trino. This overview +provides an introduction to different aspects of configuring security for your +Trino cluster. + +## Aspects of configuring security + +The default installation of Trino has no security features enabled. Security +can be enabled for different parts of the Trino architecture: + +- {ref}`security-client` +- {ref}`security-inside-cluster` +- {ref}`security-data-sources` + +## Suggested configuration workflow + +To configure security for a new Trino cluster, follow this best practice +order of steps. Do not skip or combine steps. + +1. **Enable** {doc}`TLS/HTTPS ` + + - Work with your security team. + - Use a {ref}`load balancer or proxy ` to terminate + HTTPS, if possible. + - Use a globally trusted TLS certificate. + + {ref}`Verify this step is working correctly.` + +2. **Configure** a {doc}`a shared secret ` + + {ref}`Verify this step is working correctly.` + +3. **Enable authentication** + + - Start with {doc}`password file authentication ` to get up + and running. + - Then configure your preferred authentication provider, such as {doc}`LDAP + `. + - Avoid the complexity of Kerberos for client authentication, if possible. + + {ref}`Verify this step is working correctly.` + +4. **Enable authorization and access control** + + - Start with {doc}`file-based rules `. + - Then configure another access control method as required. + + {ref}`Verify this step is working correctly. ` + +Configure one step at a time. Always restart the Trino server after each +change, and verify the results before proceeding. + +(security-client)= + +## Securing client access to the cluster + +Trino {doc}`clients ` include the Trino {doc}`CLI `, +the {doc}`Web UI `, the {doc}`JDBC driver +`, [Python, Go, or other clients](https://trino.io/resources.html), and any applications using these tools. + +All access to the Trino cluster is managed by the coordinator. Thus, securing +access to the cluster means securing access to the coordinator. + +There are three aspects to consider: + +- {ref}`cl-access-encrypt`: protecting the integrity of client to server + communication in transit. +- {ref}`cl-access-auth`: identifying users and user name management. +- {ref}`cl-access-control`: validating each user's access rights. + +(cl-access-encrypt)= + +### Encryption + +The Trino server uses the standard {doc}`HTTPS protocol and TLS encryption +`, formerly known as SSL. + +(cl-access-auth)= + +### Authentication + +Trino supports several authentication providers. When setting up a new cluster, +start with simple password file authentication before configuring another +provider. + +- {doc}`Password file authentication ` +- {doc}`LDAP authentication ` +- {doc}`Salesforce authentication ` +- {doc}`OAuth 2.0 authentication ` +- {doc}`Certificate authentication ` +- {doc}`JSON Web Token (JWT) authentication ` +- {doc}`Kerberos authentication ` + +(user-name-management)= + +#### User name management + +Trino provides ways to map the user and group names from authentication +providers to Trino user names. + +- {doc}`User mapping ` applies to all authentication systems, + and allows for regular expression rules to be specified that map complex user + names from other systems (`alice@example.com`) to simple user names + (`alice`). +- {doc}`File group provider ` provides a way to assign a set + of user names to a group name to ease access control. + +(cl-access-control)= + +### Authorization and access control + +Trino's {doc}`default method of access control ` +allows all operations for all authenticated users. + +To implement access control, use: + +- {doc}`File-based system access control `, where + you configure JSON files that specify fine-grained user access restrictions at + the catalog, schema, or table level. + +In addition, Trino {doc}`provides an API ` that +allows you to create a custom access control method, or to extend an existing +one. + +Access control can limit access to columns of a table. The default behavior +of a query to all columns with a `SELECT *` statement is to show an error +denying access to any inaccessible columns. + +You can change this behavior to silently hide inaccessible columns with the +global property `hide-inaccessible-columns` configured in +{ref}`config-properties`: + +```properties +hide-inaccessible-columns = true +``` + +(security-inside-cluster)= + +## Securing inside the cluster + +You can {doc}`secure the internal communication ` +between coordinator and workers inside the clusters. + +Secrets in properties files, such as passwords in catalog files, can be secured +with {doc}`secrets management `. + +(security-data-sources)= + +## Securing cluster access to data sources + +Communication between the Trino cluster and data sources is configured for each +catalog. Each catalog uses a connector, which supports a variety of +security-related configurations. + +More information is available with the documentation for individual +{doc}`connectors `. + +{doc}`Secrets management ` can be used for the catalog properties files +content. diff --git a/430/_sources/security/password-file.md.txt b/430/_sources/security/password-file.md.txt new file mode 100644 index 000000000..7d2d0b576 --- /dev/null +++ b/430/_sources/security/password-file.md.txt @@ -0,0 +1,120 @@ +# Password file authentication + +Trino can be configured to enable frontend password authentication over +HTTPS for clients, such as the CLI, or the JDBC and ODBC drivers. The +username and password are validated against usernames and passwords stored +in a file. + +Password file authentication is very similar to {doc}`ldap`. Please see +the LDAP documentation for generic instructions on configuring the server +and clients to use TLS and authenticate with a username and password. + +Using {doc}`TLS ` and {doc}`a configured shared secret +` is required for password file +authentication. + +## Password authenticator configuration + +To enable password file authentication, set the {doc}`password authentication +type ` in `etc/config.properties`: + +```properties +http-server.authentication.type=PASSWORD +``` + +In addition, create a `etc/password-authenticator.properties` file on the +coordinator with the `file` authenticator name: + +```text +password-authenticator.name=file +file.password-file=/path/to/password.db +``` + +The following configuration properties are available: + +| Property | Description | +| -------------------------------- | ----------------------------------------------------------------- | +| `file.password-file` | Path of the password file. | +| `file.refresh-period` | How often to reload the password file. Defaults to `5s`. | +| `file.auth-token-cache.max-size` | Max number of cached authenticated passwords. Defaults to `1000`. | + +## Password files + +### File format + +The password file contains a list of usernames and passwords, one per line, +separated by a colon. Passwords must be securely hashed using bcrypt or PBKDF2. + +bcrypt passwords start with `$2y$` and must use a minimum cost of `8`: + +```text +test:$2y$10$BqTb8hScP5DfcpmHo5PeyugxHz5Ky/qf3wrpD7SNm8sWuA3VlGqsa +``` + +PBKDF2 passwords are composed of the iteration count, followed by the +hex encoded salt and hash: + +```text +test:1000:5b4240333032306164:f38d165fce8ce42f59d366139ef5d9e1ca1247f0e06e503ee1a611dd9ec40876bb5edb8409f5abe5504aab6628e70cfb3d3a18e99d70357d295002c3d0a308a0 +``` + +### Creating a password file + +Password files utilizing the bcrypt format can be created using the +[htpasswd](https://httpd.apache.org/docs/current/programs/htpasswd.html) +utility from the [Apache HTTP Server](https://httpd.apache.org/). +The cost must be specified, as Trino enforces a higher minimum cost +than the default. + +Create an empty password file to get started: + +```text +touch password.db +``` + +Add or update the password for the user `test`: + +```text +htpasswd -B -C 10 password.db test +``` + +(verify-authentication)= + +### Verify configuration + +To verify password file authentication, log in to the {doc}`Web UI +`, and connect with the Trino {doc}`CLI ` to +the cluster: + +- Connect to the Web UI from your browser using a URL that uses HTTPS, such as + `https://trino.example.com:8443`. Enter a username in the `Username` text + box and the corresponding password in the `Password` text box, and log in to + the UI. Confirm that you are not able to log in using an incorrect username + and password combination. A successful login displays the username in the + top right corner of the UI. +- Connect with the Trino CLI using a URL that uses HTTPS, such as + `https://trino.example.net:8443` with the addition of the `--user` and + `--password` properties: + +```text +./trino --server https://trino.example.com:8443 --user test --password +``` + +The above command quotes you for a password. Supply the password set for the +user entered for the `--user` property to use the `trino>` prompt. Sucessful +authentication allows you to run queries from the CLI. + +To test the connection, send a query: + +```text +trino> SELECT 'rocks' AS trino; + +trino +------- +rocks +(1 row) + +Query 20220919_113804_00017_54qfi, FINISHED, 1 node +Splits: 1 total, 1 done (100.00%) +0.12 [0 rows, 0B] [0 rows/s, 0B/s] +``` diff --git a/430/_sources/security/salesforce.md.txt b/430/_sources/security/salesforce.md.txt new file mode 100644 index 000000000..84757d2a3 --- /dev/null +++ b/430/_sources/security/salesforce.md.txt @@ -0,0 +1,69 @@ +# Salesforce authentication + +Trino can be configured to enable frontend password authentication over +HTTPS for clients, such as the CLI, or the JDBC and ODBC drivers. The +username and password (or password and [security token](#security-token) concatenation) +are validated by having the Trino coordinator perform a login to Salesforce. + +This allows you to enable users to authenticate to Trino via their Salesforce +basic credentials. This can also be used to secure the {ref}`Web UI `. + +:::{note} +This is *not* a Salesforce connector, and does not allow users to query +Salesforce data. Salesforce authentication is simply a means by which users +can authenticate to Trino, similar to {doc}`ldap` or {doc}`password-file`. +::: + +Using {doc}`TLS ` and {doc}`a configured shared secret +` is required for Salesforce authentication. + +## Salesforce authenticator configuration + +To enable Salesfore authentication, set the {doc}`password authentication +type ` in `etc/config.properties`: + +```properties +http-server.authentication.type=PASSWORD +``` + +In addition, create a `etc/password-authenticator.properties` file on the +coordinator with the `salesforce` authenticator name: + +```properties +password-authenticator.name=salesforce +salesforce.allowed-organizations= +``` + +The following configuration properties are available: + +| Property | Description | +| ---------------------------------- | ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| `salesforce.allowed-organizations` | Comma separated list of 18 character Salesforce.com Organization IDs for a second, simple layer of security. This option can be explicitly ignored using `all`, which bypasses any check of the authenticated user's Salesforce.com Organization ID. | +| `salesforce.cache-size` | Maximum number of cached authenticated users. Defaults to `4096`. | +| `salesforce.cache-expire-duration` | How long a cached authentication should be considered valid. Defaults to `2m`. | + +## Salesforce concepts + +There are two Salesforce specific aspects to this authenticator. They are the use of the +Salesforce security token, and configuration of one or more Salesforce.com Organization IDs. + +(security-token)= +### Security token + +Credentials are a user's Salesforce username and password if Trino is connecting from a whitelisted +IP, or username and password/[security token](https://help.salesforce.com/articleView?id=user_security_token.htm&type=5) +concatenation otherwise. For example, if Trino is *not* whitelisted, and your password is `password` +and security token is `token`, use `passwordtoken` to authenticate. + +You can configure a public IP for Trino as a trusted IP by [whitelisting an IP range](https://help.salesforce.com/articleView?id=security_networkaccess.htm&type=5). + +### Salesforce.com organization IDs + +You can configure one or more Salesforce Organization IDs for additional security. When the user authenticates, +the Salesforce API returns the *18 character* Salesforce.com Organization ID for the user. The Trino Salesforce +authenticator ensures that the ID matches one of the IDs configured in `salesforce.allowed-organizations`. + +Optionally, you can configure `all` to explicitly ignore this layer of security. + +Admins can find their Salesforce.com Organization ID using the [Salesforce Setup UI](https://help.salesforce.com/articleView?id=000325251&type=1&mode=1). This will be the 15 character +ID, which can be [converted to the 18 character ID](https://sf1518.click/). diff --git a/430/_sources/security/secrets.md.txt b/430/_sources/security/secrets.md.txt new file mode 100644 index 000000000..ed68a46b9 --- /dev/null +++ b/430/_sources/security/secrets.md.txt @@ -0,0 +1,36 @@ +# Secrets + +Trino manages configuration details in static properties files. This +configuration needs to include values such as usernames, passwords and other +strings, that are often required to be kept secret. Only a few select +administrators or the provisioning system has access to the actual value. + +The secrets support in Trino allows you to use environment variables as values +for any configuration property. All properties files used by Trino, including +`config.properties` and catalog properties files, are supported. When loading +the properties, Trino replaces the reference to the environment variable with +the value of the environment variable. + +Environment variables are the most widely-supported means of setting and +retrieving values. Environment variables can be set in the scope of the task +being performed, preventing external access. Most provisioning and configuration +management systems include support for setting environment variables. This +includes systems such as Ansible, often used for virtual machines, and +Kubernetes for container usage. You can also manually set an environment +variable on the command line. + +```text +export DB_PASSWORD=my-super-secret-pwd +``` + +To use this variable in the properties file, you reference it with the syntax +`${ENV:VARIABLE}`. For example, if you want to use the password in a catalog +properties file like `etc/catalog/db.properties`, add the following line: + +```properties +connection-password=${ENV:DB_PASSWORD} +``` + +With this setup in place, the secret is managed by the provisioning system +or by the administrators handling the machines. No secret is stored in the Trino +configuration files on the filesystem or wherever they are managed. diff --git a/430/_sources/security/tls.md.txt b/430/_sources/security/tls.md.txt new file mode 100644 index 000000000..775a20f4e --- /dev/null +++ b/430/_sources/security/tls.md.txt @@ -0,0 +1,305 @@ +# TLS and HTTPS + +Trino runs with no security by default. This allows you to connect to the server +using URLs that specify the HTTP protocol when using the Trino {doc}`CLI +`, the {doc}`Web UI `, or other +clients. + +This topic describes how to configure your Trino server to use {ref}`TLS +` to require clients to use the HTTPS connection protocol. +All authentication technologies supported by Trino require configuring TLS as +the foundational layer. + +:::{important} +This page discusses only how to prepare the Trino server for secure client +connections from outside of the Trino cluster to its coordinator. +::: + +See the {doc}`Glossary ` to clarify unfamiliar terms. + +(tls-version-and-ciphers)= + +## Supported standards + +When configured to use TLS, the Trino server responds to client connections +using TLS 1.2 and TLS 1.3 certificates. The server rejects TLS 1.1, TLS 1.0, and +all SSL format certificates. + +The Trino server does not specify a set of supported ciphers, instead deferring +to the defaults set by the JVM version in use. The documentation for Java 17 +lists its [supported cipher suites](https://docs.oracle.com/en/java/javase/17/security/oracle-providers.html#GUID-7093246A-31A3-4304-AC5F-5FB6400405E2__SUNJSSE_CIPHER_SUITES). + +Run the following two-line code on the same JVM from the same vendor as +configured on the coordinator to determine that JVM's default cipher list. + +```shell +echo "java.util.Arrays.asList(((javax.net.ssl.SSLServerSocketFactory) \ +javax.net.ssl.SSLServerSocketFactory.getDefault()).getSupportedCipherSuites()).forEach(System.out::println)" | jshell - +``` + +The default Trino server specifies a set of regular expressions that exclude +older cipher suites that do not support forward secrecy (FS). + +Use the `http-server.https.included-cipher` property to specify a +comma-separated list of ciphers in preferred use order. If one of your preferred +selections is a non-FS cipher, you must also set the +`http-server.https.excluded-cipher` property to an empty list to override the +default exclusions. For example: + +```text +http-server.https.included-cipher=TLS_RSA_WITH_AES_128_CBC_SHA,TLS_RSA_WITH_AES_128_CBC_SHA256 +http-server.https.excluded-cipher= +``` + +Specifying a different cipher suite is a complex issue that should only be +considered in conjunction with your organization's security managers. Using a +different suite may require downloading and installing a different SunJCE +implementation package. Some locales may have export restrictions on cipher +suites. See the discussion in Java documentation that begins with [Customizing +the Encryption Algorithm Providers](https://docs.oracle.com/en/java/javase/17/security/java-secure-socket-extension-jsse-reference-guide.html#GUID-316FB978-7588-442E-B829-B4973DB3B584). + +:::{note} +If you manage the coordinator's direct TLS implementatation, monitor the CPU +usage on the Trino coordinator after enabling HTTPS. Java prefers the more +CPU-intensive cipher suites, if you allow it to choose from a big list of +ciphers. If the CPU usage is unacceptably high after enabling HTTPS, you can +configure Java to use specific cipher suites as described in this section. + +However, best practice is to instead use an external load balancer, as +discussed next. +::: + +## Approaches + +To configure Trino with TLS support, consider two alternative paths: + +- Use the {ref}`load balancer or proxy ` at your site + or cloud environment to terminate TLS/HTTPS. This approach is the simplest and + strongly preferred solution. +- Secure the Trino {ref}`server directly `. This + requires you to obtain a valid certificate, and add it to the Trino + coordinator's configuration. + +(https-load-balancer)= + +## Use a load balancer to terminate TLS/HTTPS + +Your site or cloud environment may already have a {ref}`load balancer ` +or proxy server configured and running with a valid, globally trusted TLS +certificate. In this case, you can work with your network administrators to set +up your Trino server behind the load balancer. The load balancer or proxy server +accepts TLS connections and forwards them to the Trino coordinator, which +typically runs with default HTTP configuration on the default port, 8080. + +When a load balancer accepts a TLS encrypted connection, it adds a +[forwarded](https://developer.mozilla.org/docs/Web/HTTP/Proxy_servers_and_tunneling#forwarding_client_information_through_proxies) +HTTP header to the request, such as `X-Forwarded-Proto: https`. + +This tells the Trino coordinator to process the connection as if a TLS +connection has already been successfully negotiated for it. This is why you do +not need to configure `http-server.https.enabled=true` for a coordinator +behind a load balancer. + +However, to enable processing of such forwarded headers, the server's +{ref}`config properties file ` *must* include the following: + +```text +http-server.process-forwarded=true +``` + +This completes any necessary configuration for using HTTPS with a load balancer. +Client tools can access Trino with the URL exposed by the load balancer. + +(https-secure-directly)= + +## Secure Trino directly + +Instead of the preferred mechanism of using an {ref}`external load balancer +`, you can secure the Trino coordinator itself. This +requires you to obtain and install a TLS {ref}`certificate `, and +configure Trino to use it for client connections. + +### Add a TLS certificate + +Obtain a TLS certificate file for use with your Trino server. Consider the +following types of certificates: + +- **Globally trusted certificates** — A certificate that is automatically + trusted by all browsers and clients. This is the easiest type to use because + you do not need to configure clients. Obtain a certificate of this type from: + + - A commercial certificate vendor + - Your cloud infrastructure provider + - A domain name registrar, such as Verisign or GoDaddy + - A free certificate generator, such as + [letsencrypt.org](https://letsencrypt.org/) or + [sslforfree.com](https://www.sslforfree.com/) + +- **Corporate trusted certificates** — A certificate trusted by browsers and + clients in your organization. Typically, a site's IT department runs a local + {ref}`certificate authority ` and preconfigures clients and servers + to trust this CA. + +- **Generated self-signed certificates** — A certificate generated just for + Trino that is not automatically trusted by any client. Before using, make sure + you understand the {ref}`limitations of self-signed certificates + `. + +The most convenient option and strongly recommended option is a globally trusted +certificate. It may require a little more work up front, but it is worth it to +not have to configure every single client. + +### Keys and certificates + +Trino can read certificates and private keys encoded in PEM encoded PKCS #1, PEM +encoded PKCS #8, PKCS #12, and the legacy Java KeyStore (JKS) format. +Certificates and private keys encoded in a binary format such as DER must be +converted. + +Make sure you obtain a certificate that is validated by a recognized +{ref}`certificate authority `. + +### Inspect received certificates + +Before installing your certificate, inspect and validate the received key and +certificate files to make sure they reference the correct information to access +your Trino server. Much unnecessary debugging time is saved by taking the time +to validate your certificates before proceeding to configure the server. + +Inspect PEM-encoded files as described in {doc}`Inspect PEM files +`. + +Inspect PKCS # 12 and JKS keystores as described in {doc}`Inspect JKS files +`. + +### Invalid certificates + +If your certificate does not pass validation, or does not show the expected +information on inspection, contact the group or vendor who provided it for a +replacement. + +(cert-placement)= + +### Place the certificate file + +There are no location requirements for a certificate file as long as: + +- The file can be read by the Trino coordinator server process. +- The location is secure from copying or tampering by malicious actors. + +You can place your file in the Trino coordinator's `etc` directory, which +allows you to use a relative path reference in configuration files. However, +this location can require you to keep track of the certificate file, and move it +to a new `etc` directory when you upgrade your Trino version. + +(configure-https)= + +### Configure the coordinator + +On the coordinator, add the following lines to the {ref}`config properties file +` to enable TLS/HTTPS support for the server. + +:::{note} +Legacy `keystore` and `truststore` wording is used in property names, even +when directly using PEM-encoded certificates. +::: + +```text +http-server.https.enabled=true +http-server.https.port=8443 +http-server.https.keystore.path=etc/clustercoord.pem +``` + +Possible alternatives for the third line include: + +```text +http-server.https.keystore.path=etc/clustercoord.jks +http-server.https.keystore.path=/usr/local/certs/clustercoord.p12 +``` + +Relative paths are relative to the Trino server's root directory. In a +`tar.gz` installation, the root directory is one level above `etc`. + +JKS keystores always require a password, while PEM files with passwords are not +supported by Trino. For JKS, add the following line to the configuration: + +```text +http-server.https.keystore.key= +``` + +It is possible for a key inside a keystore to have its own password, +independent of the keystore's password. In this case, specify the key's password +with the following property: + +```text +http-server.https.keymanager.password= +``` + +When your Trino coordinator has an authenticator enabled along with HTTPS +enabled, HTTP access is automatically disabled for all clients, including the +{doc}`Web UI `. Although not recommended, you can +re-enable it by setting: + +```text +http-server.authentication.allow-insecure-over-http=true +``` + +(verify-tls)= + +### Verify configuration + +To verify TLS/HTTPS configuration, log in to the {doc}`Web UI +`, and send a query with the Trino {doc}`CLI +`. + +- Connect to the Web UI from your browser using a URL that uses HTTPS, such as + `https://trino.example.com:8443`. Enter any username into the `Username` + text box, and log in to the UI. The `Password` box is disabled while + {doc}`authentication ` is not configured. +- Connect with the Trino CLI using a URL that uses HTTPS, such as + `https://trino.example.com:8443`: + +```text +./trino --server https://trino.example.com:8443 +``` + +Send a query to test the connection: + +```text +trino> SELECT 'rocks' AS trino; + +trino +------- +rocks +(1 row) + +Query 20220919_113804_00017_54qfi, FINISHED, 1 node +Splits: 1 total, 1 done (100.00%) +0.12 [0 rows, 0B] [0 rows/s, 0B/s] +``` + +(self-signed-limits)= + +## Limitations of self-signed certificates + +It is possible to generate a self-signed certificate with the `openssl`, +`keytool`, or on Linux, `certtool` commands. Self-signed certificates can be +useful during development of a cluster for internal use only. We recommend never +using a self-signed certificate for a production Trino server. + +Self-signed certificates are not trusted by anyone. They are typically created +by an administrator for expediency, because they do not require getting trust +signoff from anyone. + +To use a self-signed certificate while developing your cluster requires: + +- distributing to every client a local truststore that validates the certificate +- configuring every client to use this certificate + +However, even with this client configuration, modern browsers reject these +certificates, which makes self-signed servers difficult to work with. + +There is a difference between self-signed and unsigned certificates. Both types +are created with the same tools, but unsigned certificates are meant to be +forwarded to a CA with a Certificate Signing Request (CSR). The CA returns the +certificate signed by the CA and now globally trusted. diff --git a/430/_sources/security/user-mapping.md.txt b/430/_sources/security/user-mapping.md.txt new file mode 100644 index 000000000..c297891f6 --- /dev/null +++ b/430/_sources/security/user-mapping.md.txt @@ -0,0 +1,122 @@ +# User mapping + +User mapping defines rules for mapping from users in the authentication method to Trino users. This +mapping is particularly important for {doc}`Kerberos ` or +certificate authentication where the user names +are complex, such as `alice@example` or `CN=Alice Smith,OU=Finance,O=Acme,C=US`. + +There are two ways to map the username format of a given authentication +provider into the simple username format of Trino users: + +- With a single regular expression (regex) {ref}`pattern mapping rule ` +- With a {ref}`file of regex mapping rules ` in JSON format + +(pattern-rule)= + +## Pattern mapping rule + +If you can map all of your authentication method’s usernames with a single +reqular expression, consider using a **Pattern mapping rule**. + +For example, your authentication method uses all usernames in the form +`alice@example.com`, with no exceptions. In this case, choose a regex that +breaks incoming usernames into at least two regex capture groups, such that the +first capture group includes only the name before the `@` sign. You can use +the simple regex `(.*)(@.*)` for this case. + +Trino automatically uses the first capture group – the \$1 group – as the +username to emit after the regex substitution. If the regular expression does +not match the incoming username, authentication is denied. + +Specify your regex pattern in the appropriate property in your coordinator’s +`config.properties` file, using one of the `*user-mapping.pattern` +properties from the table below that matches the authentication type of your +configured authentication provider. For example, for an {doc}`LDAP +` authentication provider: + +```text +http-server.authentication.password.user-mapping.pattern=(.*)(@.*) +``` + +Remember that an {doc}`authentication type ` +represents a category, such as `PASSWORD`, `OAUTH2`, `KERBEROS`. More than +one authentication method can have the same authentication type. For example, +the Password file, LDAP, and Salesforce authentication methods all share the +`PASSWORD` authentication type. + +You can specify different user mapping patterns for different authentication +types when multiple authentication methods are enabled: + +| Authentication type | Property | +| --------------------------------- | ------------------------------------------------------------- | +| Password (file, LDAP, Salesforce) | `http-server.authentication.password.user-mapping.pattern` | +| OAuth2 | `http-server.authentication.oauth2.user-mapping.pattern` | +| Certificate | `http-server.authentication.certificate.user-mapping.pattern` | +| Header | `http-server.authentication.header.user-mapping.pattern` | +| JSON Web Token | `http-server.authentication.jwt.user-mapping.pattern` | +| Kerberos | `http-server.authentication.krb5.user-mapping.pattern` | +| Insecure | `http-server.authentication.insecure.user-mapping.pattern` | + +(pattern-file)= + +## File mapping rules + +Use the **File mapping rules** method if your authentication provider expresses +usernames in a way that cannot be reduced to a single rule, or if you want to +exclude a set of users from accessing the cluster. + +The rules are loaded from a JSON file identified in a configuration property. +The mapping is based on the first matching rule, processed from top to bottom. +If no rules match, authentication is denied. Each rule is composed of the +following fields: + +- `pattern` (required): regex to match against the authentication method's + username. + +- `user` (optional): replacement string to substitute against *pattern*. + The default value is `$1`. + +- `allow` (optional): boolean indicating whether authentication is to be + allowed for the current match. + +- `case` (optional): one of: + + - `keep` - keep the matched username as is (default behavior) + - `lower` - lowercase the matched username; thus both `Admin` and `ADMIN` become `admin` + - `upper` - uppercase the matched username; thus both `admin` and `Admin` become `ADMIN` + +The following example maps all usernames in the form `alice@example.com` to +just `alice`, except for the `test` user, which is denied authentication. It +also maps users in the form `bob@uk.example.com` to `bob_uk`: + +```{literalinclude} user-mapping.json +:language: json +``` + +Set up the preceding example to use the {doc}`LDAP ` +authentication method with the {doc}`PASSWORD ` +authentication type by adding the following line to your coordinator's +`config.properties` file: + +```text +http-server.authentication.password.user-mapping.file=etc/user-mapping.json +``` + +You can place your user mapping JSON file in any local file system location on +the coordinator, but placement in the `etc` directory is typical. There is no +naming standard for the file or its extension, although using `.json` as the +extension is traditional. Specify an absolute path or a path relative to the +Trino installation root. + +You can specify different user mapping files for different authentication +types when multiple authentication methods are enabled: + +| Authentication type | Property | +| --------------------------------- | ---------------------------------------------------------- | +| Password (file, LDAP, Salesforce) | `http-server.authentication.password.user-mapping.file` | +| OAuth2 | `http-server.authentication.oauth2.user-mapping.file` | +| Certificate | `http-server.authentication.certificate.user-mapping.file` | +| Header | `http-server.authentication.header.user-mapping.pattern` | +| JSON Web Token | `http-server.authentication.jwt.user-mapping.file` | +| Kerberos | `http-server.authentication.krb5.user-mapping.file` | +| Insecure | `http-server.authentication.insecure.user-mapping.file` | diff --git a/430/_sources/sql.md.txt b/430/_sources/sql.md.txt new file mode 100644 index 000000000..59a65fa9d --- /dev/null +++ b/430/_sources/sql.md.txt @@ -0,0 +1,86 @@ +# SQL statement syntax + +This section describes the syntax for SQL statements that can be executed in +Trino. + +Refer to the following sections for further details: + +* [SQL data types and other general aspects](/language) +* [SQL functions and operators](/functions) + +```{toctree} +:maxdepth: 1 + +sql/alter-materialized-view +sql/alter-schema +sql/alter-table +sql/alter-view +sql/analyze +sql/call +sql/comment +sql/commit +sql/create-materialized-view +sql/create-role +sql/create-schema +sql/create-table +sql/create-table-as +sql/create-view +sql/deallocate-prepare +sql/delete +sql/deny +sql/describe +sql/describe-input +sql/describe-output +sql/drop-materialized-view +sql/drop-role +sql/drop-schema +sql/drop-table +sql/drop-view +sql/execute +sql/execute-immediate +sql/explain +sql/explain-analyze +sql/grant +sql/grant-roles +sql/insert +sql/match-recognize +sql/merge +sql/prepare +sql/refresh-materialized-view +sql/reset-session +sql/reset-session-authorization +sql/revoke +sql/revoke-roles +sql/rollback +sql/select +sql/set-path +sql/set-role +sql/set-session +sql/set-session-authorization +sql/set-time-zone +sql/show-catalogs +sql/show-columns +sql/show-create-materialized-view +sql/show-create-schema +sql/show-create-table +sql/show-create-view +sql/show-functions +sql/show-grants +sql/show-role-grants +sql/show-roles +sql/show-schemas +sql/show-session +sql/show-stats +sql/show-tables +sql/start-transaction +sql/truncate +sql/update +sql/use +sql/values +``` + +```{toctree} +:hidden: + +sql/pattern-recognition-in-window +``` diff --git a/430/_sources/sql/alter-materialized-view.md.txt b/430/_sources/sql/alter-materialized-view.md.txt new file mode 100644 index 000000000..e6c7854b5 --- /dev/null +++ b/430/_sources/sql/alter-materialized-view.md.txt @@ -0,0 +1,72 @@ +# ALTER MATERIALIZED VIEW + +## Synopsis + +```text +ALTER MATERIALIZED VIEW [ IF EXISTS ] name RENAME TO new_name +ALTER MATERIALIZED VIEW name SET PROPERTIES property_name = expression [, ...] +``` + +## Description + +Change the name of an existing materialized view. + +The optional `IF EXISTS` clause causes the error to be suppressed if the +materialized view does not exist. The error is not suppressed if the +materialized view does not exist, but a table or view with the given name +exists. + +(alter-materialized-view-set-properties)= + +### SET PROPERTIES + +The `ALTER MATERIALIZED VIEW SET PROPERTIES` statement followed by some number +of `property_name` and `expression` pairs applies the specified properties +and values to a materialized view. Ommitting an already-set property from this +statement leaves that property unchanged in the materialized view. + +A property in a `SET PROPERTIES` statement can be set to `DEFAULT`, which +reverts its value back to the default in that materialized view. + +Support for `ALTER MATERIALIZED VIEW SET PROPERTIES` varies between +connectors. Refer to the connector documentation for more details. + +## Examples + +Rename materialized view `people` to `users` in the current schema: + +``` +ALTER MATERIALIZED VIEW people RENAME TO users; +``` + +Rename materialized view `people` to `users`, if materialized view +`people` exists in the current catalog and schema: + +``` +ALTER MATERIALIZED VIEW IF EXISTS people RENAME TO users; +``` + +Set view properties (`x = y`) in materialized view `people`: + +``` +ALTER MATERIALIZED VIEW people SET PROPERTIES x = 'y'; +``` + +Set multiple view properties (`foo = 123` and `foo bar = 456`) in +materialized view `people`: + +``` +ALTER MATERIALIZED VIEW people SET PROPERTIES foo = 123, "foo bar" = 456; +``` + +Set view property `x` to its default value in materialized view `people`: + +``` +ALTER MATERIALIZED VIEW people SET PROPERTIES x = DEFAULT; +``` + +## See also + +- {doc}`create-materialized-view` +- {doc}`refresh-materialized-view` +- {doc}`drop-materialized-view` diff --git a/430/_sources/sql/alter-schema.md.txt b/430/_sources/sql/alter-schema.md.txt new file mode 100644 index 000000000..ead700bcf --- /dev/null +++ b/430/_sources/sql/alter-schema.md.txt @@ -0,0 +1,36 @@ +# ALTER SCHEMA + +## Synopsis + +```text +ALTER SCHEMA name RENAME TO new_name +ALTER SCHEMA name SET AUTHORIZATION ( user | USER user | ROLE role ) +``` + +## Description + +Change the definition of an existing schema. + +## Examples + +Rename schema `web` to `traffic`: + +``` +ALTER SCHEMA web RENAME TO traffic +``` + +Change owner of schema `web` to user `alice`: + +``` +ALTER SCHEMA web SET AUTHORIZATION alice +``` + +Allow everyone to drop schema and create tables in schema `web`: + +``` +ALTER SCHEMA web SET AUTHORIZATION ROLE PUBLIC +``` + +## See Also + +{doc}`create-schema` diff --git a/430/_sources/sql/alter-table.md.txt b/430/_sources/sql/alter-table.md.txt new file mode 100644 index 000000000..08e2a43fb --- /dev/null +++ b/430/_sources/sql/alter-table.md.txt @@ -0,0 +1,156 @@ +# ALTER TABLE + +## Synopsis + +```text +ALTER TABLE [ IF EXISTS ] name RENAME TO new_name +ALTER TABLE [ IF EXISTS ] name ADD COLUMN [ IF NOT EXISTS ] column_name data_type + [ NOT NULL ] [ COMMENT comment ] + [ WITH ( property_name = expression [, ...] ) ] +ALTER TABLE [ IF EXISTS ] name DROP COLUMN [ IF EXISTS ] column_name +ALTER TABLE [ IF EXISTS ] name RENAME COLUMN [ IF EXISTS ] old_name TO new_name +ALTER TABLE [ IF EXISTS ] name ALTER COLUMN column_name SET DATA TYPE new_type +ALTER TABLE name SET AUTHORIZATION ( user | USER user | ROLE role ) +ALTER TABLE name SET PROPERTIES property_name = expression [, ...] +ALTER TABLE name EXECUTE command [ ( parameter => expression [, ... ] ) ] + [ WHERE expression ] +``` + +## Description + +Change the definition of an existing table. + +The optional `IF EXISTS` (when used before the table name) clause causes the error to be suppressed if the table does not exists. + +The optional `IF EXISTS` (when used before the column name) clause causes the error to be suppressed if the column does not exists. + +The optional `IF NOT EXISTS` clause causes the error to be suppressed if the column already exists. + +(alter-table-set-properties)= + +### SET PROPERTIES + +The `ALTER TABLE SET PROPERTIES` statement followed by some number +of `property_name` and `expression` pairs applies the specified properties +and values to a table. Ommitting an already-set property from this +statement leaves that property unchanged in the table. + +A property in a `SET PROPERTIES` statement can be set to `DEFAULT`, which +reverts its value back to the default in that table. + +Support for `ALTER TABLE SET PROPERTIES` varies between +connectors, as not all connectors support modifying table properties. + +(alter-table-execute)= + +### EXECUTE + +The `ALTER TABLE EXECUTE` statement followed by a `command` and +`parameters` modifies the table according to the specified command and +parameters. `ALTER TABLE EXECUTE` supports different commands on a +per-connector basis. + +You can use the `=>` operator for passing named parameter values. +The left side is the name of the parameter, the right side is the value being passed: + +``` +ALTER TABLE hive.schema.test_table EXECUTE optimize(file_size_threshold => '10MB') +``` + +## Examples + +Rename table `users` to `people`: + +``` +ALTER TABLE users RENAME TO people; +``` + +Rename table `users` to `people` if table `users` exists: + +``` +ALTER TABLE IF EXISTS users RENAME TO people; +``` + +Add column `zip` to the `users` table: + +``` +ALTER TABLE users ADD COLUMN zip varchar; +``` + +Add column `zip` to the `users` table if table `users` exists and column `zip` not already exists: + +``` +ALTER TABLE IF EXISTS users ADD COLUMN IF NOT EXISTS zip varchar; +``` + +Drop column `zip` from the `users` table: + +``` +ALTER TABLE users DROP COLUMN zip; +``` + +Drop column `zip` from the `users` table if table `users` and column `zip` exists: + +``` +ALTER TABLE IF EXISTS users DROP COLUMN IF EXISTS zip; +``` + +Rename column `id` to `user_id` in the `users` table: + +``` +ALTER TABLE users RENAME COLUMN id TO user_id; +``` + +Rename column `id` to `user_id` in the `users` table if table `users` and column `id` exists: + +``` +ALTER TABLE IF EXISTS users RENAME column IF EXISTS id to user_id; +``` + +Change type of column `id` to `bigint` in the `users` table: + +``` +ALTER TABLE users ALTER COLUMN id SET DATA TYPE bigint; +``` + +Change owner of table `people` to user `alice`: + +``` +ALTER TABLE people SET AUTHORIZATION alice +``` + +Allow everyone with role public to drop and alter table `people`: + +``` +ALTER TABLE people SET AUTHORIZATION ROLE PUBLIC +``` + +Set table properties (`x = y`) in table `people`: + +``` +ALTER TABLE people SET PROPERTIES x = 'y'; +``` + +Set multiple table properties (`foo = 123` and `foo bar = 456`) in +table `people`: + +``` +ALTER TABLE people SET PROPERTIES foo = 123, "foo bar" = 456; +``` + +Set table property `x` to its default value in table\`\`people\`\`: + +``` +ALTER TABLE people SET PROPERTIES x = DEFAULT; +``` + +Collapse files in a table that are over 10 megabytes in size, as supported by +the Hive connector: + +``` +ALTER TABLE hive.schema.test_table EXECUTE optimize(file_size_threshold => '10MB') +``` + +## See also + +{doc}`create-table` diff --git a/430/_sources/sql/alter-view.md.txt b/430/_sources/sql/alter-view.md.txt new file mode 100644 index 000000000..9c9d91aa3 --- /dev/null +++ b/430/_sources/sql/alter-view.md.txt @@ -0,0 +1,30 @@ +# ALTER VIEW + +## Synopsis + +```text +ALTER VIEW name RENAME TO new_name +ALTER VIEW name SET AUTHORIZATION ( user | USER user | ROLE role ) +``` + +## Description + +Change the definition of an existing view. + +## Examples + +Rename view `people` to `users`: + +``` +ALTER VIEW people RENAME TO users +``` + +Change owner of VIEW `people` to user `alice`: + +``` +ALTER VIEW people SET AUTHORIZATION alice +``` + +## See also + +{doc}`create-view` diff --git a/430/_sources/sql/analyze.md.txt b/430/_sources/sql/analyze.md.txt new file mode 100644 index 000000000..f3e03af5e --- /dev/null +++ b/430/_sources/sql/analyze.md.txt @@ -0,0 +1,53 @@ +# ANALYZE + +## Synopsis + +```text +ANALYZE table_name [ WITH ( property_name = expression [, ...] ) ] +``` + +## Description + +Collects table and column statistics for a given table. + +The optional `WITH` clause can be used to provide connector-specific properties. +To list all available properties, run the following query: + +``` +SELECT * FROM system.metadata.analyze_properties +``` + +## Examples + +Analyze table `web` to collect table and column statistics: + +``` +ANALYZE web; +``` + +Analyze table `stores` in catalog `hive` and schema `default`: + +``` +ANALYZE hive.default.stores; +``` + +Analyze partitions `'1992-01-01', '1992-01-02'` from a Hive partitioned table `sales`: + +``` +ANALYZE hive.default.sales WITH (partitions = ARRAY[ARRAY['1992-01-01'], ARRAY['1992-01-02']]); +``` + +Analyze partitions with complex partition key (`state` and `city` columns) from a Hive partitioned table `customers`: + +``` +ANALYZE hive.default.customers WITH (partitions = ARRAY[ARRAY['CA', 'San Francisco'], ARRAY['NY', 'NY']]); +``` + +Analyze only columns `department` and `product_id` for partitions `'1992-01-01', '1992-01-02'` from a Hive partitioned +table `sales`: + +``` +ANALYZE hive.default.sales WITH ( + partitions = ARRAY[ARRAY['1992-01-01'], ARRAY['1992-01-02']], + columns = ARRAY['department', 'product_id']); +``` diff --git a/430/_sources/sql/call.md.txt b/430/_sources/sql/call.md.txt new file mode 100644 index 000000000..c03dacda5 --- /dev/null +++ b/430/_sources/sql/call.md.txt @@ -0,0 +1,42 @@ +# CALL + +## Synopsis + +```text +CALL procedure_name ( [ name => ] expression [, ...] ) +``` + +## Description + +Call a procedure. + +Procedures can be provided by connectors to perform data manipulation or +administrative tasks. For example, the {doc}`/connector/system` defines a +procedure for killing a running query. + +Some connectors, such as the {doc}`/connector/postgresql`, are for systems +that have their own stored procedures. These stored procedures are separate +from the connector-defined procedures discussed here and thus are not +directly callable via `CALL`. + +See connector documentation for details on available procedures. + +## Examples + +Call a procedure using positional arguments: + +``` +CALL test(123, 'apple'); +``` + +Call a procedure using named arguments: + +``` +CALL test(name => 'apple', id => 123); +``` + +Call a procedure using a fully qualified name: + +``` +CALL catalog.schema.test(); +``` diff --git a/430/_sources/sql/comment.md.txt b/430/_sources/sql/comment.md.txt new file mode 100644 index 000000000..8ed32a746 --- /dev/null +++ b/430/_sources/sql/comment.md.txt @@ -0,0 +1,35 @@ +# COMMENT + +## Synopsis + +```text +COMMENT ON ( TABLE | VIEW | COLUMN ) name IS 'comments' +``` + +## Description + +Set the comment for a object. The comment can be removed by setting the comment to `NULL`. + +## Examples + +Change the comment for the `users` table to be `master table`: + +``` +COMMENT ON TABLE users IS 'master table'; +``` + +Change the comment for the `users` view to be `master view`: + +``` +COMMENT ON VIEW users IS 'master view'; +``` + +Change the comment for the `users.name` column to be `full name`: + +``` +COMMENT ON COLUMN users.name IS 'full name'; +``` + +## See also + +[](/language/comments) diff --git a/430/_sources/sql/commit.md.txt b/430/_sources/sql/commit.md.txt new file mode 100644 index 000000000..3cd262378 --- /dev/null +++ b/430/_sources/sql/commit.md.txt @@ -0,0 +1,22 @@ +# COMMIT + +## Synopsis + +```text +COMMIT [ WORK ] +``` + +## Description + +Commit the current transaction. + +## Examples + +```sql +COMMIT; +COMMIT WORK; +``` + +## See also + +{doc}`rollback`, {doc}`start-transaction` diff --git a/430/_sources/sql/create-materialized-view.md.txt b/430/_sources/sql/create-materialized-view.md.txt new file mode 100644 index 000000000..3c2f1bc06 --- /dev/null +++ b/430/_sources/sql/create-materialized-view.md.txt @@ -0,0 +1,120 @@ +# CREATE MATERIALIZED VIEW + +## Synopsis + +```text +CREATE [ OR REPLACE ] MATERIALIZED VIEW +[ IF NOT EXISTS ] view_name +[ GRACE PERIOD interval ] +[ COMMENT string ] +[ WITH properties ] +AS query +``` + +## Description + +Create and validate the definition of a new materialized view `view_name` of a +{doc}`select` `query`. You need to run the {doc}`refresh-materialized-view` +statement after the creation to populate the materialized view with data. This +materialized view is a physical manifestation of the query results at time of +refresh. The data is stored, and can be referenced by future queries. + +Queries accessing materialized views are typically faster than retrieving data +from a view created with the same query. Any computation, aggregation, and other +operation to create the data is performed once during refresh of the +materialized views, as compared to each time of accessing the view. Multiple +reads of view data over time, or by multiple users, all trigger repeated +processing. This is avoided for materialized views. + +The optional `OR REPLACE` clause causes the materialized view to be replaced +if it already exists rather than raising an error. + +The optional `IF NOT EXISTS` clause causes the materialized view only to be +created if it does not exist yet. + +Note that `OR REPLACE` and `IF NOT EXISTS` are mutually exclusive clauses. + +The optional `GRACE PERIOD` clause specifies how long the query materialization +is used for querying. If the time elapsed since last materialized view refresh +is greater than the grace period, the materialized view acts as a normal view and +the materialized data is not used. If not specified, the grace period defaults to +infinity. See {doc}`refresh-materialized-view` for more about refreshing +materialized views. + +The optional `COMMENT` clause causes a `string` comment to be stored with +the metadata about the materialized view. The comment is displayed with the +{doc}`show-create-materialized-view` statement and is available in the table +`system.metadata.materialized_view_properties`. + +The optional `WITH` clause is used to define properties for the materialized +view creation. Separate multiple property/value pairs by commas. The connector +uses the properties as input parameters for the materialized view refresh +operation. The supported properties are different for each connector and +detailed in the SQL support section of the specific connector's documentation. + +After successful creation, all metadata about the materialized view is available +in a {ref}`system table `. + +## Examples + +Create a simple materialized view `cancelled_orders` over the `orders` table +that only includes cancelled orders. Note that `orderstatus` is a numeric +value that is potentially meaningless to a consumer, yet the name of the view +clarifies the content: + +``` +CREATE MATERIALIZED VIEW cancelled_orders +AS + SELECT orderkey, totalprice + FROM orders + WHERE orderstatus = 3; +``` + +Create or replace a materialized view `order_totals_by_date` that summarizes +`orders` across all orders from all customers: + +``` +CREATE OR REPLACE MATERIALIZED VIEW order_totals_by_date +AS + SELECT orderdate, sum(totalprice) AS price + FROM orders + GROUP BY orderdate; +``` + +Create a materialized view for a catalog using the Iceberg connector, with a +comment and partitioning on two fields in the storage: + +``` +CREATE MATERIALIZED VIEW orders_nation_mkgsegment +COMMENT 'Orders with nation and market segment data' +WITH ( partitioning = ARRAY['mktsegment', 'nationkey'] ) +AS + SELECT o.*, c.nationkey, c.mktsegment + FROM orders AS o + JOIN customer AS c + ON o.custkey = c.custkey; +``` + +Set multiple properties: + +``` +WITH ( format = 'ORC', partitioning = ARRAY['_date'] ) +``` + +Show defined materialized view properties for all catalogs: + +``` +SELECT * FROM system.metadata.materialized_view_properties; +``` + +Show metadata about the materialized views in all catalogs: + +``` +SELECT * FROM system.metadata.materialized_views; +``` + +## See also + +- {doc}`drop-materialized-view` +- {doc}`show-create-materialized-view` +- {doc}`refresh-materialized-view` diff --git a/430/_sources/sql/create-role.md.txt b/430/_sources/sql/create-role.md.txt new file mode 100644 index 000000000..524894328 --- /dev/null +++ b/430/_sources/sql/create-role.md.txt @@ -0,0 +1,44 @@ +# CREATE ROLE + +## Synopsis + +```text +CREATE ROLE role_name +[ WITH ADMIN ( user | USER user | ROLE role | CURRENT_USER | CURRENT_ROLE ) ] +[ IN catalog ] +``` + +## Description + +`CREATE ROLE` creates the specified role. + +The optional `WITH ADMIN` clause causes the role to be created with +the specified user as a role admin. A role admin has permission to drop +or grant a role. If the optional `WITH ADMIN` clause is not +specified, the role is created with current user as admin. + +The optional `IN catalog` clause creates the role in a catalog as opposed +to a system role. + +## Examples + +Create role `admin` + +``` +CREATE ROLE admin; +``` + +Create role `moderator` with admin `bob`: + +``` +CREATE ROLE moderator WITH ADMIN USER bob; +``` + +## Limitations + +Some connectors do not support role management. +See connector documentation for more details. + +## See also + +{doc}`drop-role`, {doc}`set-role`, {doc}`grant-roles`, {doc}`revoke-roles` diff --git a/430/_sources/sql/create-schema.md.txt b/430/_sources/sql/create-schema.md.txt new file mode 100644 index 000000000..4cf2d22a6 --- /dev/null +++ b/430/_sources/sql/create-schema.md.txt @@ -0,0 +1,79 @@ +# CREATE SCHEMA + +## Synopsis + +```text +CREATE SCHEMA [ IF NOT EXISTS ] schema_name +[ AUTHORIZATION ( user | USER user | ROLE role ) ] +[ WITH ( property_name = expression [, ...] ) ] +``` + +## Description + +Create a new, empty schema. A schema is a container that +holds tables, views and other database objects. + +The optional `IF NOT EXISTS` clause causes the error to be +suppressed if the schema already exists. + +The optional `AUTHORIZATION` clause can be used to set the +owner of the newly created schema to a user or role. + +The optional `WITH` clause can be used to set properties +on the newly created schema. To list all available schema +properties, run the following query: + +``` +SELECT * FROM system.metadata.schema_properties +``` + +## Examples + +Create a new schema `web` in the current catalog: + +``` +CREATE SCHEMA web +``` + +Create a new schema `sales` in the `hive` catalog: + +``` +CREATE SCHEMA hive.sales +``` + +Create the schema `traffic` if it does not already exist: + +``` +CREATE SCHEMA IF NOT EXISTS traffic +``` + +Create a new schema `web` and set the owner to user `alice`: + +``` +CREATE SCHEMA web AUTHORIZATION alice +``` + +Create a new schema `web`, set the `LOCATION` property to `/hive/data/web` +and set the owner to user `alice`: + +``` +CREATE SCHEMA web AUTHORIZATION alice WITH ( LOCATION = '/hive/data/web' ) +``` + +Create a new schema `web` and allow everyone to drop schema and create tables +in schema `web`: + +``` +CREATE SCHEMA web AUTHORIZATION ROLE PUBLIC +``` + +Create a new schema `web`, set the `LOCATION` property to `/hive/data/web` +and allow everyone to drop schema and create tables in schema `web`: + +``` +CREATE SCHEMA web AUTHORIZATION ROLE PUBLIC WITH ( LOCATION = '/hive/data/web' ) +``` + +## See also + +{doc}`alter-schema`, {doc}`drop-schema` diff --git a/430/_sources/sql/create-table-as.md.txt b/430/_sources/sql/create-table-as.md.txt new file mode 100644 index 000000000..bc2cb58f0 --- /dev/null +++ b/430/_sources/sql/create-table-as.md.txt @@ -0,0 +1,72 @@ +# CREATE TABLE AS + +## Synopsis + +```text +CREATE TABLE [ IF NOT EXISTS ] table_name [ ( column_alias, ... ) ] +[ COMMENT table_comment ] +[ WITH ( property_name = expression [, ...] ) ] +AS query +[ WITH [ NO ] DATA ] +``` + +## Description + +Create a new table containing the result of a {doc}`select` query. +Use {doc}`create-table` to create an empty table. + +The optional `IF NOT EXISTS` clause causes the error to be +suppressed if the table already exists. + +The optional `WITH` clause can be used to set properties +on the newly created table. To list all available table +properties, run the following query: + +``` +SELECT * FROM system.metadata.table_properties +``` + +## Examples + +Create a new table `orders_column_aliased` with the results of a query and the given column names: + +``` +CREATE TABLE orders_column_aliased (order_date, total_price) +AS +SELECT orderdate, totalprice +FROM orders +``` + +Create a new table `orders_by_date` that summarizes `orders`: + +``` +CREATE TABLE orders_by_date +COMMENT 'Summary of orders by date' +WITH (format = 'ORC') +AS +SELECT orderdate, sum(totalprice) AS price +FROM orders +GROUP BY orderdate +``` + +Create the table `orders_by_date` if it does not already exist: + +``` +CREATE TABLE IF NOT EXISTS orders_by_date AS +SELECT orderdate, sum(totalprice) AS price +FROM orders +GROUP BY orderdate +``` + +Create a new `empty_nation` table with the same schema as `nation` and no data: + +``` +CREATE TABLE empty_nation AS +SELECT * +FROM nation +WITH NO DATA +``` + +## See also + +{doc}`create-table`, {doc}`select` diff --git a/430/_sources/sql/create-table.md.txt b/430/_sources/sql/create-table.md.txt new file mode 100644 index 000000000..56e4ebd77 --- /dev/null +++ b/430/_sources/sql/create-table.md.txt @@ -0,0 +1,92 @@ +# CREATE TABLE + +## Synopsis + +```text +CREATE TABLE [ IF NOT EXISTS ] +table_name ( + { column_name data_type [ NOT NULL ] + [ COMMENT comment ] + [ WITH ( property_name = expression [, ...] ) ] + | LIKE existing_table_name + [ { INCLUDING | EXCLUDING } PROPERTIES ] + } + [, ...] +) +[ COMMENT table_comment ] +[ WITH ( property_name = expression [, ...] ) ] +``` + +## Description + +Create a new, empty table with the specified columns. +Use {doc}`create-table-as` to create a table with data. + +The optional `IF NOT EXISTS` clause causes the error to be +suppressed if the table already exists. + +The optional `WITH` clause can be used to set properties +on the newly created table or on single columns. To list all available table +properties, run the following query: + +``` +SELECT * FROM system.metadata.table_properties +``` + +To list all available column properties, run the following query: + +``` +SELECT * FROM system.metadata.column_properties +``` + +The `LIKE` clause can be used to include all the column definitions from +an existing table in the new table. Multiple `LIKE` clauses may be +specified, which allows copying the columns from multiple tables. + +If `INCLUDING PROPERTIES` is specified, all of the table properties are +copied to the new table. If the `WITH` clause specifies the same property +name as one of the copied properties, the value from the `WITH` clause +will be used. The default behavior is `EXCLUDING PROPERTIES`. The +`INCLUDING PROPERTIES` option maybe specified for at most one table. + +## Examples + +Create a new table `orders`: + +``` +CREATE TABLE orders ( + orderkey bigint, + orderstatus varchar, + totalprice double, + orderdate date +) +WITH (format = 'ORC') +``` + +Create the table `orders` if it does not already exist, adding a table comment +and a column comment: + +``` +CREATE TABLE IF NOT EXISTS orders ( + orderkey bigint, + orderstatus varchar, + totalprice double COMMENT 'Price in cents.', + orderdate date +) +COMMENT 'A table to keep track of orders.' +``` + +Create the table `bigger_orders` using the columns from `orders` +plus additional columns at the start and end: + +``` +CREATE TABLE bigger_orders ( + another_orderkey bigint, + LIKE orders, + another_orderdate date +) +``` + +## See also + +{doc}`alter-table`, {doc}`drop-table`, {doc}`create-table-as`, {doc}`show-create-table` diff --git a/430/_sources/sql/create-view.md.txt b/430/_sources/sql/create-view.md.txt new file mode 100644 index 000000000..ff39a3b9a --- /dev/null +++ b/430/_sources/sql/create-view.md.txt @@ -0,0 +1,77 @@ +# CREATE VIEW + +## Synopsis + +```text +CREATE [ OR REPLACE ] VIEW view_name +[ COMMENT view_comment ] +[ SECURITY { DEFINER | INVOKER } ] +AS query +``` + +## Description + +Create a new view of a {doc}`select` query. The view is a logical table +that can be referenced by future queries. Views do not contain any data. +Instead, the query stored by the view is executed every time the view is +referenced by another query. + +The optional `OR REPLACE` clause causes the view to be replaced if it +already exists rather than raising an error. + +## Security + +In the default `DEFINER` security mode, tables referenced in the view +are accessed using the permissions of the view owner (the *creator* or +*definer* of the view) rather than the user executing the query. This +allows providing restricted access to the underlying tables, for which +the user may not be allowed to access directly. + +In the `INVOKER` security mode, tables referenced in the view are accessed +using the permissions of the user executing the query (the *invoker* of the view). +A view created in this mode is simply a stored query. + +Regardless of the security mode, the `current_user` function will +always return the user executing the query and thus may be used +within views to filter out rows or otherwise restrict access. + +## Examples + +Create a simple view `test` over the `orders` table: + +``` +CREATE VIEW test AS +SELECT orderkey, orderstatus, totalprice / 2 AS half +FROM orders +``` + +Create a view `test_with_comment` with a view comment: + +``` +CREATE VIEW test_with_comment +COMMENT 'A view to keep track of orders.' +AS +SELECT orderkey, orderstatus, totalprice +FROM orders +``` + +Create a view `orders_by_date` that summarizes `orders`: + +``` +CREATE VIEW orders_by_date AS +SELECT orderdate, sum(totalprice) AS price +FROM orders +GROUP BY orderdate +``` + +Create a view that replaces an existing view: + +``` +CREATE OR REPLACE VIEW test AS +SELECT orderkey, orderstatus, totalprice / 4 AS quarter +FROM orders +``` + +## See also + +{doc}`drop-view`, {doc}`show-create-view` diff --git a/430/_sources/sql/deallocate-prepare.md.txt b/430/_sources/sql/deallocate-prepare.md.txt new file mode 100644 index 000000000..e4c1fcb5c --- /dev/null +++ b/430/_sources/sql/deallocate-prepare.md.txt @@ -0,0 +1,24 @@ +# DEALLOCATE PREPARE + +## Synopsis + +```text +DEALLOCATE PREPARE statement_name +``` + +## Description + +Removes a statement with the name `statement_name` from the list of prepared +statements in a session. + +## Examples + +Deallocate a statement with the name `my_query`: + +``` +DEALLOCATE PREPARE my_query; +``` + +## See also + +{doc}`prepare`, {doc}`execute`, {doc}`execute-immediate` diff --git a/430/_sources/sql/delete.md.txt b/430/_sources/sql/delete.md.txt new file mode 100644 index 000000000..622405fab --- /dev/null +++ b/430/_sources/sql/delete.md.txt @@ -0,0 +1,38 @@ +# DELETE + +## Synopsis + +```text +DELETE FROM table_name [ WHERE condition ] +``` + +## Description + +Delete rows from a table. If the `WHERE` clause is specified, only the +matching rows are deleted. Otherwise, all rows from the table are deleted. + +## Examples + +Delete all line items shipped by air: + +``` +DELETE FROM lineitem WHERE shipmode = 'AIR'; +``` + +Delete all line items for low priority orders: + +``` +DELETE FROM lineitem +WHERE orderkey IN (SELECT orderkey FROM orders WHERE priority = 'LOW'); +``` + +Delete all orders: + +``` +DELETE FROM orders; +``` + +## Limitations + +Some connectors have limited or no support for `DELETE`. +See connector documentation for more details. diff --git a/430/_sources/sql/deny.md.txt b/430/_sources/sql/deny.md.txt new file mode 100644 index 000000000..d53338275 --- /dev/null +++ b/430/_sources/sql/deny.md.txt @@ -0,0 +1,49 @@ +# DENY + +## Synopsis + +```text +DENY ( privilege [, ...] | ( ALL PRIVILEGES ) ) +ON ( table_name | TABLE table_name | SCHEMA schema_name) +TO ( user | USER user | ROLE role ) +``` + +## Description + +Denies the specified privileges to the specified grantee. + +Deny on a table rejects the specified privilege on all current and future +columns of the table. + +Deny on a schema rejects the specified privilege on all current and future +columns of all current and future tables of the schema. + +## Examples + +Deny `INSERT` and `SELECT` privileges on the table `orders` +to user `alice`: + +``` +DENY INSERT, SELECT ON orders TO alice; +``` + +Deny `DELETE` privilege on the schema `finance` to user `bob`: + +``` +DENY DELETE ON SCHEMA finance TO bob; +``` + +Deny `SELECT` privilege on the table `orders` to everyone: + +``` +DENY SELECT ON orders TO ROLE PUBLIC; +``` + +## Limitations + +The system access controls as well as the connectors provided by default +in Trino have no support for `DENY`. + +## See also + +{doc}`grant`, {doc}`revoke`, {doc}`show-grants` diff --git a/430/_sources/sql/describe-input.md.txt b/430/_sources/sql/describe-input.md.txt new file mode 100644 index 000000000..e513323b2 --- /dev/null +++ b/430/_sources/sql/describe-input.md.txt @@ -0,0 +1,56 @@ +# DESCRIBE INPUT + +## Synopsis + +```text +DESCRIBE INPUT statement_name +``` + +## Description + +Lists the input parameters of a prepared statement along with the +position and type of each parameter. Parameter types that cannot be +determined will appear as `unknown`. + +## Examples + +Prepare and describe a query with three parameters: + +```sql +PREPARE my_select1 FROM +SELECT ? FROM nation WHERE regionkey = ? AND name < ?; +``` + +```sql +DESCRIBE INPUT my_select1; +``` + +```text + Position | Type +-------------------- + 0 | unknown + 1 | bigint + 2 | varchar +(3 rows) +``` + +Prepare and describe a query with no parameters: + +```sql +PREPARE my_select2 FROM +SELECT * FROM nation; +``` + +```sql +DESCRIBE INPUT my_select2; +``` + +```text + Position | Type +----------------- +(0 rows) +``` + +## See also + +{doc}`prepare` diff --git a/430/_sources/sql/describe-output.md.txt b/430/_sources/sql/describe-output.md.txt new file mode 100644 index 000000000..b2636aeb6 --- /dev/null +++ b/430/_sources/sql/describe-output.md.txt @@ -0,0 +1,77 @@ +# DESCRIBE OUTPUT + +## Synopsis + +```text +DESCRIBE OUTPUT statement_name +``` + +## Description + +List the output columns of a prepared statement, including the +column name (or alias), catalog, schema, table, type, type size in +bytes, and a boolean indicating if the column is aliased. + +## Examples + +Prepare and describe a query with four output columns: + +``` +PREPARE my_select1 FROM +SELECT * FROM nation; +``` + +```sql +DESCRIBE OUTPUT my_select1; +``` + +```text + Column Name | Catalog | Schema | Table | Type | Type Size | Aliased +-------------+---------+--------+--------+---------+-----------+--------- + nationkey | tpch | sf1 | nation | bigint | 8 | false + name | tpch | sf1 | nation | varchar | 0 | false + regionkey | tpch | sf1 | nation | bigint | 8 | false + comment | tpch | sf1 | nation | varchar | 0 | false +(4 rows) +``` + +Prepare and describe a query whose output columns are expressions: + +``` +PREPARE my_select2 FROM +SELECT count(*) as my_count, 1+2 FROM nation; +``` + +```sql +DESCRIBE OUTPUT my_select2; +``` + +```text + Column Name | Catalog | Schema | Table | Type | Type Size | Aliased +-------------+---------+--------+-------+--------+-----------+--------- + my_count | | | | bigint | 8 | true + _col1 | | | | bigint | 8 | false +(2 rows) +``` + +Prepare and describe a row count query: + +``` +PREPARE my_create FROM +CREATE TABLE foo AS SELECT * FROM nation; +``` + +```sql +DESCRIBE OUTPUT my_create; +``` + +```text + Column Name | Catalog | Schema | Table | Type | Type Size | Aliased +-------------+---------+--------+-------+--------+-----------+--------- + rows | | | | bigint | 8 | false +(1 row) +``` + +## See also + +{doc}`prepare` diff --git a/430/_sources/sql/describe.md.txt b/430/_sources/sql/describe.md.txt new file mode 100644 index 000000000..6db8a20d0 --- /dev/null +++ b/430/_sources/sql/describe.md.txt @@ -0,0 +1,11 @@ +# DESCRIBE + +## Synopsis + +```text +DESCRIBE table_name +``` + +## Description + +`DESCRIBE` is an alias for {doc}`show-columns`. diff --git a/430/_sources/sql/drop-materialized-view.md.txt b/430/_sources/sql/drop-materialized-view.md.txt new file mode 100644 index 000000000..50027fe9c --- /dev/null +++ b/430/_sources/sql/drop-materialized-view.md.txt @@ -0,0 +1,34 @@ +# DROP MATERIALIZED VIEW + +## Synopsis + +```text +DROP MATERIALIZED VIEW [ IF EXISTS ] view_name +``` + +## Description + +Drop an existing materialized view `view_name`. + +The optional `IF EXISTS` clause causes the error to be suppressed if +the materialized view does not exist. + +## Examples + +Drop the materialized view `orders_by_date`: + +``` +DROP MATERIALIZED VIEW orders_by_date; +``` + +Drop the materialized view `orders_by_date` if it exists: + +``` +DROP MATERIALIZED VIEW IF EXISTS orders_by_date; +``` + +## See also + +- {doc}`create-materialized-view` +- {doc}`show-create-materialized-view` +- {doc}`refresh-materialized-view` diff --git a/430/_sources/sql/drop-role.md.txt b/430/_sources/sql/drop-role.md.txt new file mode 100644 index 000000000..9e0c84817 --- /dev/null +++ b/430/_sources/sql/drop-role.md.txt @@ -0,0 +1,35 @@ +# DROP ROLE + +## Synopsis + +```text +DROP ROLE role_name +[ IN catalog ] +``` + +## Description + +`DROP ROLE` drops the specified role. + +For `DROP ROLE` statement to succeed, the user executing it should possess +admin privileges for the given role. + +The optional `IN catalog` clause drops the role in a catalog as opposed +to a system role. + +## Examples + +Drop role `admin` + +``` +DROP ROLE admin; +``` + +## Limitations + +Some connectors do not support role management. +See connector documentation for more details. + +## See also + +{doc}`create-role`, {doc}`set-role`, {doc}`grant-roles`, {doc}`revoke-roles` diff --git a/430/_sources/sql/drop-schema.md.txt b/430/_sources/sql/drop-schema.md.txt new file mode 100644 index 000000000..b1345d736 --- /dev/null +++ b/430/_sources/sql/drop-schema.md.txt @@ -0,0 +1,44 @@ +# DROP SCHEMA + +## Synopsis + +```text +DROP SCHEMA [ IF EXISTS ] schema_name [ CASCADE | RESTRICT ] +``` + +## Description + +Drop an existing schema. The schema must be empty. + +The optional `IF EXISTS` clause causes the error to be suppressed if +the schema does not exist. + +## Examples + +Drop the schema `web`: + +``` +DROP SCHEMA web +``` + +Drop the schema `sales` if it exists: + +``` +DROP SCHEMA IF EXISTS sales +``` + +Drop the schema `archive`, along with everything it contains: + +``` +DROP SCHEMA archive CASCADE +``` + +Drop the schema `archive`, only if there are no objects contained in the schema: + +``` +DROP SCHEMA archive RESTRICT +``` + +## See also + +{doc}`alter-schema`, {doc}`create-schema` diff --git a/430/_sources/sql/drop-table.md.txt b/430/_sources/sql/drop-table.md.txt new file mode 100644 index 000000000..7c8f70c6c --- /dev/null +++ b/430/_sources/sql/drop-table.md.txt @@ -0,0 +1,32 @@ +# DROP TABLE + +## Synopsis + +```text +DROP TABLE [ IF EXISTS ] table_name +``` + +## Description + +Drops an existing table. + +The optional `IF EXISTS` clause causes the error to be suppressed if +the table does not exist. + +## Examples + +Drop the table `orders_by_date`: + +``` +DROP TABLE orders_by_date +``` + +Drop the table `orders_by_date` if it exists: + +``` +DROP TABLE IF EXISTS orders_by_date +``` + +## See also + +{doc}`alter-table`, {doc}`create-table` diff --git a/430/_sources/sql/drop-view.md.txt b/430/_sources/sql/drop-view.md.txt new file mode 100644 index 000000000..62b0fd9be --- /dev/null +++ b/430/_sources/sql/drop-view.md.txt @@ -0,0 +1,32 @@ +# DROP VIEW + +## Synopsis + +```text +DROP VIEW [ IF EXISTS ] view_name +``` + +## Description + +Drop an existing view. + +The optional `IF EXISTS` clause causes the error to be suppressed if +the view does not exist. + +## Examples + +Drop the view `orders_by_date`: + +``` +DROP VIEW orders_by_date +``` + +Drop the view `orders_by_date` if it exists: + +``` +DROP VIEW IF EXISTS orders_by_date +``` + +## See also + +{doc}`create-view` diff --git a/430/_sources/sql/execute-immediate.md.txt b/430/_sources/sql/execute-immediate.md.txt new file mode 100644 index 000000000..0641d614e --- /dev/null +++ b/430/_sources/sql/execute-immediate.md.txt @@ -0,0 +1,41 @@ +# EXECUTE IMMEDIATE + +## Synopsis + +```text +EXECUTE IMMEDIATE `statement` [ USING parameter1 [ , parameter2, ... ] ] +``` + +## Description + +Executes a statement without the need to prepare or deallocate the statement. +Parameter values are defined in the `USING` clause. + +## Examples + +Execute a query with no parameters: + +``` +EXECUTE IMMEDIATE +'SELECT name FROM nation'; +``` + +Execute a query with two parameters: + +``` +EXECUTE IMMEDIATE +'SELECT name FROM nation WHERE regionkey = ? and nationkey < ?' +USING 1, 3; +``` + +This is equivalent to: + +``` +PREPARE statement_name FROM SELECT name FROM nation WHERE regionkey = ? and nationkey < ? +EXECUTE statement_name USING 1, 3 +DEALLOCATE PREPARE statement_name +``` + +## See also + +{doc}`execute`, {doc}`prepare`, {doc}`deallocate-prepare` diff --git a/430/_sources/sql/execute.md.txt b/430/_sources/sql/execute.md.txt new file mode 100644 index 000000000..c3def5586 --- /dev/null +++ b/430/_sources/sql/execute.md.txt @@ -0,0 +1,46 @@ +# EXECUTE + +## Synopsis + +```text +EXECUTE statement_name [ USING parameter1 [ , parameter2, ... ] ] +``` + +## Description + +Executes a prepared statement with the name `statement_name`. Parameter values +are defined in the `USING` clause. + +## Examples + +Prepare and execute a query with no parameters: + +``` +PREPARE my_select1 FROM +SELECT name FROM nation; +``` + +```sql +EXECUTE my_select1; +``` + +Prepare and execute a query with two parameters: + +``` +PREPARE my_select2 FROM +SELECT name FROM nation WHERE regionkey = ? and nationkey < ?; +``` + +```sql +EXECUTE my_select2 USING 1, 3; +``` + +This is equivalent to: + +``` +SELECT name FROM nation WHERE regionkey = 1 AND nationkey < 3; +``` + +## See also + +{doc}`prepare`, {doc}`deallocate-prepare`, {doc}`execute-immediate` diff --git a/430/_sources/sql/explain-analyze.md.txt b/430/_sources/sql/explain-analyze.md.txt new file mode 100644 index 000000000..f49b77771 --- /dev/null +++ b/430/_sources/sql/explain-analyze.md.txt @@ -0,0 +1,115 @@ +# EXPLAIN ANALYZE + +## Synopsis + +```text +EXPLAIN ANALYZE [VERBOSE] statement +``` + +## Description + +Execute the statement and show the distributed execution plan of the statement +along with the cost of each operation. + +The `VERBOSE` option will give more detailed information and low-level statistics; +understanding these may require knowledge of Trino internals and implementation details. + +:::{note} +The stats may not be entirely accurate, especially for queries that complete quickly. +::: + +## Examples + +In the example below, you can see the CPU time spent in each stage, as well as the relative +cost of each plan node in the stage. Note that the relative cost of the plan nodes is based on +wall time, which may or may not be correlated to CPU time. For each plan node you can see +some additional statistics (e.g: average input per node instance). Such statistics are useful +when one wants to detect data anomalies for a query (e.g: skewness). + +```sql +EXPLAIN ANALYZE SELECT count(*), clerk FROM orders +WHERE orderdate > date '1995-01-01' GROUP BY clerk; +``` + +```text + Query Plan +----------------------------------------------------------------------------------------------- +Trino version: version +Queued: 374.17us, Analysis: 190.96ms, Planning: 179.03ms, Execution: 3.06s +Fragment 1 [HASH] + CPU: 22.58ms, Scheduled: 96.72ms, Blocked 46.21s (Input: 23.06s, Output: 0.00ns), Input: 1000 rows (37.11kB); per task: avg.: 1000.00 std.dev.: 0.00, Output: 1000 rows (28.32kB) + Output layout: [clerk, count] + Output partitioning: SINGLE [] + Project[] + │ Layout: [clerk:varchar(15), count:bigint] + │ Estimates: {rows: ? (?), cpu: ?, memory: 0B, network: 0B} + │ CPU: 8.00ms (3.51%), Scheduled: 63.00ms (15.11%), Blocked: 0.00ns (0.00%), Output: 1000 rows (28.32kB) + │ Input avg.: 15.63 rows, Input std.dev.: 24.36% + └─ Aggregate[type = FINAL, keys = [clerk], hash = [$hashvalue]] + │ Layout: [clerk:varchar(15), $hashvalue:bigint, count:bigint] + │ Estimates: {rows: ? (?), cpu: ?, memory: ?, network: 0B} + │ CPU: 8.00ms (3.51%), Scheduled: 22.00ms (5.28%), Blocked: 0.00ns (0.00%), Output: 1000 rows (37.11kB) + │ Input avg.: 15.63 rows, Input std.dev.: 24.36% + │ count := count("count_0") + └─ LocalExchange[partitioning = HASH, hashColumn = [$hashvalue], arguments = ["clerk"]] + │ Layout: [clerk:varchar(15), count_0:bigint, $hashvalue:bigint] + │ Estimates: {rows: ? (?), cpu: ?, memory: 0B, network: 0B} + │ CPU: 2.00ms (0.88%), Scheduled: 4.00ms (0.96%), Blocked: 23.15s (50.10%), Output: 1000 rows (37.11kB) + │ Input avg.: 15.63 rows, Input std.dev.: 793.73% + └─ RemoteSource[sourceFragmentIds = [2]] + Layout: [clerk:varchar(15), count_0:bigint, $hashvalue_1:bigint] + CPU: 0.00ns (0.00%), Scheduled: 0.00ns (0.00%), Blocked: 23.06s (49.90%), Output: 1000 rows (37.11kB) + Input avg.: 15.63 rows, Input std.dev.: 793.73% + +Fragment 2 [SOURCE] + CPU: 210.60ms, Scheduled: 327.92ms, Blocked 0.00ns (Input: 0.00ns, Output: 0.00ns), Input: 1500000 rows (18.17MB); per task: avg.: 1500000.00 std.dev.: 0.00, Output: 1000 rows (37.11kB) + Output layout: [clerk, count_0, $hashvalue_2] + Output partitioning: HASH [clerk][$hashvalue_2] + Aggregate[type = PARTIAL, keys = [clerk], hash = [$hashvalue_2]] + │ Layout: [clerk:varchar(15), $hashvalue_2:bigint, count_0:bigint] + │ CPU: 30.00ms (13.16%), Scheduled: 30.00ms (7.19%), Blocked: 0.00ns (0.00%), Output: 1000 rows (37.11kB) + │ Input avg.: 818058.00 rows, Input std.dev.: 0.00% + │ count_0 := count(*) + └─ ScanFilterProject[table = hive:sf1:orders, filterPredicate = ("orderdate" > DATE '1995-01-01')] + Layout: [clerk:varchar(15), $hashvalue_2:bigint] + Estimates: {rows: 1500000 (41.48MB), cpu: 35.76M, memory: 0B, network: 0B}/{rows: 816424 (22.58MB), cpu: 35.76M, memory: 0B, network: 0B}/{rows: 816424 (22.58MB), cpu: 22.58M, memory: 0B, network: 0B} + CPU: 180.00ms (78.95%), Scheduled: 298.00ms (71.46%), Blocked: 0.00ns (0.00%), Output: 818058 rows (12.98MB) + Input avg.: 1500000.00 rows, Input std.dev.: 0.00% + $hashvalue_2 := combine_hash(bigint '0', COALESCE("$operator$hash_code"("clerk"), 0)) + clerk := clerk:varchar(15):REGULAR + orderdate := orderdate:date:REGULAR + Input: 1500000 rows (18.17MB), Filtered: 45.46%, Physical Input: 4.51MB +``` + +When the `VERBOSE` option is used, some operators may report additional information. +For example, the window function operator will output the following: + +``` +EXPLAIN ANALYZE VERBOSE SELECT count(clerk) OVER() FROM orders +WHERE orderdate > date '1995-01-01'; +``` + +```text + Query Plan +----------------------------------------------------------------------------------------------- + ... + ─ Window[] + │ Layout: [clerk:varchar(15), count:bigint] + │ CPU: 157.00ms (53.40%), Scheduled: 158.00ms (37.71%), Blocked: 0.00ns (0.00%), Output: 818058 rows (22.62MB) + │ metrics: + │ 'CPU time distribution (s)' = {count=1.00, p01=0.16, p05=0.16, p10=0.16, p25=0.16, p50=0.16, p75=0.16, p90=0.16, p95=0.16, p99=0.16, min=0.16, max=0.16} + │ 'Input rows distribution' = {count=1.00, p01=818058.00, p05=818058.00, p10=818058.00, p25=818058.00, p50=818058.00, p75=818058.00, p90=818058.00, p95=818058.00, p99=818058.00, min=818058.00, max=818058.00} + │ 'Scheduled time distribution (s)' = {count=1.00, p01=0.16, p05=0.16, p10=0.16, p25=0.16, p50=0.16, p75=0.16, p90=0.16, p95=0.16, p99=0.16, min=0.16, max=0.16} + │ Input avg.: 818058.00 rows, Input std.dev.: 0.00% + │ Active Drivers: [ 1 / 1 ] + │ Index size: std.dev.: 0.00 bytes, 0.00 rows + │ Index count per driver: std.dev.: 0.00 + │ Rows per driver: std.dev.: 0.00 + │ Size of partition: std.dev.: 0.00 + │ count := count("clerk") RANGE UNBOUNDED_PRECEDING CURRENT_ROW + ... +``` + +## See also + +{doc}`explain` diff --git a/430/_sources/sql/explain.md.txt b/430/_sources/sql/explain.md.txt new file mode 100644 index 000000000..a9b5686e7 --- /dev/null +++ b/430/_sources/sql/explain.md.txt @@ -0,0 +1,782 @@ +# EXPLAIN + +## Synopsis + +```text +EXPLAIN [ ( option [, ...] ) ] statement +``` + +where `option` can be one of: + +```text +FORMAT { TEXT | GRAPHVIZ | JSON } +TYPE { LOGICAL | DISTRIBUTED | VALIDATE | IO } +``` + +## Description + +Show the logical or distributed execution plan of a statement, or validate the statement. +The distributed plan is shown by default. Each plan fragment of the distributed plan is executed by +a single or multiple Trino nodes. Fragments separation represent the data exchange between Trino nodes. +Fragment type specifies how the fragment is executed by Trino nodes and how the data is +distributed between fragments: + +`SINGLE` + +: Fragment is executed on a single node. + +`HASH` + +: Fragment is executed on a fixed number of nodes with the input data + distributed using a hash function. + +`ROUND_ROBIN` + +: Fragment is executed on a fixed number of nodes with the input data + distributed in a round-robin fashion. + +`BROADCAST` + +: Fragment is executed on a fixed number of nodes with the input data + broadcasted to all nodes. + +`SOURCE` + +: Fragment is executed on nodes where input splits are accessed. + +## Examples + +### EXPLAIN (TYPE LOGICAL) + +Process the supplied query statement and create a logical plan in text format: + +``` +EXPLAIN (TYPE LOGICAL) SELECT regionkey, count(*) FROM nation GROUP BY 1; +``` + +```text + Query Plan +----------------------------------------------------------------------------------------------------------------- + Trino version: version + Output[regionkey, _col1] + │ Layout: [regionkey:bigint, count:bigint] + │ Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?} + │ _col1 := count + └─ RemoteExchange[GATHER] + │ Layout: [regionkey:bigint, count:bigint] + │ Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?} + └─ Aggregate(FINAL)[regionkey] + │ Layout: [regionkey:bigint, count:bigint] + │ Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?} + │ count := count("count_8") + └─ LocalExchange[HASH][$hashvalue] ("regionkey") + │ Layout: [regionkey:bigint, count_8:bigint, $hashvalue:bigint] + │ Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?} + └─ RemoteExchange[REPARTITION][$hashvalue_9] + │ Layout: [regionkey:bigint, count_8:bigint, $hashvalue_9:bigint] + │ Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?} + └─ Project[] + │ Layout: [regionkey:bigint, count_8:bigint, $hashvalue_10:bigint] + │ Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?} + │ $hashvalue_10 := "combine_hash"(bigint '0', COALESCE("$operator$hash_code"("regionkey"), 0)) + └─ Aggregate(PARTIAL)[regionkey] + │ Layout: [regionkey:bigint, count_8:bigint] + │ count_8 := count(*) + └─ TableScan[tpch:nation:sf0.01] + Layout: [regionkey:bigint] + Estimates: {rows: 25 (225B), cpu: 225, memory: 0B, network: 0B} + regionkey := tpch:regionkey +``` + +### EXPLAIN (TYPE LOGICAL, FORMAT JSON) + +:::{warning} +The output format is not guaranteed to be backward compatible across Trino versions. +::: + +Process the supplied query statement and create a logical plan in JSON format: + +``` +EXPLAIN (TYPE LOGICAL, FORMAT JSON) SELECT regionkey, count(*) FROM nation GROUP BY 1; +``` + +```json +{ + "id": "9", + "name": "Output", + "descriptor": { + "columnNames": "[regionkey, _col1]" + }, + "outputs": [ + { + "symbol": "regionkey", + "type": "bigint" + }, + { + "symbol": "count", + "type": "bigint" + } + ], + "details": [ + "_col1 := count" + ], + "estimates": [ + { + "outputRowCount": "NaN", + "outputSizeInBytes": "NaN", + "cpuCost": "NaN", + "memoryCost": "NaN", + "networkCost": "NaN" + } + ], + "children": [ + { + "id": "145", + "name": "RemoteExchange", + "descriptor": { + "type": "GATHER", + "isReplicateNullsAndAny": "", + "hashColumn": "" + }, + "outputs": [ + { + "symbol": "regionkey", + "type": "bigint" + }, + { + "symbol": "count", + "type": "bigint" + } + ], + "details": [ + + ], + "estimates": [ + { + "outputRowCount": "NaN", + "outputSizeInBytes": "NaN", + "cpuCost": "NaN", + "memoryCost": "NaN", + "networkCost": "NaN" + } + ], + "children": [ + { + "id": "4", + "name": "Aggregate", + "descriptor": { + "type": "FINAL", + "keys": "[regionkey]", + "hash": "" + }, + "outputs": [ + { + "symbol": "regionkey", + "type": "bigint" + }, + { + "symbol": "count", + "type": "bigint" + } + ], + "details": [ + "count := count(\"count_0\")" + ], + "estimates": [ + { + "outputRowCount": "NaN", + "outputSizeInBytes": "NaN", + "cpuCost": "NaN", + "memoryCost": "NaN", + "networkCost": "NaN" + } + ], + "children": [ + { + "id": "194", + "name": "LocalExchange", + "descriptor": { + "partitioning": "HASH", + "isReplicateNullsAndAny": "", + "hashColumn": "[$hashvalue]", + "arguments": "[\"regionkey\"]" + }, + "outputs": [ + { + "symbol": "regionkey", + "type": "bigint" + }, + { + "symbol": "count_0", + "type": "bigint" + }, + { + "symbol": "$hashvalue", + "type": "bigint" + } + ], + "details":[], + "estimates": [ + { + "outputRowCount": "NaN", + "outputSizeInBytes": "NaN", + "cpuCost": "NaN", + "memoryCost": "NaN", + "networkCost": "NaN" + } + ], + "children": [ + { + "id": "200", + "name": "RemoteExchange", + "descriptor": { + "type": "REPARTITION", + "isReplicateNullsAndAny": "", + "hashColumn": "[$hashvalue_1]" + }, + "outputs": [ + { + "symbol": "regionkey", + "type": "bigint" + }, + { + "symbol": "count_0", + "type": "bigint" + }, + { + "symbol": "$hashvalue_1", + "type": "bigint" + } + ], + "details":[], + "estimates": [ + { + "outputRowCount": "NaN", + "outputSizeInBytes": "NaN", + "cpuCost": "NaN", + "memoryCost": "NaN", + "networkCost": "NaN" + } + ], + "children": [ + { + "id": "226", + "name": "Project", + "descriptor": {} + "outputs": [ + { + "symbol": "regionkey", + "type": "bigint" + }, + { + "symbol": "count_0", + "type": "bigint" + }, + { + "symbol": "$hashvalue_2", + "type": "bigint" + } + ], + "details": [ + "$hashvalue_2 := combine_hash(bigint '0', COALESCE(\"$operator$hash_code\"(\"regionkey\"), 0))" + ], + "estimates": [ + { + "outputRowCount": "NaN", + "outputSizeInBytes": "NaN", + "cpuCost": "NaN", + "memoryCost": "NaN", + "networkCost": "NaN" + } + ], + "children": [ + { + "id": "198", + "name": "Aggregate", + "descriptor": { + "type": "PARTIAL", + "keys": "[regionkey]", + "hash": "" + }, + "outputs": [ + { + "symbol": "regionkey", + "type": "bigint" + }, + { + "symbol": "count_0", + "type": "bigint" + } + ], + "details": [ + "count_0 := count(*)" + ], + "estimates":[], + "children": [ + { + "id": "0", + "name": "TableScan", + "descriptor": { + "table": "hive:tpch_sf1_orc_part:nation" + }, + "outputs": [ + { + "symbol": "regionkey", + "type": "bigint" + } + ], + "details": [ + "regionkey := regionkey:bigint:REGULAR" + ], + "estimates": [ + { + "outputRowCount": 25, + "outputSizeInBytes": 225, + "cpuCost": 225, + "memoryCost": 0, + "networkCost": 0 + } + ], + "children": [] + } + ] + } + ] + } + ] + } + ] + } + ] + } + ] + } + ] +} +``` + +### EXPLAIN (TYPE DISTRIBUTED) + +Process the supplied query statement and create a distributed plan in text +format. The distributed plan splits the logical plan into stages, and therefore +explicitly shows the data exchange between workers: + +``` +EXPLAIN (TYPE DISTRIBUTED) SELECT regionkey, count(*) FROM nation GROUP BY 1; +``` + +```text + Query Plan +------------------------------------------------------------------------------------------------------ + Trino version: version + Fragment 0 [SINGLE] + Output layout: [regionkey, count] + Output partitioning: SINGLE [] + Output[regionkey, _col1] + │ Layout: [regionkey:bigint, count:bigint] + │ Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?} + │ _col1 := count + └─ RemoteSource[1] + Layout: [regionkey:bigint, count:bigint] + + Fragment 1 [HASH] + Output layout: [regionkey, count] + Output partitioning: SINGLE [] + Aggregate(FINAL)[regionkey] + │ Layout: [regionkey:bigint, count:bigint] + │ Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?} + │ count := count("count_8") + └─ LocalExchange[HASH][$hashvalue] ("regionkey") + │ Layout: [regionkey:bigint, count_8:bigint, $hashvalue:bigint] + │ Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?} + └─ RemoteSource[2] + Layout: [regionkey:bigint, count_8:bigint, $hashvalue_9:bigint] + + Fragment 2 [SOURCE] + Output layout: [regionkey, count_8, $hashvalue_10] + Output partitioning: HASH [regionkey][$hashvalue_10] + Project[] + │ Layout: [regionkey:bigint, count_8:bigint, $hashvalue_10:bigint] + │ Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?} + │ $hashvalue_10 := "combine_hash"(bigint '0', COALESCE("$operator$hash_code"("regionkey"), 0)) + └─ Aggregate(PARTIAL)[regionkey] + │ Layout: [regionkey:bigint, count_8:bigint] + │ count_8 := count(*) + └─ TableScan[tpch:nation:sf0.01, grouped = false] + Layout: [regionkey:bigint] + Estimates: {rows: 25 (225B), cpu: 225, memory: 0B, network: 0B} + regionkey := tpch:regionkey +``` + +### EXPLAIN (TYPE DISTRIBUTED, FORMAT JSON) + +:::{warning} +The output format is not guaranteed to be backward compatible across Trino versions. +::: + +Process the supplied query statement and create a distributed plan in JSON +format. The distributed plan splits the logical plan into stages, and therefore +explicitly shows the data exchange between workers: + +``` +EXPLAIN (TYPE DISTRIBUTED, FORMAT JSON) SELECT regionkey, count(*) FROM nation GROUP BY 1; +``` + +```json +{ + "0" : { + "id" : "9", + "name" : "Output", + "descriptor" : { + "columnNames" : "[regionkey, _col1]" + }, + "outputs" : [ { + "symbol" : "regionkey", + "type" : "bigint" + }, { + "symbol" : "count", + "type" : "bigint" + } ], + "details" : [ "_col1 := count" ], + "estimates" : [ { + "outputRowCount" : "NaN", + "outputSizeInBytes" : "NaN", + "cpuCost" : "NaN", + "memoryCost" : "NaN", + "networkCost" : "NaN" + } ], + "children" : [ { + "id" : "145", + "name" : "RemoteSource", + "descriptor" : { + "sourceFragmentIds" : "[1]" + }, + "outputs" : [ { + "symbol" : "regionkey", + "type" : "bigint" + }, { + "symbol" : "count", + "type" : "bigint" + } ], + "details" : [ ], + "estimates" : [ ], + "children" : [ ] + } ] + }, + "1" : { + "id" : "4", + "name" : "Aggregate", + "descriptor" : { + "type" : "FINAL", + "keys" : "[regionkey]", + "hash" : "[]" + }, + "outputs" : [ { + "symbol" : "regionkey", + "type" : "bigint" + }, { + "symbol" : "count", + "type" : "bigint" + } ], + "details" : [ "count := count(\"count_0\")" ], + "estimates" : [ { + "outputRowCount" : "NaN", + "outputSizeInBytes" : "NaN", + "cpuCost" : "NaN", + "memoryCost" : "NaN", + "networkCost" : "NaN" + } ], + "children" : [ { + "id" : "194", + "name" : "LocalExchange", + "descriptor" : { + "partitioning" : "SINGLE", + "isReplicateNullsAndAny" : "", + "hashColumn" : "[]", + "arguments" : "[]" + }, + "outputs" : [ { + "symbol" : "regionkey", + "type" : "bigint" + }, { + "symbol" : "count_0", + "type" : "bigint" + } ], + "details" : [ ], + "estimates" : [ { + "outputRowCount" : "NaN", + "outputSizeInBytes" : "NaN", + "cpuCost" : "NaN", + "memoryCost" : "NaN", + "networkCost" : "NaN" + } ], + "children" : [ { + "id" : "227", + "name" : "Project", + "descriptor" : { }, + "outputs" : [ { + "symbol" : "regionkey", + "type" : "bigint" + }, { + "symbol" : "count_0", + "type" : "bigint" + } ], + "details" : [ ], + "estimates" : [ { + "outputRowCount" : "NaN", + "outputSizeInBytes" : "NaN", + "cpuCost" : "NaN", + "memoryCost" : "NaN", + "networkCost" : "NaN" + } ], + "children" : [ { + "id" : "200", + "name" : "RemoteSource", + "descriptor" : { + "sourceFragmentIds" : "[2]" + }, + "outputs" : [ { + "symbol" : "regionkey", + "type" : "bigint" + }, { + "symbol" : "count_0", + "type" : "bigint" + }, { + "symbol" : "$hashvalue", + "type" : "bigint" + } ], + "details" : [ ], + "estimates" : [ ], + "children" : [ ] + } ] + } ] + } ] + }, + "2" : { + "id" : "226", + "name" : "Project", + "descriptor" : { }, + "outputs" : [ { + "symbol" : "regionkey", + "type" : "bigint" + }, { + "symbol" : "count_0", + "type" : "bigint" + }, { + "symbol" : "$hashvalue_1", + "type" : "bigint" + } ], + "details" : [ "$hashvalue_1 := combine_hash(bigint '0', COALESCE(\"$operator$hash_code\"(\"regionkey\"), 0))" ], + "estimates" : [ { + "outputRowCount" : "NaN", + "outputSizeInBytes" : "NaN", + "cpuCost" : "NaN", + "memoryCost" : "NaN", + "networkCost" : "NaN" + } ], + "children" : [ { + "id" : "198", + "name" : "Aggregate", + "descriptor" : { + "type" : "PARTIAL", + "keys" : "[regionkey]", + "hash" : "[]" + }, + "outputs" : [ { + "symbol" : "regionkey", + "type" : "bigint" + }, { + "symbol" : "count_0", + "type" : "bigint" + } ], + "details" : [ "count_0 := count(*)" ], + "estimates" : [ ], + "children" : [ { + "id" : "0", + "name" : "TableScan", + "descriptor" : { + "table" : "tpch:tiny:nation" + }, + "outputs" : [ { + "symbol" : "regionkey", + "type" : "bigint" + } ], + "details" : [ "regionkey := tpch:regionkey" ], + "estimates" : [ { + "outputRowCount" : 25.0, + "outputSizeInBytes" : 225.0, + "cpuCost" : 225.0, + "memoryCost" : 0.0, + "networkCost" : 0.0 + } ], + "children" : [ ] + } ] + } ] + } +} +``` + +### EXPLAIN (TYPE VALIDATE) + +Validate the supplied query statement for syntactical and semantic correctness. +Returns true if the statement is valid: + +``` +EXPLAIN (TYPE VALIDATE) SELECT regionkey, count(*) FROM nation GROUP BY 1; +``` + +```text + Valid +------- + true +``` + +If the statement is not correct because a syntax error, such as an unknown +keyword, is found the error message details the problem: + +``` +EXPLAIN (TYPE VALIDATE) SELET 1=0; +``` + +```text +Query 20220929_234840_00001_vjwxj failed: line 1:25: mismatched input 'SELET'. +Expecting: 'ALTER', 'ANALYZE', 'CALL', 'COMMENT', 'COMMIT', 'CREATE', +'DEALLOCATE', 'DELETE', 'DENY', 'DESC', 'DESCRIBE', 'DROP', 'EXECUTE', +'EXPLAIN', 'GRANT', 'INSERT', 'MERGE', 'PREPARE', 'REFRESH', 'RESET', +'REVOKE', 'ROLLBACK', 'SET', 'SHOW', 'START', 'TRUNCATE', 'UPDATE', 'USE', + +``` + +Similarly if semantic issues are detected, such as an invalid object name +`nations` instead of `nation`, the error message returns useful +information: + +``` +EXPLAIN(TYPE VALIDATE) SELECT * FROM tpch.tiny.nations; +``` + +```text +Query 20220929_235059_00003_vjwxj failed: line 1:15: Table 'tpch.tiny.nations' does not exist +SELECT * FROM tpch.tiny.nations +``` + +### EXPLAIN (TYPE IO) + +Process the supplied query statement and create a plan with input and output +details about the accessed objects in JSON format: + +``` +EXPLAIN (TYPE IO, FORMAT JSON) INSERT INTO test_lineitem +SELECT * FROM lineitem WHERE shipdate = '2020-02-01' AND quantity > 10; +``` + +```text + Query Plan +----------------------------------- +{ + inputTableColumnInfos: [ + { + table: { + catalog: "hive", + schemaTable: { + schema: "tpch", + table: "test_orders" + } + }, + columnConstraints: [ + { + columnName: "orderkey", + type: "bigint", + domain: { + nullsAllowed: false, + ranges: [ + { + low: { + value: "1", + bound: "EXACTLY" + }, + high: { + value: "1", + bound: "EXACTLY" + } + }, + { + low: { + value: "2", + bound: "EXACTLY" + }, + high: { + value: "2", + bound: "EXACTLY" + } + } + ] + } + }, + { + columnName: "processing", + type: "boolean", + domain: { + nullsAllowed: false, + ranges: [ + { + low: { + value: "false", + bound: "EXACTLY" + }, + high: { + value: "false", + bound: "EXACTLY" + } + } + ] + } + }, + { + columnName: "custkey", + type: "bigint", + domain: { + nullsAllowed: false, + ranges: [ + { + low: { + bound: "ABOVE" + }, + high: { + value: "10", + bound: "EXACTLY" + } + } + ] + } + } + ], + estimate: { + outputRowCount: 2, + outputSizeInBytes: 40, + cpuCost: 40, + maxMemory: 0, + networkCost: 0 + } + } + ], + outputTable: { + catalog: "hive", + schemaTable: { + schema: "tpch", + table: "test_orders" + } + }, + estimate: { + outputRowCount: "NaN", + outputSizeInBytes: "NaN", + cpuCost: "NaN", + maxMemory: "NaN", + networkCost: "NaN" + } +} +``` + +## See also + +{doc}`explain-analyze` diff --git a/430/_sources/sql/grant-roles.md.txt b/430/_sources/sql/grant-roles.md.txt new file mode 100644 index 000000000..a11d313a0 --- /dev/null +++ b/430/_sources/sql/grant-roles.md.txt @@ -0,0 +1,51 @@ +# GRANT ROLES + +## Synopsis + +```text +GRANT role [, ...] +TO ( user | USER user | ROLE role) [, ...] +[ GRANTED BY ( user | USER user | ROLE role | CURRENT_USER | CURRENT_ROLE ) ] +[ WITH ADMIN OPTION ] +[ IN catalog ] +``` + +## Description + +Grants the specified role(s) to the specified principal(s). + +If the `WITH ADMIN OPTION` clause is specified, the role(s) are granted +to the users with `GRANT` option. + +For the `GRANT` statement for roles to succeed, the user executing it either should +be the role admin or should possess the `GRANT` option for the given role. + +The optional `GRANTED BY` clause causes the role(s) to be granted with +the specified principal as a grantor. If the `GRANTED BY` clause is not +specified, the roles are granted with the current user as a grantor. + +The optional `IN catalog` clause grants the roles in a catalog as opposed +to a system roles. + +## Examples + +Grant role `bar` to user `foo` + +``` +GRANT bar TO USER foo; +``` + +Grant roles `bar` and `foo` to user `baz` and role `qux` with admin option + +``` +GRANT bar, foo TO USER baz, ROLE qux WITH ADMIN OPTION; +``` + +## Limitations + +Some connectors do not support role management. +See connector documentation for more details. + +## See also + +{doc}`create-role`, {doc}`drop-role`, {doc}`set-role`, {doc}`revoke-roles` diff --git a/430/_sources/sql/grant.md.txt b/430/_sources/sql/grant.md.txt new file mode 100644 index 000000000..1177c7682 --- /dev/null +++ b/430/_sources/sql/grant.md.txt @@ -0,0 +1,61 @@ +# GRANT + +## Synopsis + +```text +GRANT ( privilege [, ...] | ( ALL PRIVILEGES ) ) +ON ( table_name | TABLE table_name | SCHEMA schema_name) +TO ( user | USER user | ROLE role ) +[ WITH GRANT OPTION ] +``` + +## Description + +Grants the specified privileges to the specified grantee. + +Specifying `ALL PRIVILEGES` grants {doc}`delete`, {doc}`insert`, {doc}`update` and {doc}`select` privileges. + +Specifying `ROLE PUBLIC` grants privileges to the `PUBLIC` role and hence to all users. + +The optional `WITH GRANT OPTION` clause allows the grantee to grant these same privileges to others. + +For `GRANT` statement to succeed, the user executing it should possess the specified privileges as well as the `GRANT OPTION` for those privileges. + +Grant on a table grants the specified privilege on all current and future columns of the table. + +Grant on a schema grants the specified privilege on all current and future columns of all current and future tables of the schema. + +## Examples + +Grant `INSERT` and `SELECT` privileges on the table `orders` to user `alice`: + +``` +GRANT INSERT, SELECT ON orders TO alice; +``` + +Grant `DELETE` privilege on the schema `finance` to user `bob`: + +``` +GRANT DELETE ON SCHEMA finance TO bob; +``` + +Grant `SELECT` privilege on the table `nation` to user `alice`, additionally allowing `alice` to grant `SELECT` privilege to others: + +``` +GRANT SELECT ON nation TO alice WITH GRANT OPTION; +``` + +Grant `SELECT` privilege on the table `orders` to everyone: + +``` +GRANT SELECT ON orders TO ROLE PUBLIC; +``` + +## Limitations + +Some connectors have no support for `GRANT`. +See connector documentation for more details. + +## See also + +{doc}`deny`, {doc}`revoke`, {doc}`show-grants` diff --git a/430/_sources/sql/insert.md.txt b/430/_sources/sql/insert.md.txt new file mode 100644 index 000000000..677fc9a93 --- /dev/null +++ b/430/_sources/sql/insert.md.txt @@ -0,0 +1,57 @@ +# INSERT + +## Synopsis + +```text +INSERT INTO table_name [ ( column [, ... ] ) ] query +``` + +## Description + +Insert new rows into a table. + +If the list of column names is specified, they must exactly match the list +of columns produced by the query. Each column in the table not present in the +column list will be filled with a `null` value. Otherwise, if the list of +columns is not specified, the columns produced by the query must exactly match +the columns in the table being inserted into. + +## Examples + +Load additional rows into the `orders` table from the `new_orders` table: + +``` +INSERT INTO orders +SELECT * FROM new_orders; +``` + +Insert a single row into the `cities` table: + +``` +INSERT INTO cities VALUES (1, 'San Francisco'); +``` + +Insert multiple rows into the `cities` table: + +``` +INSERT INTO cities VALUES (2, 'San Jose'), (3, 'Oakland'); +``` + +Insert a single row into the `nation` table with the specified column list: + +``` +INSERT INTO nation (nationkey, name, regionkey, comment) +VALUES (26, 'POLAND', 3, 'no comment'); +``` + +Insert a row without specifying the `comment` column. +That column will be `null`: + +``` +INSERT INTO nation (nationkey, name, regionkey) +VALUES (26, 'POLAND', 3); +``` + +## See also + +{doc}`values` diff --git a/430/_sources/sql/match-recognize.md.txt b/430/_sources/sql/match-recognize.md.txt new file mode 100644 index 000000000..cffddbccd --- /dev/null +++ b/430/_sources/sql/match-recognize.md.txt @@ -0,0 +1,809 @@ +# MATCH_RECOGNIZE + +## Synopsis + +```text +MATCH_RECOGNIZE ( + [ PARTITION BY column [, ...] ] + [ ORDER BY column [, ...] ] + [ MEASURES measure_definition [, ...] ] + [ rows_per_match ] + [ AFTER MATCH skip_to ] + PATTERN ( row_pattern ) + [ SUBSET subset_definition [, ...] ] + DEFINE variable_definition [, ...] + ) +``` + +## Description + +The `MATCH_RECOGNIZE` clause is an optional subclause of the `FROM` clause. +It is used to detect patterns in a set of rows. Patterns of interest are +specified using row pattern syntax based on regular expressions. The input to +pattern matching is a table, a view or a subquery. For each detected match, one +or more rows are returned. They contain requested information about the match. + +Row pattern matching is a powerful tool when analyzing complex sequences of +events. The following examples show some of the typical use cases: + +- in trade applications, tracking trends or identifying customers with specific + behavioral patterns +- in shipping applications, tracking packages through all possible valid paths, +- in financial applications, detecting unusual incidents, which might signal + fraud + +## Example + +In the following example, the pattern describes a V-shape over the +`totalprice` column. A match is found whenever orders made by a customer +first decrease in price, and then increase past the starting point: + +``` +SELECT * FROM orders MATCH_RECOGNIZE( + PARTITION BY custkey + ORDER BY orderdate + MEASURES + A.totalprice AS starting_price, + LAST(B.totalprice) AS bottom_price, + LAST(U.totalprice) AS top_price + ONE ROW PER MATCH + AFTER MATCH SKIP PAST LAST ROW + PATTERN (A B+ C+ D+) + SUBSET U = (C, D) + DEFINE + B AS totalprice < PREV(totalprice), + C AS totalprice > PREV(totalprice) AND totalprice <= A.totalprice, + D AS totalprice > PREV(totalprice) + ) +``` + +In the following sections, all subclauses of the `MATCH_RECOGNIZE` clause are +explained with this example query. + +## Partitioning and ordering + +```sql +PARTITION BY custkey +``` + +The `PARTITION BY` clause allows you to break up the input table into +separate sections, that are independently processed for pattern matching. +Without a partition declaration, the whole input table is used. This behavior +is analogous to the semantics of `PARTITION BY` clause in {ref}`window +specification`. In the example, the `orders` table is +partitioned by the `custkey` value, so that pattern matching is performed for +all orders of a specific customer independently from orders of other +customers. + +```sql +ORDER BY orderdate +``` + +The optional `ORDER BY` clause is generally useful to allow matching on an +ordered data set. For example, sorting the input by `orderdate` allows for +matching on a trend of changes over time. + +(row-pattern-measures)= + +## Row pattern measures + +The `MEASURES` clause allows to specify what information is retrieved from a +matched sequence of rows. + +```text +MEASURES measure_expression AS measure_name [, ...] +``` + +A measure expression is a scalar expression whose value is computed based on a +match. In the example, three row pattern measures are specified: + +`A.totalprice AS starting_price` returns the price in the first row of the +match, which is the only row associated with `A` according to the pattern. + +`LAST(B.totalprice) AS bottom_price` returns the lowest price (corresponding +to the bottom of the "V" in the pattern). It is the price in the last row +associated with `B`, which is the last row of the descending section. + +`LAST(U.totalprice) AS top_price` returns the highest price in the match. It +is the price in the last row associated with `C` or `D`, which is also the +final row of the match. + +Measure expressions can refer to the columns of the input table. They also +allow special syntax to combine the input information with the details of the +match (see {ref}`pattern-recognition-expressions`). + +Each measure defines an output column of the pattern recognition. The column +can be referenced with the `measure_name`. + +The `MEASURES` clause is optional. When no measures are specified, certain +input columns (depending on {ref}`ROWS PER MATCH` clause) are +the output of the pattern recognition. + +(rows-per-match)= + +## Rows per match + +This clause can be used to specify the quantity of output rows. There are two +main options: + +``` +ONE ROW PER MATCH +``` + +and + +```sql +ALL ROWS PER MATCH +``` + +`ONE ROW PER MATCH` is the default option. For every match, a single row of +output is produced. Output consists of `PARTITION BY` columns and measures. +The output is also produced for empty matches, based on their starting rows. +Rows that are unmatched (that is, neither included in some non-empty match, nor +being the starting row of an empty match), are not included in the output. + +For `ALL ROWS PER MATCH`, every row of a match produces an output row, unless +it is excluded from the output by the {ref}`exclusion-syntax`. Output consists +of `PARTITION BY` columns, `ORDER BY` columns, measures and remaining +columns from the input table. By default, empty matches are shown and unmatched +rows are skipped, similarly as with the `ONE ROW PER MATCH` option. However, +this behavior can be changed by modifiers: + +``` +ALL ROWS PER MATCH SHOW EMPTY MATCHES +``` + +shows empty matches and skips unmatched rows, like the default. + +```sql +ALL ROWS PER MATCH OMIT EMPTY MATCHES +``` + +excludes empty matches from the output. + +```sql +ALL ROWS PER MATCH WITH UNMATCHED ROWS +``` + +shows empty matches and produces additional output row for each unmatched row. + +There are special rules for computing row pattern measures for empty matches +and unmatched rows. They are explained in +{ref}`empty-matches-and-unmatched-rows`. + +Unmatched rows can only occur when the pattern does not allow an empty match. +Otherwise, they are considered as starting rows of empty matches. The option +`ALL ROWS PER MATCH WITH UNMATCHED ROWS` is recommended when pattern +recognition is expected to pass all input rows, and it is not certain whether +the pattern allows an empty match. + +(after-match-skip)= + +## After match skip + +The `AFTER MATCH SKIP` clause specifies where pattern matching resumes after +a non-empty match is found. + +The default option is: + +``` +AFTER MATCH SKIP PAST LAST ROW +``` + +With this option, pattern matching starts from the row after the last row of +the match. Overlapping matches are not detected. + +With the following option, pattern matching starts from the second row of the +match: + +``` +AFTER MATCH SKIP TO NEXT ROW +``` + +In the example, if a V-shape is detected, further overlapping matches are +found, starting from consecutive rows on the descending slope of the "V". +Skipping to the next row is the default behavior after detecting an empty match +or unmatched row. + +The following `AFTER MATCH SKIP` options allow to resume pattern matching +based on the components of the pattern. Pattern matching starts from the last +(default) or first row matched to a certain row pattern variable. It can be +either a primary pattern variable (they are explained in +{ref}`row-pattern-syntax`) or a +{ref}`union variable`: + +``` +AFTER MATCH SKIP TO [ FIRST | LAST ] pattern_variable +``` + +It is forbidden to skip to the first row of the current match, because it +results in an infinite loop. For example specifying `AFTER MATCH SKIP TO A` +fails, because `A` is the first element of the pattern, and jumping back to +it creates an infinite loop. Similarly, skipping to a pattern variable which is +not present in the match causes failure. + +All other options than the default `AFTER MATCH SKIP PAST LAST ROW` allow +detection of overlapping matches. The combination of `ALL ROWS PER MATCH WITH +UNMATCHED ROWS` with `AFTER MATCH SKIP PAST LAST ROW` is the only +configuration that guarantees exactly one output row for each input row. + +(row-pattern-syntax)= + +## Row pattern syntax + +Row pattern is a form of a regular expression with some syntactical extensions +specific to row pattern recognition. It is specified in the `PATTERN` +clause: + +``` +PATTERN ( row_pattern ) +``` + +The basic element of row pattern is a primary pattern variable. Like pattern +matching in character strings searches for characters, pattern matching in row +sequences searches for rows which can be "labeled" with certain primary pattern +variables. A primary pattern variable has a form of an identifier and is +{ref}`defined` by a boolean condition. This +condition determines whether a particular input row can be mapped to this +variable and take part in the match. + +In the example `PATTERN (A B+ C+ D+)`, there are four primary pattern +variables: `A`, `B`, `C`, and `D`. + +Row pattern syntax includes the following usage: + +### concatenation + +```text +A B+ C+ D+ +``` + +It is a sequence of components without operators between them. All components +are matched in the same order as they are specified. + +### alternation + +```text +A | B | C +``` + +It is a sequence of components separated by `|`. Exactly one of the +components is matched. In case when multiple components can be matched, the +leftmost matching component is chosen. + +(permute-function)= + +### permutation + +```text +PERMUTE(A, B, C) +``` + +It is equivalent to alternation of all permutations of its components. All +components are matched in some order. If multiple matches are possible for +different orderings of the components, the match is chosen based on the +lexicographical order established by the order of components in the `PERMUTE` +list. In the above example, the most preferred option is `A B C`, and the +least preferred option is `C B A`. + +### grouping + +```text +(A B C) +``` + +### partition start anchor + +```text +^ +``` + +### partition end anchor + +```text +$ +``` + +### empty pattern + +```text +() +``` + +(exclusion-syntax)= + +### exclusion syntax + +```text +{- row_pattern -} +``` + +Exclusion syntax is used to specify portions of the match to exclude from the +output. It is useful in combination with the `ALL ROWS PER MATCH` option, +when only certain sections of the match are interesting. + +If you change the example to use `ALL ROWS PER MATCH`, and the pattern is +modified to `PATTERN (A {- B+ C+ -} D+)`, the result consists of the initial +matched row and the trailing section of rows. + +Specifying pattern exclusions does not affect the computation of expressions in +`MEASURES` and `DEFINE` clauses. Exclusions also do not affect pattern +matching. They have the same semantics as regular grouping with parentheses. + +It is forbidden to specify pattern exclusions with the option `ALL ROWS PER +MATCH WITH UNMATCHED ROWS`. + +### quantifiers + +Pattern quantifiers allow to specify the desired number of repetitions of a +sub-pattern in a match. They are appended after the relevant pattern +component: + +``` +(A | B)* +``` + +There are following row pattern quantifiers: + +- zero or more repetitions: + +```text +* +``` + +- one or more repetitions: + +```text ++ +``` + +- zero or one repetition: + +```text +? +``` + +- exact number of repetitions, specified by a non-negative integer number: + +```text +{n} +``` + +- number of repetitions ranging between bounds, specified by non-negative + integer numbers: + +```text +{m, n} +``` + +Specifying bounds is optional. If the left bound is omitted, it defaults to +`0`. So, `{, 5}` can be described as "between zero and five repetitions". +If the right bound is omitted, the number of accepted repetitions is unbounded. +So, `{5, }` can be described as "at least five repetitions". Also, `{,}` is +equivalent to `*`. + +Quantifiers are greedy by default. It means that higher number of repetitions +is preferred over lower number. This behavior can be changed to reluctant by +appending `?` immediately after the quantifier. With `{3, 5}`, 3 +repetitions is the least desired option and 5 repetitions -- the most desired. +With `{3, 5}?`, 3 repetitions are most desired. Similarly, `?` prefers 1 +repetition, while `??` prefers 0 repetitions. + +(row-pattern-union-variables)= + +## Row pattern union variables + +As explained in {ref}`row-pattern-syntax`, primary pattern variables are the +basic elements of row pattern. In addition to primary pattern variables, you +can define union variables. They are introduced in the `SUBSET` clause: + +``` +SUBSET U = (C, D), ... +``` + +In the preceding example, union variable `U` is defined as union of primary +variables `C` and `D`. Union variables are useful in `MEASURES`, +`DEFINE` and `AFTER MATCH SKIP` clauses. They allow you to refer to set of +rows matched to either primary variable from a subset. + +With the pattern: `PATTERN((A | B){5} C+)` it cannot be determined upfront if +the match contains any `A` or any `B`. A union variable can be used to +access the last row matched to either `A` or `B`. Define `SUBSET U = +(A, B)`, and the expression `LAST(U.totalprice)` returns the value of the +`totalprice` column from the last row mapped to either `A` or `B`. Also, +`AFTER MATCH SKIP TO LAST A` or `AFTER MATCH SKIP TO LAST B` can result in +failure if `A` or `B` is not present in the match. `AFTER MATCH SKIP TO +LAST U` does not fail. + +(row-pattern-variable-definitions)= + +## Row pattern variable definitions + +The `DEFINE` clause is where row pattern primary variables are defined. Each +variable is associated with a boolean condition: + +``` +DEFINE B AS totalprice < PREV(totalprice), ... +``` + +During pattern matching, when a certain variable is considered for the next +step of the match, the boolean condition is evaluated in context of the current +match. If the result is `true`, then the current row, "labeled" with the +variable, becomes part of the match. + +In the preceding example, assume that the pattern allows to match `B` at some +point. There are some rows already matched to some pattern variables. Now, +variable `B` is being considered for the current row. Before the match is +made, the defining condition for `B` is evaluated. In this example, it is +only true if the value of the `totalprice` column in the current row is lower +than `totalprice` in the preceding row. + +The mechanism of matching variables to rows shows the difference between +pattern matching in row sequences and regular expression matching in text. In +text, characters remain constantly in their positions. In row pattern matching, +a row can be mapped to different variables in different matches, depending on +the preceding part of the match, and even on the match number. + +It is not required that every primary variable has a definition in the +`DEFINE` clause. Variables not mentioned in the `DEFINE` clause are +implicitly associated with `true` condition, which means that they can be +matched to every row. + +Boolean expressions in the `DEFINE` clause allow the same special syntax as +expressions in the `MEASURES` clause. Details are explained in +{ref}`pattern-recognition-expressions`. + +(pattern-recognition-expressions)= + +## Row pattern recognition expressions + +Expressions in {ref}`MEASURES` and +{ref}`DEFINE` clauses are scalar expressions +evaluated over rows of the input table. They support special syntax, specific +to pattern recognition context. They can combine input information with the +information about the current match. Special syntax allows to access pattern +variables assigned to rows, browse rows based on how they are matched, and +refer to the sequential number of the match. + +### pattern variable references + +```sql +A.totalprice + +U.orderdate + +orderstatus +``` + +A column name prefixed with a pattern variable refers to values of this column +in all rows matched to this variable, or to any variable from the subset in +case of union variable. If a column name is not prefixed, it is considered as +prefixed with the `universal pattern variable`, defined as union of all +primary pattern variables. In other words, a non-prefixed column name refers to +all rows of the current match. + +It is forbidden to prefix a column name with a table name in the pattern +recognition context. + +(classifier-function)= + +### classifier function + +```sql +CLASSIFIER() + +CLASSIFIER(A) + +CLASSIFIER(U) +``` + +The `classifier` function returns the primary pattern variable associated +with the row. The return type is `varchar`. The optional argument is a +pattern variable. It limits the rows of interest, the same way as with prefixed +column references. The `classifier` function is particularly useful with a +union variable as the argument. It allows you to determine which variable from +the subset actually matched. + +(match-number-function)= + +### match_number function + +```sql +MATCH_NUMBER() +``` + +The `match_number` function returns the sequential number of the match within +partition, starting from `1`. Empty matches are assigned sequential numbers +as well as non-empty matches. The return type is `bigint`. + +(logical-navigation-functions)= + +### logical navigation functions + +```sql +FIRST(A.totalprice, 2) +``` + +In the above example, the `first` function navigates to the first row matched +to pattern variable `A`, and then searches forward until it finds two more +occurrences of variable `A` within the match. The result is the value of the +`totalprice` column in that row. + +```sql +LAST(A.totalprice, 2) +``` + +In the above example, the `last` function navigates to the last row matched +to pattern variable `A`, and then searches backwards until it finds two more +occurrences of variable `A` within the match. The result is the value of the +`totalprice` column in that row. + +With the `first` and `last` functions the result is `null`, if the +searched row is not found in the mach. + +The second argument is optional. The default value is `0`, which means that +by default these functions navigate to the first or last row of interest. If +specified, the second argument must be a non-negative integer number. + +(physical-navigation-functions)= + +### physical navigation functions + +```sql +PREV(A.totalprice, 2) +``` + +In the above example, the `prev` function navigates to the last row matched +to pattern variable `A`, and then searches two rows backward. The result is +the value of the `totalprice` column in that row. + +```sql +NEXT(A.totalprice, 2) +``` + +In the above example, the `next` function navigates to the last row matched +to pattern variable `A`, and then searches two rows forward. The result is +the value of the `totalprice` column in that row. + +With the `prev` and `next` functions, it is possible to navigate and +retrieve values outside the match. If the navigation goes beyond partition +bounds, the result is `null`. + +The second argument is optional. The default value is `1`, which means that +by default these functions navigate to previous or next row. If specified, the +second argument must be a non-negative integer number. + +### nesting of navigation functions + +It is possible to nest logical navigation functions within physical navigation +functions: + +```sql +PREV(FIRST(A.totalprice, 3), 2) +``` + +In case of nesting, first the logical navigation is performed. It establishes +the starting row for the physical navigation. When both navigation operations +succeed, the value is retrieved from the designated row. + +Pattern navigation functions require at least one column reference or +`classifier` function inside of their first argument. The following examples +are correct: + +``` +LAST("pattern_variable_" || CLASSIFIER()) + +NEXT(U.totalprice + 10) +``` + +This is incorrect: + +``` +LAST(1) +``` + +It is also required that all column references and all `classifier` calls +inside a pattern navigation function are consistent in referred pattern +variables. They must all refer either to the same primary variable, the same +union variable, or to the implicit universal pattern variable. The following +examples are correct: + +``` +LAST(CLASSIFIER() = 'A' OR totalprice > 10) /* universal pattern variable */ + +LAST(CLASSIFIER(U) = 'A' OR U.totalprice > 10) /* pattern variable U */ +``` + +This is incorrect: + +``` +LAST(A.totalprice + B.totalprice) +``` + +### Aggregate functions + +It is allowed to use aggregate functions in a row pattern recognition context. +Aggregate functions are evaluated over all rows of the current match or over a +subset of rows based on the matched pattern variables. The +{ref}`running and final semantics` are supported, with +`running` as the default. + +The following expression returns the average value of the `totalprice` column +for all rows matched to pattern variable `A`: + +``` +avg(A.totalprice) +``` + +The following expression returns the average value of the `totalprice` column +for all rows matched to pattern variables from subset `U`: + +``` +avg(U.totalprice) +``` + +The following expression returns the average value of the `totalprice` column +for all rows of the match: + +``` +avg(totalprice) +``` + +#### Aggregation arguments + +In case when the aggregate function has multiple arguments, it is required that +all arguments refer consistently to the same set of rows: + +``` +max_by(totalprice, tax) /* aggregate over all rows of the match */ + +max_by(CLASSIFIER(A), A.tax) /* aggregate over all rows matched to A */ +``` + +This is incorrect: + +``` +max_by(A.totalprice, tax) + +max_by(A.totalprice, A.tax + B.tax) +``` + +If an aggregate argument does not contain any column reference or +`classifier` function, it does not refer to any pattern variable. In such a +case other aggregate arguments determine the set of rows to aggregate over. If +none of the arguments contains a pattern variable reference, the universal row +pattern variable is implicit. This means that the aggregate function applies to +all rows of the match: + +``` +count(1) /* aggregate over all rows of the match */ + +min_by(1, 2) /* aggregate over all rows of the match */ + +min_by(1, totalprice) /* aggregate over all rows of the match */ + +min_by(totalprice, 1) /* aggregate over all rows of the match */ + +min_by(A.totalprice, 1) /* aggregate over all rows matched to A */ + +max_by(1, A.totalprice) /* aggregate over all rows matched to A */ +``` + +#### Nesting of aggregate functions + +Aggregate function arguments must not contain pattern navigation functions. +Similarly, aggregate functions cannot be nested in pattern navigation +functions. + +#### Usage of the `classifier` and `match_number` functions + +It is allowed to use the `classifier` and `match_number` functions in +aggregate function arguments. The following expression returns an array +containing all matched pattern variables: + +``` +array_agg(CLASSIFIER()) +``` + +This is particularly useful in combination with the option +`ONE ROW PER MATCH`. It allows to get all the components of the match while +keeping the output size reduced. + +#### Row pattern count aggregation + +Like other aggregate functions in a row pattern recognition context, the +`count` function can be applied to all rows of the match, or to rows +associated with certain row pattern variables: + +``` +count(*), count() /* count all rows of the match */ + +count(totalprice) /* count non-null values of the totalprice column + in all rows of the match */ + +count(A.totalprice) /* count non-null values of the totalprice column + in all rows matched to A */ +``` + +The `count` function in a row pattern recognition context allows special syntax +to support the `count(*)` behavior over a limited set of rows: + +``` +count(A.*) /* count rows matched to A */ + +count(U.*) /* count rows matched to pattern variables from subset U */ +``` + +(running-and-final)= + +### `RUNNING` and `FINAL` semantics + +During pattern matching in a sequence of rows, one row after another is +examined to determine if it fits the pattern. At any step, a partial match is +known, but it is not yet known what rows will be added in the future or what +pattern variables they will be mapped to. So, when evaluating a boolean +condition in the `DEFINE` clause for the current row, only the preceding part +of the match (plus the current row) is "visible". This is the `running` +semantics. + +When evaluating expressions in the `MEASURES` clause, the match is complete. +It is then possible to apply the `final` semantics. In the `final` +semantics, the whole match is "visible" as from the position of the final row. + +In the `MEASURES` clause, the `running` semantics can also be applied. When +outputting information row by row (as in `ALL ROWS PER MATCH`), the +`running` semantics evaluate expressions from the positions of consecutive +rows. + +The `running` and `final` semantics are denoted by the keywords: +`RUNNING` and `FINAL`, preceding a logical navigation function `first` or +`last`, or an aggregate function: + +``` +RUNNING LAST(A.totalprice) + +FINAL LAST(A.totalprice) + +RUNNING avg(A.totalprice) + +FINAL count(A.*) +``` + +The `running` semantics is default in `MEASURES` and `DEFINE` clauses. +`FINAL` can only be specified in the `MEASURES` clause. + +With the option `ONE ROW PER MATCH`, row pattern measures are evaluated from +the position of the final row in the match. Therefore, `running` and +`final` semantics are the same. + +(empty-matches-and-unmatched-rows)= + +## Evaluating expressions in empty matches and unmatched rows + +An empty match occurs when the row pattern is successfully matched, but no +pattern variables are assigned. The following pattern produces an empty match +for every row: + +``` +PATTERN(()) +``` + +When evaluating row pattern measures for an empty match: + +- all column references return `null` +- all navigation operations return `null` +- `classifier` function returns `null` +- `match_number` function returns the sequential number of the match +- all aggregate functions are evaluated over an empty set of rows + +Like every match, an empty match has its starting row. All input values which +are to be output along with the measures (as explained in +{ref}`rows-per-match`), are the values from the starting row. + +An unmatched row is a row that is neither part of any non-empty match nor the +starting row of an empty match. With the option `ALL ROWS PER MATCH WITH +UNMATCHED ROWS`, a single output row is produced. In that row, all row pattern +measures are `null`. All input values which are to be output along with the +measures (as explained in {ref}`rows-per-match`), are the values from the +unmatched row. Using the `match_number` function as a measure can help +differentiate between an empty match and unmatched row. diff --git a/430/_sources/sql/merge.md.txt b/430/_sources/sql/merge.md.txt new file mode 100644 index 000000000..e7b652577 --- /dev/null +++ b/430/_sources/sql/merge.md.txt @@ -0,0 +1,99 @@ +# MERGE + +## Synopsis + +```text +MERGE INTO target_table [ [ AS ] target_alias ] +USING { source_table | query } [ [ AS ] source_alias ] +ON search_condition +when_clause [...] +``` + +where `when_clause` is one of + +```text +WHEN MATCHED [ AND condition ] + THEN DELETE +``` + +```text +WHEN MATCHED [ AND condition ] + THEN UPDATE SET ( column = expression [, ...] ) +``` + +```text +WHEN NOT MATCHED [ AND condition ] + THEN INSERT [ column_list ] VALUES (expression, ...) +``` + +## Description + +Conditionally update and/or delete rows of a table and/or insert new +rows into a table. + +`MERGE` supports an arbitrary number of `WHEN` clauses with different +`MATCHED` conditions, executing the `DELETE`, `UPDATE` or `INSERT` +operation in the first `WHEN` clause selected by the `MATCHED` +state and the match condition. + +For each source row, the `WHEN` clauses are processed in order. Only +the first first matching `WHEN` clause is executed and subsequent clauses +are ignored. A `MERGE_TARGET_ROW_MULTIPLE_MATCHES` exception is +raised when a single target table row matches more than one source row. + +If a source row is not matched by any `WHEN` clause and there is no +`WHEN NOT MATCHED` clause, the source row is ignored. + +In `WHEN` clauses with `UPDATE` operations, the column value expressions +can depend on any field of the target or the source. In the `NOT MATCHED` +case, the `INSERT` expressions can depend on any field of the source. + +## Examples + +Delete all customers mentioned in the source table: + +``` +MERGE INTO accounts t USING monthly_accounts_update s + ON t.customer = s.customer + WHEN MATCHED + THEN DELETE +``` + +For matching customer rows, increment the purchases, and if there is no +match, insert the row from the source table: + +``` +MERGE INTO accounts t USING monthly_accounts_update s + ON (t.customer = s.customer) + WHEN MATCHED + THEN UPDATE SET purchases = s.purchases + t.purchases + WHEN NOT MATCHED + THEN INSERT (customer, purchases, address) + VALUES(s.customer, s.purchases, s.address) +``` + +`MERGE` into the target table from the source table, deleting any matching +target row for which the source address is Centreville. For all other +matching rows, add the source purchases and set the address to the source +address, if there is no match in the target table, insert the source +table row: + +``` +MERGE INTO accounts t USING monthly_accounts_update s + ON (t.customer = s.customer) + WHEN MATCHED AND s.address = 'Centreville' + THEN DELETE + WHEN MATCHED + THEN UPDATE + SET purchases = s.purchases + t.purchases, address = s.address + WHEN NOT MATCHED + THEN INSERT (customer, purchases, address) + VALUES(s.customer, s.purchases, s.address) +``` + +## Limitations + +Any connector can be used as a source table for a `MERGE` statement. +Only connectors which support the `MERGE` statement can be the target of a +merge operation. See the {doc}`connector documentation ` for more +information. diff --git a/430/_sources/sql/pattern-recognition-in-window.md.txt b/430/_sources/sql/pattern-recognition-in-window.md.txt new file mode 100644 index 000000000..d1e533ab7 --- /dev/null +++ b/430/_sources/sql/pattern-recognition-in-window.md.txt @@ -0,0 +1,259 @@ +# Row pattern recognition in window structures + +A window structure can be defined in the `WINDOW` clause or in the `OVER` +clause of a window operation. In both cases, the window specification can +include row pattern recognition clauses. They are part of the window frame. The +syntax and semantics of row pattern recognition in window are similar to those +of the {doc}`MATCH_RECOGNIZE` clause. + +This section explains the details of row pattern recognition in window +structures, and highlights the similarities and the differences between both +pattern recognition mechanisms. + +## Window with row pattern recognition + +**Window specification:** + +```text +( +[ existing_window_name ] +[ PARTITION BY column [, ...] ] +[ ORDER BY column [, ...] ] +[ window_frame ] +) +``` + +**Window frame:** + +```text +[ MEASURES measure_definition [, ...] ] +frame_extent +[ AFTER MATCH skip_to ] +[ INITIAL | SEEK ] +[ PATTERN ( row_pattern ) ] +[ SUBSET subset_definition [, ...] ] +[ DEFINE variable_definition [, ...] ] +``` + +Generally, a window frame specifies the `frame_extent`, which defines the +"sliding window" of rows to be processed by a window function. It can be +defined in terms of `ROWS`, `RANGE` or `GROUPS`. + +A window frame with row pattern recognition involves many other syntactical +components, mandatory or optional, and enforces certain limitations on the +`frame_extent`. + +**Window frame with row pattern recognition:** + +```text +[ MEASURES measure_definition [, ...] ] +ROWS BETWEEN CURRENT ROW AND frame_end +[ AFTER MATCH skip_to ] +[ INITIAL | SEEK ] +PATTERN ( row_pattern ) +[ SUBSET subset_definition [, ...] ] +DEFINE variable_definition [, ...] +``` + +## Description of the pattern recognition clauses + +The `frame_extent` with row pattern recognition must be defined in terms of +`ROWS`. The frame start must be at the `CURRENT ROW`, which limits the +allowed frame extent values to the following: + +``` +ROWS BETWEEN CURRENT ROW AND CURRENT ROW + +ROWS BETWEEN CURRENT ROW AND FOLLOWING + +ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING +``` + +For every input row processed by the window, the portion of rows enclosed by +the `frame_extent` limits the search area for row pattern recognition. Unlike +in `MATCH_RECOGNIZE`, where the pattern search can explore all rows until the +partition end, and all rows of the partition are available for computations, in +window structures the pattern matching can neither match rows nor retrieve +input values outside the frame. + +Besides the `frame_extent`, pattern matching requires the `PATTERN` and +`DEFINE` clauses. + +The `PATTERN` clause specifies a row pattern, which is a form of a regular +expression with some syntactical extensions. The row pattern syntax is similar +to the {ref}`row pattern syntax in MATCH_RECOGNIZE`. +However, the anchor patterns `^` and `$` are not allowed in a window +specification. + +The `DEFINE` clause defines the row pattern primary variables in terms of +boolean conditions that must be satisfied. It is similar to the +{ref}`DEFINE clause of MATCH_RECOGNIZE`. +The only difference is that the window syntax does not support the +`MATCH_NUMBER` function. + +The `MEASURES` clause is syntactically similar to the +{ref}`MEASURES clause of MATCH_RECOGNIZE`. The only +limitation is that the `MATCH_NUMBER` function is not allowed. However, the +semantics of this clause differs between `MATCH_RECOGNIZE` and window. +While in `MATCH_RECOGNIZE` every measure produces an output column, the +measures in window should be considered as **definitions** associated with the +window structure. They can be called over the window, in the same manner as +regular window functions: + +``` +SELECT cust_key, value OVER w, label OVER w + FROM orders + WINDOW w AS ( + PARTITION BY cust_key + ORDER BY order_date + MEASURES + RUNNING LAST(total_price) AS value, + CLASSIFIER() AS label + ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING + PATTERN (A B+ C+) + DEFINE + B AS B.value < PREV (B.value), + C AS C.value > PREV (C.value) + ) +``` + +Measures defined in a window can be referenced in the `SELECT` clause and in +the `ORDER BY` clause of the enclosing query. + +The `RUNNING` and `FINAL` keywords are allowed in the `MEASURES` clause. +They can precede a logical navigation function `FIRST` or `LAST`, or an +aggregate function. However, they have no effect. Every computation is +performed from the position of the final row of the match, so the semantics is +effectively `FINAL`. + +The `AFTER MATCH SKIP` clause has the same syntax as the +{ref}`AFTER MATCH SKIP clause of MATCH_RECOGNIZE`. + +The `INITIAL` or `SEEK` modifier is specific to row pattern recognition in +window. With `INITIAL`, which is the default, the pattern match for an input +row can only be found starting from that row. With `SEEK`, if there is no +match starting from the current row, the engine tries to find a match starting +from subsequent rows within the frame. As a result, it is possible to associate +an input row with a match which is detached from that row. + +The `SUBSET` clause is used to define {ref}`union variables +` as sets of primary pattern variables. You can +use union variables to refer to a set of rows matched to any primary pattern +variable from the subset: + +``` +SUBSET U = (A, B) +``` + +The following expression returns the `total_price` value from the last row +matched to either `A` or `B`: + +``` +LAST(U.total_price) +``` + +If you want to refer to all rows of the match, there is no need to define a +`SUBSET` containing all pattern variables. There is an implicit *universal +pattern variable* applied to any non prefixed column name and any +`CLASSIFIER` call without an argument. The following expression returns the +`total_price` value from the last matched row: + +``` +LAST(total_price) +``` + +The following call returns the primary pattern variable of the first matched +row: + +``` +FIRST(CLASSIFIER()) +``` + +In window, unlike in `MATCH_RECOGNIZE`, you cannot specify `ONE ROW PER +MATCH` or `ALL ROWS PER MATCH`. This is because all calls over window, +whether they are regular window functions or measures, must comply with the +window semantics. A call over window is supposed to produce exactly one output +row for every input row. And so, the output mode of pattern recognition in +window is a combination of `ONE ROW PER MATCH` and `WITH UNMATCHED ROWS`. + +## Processing input with row pattern recognition + +Pattern recognition in window processes input rows in two different cases: + +- upon a row pattern measure call over the window: + + ``` + some_measure OVER w + ``` + +- upon a window function call over the window: + + ``` + sum(total_price) OVER w + ``` + +The output row produced for each input row, consists of: + +- all values from the input row +- the value of the called measure or window function, computed with respect to + the pattern match associated with the row + +Processing the input can be described as the following sequence of steps: + +- Partition the input data accordingly to `PARTITION BY` +- Order each partition by the `ORDER BY` expressions +- For every row of the ordered partition: + : If the row is 'skipped' by a match of some previous row: + : - For a measure, produce a one-row output as for an unmatched row + - For a window function, evaluate the function over an empty frame + and produce a one-row output + + Otherwise: + : - Determine the frame extent + - Try match the row pattern starting from the current row within + the frame extent + - If no match is found, and `SEEK` is specified, try to find a match + starting from subsequent rows within the frame extent + + If no match is found: + : - For a measure, produce a one-row output for an unmatched row + - For a window function, evaluate the function over an empty + frame and produce a one-row output + + Otherwise: + : - For a measure, produce a one-row output for the match + - For a window function, evaluate the function over a frame + limited to the matched rows sequence and produce a one-row + output + - Evaluate the `AFTER MATCH SKIP` clause, and mark the 'skipped' + rows + +## Empty matches and unmatched rows + +If no match can be associated with a particular input row, the row is +*unmatched*. This happens when no match can be found for the row. This also +happens when no match is attempted for the row, because it is skipped by the +`AFTER MATCH SKIP` clause of some preceding row. For an unmatched row, +every row pattern measure is `null`. Every window function is evaluated over +an empty frame. + +An *empty match* is a successful match which does not involve any pattern +variables. In other words, an empty match does not contain any rows. If an +empty match is associated with an input row, every row pattern measure for that +row is evaluated over an empty sequence of rows. All navigation operations and +the `CLASSIFIER` function return `null`. Every window function is evaluated +over an empty frame. + +In most cases, the results for empty matches and unmatched rows are the same. +A constant measure can be helpful to distinguish between them: + +The following call returns `'matched'` for every matched row, including empty +matches, and `null` for every unmatched row: + +``` +matched OVER ( + ... + MEASURES 'matched' AS matched + ... + ) +``` diff --git a/430/_sources/sql/prepare.md.txt b/430/_sources/sql/prepare.md.txt new file mode 100644 index 000000000..d6bb0aef2 --- /dev/null +++ b/430/_sources/sql/prepare.md.txt @@ -0,0 +1,42 @@ +# PREPARE + +## Synopsis + +```text +PREPARE statement_name FROM statement +``` + +## Description + +Prepares a statement for execution at a later time. Prepared statements are +queries that are saved in a session with a given name. The statement can +include parameters in place of literals to be replaced at execution time. +Parameters are represented by question marks. + +## Examples + +Prepare a select query: + +``` +PREPARE my_select1 FROM +SELECT * FROM nation; +``` + +Prepare a select query that includes parameters. The values to compare with +`regionkey` and `nationkey` will be filled in with the {doc}`execute` statement: + +``` +PREPARE my_select2 FROM +SELECT name FROM nation WHERE regionkey = ? AND nationkey < ?; +``` + +Prepare an insert query: + +``` +PREPARE my_insert FROM +INSERT INTO cities VALUES (1, 'San Francisco'); +``` + +## See also + +{doc}`execute`, {doc}`deallocate-prepare`, {doc}`execute-immediate`, {doc}`describe-input`, {doc}`describe-output` diff --git a/430/_sources/sql/refresh-materialized-view.md.txt b/430/_sources/sql/refresh-materialized-view.md.txt new file mode 100644 index 000000000..61a4d74cf --- /dev/null +++ b/430/_sources/sql/refresh-materialized-view.md.txt @@ -0,0 +1,29 @@ +# REFRESH MATERIALIZED VIEW + +## Synopsis + +```text +REFRESH MATERIALIZED VIEW view_name +``` + +## Description + +Initially populate or refresh the data stored in the materialized view +`view_name`. The materialized view must be defined with +{doc}`create-materialized-view`. Data is retrieved from the underlying tables +accessed by the defined query. + +The initial population of the materialized view is typically processing +intensive since it reads the data from the source tables and performs physical +write operations. + +The refresh operation can be less intensive, if the underlying data has not +changed and the connector has implemented a mechanism to be aware of that. The +specific implementation and performance varies by connector used to create the +materialized view. + +## See also + +- {doc}`create-materialized-view` +- {doc}`drop-materialized-view` +- {doc}`show-create-materialized-view` diff --git a/430/_sources/sql/reset-session-authorization.rst.txt b/430/_sources/sql/reset-session-authorization.rst.txt new file mode 100644 index 000000000..b1b163a5c --- /dev/null +++ b/430/_sources/sql/reset-session-authorization.rst.txt @@ -0,0 +1,22 @@ +=========================== +RESET SESSION AUTHORIZATION +=========================== + +Synopsis +-------- + +.. code-block:: none + + RESET SESSION AUTHORIZATION + +Description +----------- + +Resets the current authorization user back to the original user. +The original user is usually the authenticated user (principal), +or it can be the session user when the session user is provided by the client. + +See Also +-------- + +:doc:`set-session-authorization` diff --git a/430/_sources/sql/reset-session.md.txt b/430/_sources/sql/reset-session.md.txt new file mode 100644 index 000000000..431b78dda --- /dev/null +++ b/430/_sources/sql/reset-session.md.txt @@ -0,0 +1,24 @@ +# RESET SESSION + +## Synopsis + +```text +RESET SESSION name +RESET SESSION catalog.name +``` + +## Description + +Reset a {ref}`session property ` value to the +default value. + +## Examples + +```sql +RESET SESSION optimize_hash_generation; +RESET SESSION hive.optimized_reader_enabled; +``` + +## See also + +{doc}`set-session`, {doc}`show-session` diff --git a/430/_sources/sql/revoke-roles.md.txt b/430/_sources/sql/revoke-roles.md.txt new file mode 100644 index 000000000..f7f6ecb4f --- /dev/null +++ b/430/_sources/sql/revoke-roles.md.txt @@ -0,0 +1,52 @@ +# REVOKE ROLES + +## Synopsis + +```text +REVOKE +[ ADMIN OPTION FOR ] +role [, ...] +FROM ( user | USER user | ROLE role) [, ...] +[ GRANTED BY ( user | USER user | ROLE role | CURRENT_USER | CURRENT_ROLE ) ] +[ IN catalog ] +``` + +## Description + +Revokes the specified role(s) from the specified principal(s). + +If the `ADMIN OPTION FOR` clause is specified, the `GRANT` permission is +revoked instead of the role. + +For the `REVOKE` statement for roles to succeed, the user executing it either should +be the role admin or should possess the `GRANT` option for the given role. + +The optional `GRANTED BY` clause causes the role(s) to be revoked with +the specified principal as a revoker. If the `GRANTED BY` clause is not +specified, the roles are revoked by the current user as a revoker. + +The optional `IN catalog` clause revokes the roles in a catalog as opposed +to a system roles. + +## Examples + +Revoke role `bar` from user `foo` + +``` +REVOKE bar FROM USER foo; +``` + +Revoke admin option for roles `bar` and `foo` from user `baz` and role `qux` + +``` +REVOKE ADMIN OPTION FOR bar, foo FROM USER baz, ROLE qux; +``` + +## Limitations + +Some connectors do not support role management. +See connector documentation for more details. + +## See also + +{doc}`create-role`, {doc}`drop-role`, {doc}`set-role`, {doc}`grant-roles` diff --git a/430/_sources/sql/revoke.md.txt b/430/_sources/sql/revoke.md.txt new file mode 100644 index 000000000..5621cba98 --- /dev/null +++ b/430/_sources/sql/revoke.md.txt @@ -0,0 +1,62 @@ +# REVOKE + +## Synopsis + +```text +REVOKE [ GRANT OPTION FOR ] +( privilege [, ...] | ALL PRIVILEGES ) +ON ( table_name | TABLE table_name | SCHEMA schema_name ) +FROM ( user | USER user | ROLE role ) +``` + +## Description + +Revokes the specified privileges from the specified grantee. + +Specifying `ALL PRIVILEGES` revokes {doc}`delete`, {doc}`insert` and {doc}`select` privileges. + +Specifying `ROLE PUBLIC` revokes privileges from the `PUBLIC` role. Users will retain privileges assigned to them directly or via other roles. + +If the optional `GRANT OPTION FOR` clause is specified, only the `GRANT OPTION` +is removed. Otherwise, both the `GRANT` and `GRANT OPTION` are revoked. + +For `REVOKE` statement to succeed, the user executing it should possess the specified privileges as well as the `GRANT OPTION` for those privileges. + +Revoke on a table revokes the specified privilege on all columns of the table. + +Revoke on a schema revokes the specified privilege on all columns of all tables of the schema. + +## Examples + +Revoke `INSERT` and `SELECT` privileges on the table `orders` from user `alice`: + +``` +REVOKE INSERT, SELECT ON orders FROM alice; +``` + +Revoke `DELETE` privilege on the schema `finance` from user `bob`: + +``` +REVOKE DELETE ON SCHEMA finance FROM bob; +``` + +Revoke `SELECT` privilege on the table `nation` from everyone, additionally revoking the privilege to grant `SELECT` privilege: + +``` +REVOKE GRANT OPTION FOR SELECT ON nation FROM ROLE PUBLIC; +``` + +Revoke all privileges on the table `test` from user `alice`: + +``` +REVOKE ALL PRIVILEGES ON test FROM alice; +``` + +## Limitations + +Some connectors have no support for `REVOKE`. +See connector documentation for more details. + +## See also + +{doc}`deny`, {doc}`grant`, {doc}`show-grants` diff --git a/430/_sources/sql/rollback.md.txt b/430/_sources/sql/rollback.md.txt new file mode 100644 index 000000000..abd53cae3 --- /dev/null +++ b/430/_sources/sql/rollback.md.txt @@ -0,0 +1,22 @@ +# ROLLBACK + +## Synopsis + +```text +ROLLBACK [ WORK ] +``` + +## Description + +Rollback the current transaction. + +## Examples + +```sql +ROLLBACK; +ROLLBACK WORK; +``` + +## See also + +{doc}`commit`, {doc}`start-transaction` diff --git a/430/_sources/sql/select.md.txt b/430/_sources/sql/select.md.txt new file mode 100644 index 000000000..c3d9fc092 --- /dev/null +++ b/430/_sources/sql/select.md.txt @@ -0,0 +1,1418 @@ +# SELECT + +## Synopsis + +```text +[ WITH [ RECURSIVE ] with_query [, ...] ] +SELECT [ ALL | DISTINCT ] select_expression [, ...] +[ FROM from_item [, ...] ] +[ WHERE condition ] +[ GROUP BY [ ALL | DISTINCT ] grouping_element [, ...] ] +[ HAVING condition] +[ WINDOW window_definition_list] +[ { UNION | INTERSECT | EXCEPT } [ ALL | DISTINCT ] select ] +[ ORDER BY expression [ ASC | DESC ] [, ...] ] +[ OFFSET count [ ROW | ROWS ] ] +[ LIMIT { count | ALL } ] +[ FETCH { FIRST | NEXT } [ count ] { ROW | ROWS } { ONLY | WITH TIES } ] +``` + +where `from_item` is one of + +```text +table_name [ [ AS ] alias [ ( column_alias [, ...] ) ] ] +``` + +```text +from_item join_type from_item + [ ON join_condition | USING ( join_column [, ...] ) ] +``` + +```text +table_name [ [ AS ] alias [ ( column_alias [, ...] ) ] ] + MATCH_RECOGNIZE pattern_recognition_specification + [ [ AS ] alias [ ( column_alias [, ...] ) ] ] +``` + +For detailed description of `MATCH_RECOGNIZE` clause, see {doc}`pattern +recognition in FROM clause`. + +```text +TABLE (table_function_invocation) [ [ AS ] alias [ ( column_alias [, ...] ) ] ] +``` + +For description of table functions usage, see {doc}`table functions`. + +and `join_type` is one of + +```text +[ INNER ] JOIN +LEFT [ OUTER ] JOIN +RIGHT [ OUTER ] JOIN +FULL [ OUTER ] JOIN +CROSS JOIN +``` + +and `grouping_element` is one of + +```text +() +expression +GROUPING SETS ( ( column [, ...] ) [, ...] ) +CUBE ( column [, ...] ) +ROLLUP ( column [, ...] ) +``` + +## Description + +Retrieve rows from zero or more tables. + +## WITH clause + +The `WITH` clause defines named relations for use within a query. +It allows flattening nested queries or simplifying subqueries. +For example, the following queries are equivalent: + +``` +SELECT a, b +FROM ( + SELECT a, MAX(b) AS b FROM t GROUP BY a +) AS x; + +WITH x AS (SELECT a, MAX(b) AS b FROM t GROUP BY a) +SELECT a, b FROM x; +``` + +This also works with multiple subqueries: + +``` +WITH + t1 AS (SELECT a, MAX(b) AS b FROM x GROUP BY a), + t2 AS (SELECT a, AVG(d) AS d FROM y GROUP BY a) +SELECT t1.*, t2.* +FROM t1 +JOIN t2 ON t1.a = t2.a; +``` + +Additionally, the relations within a `WITH` clause can chain: + +``` +WITH + x AS (SELECT a FROM t), + y AS (SELECT a AS b FROM x), + z AS (SELECT b AS c FROM y) +SELECT c FROM z; +``` + +:::{warning} +Currently, the SQL for the `WITH` clause will be inlined anywhere the named +relation is used. This means that if the relation is used more than once and the query +is non-deterministic, the results may be different each time. +::: + +## WITH RECURSIVE clause + +The `WITH RECURSIVE` clause is a variant of the `WITH` clause. It defines +a list of queries to process, including recursive processing of suitable +queries. + +:::{warning} +This feature is experimental only. Proceed to use it only if you understand +potential query failures and the impact of the recursion processing on your +workload. +::: + +A recursive `WITH`-query must be shaped as a `UNION` of two relations. The +first relation is called the *recursion base*, and the second relation is called +the *recursion step*. Trino supports recursive `WITH`-queries with a single +recursive reference to a `WITH`-query from within the query. The name `T` of +the query `T` can be mentioned once in the `FROM` clause of the recursion +step relation. + +The following listing shows a simple example, that displays a commonly used +form of a single query in the list: + +```text +WITH RECURSIVE t(n) AS ( + VALUES (1) + UNION ALL + SELECT n + 1 FROM t WHERE n < 4 +) +SELECT sum(n) FROM t; +``` + +In the preceding query the simple assignment `VALUES (1)` defines the +recursion base relation. `SELECT n + 1 FROM t WHERE n < 4` defines the +recursion step relation. The recursion processing performs these steps: + +- recursive base yields `1` +- first recursion yields `1 + 1 = 2` +- second recursion uses the result from the first and adds one: `2 + 1 = 3` +- third recursion uses the result from the second and adds one again: + `3 + 1 = 4` +- fourth recursion aborts since `n = 4` +- this results in `t` having values `1`, `2`, `3` and `4` +- the final statement performs the sum operation of these elements with the + final result value `10` + +The types of the returned columns are those of the base relation. Therefore it +is required that types in the step relation can be coerced to base relation +types. + +The `RECURSIVE` clause applies to all queries in the `WITH` list, but not +all of them must be recursive. If a `WITH`-query is not shaped according to +the rules mentioned above or it does not contain a recursive reference, it is +processed like a regular `WITH`-query. Column aliases are mandatory for all +the queries in the recursive `WITH` list. + +The following limitations apply as a result of following the SQL standard and +due to implementation choices, in addition to `WITH` clause limitations: + +- only single-element recursive cycles are supported. Like in regular + `WITH`-queries, references to previous queries in the `WITH` list are + allowed. References to following queries are forbidden. +- usage of outer joins, set operations, limit clause, and others is not always + allowed in the step relation +- recursion depth is fixed, defaults to `10`, and doesn't depend on the actual + query results + +You can adjust the recursion depth with the {doc}`session property +` `max_recursion_depth`. When changing the value consider +that the size of the query plan growth is quadratic with the recursion depth. + +## SELECT clause + +The `SELECT` clause specifies the output of the query. Each `select_expression` +defines a column or columns to be included in the result. + +```text +SELECT [ ALL | DISTINCT ] select_expression [, ...] +``` + +The `ALL` and `DISTINCT` quantifiers determine whether duplicate rows +are included in the result set. If the argument `ALL` is specified, +all rows are included. If the argument `DISTINCT` is specified, only unique +rows are included in the result set. In this case, each output column must +be of a type that allows comparison. If neither argument is specified, +the behavior defaults to `ALL`. + +### Select expressions + +Each `select_expression` must be in one of the following forms: + +```text +expression [ [ AS ] column_alias ] +``` + +```text +row_expression.* [ AS ( column_alias [, ...] ) ] +``` + +```text +relation.* +``` + +```text +* +``` + +In the case of `expression [ [ AS ] column_alias ]`, a single output column +is defined. + +In the case of `row_expression.* [ AS ( column_alias [, ...] ) ]`, +the `row_expression` is an arbitrary expression of type `ROW`. +All fields of the row define output columns to be included in the result set. + +In the case of `relation.*`, all columns of `relation` are included +in the result set. In this case column aliases are not allowed. + +In the case of `*`, all columns of the relation defined by the query +are included in the result set. + +In the result set, the order of columns is the same as the order of their +specification by the select expressions. If a select expression returns multiple +columns, they are ordered the same way they were ordered in the source +relation or row type expression. + +If column aliases are specified, they override any preexisting column +or row field names: + +``` +SELECT (CAST(ROW(1, true) AS ROW(field1 bigint, field2 boolean))).* AS (alias1, alias2); +``` + +```text + alias1 | alias2 +--------+-------- + 1 | true +(1 row) +``` + +Otherwise, the existing names are used: + +``` +SELECT (CAST(ROW(1, true) AS ROW(field1 bigint, field2 boolean))).*; +``` + +```text + field1 | field2 +--------+-------- + 1 | true +(1 row) +``` + +and in their absence, anonymous columns are produced: + +``` +SELECT (ROW(1, true)).*; +``` + +```text + _col0 | _col1 +-------+------- + 1 | true +(1 row) +``` + +## GROUP BY clause + +The `GROUP BY` clause divides the output of a `SELECT` statement into +groups of rows containing matching values. A simple `GROUP BY` clause may +contain any expression composed of input columns or it may be an ordinal +number selecting an output column by position (starting at one). + +The following queries are equivalent. They both group the output by +the `nationkey` input column with the first query using the ordinal +position of the output column and the second query using the input +column name: + +``` +SELECT count(*), nationkey FROM customer GROUP BY 2; + +SELECT count(*), nationkey FROM customer GROUP BY nationkey; +``` + +`GROUP BY` clauses can group output by input column names not appearing in +the output of a select statement. For example, the following query generates +row counts for the `customer` table using the input column `mktsegment`: + +``` +SELECT count(*) FROM customer GROUP BY mktsegment; +``` + +```text + _col0 +------- + 29968 + 30142 + 30189 + 29949 + 29752 +(5 rows) +``` + +When a `GROUP BY` clause is used in a `SELECT` statement all output +expressions must be either aggregate functions or columns present in +the `GROUP BY` clause. + +(complex-grouping-operations)= + +### Complex grouping operations + +Trino also supports complex aggregations using the `GROUPING SETS`, `CUBE` +and `ROLLUP` syntax. This syntax allows users to perform analysis that requires +aggregation on multiple sets of columns in a single query. Complex grouping +operations do not support grouping on expressions composed of input columns. +Only column names are allowed. + +Complex grouping operations are often equivalent to a `UNION ALL` of simple +`GROUP BY` expressions, as shown in the following examples. This equivalence +does not apply, however, when the source of data for the aggregation +is non-deterministic. + +### GROUPING SETS + +Grouping sets allow users to specify multiple lists of columns to group on. +The columns not part of a given sublist of grouping columns are set to `NULL`. + +``` +SELECT * FROM shipping; +``` + +```text + origin_state | origin_zip | destination_state | destination_zip | package_weight +--------------+------------+-------------------+-----------------+---------------- + California | 94131 | New Jersey | 8648 | 13 + California | 94131 | New Jersey | 8540 | 42 + New Jersey | 7081 | Connecticut | 6708 | 225 + California | 90210 | Connecticut | 6927 | 1337 + California | 94131 | Colorado | 80302 | 5 + New York | 10002 | New Jersey | 8540 | 3 +(6 rows) +``` + +`GROUPING SETS` semantics are demonstrated by this example query: + +``` +SELECT origin_state, origin_zip, destination_state, sum(package_weight) +FROM shipping +GROUP BY GROUPING SETS ( + (origin_state), + (origin_state, origin_zip), + (destination_state)); +``` + +```text + origin_state | origin_zip | destination_state | _col0 +--------------+------------+-------------------+------- + New Jersey | NULL | NULL | 225 + California | NULL | NULL | 1397 + New York | NULL | NULL | 3 + California | 90210 | NULL | 1337 + California | 94131 | NULL | 60 + New Jersey | 7081 | NULL | 225 + New York | 10002 | NULL | 3 + NULL | NULL | Colorado | 5 + NULL | NULL | New Jersey | 58 + NULL | NULL | Connecticut | 1562 +(10 rows) +``` + +The preceding query may be considered logically equivalent to a `UNION ALL` of +multiple `GROUP BY` queries: + +``` +SELECT origin_state, NULL, NULL, sum(package_weight) +FROM shipping GROUP BY origin_state + +UNION ALL + +SELECT origin_state, origin_zip, NULL, sum(package_weight) +FROM shipping GROUP BY origin_state, origin_zip + +UNION ALL + +SELECT NULL, NULL, destination_state, sum(package_weight) +FROM shipping GROUP BY destination_state; +``` + +However, the query with the complex grouping syntax (`GROUPING SETS`, `CUBE` +or `ROLLUP`) will only read from the underlying data source once, while the +query with the `UNION ALL` reads the underlying data three times. This is why +queries with a `UNION ALL` may produce inconsistent results when the data +source is not deterministic. + +### CUBE + +The `CUBE` operator generates all possible grouping sets (i.e. a power set) +for a given set of columns. For example, the query: + +``` +SELECT origin_state, destination_state, sum(package_weight) +FROM shipping +GROUP BY CUBE (origin_state, destination_state); +``` + +is equivalent to: + +``` +SELECT origin_state, destination_state, sum(package_weight) +FROM shipping +GROUP BY GROUPING SETS ( + (origin_state, destination_state), + (origin_state), + (destination_state), + () +); +``` + +```text + origin_state | destination_state | _col0 +--------------+-------------------+------- + California | New Jersey | 55 + California | Colorado | 5 + New York | New Jersey | 3 + New Jersey | Connecticut | 225 + California | Connecticut | 1337 + California | NULL | 1397 + New York | NULL | 3 + New Jersey | NULL | 225 + NULL | New Jersey | 58 + NULL | Connecticut | 1562 + NULL | Colorado | 5 + NULL | NULL | 1625 +(12 rows) +``` + +### ROLLUP + +The `ROLLUP` operator generates all possible subtotals for a given set of +columns. For example, the query: + +``` +SELECT origin_state, origin_zip, sum(package_weight) +FROM shipping +GROUP BY ROLLUP (origin_state, origin_zip); +``` + +```text + origin_state | origin_zip | _col2 +--------------+------------+------- + California | 94131 | 60 + California | 90210 | 1337 + New Jersey | 7081 | 225 + New York | 10002 | 3 + California | NULL | 1397 + New York | NULL | 3 + New Jersey | NULL | 225 + NULL | NULL | 1625 +(8 rows) +``` + +is equivalent to: + +``` +SELECT origin_state, origin_zip, sum(package_weight) +FROM shipping +GROUP BY GROUPING SETS ((origin_state, origin_zip), (origin_state), ()); +``` + +### Combining multiple grouping expressions + +Multiple grouping expressions in the same query are interpreted as having +cross-product semantics. For example, the following query: + +``` +SELECT origin_state, destination_state, origin_zip, sum(package_weight) +FROM shipping +GROUP BY + GROUPING SETS ((origin_state, destination_state)), + ROLLUP (origin_zip); +``` + +which can be rewritten as: + +``` +SELECT origin_state, destination_state, origin_zip, sum(package_weight) +FROM shipping +GROUP BY + GROUPING SETS ((origin_state, destination_state)), + GROUPING SETS ((origin_zip), ()); +``` + +is logically equivalent to: + +``` +SELECT origin_state, destination_state, origin_zip, sum(package_weight) +FROM shipping +GROUP BY GROUPING SETS ( + (origin_state, destination_state, origin_zip), + (origin_state, destination_state) +); +``` + +```text + origin_state | destination_state | origin_zip | _col3 +--------------+-------------------+------------+------- + New York | New Jersey | 10002 | 3 + California | New Jersey | 94131 | 55 + New Jersey | Connecticut | 7081 | 225 + California | Connecticut | 90210 | 1337 + California | Colorado | 94131 | 5 + New York | New Jersey | NULL | 3 + New Jersey | Connecticut | NULL | 225 + California | Colorado | NULL | 5 + California | Connecticut | NULL | 1337 + California | New Jersey | NULL | 55 +(10 rows) +``` + +The `ALL` and `DISTINCT` quantifiers determine whether duplicate grouping +sets each produce distinct output rows. This is particularly useful when +multiple complex grouping sets are combined in the same query. For example, the +following query: + +``` +SELECT origin_state, destination_state, origin_zip, sum(package_weight) +FROM shipping +GROUP BY ALL + CUBE (origin_state, destination_state), + ROLLUP (origin_state, origin_zip); +``` + +is equivalent to: + +``` +SELECT origin_state, destination_state, origin_zip, sum(package_weight) +FROM shipping +GROUP BY GROUPING SETS ( + (origin_state, destination_state, origin_zip), + (origin_state, origin_zip), + (origin_state, destination_state, origin_zip), + (origin_state, origin_zip), + (origin_state, destination_state), + (origin_state), + (origin_state, destination_state), + (origin_state), + (origin_state, destination_state), + (origin_state), + (destination_state), + () +); +``` + +However, if the query uses the `DISTINCT` quantifier for the `GROUP BY`: + +``` +SELECT origin_state, destination_state, origin_zip, sum(package_weight) +FROM shipping +GROUP BY DISTINCT + CUBE (origin_state, destination_state), + ROLLUP (origin_state, origin_zip); +``` + +only unique grouping sets are generated: + +``` +SELECT origin_state, destination_state, origin_zip, sum(package_weight) +FROM shipping +GROUP BY GROUPING SETS ( + (origin_state, destination_state, origin_zip), + (origin_state, origin_zip), + (origin_state, destination_state), + (origin_state), + (destination_state), + () +); +``` + +The default set quantifier is `ALL`. + +### GROUPING operation + +`grouping(col1, ..., colN) -> bigint` + +The grouping operation returns a bit set converted to decimal, indicating which columns are present in a +grouping. It must be used in conjunction with `GROUPING SETS`, `ROLLUP`, `CUBE` or `GROUP BY` +and its arguments must match exactly the columns referenced in the corresponding `GROUPING SETS`, +`ROLLUP`, `CUBE` or `GROUP BY` clause. + +To compute the resulting bit set for a particular row, bits are assigned to the argument columns with +the rightmost column being the least significant bit. For a given grouping, a bit is set to 0 if the +corresponding column is included in the grouping and to 1 otherwise. For example, consider the query +below: + +``` +SELECT origin_state, origin_zip, destination_state, sum(package_weight), + grouping(origin_state, origin_zip, destination_state) +FROM shipping +GROUP BY GROUPING SETS ( + (origin_state), + (origin_state, origin_zip), + (destination_state) +); +``` + +```text +origin_state | origin_zip | destination_state | _col3 | _col4 +--------------+------------+-------------------+-------+------- +California | NULL | NULL | 1397 | 3 +New Jersey | NULL | NULL | 225 | 3 +New York | NULL | NULL | 3 | 3 +California | 94131 | NULL | 60 | 1 +New Jersey | 7081 | NULL | 225 | 1 +California | 90210 | NULL | 1337 | 1 +New York | 10002 | NULL | 3 | 1 +NULL | NULL | New Jersey | 58 | 6 +NULL | NULL | Connecticut | 1562 | 6 +NULL | NULL | Colorado | 5 | 6 +(10 rows) +``` + +The first grouping in the above result only includes the `origin_state` column and excludes +the `origin_zip` and `destination_state` columns. The bit set constructed for that grouping +is `011` where the most significant bit represents `origin_state`. + +## HAVING clause + +The `HAVING` clause is used in conjunction with aggregate functions and +the `GROUP BY` clause to control which groups are selected. A `HAVING` +clause eliminates groups that do not satisfy the given conditions. +`HAVING` filters groups after groups and aggregates are computed. + +The following example queries the `customer` table and selects groups +with an account balance greater than the specified value: + +``` +SELECT count(*), mktsegment, nationkey, + CAST(sum(acctbal) AS bigint) AS totalbal +FROM customer +GROUP BY mktsegment, nationkey +HAVING sum(acctbal) > 5700000 +ORDER BY totalbal DESC; +``` + +```text + _col0 | mktsegment | nationkey | totalbal +-------+------------+-----------+---------- + 1272 | AUTOMOBILE | 19 | 5856939 + 1253 | FURNITURE | 14 | 5794887 + 1248 | FURNITURE | 9 | 5784628 + 1243 | FURNITURE | 12 | 5757371 + 1231 | HOUSEHOLD | 3 | 5753216 + 1251 | MACHINERY | 2 | 5719140 + 1247 | FURNITURE | 8 | 5701952 +(7 rows) +``` + +(window-clause)= + +## WINDOW clause + +The `WINDOW` clause is used to define named window specifications. The defined named +window specifications can be referred to in the `SELECT` and `ORDER BY` clauses +of the enclosing query: + +``` +SELECT orderkey, clerk, totalprice, + rank() OVER w AS rnk +FROM orders +WINDOW w AS (PARTITION BY clerk ORDER BY totalprice DESC) +ORDER BY count() OVER w, clerk, rnk +``` + +The window definition list of `WINDOW` clause can contain one or multiple named window +specifications of the form + +```none +window_name AS (window_specification) +``` + +A window specification has the following components: + +- The existing window name, which refers to a named window specification in the + `WINDOW` clause. The window specification associated with the referenced name + is the basis of the current specification. +- The partition specification, which separates the input rows into different + partitions. This is analogous to how the `GROUP BY` clause separates rows + into different groups for aggregate functions. +- The ordering specification, which determines the order in which input rows + will be processed by the window function. +- The window frame, which specifies a sliding window of rows to be processed + by the function for a given row. If the frame is not specified, it defaults + to `RANGE UNBOUNDED PRECEDING`, which is the same as + `RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW`. This frame contains all + rows from the start of the partition up to the last peer of the current row. + In the absence of `ORDER BY`, all rows are considered peers, so `RANGE + BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW` is equivalent to `BETWEEN + UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING`. The window frame syntax + supports additional clauses for row pattern recognition. If the row pattern + recognition clauses are specified, the window frame for a particular row + consists of the rows matched by a pattern starting from that row. + Additionally, if the frame specifies row pattern measures, they can be + called over the window, similarly to window functions. For more details, see + [Row pattern recognition in window structures](/sql/pattern-recognition-in-window) . + +Each window component is optional. If a window specification does not specify +window partitioning, ordering or frame, those components are obtained from +the window specification referenced by the `existing window name`, or from +another window specification in the reference chain. In case when there is no +`existing window name` specified, or none of the referenced window +specifications contains the component, the default value is used. + +## Set operations + +`UNION` `INTERSECT` and `EXCEPT` are all set operations. These clauses are used +to combine the results of more than one select statement into a single result set: + +```text +query UNION [ALL | DISTINCT] query +``` + +```text +query INTERSECT [ALL | DISTINCT] query +``` + +```text +query EXCEPT [ALL | DISTINCT] query +``` + +The argument `ALL` or `DISTINCT` controls which rows are included in +the final result set. If the argument `ALL` is specified all rows are +included even if the rows are identical. If the argument `DISTINCT` +is specified only unique rows are included in the combined result set. +If neither is specified, the behavior defaults to `DISTINCT`. + +Multiple set operations are processed left to right, unless the order is explicitly +specified via parentheses. Additionally, `INTERSECT` binds more tightly +than `EXCEPT` and `UNION`. That means `A UNION B INTERSECT C EXCEPT D` +is the same as `A UNION (B INTERSECT C) EXCEPT D`. + +### UNION clause + +`UNION` combines all the rows that are in the result set from the +first query with those that are in the result set for the second query. +The following is an example of one of the simplest possible `UNION` clauses. +It selects the value `13` and combines this result set with a second query +that selects the value `42`: + +``` +SELECT 13 +UNION +SELECT 42; +``` + +```text + _col0 +------- + 13 + 42 +(2 rows) +``` + +The following query demonstrates the difference between `UNION` and `UNION ALL`. +It selects the value `13` and combines this result set with a second query that +selects the values `42` and `13`: + +``` +SELECT 13 +UNION +SELECT * FROM (VALUES 42, 13); +``` + +```text + _col0 +------- + 13 + 42 +(2 rows) +``` + +``` +SELECT 13 +UNION ALL +SELECT * FROM (VALUES 42, 13); +``` + +```text + _col0 +------- + 13 + 42 + 13 +(2 rows) +``` + +### INTERSECT clause + +`INTERSECT` returns only the rows that are in the result sets of both the first and +the second queries. The following is an example of one of the simplest +possible `INTERSECT` clauses. It selects the values `13` and `42` and combines +this result set with a second query that selects the value `13`. Since `42` +is only in the result set of the first query, it is not included in the final results.: + +``` +SELECT * FROM (VALUES 13, 42) +INTERSECT +SELECT 13; +``` + +```text + _col0 +------- + 13 +(2 rows) +``` + +### EXCEPT clause + +`EXCEPT` returns the rows that are in the result set of the first query, +but not the second. The following is an example of one of the simplest +possible `EXCEPT` clauses. It selects the values `13` and `42` and combines +this result set with a second query that selects the value `13`. Since `13` +is also in the result set of the second query, it is not included in the final result.: + +``` +SELECT * FROM (VALUES 13, 42) +EXCEPT +SELECT 13; +``` + +```text + _col0 +------- + 42 +(2 rows) +``` + +(order-by-clause)= + +## ORDER BY clause + +The `ORDER BY` clause is used to sort a result set by one or more +output expressions: + +```text +ORDER BY expression [ ASC | DESC ] [ NULLS { FIRST | LAST } ] [, ...] +``` + +Each expression may be composed of output columns, or it may be an ordinal +number selecting an output column by position, starting at one. The +`ORDER BY` clause is evaluated after any `GROUP BY` or `HAVING` clause, +and before any `OFFSET`, `LIMIT` or `FETCH FIRST` clause. +The default null ordering is `NULLS LAST`, regardless of the ordering direction. + +Note that, following the SQL specification, an `ORDER BY` clause only +affects the order of rows for queries that immediately contain the clause. +Trino follows that specification, and drops redundant usage of the clause to +avoid negative performance impacts. + +In the following example, the clause only applies to the select statement. + +```SQL +INSERT INTO some_table +SELECT * FROM another_table +ORDER BY field; +``` + +Since tables in SQL are inherently unordered, and the `ORDER BY` clause in +this case does not result in any difference, but negatively impacts performance +of running the overall insert statement, Trino skips the sort operation. + +Another example where the `ORDER BY` clause is redundant, and does not affect +the outcome of the overall statement, is a nested query: + +```SQL +SELECT * +FROM some_table + JOIN (SELECT * FROM another_table ORDER BY field) u + ON some_table.key = u.key; +``` + +More background information and details can be found in +[a blog post about this optimization](https://trino.io/blog/2019/06/03/redundant-order-by.html). + +(offset-clause)= + +## OFFSET clause + +The `OFFSET` clause is used to discard a number of leading rows +from the result set: + +```text +OFFSET count [ ROW | ROWS ] +``` + +If the `ORDER BY` clause is present, the `OFFSET` clause is evaluated +over a sorted result set, and the set remains sorted after the +leading rows are discarded: + +``` +SELECT name FROM nation ORDER BY name OFFSET 22; +``` + +```text + name +---------------- + UNITED KINGDOM + UNITED STATES + VIETNAM +(3 rows) +``` + +Otherwise, it is arbitrary which rows are discarded. +If the count specified in the `OFFSET` clause equals or exceeds the size +of the result set, the final result is empty. + +(limit-clause)= + +## LIMIT or FETCH FIRST clause + +The `LIMIT` or `FETCH FIRST` clause restricts the number of rows +in the result set. + +```text +LIMIT { count | ALL } +``` + +```text +FETCH { FIRST | NEXT } [ count ] { ROW | ROWS } { ONLY | WITH TIES } +``` + +The following example queries a large table, but the `LIMIT` clause +restricts the output to only have five rows (because the query lacks an `ORDER BY`, +exactly which rows are returned is arbitrary): + +``` +SELECT orderdate FROM orders LIMIT 5; +``` + +```text + orderdate +------------ + 1994-07-25 + 1993-11-12 + 1992-10-06 + 1994-01-04 + 1997-12-28 +(5 rows) +``` + +`LIMIT ALL` is the same as omitting the `LIMIT` clause. + +The `FETCH FIRST` clause supports either the `FIRST` or `NEXT` keywords +and the `ROW` or `ROWS` keywords. These keywords are equivalent and +the choice of keyword has no effect on query execution. + +If the count is not specified in the `FETCH FIRST` clause, it defaults to `1`: + +``` +SELECT orderdate FROM orders FETCH FIRST ROW ONLY; +``` + +```text + orderdate +------------ + 1994-02-12 +(1 row) +``` + +If the `OFFSET` clause is present, the `LIMIT` or `FETCH FIRST` clause +is evaluated after the `OFFSET` clause: + +``` +SELECT * FROM (VALUES 5, 2, 4, 1, 3) t(x) ORDER BY x OFFSET 2 LIMIT 2; +``` + +```text + x +--- + 3 + 4 +(2 rows) +``` + +For the `FETCH FIRST` clause, the argument `ONLY` or `WITH TIES` +controls which rows are included in the result set. + +If the argument `ONLY` is specified, the result set is limited to the exact +number of leading rows determined by the count. + +If the argument `WITH TIES` is specified, it is required that the `ORDER BY` +clause be present. The result set consists of the same set of leading rows +and all of the rows in the same peer group as the last of them ('ties') +as established by the ordering in the `ORDER BY` clause. The result set is sorted: + +``` +SELECT name, regionkey +FROM nation +ORDER BY regionkey FETCH FIRST ROW WITH TIES; +``` + +```text + name | regionkey +------------+----------- + ETHIOPIA | 0 + MOROCCO | 0 + KENYA | 0 + ALGERIA | 0 + MOZAMBIQUE | 0 +(5 rows) +``` + +## TABLESAMPLE + +There are multiple sample methods: + +`BERNOULLI` + +: Each row is selected to be in the table sample with a probability of + the sample percentage. When a table is sampled using the Bernoulli + method, all physical blocks of the table are scanned and certain + rows are skipped (based on a comparison between the sample percentage + and a random value calculated at runtime). + + The probability of a row being included in the result is independent + from any other row. This does not reduce the time required to read + the sampled table from disk. It may have an impact on the total + query time if the sampled output is processed further. + +`SYSTEM` + +: This sampling method divides the table into logical segments of data + and samples the table at this granularity. This sampling method either + selects all the rows from a particular segment of data or skips it + (based on a comparison between the sample percentage and a random + value calculated at runtime). + + The rows selected in a system sampling will be dependent on which + connector is used. For example, when used with Hive, it is dependent + on how the data is laid out on HDFS. This method does not guarantee + independent sampling probabilities. + +:::{note} +Neither of the two methods allow deterministic bounds on the number of rows returned. +::: + +Examples: + +``` +SELECT * +FROM users TABLESAMPLE BERNOULLI (50); + +SELECT * +FROM users TABLESAMPLE SYSTEM (75); +``` + +Using sampling with joins: + +``` +SELECT o.*, i.* +FROM orders o TABLESAMPLE SYSTEM (10) +JOIN lineitem i TABLESAMPLE BERNOULLI (40) + ON o.orderkey = i.orderkey; +``` + +(unnest)= + +## UNNEST + +`UNNEST` can be used to expand an {ref}`array-type` or {ref}`map-type` into a relation. +Arrays are expanded into a single column: + +``` +SELECT * FROM UNNEST(ARRAY[1,2]) AS t(number); +``` + +```text + number +-------- + 1 + 2 +(2 rows) +``` + +Maps are expanded into two columns (key, value): + +``` +SELECT * FROM UNNEST( + map_from_entries( + ARRAY[ + ('SQL',1974), + ('Java', 1995) + ] + ) +) AS t(language, first_appeared_year); +``` + +```text + language | first_appeared_year +----------+--------------------- + SQL | 1974 + Java | 1995 +(2 rows) +``` + +`UNNEST` can be used in combination with an `ARRAY` of {ref}`row-type` structures for expanding each +field of the `ROW` into a corresponding column: + +``` +SELECT * +FROM UNNEST( + ARRAY[ + ROW('Java', 1995), + ROW('SQL' , 1974)], + ARRAY[ + ROW(false), + ROW(true)] +) as t(language,first_appeared_year,declarative); +``` + +```text + language | first_appeared_year | declarative +----------+---------------------+------------- + Java | 1995 | false + SQL | 1974 | true +(2 rows) +``` + +`UNNEST` can optionally have a `WITH ORDINALITY` clause, in which case an additional ordinality column +is added to the end: + +``` +SELECT a, b, rownumber +FROM UNNEST ( + ARRAY[2, 5], + ARRAY[7, 8, 9] + ) WITH ORDINALITY AS t(a, b, rownumber); +``` + +```text + a | b | rownumber +------+---+----------- + 2 | 7 | 1 + 5 | 8 | 2 + NULL | 9 | 3 +(3 rows) +``` + +`UNNEST` returns zero entries when the array/map is empty: + +``` +SELECT * FROM UNNEST (ARRAY[]) AS t(value); +``` + +```text + value +------- +(0 rows) +``` + +`UNNEST` returns zero entries when the array/map is null: + +``` +SELECT * FROM UNNEST (CAST(null AS ARRAY(integer))) AS t(number); +``` + +```text + number +-------- +(0 rows) +``` + +`UNNEST` is normally used with a `JOIN`, and can reference columns +from relations on the left side of the join: + +``` +SELECT student, score +FROM ( + VALUES + ('John', ARRAY[7, 10, 9]), + ('Mary', ARRAY[4, 8, 9]) +) AS tests (student, scores) +CROSS JOIN UNNEST(scores) AS t(score); +``` + +```text + student | score +---------+------- + John | 7 + John | 10 + John | 9 + Mary | 4 + Mary | 8 + Mary | 9 +(6 rows) +``` + +`UNNEST` can also be used with multiple arguments, in which case they are expanded into multiple columns, +with as many rows as the highest cardinality argument (the other columns are padded with nulls): + +``` +SELECT numbers, animals, n, a +FROM ( + VALUES + (ARRAY[2, 5], ARRAY['dog', 'cat', 'bird']), + (ARRAY[7, 8, 9], ARRAY['cow', 'pig']) +) AS x (numbers, animals) +CROSS JOIN UNNEST(numbers, animals) AS t (n, a); +``` + +```text + numbers | animals | n | a +-----------+------------------+------+------ + [2, 5] | [dog, cat, bird] | 2 | dog + [2, 5] | [dog, cat, bird] | 5 | cat + [2, 5] | [dog, cat, bird] | NULL | bird + [7, 8, 9] | [cow, pig] | 7 | cow + [7, 8, 9] | [cow, pig] | 8 | pig + [7, 8, 9] | [cow, pig] | 9 | NULL +(6 rows) +``` + +`LEFT JOIN` is preferable in order to avoid losing the the row containing the array/map field in question +when referenced columns from relations on the left side of the join can be empty or have `NULL` values: + +``` +SELECT runner, checkpoint +FROM ( + VALUES + ('Joe', ARRAY[10, 20, 30, 42]), + ('Roger', ARRAY[10]), + ('Dave', ARRAY[]), + ('Levi', NULL) +) AS marathon (runner, checkpoints) +LEFT JOIN UNNEST(checkpoints) AS t(checkpoint) ON TRUE; +``` + +```text + runner | checkpoint +--------+------------ + Joe | 10 + Joe | 20 + Joe | 30 + Joe | 42 + Roger | 10 + Dave | NULL + Levi | NULL +(7 rows) +``` + +Note that in case of using `LEFT JOIN` the only condition supported by the current implementation is `ON TRUE`. + +## Joins + +Joins allow you to combine data from multiple relations. + +### CROSS JOIN + +A cross join returns the Cartesian product (all combinations) of two +relations. Cross joins can either be specified using the explit +`CROSS JOIN` syntax or by specifying multiple relations in the +`FROM` clause. + +Both of the following queries are equivalent: + +``` +SELECT * +FROM nation +CROSS JOIN region; + +SELECT * +FROM nation, region; +``` + +The `nation` table contains 25 rows and the `region` table contains 5 rows, +so a cross join between the two tables produces 125 rows: + +``` +SELECT n.name AS nation, r.name AS region +FROM nation AS n +CROSS JOIN region AS r +ORDER BY 1, 2; +``` + +```text + nation | region +----------------+------------- + ALGERIA | AFRICA + ALGERIA | AMERICA + ALGERIA | ASIA + ALGERIA | EUROPE + ALGERIA | MIDDLE EAST + ARGENTINA | AFRICA + ARGENTINA | AMERICA +... +(125 rows) +``` + +### LATERAL + +Subqueries appearing in the `FROM` clause can be preceded by the keyword `LATERAL`. +This allows them to reference columns provided by preceding `FROM` items. + +A `LATERAL` join can appear at the top level in the `FROM` list, or anywhere +within a parenthesized join tree. In the latter case, it can also refer to any items +that are on the left-hand side of a `JOIN` for which it is on the right-hand side. + +When a `FROM` item contains `LATERAL` cross-references, evaluation proceeds as follows: +for each row of the `FROM` item providing the cross-referenced columns, +the `LATERAL` item is evaluated using that row set's values of the columns. +The resulting rows are joined as usual with the rows they were computed from. +This is repeated for set of rows from the column source tables. + +`LATERAL` is primarily useful when the cross-referenced column is necessary for +computing the rows to be joined: + +``` +SELECT name, x, y +FROM nation +CROSS JOIN LATERAL (SELECT name || ' :-' AS x) +CROSS JOIN LATERAL (SELECT x || ')' AS y); +``` + +### Qualifying column names + +When two relations in a join have columns with the same name, the column +references must be qualified using the relation alias (if the relation +has an alias), or with the relation name: + +``` +SELECT nation.name, region.name +FROM nation +CROSS JOIN region; + +SELECT n.name, r.name +FROM nation AS n +CROSS JOIN region AS r; + +SELECT n.name, r.name +FROM nation n +CROSS JOIN region r; +``` + +The following query will fail with the error `Column 'name' is ambiguous`: + +``` +SELECT name +FROM nation +CROSS JOIN region; +``` + +## Subqueries + +A subquery is an expression which is composed of a query. The subquery +is correlated when it refers to columns outside of the subquery. +Logically, the subquery will be evaluated for each row in the surrounding +query. The referenced columns will thus be constant during any single +evaluation of the subquery. + +:::{note} +Support for correlated subqueries is limited. Not every standard form is supported. +::: + +### EXISTS + +The `EXISTS` predicate determines if a subquery returns any rows: + +``` +SELECT name +FROM nation +WHERE EXISTS ( + SELECT * + FROM region + WHERE region.regionkey = nation.regionkey +); +``` + +### IN + +The `IN` predicate determines if any values produced by the subquery +are equal to the provided expression. The result of `IN` follows the +standard rules for nulls. The subquery must produce exactly one column: + +``` +SELECT name +FROM nation +WHERE regionkey IN ( + SELECT regionkey + FROM region + WHERE name = 'AMERICA' OR name = 'AFRICA' +); +``` + +### Scalar subquery + +A scalar subquery is a non-correlated subquery that returns zero or +one row. It is an error for the subquery to produce more than one +row. The returned value is `NULL` if the subquery produces no rows: + +``` +SELECT name +FROM nation +WHERE regionkey = (SELECT max(regionkey) FROM region); +``` + +:::{note} +Currently only single column can be returned from the scalar subquery. +::: diff --git a/430/_sources/sql/set-path.md.txt b/430/_sources/sql/set-path.md.txt new file mode 100644 index 000000000..cbb9d33ea --- /dev/null +++ b/430/_sources/sql/set-path.md.txt @@ -0,0 +1,49 @@ +# SET PATH + +## Synopsis + +```text +SET PATH path-element[, ...] +``` + +## Description + +Define a collection of paths to functions or table functions in specific +catalogs and schemas for the current session. + +Each path-element uses a period-separated syntax to specify the catalog name and +schema location `.` of the function, or only the schema +location `` in the current catalog. The current catalog is set with +{doc}`use`, or as part of a client tool connection. Catalog and schema must +exist. + +## Examples + +The following example sets a path to access functions in the `system` schema +of the `example` catalog: + +``` +SET PATH example.system; +``` + +The catalog uses the PostgreSQL connector, and you can therefore use the +{ref}`query table function ` directly, without the +full catalog and schema qualifiers: + +``` +SELECT + * +FROM + TABLE( + query( + query => 'SELECT + * + FROM + tpch.nation' + ) + ); +``` + +## See also + +{doc}`use` diff --git a/430/_sources/sql/set-role.md.txt b/430/_sources/sql/set-role.md.txt new file mode 100644 index 000000000..338d75071 --- /dev/null +++ b/430/_sources/sql/set-role.md.txt @@ -0,0 +1,34 @@ +# SET ROLE + +## Synopsis + +```text +SET ROLE ( role | ALL | NONE ) +[ IN catalog ] +``` + +## Description + +`SET ROLE` sets the enabled role for the current session. + +`SET ROLE role` enables a single specified role for the current session. +For the `SET ROLE role` statement to succeed, the user executing it should +have a grant for the given role. + +`SET ROLE ALL` enables all roles that the current user has been granted for the +current session. + +`SET ROLE NONE` disables all the roles granted to the current user for the +current session. + +The optional `IN catalog` clause sets the role in a catalog as opposed +to a system role. + +## Limitations + +Some connectors do not support role management. +See connector documentation for more details. + +## See also + +{doc}`create-role`, {doc}`drop-role`, {doc}`grant-roles`, {doc}`revoke-roles` diff --git a/430/_sources/sql/set-session-authorization.rst.txt b/430/_sources/sql/set-session-authorization.rst.txt new file mode 100644 index 000000000..98634dd11 --- /dev/null +++ b/430/_sources/sql/set-session-authorization.rst.txt @@ -0,0 +1,47 @@ +========================= +SET SESSION AUTHORIZATION +========================= + +Synopsis +-------- + +.. code-block:: none + + SET SESSION AUTHORIZATION username + +Description +----------- + +Changes the current user of the session. +For the ``SET SESSION AUTHORIZATION username`` statement to succeed, +the the original user (that the client connected with) must be able to impersonate the specified user. +User impersonation can be enabled in the system access control. + +Examples +-------- + +In the following example, the original user when the connection to Trino is made is Kevin. +The following sets the session authorization user to John:: + + SET SESSION AUTHORIZATION 'John'; + +Queries will now execute as John instead of Kevin. + +All supported syntax to change the session authorization users are shown below. + +Changing the session authorization with single quotes:: + + SET SESSION AUTHORIZATION 'John'; + +Changing the session authorization with double quotes:: + + SET SESSION AUTHORIZATION "John"; + +Changing the session authorization without quotes:: + + SET SESSION AUTHORIZATION John; + +See Also +-------- + +:doc:`reset-session-authorization` diff --git a/430/_sources/sql/set-session.md.txt b/430/_sources/sql/set-session.md.txt new file mode 100644 index 000000000..396f6d068 --- /dev/null +++ b/430/_sources/sql/set-session.md.txt @@ -0,0 +1,61 @@ +# SET SESSION + +## Synopsis + +```text +SET SESSION name = expression +SET SESSION catalog.name = expression +``` + +## Description + +Set a session property value or a catalog session property. + +(session-properties-definition)= + +## Session properties + +A session property is a {doc}`configuration property ` that +can be temporarily modified by a user for the duration of the current +connection session to the Trino cluster. Many configuration properties have a +corresponding session property that accepts the same values as the config +property. + +There are two types of session properties: + +- **System session properties** apply to the whole cluster. Most session + properties are system session properties unless specified otherwise. +- **Catalog session properties** are connector-defined session properties that + can be set on a per-catalog basis. These properties must be set separately for + each catalog by including the catalog name as a prefix, such as + `catalogname.property_name`. + +Session properties are tied to the current session, so a user can have multiple +connections to a cluster that each have different values for the same session +properties. Once a session ends, either by disconnecting or creating a new +session, any changes made to session properties during the previous session are +lost. + +## Examples + +The following example sets a system session property to enable optimized hash +generation: + +``` +SET SESSION optimize_hash_generation = true; +``` + +The following example sets the `optimize_locality_enabled` catalog session +property for an {doc}`Accumulo catalog ` named `acc01`: + +``` +SET SESSION acc01.optimize_locality_enabled = false; +``` + +The example `acc01.optimize_locality_enabled` catalog session property +does not apply to any other catalog, even if another catalog also uses the +Accumulo connector. + +## See also + +{doc}`reset-session`, {doc}`show-session` diff --git a/430/_sources/sql/set-time-zone.md.txt b/430/_sources/sql/set-time-zone.md.txt new file mode 100644 index 000000000..9921f9bb6 --- /dev/null +++ b/430/_sources/sql/set-time-zone.md.txt @@ -0,0 +1,67 @@ +# SET TIME ZONE + +## Synopsis + +```text +SET TIME ZONE LOCAL +SET TIME ZONE expression +``` + +## Description + +Sets the default time zone for the current session. + +If the `LOCAL` option is specified, the time zone for the current session +is set to the initial time zone of the session. + +If the `expression` option is specified: + +- if the type of the `expression` is a string, the time zone for the current + session is set to the corresponding region-based time zone ID or the + corresponding zone offset. +- if the type of the `expression` is an interval, the time zone for the + current session is set to the corresponding zone offset relative to UTC. + It must be in the range of \[-14,14\] hours. + +## Examples + +Use the default time zone for the current session: + +``` +SET TIME ZONE LOCAL; +``` + +Use a zone offset for specifying the time zone: + +``` +SET TIME ZONE '-08:00'; +``` + +Use an interval literal for specifying the time zone: + +``` +SET TIME ZONE INTERVAL '10' HOUR; +SET TIME ZONE INTERVAL -'08:00' HOUR TO MINUTE; +``` + +Use a region-based time zone identifier for specifying the time zone: + +``` +SET TIME ZONE 'America/Los_Angeles'; +``` + +The time zone identifier to be used can be passed as the output of a +function call: + +``` +SET TIME ZONE concat_ws('/', 'America', 'Los_Angeles'); +``` + +## Limitations + +Setting the default time zone for the session has no effect if +the `sql.forced-session-time-zone` configuration property is already set. + +## See also + +- {func}`current_timezone` diff --git a/430/_sources/sql/show-catalogs.md.txt b/430/_sources/sql/show-catalogs.md.txt new file mode 100644 index 000000000..1f253d794 --- /dev/null +++ b/430/_sources/sql/show-catalogs.md.txt @@ -0,0 +1,19 @@ +# SHOW CATALOGS + +## Synopsis + +```text +SHOW CATALOGS [ LIKE pattern ] +``` + +## Description + +List the available catalogs. + +{ref}`Specify a pattern ` in the optional `LIKE` clause to +filter the results to the desired subset. For example, the following query +allows you to find catalogs that begin with `t`: + +``` +SHOW CATALOGS LIKE 't%' +``` diff --git a/430/_sources/sql/show-columns.md.txt b/430/_sources/sql/show-columns.md.txt new file mode 100644 index 000000000..3a399296a --- /dev/null +++ b/430/_sources/sql/show-columns.md.txt @@ -0,0 +1,39 @@ +# SHOW COLUMNS + +## Synopsis + +```text +SHOW COLUMNS FROM table [ LIKE pattern ] +``` + +## Description + +List the columns in a `table` along with their data type and other attributes: + +``` +SHOW COLUMNS FROM nation; +``` + +```text + Column | Type | Extra | Comment +-----------+--------------+-------+--------- + nationkey | bigint | | + name | varchar(25) | | + regionkey | bigint | | + comment | varchar(152) | | +``` + +{ref}`Specify a pattern ` in the optional `LIKE` clause to +filter the results to the desired subset. For example, the following query +allows you to find columns ending in `key`: + +``` +SHOW COLUMNS FROM nation LIKE '%key'; +``` + +```text + Column | Type | Extra | Comment +-----------+--------------+-------+--------- + nationkey | bigint | | + regionkey | bigint | | +``` diff --git a/430/_sources/sql/show-create-materialized-view.md.txt b/430/_sources/sql/show-create-materialized-view.md.txt new file mode 100644 index 000000000..915a49c83 --- /dev/null +++ b/430/_sources/sql/show-create-materialized-view.md.txt @@ -0,0 +1,18 @@ +# SHOW CREATE MATERIALIZED VIEW + +## Synopsis + +```text +SHOW CREATE MATERIALIZED VIEW view_name +``` + +## Description + +Show the SQL statement that creates the specified materialized view +`view_name`. + +## See also + +- {doc}`create-materialized-view` +- {doc}`drop-materialized-view` +- {doc}`refresh-materialized-view` diff --git a/430/_sources/sql/show-create-schema.md.txt b/430/_sources/sql/show-create-schema.md.txt new file mode 100644 index 000000000..fdf9b01c6 --- /dev/null +++ b/430/_sources/sql/show-create-schema.md.txt @@ -0,0 +1,15 @@ +# SHOW CREATE SCHEMA + +## Synopsis + +```text +SHOW CREATE SCHEMA schema_name +``` + +## Description + +Show the SQL statement that creates the specified schema. + +## See also + +{doc}`create-schema` diff --git a/430/_sources/sql/show-create-table.md.txt b/430/_sources/sql/show-create-table.md.txt new file mode 100644 index 000000000..41d428aec --- /dev/null +++ b/430/_sources/sql/show-create-table.md.txt @@ -0,0 +1,39 @@ +# SHOW CREATE TABLE + +## Synopsis + +```text +SHOW CREATE TABLE table_name +``` + +## Description + +Show the SQL statement that creates the specified table. + +## Examples + +Show the SQL that can be run to create the `orders` table: + +``` +SHOW CREATE TABLE sf1.orders; +``` + +```text + Create Table +----------------------------------------- + CREATE TABLE tpch.sf1.orders ( + orderkey bigint, + orderstatus varchar, + totalprice double, + orderdate varchar + ) + WITH ( + format = 'ORC', + partitioned_by = ARRAY['orderdate'] + ) +(1 row) +``` + +## See also + +{doc}`create-table` diff --git a/430/_sources/sql/show-create-view.md.txt b/430/_sources/sql/show-create-view.md.txt new file mode 100644 index 000000000..dd1377d12 --- /dev/null +++ b/430/_sources/sql/show-create-view.md.txt @@ -0,0 +1,15 @@ +# SHOW CREATE VIEW + +## Synopsis + +```text +SHOW CREATE VIEW view_name +``` + +## Description + +Show the SQL statement that creates the specified view. + +## See also + +{doc}`create-view` diff --git a/430/_sources/sql/show-functions.md.txt b/430/_sources/sql/show-functions.md.txt new file mode 100644 index 000000000..de286d797 --- /dev/null +++ b/430/_sources/sql/show-functions.md.txt @@ -0,0 +1,41 @@ +# SHOW FUNCTIONS + +## Synopsis + +```text +SHOW FUNCTIONS [ FROM schema ] [ LIKE pattern ] +``` + +## Description + +List functions in `schema` or all functions in the current session path. +For each function returned, the following information is displayed: + +- Function name +- Return type +- Argument types +- Function type +- Deterministic +- Description + +{ref}`Specify a pattern ` in the optional `LIKE` clause to +filter the results to the desired subset. For example, the following query +allows you to find functions beginning with `array`: + +``` +SHOW FUNCTIONS LIKE 'array%'; +``` + +`SHOW FUNCTIONS` works with built-in functions as well as with {doc}`custom +functions `. In the following example, three custom +functions beginning with `cf` are available: + +```text +SHOW FUNCTIONS LIKE 'cf%'; + + Function | Return Type | Argument Types | Function Type | Deterministic | Description + ------------------+-------------+----------------+---------------+---------------+----------------------------------------- + cf_getgroups | varchar | | scalar | true | Returns the current session's groups + cf_getprincipal | varchar | | scalar | true | Returns the current session's principal + cf_getuser | varchar | | scalar | true | Returns the current session's user +``` diff --git a/430/_sources/sql/show-grants.md.txt b/430/_sources/sql/show-grants.md.txt new file mode 100644 index 000000000..2ff6f5bf3 --- /dev/null +++ b/430/_sources/sql/show-grants.md.txt @@ -0,0 +1,42 @@ +# SHOW GRANTS + +## Synopsis + +```text +SHOW GRANTS [ ON [ TABLE ] table_name ] +``` + +## Description + +List the grants for the current user on the specified table in the current catalog. + +If no table name is specified, the command lists the grants for the current user on all the tables in all schemas of the current catalog. + +The command requires the current catalog to be set. + +:::{note} +Ensure that authentication has been enabled before running any of the authorization commands. +::: + +## Examples + +List the grants for the current user on table `orders`: + +``` +SHOW GRANTS ON TABLE orders; +``` + +List the grants for the current user on all the tables in all schemas of the current catalog: + +``` +SHOW GRANTS; +``` + +## Limitations + +Some connectors have no support for `SHOW GRANTS`. +See connector documentation for more details. + +## See also + +{doc}`grant`, {doc}`revoke` diff --git a/430/_sources/sql/show-role-grants.md.txt b/430/_sources/sql/show-role-grants.md.txt new file mode 100644 index 000000000..1170ae13b --- /dev/null +++ b/430/_sources/sql/show-role-grants.md.txt @@ -0,0 +1,11 @@ +# SHOW ROLE GRANTS + +## Synopsis + +```text +SHOW ROLE GRANTS [ FROM catalog ] +``` + +## Description + +List non-recursively the system roles or roles in `catalog` that have been granted to the session user. diff --git a/430/_sources/sql/show-roles.md.txt b/430/_sources/sql/show-roles.md.txt new file mode 100644 index 000000000..4476048c4 --- /dev/null +++ b/430/_sources/sql/show-roles.md.txt @@ -0,0 +1,13 @@ +# SHOW ROLES + +## Synopsis + +```text +SHOW [CURRENT] ROLES [ FROM catalog ] +``` + +## Description + +`SHOW ROLES` lists all the system roles or all the roles in `catalog`. + +`SHOW CURRENT ROLES` lists the enabled system roles or roles in `catalog`. diff --git a/430/_sources/sql/show-schemas.md.txt b/430/_sources/sql/show-schemas.md.txt new file mode 100644 index 000000000..0c5160457 --- /dev/null +++ b/430/_sources/sql/show-schemas.md.txt @@ -0,0 +1,19 @@ +# SHOW SCHEMAS + +## Synopsis + +```text +SHOW SCHEMAS [ FROM catalog ] [ LIKE pattern ] +``` + +## Description + +List the schemas in `catalog` or in the current catalog. + +{ref}`Specify a pattern ` in the optional `LIKE` clause to +filter the results to the desired subset. For example, the following query +allows you to find schemas that have `3` as the third character: + +``` +SHOW SCHEMAS FROM tpch LIKE '__3%' +``` diff --git a/430/_sources/sql/show-session.md.txt b/430/_sources/sql/show-session.md.txt new file mode 100644 index 000000000..e19846475 --- /dev/null +++ b/430/_sources/sql/show-session.md.txt @@ -0,0 +1,23 @@ +# SHOW SESSION + +## Synopsis + +```text +SHOW SESSION [ LIKE pattern ] +``` + +## Description + +List the current {ref}`session properties `. + +{ref}`Specify a pattern ` in the optional `LIKE` clause to +filter the results to the desired subset. For example, the following query +allows you to find session properties that begin with `query`: + +``` +SHOW SESSION LIKE 'query%' +``` + +## See also + +{doc}`reset-session`, {doc}`set-session` diff --git a/430/_sources/sql/show-stats.md.txt b/430/_sources/sql/show-stats.md.txt new file mode 100644 index 000000000..2c5f27af0 --- /dev/null +++ b/430/_sources/sql/show-stats.md.txt @@ -0,0 +1,57 @@ +# SHOW STATS + +## Synopsis + +```text +SHOW STATS FOR table +SHOW STATS FOR ( query ) +``` + +## Description + +Returns approximated statistics for the named table or for the results of a +query. Returns `NULL` for any statistics that are not populated or +unavailable on the data source. + +Statistics are returned as a row for each column, plus a summary row for +the table (identifiable by a `NULL` value for `column_name`). The following +table lists the returned columns and what statistics they represent. Any +additional statistics collected on the data source, other than those listed +here, are not included. + +:::{list-table} Statistics +:widths: 20, 40, 40 +:header-rows: 1 + +* - Column + - Description + - Notes +* - `column_name` + - The name of the column + - `NULL` in the table summary row +* - `data_size` + - The total size in bytes of all of the values in the column + - `NULL` in the table summary row. Available for columns of + [string](string-data-types) data types with variable widths. +* - `distinct_values_count` + - The estimated number of distinct values in the column + - `NULL` in the table summary row +* - `nulls_fractions` + - The portion of the values in the column that are `NULL` + - `NULL` in the table summary row. +* - `row_count` + - The estimated number of rows in the table + - `NULL` in column statistic rows +* - `low_value` + - The lowest value found in this column + - `NULL` in the table summary row. Available for columns of + [DATE](date-data-type), [integer](integer-data-types), + [floating-point](floating-point-data-types), and + [fixed-precision](fixed-precision-data-types) data types. +* - `high_value` + - The highest value found in this column + - `NULL` in the table summary row. Available for columns of + [DATE](date-data-type), [integer](integer-data-types), + [floating-point](floating-point-data-types), and + [fixed-precision](fixed-precision-data-types) data types. + ::: diff --git a/430/_sources/sql/show-tables.md.txt b/430/_sources/sql/show-tables.md.txt new file mode 100644 index 000000000..17dce4ad8 --- /dev/null +++ b/430/_sources/sql/show-tables.md.txt @@ -0,0 +1,19 @@ +# SHOW TABLES + +## Synopsis + +```text +SHOW TABLES [ FROM schema ] [ LIKE pattern ] +``` + +## Description + +List the tables in `schema` or in the current schema. + +{ref}`Specify a pattern ` in the optional `LIKE` clause to +filter the results to the desired subset.. For example, the following query +allows you to find tables that begin with `p`: + +``` +SHOW TABLES FROM tpch.tiny LIKE 'p%'; +``` diff --git a/430/_sources/sql/start-transaction.md.txt b/430/_sources/sql/start-transaction.md.txt new file mode 100644 index 000000000..e293dc5c0 --- /dev/null +++ b/430/_sources/sql/start-transaction.md.txt @@ -0,0 +1,32 @@ +# START TRANSACTION + +## Synopsis + +```text +START TRANSACTION [ mode [, ...] ] +``` + +where `mode` is one of + +```text +ISOLATION LEVEL { READ UNCOMMITTED | READ COMMITTED | REPEATABLE READ | SERIALIZABLE } +READ { ONLY | WRITE } +``` + +## Description + +Start a new transaction for the current session. + +## Examples + +```sql +START TRANSACTION; +START TRANSACTION ISOLATION LEVEL REPEATABLE READ; +START TRANSACTION READ WRITE; +START TRANSACTION ISOLATION LEVEL READ COMMITTED, READ ONLY; +START TRANSACTION READ WRITE, ISOLATION LEVEL SERIALIZABLE; +``` + +## See also + +{doc}`commit`, {doc}`rollback` diff --git a/430/_sources/sql/truncate.md.txt b/430/_sources/sql/truncate.md.txt new file mode 100644 index 000000000..e1f634b5c --- /dev/null +++ b/430/_sources/sql/truncate.md.txt @@ -0,0 +1,19 @@ +# TRUNCATE + +## Synopsis + +```none +TRUNCATE TABLE table_name +``` + +## Description + +Delete all rows from a table. + +## Examples + +Truncate the table `orders`: + +``` +TRUNCATE TABLE orders; +``` diff --git a/430/_sources/sql/update.md.txt b/430/_sources/sql/update.md.txt new file mode 100644 index 000000000..0f6a698f1 --- /dev/null +++ b/430/_sources/sql/update.md.txt @@ -0,0 +1,61 @@ +# UPDATE + +## Synopsis + +```text +UPDATE table_name SET [ ( column = expression [, ... ] ) ] [ WHERE condition ] +``` + +## Description + +Update selected columns values in existing rows in a table. + +The columns named in the `column = expression` assignments will be updated +for all rows that match the `WHERE` condition. The values of all column update +expressions for a matching row are evaluated before any column value is changed. +When the type of the expression and the type of the column differ, the usual implicit +CASTs, such as widening numeric fields, are applied to the `UPDATE` expression values. + +## Examples + +Update the status of all purchases that haven't been assigned a ship date: + +``` +UPDATE + purchases +SET + status = 'OVERDUE' +WHERE + ship_date IS NULL; +``` + +Update the account manager and account assign date for all customers: + +``` +UPDATE + customers +SET + account_manager = 'John Henry', + assign_date = now(); +``` + +Update the manager to be the name of the employee who matches the manager ID: + +``` +UPDATE + new_hires +SET + manager = ( + SELECT + e.name + FROM + employees e + WHERE + e.employee_id = new_hires.manager_id + ); +``` + +## Limitations + +Some connectors have limited or no support for `UPDATE`. +See connector documentation for more details. diff --git a/430/_sources/sql/use.md.txt b/430/_sources/sql/use.md.txt new file mode 100644 index 000000000..ae693eeed --- /dev/null +++ b/430/_sources/sql/use.md.txt @@ -0,0 +1,21 @@ +# USE + +## Synopsis + +```text +USE catalog.schema +USE schema +``` + +## Description + +Update the session to use the specified catalog and schema. If a +catalog is not specified, the schema is resolved relative to the +current catalog. + +## Examples + +```sql +USE hive.finance; +USE information_schema; +``` diff --git a/430/_sources/sql/values.md.txt b/430/_sources/sql/values.md.txt new file mode 100644 index 000000000..3b478c073 --- /dev/null +++ b/430/_sources/sql/values.md.txt @@ -0,0 +1,66 @@ +# VALUES + +## Synopsis + +```text +VALUES row [, ...] +``` + +where `row` is a single expression or + +```text +( column_expression [, ...] ) +``` + +## Description + +Defines a literal inline table. + +`VALUES` can be used anywhere a query can be used (e.g., the `FROM` clause +of a {doc}`select`, an {doc}`insert`, or even at the top level). `VALUES` creates +an anonymous table without column names, but the table and columns can be named +using an `AS` clause with column aliases. + +## Examples + +Return a table with one column and three rows: + +``` +VALUES 1, 2, 3 +``` + +Return a table with two columns and three rows: + +``` +VALUES + (1, 'a'), + (2, 'b'), + (3, 'c') +``` + +Return table with column `id` and `name`: + +``` +SELECT * FROM ( + VALUES + (1, 'a'), + (2, 'b'), + (3, 'c') +) AS t (id, name) +``` + +Create a new table with column `id` and `name`: + +``` +CREATE TABLE example AS +SELECT * FROM ( + VALUES + (1, 'a'), + (2, 'b'), + (3, 'c') +) AS t (id, name) +``` + +## See also + +{doc}`insert`, {doc}`select` diff --git a/430/_static/_sphinx_javascript_frameworks_compat.js b/430/_static/_sphinx_javascript_frameworks_compat.js new file mode 100644 index 000000000..8549469dc --- /dev/null +++ b/430/_static/_sphinx_javascript_frameworks_compat.js @@ -0,0 +1,134 @@ +/* + * _sphinx_javascript_frameworks_compat.js + * ~~~~~~~~~~ + * + * Compatability shim for jQuery and underscores.js. + * + * WILL BE REMOVED IN Sphinx 6.0 + * xref RemovedInSphinx60Warning + * + */ + +/** + * select a different prefix for underscore + */ +$u = _.noConflict(); + + +/** + * small helper function to urldecode strings + * + * See https://developer.mozilla.org/en-US/docs/Web/JavaScript/Reference/Global_Objects/decodeURIComponent#Decoding_query_parameters_from_a_URL + */ +jQuery.urldecode = function(x) { + if (!x) { + return x + } + return decodeURIComponent(x.replace(/\+/g, ' ')); +}; + +/** + * small helper function to urlencode strings + */ +jQuery.urlencode = encodeURIComponent; + +/** + * This function returns the parsed url parameters of the + * current request. Multiple values per key are supported, + * it will always return arrays of strings for the value parts. + */ +jQuery.getQueryParameters = function(s) { + if (typeof s === 'undefined') + s = document.location.search; + var parts = s.substr(s.indexOf('?') + 1).split('&'); + var result = {}; + for (var i = 0; i < parts.length; i++) { + var tmp = parts[i].split('=', 2); + var key = jQuery.urldecode(tmp[0]); + var value = jQuery.urldecode(tmp[1]); + if (key in result) + result[key].push(value); + else + result[key] = [value]; + } + return result; +}; + +/** + * highlight a given string on a jquery object by wrapping it in + * span elements with the given class name. + */ +jQuery.fn.highlightText = function(text, className) { + function highlight(node, addItems) { + if (node.nodeType === 3) { + var val = node.nodeValue; + var pos = val.toLowerCase().indexOf(text); + if (pos >= 0 && + !jQuery(node.parentNode).hasClass(className) && + !jQuery(node.parentNode).hasClass("nohighlight")) { + var span; + var isInSVG = jQuery(node).closest("body, svg, foreignObject").is("svg"); + if (isInSVG) { + span = document.createElementNS("http://www.w3.org/2000/svg", "tspan"); + } else { + span = document.createElement("span"); + span.className = className; + } + span.appendChild(document.createTextNode(val.substr(pos, text.length))); + node.parentNode.insertBefore(span, node.parentNode.insertBefore( + document.createTextNode(val.substr(pos + text.length)), + node.nextSibling)); + node.nodeValue = val.substr(0, pos); + if (isInSVG) { + var rect = document.createElementNS("http://www.w3.org/2000/svg", "rect"); + var bbox = node.parentElement.getBBox(); + rect.x.baseVal.value = bbox.x; + rect.y.baseVal.value = bbox.y; + rect.width.baseVal.value = bbox.width; + rect.height.baseVal.value = bbox.height; + rect.setAttribute('class', className); + addItems.push({ + "parent": node.parentNode, + "target": rect}); + } + } + } + else if (!jQuery(node).is("button, select, textarea")) { + jQuery.each(node.childNodes, function() { + highlight(this, addItems); + }); + } + } + var addItems = []; + var result = this.each(function() { + highlight(this, addItems); + }); + for (var i = 0; i < addItems.length; ++i) { + jQuery(addItems[i].parent).before(addItems[i].target); + } + return result; +}; + +/* + * backward compatibility for jQuery.browser + * This will be supported until firefox bug is fixed. + */ +if (!jQuery.browser) { + jQuery.uaMatch = function(ua) { + ua = ua.toLowerCase(); + + var match = /(chrome)[ \/]([\w.]+)/.exec(ua) || + /(webkit)[ \/]([\w.]+)/.exec(ua) || + /(opera)(?:.*version|)[ \/]([\w.]+)/.exec(ua) || + /(msie) ([\w.]+)/.exec(ua) || + ua.indexOf("compatible") < 0 && /(mozilla)(?:.*? rv:([\w.]+)|)/.exec(ua) || + []; + + return { + browser: match[ 1 ] || "", + version: match[ 2 ] || "0" + }; + }; + jQuery.browser = {}; + jQuery.browser[jQuery.uaMatch(navigator.userAgent).browser] = true; +} diff --git a/430/_static/basic.css b/430/_static/basic.css new file mode 100644 index 000000000..088967717 --- /dev/null +++ b/430/_static/basic.css @@ -0,0 +1,930 @@ +/* + * basic.css + * ~~~~~~~~~ + * + * Sphinx stylesheet -- basic theme. + * + * :copyright: Copyright 2007-2022 by the Sphinx team, see AUTHORS. + * :license: BSD, see LICENSE for details. + * + */ + +/* -- main layout ----------------------------------------------------------- */ + +div.clearer { + clear: both; +} + +div.section::after { + display: block; + content: ''; + clear: left; +} + +/* -- relbar ---------------------------------------------------------------- */ + +div.related { + width: 100%; + font-size: 90%; +} + +div.related h3 { + display: none; +} + +div.related ul { + margin: 0; + padding: 0 0 0 10px; + list-style: none; +} + +div.related li { + display: inline; +} + +div.related li.right { + float: right; + margin-right: 5px; +} + +/* -- sidebar --------------------------------------------------------------- */ + +div.sphinxsidebarwrapper { + padding: 10px 5px 0 10px; +} + +div.sphinxsidebar { + float: left; + width: 230px; + margin-left: -100%; + font-size: 90%; + word-wrap: break-word; + overflow-wrap : break-word; +} + +div.sphinxsidebar ul { + list-style: none; +} + +div.sphinxsidebar ul ul, +div.sphinxsidebar ul.want-points { + margin-left: 20px; + list-style: square; +} + +div.sphinxsidebar ul ul { + margin-top: 0; + margin-bottom: 0; +} + +div.sphinxsidebar form { + margin-top: 10px; +} + +div.sphinxsidebar input { + border: 1px solid #98dbcc; + font-family: sans-serif; + font-size: 1em; +} + +div.sphinxsidebar #searchbox form.search { + overflow: hidden; +} + +div.sphinxsidebar #searchbox input[type="text"] { + float: left; + width: 80%; + padding: 0.25em; + box-sizing: border-box; +} + +div.sphinxsidebar #searchbox input[type="submit"] { + float: left; + width: 20%; + border-left: none; + padding: 0.25em; + box-sizing: border-box; +} + + +img { + border: 0; + max-width: 100%; +} + +/* -- search page ----------------------------------------------------------- */ + +ul.search { + margin: 10px 0 0 20px; + padding: 0; +} + +ul.search li { + padding: 5px 0 5px 20px; + background-image: url(file.png); + background-repeat: no-repeat; + background-position: 0 7px; +} + +ul.search li a { + font-weight: bold; +} + +ul.search li p.context { + color: #888; + margin: 2px 0 0 30px; + text-align: left; +} + +ul.keywordmatches li.goodmatch a { + font-weight: bold; +} + +/* -- index page ------------------------------------------------------------ */ + +table.contentstable { + width: 90%; + margin-left: auto; + margin-right: auto; +} + +table.contentstable p.biglink { + line-height: 150%; +} + +a.biglink { + font-size: 1.3em; +} + +span.linkdescr { + font-style: italic; + padding-top: 5px; + font-size: 90%; +} + +/* -- general index --------------------------------------------------------- */ + +table.indextable { + width: 100%; +} + +table.indextable td { + text-align: left; + vertical-align: top; +} + +table.indextable ul { + margin-top: 0; + margin-bottom: 0; + list-style-type: none; +} + +table.indextable > tbody > tr > td > ul { + padding-left: 0em; +} + +table.indextable tr.pcap { + height: 10px; +} + +table.indextable tr.cap { + margin-top: 10px; + background-color: #f2f2f2; +} + +img.toggler { + margin-right: 3px; + margin-top: 3px; + cursor: pointer; +} + +div.modindex-jumpbox { + border-top: 1px solid #ddd; + border-bottom: 1px solid #ddd; + margin: 1em 0 1em 0; + padding: 0.4em; +} + +div.genindex-jumpbox { + border-top: 1px solid #ddd; + border-bottom: 1px solid #ddd; + margin: 1em 0 1em 0; + padding: 0.4em; +} + +/* -- domain module index --------------------------------------------------- */ + +table.modindextable td { + padding: 2px; + border-collapse: collapse; +} + +/* -- general body styles --------------------------------------------------- */ + +div.body { + min-width: 360px; + max-width: 800px; +} + +div.body p, div.body dd, div.body li, div.body blockquote { + -moz-hyphens: auto; + -ms-hyphens: auto; + -webkit-hyphens: auto; + hyphens: auto; +} + +a.headerlink { + visibility: hidden; +} + +h1:hover > a.headerlink, +h2:hover > a.headerlink, +h3:hover > a.headerlink, +h4:hover > a.headerlink, +h5:hover > a.headerlink, +h6:hover > a.headerlink, +dt:hover > a.headerlink, +caption:hover > a.headerlink, +p.caption:hover > a.headerlink, +div.code-block-caption:hover > a.headerlink { + visibility: visible; +} + +div.body p.caption { + text-align: inherit; +} + +div.body td { + text-align: left; +} + +.first { + margin-top: 0 !important; +} + +p.rubric { + margin-top: 30px; + font-weight: bold; +} + +img.align-left, figure.align-left, .figure.align-left, object.align-left { + clear: left; + float: left; + margin-right: 1em; +} + +img.align-right, figure.align-right, .figure.align-right, object.align-right { + clear: right; + float: right; + margin-left: 1em; +} + +img.align-center, figure.align-center, .figure.align-center, object.align-center { + display: block; + margin-left: auto; + margin-right: auto; +} + +img.align-default, figure.align-default, .figure.align-default { + display: block; + margin-left: auto; + margin-right: auto; +} + +.align-left { + text-align: left; +} + +.align-center { + text-align: center; +} + +.align-default { + text-align: center; +} + +.align-right { + text-align: right; +} + +/* -- sidebars -------------------------------------------------------------- */ + +div.sidebar, +aside.sidebar { + margin: 0 0 0.5em 1em; + border: 1px solid #ddb; + padding: 7px; + background-color: #ffe; + width: 40%; + float: right; + clear: right; + overflow-x: auto; +} + +p.sidebar-title { + font-weight: bold; +} +nav.contents, +aside.topic, + +div.admonition, div.topic, blockquote { + clear: left; +} + +/* -- topics ---------------------------------------------------------------- */ +nav.contents, +aside.topic, + +div.topic { + border: 1px solid #ccc; + padding: 7px; + margin: 10px 0 10px 0; +} + +p.topic-title { + font-size: 1.1em; + font-weight: bold; + margin-top: 10px; +} + +/* -- admonitions ----------------------------------------------------------- */ + +div.admonition { + margin-top: 10px; + margin-bottom: 10px; + padding: 7px; +} + +div.admonition dt { + font-weight: bold; +} + +p.admonition-title { + margin: 0px 10px 5px 0px; + font-weight: bold; +} + +div.body p.centered { + text-align: center; + margin-top: 25px; +} + +/* -- content of sidebars/topics/admonitions -------------------------------- */ + +div.sidebar > :last-child, +aside.sidebar > :last-child, +nav.contents > :last-child, +aside.topic > :last-child, + +div.topic > :last-child, +div.admonition > :last-child { + margin-bottom: 0; +} + +div.sidebar::after, +aside.sidebar::after, +nav.contents::after, +aside.topic::after, + +div.topic::after, +div.admonition::after, +blockquote::after { + display: block; + content: ''; + clear: both; +} + +/* -- tables ---------------------------------------------------------------- */ + +table.docutils { + margin-top: 10px; + margin-bottom: 10px; + border: 0; + border-collapse: collapse; +} + +table.align-center { + margin-left: auto; + margin-right: auto; +} + +table.align-default { + margin-left: auto; + margin-right: auto; +} + +table caption span.caption-number { + font-style: italic; +} + +table caption span.caption-text { +} + +table.docutils td, table.docutils th { + padding: 1px 8px 1px 5px; + border-top: 0; + border-left: 0; + border-right: 0; + border-bottom: 1px solid #aaa; +} + +th { + text-align: left; + padding-right: 5px; +} + +table.citation { + border-left: solid 1px gray; + margin-left: 1px; +} + +table.citation td { + border-bottom: none; +} + +th > :first-child, +td > :first-child { + margin-top: 0px; +} + +th > :last-child, +td > :last-child { + margin-bottom: 0px; +} + +/* -- figures --------------------------------------------------------------- */ + +div.figure, figure { + margin: 0.5em; + padding: 0.5em; +} + +div.figure p.caption, figcaption { + padding: 0.3em; +} + +div.figure p.caption span.caption-number, +figcaption span.caption-number { + font-style: italic; +} + +div.figure p.caption span.caption-text, +figcaption span.caption-text { +} + +/* -- field list styles ----------------------------------------------------- */ + +table.field-list td, table.field-list th { + border: 0 !important; +} + +.field-list ul { + margin: 0; + padding-left: 1em; +} + +.field-list p { + margin: 0; +} + +.field-name { + -moz-hyphens: manual; + -ms-hyphens: manual; + -webkit-hyphens: manual; + hyphens: manual; +} + +/* -- hlist styles ---------------------------------------------------------- */ + +table.hlist { + margin: 1em 0; +} + +table.hlist td { + vertical-align: top; +} + +/* -- object description styles --------------------------------------------- */ + +.sig { + font-family: 'Consolas', 'Menlo', 'DejaVu Sans Mono', 'Bitstream Vera Sans Mono', monospace; +} + +.sig-name, code.descname { + background-color: transparent; + font-weight: bold; +} + +.sig-name { + font-size: 1.1em; +} + +code.descname { + font-size: 1.2em; +} + +.sig-prename, code.descclassname { + background-color: transparent; +} + +.optional { + font-size: 1.3em; +} + +.sig-paren { + font-size: larger; +} + +.sig-param.n { + font-style: italic; +} + +/* C++ specific styling */ + +.sig-inline.c-texpr, +.sig-inline.cpp-texpr { + font-family: unset; +} + +.sig.c .k, .sig.c .kt, +.sig.cpp .k, .sig.cpp .kt { + color: #0033B3; +} + +.sig.c .m, +.sig.cpp .m { + color: #1750EB; +} + +.sig.c .s, .sig.c .sc, +.sig.cpp .s, .sig.cpp .sc { + color: #067D17; +} + + +/* -- other body styles ----------------------------------------------------- */ + +ol.arabic { + list-style: decimal; +} + +ol.loweralpha { + list-style: lower-alpha; +} + +ol.upperalpha { + list-style: upper-alpha; +} + +ol.lowerroman { + list-style: lower-roman; +} + +ol.upperroman { + list-style: upper-roman; +} + +:not(li) > ol > li:first-child > :first-child, +:not(li) > ul > li:first-child > :first-child { + margin-top: 0px; +} + +:not(li) > ol > li:last-child > :last-child, +:not(li) > ul > li:last-child > :last-child { + margin-bottom: 0px; +} + +ol.simple ol p, +ol.simple ul p, +ul.simple ol p, +ul.simple ul p { + margin-top: 0; +} + +ol.simple > li:not(:first-child) > p, +ul.simple > li:not(:first-child) > p { + margin-top: 0; +} + +ol.simple p, +ul.simple p { + margin-bottom: 0; +} + +/* Docutils 0.17 and older (footnotes & citations) */ +dl.footnote > dt, +dl.citation > dt { + float: left; + margin-right: 0.5em; +} + +dl.footnote > dd, +dl.citation > dd { + margin-bottom: 0em; +} + +dl.footnote > dd:after, +dl.citation > dd:after { + content: ""; + clear: both; +} + +/* Docutils 0.18+ (footnotes & citations) */ +aside.footnote > span, +div.citation > span { + float: left; +} +aside.footnote > span:last-of-type, +div.citation > span:last-of-type { + padding-right: 0.5em; +} +aside.footnote > p { + margin-left: 2em; +} +div.citation > p { + margin-left: 4em; +} +aside.footnote > p:last-of-type, +div.citation > p:last-of-type { + margin-bottom: 0em; +} +aside.footnote > p:last-of-type:after, +div.citation > p:last-of-type:after { + content: ""; + clear: both; +} + +/* Footnotes & citations ends */ + +dl.field-list { + display: grid; + grid-template-columns: fit-content(30%) auto; +} + +dl.field-list > dt { + font-weight: bold; + word-break: break-word; + padding-left: 0.5em; + padding-right: 5px; +} + +dl.field-list > dt:after { + content: ":"; +} + +dl.field-list > dd { + padding-left: 0.5em; + margin-top: 0em; + margin-left: 0em; + margin-bottom: 0em; +} + +dl { + margin-bottom: 15px; +} + +dd > :first-child { + margin-top: 0px; +} + +dd ul, dd table { + margin-bottom: 10px; +} + +dd { + margin-top: 3px; + margin-bottom: 10px; + margin-left: 30px; +} + +dl > dd:last-child, +dl > dd:last-child > :last-child { + margin-bottom: 0; +} + +dt:target, span.highlighted { + background-color: #fbe54e; +} + +rect.highlighted { + fill: #fbe54e; +} + +dl.glossary dt { + font-weight: bold; + font-size: 1.1em; +} + +.versionmodified { + font-style: italic; +} + +.system-message { + background-color: #fda; + padding: 5px; + border: 3px solid red; +} + +.footnote:target { + background-color: #ffa; +} + +.line-block { + display: block; + margin-top: 1em; + margin-bottom: 1em; +} + +.line-block .line-block { + margin-top: 0; + margin-bottom: 0; + margin-left: 1.5em; +} + +.guilabel, .menuselection { + font-family: sans-serif; +} + +.accelerator { + text-decoration: underline; +} + +.classifier { + font-style: oblique; +} + +.classifier:before { + font-style: normal; + margin: 0 0.5em; + content: ":"; + display: inline-block; +} + +abbr, acronym { + border-bottom: dotted 1px; + cursor: help; +} + +/* -- code displays --------------------------------------------------------- */ + +pre { + overflow: auto; + overflow-y: hidden; /* fixes display issues on Chrome browsers */ +} + +pre, div[class*="highlight-"] { + clear: both; +} + +span.pre { + -moz-hyphens: none; + -ms-hyphens: none; + -webkit-hyphens: none; + hyphens: none; + white-space: nowrap; +} + +div[class*="highlight-"] { + margin: 1em 0; +} + +td.linenos pre { + border: 0; + background-color: transparent; + color: #aaa; +} + +table.highlighttable { + display: block; +} + +table.highlighttable tbody { + display: block; +} + +table.highlighttable tr { + display: flex; +} + +table.highlighttable td { + margin: 0; + padding: 0; +} + +table.highlighttable td.linenos { + padding-right: 0.5em; +} + +table.highlighttable td.code { + flex: 1; + overflow: hidden; +} + +.highlight .hll { + display: block; +} + +div.highlight pre, +table.highlighttable pre { + margin: 0; +} + +div.code-block-caption + div { + margin-top: 0; +} + +div.code-block-caption { + margin-top: 1em; + padding: 2px 5px; + font-size: small; +} + +div.code-block-caption code { + background-color: transparent; +} + +table.highlighttable td.linenos, +span.linenos, +div.highlight span.gp { /* gp: Generic.Prompt */ + user-select: none; + -webkit-user-select: text; /* Safari fallback only */ + -webkit-user-select: none; /* Chrome/Safari */ + -moz-user-select: none; /* Firefox */ + -ms-user-select: none; /* IE10+ */ +} + +div.code-block-caption span.caption-number { + padding: 0.1em 0.3em; + font-style: italic; +} + +div.code-block-caption span.caption-text { +} + +div.literal-block-wrapper { + margin: 1em 0; +} + +code.xref, a code { + background-color: transparent; + font-weight: bold; +} + +h1 code, h2 code, h3 code, h4 code, h5 code, h6 code { + background-color: transparent; +} + +.viewcode-link { + float: right; +} + +.viewcode-back { + float: right; + font-family: sans-serif; +} + +div.viewcode-block:target { + margin: -1px -10px; + padding: 0 10px; +} + +/* -- math display ---------------------------------------------------------- */ + +img.math { + vertical-align: middle; +} + +div.body div.math p { + text-align: center; +} + +span.eqno { + float: right; +} + +span.eqno a.headerlink { + position: absolute; + z-index: 1; +} + +div.math:hover a.headerlink { + visibility: visible; +} + +/* -- printout stylesheet --------------------------------------------------- */ + +@media print { + div.document, + div.documentwrapper, + div.bodywrapper { + margin: 0 !important; + width: 100%; + } + + div.sphinxsidebar, + div.related, + div.footer, + #top-link { + display: none; + } +} \ No newline at end of file diff --git a/430/_static/clipboard.min.js b/430/_static/clipboard.min.js new file mode 100644 index 000000000..02c549e35 --- /dev/null +++ b/430/_static/clipboard.min.js @@ -0,0 +1,7 @@ +/*! + * clipboard.js v2.0.4 + * https://zenorocha.github.io/clipboard.js + * + * Licensed MIT © Zeno Rocha + */ +!function(t,e){"object"==typeof exports&&"object"==typeof module?module.exports=e():"function"==typeof define&&define.amd?define([],e):"object"==typeof exports?exports.ClipboardJS=e():t.ClipboardJS=e()}(this,function(){return function(n){var o={};function r(t){if(o[t])return o[t].exports;var e=o[t]={i:t,l:!1,exports:{}};return n[t].call(e.exports,e,e.exports,r),e.l=!0,e.exports}return r.m=n,r.c=o,r.d=function(t,e,n){r.o(t,e)||Object.defineProperty(t,e,{enumerable:!0,get:n})},r.r=function(t){"undefined"!=typeof Symbol&&Symbol.toStringTag&&Object.defineProperty(t,Symbol.toStringTag,{value:"Module"}),Object.defineProperty(t,"__esModule",{value:!0})},r.t=function(e,t){if(1&t&&(e=r(e)),8&t)return e;if(4&t&&"object"==typeof e&&e&&e.__esModule)return e;var n=Object.create(null);if(r.r(n),Object.defineProperty(n,"default",{enumerable:!0,value:e}),2&t&&"string"!=typeof e)for(var o in e)r.d(n,o,function(t){return e[t]}.bind(null,o));return n},r.n=function(t){var e=t&&t.__esModule?function(){return t.default}:function(){return t};return r.d(e,"a",e),e},r.o=function(t,e){return Object.prototype.hasOwnProperty.call(t,e)},r.p="",r(r.s=0)}([function(t,e,n){"use strict";var r="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol&&t!==Symbol.prototype?"symbol":typeof t},i=function(){function o(t,e){for(var n=0;n + + + + diff --git a/430/_static/copybutton.css b/430/_static/copybutton.css new file mode 100644 index 000000000..75b17a83d --- /dev/null +++ b/430/_static/copybutton.css @@ -0,0 +1,67 @@ +/* Copy buttons */ +a.copybtn { + position: absolute; + top: .2em; + right: .2em; + width: 1em; + height: 1em; + opacity: .3; + transition: opacity 0.5s; + border: none; + user-select: none; +} + +div.highlight { + position: relative; +} + +a.copybtn > img { + vertical-align: top; + margin: 0; + top: 0; + left: 0; + position: absolute; +} + +.highlight:hover .copybtn { + opacity: 1; +} + +/** + * A minimal CSS-only tooltip copied from: + * https://codepen.io/mildrenben/pen/rVBrpK + * + * To use, write HTML like the following: + * + *

Short

+ */ + .o-tooltip--left { + position: relative; + } + + .o-tooltip--left:after { + opacity: 0; + visibility: hidden; + position: absolute; + content: attr(data-tooltip); + padding: 2px; + top: 0; + left: -.2em; + background: grey; + font-size: 1rem; + color: white; + white-space: nowrap; + z-index: 2; + border-radius: 2px; + transform: translateX(-102%) translateY(0); + transition: opacity 0.2s cubic-bezier(0.64, 0.09, 0.08, 1), transform 0.2s cubic-bezier(0.64, 0.09, 0.08, 1); +} + +.o-tooltip--left:hover:after { + display: block; + opacity: 1; + visibility: visible; + transform: translateX(-100%) translateY(0); + transition: opacity 0.2s cubic-bezier(0.64, 0.09, 0.08, 1), transform 0.2s cubic-bezier(0.64, 0.09, 0.08, 1); + transition-delay: .5s; +} diff --git a/430/_static/copybutton.js b/430/_static/copybutton.js new file mode 100644 index 000000000..65a59167a --- /dev/null +++ b/430/_static/copybutton.js @@ -0,0 +1,153 @@ +// Localization support +const messages = { + 'en': { + 'copy': 'Copy', + 'copy_to_clipboard': 'Copy to clipboard', + 'copy_success': 'Copied!', + 'copy_failure': 'Failed to copy', + }, + 'es' : { + 'copy': 'Copiar', + 'copy_to_clipboard': 'Copiar al portapapeles', + 'copy_success': '¡Copiado!', + 'copy_failure': 'Error al copiar', + }, + 'de' : { + 'copy': 'Kopieren', + 'copy_to_clipboard': 'In die Zwischenablage kopieren', + 'copy_success': 'Kopiert!', + 'copy_failure': 'Fehler beim Kopieren', + } +} + +let locale = 'en' +if( document.documentElement.lang !== undefined + && messages[document.documentElement.lang] !== undefined ) { + locale = document.documentElement.lang +} + +/** + * Set up copy/paste for code blocks + */ + +const runWhenDOMLoaded = cb => { + if (document.readyState != 'loading') { + cb() + } else if (document.addEventListener) { + document.addEventListener('DOMContentLoaded', cb) + } else { + document.attachEvent('onreadystatechange', function() { + if (document.readyState == 'complete') cb() + }) + } +} + +const codeCellId = index => `codecell${index}` + +// Clears selected text since ClipboardJS will select the text when copying +const clearSelection = () => { + if (window.getSelection) { + window.getSelection().removeAllRanges() + } else if (document.selection) { + document.selection.empty() + } +} + +// Changes tooltip text for two seconds, then changes it back +const temporarilyChangeTooltip = (el, newText) => { + const oldText = el.getAttribute('data-tooltip') + el.setAttribute('data-tooltip', newText) + setTimeout(() => el.setAttribute('data-tooltip', oldText), 2000) +} + +const addCopyButtonToCodeCells = () => { + // If ClipboardJS hasn't loaded, wait a bit and try again. This + // happens because we load ClipboardJS asynchronously. + if (window.ClipboardJS === undefined) { + setTimeout(addCopyButtonToCodeCells, 250) + return + } + + // Add copybuttons to all of our code cells + const codeCells = document.querySelectorAll('div.highlight pre') + codeCells.forEach((codeCell, index) => { + const id = codeCellId(index) + codeCell.setAttribute('id', id) + const pre_bg = getComputedStyle(codeCell).backgroundColor; + + const clipboardButton = id => + ` + ${messages[locale]['copy_to_clipboard']} + ` + codeCell.insertAdjacentHTML('afterend', clipboardButton(id)) + }) + +function escapeRegExp(string) { + return string.replace(/[.*+?^${}()|[\]\\]/g, '\\$&'); // $& means the whole matched string +} + +// Callback when a copy button is clicked. Will be passed the node that was clicked +// should then grab the text and replace pieces of text that shouldn't be used in output +function formatCopyText(textContent, copybuttonPromptText, isRegexp = false, onlyCopyPromptLines = true, removePrompts = true) { + + var regexp; + var match; + + // create regexp to capture prompt and remaining line + if (isRegexp) { + regexp = new RegExp('^(' + copybuttonPromptText + ')(.*)') + } else { + regexp = new RegExp('^(' + escapeRegExp(copybuttonPromptText) + ')(.*)') + } + + const outputLines = []; + var promptFound = false; + for (const line of textContent.split('\n')) { + match = line.match(regexp) + if (match) { + promptFound = true + if (removePrompts) { + outputLines.push(match[2]) + } else { + outputLines.push(line) + } + } else { + if (!onlyCopyPromptLines) { + outputLines.push(line) + } + } + } + + // If no lines with the prompt were found then just use original lines + if (promptFound) { + textContent = outputLines.join('\n'); + } + + // Remove a trailing newline to avoid auto-running when pasting + if (textContent.endsWith("\n")) { + textContent = textContent.slice(0, -1) + } + return textContent +} + + +var copyTargetText = (trigger) => { + var target = document.querySelector(trigger.attributes['data-clipboard-target'].value); + return formatCopyText(target.innerText, '', false, true, true) +} + + // Initialize with a callback so we can modify the text before copy + const clipboard = new ClipboardJS('.copybtn', {text: copyTargetText}) + + // Update UI with error/success messages + clipboard.on('success', event => { + clearSelection() + temporarilyChangeTooltip(event.trigger, messages[locale]['copy_success']) + }) + + clipboard.on('error', event => { + temporarilyChangeTooltip(event.trigger, messages[locale]['copy_failure']) + }) +} + +runWhenDOMLoaded(addCopyButtonToCodeCells) \ No newline at end of file diff --git a/430/_static/copybutton_funcs.js b/430/_static/copybutton_funcs.js new file mode 100644 index 000000000..57caa5585 --- /dev/null +++ b/430/_static/copybutton_funcs.js @@ -0,0 +1,47 @@ +function escapeRegExp(string) { + return string.replace(/[.*+?^${}()|[\]\\]/g, '\\$&'); // $& means the whole matched string +} + +// Callback when a copy button is clicked. Will be passed the node that was clicked +// should then grab the text and replace pieces of text that shouldn't be used in output +export function formatCopyText(textContent, copybuttonPromptText, isRegexp = false, onlyCopyPromptLines = true, removePrompts = true) { + + var regexp; + var match; + + // create regexp to capture prompt and remaining line + if (isRegexp) { + regexp = new RegExp('^(' + copybuttonPromptText + ')(.*)') + } else { + regexp = new RegExp('^(' + escapeRegExp(copybuttonPromptText) + ')(.*)') + } + + const outputLines = []; + var promptFound = false; + for (const line of textContent.split('\n')) { + match = line.match(regexp) + if (match) { + promptFound = true + if (removePrompts) { + outputLines.push(match[2]) + } else { + outputLines.push(line) + } + } else { + if (!onlyCopyPromptLines) { + outputLines.push(line) + } + } + } + + // If no lines with the prompt were found then just use original lines + if (promptFound) { + textContent = outputLines.join('\n'); + } + + // Remove a trailing newline to avoid auto-running when pasting + if (textContent.endsWith("\n")) { + textContent = textContent.slice(0, -1) + } + return textContent +} diff --git a/430/_static/doctools.js b/430/_static/doctools.js new file mode 100644 index 000000000..c3db08d1c --- /dev/null +++ b/430/_static/doctools.js @@ -0,0 +1,264 @@ +/* + * doctools.js + * ~~~~~~~~~~~ + * + * Base JavaScript utilities for all Sphinx HTML documentation. + * + * :copyright: Copyright 2007-2022 by the Sphinx team, see AUTHORS. + * :license: BSD, see LICENSE for details. + * + */ +"use strict"; + +const _ready = (callback) => { + if (document.readyState !== "loading") { + callback(); + } else { + document.addEventListener("DOMContentLoaded", callback); + } +}; + +/** + * highlight a given string on a node by wrapping it in + * span elements with the given class name. + */ +const _highlight = (node, addItems, text, className) => { + if (node.nodeType === Node.TEXT_NODE) { + const val = node.nodeValue; + const parent = node.parentNode; + const pos = val.toLowerCase().indexOf(text); + if ( + pos >= 0 && + !parent.classList.contains(className) && + !parent.classList.contains("nohighlight") + ) { + let span; + + const closestNode = parent.closest("body, svg, foreignObject"); + const isInSVG = closestNode && closestNode.matches("svg"); + if (isInSVG) { + span = document.createElementNS("http://www.w3.org/2000/svg", "tspan"); + } else { + span = document.createElement("span"); + span.classList.add(className); + } + + span.appendChild(document.createTextNode(val.substr(pos, text.length))); + parent.insertBefore( + span, + parent.insertBefore( + document.createTextNode(val.substr(pos + text.length)), + node.nextSibling + ) + ); + node.nodeValue = val.substr(0, pos); + + if (isInSVG) { + const rect = document.createElementNS( + "http://www.w3.org/2000/svg", + "rect" + ); + const bbox = parent.getBBox(); + rect.x.baseVal.value = bbox.x; + rect.y.baseVal.value = bbox.y; + rect.width.baseVal.value = bbox.width; + rect.height.baseVal.value = bbox.height; + rect.setAttribute("class", className); + addItems.push({ parent: parent, target: rect }); + } + } + } else if (node.matches && !node.matches("button, select, textarea")) { + node.childNodes.forEach((el) => _highlight(el, addItems, text, className)); + } +}; +const _highlightText = (thisNode, text, className) => { + let addItems = []; + _highlight(thisNode, addItems, text, className); + addItems.forEach((obj) => + obj.parent.insertAdjacentElement("beforebegin", obj.target) + ); +}; + +/** + * Small JavaScript module for the documentation. + */ +const Documentation = { + init: () => { + Documentation.highlightSearchWords(); + Documentation.initDomainIndexTable(); + Documentation.initOnKeyListeners(); + }, + + /** + * i18n support + */ + TRANSLATIONS: {}, + PLURAL_EXPR: (n) => (n === 1 ? 0 : 1), + LOCALE: "unknown", + + // gettext and ngettext don't access this so that the functions + // can safely bound to a different name (_ = Documentation.gettext) + gettext: (string) => { + const translated = Documentation.TRANSLATIONS[string]; + switch (typeof translated) { + case "undefined": + return string; // no translation + case "string": + return translated; // translation exists + default: + return translated[0]; // (singular, plural) translation tuple exists + } + }, + + ngettext: (singular, plural, n) => { + const translated = Documentation.TRANSLATIONS[singular]; + if (typeof translated !== "undefined") + return translated[Documentation.PLURAL_EXPR(n)]; + return n === 1 ? singular : plural; + }, + + addTranslations: (catalog) => { + Object.assign(Documentation.TRANSLATIONS, catalog.messages); + Documentation.PLURAL_EXPR = new Function( + "n", + `return (${catalog.plural_expr})` + ); + Documentation.LOCALE = catalog.locale; + }, + + /** + * highlight the search words provided in the url in the text + */ + highlightSearchWords: () => { + const highlight = + new URLSearchParams(window.location.search).get("highlight") || ""; + const terms = highlight.toLowerCase().split(/\s+/).filter(x => x); + if (terms.length === 0) return; // nothing to do + + // There should never be more than one element matching "div.body" + const divBody = document.querySelectorAll("div.body"); + const body = divBody.length ? divBody[0] : document.querySelector("body"); + window.setTimeout(() => { + terms.forEach((term) => _highlightText(body, term, "highlighted")); + }, 10); + + const searchBox = document.getElementById("searchbox"); + if (searchBox === null) return; + searchBox.appendChild( + document + .createRange() + .createContextualFragment( + '" + ) + ); + }, + + /** + * helper function to hide the search marks again + */ + hideSearchWords: () => { + document + .querySelectorAll("#searchbox .highlight-link") + .forEach((el) => el.remove()); + document + .querySelectorAll("span.highlighted") + .forEach((el) => el.classList.remove("highlighted")); + const url = new URL(window.location); + url.searchParams.delete("highlight"); + window.history.replaceState({}, "", url); + }, + + /** + * helper function to focus on search bar + */ + focusSearchBar: () => { + document.querySelectorAll("input[name=q]")[0]?.focus(); + }, + + /** + * Initialise the domain index toggle buttons + */ + initDomainIndexTable: () => { + const toggler = (el) => { + const idNumber = el.id.substr(7); + const toggledRows = document.querySelectorAll(`tr.cg-${idNumber}`); + if (el.src.substr(-9) === "minus.png") { + el.src = `${el.src.substr(0, el.src.length - 9)}plus.png`; + toggledRows.forEach((el) => (el.style.display = "none")); + } else { + el.src = `${el.src.substr(0, el.src.length - 8)}minus.png`; + toggledRows.forEach((el) => (el.style.display = "")); + } + }; + + const togglerElements = document.querySelectorAll("img.toggler"); + togglerElements.forEach((el) => + el.addEventListener("click", (event) => toggler(event.currentTarget)) + ); + togglerElements.forEach((el) => (el.style.display = "")); + if (DOCUMENTATION_OPTIONS.COLLAPSE_INDEX) togglerElements.forEach(toggler); + }, + + initOnKeyListeners: () => { + // only install a listener if it is really needed + if ( + !DOCUMENTATION_OPTIONS.NAVIGATION_WITH_KEYS && + !DOCUMENTATION_OPTIONS.ENABLE_SEARCH_SHORTCUTS + ) + return; + + const blacklistedElements = new Set([ + "TEXTAREA", + "INPUT", + "SELECT", + "BUTTON", + ]); + document.addEventListener("keydown", (event) => { + if (blacklistedElements.has(document.activeElement.tagName)) return; // bail for input elements + if (event.altKey || event.ctrlKey || event.metaKey) return; // bail with special keys + + if (!event.shiftKey) { + switch (event.key) { + case "ArrowLeft": + if (!DOCUMENTATION_OPTIONS.NAVIGATION_WITH_KEYS) break; + + const prevLink = document.querySelector('link[rel="prev"]'); + if (prevLink && prevLink.href) { + window.location.href = prevLink.href; + event.preventDefault(); + } + break; + case "ArrowRight": + if (!DOCUMENTATION_OPTIONS.NAVIGATION_WITH_KEYS) break; + + const nextLink = document.querySelector('link[rel="next"]'); + if (nextLink && nextLink.href) { + window.location.href = nextLink.href; + event.preventDefault(); + } + break; + case "Escape": + if (!DOCUMENTATION_OPTIONS.ENABLE_SEARCH_SHORTCUTS) break; + Documentation.hideSearchWords(); + event.preventDefault(); + } + } + + // some keyboard layouts may need Shift to get / + switch (event.key) { + case "/": + if (!DOCUMENTATION_OPTIONS.ENABLE_SEARCH_SHORTCUTS) break; + Documentation.focusSearchBar(); + event.preventDefault(); + } + }); + }, +}; + +// quick alias for translations +const _ = Documentation.gettext; + +_ready(Documentation.init); diff --git a/430/_static/documentation_options.js b/430/_static/documentation_options.js new file mode 100644 index 000000000..d5155ec56 --- /dev/null +++ b/430/_static/documentation_options.js @@ -0,0 +1,14 @@ +var DOCUMENTATION_OPTIONS = { + URL_ROOT: document.getElementById("documentation_options").getAttribute('data-url_root'), + VERSION: '430', + LANGUAGE: 'en', + COLLAPSE_INDEX: false, + BUILDER: 'html', + FILE_SUFFIX: '.html', + LINK_SUFFIX: '.html', + HAS_SOURCE: true, + SOURCELINK_SUFFIX: '.txt', + NAVIGATION_WITH_KEYS: false, + SHOW_SEARCH_SUMMARY: true, + ENABLE_SEARCH_SHORTCUTS: false, +}; \ No newline at end of file diff --git a/430/_static/file.png b/430/_static/file.png new file mode 100644 index 000000000..a858a410e Binary files /dev/null and b/430/_static/file.png differ diff --git a/430/_static/fonts/font-awesome.css b/430/_static/fonts/font-awesome.css new file mode 100644 index 000000000..b476b53e3 --- /dev/null +++ b/430/_static/fonts/font-awesome.css @@ -0,0 +1,4 @@ +/*! + * Font Awesome 4.7.0 by @davegandy - http://fontawesome.io - @fontawesome + * License - http://fontawesome.io/license (Font: SIL OFL 1.1, CSS: MIT License) + */@font-face{font-family:FontAwesome;font-style:normal;font-weight:400;src:url("specimen/FontAwesome.woff2") format("woff2"),url("specimen/FontAwesome.woff") format("woff"),url("specimen/FontAwesome.ttf") format("truetype")}.fa{display:inline-block;font:normal normal normal 14px/1 FontAwesome;font-size:inherit;text-rendering:auto;-webkit-font-smoothing:antialiased;-moz-osx-font-smoothing:grayscale}.fa-lg{font-size:1.33333333em;line-height:.75em;vertical-align:-15%}.fa-2x{font-size:2em}.fa-3x{font-size:3em}.fa-4x{font-size:4em}.fa-5x{font-size:5em}.fa-fw{width:1.28571429em;text-align:center}.fa-ul{padding-left:0;margin-left:2.14285714em;list-style-type:none}.fa-ul>li{position:relative}.fa-li{position:absolute;left:-2.14285714em;width:2.14285714em;top:.14285714em;text-align:center}.fa-li.fa-lg{left:-1.85714286em}.fa-border{padding:.2em .25em .15em;border:solid .08em #eee;border-radius:.1em}.fa-pull-left{float:left}.fa-pull-right{float:right}.fa.fa-pull-left{margin-right:.3em}.fa.fa-pull-right{margin-left:.3em}.pull-right{float:right}.pull-left{float:left}.fa.pull-left{margin-right:.3em}.fa.pull-right{margin-left:.3em}.fa-spin{-webkit-animation:fa-spin 2s infinite linear;animation:fa-spin 2s infinite linear}.fa-pulse{-webkit-animation:fa-spin 1s infinite steps(8);animation:fa-spin 1s infinite steps(8)}@-webkit-keyframes fa-spin{0%{-webkit-transform:rotate(0deg);transform:rotate(0deg)}100%{-webkit-transform:rotate(359deg);transform:rotate(359deg)}}@keyframes fa-spin{0%{-webkit-transform:rotate(0deg);transform:rotate(0deg)}100%{-webkit-transform:rotate(359deg);transform:rotate(359deg)}}.fa-rotate-90{-ms-filter:"progid:DXImageTransform.Microsoft.BasicImage(rotation=1)";-webkit-transform:rotate(90deg);-ms-transform:rotate(90deg);transform:rotate(90deg)}.fa-rotate-180{-ms-filter:"progid:DXImageTransform.Microsoft.BasicImage(rotation=2)";-webkit-transform:rotate(180deg);-ms-transform:rotate(180deg);transform:rotate(180deg)}.fa-rotate-270{-ms-filter:"progid:DXImageTransform.Microsoft.BasicImage(rotation=3)";-webkit-transform:rotate(270deg);-ms-transform:rotate(270deg);transform:rotate(270deg)}.fa-flip-horizontal{-ms-filter:"progid:DXImageTransform.Microsoft.BasicImage(rotation=0, mirror=1)";-webkit-transform:scale(-1,1);-ms-transform:scale(-1,1);transform:scale(-1,1)}.fa-flip-vertical{-ms-filter:"progid:DXImageTransform.Microsoft.BasicImage(rotation=2, mirror=1)";-webkit-transform:scale(1,-1);-ms-transform:scale(1,-1);transform:scale(1,-1)}:root .fa-rotate-90,:root .fa-rotate-180,:root .fa-rotate-270,:root .fa-flip-horizontal,:root .fa-flip-vertical{filter:none}.fa-stack{position:relative;display:inline-block;width:2em;height:2em;line-height:2em;vertical-align:middle}.fa-stack-1x,.fa-stack-2x{position:absolute;left:0;width:100%;text-align:center}.fa-stack-1x{line-height:inherit}.fa-stack-2x{font-size:2em}.fa-inverse{color:#fff}.fa-glass:before{content:"\f000"}.fa-music:before{content:"\f001"}.fa-search:before{content:"\f002"}.fa-envelope-o:before{content:"\f003"}.fa-heart:before{content:"\f004"}.fa-star:before{content:"\f005"}.fa-star-o:before{content:"\f006"}.fa-user:before{content:"\f007"}.fa-film:before{content:"\f008"}.fa-th-large:before{content:"\f009"}.fa-th:before{content:"\f00a"}.fa-th-list:before{content:"\f00b"}.fa-check:before{content:"\f00c"}.fa-remove:before,.fa-close:before,.fa-times:before{content:"\f00d"}.fa-search-plus:before{content:"\f00e"}.fa-search-minus:before{content:"\f010"}.fa-power-off:before{content:"\f011"}.fa-signal:before{content:"\f012"}.fa-gear:before,.fa-cog:before{content:"\f013"}.fa-trash-o:before{content:"\f014"}.fa-home:before{content:"\f015"}.fa-file-o:before{content:"\f016"}.fa-clock-o:before{content:"\f017"}.fa-road:before{content:"\f018"}.fa-download:before{content:"\f019"}.fa-arrow-circle-o-down:before{content:"\f01a"}.fa-arrow-circle-o-up:before{content:"\f01b"}.fa-inbox:before{content:"\f01c"}.fa-play-circle-o:before{content:"\f01d"}.fa-rotate-right:before,.fa-repeat:before{content:"\f01e"}.fa-refresh:before{content:"\f021"}.fa-list-alt:before{content:"\f022"}.fa-lock:before{content:"\f023"}.fa-flag:before{content:"\f024"}.fa-headphones:before{content:"\f025"}.fa-volume-off:before{content:"\f026"}.fa-volume-down:before{content:"\f027"}.fa-volume-up:before{content:"\f028"}.fa-qrcode:before{content:"\f029"}.fa-barcode:before{content:"\f02a"}.fa-tag:before{content:"\f02b"}.fa-tags:before{content:"\f02c"}.fa-book:before{content:"\f02d"}.fa-bookmark:before{content:"\f02e"}.fa-print:before{content:"\f02f"}.fa-camera:before{content:"\f030"}.fa-font:before{content:"\f031"}.fa-bold:before{content:"\f032"}.fa-italic:before{content:"\f033"}.fa-text-height:before{content:"\f034"}.fa-text-width:before{content:"\f035"}.fa-align-left:before{content:"\f036"}.fa-align-center:before{content:"\f037"}.fa-align-right:before{content:"\f038"}.fa-align-justify:before{content:"\f039"}.fa-list:before{content:"\f03a"}.fa-dedent:before,.fa-outdent:before{content:"\f03b"}.fa-indent:before{content:"\f03c"}.fa-video-camera:before{content:"\f03d"}.fa-photo:before,.fa-image:before,.fa-picture-o:before{content:"\f03e"}.fa-pencil:before{content:"\f040"}.fa-map-marker:before{content:"\f041"}.fa-adjust:before{content:"\f042"}.fa-tint:before{content:"\f043"}.fa-edit:before,.fa-pencil-square-o:before{content:"\f044"}.fa-share-square-o:before{content:"\f045"}.fa-check-square-o:before{content:"\f046"}.fa-arrows:before{content:"\f047"}.fa-step-backward:before{content:"\f048"}.fa-fast-backward:before{content:"\f049"}.fa-backward:before{content:"\f04a"}.fa-play:before{content:"\f04b"}.fa-pause:before{content:"\f04c"}.fa-stop:before{content:"\f04d"}.fa-forward:before{content:"\f04e"}.fa-fast-forward:before{content:"\f050"}.fa-step-forward:before{content:"\f051"}.fa-eject:before{content:"\f052"}.fa-chevron-left:before{content:"\f053"}.fa-chevron-right:before{content:"\f054"}.fa-plus-circle:before{content:"\f055"}.fa-minus-circle:before{content:"\f056"}.fa-times-circle:before{content:"\f057"}.fa-check-circle:before{content:"\f058"}.fa-question-circle:before{content:"\f059"}.fa-info-circle:before{content:"\f05a"}.fa-crosshairs:before{content:"\f05b"}.fa-times-circle-o:before{content:"\f05c"}.fa-check-circle-o:before{content:"\f05d"}.fa-ban:before{content:"\f05e"}.fa-arrow-left:before{content:"\f060"}.fa-arrow-right:before{content:"\f061"}.fa-arrow-up:before{content:"\f062"}.fa-arrow-down:before{content:"\f063"}.fa-mail-forward:before,.fa-share:before{content:"\f064"}.fa-expand:before{content:"\f065"}.fa-compress:before{content:"\f066"}.fa-plus:before{content:"\f067"}.fa-minus:before{content:"\f068"}.fa-asterisk:before{content:"\f069"}.fa-exclamation-circle:before{content:"\f06a"}.fa-gift:before{content:"\f06b"}.fa-leaf:before{content:"\f06c"}.fa-fire:before{content:"\f06d"}.fa-eye:before{content:"\f06e"}.fa-eye-slash:before{content:"\f070"}.fa-warning:before,.fa-exclamation-triangle:before{content:"\f071"}.fa-plane:before{content:"\f072"}.fa-calendar:before{content:"\f073"}.fa-random:before{content:"\f074"}.fa-comment:before{content:"\f075"}.fa-magnet:before{content:"\f076"}.fa-chevron-up:before{content:"\f077"}.fa-chevron-down:before{content:"\f078"}.fa-retweet:before{content:"\f079"}.fa-shopping-cart:before{content:"\f07a"}.fa-folder:before{content:"\f07b"}.fa-folder-open:before{content:"\f07c"}.fa-arrows-v:before{content:"\f07d"}.fa-arrows-h:before{content:"\f07e"}.fa-bar-chart-o:before,.fa-bar-chart:before{content:"\f080"}.fa-twitter-square:before{content:"\f081"}.fa-facebook-square:before{content:"\f082"}.fa-camera-retro:before{content:"\f083"}.fa-key:before{content:"\f084"}.fa-gears:before,.fa-cogs:before{content:"\f085"}.fa-comments:before{content:"\f086"}.fa-thumbs-o-up:before{content:"\f087"}.fa-thumbs-o-down:before{content:"\f088"}.fa-star-half:before{content:"\f089"}.fa-heart-o:before{content:"\f08a"}.fa-sign-out:before{content:"\f08b"}.fa-linkedin-square:before{content:"\f08c"}.fa-thumb-tack:before{content:"\f08d"}.fa-external-link:before{content:"\f08e"}.fa-sign-in:before{content:"\f090"}.fa-trophy:before{content:"\f091"}.fa-github-square:before{content:"\f092"}.fa-upload:before{content:"\f093"}.fa-lemon-o:before{content:"\f094"}.fa-phone:before{content:"\f095"}.fa-square-o:before{content:"\f096"}.fa-bookmark-o:before{content:"\f097"}.fa-phone-square:before{content:"\f098"}.fa-twitter:before{content:"\f099"}.fa-facebook-f:before,.fa-facebook:before{content:"\f09a"}.fa-github:before{content:"\f09b"}.fa-unlock:before{content:"\f09c"}.fa-credit-card:before{content:"\f09d"}.fa-feed:before,.fa-rss:before{content:"\f09e"}.fa-hdd-o:before{content:"\f0a0"}.fa-bullhorn:before{content:"\f0a1"}.fa-bell:before{content:"\f0f3"}.fa-certificate:before{content:"\f0a3"}.fa-hand-o-right:before{content:"\f0a4"}.fa-hand-o-left:before{content:"\f0a5"}.fa-hand-o-up:before{content:"\f0a6"}.fa-hand-o-down:before{content:"\f0a7"}.fa-arrow-circle-left:before{content:"\f0a8"}.fa-arrow-circle-right:before{content:"\f0a9"}.fa-arrow-circle-up:before{content:"\f0aa"}.fa-arrow-circle-down:before{content:"\f0ab"}.fa-globe:before{content:"\f0ac"}.fa-wrench:before{content:"\f0ad"}.fa-tasks:before{content:"\f0ae"}.fa-filter:before{content:"\f0b0"}.fa-briefcase:before{content:"\f0b1"}.fa-arrows-alt:before{content:"\f0b2"}.fa-group:before,.fa-users:before{content:"\f0c0"}.fa-chain:before,.fa-link:before{content:"\f0c1"}.fa-cloud:before{content:"\f0c2"}.fa-flask:before{content:"\f0c3"}.fa-cut:before,.fa-scissors:before{content:"\f0c4"}.fa-copy:before,.fa-files-o:before{content:"\f0c5"}.fa-paperclip:before{content:"\f0c6"}.fa-save:before,.fa-floppy-o:before{content:"\f0c7"}.fa-square:before{content:"\f0c8"}.fa-navicon:before,.fa-reorder:before,.fa-bars:before{content:"\f0c9"}.fa-list-ul:before{content:"\f0ca"}.fa-list-ol:before{content:"\f0cb"}.fa-strikethrough:before{content:"\f0cc"}.fa-underline:before{content:"\f0cd"}.fa-table:before{content:"\f0ce"}.fa-magic:before{content:"\f0d0"}.fa-truck:before{content:"\f0d1"}.fa-pinterest:before{content:"\f0d2"}.fa-pinterest-square:before{content:"\f0d3"}.fa-google-plus-square:before{content:"\f0d4"}.fa-google-plus:before{content:"\f0d5"}.fa-money:before{content:"\f0d6"}.fa-caret-down:before{content:"\f0d7"}.fa-caret-up:before{content:"\f0d8"}.fa-caret-left:before{content:"\f0d9"}.fa-caret-right:before{content:"\f0da"}.fa-columns:before{content:"\f0db"}.fa-unsorted:before,.fa-sort:before{content:"\f0dc"}.fa-sort-down:before,.fa-sort-desc:before{content:"\f0dd"}.fa-sort-up:before,.fa-sort-asc:before{content:"\f0de"}.fa-envelope:before{content:"\f0e0"}.fa-linkedin:before{content:"\f0e1"}.fa-rotate-left:before,.fa-undo:before{content:"\f0e2"}.fa-legal:before,.fa-gavel:before{content:"\f0e3"}.fa-dashboard:before,.fa-tachometer:before{content:"\f0e4"}.fa-comment-o:before{content:"\f0e5"}.fa-comments-o:before{content:"\f0e6"}.fa-flash:before,.fa-bolt:before{content:"\f0e7"}.fa-sitemap:before{content:"\f0e8"}.fa-umbrella:before{content:"\f0e9"}.fa-paste:before,.fa-clipboard:before{content:"\f0ea"}.fa-lightbulb-o:before{content:"\f0eb"}.fa-exchange:before{content:"\f0ec"}.fa-cloud-download:before{content:"\f0ed"}.fa-cloud-upload:before{content:"\f0ee"}.fa-user-md:before{content:"\f0f0"}.fa-stethoscope:before{content:"\f0f1"}.fa-suitcase:before{content:"\f0f2"}.fa-bell-o:before{content:"\f0a2"}.fa-coffee:before{content:"\f0f4"}.fa-cutlery:before{content:"\f0f5"}.fa-file-text-o:before{content:"\f0f6"}.fa-building-o:before{content:"\f0f7"}.fa-hospital-o:before{content:"\f0f8"}.fa-ambulance:before{content:"\f0f9"}.fa-medkit:before{content:"\f0fa"}.fa-fighter-jet:before{content:"\f0fb"}.fa-beer:before{content:"\f0fc"}.fa-h-square:before{content:"\f0fd"}.fa-plus-square:before{content:"\f0fe"}.fa-angle-double-left:before{content:"\f100"}.fa-angle-double-right:before{content:"\f101"}.fa-angle-double-up:before{content:"\f102"}.fa-angle-double-down:before{content:"\f103"}.fa-angle-left:before{content:"\f104"}.fa-angle-right:before{content:"\f105"}.fa-angle-up:before{content:"\f106"}.fa-angle-down:before{content:"\f107"}.fa-desktop:before{content:"\f108"}.fa-laptop:before{content:"\f109"}.fa-tablet:before{content:"\f10a"}.fa-mobile-phone:before,.fa-mobile:before{content:"\f10b"}.fa-circle-o:before{content:"\f10c"}.fa-quote-left:before{content:"\f10d"}.fa-quote-right:before{content:"\f10e"}.fa-spinner:before{content:"\f110"}.fa-circle:before{content:"\f111"}.fa-mail-reply:before,.fa-reply:before{content:"\f112"}.fa-github-alt:before{content:"\f113"}.fa-folder-o:before{content:"\f114"}.fa-folder-open-o:before{content:"\f115"}.fa-smile-o:before{content:"\f118"}.fa-frown-o:before{content:"\f119"}.fa-meh-o:before{content:"\f11a"}.fa-gamepad:before{content:"\f11b"}.fa-keyboard-o:before{content:"\f11c"}.fa-flag-o:before{content:"\f11d"}.fa-flag-checkered:before{content:"\f11e"}.fa-terminal:before{content:"\f120"}.fa-code:before{content:"\f121"}.fa-mail-reply-all:before,.fa-reply-all:before{content:"\f122"}.fa-star-half-empty:before,.fa-star-half-full:before,.fa-star-half-o:before{content:"\f123"}.fa-location-arrow:before{content:"\f124"}.fa-crop:before{content:"\f125"}.fa-code-fork:before{content:"\f126"}.fa-unlink:before,.fa-chain-broken:before{content:"\f127"}.fa-question:before{content:"\f128"}.fa-info:before{content:"\f129"}.fa-exclamation:before{content:"\f12a"}.fa-superscript:before{content:"\f12b"}.fa-subscript:before{content:"\f12c"}.fa-eraser:before{content:"\f12d"}.fa-puzzle-piece:before{content:"\f12e"}.fa-microphone:before{content:"\f130"}.fa-microphone-slash:before{content:"\f131"}.fa-shield:before{content:"\f132"}.fa-calendar-o:before{content:"\f133"}.fa-fire-extinguisher:before{content:"\f134"}.fa-rocket:before{content:"\f135"}.fa-maxcdn:before{content:"\f136"}.fa-chevron-circle-left:before{content:"\f137"}.fa-chevron-circle-right:before{content:"\f138"}.fa-chevron-circle-up:before{content:"\f139"}.fa-chevron-circle-down:before{content:"\f13a"}.fa-html5:before{content:"\f13b"}.fa-css3:before{content:"\f13c"}.fa-anchor:before{content:"\f13d"}.fa-unlock-alt:before{content:"\f13e"}.fa-bullseye:before{content:"\f140"}.fa-ellipsis-h:before{content:"\f141"}.fa-ellipsis-v:before{content:"\f142"}.fa-rss-square:before{content:"\f143"}.fa-play-circle:before{content:"\f144"}.fa-ticket:before{content:"\f145"}.fa-minus-square:before{content:"\f146"}.fa-minus-square-o:before{content:"\f147"}.fa-level-up:before{content:"\f148"}.fa-level-down:before{content:"\f149"}.fa-check-square:before{content:"\f14a"}.fa-pencil-square:before{content:"\f14b"}.fa-external-link-square:before{content:"\f14c"}.fa-share-square:before{content:"\f14d"}.fa-compass:before{content:"\f14e"}.fa-toggle-down:before,.fa-caret-square-o-down:before{content:"\f150"}.fa-toggle-up:before,.fa-caret-square-o-up:before{content:"\f151"}.fa-toggle-right:before,.fa-caret-square-o-right:before{content:"\f152"}.fa-euro:before,.fa-eur:before{content:"\f153"}.fa-gbp:before{content:"\f154"}.fa-dollar:before,.fa-usd:before{content:"\f155"}.fa-rupee:before,.fa-inr:before{content:"\f156"}.fa-cny:before,.fa-rmb:before,.fa-yen:before,.fa-jpy:before{content:"\f157"}.fa-ruble:before,.fa-rouble:before,.fa-rub:before{content:"\f158"}.fa-won:before,.fa-krw:before{content:"\f159"}.fa-bitcoin:before,.fa-btc:before{content:"\f15a"}.fa-file:before{content:"\f15b"}.fa-file-text:before{content:"\f15c"}.fa-sort-alpha-asc:before{content:"\f15d"}.fa-sort-alpha-desc:before{content:"\f15e"}.fa-sort-amount-asc:before{content:"\f160"}.fa-sort-amount-desc:before{content:"\f161"}.fa-sort-numeric-asc:before{content:"\f162"}.fa-sort-numeric-desc:before{content:"\f163"}.fa-thumbs-up:before{content:"\f164"}.fa-thumbs-down:before{content:"\f165"}.fa-youtube-square:before{content:"\f166"}.fa-youtube:before{content:"\f167"}.fa-xing:before{content:"\f168"}.fa-xing-square:before{content:"\f169"}.fa-youtube-play:before{content:"\f16a"}.fa-dropbox:before{content:"\f16b"}.fa-stack-overflow:before{content:"\f16c"}.fa-instagram:before{content:"\f16d"}.fa-flickr:before{content:"\f16e"}.fa-adn:before{content:"\f170"}.fa-bitbucket:before{content:"\f171"}.fa-bitbucket-square:before{content:"\f172"}.fa-tumblr:before{content:"\f173"}.fa-tumblr-square:before{content:"\f174"}.fa-long-arrow-down:before{content:"\f175"}.fa-long-arrow-up:before{content:"\f176"}.fa-long-arrow-left:before{content:"\f177"}.fa-long-arrow-right:before{content:"\f178"}.fa-apple:before{content:"\f179"}.fa-windows:before{content:"\f17a"}.fa-android:before{content:"\f17b"}.fa-linux:before{content:"\f17c"}.fa-dribbble:before{content:"\f17d"}.fa-skype:before{content:"\f17e"}.fa-foursquare:before{content:"\f180"}.fa-trello:before{content:"\f181"}.fa-female:before{content:"\f182"}.fa-male:before{content:"\f183"}.fa-gittip:before,.fa-gratipay:before{content:"\f184"}.fa-sun-o:before{content:"\f185"}.fa-moon-o:before{content:"\f186"}.fa-archive:before{content:"\f187"}.fa-bug:before{content:"\f188"}.fa-vk:before{content:"\f189"}.fa-weibo:before{content:"\f18a"}.fa-renren:before{content:"\f18b"}.fa-pagelines:before{content:"\f18c"}.fa-stack-exchange:before{content:"\f18d"}.fa-arrow-circle-o-right:before{content:"\f18e"}.fa-arrow-circle-o-left:before{content:"\f190"}.fa-toggle-left:before,.fa-caret-square-o-left:before{content:"\f191"}.fa-dot-circle-o:before{content:"\f192"}.fa-wheelchair:before{content:"\f193"}.fa-vimeo-square:before{content:"\f194"}.fa-turkish-lira:before,.fa-try:before{content:"\f195"}.fa-plus-square-o:before{content:"\f196"}.fa-space-shuttle:before{content:"\f197"}.fa-slack:before{content:"\f198"}.fa-envelope-square:before{content:"\f199"}.fa-wordpress:before{content:"\f19a"}.fa-openid:before{content:"\f19b"}.fa-institution:before,.fa-bank:before,.fa-university:before{content:"\f19c"}.fa-mortar-board:before,.fa-graduation-cap:before{content:"\f19d"}.fa-yahoo:before{content:"\f19e"}.fa-google:before{content:"\f1a0"}.fa-reddit:before{content:"\f1a1"}.fa-reddit-square:before{content:"\f1a2"}.fa-stumbleupon-circle:before{content:"\f1a3"}.fa-stumbleupon:before{content:"\f1a4"}.fa-delicious:before{content:"\f1a5"}.fa-digg:before{content:"\f1a6"}.fa-pied-piper-pp:before{content:"\f1a7"}.fa-pied-piper-alt:before{content:"\f1a8"}.fa-drupal:before{content:"\f1a9"}.fa-joomla:before{content:"\f1aa"}.fa-language:before{content:"\f1ab"}.fa-fax:before{content:"\f1ac"}.fa-building:before{content:"\f1ad"}.fa-child:before{content:"\f1ae"}.fa-paw:before{content:"\f1b0"}.fa-spoon:before{content:"\f1b1"}.fa-cube:before{content:"\f1b2"}.fa-cubes:before{content:"\f1b3"}.fa-behance:before{content:"\f1b4"}.fa-behance-square:before{content:"\f1b5"}.fa-steam:before{content:"\f1b6"}.fa-steam-square:before{content:"\f1b7"}.fa-recycle:before{content:"\f1b8"}.fa-automobile:before,.fa-car:before{content:"\f1b9"}.fa-cab:before,.fa-taxi:before{content:"\f1ba"}.fa-tree:before{content:"\f1bb"}.fa-spotify:before{content:"\f1bc"}.fa-deviantart:before{content:"\f1bd"}.fa-soundcloud:before{content:"\f1be"}.fa-database:before{content:"\f1c0"}.fa-file-pdf-o:before{content:"\f1c1"}.fa-file-word-o:before{content:"\f1c2"}.fa-file-excel-o:before{content:"\f1c3"}.fa-file-powerpoint-o:before{content:"\f1c4"}.fa-file-photo-o:before,.fa-file-picture-o:before,.fa-file-image-o:before{content:"\f1c5"}.fa-file-zip-o:before,.fa-file-archive-o:before{content:"\f1c6"}.fa-file-sound-o:before,.fa-file-audio-o:before{content:"\f1c7"}.fa-file-movie-o:before,.fa-file-video-o:before{content:"\f1c8"}.fa-file-code-o:before{content:"\f1c9"}.fa-vine:before{content:"\f1ca"}.fa-codepen:before{content:"\f1cb"}.fa-jsfiddle:before{content:"\f1cc"}.fa-life-bouy:before,.fa-life-buoy:before,.fa-life-saver:before,.fa-support:before,.fa-life-ring:before{content:"\f1cd"}.fa-circle-o-notch:before{content:"\f1ce"}.fa-ra:before,.fa-resistance:before,.fa-rebel:before{content:"\f1d0"}.fa-ge:before,.fa-empire:before{content:"\f1d1"}.fa-git-square:before{content:"\f1d2"}.fa-git:before{content:"\f1d3"}.fa-y-combinator-square:before,.fa-yc-square:before,.fa-hacker-news:before{content:"\f1d4"}.fa-tencent-weibo:before{content:"\f1d5"}.fa-qq:before{content:"\f1d6"}.fa-wechat:before,.fa-weixin:before{content:"\f1d7"}.fa-send:before,.fa-paper-plane:before{content:"\f1d8"}.fa-send-o:before,.fa-paper-plane-o:before{content:"\f1d9"}.fa-history:before{content:"\f1da"}.fa-circle-thin:before{content:"\f1db"}.fa-header:before{content:"\f1dc"}.fa-paragraph:before{content:"\f1dd"}.fa-sliders:before{content:"\f1de"}.fa-share-alt:before{content:"\f1e0"}.fa-share-alt-square:before{content:"\f1e1"}.fa-bomb:before{content:"\f1e2"}.fa-soccer-ball-o:before,.fa-futbol-o:before{content:"\f1e3"}.fa-tty:before{content:"\f1e4"}.fa-binoculars:before{content:"\f1e5"}.fa-plug:before{content:"\f1e6"}.fa-slideshare:before{content:"\f1e7"}.fa-twitch:before{content:"\f1e8"}.fa-yelp:before{content:"\f1e9"}.fa-newspaper-o:before{content:"\f1ea"}.fa-wifi:before{content:"\f1eb"}.fa-calculator:before{content:"\f1ec"}.fa-paypal:before{content:"\f1ed"}.fa-google-wallet:before{content:"\f1ee"}.fa-cc-visa:before{content:"\f1f0"}.fa-cc-mastercard:before{content:"\f1f1"}.fa-cc-discover:before{content:"\f1f2"}.fa-cc-amex:before{content:"\f1f3"}.fa-cc-paypal:before{content:"\f1f4"}.fa-cc-stripe:before{content:"\f1f5"}.fa-bell-slash:before{content:"\f1f6"}.fa-bell-slash-o:before{content:"\f1f7"}.fa-trash:before{content:"\f1f8"}.fa-copyright:before{content:"\f1f9"}.fa-at:before{content:"\f1fa"}.fa-eyedropper:before{content:"\f1fb"}.fa-paint-brush:before{content:"\f1fc"}.fa-birthday-cake:before{content:"\f1fd"}.fa-area-chart:before{content:"\f1fe"}.fa-pie-chart:before{content:"\f200"}.fa-line-chart:before{content:"\f201"}.fa-lastfm:before{content:"\f202"}.fa-lastfm-square:before{content:"\f203"}.fa-toggle-off:before{content:"\f204"}.fa-toggle-on:before{content:"\f205"}.fa-bicycle:before{content:"\f206"}.fa-bus:before{content:"\f207"}.fa-ioxhost:before{content:"\f208"}.fa-angellist:before{content:"\f209"}.fa-cc:before{content:"\f20a"}.fa-shekel:before,.fa-sheqel:before,.fa-ils:before{content:"\f20b"}.fa-meanpath:before{content:"\f20c"}.fa-buysellads:before{content:"\f20d"}.fa-connectdevelop:before{content:"\f20e"}.fa-dashcube:before{content:"\f210"}.fa-forumbee:before{content:"\f211"}.fa-leanpub:before{content:"\f212"}.fa-sellsy:before{content:"\f213"}.fa-shirtsinbulk:before{content:"\f214"}.fa-simplybuilt:before{content:"\f215"}.fa-skyatlas:before{content:"\f216"}.fa-cart-plus:before{content:"\f217"}.fa-cart-arrow-down:before{content:"\f218"}.fa-diamond:before{content:"\f219"}.fa-ship:before{content:"\f21a"}.fa-user-secret:before{content:"\f21b"}.fa-motorcycle:before{content:"\f21c"}.fa-street-view:before{content:"\f21d"}.fa-heartbeat:before{content:"\f21e"}.fa-venus:before{content:"\f221"}.fa-mars:before{content:"\f222"}.fa-mercury:before{content:"\f223"}.fa-intersex:before,.fa-transgender:before{content:"\f224"}.fa-transgender-alt:before{content:"\f225"}.fa-venus-double:before{content:"\f226"}.fa-mars-double:before{content:"\f227"}.fa-venus-mars:before{content:"\f228"}.fa-mars-stroke:before{content:"\f229"}.fa-mars-stroke-v:before{content:"\f22a"}.fa-mars-stroke-h:before{content:"\f22b"}.fa-neuter:before{content:"\f22c"}.fa-genderless:before{content:"\f22d"}.fa-facebook-official:before{content:"\f230"}.fa-pinterest-p:before{content:"\f231"}.fa-whatsapp:before{content:"\f232"}.fa-server:before{content:"\f233"}.fa-user-plus:before{content:"\f234"}.fa-user-times:before{content:"\f235"}.fa-hotel:before,.fa-bed:before{content:"\f236"}.fa-viacoin:before{content:"\f237"}.fa-train:before{content:"\f238"}.fa-subway:before{content:"\f239"}.fa-medium:before{content:"\f23a"}.fa-yc:before,.fa-y-combinator:before{content:"\f23b"}.fa-optin-monster:before{content:"\f23c"}.fa-opencart:before{content:"\f23d"}.fa-expeditedssl:before{content:"\f23e"}.fa-battery-4:before,.fa-battery:before,.fa-battery-full:before{content:"\f240"}.fa-battery-3:before,.fa-battery-three-quarters:before{content:"\f241"}.fa-battery-2:before,.fa-battery-half:before{content:"\f242"}.fa-battery-1:before,.fa-battery-quarter:before{content:"\f243"}.fa-battery-0:before,.fa-battery-empty:before{content:"\f244"}.fa-mouse-pointer:before{content:"\f245"}.fa-i-cursor:before{content:"\f246"}.fa-object-group:before{content:"\f247"}.fa-object-ungroup:before{content:"\f248"}.fa-sticky-note:before{content:"\f249"}.fa-sticky-note-o:before{content:"\f24a"}.fa-cc-jcb:before{content:"\f24b"}.fa-cc-diners-club:before{content:"\f24c"}.fa-clone:before{content:"\f24d"}.fa-balance-scale:before{content:"\f24e"}.fa-hourglass-o:before{content:"\f250"}.fa-hourglass-1:before,.fa-hourglass-start:before{content:"\f251"}.fa-hourglass-2:before,.fa-hourglass-half:before{content:"\f252"}.fa-hourglass-3:before,.fa-hourglass-end:before{content:"\f253"}.fa-hourglass:before{content:"\f254"}.fa-hand-grab-o:before,.fa-hand-rock-o:before{content:"\f255"}.fa-hand-stop-o:before,.fa-hand-paper-o:before{content:"\f256"}.fa-hand-scissors-o:before{content:"\f257"}.fa-hand-lizard-o:before{content:"\f258"}.fa-hand-spock-o:before{content:"\f259"}.fa-hand-pointer-o:before{content:"\f25a"}.fa-hand-peace-o:before{content:"\f25b"}.fa-trademark:before{content:"\f25c"}.fa-registered:before{content:"\f25d"}.fa-creative-commons:before{content:"\f25e"}.fa-gg:before{content:"\f260"}.fa-gg-circle:before{content:"\f261"}.fa-tripadvisor:before{content:"\f262"}.fa-odnoklassniki:before{content:"\f263"}.fa-odnoklassniki-square:before{content:"\f264"}.fa-get-pocket:before{content:"\f265"}.fa-wikipedia-w:before{content:"\f266"}.fa-safari:before{content:"\f267"}.fa-chrome:before{content:"\f268"}.fa-firefox:before{content:"\f269"}.fa-opera:before{content:"\f26a"}.fa-internet-explorer:before{content:"\f26b"}.fa-tv:before,.fa-television:before{content:"\f26c"}.fa-contao:before{content:"\f26d"}.fa-500px:before{content:"\f26e"}.fa-amazon:before{content:"\f270"}.fa-calendar-plus-o:before{content:"\f271"}.fa-calendar-minus-o:before{content:"\f272"}.fa-calendar-times-o:before{content:"\f273"}.fa-calendar-check-o:before{content:"\f274"}.fa-industry:before{content:"\f275"}.fa-map-pin:before{content:"\f276"}.fa-map-signs:before{content:"\f277"}.fa-map-o:before{content:"\f278"}.fa-map:before{content:"\f279"}.fa-commenting:before{content:"\f27a"}.fa-commenting-o:before{content:"\f27b"}.fa-houzz:before{content:"\f27c"}.fa-vimeo:before{content:"\f27d"}.fa-black-tie:before{content:"\f27e"}.fa-fonticons:before{content:"\f280"}.fa-reddit-alien:before{content:"\f281"}.fa-edge:before{content:"\f282"}.fa-credit-card-alt:before{content:"\f283"}.fa-codiepie:before{content:"\f284"}.fa-modx:before{content:"\f285"}.fa-fort-awesome:before{content:"\f286"}.fa-usb:before{content:"\f287"}.fa-product-hunt:before{content:"\f288"}.fa-mixcloud:before{content:"\f289"}.fa-scribd:before{content:"\f28a"}.fa-pause-circle:before{content:"\f28b"}.fa-pause-circle-o:before{content:"\f28c"}.fa-stop-circle:before{content:"\f28d"}.fa-stop-circle-o:before{content:"\f28e"}.fa-shopping-bag:before{content:"\f290"}.fa-shopping-basket:before{content:"\f291"}.fa-hashtag:before{content:"\f292"}.fa-bluetooth:before{content:"\f293"}.fa-bluetooth-b:before{content:"\f294"}.fa-percent:before{content:"\f295"}.fa-gitlab:before{content:"\f296"}.fa-wpbeginner:before{content:"\f297"}.fa-wpforms:before{content:"\f298"}.fa-envira:before{content:"\f299"}.fa-universal-access:before{content:"\f29a"}.fa-wheelchair-alt:before{content:"\f29b"}.fa-question-circle-o:before{content:"\f29c"}.fa-blind:before{content:"\f29d"}.fa-audio-description:before{content:"\f29e"}.fa-volume-control-phone:before{content:"\f2a0"}.fa-braille:before{content:"\f2a1"}.fa-assistive-listening-systems:before{content:"\f2a2"}.fa-asl-interpreting:before,.fa-american-sign-language-interpreting:before{content:"\f2a3"}.fa-deafness:before,.fa-hard-of-hearing:before,.fa-deaf:before{content:"\f2a4"}.fa-glide:before{content:"\f2a5"}.fa-glide-g:before{content:"\f2a6"}.fa-signing:before,.fa-sign-language:before{content:"\f2a7"}.fa-low-vision:before{content:"\f2a8"}.fa-viadeo:before{content:"\f2a9"}.fa-viadeo-square:before{content:"\f2aa"}.fa-snapchat:before{content:"\f2ab"}.fa-snapchat-ghost:before{content:"\f2ac"}.fa-snapchat-square:before{content:"\f2ad"}.fa-pied-piper:before{content:"\f2ae"}.fa-first-order:before{content:"\f2b0"}.fa-yoast:before{content:"\f2b1"}.fa-themeisle:before{content:"\f2b2"}.fa-google-plus-circle:before,.fa-google-plus-official:before{content:"\f2b3"}.fa-fa:before,.fa-font-awesome:before{content:"\f2b4"}.fa-handshake-o:before{content:"\f2b5"}.fa-envelope-open:before{content:"\f2b6"}.fa-envelope-open-o:before{content:"\f2b7"}.fa-linode:before{content:"\f2b8"}.fa-address-book:before{content:"\f2b9"}.fa-address-book-o:before{content:"\f2ba"}.fa-vcard:before,.fa-address-card:before{content:"\f2bb"}.fa-vcard-o:before,.fa-address-card-o:before{content:"\f2bc"}.fa-user-circle:before{content:"\f2bd"}.fa-user-circle-o:before{content:"\f2be"}.fa-user-o:before{content:"\f2c0"}.fa-id-badge:before{content:"\f2c1"}.fa-drivers-license:before,.fa-id-card:before{content:"\f2c2"}.fa-drivers-license-o:before,.fa-id-card-o:before{content:"\f2c3"}.fa-quora:before{content:"\f2c4"}.fa-free-code-camp:before{content:"\f2c5"}.fa-telegram:before{content:"\f2c6"}.fa-thermometer-4:before,.fa-thermometer:before,.fa-thermometer-full:before{content:"\f2c7"}.fa-thermometer-3:before,.fa-thermometer-three-quarters:before{content:"\f2c8"}.fa-thermometer-2:before,.fa-thermometer-half:before{content:"\f2c9"}.fa-thermometer-1:before,.fa-thermometer-quarter:before{content:"\f2ca"}.fa-thermometer-0:before,.fa-thermometer-empty:before{content:"\f2cb"}.fa-shower:before{content:"\f2cc"}.fa-bathtub:before,.fa-s15:before,.fa-bath:before{content:"\f2cd"}.fa-podcast:before{content:"\f2ce"}.fa-window-maximize:before{content:"\f2d0"}.fa-window-minimize:before{content:"\f2d1"}.fa-window-restore:before{content:"\f2d2"}.fa-times-rectangle:before,.fa-window-close:before{content:"\f2d3"}.fa-times-rectangle-o:before,.fa-window-close-o:before{content:"\f2d4"}.fa-bandcamp:before{content:"\f2d5"}.fa-grav:before{content:"\f2d6"}.fa-etsy:before{content:"\f2d7"}.fa-imdb:before{content:"\f2d8"}.fa-ravelry:before{content:"\f2d9"}.fa-eercast:before{content:"\f2da"}.fa-microchip:before{content:"\f2db"}.fa-snowflake-o:before{content:"\f2dc"}.fa-superpowers:before{content:"\f2dd"}.fa-wpexplorer:before{content:"\f2de"}.fa-meetup:before{content:"\f2e0"}.sr-only{position:absolute;width:1px;height:1px;padding:0;margin:-1px;overflow:hidden;clip:rect(0,0,0,0);border:0}.sr-only-focusable:active,.sr-only-focusable:focus{position:static;width:auto;height:auto;margin:0;overflow:visible;clip:auto} \ No newline at end of file diff --git a/430/_static/fonts/material-icons.css b/430/_static/fonts/material-icons.css new file mode 100644 index 000000000..63130b012 --- /dev/null +++ b/430/_static/fonts/material-icons.css @@ -0,0 +1,13 @@ +/*! + * Licensed 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. + */@font-face{font-display:swap;font-family:"Material Icons";font-style:normal;font-weight:400;src:local("Material Icons"),local("MaterialIcons-Regular"),url("specimen/MaterialIcons-Regular.woff2") format("woff2"),url("specimen/MaterialIcons-Regular.woff") format("woff"),url("specimen/MaterialIcons-Regular.ttf") format("truetype")} \ No newline at end of file diff --git a/430/_static/fonts/specimen/FontAwesome.ttf b/430/_static/fonts/specimen/FontAwesome.ttf new file mode 100644 index 000000000..35acda2fa Binary files /dev/null and b/430/_static/fonts/specimen/FontAwesome.ttf differ diff --git a/430/_static/fonts/specimen/FontAwesome.woff b/430/_static/fonts/specimen/FontAwesome.woff new file mode 100644 index 000000000..400014a4b Binary files /dev/null and b/430/_static/fonts/specimen/FontAwesome.woff differ diff --git a/430/_static/fonts/specimen/FontAwesome.woff2 b/430/_static/fonts/specimen/FontAwesome.woff2 new file mode 100644 index 000000000..4d13fc604 Binary files /dev/null and b/430/_static/fonts/specimen/FontAwesome.woff2 differ diff --git a/430/_static/fonts/specimen/MaterialIcons-Regular.ttf b/430/_static/fonts/specimen/MaterialIcons-Regular.ttf new file mode 100644 index 000000000..7015564ad Binary files /dev/null and b/430/_static/fonts/specimen/MaterialIcons-Regular.ttf differ diff --git a/430/_static/fonts/specimen/MaterialIcons-Regular.woff b/430/_static/fonts/specimen/MaterialIcons-Regular.woff new file mode 100644 index 000000000..b648a3eea Binary files /dev/null and b/430/_static/fonts/specimen/MaterialIcons-Regular.woff differ diff --git a/430/_static/fonts/specimen/MaterialIcons-Regular.woff2 b/430/_static/fonts/specimen/MaterialIcons-Regular.woff2 new file mode 100644 index 000000000..9fa211252 Binary files /dev/null and b/430/_static/fonts/specimen/MaterialIcons-Regular.woff2 differ diff --git a/430/_static/images/favicon.png b/430/_static/images/favicon.png new file mode 100644 index 000000000..76d17f57a Binary files /dev/null and b/430/_static/images/favicon.png differ diff --git a/430/_static/images/icons/bitbucket.1b09e088.svg b/430/_static/images/icons/bitbucket.1b09e088.svg new file mode 100644 index 000000000..cf58c14fb --- /dev/null +++ b/430/_static/images/icons/bitbucket.1b09e088.svg @@ -0,0 +1 @@ + \ No newline at end of file diff --git a/430/_static/images/icons/bitbucket.svg b/430/_static/images/icons/bitbucket.svg new file mode 100644 index 000000000..cf58c14fb --- /dev/null +++ b/430/_static/images/icons/bitbucket.svg @@ -0,0 +1 @@ + \ No newline at end of file diff --git a/430/_static/images/icons/github.f0b8504a.svg b/430/_static/images/icons/github.f0b8504a.svg new file mode 100644 index 000000000..3d13b1975 --- /dev/null +++ b/430/_static/images/icons/github.f0b8504a.svg @@ -0,0 +1 @@ + \ No newline at end of file diff --git a/430/_static/images/icons/github.svg b/430/_static/images/icons/github.svg new file mode 100644 index 000000000..3d13b1975 --- /dev/null +++ b/430/_static/images/icons/github.svg @@ -0,0 +1 @@ + \ No newline at end of file diff --git a/430/_static/images/icons/gitlab.6dd19c00.svg b/430/_static/images/icons/gitlab.6dd19c00.svg new file mode 100644 index 000000000..1d9fffa74 --- /dev/null +++ b/430/_static/images/icons/gitlab.6dd19c00.svg @@ -0,0 +1 @@ + \ No newline at end of file diff --git a/430/_static/images/icons/gitlab.svg b/430/_static/images/icons/gitlab.svg new file mode 100644 index 000000000..1d9fffa74 --- /dev/null +++ b/430/_static/images/icons/gitlab.svg @@ -0,0 +1 @@ + \ No newline at end of file diff --git a/430/_static/img/accumulo.png b/430/_static/img/accumulo.png new file mode 100644 index 000000000..2e5430bf2 Binary files /dev/null and b/430/_static/img/accumulo.png differ diff --git a/430/_static/img/bigquery.png b/430/_static/img/bigquery.png new file mode 100644 index 000000000..f11683ee7 Binary files /dev/null and b/430/_static/img/bigquery.png differ diff --git a/430/_static/img/cassandra.png b/430/_static/img/cassandra.png new file mode 100644 index 000000000..833381c55 Binary files /dev/null and b/430/_static/img/cassandra.png differ diff --git a/430/_static/img/clickhouse.png b/430/_static/img/clickhouse.png new file mode 100644 index 000000000..7f8a2badd Binary files /dev/null and b/430/_static/img/clickhouse.png differ diff --git a/430/_static/img/delta-lake.png b/430/_static/img/delta-lake.png new file mode 100644 index 000000000..df28c59ee Binary files /dev/null and b/430/_static/img/delta-lake.png differ diff --git a/430/_static/img/druid.png b/430/_static/img/druid.png new file mode 100644 index 000000000..fb8ad5f67 Binary files /dev/null and b/430/_static/img/druid.png differ diff --git a/430/_static/img/elasticsearch.png b/430/_static/img/elasticsearch.png new file mode 100644 index 000000000..38f54bd32 Binary files /dev/null and b/430/_static/img/elasticsearch.png differ diff --git a/430/_static/img/google-sheets.png b/430/_static/img/google-sheets.png new file mode 100644 index 000000000..f7843e2e7 Binary files /dev/null and b/430/_static/img/google-sheets.png differ diff --git a/430/_static/img/hive.png b/430/_static/img/hive.png new file mode 100644 index 000000000..22b84870f Binary files /dev/null and b/430/_static/img/hive.png differ diff --git a/430/_static/img/hudi.png b/430/_static/img/hudi.png new file mode 100644 index 000000000..96ae05e39 Binary files /dev/null and b/430/_static/img/hudi.png differ diff --git a/430/_static/img/iceberg.png b/430/_static/img/iceberg.png new file mode 100644 index 000000000..08f887321 Binary files /dev/null and b/430/_static/img/iceberg.png differ diff --git a/430/_static/img/ignite.png b/430/_static/img/ignite.png new file mode 100644 index 000000000..7435f0173 Binary files /dev/null and b/430/_static/img/ignite.png differ diff --git a/430/_static/img/kafka.png b/430/_static/img/kafka.png new file mode 100644 index 000000000..336958040 Binary files /dev/null and b/430/_static/img/kafka.png differ diff --git a/430/_static/img/kinesis.png b/430/_static/img/kinesis.png new file mode 100644 index 000000000..7f5619032 Binary files /dev/null and b/430/_static/img/kinesis.png differ diff --git a/430/_static/img/kudu.png b/430/_static/img/kudu.png new file mode 100644 index 000000000..99f3f3dd1 Binary files /dev/null and b/430/_static/img/kudu.png differ diff --git a/430/_static/img/mariadb.png b/430/_static/img/mariadb.png new file mode 100644 index 000000000..b57019d21 Binary files /dev/null and b/430/_static/img/mariadb.png differ diff --git a/430/_static/img/mongodb.png b/430/_static/img/mongodb.png new file mode 100644 index 000000000..607cba323 Binary files /dev/null and b/430/_static/img/mongodb.png differ diff --git a/430/_static/img/mysql.png b/430/_static/img/mysql.png new file mode 100644 index 000000000..d2e345bcc Binary files /dev/null and b/430/_static/img/mysql.png differ diff --git a/430/_static/img/oracle.png b/430/_static/img/oracle.png new file mode 100644 index 000000000..067a2da5c Binary files /dev/null and b/430/_static/img/oracle.png differ diff --git a/430/_static/img/phoenix.png b/430/_static/img/phoenix.png new file mode 100644 index 000000000..0b886afba Binary files /dev/null and b/430/_static/img/phoenix.png differ diff --git a/430/_static/img/pinot.png b/430/_static/img/pinot.png new file mode 100644 index 000000000..ad9d8c423 Binary files /dev/null and b/430/_static/img/pinot.png differ diff --git a/430/_static/img/postgresql.png b/430/_static/img/postgresql.png new file mode 100644 index 000000000..b15485dc5 Binary files /dev/null and b/430/_static/img/postgresql.png differ diff --git a/430/_static/img/prometheus.png b/430/_static/img/prometheus.png new file mode 100644 index 000000000..94ed675ad Binary files /dev/null and b/430/_static/img/prometheus.png differ diff --git a/430/_static/img/redis.png b/430/_static/img/redis.png new file mode 100644 index 000000000..23aa10a80 Binary files /dev/null and b/430/_static/img/redis.png differ diff --git a/430/_static/img/redshift.png b/430/_static/img/redshift.png new file mode 100644 index 000000000..de9ccabfb Binary files /dev/null and b/430/_static/img/redshift.png differ diff --git a/430/_static/img/singlestore.png b/430/_static/img/singlestore.png new file mode 100644 index 000000000..015ec81bf Binary files /dev/null and b/430/_static/img/singlestore.png differ diff --git a/430/_static/img/sqlserver.png b/430/_static/img/sqlserver.png new file mode 100644 index 000000000..79736a5bd Binary files /dev/null and b/430/_static/img/sqlserver.png differ diff --git a/430/_static/javascripts/application.js b/430/_static/javascripts/application.js new file mode 100644 index 000000000..7c724d2e4 --- /dev/null +++ b/430/_static/javascripts/application.js @@ -0,0 +1,2540 @@ +! function(e, t) { + for (var n in t) e[n] = t[n] +}(window, function(n) { + var r = {}; + + function i(e) { + if (r[e]) return r[e].exports; + var t = r[e] = { + i: e, + l: !1, + exports: {} + }; + return n[e].call(t.exports, t, t.exports, i), t.l = !0, t.exports + } + return i.m = n, i.c = r, i.d = function(e, t, n) { + i.o(e, t) || Object.defineProperty(e, t, { + enumerable: !0, + get: n + }) + }, i.r = function(e) { + "undefined" != typeof Symbol && Symbol.toStringTag && Object.defineProperty(e, Symbol.toStringTag, { + value: "Module" + }), Object.defineProperty(e, "__esModule", { + value: !0 + }) + }, i.t = function(t, e) { + if (1 & e && (t = i(t)), 8 & e) return t; + if (4 & e && "object" == typeof t && t && t.__esModule) return t; + var n = Object.create(null); + if (i.r(n), Object.defineProperty(n, "default", { + enumerable: !0, + value: t + }), 2 & e && "string" != typeof t) + for (var r in t) i.d(n, r, function(e) { + return t[e] + }.bind(null, r)); + return n + }, i.n = function(e) { + var t = e && e.__esModule ? function() { + return e.default + } : function() { + return e + }; + return i.d(t, "a", t), t + }, i.o = function(e, t) { + return Object.prototype.hasOwnProperty.call(e, t) + }, i.p = "", i(i.s = 13) +}([function(e, t, n) { + "use strict"; + var r = { + Listener: function() { + function e(e, t, n) { + var r = this; + this.els_ = Array.prototype.slice.call("string" == typeof e ? document.querySelectorAll(e) : [].concat(e)), this.handler_ = "function" == typeof n ? { + update: n + } : n, this.events_ = [].concat(t), this.update_ = function(e) { + return r.handler_.update(e) + } + } + var t = e.prototype; + return t.listen = function() { + var n = this; + this.els_.forEach(function(t) { + n.events_.forEach(function(e) { + t.addEventListener(e, n.update_, !1) + }) + }), "function" == typeof this.handler_.setup && this.handler_.setup() + }, t.unlisten = function() { + var n = this; + this.els_.forEach(function(t) { + n.events_.forEach(function(e) { + t.removeEventListener(e, n.update_) + }) + }), "function" == typeof this.handler_.reset && this.handler_.reset() + }, e + }(), + MatchMedia: function(e, t) { + this.handler_ = function(e) { + e.matches ? t.listen() : t.unlisten() + }; + var n = window.matchMedia(e); + n.addListener(this.handler_), this.handler_(n) + } + }, + i = { + Shadow: function() { + function e(e, t) { + var n = "string" == typeof e ? document.querySelector(e) : e; + if (!(n instanceof HTMLElement && n.parentNode instanceof HTMLElement)) throw new ReferenceError; + if (this.el_ = n.parentNode, !((n = "string" == typeof t ? document.querySelector(t) : t) instanceof HTMLElement)) throw new ReferenceError; + this.header_ = n, this.height_ = 0, this.active_ = !1 + } + var t = e.prototype; + return t.setup = function() { + for (var e = this.el_; e = e.previousElementSibling;) { + if (!(e instanceof HTMLElement)) throw new ReferenceError; + this.height_ += e.offsetHeight + } + this.update() + }, t.update = function(e) { + if (!e || "resize" !== e.type && "orientationchange" !== e.type) { + var t = window.pageYOffset >= this.height_; + t !== this.active_ && (this.header_.dataset.mdState = (this.active_ = t) ? "shadow" : "") + } else this.height_ = 0, this.setup() + }, t.reset = function() { + this.header_.dataset.mdState = "", this.height_ = 0, this.active_ = !1 + }, e + }(), + Title: function() { + function e(e, t) { + var n = "string" == typeof e ? document.querySelector(e) : e; + if (!(n instanceof HTMLElement)) throw new ReferenceError; + if (this.el_ = n, !((n = "string" == typeof t ? document.querySelector(t) : t) instanceof HTMLHeadingElement)) throw new ReferenceError; + this.header_ = n, this.active_ = !1 + } + var t = e.prototype; + return t.setup = function() { + var t = this; + Array.prototype.forEach.call(this.el_.children, function(e) { + e.style.width = t.el_.offsetWidth - 20 + "px" + }) + }, t.update = function(e) { + var t = this, + n = window.pageYOffset >= this.header_.offsetTop; + n !== this.active_ && (this.el_.dataset.mdState = (this.active_ = n) ? "active" : ""), "resize" !== e.type && "orientationchange" !== e.type || Array.prototype.forEach.call(this.el_.children, function(e) { + e.style.width = t.el_.offsetWidth - 20 + "px" + }) + }, t.reset = function() { + this.el_.dataset.mdState = "", this.el_.style.width = "", this.active_ = !1 + }, e + }() + }, + o = { + Blur: function() { + function e(e) { + this.els_ = "string" == typeof e ? document.querySelectorAll(e) : e, this.index_ = 0, this.offset_ = window.pageYOffset, this.dir_ = !1, this.anchors_ = [].reduce.call(this.els_, function(e, t) { + var n = decodeURIComponent(t.hash); + return e.concat(document.getElementById(n.substring(1)) || []) + }, []) + } + var t = e.prototype; + return t.setup = function() { + this.update() + }, t.update = function() { + var e = window.pageYOffset, + t = this.offset_ - e < 0; + if (this.dir_ !== t && (this.index_ = this.index_ = t ? 0 : this.els_.length - 1), 0 !== this.anchors_.length) { + if (this.offset_ <= e) + for (var n = this.index_ + 1; n < this.els_.length && this.anchors_[n].offsetTop - 80 <= e; n++) 0 < n && (this.els_[n - 1].dataset.mdState = "blur"), this.index_ = n; + else + for (var r = this.index_; 0 <= r; r--) { + if (!(this.anchors_[r].offsetTop - 80 > e)) { + this.index_ = r; + break + } + 0 < r && (this.els_[r - 1].dataset.mdState = "") + } + this.offset_ = e, this.dir_ = t + } + }, t.reset = function() { + Array.prototype.forEach.call(this.els_, function(e) { + e.dataset.mdState = "" + }), this.index_ = 0, this.offset_ = window.pageYOffset + }, e + }(), + Collapse: function() { + function e(e) { + var t = "string" == typeof e ? document.querySelector(e) : e; + if (!(t instanceof HTMLElement)) throw new ReferenceError; + this.el_ = t + } + var t = e.prototype; + return t.setup = function() { + var e = this.el_.getBoundingClientRect().height; + this.el_.style.display = e ? "block" : "none", this.el_.style.overflow = e ? "visible" : "hidden" + }, t.update = function() { + var e = this, + t = this.el_.getBoundingClientRect().height; + this.el_.style.display = "block", this.el_.style.overflow = ""; + var r = this.el_.previousElementSibling.previousElementSibling.checked; + if (r) this.el_.style.maxHeight = t + "px", requestAnimationFrame(function() { + e.el_.setAttribute("data-md-state", "animate"), e.el_.style.maxHeight = "0px" + }); + else { + this.el_.setAttribute("data-md-state", "expand"), this.el_.style.maxHeight = ""; + var n = this.el_.getBoundingClientRect().height; + this.el_.removeAttribute("data-md-state"), this.el_.style.maxHeight = "0px", requestAnimationFrame(function() { + e.el_.setAttribute("data-md-state", "animate"), e.el_.style.maxHeight = n + "px" + }) + } + this.el_.addEventListener("transitionend", function e(t) { + var n = t.target; + if (!(n instanceof HTMLElement)) throw new ReferenceError; + n.removeAttribute("data-md-state"), n.style.maxHeight = "", n.style.display = r ? "none" : "block", n.style.overflow = r ? "hidden" : "visible", n.removeEventListener("transitionend", e) + }, !1) + }, t.reset = function() { + this.el_.dataset.mdState = "", this.el_.style.maxHeight = "", this.el_.style.display = "", this.el_.style.overflow = "" + }, e + }(), + Scrolling: function() { + function e(e) { + var t = "string" == typeof e ? document.querySelector(e) : e; + if (!(t instanceof HTMLElement)) throw new ReferenceError; + this.el_ = t + } + var t = e.prototype; + return t.setup = function() { + this.el_.children[this.el_.children.length - 1].style.webkitOverflowScrolling = "touch"; + var e = this.el_.querySelectorAll("[data-md-toggle]"); + Array.prototype.forEach.call(e, function(e) { + if (!(e instanceof HTMLInputElement)) throw new ReferenceError; + if (e.checked) { + var t = e.nextElementSibling; + if (!(t instanceof HTMLElement)) throw new ReferenceError; + for (; + "NAV" !== t.tagName && t.nextElementSibling;) t = t.nextElementSibling; + if (!(e.parentNode instanceof HTMLElement && e.parentNode.parentNode instanceof HTMLElement)) throw new ReferenceError; + var n = e.parentNode.parentNode, + r = t.children[t.children.length - 1]; + n.style.webkitOverflowScrolling = "", r.style.webkitOverflowScrolling = "touch" + } + }) + }, t.update = function(e) { + var t = e.target; + if (!(t instanceof HTMLElement)) throw new ReferenceError; + var n = t.nextElementSibling; + if (!(n instanceof HTMLElement)) throw new ReferenceError; + for (; + "NAV" !== n.tagName && n.nextElementSibling;) n = n.nextElementSibling; + if (!(t.parentNode instanceof HTMLElement && t.parentNode.parentNode instanceof HTMLElement)) throw new ReferenceError; + var r = t.parentNode.parentNode, + i = n.children[n.children.length - 1]; + if (r.style.webkitOverflowScrolling = "", i.style.webkitOverflowScrolling = "", !t.checked) { + n.addEventListener("transitionend", function e() { + n instanceof HTMLElement && (r.style.webkitOverflowScrolling = "touch", n.removeEventListener("transitionend", e)) + }, !1) + } + if (t.checked) { + n.addEventListener("transitionend", function e() { + n instanceof HTMLElement && (i.style.webkitOverflowScrolling = "touch", n.removeEventListener("transitionend", e)) + }, !1) + } + }, t.reset = function() { + this.el_.children[1].style.webkitOverflowScrolling = ""; + var e = this.el_.querySelectorAll("[data-md-toggle]"); + Array.prototype.forEach.call(e, function(e) { + if (!(e instanceof HTMLInputElement)) throw new ReferenceError; + if (e.checked) { + var t = e.nextElementSibling; + if (!(t instanceof HTMLElement)) throw new ReferenceError; + for (; + "NAV" !== t.tagName && t.nextElementSibling;) t = t.nextElementSibling; + if (!(e.parentNode instanceof HTMLElement && e.parentNode.parentNode instanceof HTMLElement)) throw new ReferenceError; + var n = e.parentNode.parentNode, + r = t.children[t.children.length - 1]; + n.style.webkitOverflowScrolling = "", r.style.webkitOverflowScrolling = "" + } + }) + }, e + }() + }, + a = { + Lock: function() { + function e(e) { + var t = "string" == typeof e ? document.querySelector(e) : e; + if (!(t instanceof HTMLInputElement)) throw new ReferenceError; + if (this.el_ = t, !document.body) throw new ReferenceError; + this.lock_ = document.body + } + var t = e.prototype; + return t.setup = function() { + this.update() + }, t.update = function() { + var e = this; + this.el_.checked ? (this.offset_ = window.pageYOffset, setTimeout(function() { + window.scrollTo(0, 0), e.el_.checked && (e.lock_.dataset.mdState = "lock") + }, 400)) : (this.lock_.dataset.mdState = "", setTimeout(function() { + void 0 !== e.offset_ && window.scrollTo(0, e.offset_) + }, 100)) + }, t.reset = function() { + "lock" === this.lock_.dataset.mdState && window.scrollTo(0, this.offset_), this.lock_.dataset.mdState = "" + }, e + }(), + Result: n(9).a + }, + s = { + Position: function() { + function e(e, t) { + var n = "string" == typeof e ? document.querySelector(e) : e; + if (!(n instanceof HTMLElement && n.parentNode instanceof HTMLElement)) throw new ReferenceError; + if (this.el_ = n, this.parent_ = n.parentNode, !((n = "string" == typeof t ? document.querySelector(t) : t) instanceof HTMLElement)) throw new ReferenceError; + this.header_ = n, this.height_ = 0, this.pad_ = "fixed" === window.getComputedStyle(this.header_).position + } + var t = e.prototype; + return t.setup = function() { + var e = Array.prototype.reduce.call(this.parent_.children, function(e, t) { + return Math.max(e, t.offsetTop) + }, 0); + this.offset_ = e - (this.pad_ ? this.header_.offsetHeight : 0), this.update() + }, t.update = function(e) { + var t = window.pageYOffset, + n = window.innerHeight; + e && "resize" === e.type && this.setup(); + var r = this.pad_ ? this.header_.offsetHeight : 0, + i = this.parent_.offsetTop + this.parent_.offsetHeight, + o = n - r - Math.max(0, this.offset_ - t) - Math.max(0, t + n - i); + o !== this.height_ && (this.el_.style.height = (this.height_ = o) + "px"), t >= this.offset_ ? "lock" !== this.el_.dataset.mdState && (this.el_.dataset.mdState = "lock") : "lock" === this.el_.dataset.mdState && (this.el_.dataset.mdState = "") + }, t.reset = function() { + this.el_.dataset.mdState = "", this.el_.style.height = "", this.height_ = 0 + }, e + }() + }, + c = n(6), + l = n.n(c); + var u = { + Adapter: { + GitHub: function(o) { + var e, t; + + function n(e) { + var t; + t = o.call(this, e) || this; + var n = /^.+github\.com\/([^/]+)\/?([^/]+)?.*$/.exec(t.base_); + if (n && 3 === n.length) { + var r = n[1], + i = n[2]; + t.base_ = "https://api.github.com/users/" + r + "/repos", t.name_ = i + } + return t + } + return t = o, (e = n).prototype = Object.create(t.prototype), (e.prototype.constructor = e).__proto__ = t, n.prototype.fetch_ = function() { + var i = this; + return function n(r) { + return void 0 === r && (r = 0), fetch(i.base_ + "?per_page=30&page=" + r).then(function(e) { + return e.json() + }).then(function(e) { + if (!(e instanceof Array)) throw new TypeError; + if (i.name_) { + var t = e.find(function(e) { + return e.name === i.name_ + }); + return t || 30 !== e.length ? t ? [i.format_(t.stargazers_count) + " Stars", i.format_(t.forks_count) + " Forks"] : [] : n(r + 1) + } + return [e.length + " Repositories"] + }) + }() + }, n + }(function() { + function e(e) { + var t = "string" == typeof e ? document.querySelector(e) : e; + if (!(t instanceof HTMLAnchorElement)) throw new ReferenceError; + this.el_ = t, this.base_ = this.el_.href, this.salt_ = this.hash_(this.base_) + } + var t = e.prototype; + return t.fetch = function() { + var n = this; + return new Promise(function(t) { + var e = l.a.getJSON(n.salt_ + ".cache-source"); + void 0 !== e ? t(e) : n.fetch_().then(function(e) { + l.a.set(n.salt_ + ".cache-source", e, { + expires: 1 / 96 + }), t(e) + }) + }) + }, t.fetch_ = function() { + throw new Error("fetch_(): Not implemented") + }, t.format_ = function(e) { + return 1e4 < e ? (e / 1e3).toFixed(0) + "k" : 1e3 < e ? (e / 1e3).toFixed(1) + "k" : "" + e + }, t.hash_ = function(e) { + var t = 0; + if (0 === e.length) return t; + for (var n = 0, r = e.length; n < r; n++) t = (t << 5) - t + e.charCodeAt(n), t |= 0; + return t + }, e + }()) + }, + Repository: n(10).a + }, + f = { + Toggle: function() { + function e(e) { + var t = "string" == typeof e ? document.querySelector(e) : e; + if (!(t instanceof Node)) throw new ReferenceError; + this.el_ = t; + var n = document.querySelector("[data-md-component=header]"); + this.height_ = n.offsetHeight, this.active_ = !1 + } + var t = e.prototype; + return t.update = function() { + var e = window.pageYOffset >= this.el_.children[0].offsetTop + (5 - this.height_); + e !== this.active_ && (this.el_.dataset.mdState = (this.active_ = e) ? "hidden" : "") + }, t.reset = function() { + this.el_.dataset.mdState = "", this.active_ = !1 + }, e + }() + }; + t.a = { + Event: r, + Header: i, + Nav: o, + Search: a, + Sidebar: s, + Source: u, + Tabs: f + } +}, function(t, e, n) { + (function(e) { + t.exports = e.lunr = n(24) + }).call(this, n(4)) +}, function(e, f, d) { + "use strict"; + (function(t) { + var e = d(8), + n = setTimeout; + + function r() {} + + function o(e) { + if (!(this instanceof o)) throw new TypeError("Promises must be constructed via new"); + if ("function" != typeof e) throw new TypeError("not a function"); + this._state = 0, this._handled = !1, this._value = void 0, this._deferreds = [], u(e, this) + } + + function i(n, r) { + for (; 3 === n._state;) n = n._value; + 0 !== n._state ? (n._handled = !0, o._immediateFn(function() { + var e = 1 === n._state ? r.onFulfilled : r.onRejected; + if (null !== e) { + var t; + try { + t = e(n._value) + } catch (e) { + return void s(r.promise, e) + } + a(r.promise, t) + } else(1 === n._state ? a : s)(r.promise, n._value) + })) : n._deferreds.push(r) + } + + function a(t, e) { + try { + if (e === t) throw new TypeError("A promise cannot be resolved with itself."); + if (e && ("object" == typeof e || "function" == typeof e)) { + var n = e.then; + if (e instanceof o) return t._state = 3, t._value = e, void c(t); + if ("function" == typeof n) return void u((r = n, i = e, function() { + r.apply(i, arguments) + }), t) + } + t._state = 1, t._value = e, c(t) + } catch (e) { + s(t, e) + } + var r, i + } + + function s(e, t) { + e._state = 2, e._value = t, c(e) + } + + function c(e) { + 2 === e._state && 0 === e._deferreds.length && o._immediateFn(function() { + e._handled || o._unhandledRejectionFn(e._value) + }); + for (var t = 0, n = e._deferreds.length; t < n; t++) i(e, e._deferreds[t]); + e._deferreds = null + } + + function l(e, t, n) { + this.onFulfilled = "function" == typeof e ? e : null, this.onRejected = "function" == typeof t ? t : null, this.promise = n + } + + function u(e, t) { + var n = !1; + try { + e(function(e) { + n || (n = !0, a(t, e)) + }, function(e) { + n || (n = !0, s(t, e)) + }) + } catch (e) { + if (n) return; + n = !0, s(t, e) + } + } + o.prototype.catch = function(e) { + return this.then(null, e) + }, o.prototype.then = function(e, t) { + var n = new this.constructor(r); + return i(this, new l(e, t, n)), n + }, o.prototype.finally = e.a, o.all = function(t) { + return new o(function(r, i) { + if (!t || void 0 === t.length) throw new TypeError("Promise.all accepts an array"); + var o = Array.prototype.slice.call(t); + if (0 === o.length) return r([]); + var a = o.length; + + function s(t, e) { + try { + if (e && ("object" == typeof e || "function" == typeof e)) { + var n = e.then; + if ("function" == typeof n) return void n.call(e, function(e) { + s(t, e) + }, i) + } + o[t] = e, 0 == --a && r(o) + } catch (e) { + i(e) + } + } + for (var e = 0; e < o.length; e++) s(e, o[e]) + }) + }, o.resolve = function(t) { + return t && "object" == typeof t && t.constructor === o ? t : new o(function(e) { + e(t) + }) + }, o.reject = function(n) { + return new o(function(e, t) { + t(n) + }) + }, o.race = function(i) { + return new o(function(e, t) { + for (var n = 0, r = i.length; n < r; n++) i[n].then(e, t) + }) + }, o._immediateFn = "function" == typeof t && function(e) { + t(e) + } || function(e) { + n(e, 0) + }, o._unhandledRejectionFn = function(e) { + "undefined" != typeof console && console && console.warn("Possible Unhandled Promise Rejection:", e) + }, f.a = o + }).call(this, d(21).setImmediate) +}, function(e, t, n) { + "use strict"; + + function r(e, t) { + var n = document.createElement(e); + t && Array.prototype.forEach.call(Object.keys(t), function(e) { + n.setAttribute(e, t[e]) + }); + for (var r = arguments.length, i = new Array(2 < r ? r - 2 : 0), o = 2; o < r; o++) i[o - 2] = arguments[o]; + return function t(e) { + Array.prototype.forEach.call(e, function(e) { + "string" == typeof e || "number" == typeof e ? n.textContent += e : Array.isArray(e) ? t(e) : void 0 !== e.__html ? n.innerHTML += e.__html : e instanceof Node && n.appendChild(e) + }) + }(i), n + } + n.r(t), n.d(t, "createElement", function() { + return r + }) +}, function(e, t) { + var n; + n = function() { + return this + }(); + try { + n = n || new Function("return this")() + } catch (e) { + "object" == typeof window && (n = window) + } + e.exports = n +}, function(e, t, n) { + var r; + r = function() { + return function(n) { + var r = {}; + + function i(e) { + if (r[e]) return r[e].exports; + var t = r[e] = { + i: e, + l: !1, + exports: {} + }; + return n[e].call(t.exports, t, t.exports, i), t.l = !0, t.exports + } + return i.m = n, i.c = r, i.d = function(e, t, n) { + i.o(e, t) || Object.defineProperty(e, t, { + enumerable: !0, + get: n + }) + }, i.r = function(e) { + "undefined" != typeof Symbol && Symbol.toStringTag && Object.defineProperty(e, Symbol.toStringTag, { + value: "Module" + }), Object.defineProperty(e, "__esModule", { + value: !0 + }) + }, i.t = function(t, e) { + if (1 & e && (t = i(t)), 8 & e) return t; + if (4 & e && "object" == typeof t && t && t.__esModule) return t; + var n = Object.create(null); + if (i.r(n), Object.defineProperty(n, "default", { + enumerable: !0, + value: t + }), 2 & e && "string" != typeof t) + for (var r in t) i.d(n, r, function(e) { + return t[e] + }.bind(null, r)); + return n + }, i.n = function(e) { + var t = e && e.__esModule ? function() { + return e.default + } : function() { + return e + }; + return i.d(t, "a", t), t + }, i.o = function(e, t) { + return Object.prototype.hasOwnProperty.call(e, t) + }, i.p = "", i(i.s = 0) + }([function(e, t, n) { + "use strict"; + var i = "function" == typeof Symbol && "symbol" == typeof Symbol.iterator ? function(e) { + return typeof e + } : function(e) { + return e && "function" == typeof Symbol && e.constructor === Symbol && e !== Symbol.prototype ? "symbol" : typeof e + }, + o = function() { + function r(e, t) { + for (var n = 0; n < t.length; n++) { + var r = t[n]; + r.enumerable = r.enumerable || !1, r.configurable = !0, "value" in r && (r.writable = !0), Object.defineProperty(e, r.key, r) + } + } + return function(e, t, n) { + return t && r(e.prototype, t), n && r(e, n), e + } + }(), + a = r(n(1)), + s = r(n(3)), + c = r(n(4)); + + function r(e) { + return e && e.__esModule ? e : { + default: e + } + } + var l = function(e) { + function r(e, t) { + ! function(e, t) { + if (!(e instanceof t)) throw new TypeError("Cannot call a class as a function") + }(this, r); + var n = function(e, t) { + if (!e) throw new ReferenceError("this hasn't been initialised - super() hasn't been called"); + return !t || "object" != typeof t && "function" != typeof t ? e : t + }(this, (r.__proto__ || Object.getPrototypeOf(r)).call(this)); + return n.resolveOptions(t), n.listenClick(e), n + } + return function(e, t) { + if ("function" != typeof t && null !== t) throw new TypeError("Super expression must either be null or a function, not " + typeof t); + e.prototype = Object.create(t && t.prototype, { + constructor: { + value: e, + enumerable: !1, + writable: !0, + configurable: !0 + } + }), t && (Object.setPrototypeOf ? Object.setPrototypeOf(e, t) : e.__proto__ = t) + }(r, s.default), o(r, [{ + key: "resolveOptions", + value: function() { + var e = 0 < arguments.length && void 0 !== arguments[0] ? arguments[0] : {}; + this.action = "function" == typeof e.action ? e.action : this.defaultAction, this.target = "function" == typeof e.target ? e.target : this.defaultTarget, this.text = "function" == typeof e.text ? e.text : this.defaultText, this.container = "object" === i(e.container) ? e.container : document.body + } + }, { + key: "listenClick", + value: function(e) { + var t = this; + this.listener = (0, c.default)(e, "click", function(e) { + return t.onClick(e) + }) + } + }, { + key: "onClick", + value: function(e) { + var t = e.delegateTarget || e.currentTarget; + this.clipboardAction && (this.clipboardAction = null), this.clipboardAction = new a.default({ + action: this.action(t), + target: this.target(t), + text: this.text(t), + container: this.container, + trigger: t, + emitter: this + }) + } + }, { + key: "defaultAction", + value: function(e) { + return u("action", e) + } + }, { + key: "defaultTarget", + value: function(e) { + var t = u("target", e); + if (t) return document.querySelector(t) + } + }, { + key: "defaultText", + value: function(e) { + return u("text", e) + } + }, { + key: "destroy", + value: function() { + this.listener.destroy(), this.clipboardAction && (this.clipboardAction.destroy(), this.clipboardAction = null) + } + }], [{ + key: "isSupported", + value: function() { + var e = 0 < arguments.length && void 0 !== arguments[0] ? arguments[0] : ["copy", "cut"], + t = "string" == typeof e ? [e] : e, + n = !!document.queryCommandSupported; + return t.forEach(function(e) { + n = n && !!document.queryCommandSupported(e) + }), n + } + }]), r + }(); + + function u(e, t) { + var n = "data-clipboard-" + e; + if (t.hasAttribute(n)) return t.getAttribute(n) + } + e.exports = l + }, function(e, t, n) { + "use strict"; + var r, i = "function" == typeof Symbol && "symbol" == typeof Symbol.iterator ? function(e) { + return typeof e + } : function(e) { + return e && "function" == typeof Symbol && e.constructor === Symbol && e !== Symbol.prototype ? "symbol" : typeof e + }, + o = function() { + function r(e, t) { + for (var n = 0; n < t.length; n++) { + var r = t[n]; + r.enumerable = r.enumerable || !1, r.configurable = !0, "value" in r && (r.writable = !0), Object.defineProperty(e, r.key, r) + } + } + return function(e, t, n) { + return t && r(e.prototype, t), n && r(e, n), e + } + }(), + a = n(2), + s = (r = a) && r.__esModule ? r : { + default: r + }; + var c = function() { + function t(e) { + ! function(e, t) { + if (!(e instanceof t)) throw new TypeError("Cannot call a class as a function") + }(this, t), this.resolveOptions(e), this.initSelection() + } + return o(t, [{ + key: "resolveOptions", + value: function() { + var e = 0 < arguments.length && void 0 !== arguments[0] ? arguments[0] : {}; + this.action = e.action, this.container = e.container, this.emitter = e.emitter, this.target = e.target, this.text = e.text, this.trigger = e.trigger, this.selectedText = "" + } + }, { + key: "initSelection", + value: function() { + this.text ? this.selectFake() : this.target && this.selectTarget() + } + }, { + key: "selectFake", + value: function() { + var e = this, + t = "rtl" == document.documentElement.getAttribute("dir"); + this.removeFake(), this.fakeHandlerCallback = function() { + return e.removeFake() + }, this.fakeHandler = this.container.addEventListener("click", this.fakeHandlerCallback) || !0, this.fakeElem = document.createElement("textarea"), this.fakeElem.style.fontSize = "12pt", this.fakeElem.style.border = "0", this.fakeElem.style.padding = "0", this.fakeElem.style.margin = "0", this.fakeElem.style.position = "absolute", this.fakeElem.style[t ? "right" : "left"] = "-9999px"; + var n = window.pageYOffset || document.documentElement.scrollTop; + this.fakeElem.style.top = n + "px", this.fakeElem.setAttribute("readonly", ""), this.fakeElem.value = this.text, this.container.appendChild(this.fakeElem), this.selectedText = (0, s.default)(this.fakeElem), this.copyText() + } + }, { + key: "removeFake", + value: function() { + this.fakeHandler && (this.container.removeEventListener("click", this.fakeHandlerCallback), this.fakeHandler = null, this.fakeHandlerCallback = null), this.fakeElem && (this.container.removeChild(this.fakeElem), this.fakeElem = null) + } + }, { + key: "selectTarget", + value: function() { + this.selectedText = (0, s.default)(this.target), this.copyText() + } + }, { + key: "copyText", + value: function() { + var t = void 0; + try { + t = document.execCommand(this.action) + } catch (e) { + t = !1 + } + this.handleResult(t) + } + }, { + key: "handleResult", + value: function(e) { + this.emitter.emit(e ? "success" : "error", { + action: this.action, + text: this.selectedText, + trigger: this.trigger, + clearSelection: this.clearSelection.bind(this) + }) + } + }, { + key: "clearSelection", + value: function() { + this.trigger && this.trigger.focus(), window.getSelection().removeAllRanges() + } + }, { + key: "destroy", + value: function() { + this.removeFake() + } + }, { + key: "action", + set: function() { + var e = 0 < arguments.length && void 0 !== arguments[0] ? arguments[0] : "copy"; + if (this._action = e, "copy" !== this._action && "cut" !== this._action) throw new Error('Invalid "action" value, use either "copy" or "cut"') + }, + get: function() { + return this._action + } + }, { + key: "target", + set: function(e) { + if (void 0 !== e) { + if (!e || "object" !== (void 0 === e ? "undefined" : i(e)) || 1 !== e.nodeType) throw new Error('Invalid "target" value, use a valid Element'); + if ("copy" === this.action && e.hasAttribute("disabled")) throw new Error('Invalid "target" attribute. Please use "readonly" instead of "disabled" attribute'); + if ("cut" === this.action && (e.hasAttribute("readonly") || e.hasAttribute("disabled"))) throw new Error('Invalid "target" attribute. You can\'t cut text from elements with "readonly" or "disabled" attributes'); + this._target = e + } + }, + get: function() { + return this._target + } + }]), t + }(); + e.exports = c + }, function(e, t) { + e.exports = function(e) { + var t; + if ("SELECT" === e.nodeName) e.focus(), t = e.value; + else if ("INPUT" === e.nodeName || "TEXTAREA" === e.nodeName) { + var n = e.hasAttribute("readonly"); + n || e.setAttribute("readonly", ""), e.select(), e.setSelectionRange(0, e.value.length), n || e.removeAttribute("readonly"), t = e.value + } else { + e.hasAttribute("contenteditable") && e.focus(); + var r = window.getSelection(), + i = document.createRange(); + i.selectNodeContents(e), r.removeAllRanges(), r.addRange(i), t = r.toString() + } + return t + } + }, function(e, t) { + function n() {} + n.prototype = { + on: function(e, t, n) { + var r = this.e || (this.e = {}); + return (r[e] || (r[e] = [])).push({ + fn: t, + ctx: n + }), this + }, + once: function(e, t, n) { + var r = this; + + function i() { + r.off(e, i), t.apply(n, arguments) + } + return i._ = t, this.on(e, i, n) + }, + emit: function(e) { + for (var t = [].slice.call(arguments, 1), n = ((this.e || (this.e = {}))[e] || []).slice(), r = 0, i = n.length; r < i; r++) n[r].fn.apply(n[r].ctx, t); + return this + }, + off: function(e, t) { + var n = this.e || (this.e = {}), + r = n[e], + i = []; + if (r && t) + for (var o = 0, a = r.length; o < a; o++) r[o].fn !== t && r[o].fn._ !== t && i.push(r[o]); + return i.length ? n[e] = i : delete n[e], this + } + }, e.exports = n + }, function(e, t, n) { + var d = n(5), + h = n(6); + e.exports = function(e, t, n) { + if (!e && !t && !n) throw new Error("Missing required arguments"); + if (!d.string(t)) throw new TypeError("Second argument must be a String"); + if (!d.fn(n)) throw new TypeError("Third argument must be a Function"); + if (d.node(e)) return u = t, f = n, (l = e).addEventListener(u, f), { + destroy: function() { + l.removeEventListener(u, f) + } + }; + if (d.nodeList(e)) return a = e, s = t, c = n, Array.prototype.forEach.call(a, function(e) { + e.addEventListener(s, c) + }), { + destroy: function() { + Array.prototype.forEach.call(a, function(e) { + e.removeEventListener(s, c) + }) + } + }; + if (d.string(e)) return r = e, i = t, o = n, h(document.body, r, i, o); + throw new TypeError("First argument must be a String, HTMLElement, HTMLCollection, or NodeList"); + var r, i, o, a, s, c, l, u, f + } + }, function(e, n) { + n.node = function(e) { + return void 0 !== e && e instanceof HTMLElement && 1 === e.nodeType + }, n.nodeList = function(e) { + var t = Object.prototype.toString.call(e); + return void 0 !== e && ("[object NodeList]" === t || "[object HTMLCollection]" === t) && "length" in e && (0 === e.length || n.node(e[0])) + }, n.string = function(e) { + return "string" == typeof e || e instanceof String + }, n.fn = function(e) { + return "[object Function]" === Object.prototype.toString.call(e) + } + }, function(e, t, n) { + var a = n(7); + + function o(e, t, n, r, i) { + var o = function(t, n, e, r) { + return function(e) { + e.delegateTarget = a(e.target, n), e.delegateTarget && r.call(t, e) + } + }.apply(this, arguments); + return e.addEventListener(n, o, i), { + destroy: function() { + e.removeEventListener(n, o, i) + } + } + } + e.exports = function(e, t, n, r, i) { + return "function" == typeof e.addEventListener ? o.apply(null, arguments) : "function" == typeof n ? o.bind(null, document).apply(null, arguments) : ("string" == typeof e && (e = document.querySelectorAll(e)), Array.prototype.map.call(e, function(e) { + return o(e, t, n, r, i) + })) + } + }, function(e, t) { + if ("undefined" != typeof Element && !Element.prototype.matches) { + var n = Element.prototype; + n.matches = n.matchesSelector || n.mozMatchesSelector || n.msMatchesSelector || n.oMatchesSelector || n.webkitMatchesSelector + } + e.exports = function(e, t) { + for (; e && 9 !== e.nodeType;) { + if ("function" == typeof e.matches && e.matches(t)) return e; + e = e.parentNode + } + } + }]) + }, e.exports = r() +}, function(r, i, o) { + var a, s; + ! function(e) { + if (void 0 === (s = "function" == typeof(a = e) ? a.call(i, o, i, r) : a) || (r.exports = s), !0, r.exports = e(), !!0) { + var t = window.Cookies, + n = window.Cookies = e(); + n.noConflict = function() { + return window.Cookies = t, n + } + } + }(function() { + function m() { + for (var e = 0, t = {}; e < arguments.length; e++) { + var n = arguments[e]; + for (var r in n) t[r] = n[r] + } + return t + } + return function e(h) { + function p(e, t, n) { + var r; + if ("undefined" != typeof document) { + if (1 < arguments.length) { + if ("number" == typeof(n = m({ + path: "/" + }, p.defaults, n)).expires) { + var i = new Date; + i.setMilliseconds(i.getMilliseconds() + 864e5 * n.expires), n.expires = i + } + n.expires = n.expires ? n.expires.toUTCString() : ""; + try { + r = JSON.stringify(t), /^[\{\[]/.test(r) && (t = r) + } catch (e) {} + t = h.write ? h.write(t, e) : encodeURIComponent(String(t)).replace(/%(23|24|26|2B|3A|3C|3E|3D|2F|3F|40|5B|5D|5E|60|7B|7D|7C)/g, decodeURIComponent), e = (e = (e = encodeURIComponent(String(e))).replace(/%(23|24|26|2B|5E|60|7C)/g, decodeURIComponent)).replace(/[\(\)]/g, escape); + var o = ""; + for (var a in n) n[a] && (o += "; " + a, !0 !== n[a] && (o += "=" + n[a])); + return document.cookie = e + "=" + t + o + } + e || (r = {}); + for (var s = document.cookie ? document.cookie.split("; ") : [], c = /(%[0-9A-Z]{2})+/g, l = 0; l < s.length; l++) { + var u = s[l].split("="), + f = u.slice(1).join("="); + this.json || '"' !== f.charAt(0) || (f = f.slice(1, -1)); + try { + var d = u[0].replace(c, decodeURIComponent); + if (f = h.read ? h.read(f, d) : h(f, d) || f.replace(c, decodeURIComponent), this.json) try { + f = JSON.parse(f) + } catch (e) {} + if (e === d) { + r = f; + break + } + e || (r[d] = f) + } catch (e) {} + } + return r + } + } + return (p.set = p).get = function(e) { + return p.call(p, e) + }, p.getJSON = function() { + return p.apply({ + json: !0 + }, [].slice.call(arguments)) + }, p.defaults = {}, p.remove = function(e, t) { + p(e, "", m(t, { + expires: -1 + })) + }, p.withConverter = e, p + }(function() {}) + }) +}, function(e, t, n) { + "use strict"; + n.r(t); + var r = "function" == typeof fetch ? fetch.bind() : function(i, o) { + return o = o || {}, new Promise(function(e, t) { + var n = new XMLHttpRequest; + for (var r in n.open(o.method || "get", i, !0), o.headers) n.setRequestHeader(r, o.headers[r]); + + function s() { + var r, i = [], + o = [], + a = {}; + return n.getAllResponseHeaders().replace(/^(.*?):[^\S\n]*([\s\S]*?)$/gm, function(e, t, n) { + i.push(t = t.toLowerCase()), o.push([t, n]), r = a[t], a[t] = r ? r + "," + n : n + }), { + ok: 2 == (n.status / 100 | 0), + status: n.status, + statusText: n.statusText, + url: n.responseURL, + clone: s, + text: function() { + return Promise.resolve(n.responseText) + }, + json: function() { + return Promise.resolve(n.responseText).then(JSON.parse) + }, + blob: function() { + return Promise.resolve(new Blob([n.response])) + }, + headers: { + keys: function() { + return i + }, + entries: function() { + return o + }, + get: function(e) { + return a[e.toLowerCase()] + }, + has: function(e) { + return e.toLowerCase() in a + } + } + } + } + n.withCredentials = "include" == o.credentials, n.onload = function() { + e(s()) + }, n.onerror = t, n.send(o.body || null) + }) + }; + t.default = r +}, function(e, t, n) { + "use strict"; + t.a = function(t) { + var n = this.constructor; + return this.then(function(e) { + return n.resolve(t()).then(function() { + return e + }) + }, function(e) { + return n.resolve(t()).then(function() { + return n.reject(e) + }) + }) + } +}, function(e, n, r) { + "use strict"; + (function(f) { + r.d(n, "a", function() { + return t + }); + var e = r(1), + d = r.n(e), + h = function(e) { + var t = document.getElementsByName("lang:" + e)[0]; + if (!(t instanceof HTMLMetaElement)) throw new ReferenceError; + return t.content + }, + t = function() { + function e(e, t) { + var n = "string" == typeof e ? document.querySelector(e) : e; + if (!(n instanceof HTMLElement)) throw new ReferenceError; + this.el_ = n; + var r = Array.prototype.slice.call(this.el_.children), + i = r[0], + o = r[1]; + this.data_ = t, this.meta_ = i, this.list_ = o, this.message_ = { + placeholder: this.meta_.textContent, + none: h("search.result.none"), + one: h("search.result.one"), + other: h("search.result.other") + }; + var a = h("search.tokenizer"); + a.length && (d.a.tokenizer.separator = a), this.lang_ = h("search.language").split(",").filter(Boolean).map(function(e) { + return e.trim() + }) + } + return e.prototype.update = function(e) { + var t, a = this; + if ("focus" !== e.type || this.index_) { + if ("focus" === e.type || "keyup" === e.type) { + var n = e.target; + if (!(n instanceof HTMLInputElement)) throw new ReferenceError; + if (!this.index_ || n.value === this.value_) return; + for (; this.list_.firstChild;) this.list_.removeChild(this.list_.firstChild); + if (this.value_ = n.value, 0 === this.value_.length) return void(this.meta_.textContent = this.message_.placeholder); + var r = this.index_.query(function(t) { + a.value_.toLowerCase().split(" ").filter(Boolean).forEach(function(e) { + t.term(e, { + wildcard: d.a.Query.wildcard.TRAILING + }) + }) + }).reduce(function(e, t) { + var n = a.docs_.get(t.ref); + if (n.parent) { + var r = n.parent.location; + e.set(r, (e.get(r) || []).concat(t)) + } else { + var i = n.location; + e.set(i, e.get(i) || []) + } + return e + }, new Map), + i = (t = this.value_.trim(), t.replace(/[|\\{}()[\]^$+*?.-]/g, "\\$&")).replace(new RegExp(d.a.tokenizer.separator, "img"), "|"), + s = new RegExp("(^|" + d.a.tokenizer.separator + ")(" + i + ")", "img"), + c = function(e, t, n) { + return t + "" + n + "" + }; + this.stack_ = [], r.forEach(function(e, t) { + var n, r = a.docs_.get(t), + i = f.createElement("li", { + class: "md-search-result__item" + }, f.createElement("a", { + href: r.location, + title: r.title, + class: "md-search-result__link", + tabindex: "-1" + }, f.createElement("article", { + class: "md-search-result__article md-search-result__article--document" + }, f.createElement("h1", { + class: "md-search-result__title" + }, { + __html: r.title.replace(s, c) + }), r.text.length ? f.createElement("p", { + class: "md-search-result__teaser" + }, { + __html: r.text.replace(s, c) + }) : {}))), + o = e.map(function(t) { + return function() { + var e = a.docs_.get(t.ref); + i.appendChild(f.createElement("a", { + href: e.location, + title: e.title, + class: "md-search-result__link", + "data-md-rel": "anchor", + tabindex: "-1" + }, f.createElement("article", { + class: "md-search-result__article" + }, f.createElement("h1", { + class: "md-search-result__title" + }, { + __html: e.title.replace(s, c) + }), e.text.length ? f.createElement("p", { + class: "md-search-result__teaser" + }, { + __html: function(e, t) { + var n = t; + if (e.length > n) { + for (; + " " !== e[n] && 0 < --n;); + return e.substring(0, n) + "..." + } + return e + }(e.text.replace(s, c), 400) + }) : {}))) + } + }); + (n = a.stack_).push.apply(n, [function() { + return a.list_.appendChild(i) + }].concat(o)) + }); + var o = this.el_.parentNode; + if (!(o instanceof HTMLElement)) throw new ReferenceError; + for (; this.stack_.length && o.offsetHeight >= o.scrollHeight - 16;) this.stack_.shift()(); + var l = this.list_.querySelectorAll("[data-md-rel=anchor]"); + switch (Array.prototype.forEach.call(l, function(r) { + ["click", "keydown"].forEach(function(n) { + r.addEventListener(n, function(e) { + if ("keydown" !== n || 13 === e.keyCode) { + var t = document.querySelector("[data-md-toggle=search]"); + if (!(t instanceof HTMLInputElement)) throw new ReferenceError; + t.checked && (t.checked = !1, t.dispatchEvent(new CustomEvent("change"))), e.preventDefault(), setTimeout(function() { + document.location.href = r.href + }, 100) + } + }) + }) + }), r.size) { + case 0: + this.meta_.textContent = this.message_.none; + break; + case 1: + this.meta_.textContent = this.message_.one; + break; + default: + this.meta_.textContent = this.message_.other.replace("#", r.size) + } + } + } else { + var u = function(e) { + a.docs_ = e.reduce(function(e, t) { + var n, r, i, o = t.location.split("#"), + a = o[0], + s = o[1]; + return t.text = (n = t.text, r = document.createTextNode(n), (i = document.createElement("p")).appendChild(r), i.innerHTML), s && (t.parent = e.get(a), t.parent && !t.parent.done && (t.parent.title = t.title, t.parent.text = t.text, t.parent.done = !0)), t.text = t.text.replace(/\n/g, " ").replace(/\s+/g, " ").replace(/\s+([,.:;!?])/g, function(e, t) { + return t + }), t.parent && t.parent.title === t.title || e.set(t.location, t), e + }, new Map); + var i = a.docs_, + o = a.lang_; + a.stack_ = [], a.index_ = d()(function() { + var e, t = this, + n = { + "search.pipeline.trimmer": d.a.trimmer, + "search.pipeline.stopwords": d.a.stopWordFilter + }, + r = Object.keys(n).reduce(function(e, t) { + return h(t).match(/^false$/i) || e.push(n[t]), e + }, []); + this.pipeline.reset(), r && (e = this.pipeline).add.apply(e, r), 1 === o.length && "en" !== o[0] && d.a[o[0]] ? this.use(d.a[o[0]]) : 1 < o.length && this.use(d.a.multiLanguage.apply(d.a, o)), this.field("title", { + boost: 10 + }), this.field("text"), this.ref("location"), i.forEach(function(e) { + return t.add(e) + }) + }); + var t = a.el_.parentNode; + if (!(t instanceof HTMLElement)) throw new ReferenceError; + t.addEventListener("scroll", function() { + for (; a.stack_.length && t.scrollTop + t.offsetHeight >= t.scrollHeight - 16;) a.stack_.splice(0, 10).forEach(function(e) { + return e() + }) + }) + }; + setTimeout(function() { + return "function" == typeof a.data_ ? a.data_().then(u) : u(a.data_) + }, 250) + } + }, e + }() + }).call(this, r(3)) +}, function(e, n, r) { + "use strict"; + (function(t) { + r.d(n, "a", function() { + return e + }); + var e = function() { + function e(e) { + var t = "string" == typeof e ? document.querySelector(e) : e; + if (!(t instanceof HTMLElement)) throw new ReferenceError; + this.el_ = t + } + return e.prototype.initialize = function(e) { + e.length && this.el_.children.length && this.el_.children[this.el_.children.length - 1].appendChild(t.createElement("ul", { + class: "md-source__facts" + }, e.map(function(e) { + return t.createElement("li", { + class: "md-source__fact" + }, e) + }))), this.el_.dataset.mdState = "done" + }, e + }() + }).call(this, r(3)) +}, , , function(e, n, c) { + "use strict"; + c.r(n), + function(o) { + c.d(n, "app", function() { + return t + }); + c(14), c(15), c(16), c(17), c(18), c(19), c(20); + var r = c(2), + e = c(5), + a = c.n(e), + i = c(0); + window.Promise = window.Promise || r.a; + var s = function(e) { + var t = document.getElementsByName("lang:" + e)[0]; + if (!(t instanceof HTMLMetaElement)) throw new ReferenceError; + return t.content + }; + var t = { + initialize: function(t) { + new i.a.Event.Listener(document, "DOMContentLoaded", function() { + if (!(document.body instanceof HTMLElement)) throw new ReferenceError; + Modernizr.addTest("ios", function() { + return !!navigator.userAgent.match(/(iPad|iPhone|iPod)/g) + }); + var e = document.querySelectorAll("table:not([class])"); + if (Array.prototype.forEach.call(e, function(e) { + var t = o.createElement("div", { + class: "md-typeset__scrollwrap" + }, o.createElement("div", { + class: "md-typeset__table" + })); + e.nextSibling ? e.parentNode.insertBefore(t, e.nextSibling) : e.parentNode.appendChild(t), t.children[0].appendChild(e) + }), a.a.isSupported()) { + var t = document.querySelectorAll(".codehilite > pre, pre > code"); + Array.prototype.forEach.call(t, function(e, t) { + var n = "__code_" + t, + r = o.createElement("button", { + class: "md-clipboard", + title: s("clipboard.copy"), + "data-clipboard-target": "#" + n + " pre, #" + n + " code" + }, o.createElement("span", { + class: "md-clipboard__message" + })), + i = e.parentNode; + i.id = n, i.insertBefore(r, e) + }), new a.a(".md-clipboard").on("success", function(e) { + var t = e.trigger.querySelector(".md-clipboard__message"); + if (!(t instanceof HTMLElement)) throw new ReferenceError; + e.clearSelection(), t.dataset.mdTimer && clearTimeout(parseInt(t.dataset.mdTimer, 10)), t.classList.add("md-clipboard__message--active"), t.innerHTML = s("clipboard.copied"), t.dataset.mdTimer = setTimeout(function() { + t.classList.remove("md-clipboard__message--active"), t.dataset.mdTimer = "" + }, 2e3).toString() + }) + } + if (!Modernizr.details) { + var n = document.querySelectorAll("details > summary"); + Array.prototype.forEach.call(n, function(e) { + e.addEventListener("click", function(e) { + var t = e.target.parentNode; + t.hasAttribute("open") ? t.removeAttribute("open") : t.setAttribute("open", "") + }) + }) + } + var r = function() { + if (document.location.hash) { + var e = document.getElementById(document.location.hash.substring(1)); + if (!e) return; + for (var t = e.parentNode; t && !(t instanceof HTMLDetailsElement);) t = t.parentNode; + if (t && !t.open) { + t.open = !0; + var n = location.hash; + location.hash = " ", location.hash = n + } + } + }; + if (window.addEventListener("hashchange", r), r(), Modernizr.ios) { + var i = document.querySelectorAll("[data-md-scrollfix]"); + Array.prototype.forEach.call(i, function(t) { + t.addEventListener("touchstart", function() { + var e = t.scrollTop; + 0 === e ? t.scrollTop = 1 : e + t.offsetHeight === t.scrollHeight && (t.scrollTop = e - 1) + }) + }) + } + }).listen(), new i.a.Event.Listener(window, ["scroll", "resize", "orientationchange"], new i.a.Header.Shadow("[data-md-component=container]", "[data-md-component=header]")).listen(), new i.a.Event.Listener(window, ["scroll", "resize", "orientationchange"], new i.a.Header.Title("[data-md-component=title]", ".md-typeset h1")).listen(), document.querySelector("[data-md-component=hero]") && new i.a.Event.Listener(window, ["scroll", "resize", "orientationchange"], new i.a.Tabs.Toggle("[data-md-component=hero]")).listen(), document.querySelector("[data-md-component=tabs]") && new i.a.Event.Listener(window, ["scroll", "resize", "orientationchange"], new i.a.Tabs.Toggle("[data-md-component=tabs]")).listen(), new i.a.Event.MatchMedia("(min-width: 1220px)", new i.a.Event.Listener(window, ["scroll", "resize", "orientationchange"], new i.a.Sidebar.Position("[data-md-component=navigation]", "[data-md-component=header]"))), document.querySelector("[data-md-component=toc]") && new i.a.Event.MatchMedia("(min-width: 960px)", new i.a.Event.Listener(window, ["scroll", "resize", "orientationchange"], new i.a.Sidebar.Position("[data-md-component=toc]", "[data-md-component=header]"))), new i.a.Event.MatchMedia("(min-width: 960px)", new i.a.Event.Listener(window, "scroll", new i.a.Nav.Blur("[data-md-component=toc] .md-nav__link"))); + var e = document.querySelectorAll("[data-md-component=collapsible]"); + Array.prototype.forEach.call(e, function(e) { + new i.a.Event.MatchMedia("(min-width: 1220px)", new i.a.Event.Listener(e.previousElementSibling, "click", new i.a.Nav.Collapse(e))) + }), new i.a.Event.MatchMedia("(max-width: 1219px)", new i.a.Event.Listener("[data-md-component=navigation] [data-md-toggle]", "change", new i.a.Nav.Scrolling("[data-md-component=navigation] nav"))), document.querySelector("[data-md-component=search]") && (new i.a.Event.MatchMedia("(max-width: 959px)", new i.a.Event.Listener("[data-md-toggle=search]", "change", new i.a.Search.Lock("[data-md-toggle=search]")))), + new i.a.Event.Listener(document.body, "keydown", function(e) { + if (9 === e.keyCode) { + var t = document.querySelectorAll("[data-md-component=navigation] .md-nav__link[for]:not([tabindex])"); + Array.prototype.forEach.call(t, function(e) { + e.offsetHeight && (e.tabIndex = 0) + }) + } + }).listen(), new i.a.Event.Listener(document.body, "mousedown", function() { + var e = document.querySelectorAll("[data-md-component=navigation] .md-nav__link[tabindex]"); + Array.prototype.forEach.call(e, function(e) { + e.removeAttribute("tabIndex") + }) + }).listen(), document.body.addEventListener("click", function() { + "tabbing" === document.body.dataset.mdState && (document.body.dataset.mdState = "") + }), new i.a.Event.MatchMedia("(max-width: 959px)", new i.a.Event.Listener("[data-md-component=navigation] [href^='#']", "click", function() { + var e = document.querySelector("[data-md-toggle=drawer]"); + if (!(e instanceof HTMLInputElement)) throw new ReferenceError; + e.checked && (e.checked = !1, e.dispatchEvent(new CustomEvent("change"))) + })), + function() { + var e = document.querySelector("[data-md-source]"); + if (!e) return r.a.resolve([]); + if (!(e instanceof HTMLAnchorElement)) throw new ReferenceError; + switch (e.dataset.mdSource) { + case "github": + return new i.a.Source.Adapter.GitHub(e).fetch(); + default: + return r.a.resolve([]) + } + }().then(function(t) { + var e = document.querySelectorAll("[data-md-source]"); + Array.prototype.forEach.call(e, function(e) { + new i.a.Source.Repository(e).initialize(t) + }) + }); + var n = function() { + var e = document.querySelectorAll("details"); + Array.prototype.forEach.call(e, function(e) { + e.setAttribute("open", "") + }) + }; + new i.a.Event.MatchMedia("print", { + listen: n, + unlisten: function() {} + }), window.onbeforeprint = n + } + } + }.call(this, c(3)) +}, function(e, t, n) { + e.exports = n.p + "assets/images/icons/bitbucket.1b09e088.svg" +}, function(e, t, n) { + e.exports = n.p + "assets/images/icons/github.f0b8504a.svg" +}, function(e, t, n) { + e.exports = n.p + "assets/images/icons/gitlab.6dd19c00.svg" +}, function(e, t) { + e.exports = "/Users/squidfunk/Desktop/General/Sources/mkdocs-material/material/application.4031d38b.css" +}, function(e, t) { + e.exports = "/Users/squidfunk/Desktop/General/Sources/mkdocs-material/material/application-palette.224b79ff.css" +}, function(e, t) { + ! function() { + if ("undefined" != typeof window) try { + var e = new window.CustomEvent("test", { + cancelable: !0 + }); + if (e.preventDefault(), !0 !== e.defaultPrevented) throw new Error("Could not prevent default") + } catch (e) { + var t = function(e, t) { + var n, r; + return (t = t || {}).bubbles = !!t.bubbles, t.cancelable = !!t.cancelable, (n = document.createEvent("CustomEvent")).initCustomEvent(e, t.bubbles, t.cancelable, t.detail), r = n.preventDefault, n.preventDefault = function() { + r.call(this); + try { + Object.defineProperty(this, "defaultPrevented", { + get: function() { + return !0 + } + }) + } catch (e) { + this.defaultPrevented = !0 + } + }, n + }; + t.prototype = window.Event.prototype, window.CustomEvent = t + } + }() +}, function(e, t, n) { + window.fetch || (window.fetch = n(7).default || n(7)) +}, function(e, i, o) { + (function(e) { + var t = void 0 !== e && e || "undefined" != typeof self && self || window, + n = Function.prototype.apply; + + function r(e, t) { + this._id = e, this._clearFn = t + } + i.setTimeout = function() { + return new r(n.call(setTimeout, t, arguments), clearTimeout) + }, i.setInterval = function() { + return new r(n.call(setInterval, t, arguments), clearInterval) + }, i.clearTimeout = i.clearInterval = function(e) { + e && e.close() + }, r.prototype.unref = r.prototype.ref = function() {}, r.prototype.close = function() { + this._clearFn.call(t, this._id) + }, i.enroll = function(e, t) { + clearTimeout(e._idleTimeoutId), e._idleTimeout = t + }, i.unenroll = function(e) { + clearTimeout(e._idleTimeoutId), e._idleTimeout = -1 + }, i._unrefActive = i.active = function(e) { + clearTimeout(e._idleTimeoutId); + var t = e._idleTimeout; + 0 <= t && (e._idleTimeoutId = setTimeout(function() { + e._onTimeout && e._onTimeout() + }, t)) + }, o(22), i.setImmediate = "undefined" != typeof self && self.setImmediate || void 0 !== e && e.setImmediate || this && this.setImmediate, i.clearImmediate = "undefined" != typeof self && self.clearImmediate || void 0 !== e && e.clearImmediate || this && this.clearImmediate + }).call(this, o(4)) +}, function(e, t, n) { + (function(e, p) { + ! function(n, r) { + "use strict"; + if (!n.setImmediate) { + var i, o, t, a, e, s = 1, + c = {}, + l = !1, + u = n.document, + f = Object.getPrototypeOf && Object.getPrototypeOf(n); + f = f && f.setTimeout ? f : n, i = "[object process]" === {}.toString.call(n.process) ? function(e) { + p.nextTick(function() { + h(e) + }) + } : function() { + if (n.postMessage && !n.importScripts) { + var e = !0, + t = n.onmessage; + return n.onmessage = function() { + e = !1 + }, n.postMessage("", "*"), n.onmessage = t, e + } + }() ? (a = "setImmediate$" + Math.random() + "$", e = function(e) { + e.source === n && "string" == typeof e.data && 0 === e.data.indexOf(a) && h(+e.data.slice(a.length)) + }, n.addEventListener ? n.addEventListener("message", e, !1) : n.attachEvent("onmessage", e), function(e) { + n.postMessage(a + e, "*") + }) : n.MessageChannel ? ((t = new MessageChannel).port1.onmessage = function(e) { + h(e.data) + }, function(e) { + t.port2.postMessage(e) + }) : u && "onreadystatechange" in u.createElement("script") ? (o = u.documentElement, function(e) { + var t = u.createElement("script"); + t.onreadystatechange = function() { + h(e), t.onreadystatechange = null, o.removeChild(t), t = null + }, o.appendChild(t) + }) : function(e) { + setTimeout(h, 0, e) + }, f.setImmediate = function(e) { + "function" != typeof e && (e = new Function("" + e)); + for (var t = new Array(arguments.length - 1), n = 0; n < t.length; n++) t[n] = arguments[n + 1]; + var r = { + callback: e, + args: t + }; + return c[s] = r, i(s), s++ + }, f.clearImmediate = d + } + + function d(e) { + delete c[e] + } + + function h(e) { + if (l) setTimeout(h, 0, e); + else { + var t = c[e]; + if (t) { + l = !0; + try { + ! function(e) { + var t = e.callback, + n = e.args; + switch (n.length) { + case 0: + t(); + break; + case 1: + t(n[0]); + break; + case 2: + t(n[0], n[1]); + break; + case 3: + t(n[0], n[1], n[2]); + break; + default: + t.apply(r, n) + } + }(t) + } finally { + d(e), l = !1 + } + } + } + } + }("undefined" == typeof self ? void 0 === e ? this : e : self) + }).call(this, n(4), n(23)) +}, function(e, t) { + var n, r, i = e.exports = {}; + + function o() { + throw new Error("setTimeout has not been defined") + } + + function a() { + throw new Error("clearTimeout has not been defined") + } + + function s(t) { + if (n === setTimeout) return setTimeout(t, 0); + if ((n === o || !n) && setTimeout) return n = setTimeout, setTimeout(t, 0); + try { + return n(t, 0) + } catch (e) { + try { + return n.call(null, t, 0) + } catch (e) { + return n.call(this, t, 0) + } + } + }! function() { + try { + n = "function" == typeof setTimeout ? setTimeout : o + } catch (e) { + n = o + } + try { + r = "function" == typeof clearTimeout ? clearTimeout : a + } catch (e) { + r = a + } + }(); + var c, l = [], + u = !1, + f = -1; + + function d() { + u && c && (u = !1, c.length ? l = c.concat(l) : f = -1, l.length && h()) + } + + function h() { + if (!u) { + var e = s(d); + u = !0; + for (var t = l.length; t;) { + for (c = l, l = []; ++f < t;) c && c[f].run(); + f = -1, t = l.length + } + c = null, u = !1, + function(t) { + if (r === clearTimeout) return clearTimeout(t); + if ((r === a || !r) && clearTimeout) return r = clearTimeout, clearTimeout(t); + try { + r(t) + } catch (e) { + try { + return r.call(null, t) + } catch (e) { + return r.call(this, t) + } + } + }(e) + } + } + + function p(e, t) { + this.fun = e, this.array = t + } + + function m() {} + i.nextTick = function(e) { + var t = new Array(arguments.length - 1); + if (1 < arguments.length) + for (var n = 1; n < arguments.length; n++) t[n - 1] = arguments[n]; + l.push(new p(e, t)), 1 !== l.length || u || s(h) + }, p.prototype.run = function() { + this.fun.apply(null, this.array) + }, i.title = "browser", i.browser = !0, i.env = {}, i.argv = [], i.version = "", i.versions = {}, i.on = m, i.addListener = m, i.once = m, i.off = m, i.removeListener = m, i.removeAllListeners = m, i.emit = m, i.prependListener = m, i.prependOnceListener = m, i.listeners = function(e) { + return [] + }, i.binding = function(e) { + throw new Error("process.binding is not supported") + }, i.cwd = function() { + return "/" + }, i.chdir = function(e) { + throw new Error("process.chdir is not supported") + }, i.umask = function() { + return 0 + } +}, function(i, o, a) { + var s, c; + /** + * lunr - http://lunrjs.com - A bit like Solr, but much smaller and not as bright - 2.3.6 + * Copyright (C) 2019 Oliver Nightingale + * @license MIT + */ + ! function() { + var t, l, u, e, n, f, d, h, p, m, y, v, g, w, _, E, x, b, k, S, T, L, R, O, C, r, D = function(e) { + var t = new D.Builder; + return t.pipeline.add(D.trimmer, D.stopWordFilter, D.stemmer), t.searchPipeline.add(D.stemmer), e.call(t, t), t.build() + }; + D.version = "2.3.6", D.utils = {}, D.utils.warn = (t = this, function(e) { + t.console && console.warn && console.warn(e) + }), D.utils.asString = function(e) { + return null == e ? "" : e.toString() + }, D.utils.clone = function(e) { + if (null == e) return e; + for (var t = Object.create(null), n = Object.keys(e), r = 0; r < n.length; r++) { + var i = n[r], + o = e[i]; + if (Array.isArray(o)) t[i] = o.slice(); + else { + if ("string" != typeof o && "number" != typeof o && "boolean" != typeof o) throw new TypeError("clone is not deep and does not support nested objects"); + t[i] = o + } + } + return t + }, D.FieldRef = function(e, t, n) { + this.docRef = e, this.fieldName = t, this._stringValue = n + }, D.FieldRef.joiner = "/", D.FieldRef.fromString = function(e) { + var t = e.indexOf(D.FieldRef.joiner); + if (-1 === t) throw "malformed field ref string"; + var n = e.slice(0, t), + r = e.slice(t + 1); + return new D.FieldRef(r, n, e) + }, D.FieldRef.prototype.toString = function() { + return null == this._stringValue && (this._stringValue = this.fieldName + D.FieldRef.joiner + this.docRef), this._stringValue + }, D.Set = function(e) { + if (this.elements = Object.create(null), e) { + this.length = e.length; + for (var t = 0; t < this.length; t++) this.elements[e[t]] = !0 + } else this.length = 0 + }, D.Set.complete = { + intersect: function(e) { + return e + }, + union: function(e) { + return e + }, + contains: function() { + return !0 + } + }, D.Set.empty = { + intersect: function() { + return this + }, + union: function(e) { + return e + }, + contains: function() { + return !1 + } + }, D.Set.prototype.contains = function(e) { + return !!this.elements[e] + }, D.Set.prototype.intersect = function(e) { + var t, n, r, i = []; + if (e === D.Set.complete) return this; + if (e === D.Set.empty) return e; + n = this.length < e.length ? (t = this, e) : (t = e, this), r = Object.keys(t.elements); + for (var o = 0; o < r.length; o++) { + var a = r[o]; + a in n.elements && i.push(a) + } + return new D.Set(i) + }, D.Set.prototype.union = function(e) { + return e === D.Set.complete ? D.Set.complete : e === D.Set.empty ? this : new D.Set(Object.keys(this.elements).concat(Object.keys(e.elements))) + }, D.idf = function(e, t) { + var n = 0; + for (var r in e) "_index" != r && (n += Object.keys(e[r]).length); + var i = (t - n + .5) / (n + .5); + return Math.log(1 + Math.abs(i)) + }, D.Token = function(e, t) { + this.str = e || "", this.metadata = t || {} + }, D.Token.prototype.toString = function() { + return this.str + }, D.Token.prototype.update = function(e) { + return this.str = e(this.str, this.metadata), this + }, D.Token.prototype.clone = function(e) { + return e = e || function(e) { + return e + }, new D.Token(e(this.str, this.metadata), this.metadata) + }, D.tokenizer = function(e, t) { + if (null == e || null == e) return []; + if (Array.isArray(e)) return e.map(function(e) { + return new D.Token(D.utils.asString(e).toLowerCase(), D.utils.clone(t)) + }); + for (var n = e.toString().trim().toLowerCase(), r = n.length, i = [], o = 0, a = 0; o <= r; o++) { + var s = o - a; + if (n.charAt(o).match(D.tokenizer.separator) || o == r) { + if (0 < s) { + var c = D.utils.clone(t) || {}; + c.position = [a, s], c.index = i.length, i.push(new D.Token(n.slice(a, o), c)) + } + a = o + 1 + } + } + return i + }, D.tokenizer.separator = /[\s\-]+/, D.Pipeline = function() { + this._stack = [] + }, D.Pipeline.registeredFunctions = Object.create(null), D.Pipeline.registerFunction = function(e, t) { + t in this.registeredFunctions && D.utils.warn("Overwriting existing registered function: " + t), e.label = t, D.Pipeline.registeredFunctions[e.label] = e + }, D.Pipeline.warnIfFunctionNotRegistered = function(e) { + e.label && e.label in this.registeredFunctions || D.utils.warn("Function is not registered with pipeline. This may cause problems when serialising the index.\n", e) + }, D.Pipeline.load = function(e) { + var n = new D.Pipeline; + return e.forEach(function(e) { + var t = D.Pipeline.registeredFunctions[e]; + if (!t) throw new Error("Cannot load unregistered function: " + e); + n.add(t) + }), n + }, D.Pipeline.prototype.add = function() { + Array.prototype.slice.call(arguments).forEach(function(e) { + D.Pipeline.warnIfFunctionNotRegistered(e), this._stack.push(e) + }, this) + }, D.Pipeline.prototype.after = function(e, t) { + D.Pipeline.warnIfFunctionNotRegistered(t); + var n = this._stack.indexOf(e); + if (-1 == n) throw new Error("Cannot find existingFn"); + n += 1, this._stack.splice(n, 0, t) + }, D.Pipeline.prototype.before = function(e, t) { + D.Pipeline.warnIfFunctionNotRegistered(t); + var n = this._stack.indexOf(e); + if (-1 == n) throw new Error("Cannot find existingFn"); + this._stack.splice(n, 0, t) + }, D.Pipeline.prototype.remove = function(e) { + var t = this._stack.indexOf(e); - 1 != t && this._stack.splice(t, 1) + }, D.Pipeline.prototype.run = function(e) { + for (var t = this._stack.length, n = 0; n < t; n++) { + for (var r = this._stack[n], i = [], o = 0; o < e.length; o++) { + var a = r(e[o], o, e); + if (void 0 !== a && "" !== a) + if (Array.isArray(a)) + for (var s = 0; s < a.length; s++) i.push(a[s]); + else i.push(a) + } + e = i + } + return e + }, D.Pipeline.prototype.runString = function(e, t) { + var n = new D.Token(e, t); + return this.run([n]).map(function(e) { + return e.toString() + }) + }, D.Pipeline.prototype.reset = function() { + this._stack = [] + }, D.Pipeline.prototype.toJSON = function() { + return this._stack.map(function(e) { + return D.Pipeline.warnIfFunctionNotRegistered(e), e.label + }) + }, D.Vector = function(e) { + this._magnitude = 0, this.elements = e || [] + }, D.Vector.prototype.positionForIndex = function(e) { + if (0 == this.elements.length) return 0; + for (var t = 0, n = this.elements.length / 2, r = n - t, i = Math.floor(r / 2), o = this.elements[2 * i]; 1 < r && (o < e && (t = i), e < o && (n = i), o != e);) r = n - t, i = t + Math.floor(r / 2), o = this.elements[2 * i]; + return o == e ? 2 * i : e < o ? 2 * i : o < e ? 2 * (i + 1) : void 0 + }, D.Vector.prototype.insert = function(e, t) { + this.upsert(e, t, function() { + throw "duplicate index" + }) + }, D.Vector.prototype.upsert = function(e, t, n) { + this._magnitude = 0; + var r = this.positionForIndex(e); + this.elements[r] == e ? this.elements[r + 1] = n(this.elements[r + 1], t) : this.elements.splice(r, 0, e, t) + }, D.Vector.prototype.magnitude = function() { + if (this._magnitude) return this._magnitude; + for (var e = 0, t = this.elements.length, n = 1; n < t; n += 2) { + var r = this.elements[n]; + e += r * r + } + return this._magnitude = Math.sqrt(e) + }, D.Vector.prototype.dot = function(e) { + for (var t = 0, n = this.elements, r = e.elements, i = n.length, o = r.length, a = 0, s = 0, c = 0, l = 0; c < i && l < o;)(a = n[c]) < (s = r[l]) ? c += 2 : s < a ? l += 2 : a == s && (t += n[c + 1] * r[l + 1], c += 2, l += 2); + return t + }, D.Vector.prototype.similarity = function(e) { + return this.dot(e) / this.magnitude() || 0 + }, D.Vector.prototype.toArray = function() { + for (var e = new Array(this.elements.length / 2), t = 1, n = 0; t < this.elements.length; t += 2, n++) e[n] = this.elements[t]; + return e + }, D.Vector.prototype.toJSON = function() { + return this.elements + }, D.stemmer = (l = { + ational: "ate", + tional: "tion", + enci: "ence", + anci: "ance", + izer: "ize", + bli: "ble", + alli: "al", + entli: "ent", + eli: "e", + ousli: "ous", + ization: "ize", + ation: "ate", + ator: "ate", + alism: "al", + iveness: "ive", + fulness: "ful", + ousness: "ous", + aliti: "al", + iviti: "ive", + biliti: "ble", + logi: "log" + }, u = { + icate: "ic", + ative: "", + alize: "al", + iciti: "ic", + ical: "ic", + ful: "", + ness: "" + }, e = "[aeiouy]", n = "[^aeiou][^aeiouy]*", f = new RegExp("^([^aeiou][^aeiouy]*)?[aeiouy][aeiou]*[^aeiou][^aeiouy]*"), d = new RegExp("^([^aeiou][^aeiouy]*)?[aeiouy][aeiou]*[^aeiou][^aeiouy]*[aeiouy][aeiou]*[^aeiou][^aeiouy]*"), h = new RegExp("^([^aeiou][^aeiouy]*)?[aeiouy][aeiou]*[^aeiou][^aeiouy]*([aeiouy][aeiou]*)?$"), p = new RegExp("^([^aeiou][^aeiouy]*)?[aeiouy]"), m = /^(.+?)(ss|i)es$/, y = /^(.+?)([^s])s$/, v = /^(.+?)eed$/, g = /^(.+?)(ed|ing)$/, w = /.$/, _ = /(at|bl|iz)$/, E = new RegExp("([^aeiouylsz])\\1$"), x = new RegExp("^" + n + e + "[^aeiouwxy]$"), b = /^(.+?[^aeiou])y$/, k = /^(.+?)(ational|tional|enci|anci|izer|bli|alli|entli|eli|ousli|ization|ation|ator|alism|iveness|fulness|ousness|aliti|iviti|biliti|logi)$/, S = /^(.+?)(icate|ative|alize|iciti|ical|ful|ness)$/, T = /^(.+?)(al|ance|ence|er|ic|able|ible|ant|ement|ment|ent|ou|ism|ate|iti|ous|ive|ize)$/, L = /^(.+?)(s|t)(ion)$/, R = /^(.+?)e$/, O = /ll$/, C = new RegExp("^" + n + e + "[^aeiouwxy]$"), r = function(e) { + var t, n, r, i, o, a, s; + if (e.length < 3) return e; + if ("y" == (r = e.substr(0, 1)) && (e = r.toUpperCase() + e.substr(1)), o = y, (i = m).test(e) ? e = e.replace(i, "$1$2") : o.test(e) && (e = e.replace(o, "$1$2")), o = g, (i = v).test(e)) { + var c = i.exec(e); + (i = f).test(c[1]) && (i = w, e = e.replace(i, "")) + } else if (o.test(e)) { + t = (c = o.exec(e))[1], (o = p).test(t) && (a = E, s = x, (o = _).test(e = t) ? e += "e" : a.test(e) ? (i = w, e = e.replace(i, "")) : s.test(e) && (e += "e")) + }(i = b).test(e) && (e = (t = (c = i.exec(e))[1]) + "i"); + (i = k).test(e) && (t = (c = i.exec(e))[1], n = c[2], (i = f).test(t) && (e = t + l[n])); + (i = S).test(e) && (t = (c = i.exec(e))[1], n = c[2], (i = f).test(t) && (e = t + u[n])); + if (o = L, (i = T).test(e)) t = (c = i.exec(e))[1], (i = d).test(t) && (e = t); + else if (o.test(e)) { + t = (c = o.exec(e))[1] + c[2], (o = d).test(t) && (e = t) + }(i = R).test(e) && (t = (c = i.exec(e))[1], o = h, a = C, ((i = d).test(t) || o.test(t) && !a.test(t)) && (e = t)); + return o = d, (i = O).test(e) && o.test(e) && (i = w, e = e.replace(i, "")), "y" == r && (e = r.toLowerCase() + e.substr(1)), e + }, function(e) { + return e.update(r) + }), D.Pipeline.registerFunction(D.stemmer, "stemmer"), D.generateStopWordFilter = function(e) { + var t = e.reduce(function(e, t) { + return e[t] = t, e + }, {}); + return function(e) { + if (e && t[e.toString()] !== e.toString()) return e + } + }, D.stopWordFilter = D.generateStopWordFilter(["a", "able", "about", "across", "after", "all", "almost", "also", "am", "among", "an", "and", "any", "are", "as", "at", "be", "because", "been", "but", "by", "can", "cannot", "could", "dear", "did", "do", "does", "either", "else", "ever", "every", "for", "from", "get", "got", "had", "has", "have", "he", "her", "hers", "him", "his", "how", "however", "i", "if", "in", "into", "is", "it", "its", "just", "least", "let", "like", "likely", "may", "me", "might", "most", "must", "my", "neither", "no", "nor", "not", "of", "off", "often", "on", "only", "or", "other", "our", "own", "rather", "said", "say", "says", "she", "should", "since", "so", "some", "than", "that", "the", "their", "them", "then", "there", "these", "they", "this", "tis", "to", "too", "twas", "us", "wants", "was", "we", "were", "what", "when", "where", "which", "while", "who", "whom", "why", "will", "with", "would", "yet", "you", "your"]), D.Pipeline.registerFunction(D.stopWordFilter, "stopWordFilter"), D.trimmer = function(e) { + return e.update(function(e) { + return e.replace(/^\W+/, "").replace(/\W+$/, "") + }) + }, D.Pipeline.registerFunction(D.trimmer, "trimmer"), D.TokenSet = function() { + this.final = !1, this.edges = {}, this.id = D.TokenSet._nextId, D.TokenSet._nextId += 1 + }, D.TokenSet._nextId = 1, D.TokenSet.fromArray = function(e) { + for (var t = new D.TokenSet.Builder, n = 0, r = e.length; n < r; n++) t.insert(e[n]); + return t.finish(), t.root + }, D.TokenSet.fromClause = function(e) { + return "editDistance" in e ? D.TokenSet.fromFuzzyString(e.term, e.editDistance) : D.TokenSet.fromString(e.term) + }, D.TokenSet.fromFuzzyString = function(e, t) { + for (var n = new D.TokenSet, r = [{ + node: n, + editsRemaining: t, + str: e + }]; r.length;) { + var i = r.pop(); + if (0 < i.str.length) { + var o, a = i.str.charAt(0); + a in i.node.edges ? o = i.node.edges[a] : (o = new D.TokenSet, i.node.edges[a] = o), 1 == i.str.length && (o.final = !0), r.push({ + node: o, + editsRemaining: i.editsRemaining, + str: i.str.slice(1) + }) + } + if (0 != i.editsRemaining) { + if ("*" in i.node.edges) var s = i.node.edges["*"]; + else { + s = new D.TokenSet; + i.node.edges["*"] = s + } + if (0 == i.str.length && (s.final = !0), r.push({ + node: s, + editsRemaining: i.editsRemaining - 1, + str: i.str + }), 1 < i.str.length && r.push({ + node: i.node, + editsRemaining: i.editsRemaining - 1, + str: i.str.slice(1) + }), 1 == i.str.length && (i.node.final = !0), 1 <= i.str.length) { + if ("*" in i.node.edges) var c = i.node.edges["*"]; + else { + c = new D.TokenSet; + i.node.edges["*"] = c + } + 1 == i.str.length && (c.final = !0), r.push({ + node: c, + editsRemaining: i.editsRemaining - 1, + str: i.str.slice(1) + }) + } + if (1 < i.str.length) { + var l, u = i.str.charAt(0), + f = i.str.charAt(1); + f in i.node.edges ? l = i.node.edges[f] : (l = new D.TokenSet, i.node.edges[f] = l), 1 == i.str.length && (l.final = !0), r.push({ + node: l, + editsRemaining: i.editsRemaining - 1, + str: u + i.str.slice(2) + }) + } + } + } + return n + }, D.TokenSet.fromString = function(e) { + for (var t = new D.TokenSet, n = t, r = 0, i = e.length; r < i; r++) { + var o = e[r], + a = r == i - 1; + if ("*" == o)(t.edges[o] = t).final = a; + else { + var s = new D.TokenSet; + s.final = a, t.edges[o] = s, t = s + } + } + return n + }, D.TokenSet.prototype.toArray = function() { + for (var e = [], t = [{ + prefix: "", + node: this + }]; t.length;) { + var n = t.pop(), + r = Object.keys(n.node.edges), + i = r.length; + n.node.final && (n.prefix.charAt(0), e.push(n.prefix)); + for (var o = 0; o < i; o++) { + var a = r[o]; + t.push({ + prefix: n.prefix.concat(a), + node: n.node.edges[a] + }) + } + } + return e + }, D.TokenSet.prototype.toString = function() { + if (this._str) return this._str; + for (var e = this.final ? "1" : "0", t = Object.keys(this.edges).sort(), n = t.length, r = 0; r < n; r++) { + var i = t[r]; + e = e + i + this.edges[i].id + } + return e + }, D.TokenSet.prototype.intersect = function(e) { + for (var t = new D.TokenSet, n = void 0, r = [{ + qNode: e, + output: t, + node: this + }]; r.length;) { + n = r.pop(); + for (var i = Object.keys(n.qNode.edges), o = i.length, a = Object.keys(n.node.edges), s = a.length, c = 0; c < o; c++) + for (var l = i[c], u = 0; u < s; u++) { + var f = a[u]; + if (f == l || "*" == l) { + var d = n.node.edges[f], + h = n.qNode.edges[l], + p = d.final && h.final, + m = void 0; + f in n.output.edges ? (m = n.output.edges[f]).final = m.final || p : ((m = new D.TokenSet).final = p, n.output.edges[f] = m), r.push({ + qNode: h, + output: m, + node: d + }) + } + } + } + return t + }, D.TokenSet.Builder = function() { + this.previousWord = "", this.root = new D.TokenSet, this.uncheckedNodes = [], this.minimizedNodes = {} + }, D.TokenSet.Builder.prototype.insert = function(e) { + var t, n = 0; + if (e < this.previousWord) throw new Error("Out of order word insertion"); + for (var r = 0; r < e.length && r < this.previousWord.length && e[r] == this.previousWord[r]; r++) n++; + this.minimize(n), t = 0 == this.uncheckedNodes.length ? this.root : this.uncheckedNodes[this.uncheckedNodes.length - 1].child; + for (r = n; r < e.length; r++) { + var i = new D.TokenSet, + o = e[r]; + t.edges[o] = i, this.uncheckedNodes.push({ + parent: t, + char: o, + child: i + }), t = i + } + t.final = !0, this.previousWord = e + }, D.TokenSet.Builder.prototype.finish = function() { + this.minimize(0) + }, D.TokenSet.Builder.prototype.minimize = function(e) { + for (var t = this.uncheckedNodes.length - 1; e <= t; t--) { + var n = this.uncheckedNodes[t], + r = n.child.toString(); + r in this.minimizedNodes ? n.parent.edges[n.char] = this.minimizedNodes[r] : (n.child._str = r, this.minimizedNodes[r] = n.child), this.uncheckedNodes.pop() + } + }, D.Index = function(e) { + this.invertedIndex = e.invertedIndex, this.fieldVectors = e.fieldVectors, this.tokenSet = e.tokenSet, this.fields = e.fields, this.pipeline = e.pipeline + }, D.Index.prototype.search = function(t) { + return this.query(function(e) { + new D.QueryParser(t, e).parse() + }) + }, D.Index.prototype.query = function(e) { + for (var t = new D.Query(this.fields), n = Object.create(null), r = Object.create(null), i = Object.create(null), o = Object.create(null), a = Object.create(null), s = 0; s < this.fields.length; s++) r[this.fields[s]] = new D.Vector; + e.call(t, t); + for (s = 0; s < t.clauses.length; s++) { + var c = t.clauses[s], + l = null, + u = D.Set.complete; + l = c.usePipeline ? this.pipeline.runString(c.term, { + fields: c.fields + }) : [c.term]; + for (var f = 0; f < l.length; f++) { + var d = l[f]; + c.term = d; + var h = D.TokenSet.fromClause(c), + p = this.tokenSet.intersect(h).toArray(); + if (0 === p.length && c.presence === D.Query.presence.REQUIRED) { + for (var m = 0; m < c.fields.length; m++) { + o[Q = c.fields[m]] = D.Set.empty + } + break + } + for (var y = 0; y < p.length; y++) { + var v = p[y], + g = this.invertedIndex[v], + w = g._index; + for (m = 0; m < c.fields.length; m++) { + var _ = g[Q = c.fields[m]], + E = Object.keys(_), + x = v + "/" + Q, + b = new D.Set(E); + if (c.presence == D.Query.presence.REQUIRED && (u = u.union(b), void 0 === o[Q] && (o[Q] = D.Set.complete)), c.presence != D.Query.presence.PROHIBITED) { + if (r[Q].upsert(w, c.boost, function(e, t) { + return e + t + }), !i[x]) { + for (var k = 0; k < E.length; k++) { + var S, T = E[k], + L = new D.FieldRef(T, Q), + R = _[T]; + void 0 === (S = n[L]) ? n[L] = new D.MatchData(v, Q, R) : S.add(v, Q, R) + } + i[x] = !0 + } + } else void 0 === a[Q] && (a[Q] = D.Set.empty), a[Q] = a[Q].union(b) + } + } + } + if (c.presence === D.Query.presence.REQUIRED) + for (m = 0; m < c.fields.length; m++) { + o[Q = c.fields[m]] = o[Q].intersect(u) + } + } + var O = D.Set.complete, + C = D.Set.empty; + for (s = 0; s < this.fields.length; s++) { + var Q; + o[Q = this.fields[s]] && (O = O.intersect(o[Q])), a[Q] && (C = C.union(a[Q])) + } + var P = Object.keys(n), + A = [], + I = Object.create(null); + if (t.isNegated()) { + P = Object.keys(this.fieldVectors); + for (s = 0; s < P.length; s++) { + L = P[s]; + var M = D.FieldRef.fromString(L); + n[L] = new D.MatchData + } + } + for (s = 0; s < P.length; s++) { + var N = (M = D.FieldRef.fromString(P[s])).docRef; + if (O.contains(N) && !C.contains(N)) { + var j, F = this.fieldVectors[M], + H = r[M.fieldName].similarity(F); + if (void 0 !== (j = I[N])) j.score += H, j.matchData.combine(n[M]); + else { + var q = { + ref: N, + score: H, + matchData: n[M] + }; + I[N] = q, A.push(q) + } + } + } + return A.sort(function(e, t) { + return t.score - e.score + }) + }, D.Index.prototype.toJSON = function() { + var e = Object.keys(this.invertedIndex).sort().map(function(e) { + return [e, this.invertedIndex[e]] + }, this), + t = Object.keys(this.fieldVectors).map(function(e) { + return [e, this.fieldVectors[e].toJSON()] + }, this); + return { + version: D.version, + fields: this.fields, + fieldVectors: t, + invertedIndex: e, + pipeline: this.pipeline.toJSON() + } + }, D.Index.load = function(e) { + var t = {}, + n = {}, + r = e.fieldVectors, + i = Object.create(null), + o = e.invertedIndex, + a = new D.TokenSet.Builder, + s = D.Pipeline.load(e.pipeline); + e.version != D.version && D.utils.warn("Version mismatch when loading serialised index. Current version of lunr '" + D.version + "' does not match serialized index '" + e.version + "'"); + for (var c = 0; c < r.length; c++) { + var l = (f = r[c])[0], + u = f[1]; + n[l] = new D.Vector(u) + } + for (c = 0; c < o.length; c++) { + var f, d = (f = o[c])[0], + h = f[1]; + a.insert(d), i[d] = h + } + return a.finish(), t.fields = e.fields, t.fieldVectors = n, t.invertedIndex = i, t.tokenSet = a.root, t.pipeline = s, new D.Index(t) + }, D.Builder = function() { + this._ref = "id", this._fields = Object.create(null), this._documents = Object.create(null), this.invertedIndex = Object.create(null), this.fieldTermFrequencies = {}, this.fieldLengths = {}, this.tokenizer = D.tokenizer, this.pipeline = new D.Pipeline, this.searchPipeline = new D.Pipeline, this.documentCount = 0, this._b = .75, this._k1 = 1.2, this.termIndex = 0, this.metadataWhitelist = [] + }, D.Builder.prototype.ref = function(e) { + this._ref = e + }, D.Builder.prototype.field = function(e, t) { + if (/\//.test(e)) throw new RangeError("Field '" + e + "' contains illegal character '/'"); + this._fields[e] = t || {} + }, D.Builder.prototype.b = function(e) { + this._b = e < 0 ? 0 : 1 < e ? 1 : e + }, D.Builder.prototype.k1 = function(e) { + this._k1 = e + }, D.Builder.prototype.add = function(e, t) { + var n = e[this._ref], + r = Object.keys(this._fields); + this._documents[n] = t || {}, this.documentCount += 1; + for (var i = 0; i < r.length; i++) { + var o = r[i], + a = this._fields[o].extractor, + s = a ? a(e) : e[o], + c = this.tokenizer(s, { + fields: [o] + }), + l = this.pipeline.run(c), + u = new D.FieldRef(n, o), + f = Object.create(null); + this.fieldTermFrequencies[u] = f, this.fieldLengths[u] = 0, this.fieldLengths[u] += l.length; + for (var d = 0; d < l.length; d++) { + var h = l[d]; + if (null == f[h] && (f[h] = 0), f[h] += 1, null == this.invertedIndex[h]) { + var p = Object.create(null); + p._index = this.termIndex, this.termIndex += 1; + for (var m = 0; m < r.length; m++) p[r[m]] = Object.create(null); + this.invertedIndex[h] = p + } + null == this.invertedIndex[h][o][n] && (this.invertedIndex[h][o][n] = Object.create(null)); + for (var y = 0; y < this.metadataWhitelist.length; y++) { + var v = this.metadataWhitelist[y], + g = h.metadata[v]; + null == this.invertedIndex[h][o][n][v] && (this.invertedIndex[h][o][n][v] = []), this.invertedIndex[h][o][n][v].push(g) + } + } + } + }, D.Builder.prototype.calculateAverageFieldLengths = function() { + for (var e = Object.keys(this.fieldLengths), t = e.length, n = {}, r = {}, i = 0; i < t; i++) { + var o = D.FieldRef.fromString(e[i]), + a = o.fieldName; + r[a] || (r[a] = 0), r[a] += 1, n[a] || (n[a] = 0), n[a] += this.fieldLengths[o] + } + var s = Object.keys(this._fields); + for (i = 0; i < s.length; i++) { + var c = s[i]; + n[c] = n[c] / r[c] + } + this.averageFieldLength = n + }, D.Builder.prototype.createFieldVectors = function() { + for (var e = {}, t = Object.keys(this.fieldTermFrequencies), n = t.length, r = Object.create(null), i = 0; i < n; i++) { + for (var o = D.FieldRef.fromString(t[i]), a = o.fieldName, s = this.fieldLengths[o], c = new D.Vector, l = this.fieldTermFrequencies[o], u = Object.keys(l), f = u.length, d = this._fields[a].boost || 1, h = this._documents[o.docRef].boost || 1, p = 0; p < f; p++) { + var m, y, v, g = u[p], + w = l[g], + _ = this.invertedIndex[g]._index; + void 0 === r[g] ? (m = D.idf(this.invertedIndex[g], this.documentCount), r[g] = m) : m = r[g], y = m * ((this._k1 + 1) * w) / (this._k1 * (1 - this._b + this._b * (s / this.averageFieldLength[a])) + w), y *= d, y *= h, v = Math.round(1e3 * y) / 1e3, c.insert(_, v) + } + e[o] = c + } + this.fieldVectors = e + }, D.Builder.prototype.createTokenSet = function() { + this.tokenSet = D.TokenSet.fromArray(Object.keys(this.invertedIndex).sort()) + }, D.Builder.prototype.build = function() { + return this.calculateAverageFieldLengths(), this.createFieldVectors(), this.createTokenSet(), new D.Index({ + invertedIndex: this.invertedIndex, + fieldVectors: this.fieldVectors, + tokenSet: this.tokenSet, + fields: Object.keys(this._fields), + pipeline: this.searchPipeline + }) + }, D.Builder.prototype.use = function(e) { + var t = Array.prototype.slice.call(arguments, 1); + t.unshift(this), e.apply(this, t) + }, D.MatchData = function(e, t, n) { + for (var r = Object.create(null), i = Object.keys(n || {}), o = 0; o < i.length; o++) { + var a = i[o]; + r[a] = n[a].slice() + } + this.metadata = Object.create(null), void 0 !== e && (this.metadata[e] = Object.create(null), this.metadata[e][t] = r) + }, D.MatchData.prototype.combine = function(e) { + for (var t = Object.keys(e.metadata), n = 0; n < t.length; n++) { + var r = t[n], + i = Object.keys(e.metadata[r]); + null == this.metadata[r] && (this.metadata[r] = Object.create(null)); + for (var o = 0; o < i.length; o++) { + var a = i[o], + s = Object.keys(e.metadata[r][a]); + null == this.metadata[r][a] && (this.metadata[r][a] = Object.create(null)); + for (var c = 0; c < s.length; c++) { + var l = s[c]; + null == this.metadata[r][a][l] ? this.metadata[r][a][l] = e.metadata[r][a][l] : this.metadata[r][a][l] = this.metadata[r][a][l].concat(e.metadata[r][a][l]) + } + } + } + }, D.MatchData.prototype.add = function(e, t, n) { + if (!(e in this.metadata)) return this.metadata[e] = Object.create(null), void(this.metadata[e][t] = n); + if (t in this.metadata[e]) + for (var r = Object.keys(n), i = 0; i < r.length; i++) { + var o = r[i]; + o in this.metadata[e][t] ? this.metadata[e][t][o] = this.metadata[e][t][o].concat(n[o]) : this.metadata[e][t][o] = n[o] + } else this.metadata[e][t] = n + }, D.Query = function(e) { + this.clauses = [], this.allFields = e + }, D.Query.wildcard = new String("*"), D.Query.wildcard.NONE = 0, D.Query.wildcard.LEADING = 1, D.Query.wildcard.TRAILING = 2, D.Query.presence = { + OPTIONAL: 1, + REQUIRED: 2, + PROHIBITED: 3 + }, D.Query.prototype.clause = function(e) { + return "fields" in e || (e.fields = this.allFields), "boost" in e || (e.boost = 1), "usePipeline" in e || (e.usePipeline = !0), "wildcard" in e || (e.wildcard = D.Query.wildcard.NONE), e.wildcard & D.Query.wildcard.LEADING && e.term.charAt(0) != D.Query.wildcard && (e.term = "*" + e.term), e.wildcard & D.Query.wildcard.TRAILING && e.term.slice(-1) != D.Query.wildcard && (e.term = e.term + "*"), "presence" in e || (e.presence = D.Query.presence.OPTIONAL), this.clauses.push(e), this + }, D.Query.prototype.isNegated = function() { + for (var e = 0; e < this.clauses.length; e++) + if (this.clauses[e].presence != D.Query.presence.PROHIBITED) return !1; + return !0 + }, D.Query.prototype.term = function(e, t) { + if (Array.isArray(e)) return e.forEach(function(e) { + this.term(e, D.utils.clone(t)) + }, this), this; + var n = t || {}; + return n.term = e.toString(), this.clause(n), this + }, D.QueryParseError = function(e, t, n) { + this.name = "QueryParseError", this.message = e, this.start = t, this.end = n + }, D.QueryParseError.prototype = new Error, D.QueryLexer = function(e) { + this.lexemes = [], this.str = e, this.length = e.length, this.pos = 0, this.start = 0, this.escapeCharPositions = [] + }, D.QueryLexer.prototype.run = function() { + for (var e = D.QueryLexer.lexText; e;) e = e(this) + }, D.QueryLexer.prototype.sliceString = function() { + for (var e = [], t = this.start, n = this.pos, r = 0; r < this.escapeCharPositions.length; r++) n = this.escapeCharPositions[r], e.push(this.str.slice(t, n)), t = n + 1; + return e.push(this.str.slice(t, this.pos)), this.escapeCharPositions.length = 0, e.join("") + }, D.QueryLexer.prototype.emit = function(e) { + this.lexemes.push({ + type: e, + str: this.sliceString(), + start: this.start, + end: this.pos + }), this.start = this.pos + }, D.QueryLexer.prototype.escapeCharacter = function() { + this.escapeCharPositions.push(this.pos - 1), this.pos += 1 + }, D.QueryLexer.prototype.next = function() { + if (this.pos >= this.length) return D.QueryLexer.EOS; + var e = this.str.charAt(this.pos); + return this.pos += 1, e + }, D.QueryLexer.prototype.width = function() { + return this.pos - this.start + }, D.QueryLexer.prototype.ignore = function() { + this.start == this.pos && (this.pos += 1), this.start = this.pos + }, D.QueryLexer.prototype.backup = function() { + this.pos -= 1 + }, D.QueryLexer.prototype.acceptDigitRun = function() { + for (var e, t; 47 < (t = (e = this.next()).charCodeAt(0)) && t < 58;); + e != D.QueryLexer.EOS && this.backup() + }, D.QueryLexer.prototype.more = function() { + return this.pos < this.length + }, D.QueryLexer.EOS = "EOS", D.QueryLexer.FIELD = "FIELD", D.QueryLexer.TERM = "TERM", D.QueryLexer.EDIT_DISTANCE = "EDIT_DISTANCE", D.QueryLexer.BOOST = "BOOST", D.QueryLexer.PRESENCE = "PRESENCE", D.QueryLexer.lexField = function(e) { + return e.backup(), e.emit(D.QueryLexer.FIELD), e.ignore(), D.QueryLexer.lexText + }, D.QueryLexer.lexTerm = function(e) { + if (1 < e.width() && (e.backup(), e.emit(D.QueryLexer.TERM)), e.ignore(), e.more()) return D.QueryLexer.lexText + }, D.QueryLexer.lexEditDistance = function(e) { + return e.ignore(), e.acceptDigitRun(), e.emit(D.QueryLexer.EDIT_DISTANCE), D.QueryLexer.lexText + }, D.QueryLexer.lexBoost = function(e) { + return e.ignore(), e.acceptDigitRun(), e.emit(D.QueryLexer.BOOST), D.QueryLexer.lexText + }, D.QueryLexer.lexEOS = function(e) { + 0 < e.width() && e.emit(D.QueryLexer.TERM) + }, D.QueryLexer.termSeparator = D.tokenizer.separator, D.QueryLexer.lexText = function(e) { + for (;;) { + var t = e.next(); + if (t == D.QueryLexer.EOS) return D.QueryLexer.lexEOS; + if (92 != t.charCodeAt(0)) { + if (":" == t) return D.QueryLexer.lexField; + if ("~" == t) return e.backup(), 0 < e.width() && e.emit(D.QueryLexer.TERM), D.QueryLexer.lexEditDistance; + if ("^" == t) return e.backup(), 0 < e.width() && e.emit(D.QueryLexer.TERM), D.QueryLexer.lexBoost; + if ("+" == t && 1 === e.width()) return e.emit(D.QueryLexer.PRESENCE), D.QueryLexer.lexText; + if ("-" == t && 1 === e.width()) return e.emit(D.QueryLexer.PRESENCE), D.QueryLexer.lexText; + if (t.match(D.QueryLexer.termSeparator)) return D.QueryLexer.lexTerm + } else e.escapeCharacter() + } + }, D.QueryParser = function(e, t) { + this.lexer = new D.QueryLexer(e), this.query = t, this.currentClause = {}, this.lexemeIdx = 0 + }, D.QueryParser.prototype.parse = function() { + this.lexer.run(), this.lexemes = this.lexer.lexemes; + for (var e = D.QueryParser.parseClause; e;) e = e(this); + return this.query + }, D.QueryParser.prototype.peekLexeme = function() { + return this.lexemes[this.lexemeIdx] + }, D.QueryParser.prototype.consumeLexeme = function() { + var e = this.peekLexeme(); + return this.lexemeIdx += 1, e + }, D.QueryParser.prototype.nextClause = function() { + var e = this.currentClause; + this.query.clause(e), this.currentClause = {} + }, D.QueryParser.parseClause = function(e) { + var t = e.peekLexeme(); + if (null != t) switch (t.type) { + case D.QueryLexer.PRESENCE: + return D.QueryParser.parsePresence; + case D.QueryLexer.FIELD: + return D.QueryParser.parseField; + case D.QueryLexer.TERM: + return D.QueryParser.parseTerm; + default: + var n = "expected either a field or a term, found " + t.type; + throw 1 <= t.str.length && (n += " with value '" + t.str + "'"), new D.QueryParseError(n, t.start, t.end) + } + }, D.QueryParser.parsePresence = function(e) { + var t = e.consumeLexeme(); + if (null != t) { + switch (t.str) { + case "-": + e.currentClause.presence = D.Query.presence.PROHIBITED; + break; + case "+": + e.currentClause.presence = D.Query.presence.REQUIRED; + break; + default: + var n = "unrecognised presence operator'" + t.str + "'"; + throw new D.QueryParseError(n, t.start, t.end) + } + var r = e.peekLexeme(); + if (null == r) { + n = "expecting term or field, found nothing"; + throw new D.QueryParseError(n, t.start, t.end) + } + switch (r.type) { + case D.QueryLexer.FIELD: + return D.QueryParser.parseField; + case D.QueryLexer.TERM: + return D.QueryParser.parseTerm; + default: + n = "expecting term or field, found '" + r.type + "'"; + throw new D.QueryParseError(n, r.start, r.end) + } + } + }, D.QueryParser.parseField = function(e) { + var t = e.consumeLexeme(); + if (null != t) { + if (-1 == e.query.allFields.indexOf(t.str)) { + var n = e.query.allFields.map(function(e) { + return "'" + e + "'" + }).join(", "), + r = "unrecognised field '" + t.str + "', possible fields: " + n; + throw new D.QueryParseError(r, t.start, t.end) + } + e.currentClause.fields = [t.str]; + var i = e.peekLexeme(); + if (null == i) { + r = "expecting term, found nothing"; + throw new D.QueryParseError(r, t.start, t.end) + } + switch (i.type) { + case D.QueryLexer.TERM: + return D.QueryParser.parseTerm; + default: + r = "expecting term, found '" + i.type + "'"; + throw new D.QueryParseError(r, i.start, i.end) + } + } + }, D.QueryParser.parseTerm = function(e) { + var t = e.consumeLexeme(); + if (null != t) { + e.currentClause.term = t.str.toLowerCase(), -1 != t.str.indexOf("*") && (e.currentClause.usePipeline = !1); + var n = e.peekLexeme(); + if (null != n) switch (n.type) { + case D.QueryLexer.TERM: + return e.nextClause(), D.QueryParser.parseTerm; + case D.QueryLexer.FIELD: + return e.nextClause(), D.QueryParser.parseField; + case D.QueryLexer.EDIT_DISTANCE: + return D.QueryParser.parseEditDistance; + case D.QueryLexer.BOOST: + return D.QueryParser.parseBoost; + case D.QueryLexer.PRESENCE: + return e.nextClause(), D.QueryParser.parsePresence; + default: + var r = "Unexpected lexeme type '" + n.type + "'"; + throw new D.QueryParseError(r, n.start, n.end) + } else e.nextClause() + } + }, D.QueryParser.parseEditDistance = function(e) { + var t = e.consumeLexeme(); + if (null != t) { + var n = parseInt(t.str, 10); + if (isNaN(n)) { + var r = "edit distance must be numeric"; + throw new D.QueryParseError(r, t.start, t.end) + } + e.currentClause.editDistance = n; + var i = e.peekLexeme(); + if (null != i) switch (i.type) { + case D.QueryLexer.TERM: + return e.nextClause(), D.QueryParser.parseTerm; + case D.QueryLexer.FIELD: + return e.nextClause(), D.QueryParser.parseField; + case D.QueryLexer.EDIT_DISTANCE: + return D.QueryParser.parseEditDistance; + case D.QueryLexer.BOOST: + return D.QueryParser.parseBoost; + case D.QueryLexer.PRESENCE: + return e.nextClause(), D.QueryParser.parsePresence; + default: + r = "Unexpected lexeme type '" + i.type + "'"; + throw new D.QueryParseError(r, i.start, i.end) + } else e.nextClause() + } + }, D.QueryParser.parseBoost = function(e) { + var t = e.consumeLexeme(); + if (null != t) { + var n = parseInt(t.str, 10); + if (isNaN(n)) { + var r = "boost must be numeric"; + throw new D.QueryParseError(r, t.start, t.end) + } + e.currentClause.boost = n; + var i = e.peekLexeme(); + if (null != i) switch (i.type) { + case D.QueryLexer.TERM: + return e.nextClause(), D.QueryParser.parseTerm; + case D.QueryLexer.FIELD: + return e.nextClause(), D.QueryParser.parseField; + case D.QueryLexer.EDIT_DISTANCE: + return D.QueryParser.parseEditDistance; + case D.QueryLexer.BOOST: + return D.QueryParser.parseBoost; + case D.QueryLexer.PRESENCE: + return e.nextClause(), D.QueryParser.parsePresence; + default: + r = "Unexpected lexeme type '" + i.type + "'"; + throw new D.QueryParseError(r, i.start, i.end) + } else e.nextClause() + } + }, void 0 === (c = "function" == typeof(s = function() { + return D + }) ? s.call(o, a, o, i) : s) || (i.exports = c) + }() +}])); \ No newline at end of file diff --git a/430/_static/javascripts/lunr/lunr.da.js b/430/_static/javascripts/lunr/lunr.da.js new file mode 100644 index 000000000..34910dfe5 --- /dev/null +++ b/430/_static/javascripts/lunr/lunr.da.js @@ -0,0 +1 @@ +!function(e,r){"function"==typeof define&&define.amd?define(r):"object"==typeof exports?module.exports=r():r()(e.lunr)}(this,function(){return function(e){if(void 0===e)throw new Error("Lunr is not present. Please include / require Lunr before this script.");if(void 0===e.stemmerSupport)throw new Error("Lunr stemmer support is not present. Please include / require Lunr stemmer support before this script.");var r,m,i;e.da=function(){this.pipeline.reset(),this.pipeline.add(e.da.trimmer,e.da.stopWordFilter,e.da.stemmer),this.searchPipeline&&(this.searchPipeline.reset(),this.searchPipeline.add(e.da.stemmer))},e.da.wordCharacters="A-Za-zªºÀ-ÖØ-öø-ʸˠ-ˤᴀ-ᴥᴬ-ᵜᵢ-ᵥᵫ-ᵷᵹ-ᶾḀ-ỿⁱⁿₐ-ₜKÅℲⅎⅠ-ↈⱠ-ⱿꜢ-ꞇꞋ-ꞭꞰ-ꞷꟷ-ꟿꬰ-ꭚꭜ-ꭤff-stA-Za-z",e.da.trimmer=e.trimmerSupport.generateTrimmer(e.da.wordCharacters),e.Pipeline.registerFunction(e.da.trimmer,"trimmer-da"),e.da.stemmer=(r=e.stemmerSupport.Among,m=e.stemmerSupport.SnowballProgram,i=new function(){var i,t,n,s=[new r("hed",-1,1),new r("ethed",0,1),new r("ered",-1,1),new r("e",-1,1),new r("erede",3,1),new r("ende",3,1),new r("erende",5,1),new r("ene",3,1),new r("erne",3,1),new r("ere",3,1),new r("en",-1,1),new r("heden",10,1),new r("eren",10,1),new r("er",-1,1),new r("heder",13,1),new r("erer",13,1),new r("s",-1,2),new r("heds",16,1),new r("es",16,1),new r("endes",18,1),new r("erendes",19,1),new r("enes",18,1),new r("ernes",18,1),new r("eres",18,1),new r("ens",16,1),new r("hedens",24,1),new r("erens",24,1),new r("ers",16,1),new r("ets",16,1),new r("erets",28,1),new r("et",-1,1),new r("eret",30,1)],o=[new r("gd",-1,-1),new r("dt",-1,-1),new r("gt",-1,-1),new r("kt",-1,-1)],a=[new r("ig",-1,1),new r("lig",0,1),new r("elig",1,1),new r("els",-1,1),new r("løst",-1,2)],d=[17,65,16,1,0,0,0,0,0,0,0,0,0,0,0,0,48,0,128],u=[239,254,42,3,0,0,0,0,0,0,0,0,0,0,0,0,16],c=new m;function l(){var e,r=c.limit-c.cursor;c.cursor>=t&&(e=c.limit_backward,c.limit_backward=t,c.ket=c.cursor,c.find_among_b(o,4)?(c.bra=c.cursor,c.limit_backward=e,c.cursor=c.limit-r,c.cursor>c.limit_backward&&(c.cursor--,c.bra=c.cursor,c.slice_del())):c.limit_backward=e)}this.setCurrent=function(e){c.setCurrent(e)},this.getCurrent=function(){return c.getCurrent()},this.stem=function(){var e,r=c.cursor;return function(){var e,r=c.cursor+3;if(t=c.limit,0<=r&&r<=c.limit){for(i=r;;){if(e=c.cursor,c.in_grouping(d,97,248)){c.cursor=e;break}if((c.cursor=e)>=c.limit)return;c.cursor++}for(;!c.out_grouping(d,97,248);){if(c.cursor>=c.limit)return;c.cursor++}(t=c.cursor)=t&&(r=c.limit_backward,c.limit_backward=t,c.ket=c.cursor,e=c.find_among_b(s,32),c.limit_backward=r,e))switch(c.bra=c.cursor,e){case 1:c.slice_del();break;case 2:c.in_grouping_b(u,97,229)&&c.slice_del()}}(),c.cursor=c.limit,l(),c.cursor=c.limit,function(){var e,r,i,n=c.limit-c.cursor;if(c.ket=c.cursor,c.eq_s_b(2,"st")&&(c.bra=c.cursor,c.eq_s_b(2,"ig")&&c.slice_del()),c.cursor=c.limit-n,c.cursor>=t&&(r=c.limit_backward,c.limit_backward=t,c.ket=c.cursor,e=c.find_among_b(a,5),c.limit_backward=r,e))switch(c.bra=c.cursor,e){case 1:c.slice_del(),i=c.limit-c.cursor,l(),c.cursor=c.limit-i;break;case 2:c.slice_from("løs")}}(),c.cursor=c.limit,c.cursor>=t&&(e=c.limit_backward,c.limit_backward=t,c.ket=c.cursor,c.out_grouping_b(d,97,248)?(c.bra=c.cursor,n=c.slice_to(n),c.limit_backward=e,c.eq_v_b(n)&&c.slice_del()):c.limit_backward=e),!0}},function(e){return"function"==typeof e.update?e.update(function(e){return i.setCurrent(e),i.stem(),i.getCurrent()}):(i.setCurrent(e),i.stem(),i.getCurrent())}),e.Pipeline.registerFunction(e.da.stemmer,"stemmer-da"),e.da.stopWordFilter=e.generateStopWordFilter("ad af alle alt anden at blev blive bliver da de dem den denne der deres det dette dig din disse dog du efter eller en end er et for fra ham han hans har havde have hende hendes her hos hun hvad hvis hvor i ikke ind jeg jer jo kunne man mange med meget men mig min mine mit mod ned noget nogle nu når og også om op os over på selv sig sin sine sit skal skulle som sådan thi til ud under var vi vil ville vor være været".split(" ")),e.Pipeline.registerFunction(e.da.stopWordFilter,"stopWordFilter-da")}}); \ No newline at end of file diff --git a/430/_static/javascripts/lunr/lunr.de.js b/430/_static/javascripts/lunr/lunr.de.js new file mode 100644 index 000000000..1529892c8 --- /dev/null +++ b/430/_static/javascripts/lunr/lunr.de.js @@ -0,0 +1 @@ +!function(e,r){"function"==typeof define&&define.amd?define(r):"object"==typeof exports?module.exports=r():r()(e.lunr)}(this,function(){return function(e){if(void 0===e)throw new Error("Lunr is not present. Please include / require Lunr before this script.");if(void 0===e.stemmerSupport)throw new Error("Lunr stemmer support is not present. Please include / require Lunr stemmer support before this script.");var _,p,r;e.de=function(){this.pipeline.reset(),this.pipeline.add(e.de.trimmer,e.de.stopWordFilter,e.de.stemmer),this.searchPipeline&&(this.searchPipeline.reset(),this.searchPipeline.add(e.de.stemmer))},e.de.wordCharacters="A-Za-zªºÀ-ÖØ-öø-ʸˠ-ˤᴀ-ᴥᴬ-ᵜᵢ-ᵥᵫ-ᵷᵹ-ᶾḀ-ỿⁱⁿₐ-ₜKÅℲⅎⅠ-ↈⱠ-ⱿꜢ-ꞇꞋ-ꞭꞰ-ꞷꟷ-ꟿꬰ-ꭚꭜ-ꭤff-stA-Za-z",e.de.trimmer=e.trimmerSupport.generateTrimmer(e.de.wordCharacters),e.Pipeline.registerFunction(e.de.trimmer,"trimmer-de"),e.de.stemmer=(_=e.stemmerSupport.Among,p=e.stemmerSupport.SnowballProgram,r=new function(){var r,n,i,s=[new _("",-1,6),new _("U",0,2),new _("Y",0,1),new _("ä",0,3),new _("ö",0,4),new _("ü",0,5)],o=[new _("e",-1,2),new _("em",-1,1),new _("en",-1,2),new _("ern",-1,1),new _("er",-1,1),new _("s",-1,3),new _("es",5,2)],c=[new _("en",-1,1),new _("er",-1,1),new _("st",-1,2),new _("est",2,1)],u=[new _("ig",-1,1),new _("lich",-1,1)],a=[new _("end",-1,1),new _("ig",-1,2),new _("ung",-1,1),new _("lich",-1,3),new _("isch",-1,2),new _("ik",-1,2),new _("heit",-1,3),new _("keit",-1,4)],t=[17,65,16,1,0,0,0,0,0,0,0,0,0,0,0,0,8,0,32,8],d=[117,30,5],l=[117,30,4],m=new p;function h(e,r,n){return!(!m.eq_s(1,e)||(m.ket=m.cursor,!m.in_grouping(t,97,252)))&&(m.slice_from(r),m.cursor=n,!0)}function w(){for(;!m.in_grouping(t,97,252);){if(m.cursor>=m.limit)return!0;m.cursor++}for(;!m.out_grouping(t,97,252);){if(m.cursor>=m.limit)return!0;m.cursor++}return!1}function f(){return i<=m.cursor}function b(){return n<=m.cursor}this.setCurrent=function(e){m.setCurrent(e)},this.getCurrent=function(){return m.getCurrent()},this.stem=function(){var e=m.cursor;return function(){for(var e,r,n,i,s=m.cursor;;)if(e=m.cursor,m.bra=e,m.eq_s(1,"ß"))m.ket=m.cursor,m.slice_from("ss");else{if(e>=m.limit)break;m.cursor=e+1}for(m.cursor=s;;)for(r=m.cursor;;){if(n=m.cursor,m.in_grouping(t,97,252)){if(i=m.cursor,m.bra=i,h("u","U",n))break;if(m.cursor=i,h("y","Y",n))break}if(n>=m.limit)return m.cursor=r;m.cursor=n+1}}(),m.cursor=e,function(){i=m.limit,n=i;var e=m.cursor+3;0<=e&&e<=m.limit&&(r=e,w()||((i=m.cursor)=m.limit)return;m.cursor++}}}(),!0}},function(e){return"function"==typeof e.update?e.update(function(e){return r.setCurrent(e),r.stem(),r.getCurrent()}):(r.setCurrent(e),r.stem(),r.getCurrent())}),e.Pipeline.registerFunction(e.de.stemmer,"stemmer-de"),e.de.stopWordFilter=e.generateStopWordFilter("aber alle allem allen aller alles als also am an ander andere anderem anderen anderer anderes anderm andern anderr anders auch auf aus bei bin bis bist da damit dann das dasselbe dazu daß dein deine deinem deinen deiner deines dem demselben den denn denselben der derer derselbe derselben des desselben dessen dich die dies diese dieselbe dieselben diesem diesen dieser dieses dir doch dort du durch ein eine einem einen einer eines einig einige einigem einigen einiger einiges einmal er es etwas euch euer eure eurem euren eurer eures für gegen gewesen hab habe haben hat hatte hatten hier hin hinter ich ihm ihn ihnen ihr ihre ihrem ihren ihrer ihres im in indem ins ist jede jedem jeden jeder jedes jene jenem jenen jener jenes jetzt kann kein keine keinem keinen keiner keines können könnte machen man manche manchem manchen mancher manches mein meine meinem meinen meiner meines mich mir mit muss musste nach nicht nichts noch nun nur ob oder ohne sehr sein seine seinem seinen seiner seines selbst sich sie sind so solche solchem solchen solcher solches soll sollte sondern sonst um und uns unse unsem unsen unser unses unter viel vom von vor war waren warst was weg weil weiter welche welchem welchen welcher welches wenn werde werden wie wieder will wir wird wirst wo wollen wollte während würde würden zu zum zur zwar zwischen über".split(" ")),e.Pipeline.registerFunction(e.de.stopWordFilter,"stopWordFilter-de")}}); \ No newline at end of file diff --git a/430/_static/javascripts/lunr/lunr.du.js b/430/_static/javascripts/lunr/lunr.du.js new file mode 100644 index 000000000..52632004a --- /dev/null +++ b/430/_static/javascripts/lunr/lunr.du.js @@ -0,0 +1 @@ +!function(e,r){"function"==typeof define&&define.amd?define(r):"object"==typeof exports?module.exports=r():r()(e.lunr)}(this,function(){return function(e){if(void 0===e)throw new Error("Lunr is not present. Please include / require Lunr before this script.");if(void 0===e.stemmerSupport)throw new Error("Lunr stemmer support is not present. Please include / require Lunr stemmer support before this script.");var v,q,r;console.warn('[Lunr Languages] Please use the "nl" instead of the "du". The "nl" code is the standard code for Dutch language, and "du" will be removed in the next major versions.'),e.du=function(){this.pipeline.reset(),this.pipeline.add(e.du.trimmer,e.du.stopWordFilter,e.du.stemmer),this.searchPipeline&&(this.searchPipeline.reset(),this.searchPipeline.add(e.du.stemmer))},e.du.wordCharacters="A-Za-zªºÀ-ÖØ-öø-ʸˠ-ˤᴀ-ᴥᴬ-ᵜᵢ-ᵥᵫ-ᵷᵹ-ᶾḀ-ỿⁱⁿₐ-ₜKÅℲⅎⅠ-ↈⱠ-ⱿꜢ-ꞇꞋ-ꞭꞰ-ꞷꟷ-ꟿꬰ-ꭚꭜ-ꭤff-stA-Za-z",e.du.trimmer=e.trimmerSupport.generateTrimmer(e.du.wordCharacters),e.Pipeline.registerFunction(e.du.trimmer,"trimmer-du"),e.du.stemmer=(v=e.stemmerSupport.Among,q=e.stemmerSupport.SnowballProgram,r=new function(){var r,i,u,o=[new v("",-1,6),new v("á",0,1),new v("ä",0,1),new v("é",0,2),new v("ë",0,2),new v("í",0,3),new v("ï",0,3),new v("ó",0,4),new v("ö",0,4),new v("ú",0,5),new v("ü",0,5)],n=[new v("",-1,3),new v("I",0,2),new v("Y",0,1)],t=[new v("dd",-1,-1),new v("kk",-1,-1),new v("tt",-1,-1)],c=[new v("ene",-1,2),new v("se",-1,3),new v("en",-1,2),new v("heden",2,1),new v("s",-1,3)],a=[new v("end",-1,1),new v("ig",-1,2),new v("ing",-1,1),new v("lijk",-1,3),new v("baar",-1,4),new v("bar",-1,5)],l=[new v("aa",-1,-1),new v("ee",-1,-1),new v("oo",-1,-1),new v("uu",-1,-1)],m=[17,65,16,1,0,0,0,0,0,0,0,0,0,0,0,0,128],d=[1,0,0,17,65,16,1,0,0,0,0,0,0,0,0,0,0,0,0,128],f=[17,67,16,1,0,0,0,0,0,0,0,0,0,0,0,0,128],_=new q;function s(e){return(_.cursor=e)>=_.limit||(_.cursor++,!1)}function w(){for(;!_.in_grouping(m,97,232);){if(_.cursor>=_.limit)return!0;_.cursor++}for(;!_.out_grouping(m,97,232);){if(_.cursor>=_.limit)return!0;_.cursor++}return!1}function b(){return i<=_.cursor}function p(){return r<=_.cursor}function g(){var e=_.limit-_.cursor;_.find_among_b(t,3)&&(_.cursor=_.limit-e,_.ket=_.cursor,_.cursor>_.limit_backward&&(_.cursor--,_.bra=_.cursor,_.slice_del()))}function h(){var e;u=!1,_.ket=_.cursor,_.eq_s_b(1,"e")&&(_.bra=_.cursor,b()&&(e=_.limit-_.cursor,_.out_grouping_b(m,97,232)&&(_.cursor=_.limit-e,_.slice_del(),u=!0,g())))}function k(){var e;b()&&(e=_.limit-_.cursor,_.out_grouping_b(m,97,232)&&(_.cursor=_.limit-e,_.eq_s_b(3,"gem")||(_.cursor=_.limit-e,_.slice_del(),g())))}this.setCurrent=function(e){_.setCurrent(e)},this.getCurrent=function(){return _.getCurrent()},this.stem=function(){var e=_.cursor;return function(){for(var e,r,i,n=_.cursor;;){if(_.bra=_.cursor,e=_.find_among(o,11))switch(_.ket=_.cursor,e){case 1:_.slice_from("a");continue;case 2:_.slice_from("e");continue;case 3:_.slice_from("i");continue;case 4:_.slice_from("o");continue;case 5:_.slice_from("u");continue;case 6:if(_.cursor>=_.limit)break;_.cursor++;continue}break}for(_.cursor=n,_.bra=n,_.eq_s(1,"y")?(_.ket=_.cursor,_.slice_from("Y")):_.cursor=n;;)if(r=_.cursor,_.in_grouping(m,97,232)){if(i=_.cursor,_.bra=i,_.eq_s(1,"i"))_.ket=_.cursor,_.in_grouping(m,97,232)&&(_.slice_from("I"),_.cursor=r);else if(_.cursor=i,_.eq_s(1,"y"))_.ket=_.cursor,_.slice_from("Y"),_.cursor=r;else if(s(r))break}else if(s(r))break}(),_.cursor=e,i=_.limit,r=i,w()||((i=_.cursor)<3&&(i=3),w()||(r=_.cursor)),_.limit_backward=e,_.cursor=_.limit,function(){var e,r,i,n,o,t,s=_.limit-_.cursor;if(_.ket=_.cursor,e=_.find_among_b(c,5))switch(_.bra=_.cursor,e){case 1:b()&&_.slice_from("heid");break;case 2:k();break;case 3:b()&&_.out_grouping_b(f,97,232)&&_.slice_del()}if(_.cursor=_.limit-s,h(),_.cursor=_.limit-s,_.ket=_.cursor,_.eq_s_b(4,"heid")&&(_.bra=_.cursor,p()&&(r=_.limit-_.cursor,_.eq_s_b(1,"c")||(_.cursor=_.limit-r,_.slice_del(),_.ket=_.cursor,_.eq_s_b(2,"en")&&(_.bra=_.cursor,k())))),_.cursor=_.limit-s,_.ket=_.cursor,e=_.find_among_b(a,6))switch(_.bra=_.cursor,e){case 1:if(p()){if(_.slice_del(),i=_.limit-_.cursor,_.ket=_.cursor,_.eq_s_b(2,"ig")&&(_.bra=_.cursor,p()&&(n=_.limit-_.cursor,!_.eq_s_b(1,"e")))){_.cursor=_.limit-n,_.slice_del();break}_.cursor=_.limit-i,g()}break;case 2:p()&&(o=_.limit-_.cursor,_.eq_s_b(1,"e")||(_.cursor=_.limit-o,_.slice_del()));break;case 3:p()&&(_.slice_del(),h());break;case 4:p()&&_.slice_del();break;case 5:p()&&u&&_.slice_del()}_.cursor=_.limit-s,_.out_grouping_b(d,73,232)&&(t=_.limit-_.cursor,_.find_among_b(l,4)&&_.out_grouping_b(m,97,232)&&(_.cursor=_.limit-t,_.ket=_.cursor,_.cursor>_.limit_backward&&(_.cursor--,_.bra=_.cursor,_.slice_del())))}(),_.cursor=_.limit_backward,function(){for(var e;;)if(_.bra=_.cursor,e=_.find_among(n,3))switch(_.ket=_.cursor,e){case 1:_.slice_from("y");break;case 2:_.slice_from("i");break;case 3:if(_.cursor>=_.limit)return;_.cursor++}}(),!0}},function(e){return"function"==typeof e.update?e.update(function(e){return r.setCurrent(e),r.stem(),r.getCurrent()}):(r.setCurrent(e),r.stem(),r.getCurrent())}),e.Pipeline.registerFunction(e.du.stemmer,"stemmer-du"),e.du.stopWordFilter=e.generateStopWordFilter(" aan al alles als altijd andere ben bij daar dan dat de der deze die dit doch doen door dus een eens en er ge geen geweest haar had heb hebben heeft hem het hier hij hoe hun iemand iets ik in is ja je kan kon kunnen maar me meer men met mij mijn moet na naar niet niets nog nu of om omdat onder ons ook op over reeds te tegen toch toen tot u uit uw van veel voor want waren was wat werd wezen wie wil worden wordt zal ze zelf zich zij zijn zo zonder zou".split(" ")),e.Pipeline.registerFunction(e.du.stopWordFilter,"stopWordFilter-du")}}); \ No newline at end of file diff --git a/430/_static/javascripts/lunr/lunr.es.js b/430/_static/javascripts/lunr/lunr.es.js new file mode 100644 index 000000000..9de6c09cb --- /dev/null +++ b/430/_static/javascripts/lunr/lunr.es.js @@ -0,0 +1 @@ +!function(e,s){"function"==typeof define&&define.amd?define(s):"object"==typeof exports?module.exports=s():s()(e.lunr)}(this,function(){return function(e){if(void 0===e)throw new Error("Lunr is not present. Please include / require Lunr before this script.");if(void 0===e.stemmerSupport)throw new Error("Lunr stemmer support is not present. Please include / require Lunr stemmer support before this script.");var C,P,s;e.es=function(){this.pipeline.reset(),this.pipeline.add(e.es.trimmer,e.es.stopWordFilter,e.es.stemmer),this.searchPipeline&&(this.searchPipeline.reset(),this.searchPipeline.add(e.es.stemmer))},e.es.wordCharacters="A-Za-zªºÀ-ÖØ-öø-ʸˠ-ˤᴀ-ᴥᴬ-ᵜᵢ-ᵥᵫ-ᵷᵹ-ᶾḀ-ỿⁱⁿₐ-ₜKÅℲⅎⅠ-ↈⱠ-ⱿꜢ-ꞇꞋ-ꞭꞰ-ꞷꟷ-ꟿꬰ-ꭚꭜ-ꭤff-stA-Za-z",e.es.trimmer=e.trimmerSupport.generateTrimmer(e.es.wordCharacters),e.Pipeline.registerFunction(e.es.trimmer,"trimmer-es"),e.es.stemmer=(C=e.stemmerSupport.Among,P=e.stemmerSupport.SnowballProgram,s=new function(){var r,n,i,a=[new C("",-1,6),new C("á",0,1),new C("é",0,2),new C("í",0,3),new C("ó",0,4),new C("ú",0,5)],t=[new C("la",-1,-1),new C("sela",0,-1),new C("le",-1,-1),new C("me",-1,-1),new C("se",-1,-1),new C("lo",-1,-1),new C("selo",5,-1),new C("las",-1,-1),new C("selas",7,-1),new C("les",-1,-1),new C("los",-1,-1),new C("selos",10,-1),new C("nos",-1,-1)],o=[new C("ando",-1,6),new C("iendo",-1,6),new C("yendo",-1,7),new C("ándo",-1,2),new C("iéndo",-1,1),new C("ar",-1,6),new C("er",-1,6),new C("ir",-1,6),new C("ár",-1,3),new C("ér",-1,4),new C("ír",-1,5)],s=[new C("ic",-1,-1),new C("ad",-1,-1),new C("os",-1,-1),new C("iv",-1,1)],u=[new C("able",-1,1),new C("ible",-1,1),new C("ante",-1,1)],w=[new C("ic",-1,1),new C("abil",-1,1),new C("iv",-1,1)],c=[new C("ica",-1,1),new C("ancia",-1,2),new C("encia",-1,5),new C("adora",-1,2),new C("osa",-1,1),new C("ista",-1,1),new C("iva",-1,9),new C("anza",-1,1),new C("logía",-1,3),new C("idad",-1,8),new C("able",-1,1),new C("ible",-1,1),new C("ante",-1,2),new C("mente",-1,7),new C("amente",13,6),new C("ación",-1,2),new C("ución",-1,4),new C("ico",-1,1),new C("ismo",-1,1),new C("oso",-1,1),new C("amiento",-1,1),new C("imiento",-1,1),new C("ivo",-1,9),new C("ador",-1,2),new C("icas",-1,1),new C("ancias",-1,2),new C("encias",-1,5),new C("adoras",-1,2),new C("osas",-1,1),new C("istas",-1,1),new C("ivas",-1,9),new C("anzas",-1,1),new C("logías",-1,3),new C("idades",-1,8),new C("ables",-1,1),new C("ibles",-1,1),new C("aciones",-1,2),new C("uciones",-1,4),new C("adores",-1,2),new C("antes",-1,2),new C("icos",-1,1),new C("ismos",-1,1),new C("osos",-1,1),new C("amientos",-1,1),new C("imientos",-1,1),new C("ivos",-1,9)],m=[new C("ya",-1,1),new C("ye",-1,1),new C("yan",-1,1),new C("yen",-1,1),new C("yeron",-1,1),new C("yendo",-1,1),new C("yo",-1,1),new C("yas",-1,1),new C("yes",-1,1),new C("yais",-1,1),new C("yamos",-1,1),new C("yó",-1,1)],l=[new C("aba",-1,2),new C("ada",-1,2),new C("ida",-1,2),new C("ara",-1,2),new C("iera",-1,2),new C("ía",-1,2),new C("aría",5,2),new C("ería",5,2),new C("iría",5,2),new C("ad",-1,2),new C("ed",-1,2),new C("id",-1,2),new C("ase",-1,2),new C("iese",-1,2),new C("aste",-1,2),new C("iste",-1,2),new C("an",-1,2),new C("aban",16,2),new C("aran",16,2),new C("ieran",16,2),new C("ían",16,2),new C("arían",20,2),new C("erían",20,2),new C("irían",20,2),new C("en",-1,1),new C("asen",24,2),new C("iesen",24,2),new C("aron",-1,2),new C("ieron",-1,2),new C("arán",-1,2),new C("erán",-1,2),new C("irán",-1,2),new C("ado",-1,2),new C("ido",-1,2),new C("ando",-1,2),new C("iendo",-1,2),new C("ar",-1,2),new C("er",-1,2),new C("ir",-1,2),new C("as",-1,2),new C("abas",39,2),new C("adas",39,2),new C("idas",39,2),new C("aras",39,2),new C("ieras",39,2),new C("ías",39,2),new C("arías",45,2),new C("erías",45,2),new C("irías",45,2),new C("es",-1,1),new C("ases",49,2),new C("ieses",49,2),new C("abais",-1,2),new C("arais",-1,2),new C("ierais",-1,2),new C("íais",-1,2),new C("aríais",55,2),new C("eríais",55,2),new C("iríais",55,2),new C("aseis",-1,2),new C("ieseis",-1,2),new C("asteis",-1,2),new C("isteis",-1,2),new C("áis",-1,2),new C("éis",-1,1),new C("aréis",64,2),new C("eréis",64,2),new C("iréis",64,2),new C("ados",-1,2),new C("idos",-1,2),new C("amos",-1,2),new C("ábamos",70,2),new C("áramos",70,2),new C("iéramos",70,2),new C("íamos",70,2),new C("aríamos",74,2),new C("eríamos",74,2),new C("iríamos",74,2),new C("emos",-1,1),new C("aremos",78,2),new C("eremos",78,2),new C("iremos",78,2),new C("ásemos",78,2),new C("iésemos",78,2),new C("imos",-1,2),new C("arás",-1,2),new C("erás",-1,2),new C("irás",-1,2),new C("ís",-1,2),new C("ará",-1,2),new C("erá",-1,2),new C("irá",-1,2),new C("aré",-1,2),new C("eré",-1,2),new C("iré",-1,2),new C("ió",-1,2)],d=[new C("a",-1,1),new C("e",-1,2),new C("o",-1,1),new C("os",-1,1),new C("á",-1,1),new C("é",-1,2),new C("í",-1,1),new C("ó",-1,1)],b=[17,65,16,0,0,0,0,0,0,0,0,0,0,0,0,0,1,17,4,10],f=new P;function _(){if(f.out_grouping(b,97,252)){for(;!f.in_grouping(b,97,252);){if(f.cursor>=f.limit)return!0;f.cursor++}return!1}return!0}function h(){var e,s=f.cursor;if(function(){if(f.in_grouping(b,97,252)){var e=f.cursor;if(_()){if(f.cursor=e,!f.in_grouping(b,97,252))return!0;for(;!f.out_grouping(b,97,252);){if(f.cursor>=f.limit)return!0;f.cursor++}}return!1}return!0}()){if(f.cursor=s,!f.out_grouping(b,97,252))return;if(e=f.cursor,_()){if(f.cursor=e,!f.in_grouping(b,97,252)||f.cursor>=f.limit)return;f.cursor++}}i=f.cursor}function v(){for(;!f.in_grouping(b,97,252);){if(f.cursor>=f.limit)return!1;f.cursor++}for(;!f.out_grouping(b,97,252);){if(f.cursor>=f.limit)return!1;f.cursor++}return!0}function p(){return i<=f.cursor}function g(){return r<=f.cursor}function k(e,s){if(!g())return!0;f.slice_del(),f.ket=f.cursor;var r=f.find_among_b(e,s);return r&&(f.bra=f.cursor,1==r&&g()&&f.slice_del()),!1}function y(e){return!g()||(f.slice_del(),f.ket=f.cursor,f.eq_s_b(2,e)&&(f.bra=f.cursor,g()&&f.slice_del()),!1)}function q(){var e;if(f.ket=f.cursor,e=f.find_among_b(c,46)){switch(f.bra=f.cursor,e){case 1:if(!g())return!1;f.slice_del();break;case 2:if(y("ic"))return!1;break;case 3:if(!g())return!1;f.slice_from("log");break;case 4:if(!g())return!1;f.slice_from("u");break;case 5:if(!g())return!1;f.slice_from("ente");break;case 6:if(!(n<=f.cursor))return!1;f.slice_del(),f.ket=f.cursor,(e=f.find_among_b(s,4))&&(f.bra=f.cursor,g()&&(f.slice_del(),1==e&&(f.ket=f.cursor,f.eq_s_b(2,"at")&&(f.bra=f.cursor,g()&&f.slice_del()))));break;case 7:if(k(u,3))return!1;break;case 8:if(k(w,3))return!1;break;case 9:if(y("at"))return!1}return!0}return!1}this.setCurrent=function(e){f.setCurrent(e)},this.getCurrent=function(){return f.getCurrent()},this.stem=function(){var e,s=f.cursor;return e=f.cursor,i=f.limit,r=n=i,h(),f.cursor=e,v()&&(n=f.cursor,v()&&(r=f.cursor)),f.limit_backward=s,f.cursor=f.limit,function(){var e;if(f.ket=f.cursor,f.find_among_b(t,13)&&(f.bra=f.cursor,(e=f.find_among_b(o,11))&&p()))switch(e){case 1:f.bra=f.cursor,f.slice_from("iendo");break;case 2:f.bra=f.cursor,f.slice_from("ando");break;case 3:f.bra=f.cursor,f.slice_from("ar");break;case 4:f.bra=f.cursor,f.slice_from("er");break;case 5:f.bra=f.cursor,f.slice_from("ir");break;case 6:f.slice_del();break;case 7:f.eq_s_b(1,"u")&&f.slice_del()}}(),f.cursor=f.limit,q()||(f.cursor=f.limit,function(){var e,s;if(f.cursor>=i&&(s=f.limit_backward,f.limit_backward=i,f.ket=f.cursor,e=f.find_among_b(m,12),f.limit_backward=s,e)){if(f.bra=f.cursor,1==e){if(!f.eq_s_b(1,"u"))return!1;f.slice_del()}return!0}return!1}()||(f.cursor=f.limit,function(){var e,s,r,n;if(f.cursor>=i&&(s=f.limit_backward,f.limit_backward=i,f.ket=f.cursor,e=f.find_among_b(l,96),f.limit_backward=s,e))switch(f.bra=f.cursor,e){case 1:r=f.limit-f.cursor,f.eq_s_b(1,"u")?(n=f.limit-f.cursor,f.eq_s_b(1,"g")?f.cursor=f.limit-n:f.cursor=f.limit-r):f.cursor=f.limit-r,f.bra=f.cursor;case 2:f.slice_del()}}())),f.cursor=f.limit,function(){var e,s;if(f.ket=f.cursor,e=f.find_among_b(d,8))switch(f.bra=f.cursor,e){case 1:p()&&f.slice_del();break;case 2:p()&&(f.slice_del(),f.ket=f.cursor,f.eq_s_b(1,"u")&&(f.bra=f.cursor,s=f.limit-f.cursor,f.eq_s_b(1,"g")&&(f.cursor=f.limit-s,p()&&f.slice_del())))}}(),f.cursor=f.limit_backward,function(){for(var e;;){if(f.bra=f.cursor,e=f.find_among(a,6))switch(f.ket=f.cursor,e){case 1:f.slice_from("a");continue;case 2:f.slice_from("e");continue;case 3:f.slice_from("i");continue;case 4:f.slice_from("o");continue;case 5:f.slice_from("u");continue;case 6:if(f.cursor>=f.limit)break;f.cursor++;continue}break}}(),!0}},function(e){return"function"==typeof e.update?e.update(function(e){return s.setCurrent(e),s.stem(),s.getCurrent()}):(s.setCurrent(e),s.stem(),s.getCurrent())}),e.Pipeline.registerFunction(e.es.stemmer,"stemmer-es"),e.es.stopWordFilter=e.generateStopWordFilter("a al algo algunas algunos ante antes como con contra cual cuando de del desde donde durante e el ella ellas ellos en entre era erais eran eras eres es esa esas ese eso esos esta estaba estabais estaban estabas estad estada estadas estado estados estamos estando estar estaremos estará estarán estarás estaré estaréis estaría estaríais estaríamos estarían estarías estas este estemos esto estos estoy estuve estuviera estuvierais estuvieran estuvieras estuvieron estuviese estuvieseis estuviesen estuvieses estuvimos estuviste estuvisteis estuviéramos estuviésemos estuvo está estábamos estáis están estás esté estéis estén estés fue fuera fuerais fueran fueras fueron fuese fueseis fuesen fueses fui fuimos fuiste fuisteis fuéramos fuésemos ha habida habidas habido habidos habiendo habremos habrá habrán habrás habré habréis habría habríais habríamos habrían habrías habéis había habíais habíamos habían habías han has hasta hay haya hayamos hayan hayas hayáis he hemos hube hubiera hubierais hubieran hubieras hubieron hubiese hubieseis hubiesen hubieses hubimos hubiste hubisteis hubiéramos hubiésemos hubo la las le les lo los me mi mis mucho muchos muy más mí mía mías mío míos nada ni no nos nosotras nosotros nuestra nuestras nuestro nuestros o os otra otras otro otros para pero poco por porque que quien quienes qué se sea seamos sean seas seremos será serán serás seré seréis sería seríais seríamos serían serías seáis sido siendo sin sobre sois somos son soy su sus suya suyas suyo suyos sí también tanto te tendremos tendrá tendrán tendrás tendré tendréis tendría tendríais tendríamos tendrían tendrías tened tenemos tenga tengamos tengan tengas tengo tengáis tenida tenidas tenido tenidos teniendo tenéis tenía teníais teníamos tenían tenías ti tiene tienen tienes todo todos tu tus tuve tuviera tuvierais tuvieran tuvieras tuvieron tuviese tuvieseis tuviesen tuvieses tuvimos tuviste tuvisteis tuviéramos tuviésemos tuvo tuya tuyas tuyo tuyos tú un una uno unos vosotras vosotros vuestra vuestras vuestro vuestros y ya yo él éramos".split(" ")),e.Pipeline.registerFunction(e.es.stopWordFilter,"stopWordFilter-es")}}); \ No newline at end of file diff --git a/430/_static/javascripts/lunr/lunr.fi.js b/430/_static/javascripts/lunr/lunr.fi.js new file mode 100644 index 000000000..2f9bf5aeb --- /dev/null +++ b/430/_static/javascripts/lunr/lunr.fi.js @@ -0,0 +1 @@ +!function(i,e){"function"==typeof define&&define.amd?define(e):"object"==typeof exports?module.exports=e():e()(i.lunr)}(this,function(){return function(i){if(void 0===i)throw new Error("Lunr is not present. Please include / require Lunr before this script.");if(void 0===i.stemmerSupport)throw new Error("Lunr stemmer support is not present. Please include / require Lunr stemmer support before this script.");var v,C,e;i.fi=function(){this.pipeline.reset(),this.pipeline.add(i.fi.trimmer,i.fi.stopWordFilter,i.fi.stemmer),this.searchPipeline&&(this.searchPipeline.reset(),this.searchPipeline.add(i.fi.stemmer))},i.fi.wordCharacters="A-Za-zªºÀ-ÖØ-öø-ʸˠ-ˤᴀ-ᴥᴬ-ᵜᵢ-ᵥᵫ-ᵷᵹ-ᶾḀ-ỿⁱⁿₐ-ₜKÅℲⅎⅠ-ↈⱠ-ⱿꜢ-ꞇꞋ-ꞭꞰ-ꞷꟷ-ꟿꬰ-ꭚꭜ-ꭤff-stA-Za-z",i.fi.trimmer=i.trimmerSupport.generateTrimmer(i.fi.wordCharacters),i.Pipeline.registerFunction(i.fi.trimmer,"trimmer-fi"),i.fi.stemmer=(v=i.stemmerSupport.Among,C=i.stemmerSupport.SnowballProgram,e=new function(){var n,t,l,o,r=[new v("pa",-1,1),new v("sti",-1,2),new v("kaan",-1,1),new v("han",-1,1),new v("kin",-1,1),new v("hän",-1,1),new v("kään",-1,1),new v("ko",-1,1),new v("pä",-1,1),new v("kö",-1,1)],s=[new v("lla",-1,-1),new v("na",-1,-1),new v("ssa",-1,-1),new v("ta",-1,-1),new v("lta",3,-1),new v("sta",3,-1)],a=[new v("llä",-1,-1),new v("nä",-1,-1),new v("ssä",-1,-1),new v("tä",-1,-1),new v("ltä",3,-1),new v("stä",3,-1)],u=[new v("lle",-1,-1),new v("ine",-1,-1)],c=[new v("nsa",-1,3),new v("mme",-1,3),new v("nne",-1,3),new v("ni",-1,2),new v("si",-1,1),new v("an",-1,4),new v("en",-1,6),new v("än",-1,5),new v("nsä",-1,3)],i=[new v("aa",-1,-1),new v("ee",-1,-1),new v("ii",-1,-1),new v("oo",-1,-1),new v("uu",-1,-1),new v("ää",-1,-1),new v("öö",-1,-1)],m=[new v("a",-1,8),new v("lla",0,-1),new v("na",0,-1),new v("ssa",0,-1),new v("ta",0,-1),new v("lta",4,-1),new v("sta",4,-1),new v("tta",4,9),new v("lle",-1,-1),new v("ine",-1,-1),new v("ksi",-1,-1),new v("n",-1,7),new v("han",11,1),new v("den",11,-1,q),new v("seen",11,-1,j),new v("hen",11,2),new v("tten",11,-1,q),new v("hin",11,3),new v("siin",11,-1,q),new v("hon",11,4),new v("hän",11,5),new v("hön",11,6),new v("ä",-1,8),new v("llä",22,-1),new v("nä",22,-1),new v("ssä",22,-1),new v("tä",22,-1),new v("ltä",26,-1),new v("stä",26,-1),new v("ttä",26,9)],w=[new v("eja",-1,-1),new v("mma",-1,1),new v("imma",1,-1),new v("mpa",-1,1),new v("impa",3,-1),new v("mmi",-1,1),new v("immi",5,-1),new v("mpi",-1,1),new v("impi",7,-1),new v("ejä",-1,-1),new v("mmä",-1,1),new v("immä",10,-1),new v("mpä",-1,1),new v("impä",12,-1)],_=[new v("i",-1,-1),new v("j",-1,-1)],k=[new v("mma",-1,1),new v("imma",0,-1)],b=[17,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,8],d=[17,65,16,1,0,0,0,0,0,0,0,0,0,0,0,0,8,0,32],e=[17,65,16,0,0,0,0,0,0,0,0,0,0,0,0,0,8,0,32],f=[17,97,24,1,0,0,0,0,0,0,0,0,0,0,0,0,8,0,32],h=new C;function p(){for(var i;i=h.cursor,!h.in_grouping(d,97,246);){if((h.cursor=i)>=h.limit)return!0;h.cursor++}for(h.cursor=i;!h.out_grouping(d,97,246);){if(h.cursor>=h.limit)return!0;h.cursor++}return!1}function g(){var i,e;if(h.cursor>=o)if(e=h.limit_backward,h.limit_backward=o,h.ket=h.cursor,i=h.find_among_b(r,10)){switch(h.bra=h.cursor,h.limit_backward=e,i){case 1:if(!h.in_grouping_b(f,97,246))return;break;case 2:if(!(l<=h.cursor))return}h.slice_del()}else h.limit_backward=e}function j(){return h.find_among_b(i,7)}function q(){return h.eq_s_b(1,"i")&&h.in_grouping_b(e,97,246)}this.setCurrent=function(i){h.setCurrent(i)},this.getCurrent=function(){return h.getCurrent()},this.stem=function(){var i,e=h.cursor;return o=h.limit,l=o,p()||(o=h.cursor,p()||(l=h.cursor)),n=!1,h.limit_backward=e,h.cursor=h.limit,g(),h.cursor=h.limit,function(){var i,e,r;if(h.cursor>=o)if(e=h.limit_backward,h.limit_backward=o,h.ket=h.cursor,i=h.find_among_b(c,9))switch(h.bra=h.cursor,h.limit_backward=e,i){case 1:r=h.limit-h.cursor,h.eq_s_b(1,"k")||(h.cursor=h.limit-r,h.slice_del());break;case 2:h.slice_del(),h.ket=h.cursor,h.eq_s_b(3,"kse")&&(h.bra=h.cursor,h.slice_from("ksi"));break;case 3:h.slice_del();break;case 4:h.find_among_b(s,6)&&h.slice_del();break;case 5:h.find_among_b(a,6)&&h.slice_del();break;case 6:h.find_among_b(u,2)&&h.slice_del()}else h.limit_backward=e}(),h.cursor=h.limit,function(){var i,e,r;if(h.cursor>=o)if(e=h.limit_backward,h.limit_backward=o,h.ket=h.cursor,i=h.find_among_b(m,30)){switch(h.bra=h.cursor,h.limit_backward=e,i){case 1:if(!h.eq_s_b(1,"a"))return;break;case 2:case 9:if(!h.eq_s_b(1,"e"))return;break;case 3:if(!h.eq_s_b(1,"i"))return;break;case 4:if(!h.eq_s_b(1,"o"))return;break;case 5:if(!h.eq_s_b(1,"ä"))return;break;case 6:if(!h.eq_s_b(1,"ö"))return;break;case 7:if(r=h.limit-h.cursor,!j()&&(h.cursor=h.limit-r,!h.eq_s_b(2,"ie"))){h.cursor=h.limit-r;break}if(h.cursor=h.limit-r,h.cursor<=h.limit_backward){h.cursor=h.limit-r;break}h.cursor--,h.bra=h.cursor;break;case 8:if(!h.in_grouping_b(d,97,246)||!h.out_grouping_b(d,97,246))return}h.slice_del(),n=!0}else h.limit_backward=e}(),h.cursor=h.limit,function(){var i,e,r;if(h.cursor>=l)if(e=h.limit_backward,h.limit_backward=l,h.ket=h.cursor,i=h.find_among_b(w,14)){if(h.bra=h.cursor,h.limit_backward=e,1==i){if(r=h.limit-h.cursor,h.eq_s_b(2,"po"))return;h.cursor=h.limit-r}h.slice_del()}else h.limit_backward=e}(),h.cursor=h.limit,h.cursor=(n?h.cursor>=o&&(i=h.limit_backward,h.limit_backward=o,h.ket=h.cursor,h.find_among_b(_,2)?(h.bra=h.cursor,h.limit_backward=i,h.slice_del()):h.limit_backward=i):(h.cursor=h.limit,function(){var i,e,r,n,t,s;if(h.cursor>=o){if(e=h.limit_backward,h.limit_backward=o,h.ket=h.cursor,h.eq_s_b(1,"t")&&(h.bra=h.cursor,r=h.limit-h.cursor,h.in_grouping_b(d,97,246)&&(h.cursor=h.limit-r,h.slice_del(),h.limit_backward=e,n=h.limit-h.cursor,h.cursor>=l&&(h.cursor=l,t=h.limit_backward,h.limit_backward=h.cursor,h.cursor=h.limit-n,h.ket=h.cursor,i=h.find_among_b(k,2))))){if(h.bra=h.cursor,h.limit_backward=t,1==i){if(s=h.limit-h.cursor,h.eq_s_b(2,"po"))return;h.cursor=h.limit-s}return h.slice_del()}h.limit_backward=e}}()),h.limit),function(){var i,e,r,n;if(h.cursor>=o){for(i=h.limit_backward,h.limit_backward=o,e=h.limit-h.cursor,j()&&(h.cursor=h.limit-e,h.ket=h.cursor,h.cursor>h.limit_backward&&(h.cursor--,h.bra=h.cursor,h.slice_del())),h.cursor=h.limit-e,h.ket=h.cursor,h.in_grouping_b(b,97,228)&&(h.bra=h.cursor,h.out_grouping_b(d,97,246)&&h.slice_del()),h.cursor=h.limit-e,h.ket=h.cursor,h.eq_s_b(1,"j")&&(h.bra=h.cursor,r=h.limit-h.cursor,h.eq_s_b(1,"o")?h.slice_del():(h.cursor=h.limit-r,h.eq_s_b(1,"u")&&h.slice_del())),h.cursor=h.limit-e,h.ket=h.cursor,h.eq_s_b(1,"o")&&(h.bra=h.cursor,h.eq_s_b(1,"j")&&h.slice_del()),h.cursor=h.limit-e,h.limit_backward=i;;){if(n=h.limit-h.cursor,h.out_grouping_b(d,97,246)){h.cursor=h.limit-n;break}if(h.cursor=h.limit-n,h.cursor<=h.limit_backward)return;h.cursor--}h.ket=h.cursor,h.cursor>h.limit_backward&&(h.cursor--,h.bra=h.cursor,t=h.slice_to(),h.eq_v_b(t)&&h.slice_del())}}(),!0}},function(i){return"function"==typeof i.update?i.update(function(i){return e.setCurrent(i),e.stem(),e.getCurrent()}):(e.setCurrent(i),e.stem(),e.getCurrent())}),i.Pipeline.registerFunction(i.fi.stemmer,"stemmer-fi"),i.fi.stopWordFilter=i.generateStopWordFilter("ei eivät emme en et ette että he heidän heidät heihin heille heillä heiltä heissä heistä heitä hän häneen hänelle hänellä häneltä hänen hänessä hänestä hänet häntä itse ja johon joiden joihin joiksi joilla joille joilta joina joissa joista joita joka joksi jolla jolle jolta jona jonka jos jossa josta jota jotka kanssa keiden keihin keiksi keille keillä keiltä keinä keissä keistä keitä keneen keneksi kenelle kenellä keneltä kenen kenenä kenessä kenestä kenet ketkä ketkä ketä koska kuin kuka kun me meidän meidät meihin meille meillä meiltä meissä meistä meitä mihin miksi mikä mille millä miltä minkä minkä minua minulla minulle minulta minun minussa minusta minut minuun minä minä missä mistä mitkä mitä mukaan mutta ne niiden niihin niiksi niille niillä niiltä niin niin niinä niissä niistä niitä noiden noihin noiksi noilla noille noilta noin noina noissa noista noita nuo nyt näiden näihin näiksi näille näillä näiltä näinä näissä näistä näitä nämä ole olemme olen olet olette oli olimme olin olisi olisimme olisin olisit olisitte olisivat olit olitte olivat olla olleet ollut on ovat poikki se sekä sen siihen siinä siitä siksi sille sillä sillä siltä sinua sinulla sinulle sinulta sinun sinussa sinusta sinut sinuun sinä sinä sitä tai te teidän teidät teihin teille teillä teiltä teissä teistä teitä tuo tuohon tuoksi tuolla tuolle tuolta tuon tuona tuossa tuosta tuota tähän täksi tälle tällä tältä tämä tämän tänä tässä tästä tätä vaan vai vaikka yli".split(" ")),i.Pipeline.registerFunction(i.fi.stopWordFilter,"stopWordFilter-fi")}}); \ No newline at end of file diff --git a/430/_static/javascripts/lunr/lunr.fr.js b/430/_static/javascripts/lunr/lunr.fr.js new file mode 100644 index 000000000..078d0cab7 --- /dev/null +++ b/430/_static/javascripts/lunr/lunr.fr.js @@ -0,0 +1 @@ +!function(e,r){"function"==typeof define&&define.amd?define(r):"object"==typeof exports?module.exports=r():r()(e.lunr)}(this,function(){return function(e){if(void 0===e)throw new Error("Lunr is not present. Please include / require Lunr before this script.");if(void 0===e.stemmerSupport)throw new Error("Lunr stemmer support is not present. Please include / require Lunr stemmer support before this script.");var r,y,s;e.fr=function(){this.pipeline.reset(),this.pipeline.add(e.fr.trimmer,e.fr.stopWordFilter,e.fr.stemmer),this.searchPipeline&&(this.searchPipeline.reset(),this.searchPipeline.add(e.fr.stemmer))},e.fr.wordCharacters="A-Za-zªºÀ-ÖØ-öø-ʸˠ-ˤᴀ-ᴥᴬ-ᵜᵢ-ᵥᵫ-ᵷᵹ-ᶾḀ-ỿⁱⁿₐ-ₜKÅℲⅎⅠ-ↈⱠ-ⱿꜢ-ꞇꞋ-ꞭꞰ-ꞷꟷ-ꟿꬰ-ꭚꭜ-ꭤff-stA-Za-z",e.fr.trimmer=e.trimmerSupport.generateTrimmer(e.fr.wordCharacters),e.Pipeline.registerFunction(e.fr.trimmer,"trimmer-fr"),e.fr.stemmer=(r=e.stemmerSupport.Among,y=e.stemmerSupport.SnowballProgram,s=new function(){var s,i,t,n=[new r("col",-1,-1),new r("par",-1,-1),new r("tap",-1,-1)],u=[new r("",-1,4),new r("I",0,1),new r("U",0,2),new r("Y",0,3)],o=[new r("iqU",-1,3),new r("abl",-1,3),new r("Ièr",-1,4),new r("ièr",-1,4),new r("eus",-1,2),new r("iv",-1,1)],c=[new r("ic",-1,2),new r("abil",-1,1),new r("iv",-1,3)],a=[new r("iqUe",-1,1),new r("atrice",-1,2),new r("ance",-1,1),new r("ence",-1,5),new r("logie",-1,3),new r("able",-1,1),new r("isme",-1,1),new r("euse",-1,11),new r("iste",-1,1),new r("ive",-1,8),new r("if",-1,8),new r("usion",-1,4),new r("ation",-1,2),new r("ution",-1,4),new r("ateur",-1,2),new r("iqUes",-1,1),new r("atrices",-1,2),new r("ances",-1,1),new r("ences",-1,5),new r("logies",-1,3),new r("ables",-1,1),new r("ismes",-1,1),new r("euses",-1,11),new r("istes",-1,1),new r("ives",-1,8),new r("ifs",-1,8),new r("usions",-1,4),new r("ations",-1,2),new r("utions",-1,4),new r("ateurs",-1,2),new r("ments",-1,15),new r("ements",30,6),new r("issements",31,12),new r("ités",-1,7),new r("ment",-1,15),new r("ement",34,6),new r("issement",35,12),new r("amment",34,13),new r("emment",34,14),new r("aux",-1,10),new r("eaux",39,9),new r("eux",-1,1),new r("ité",-1,7)],l=[new r("ira",-1,1),new r("ie",-1,1),new r("isse",-1,1),new r("issante",-1,1),new r("i",-1,1),new r("irai",4,1),new r("ir",-1,1),new r("iras",-1,1),new r("ies",-1,1),new r("îmes",-1,1),new r("isses",-1,1),new r("issantes",-1,1),new r("îtes",-1,1),new r("is",-1,1),new r("irais",13,1),new r("issais",13,1),new r("irions",-1,1),new r("issions",-1,1),new r("irons",-1,1),new r("issons",-1,1),new r("issants",-1,1),new r("it",-1,1),new r("irait",21,1),new r("issait",21,1),new r("issant",-1,1),new r("iraIent",-1,1),new r("issaIent",-1,1),new r("irent",-1,1),new r("issent",-1,1),new r("iront",-1,1),new r("ît",-1,1),new r("iriez",-1,1),new r("issiez",-1,1),new r("irez",-1,1),new r("issez",-1,1)],w=[new r("a",-1,3),new r("era",0,2),new r("asse",-1,3),new r("ante",-1,3),new r("ée",-1,2),new r("ai",-1,3),new r("erai",5,2),new r("er",-1,2),new r("as",-1,3),new r("eras",8,2),new r("âmes",-1,3),new r("asses",-1,3),new r("antes",-1,3),new r("âtes",-1,3),new r("ées",-1,2),new r("ais",-1,3),new r("erais",15,2),new r("ions",-1,1),new r("erions",17,2),new r("assions",17,3),new r("erons",-1,2),new r("ants",-1,3),new r("és",-1,2),new r("ait",-1,3),new r("erait",23,2),new r("ant",-1,3),new r("aIent",-1,3),new r("eraIent",26,2),new r("èrent",-1,2),new r("assent",-1,3),new r("eront",-1,2),new r("ât",-1,3),new r("ez",-1,2),new r("iez",32,2),new r("eriez",33,2),new r("assiez",33,3),new r("erez",32,2),new r("é",-1,2)],f=[new r("e",-1,3),new r("Ière",0,2),new r("ière",0,2),new r("ion",-1,1),new r("Ier",-1,2),new r("ier",-1,2),new r("ë",-1,4)],m=[new r("ell",-1,-1),new r("eill",-1,-1),new r("enn",-1,-1),new r("onn",-1,-1),new r("ett",-1,-1)],_=[17,65,16,1,0,0,0,0,0,0,0,0,0,0,0,128,130,103,8,5],b=[1,65,20,0,0,0,0,0,0,0,0,0,0,0,0,0,128],d=new y;function k(e,r,s){return!(!d.eq_s(1,e)||(d.ket=d.cursor,!d.in_grouping(_,97,251)))&&(d.slice_from(r),d.cursor=s,!0)}function p(e,r,s){return!!d.eq_s(1,e)&&(d.ket=d.cursor,d.slice_from(r),d.cursor=s,!0)}function g(){for(;!d.in_grouping(_,97,251);){if(d.cursor>=d.limit)return!0;d.cursor++}for(;!d.out_grouping(_,97,251);){if(d.cursor>=d.limit)return!0;d.cursor++}return!1}function q(){return t<=d.cursor}function v(){return i<=d.cursor}function h(){return s<=d.cursor}function z(){if(!function(){var e,r;if(d.ket=d.cursor,e=d.find_among_b(a,43)){switch(d.bra=d.cursor,e){case 1:if(!h())return!1;d.slice_del();break;case 2:if(!h())return!1;d.slice_del(),d.ket=d.cursor,d.eq_s_b(2,"ic")&&(d.bra=d.cursor,h()?d.slice_del():d.slice_from("iqU"));break;case 3:if(!h())return!1;d.slice_from("log");break;case 4:if(!h())return!1;d.slice_from("u");break;case 5:if(!h())return!1;d.slice_from("ent");break;case 6:if(!q())return!1;if(d.slice_del(),d.ket=d.cursor,e=d.find_among_b(o,6))switch(d.bra=d.cursor,e){case 1:h()&&(d.slice_del(),d.ket=d.cursor,d.eq_s_b(2,"at")&&(d.bra=d.cursor,h()&&d.slice_del()));break;case 2:h()?d.slice_del():v()&&d.slice_from("eux");break;case 3:h()&&d.slice_del();break;case 4:q()&&d.slice_from("i")}break;case 7:if(!h())return!1;if(d.slice_del(),d.ket=d.cursor,e=d.find_among_b(c,3))switch(d.bra=d.cursor,e){case 1:h()?d.slice_del():d.slice_from("abl");break;case 2:h()?d.slice_del():d.slice_from("iqU");break;case 3:h()&&d.slice_del()}break;case 8:if(!h())return!1;if(d.slice_del(),d.ket=d.cursor,d.eq_s_b(2,"at")&&(d.bra=d.cursor,h()&&(d.slice_del(),d.ket=d.cursor,d.eq_s_b(2,"ic")))){d.bra=d.cursor,h()?d.slice_del():d.slice_from("iqU");break}break;case 9:d.slice_from("eau");break;case 10:if(!v())return!1;d.slice_from("al");break;case 11:if(h())d.slice_del();else{if(!v())return!1;d.slice_from("eux")}break;case 12:if(!v()||!d.out_grouping_b(_,97,251))return!1;d.slice_del();break;case 13:return q()&&d.slice_from("ant"),!1;case 14:return q()&&d.slice_from("ent"),!1;case 15:return r=d.limit-d.cursor,d.in_grouping_b(_,97,251)&&q()&&(d.cursor=d.limit-r,d.slice_del()),!1}return!0}return!1}()&&(d.cursor=d.limit,!function(){var e,r;if(d.cursor=t){if(s=d.limit_backward,d.limit_backward=t,d.ket=d.cursor,e=d.find_among_b(f,7))switch(d.bra=d.cursor,e){case 1:if(h()){if(i=d.limit-d.cursor,!d.eq_s_b(1,"s")&&(d.cursor=d.limit-i,!d.eq_s_b(1,"t")))break;d.slice_del()}break;case 2:d.slice_from("i");break;case 3:d.slice_del();break;case 4:d.eq_s_b(2,"gu")&&d.slice_del()}d.limit_backward=s}}();d.cursor=d.limit,d.ket=d.cursor,d.eq_s_b(1,"Y")?(d.bra=d.cursor,d.slice_from("i")):(d.cursor=d.limit,d.eq_s_b(1,"ç")&&(d.bra=d.cursor,d.slice_from("c")))}this.setCurrent=function(e){d.setCurrent(e)},this.getCurrent=function(){return d.getCurrent()},this.stem=function(){var e,r=d.cursor;return function(){for(var e,r;;){if(e=d.cursor,d.in_grouping(_,97,251)){if(d.bra=d.cursor,r=d.cursor,k("u","U",e))continue;if(d.cursor=r,k("i","I",e))continue;if(d.cursor=r,p("y","Y",e))continue}if(d.cursor=e,!k("y","Y",d.bra=e)){if(d.cursor=e,d.eq_s(1,"q")&&(d.bra=d.cursor,p("u","U",e)))continue;if((d.cursor=e)>=d.limit)return;d.cursor++}}}(),d.cursor=r,function(){var e=d.cursor;if(t=d.limit,s=i=t,d.in_grouping(_,97,251)&&d.in_grouping(_,97,251)&&d.cursor=d.limit){d.cursor=t;break}d.cursor++}while(!d.in_grouping(_,97,251))}t=d.cursor,d.cursor=e,g()||(i=d.cursor,g()||(s=d.cursor))}(),d.limit_backward=r,d.cursor=d.limit,z(),d.cursor=d.limit,e=d.limit-d.cursor,d.find_among_b(m,5)&&(d.cursor=d.limit-e,d.ket=d.cursor,d.cursor>d.limit_backward&&(d.cursor--,d.bra=d.cursor,d.slice_del())),d.cursor=d.limit,function(){for(var e,r=1;d.out_grouping_b(_,97,251);)r--;if(r<=0){if(d.ket=d.cursor,e=d.limit-d.cursor,!d.eq_s_b(1,"é")&&(d.cursor=d.limit-e,!d.eq_s_b(1,"è")))return;d.bra=d.cursor,d.slice_from("e")}}(),d.cursor=d.limit_backward,function(){for(var e,r;r=d.cursor,d.bra=r,e=d.find_among(u,4);)switch(d.ket=d.cursor,e){case 1:d.slice_from("i");break;case 2:d.slice_from("u");break;case 3:d.slice_from("y");break;case 4:if(d.cursor>=d.limit)return;d.cursor++}}(),!0}},function(e){return"function"==typeof e.update?e.update(function(e){return s.setCurrent(e),s.stem(),s.getCurrent()}):(s.setCurrent(e),s.stem(),s.getCurrent())}),e.Pipeline.registerFunction(e.fr.stemmer,"stemmer-fr"),e.fr.stopWordFilter=e.generateStopWordFilter("ai aie aient aies ait as au aura aurai auraient aurais aurait auras aurez auriez aurions aurons auront aux avaient avais avait avec avez aviez avions avons ayant ayez ayons c ce ceci celà ces cet cette d dans de des du elle en es est et eu eue eues eurent eus eusse eussent eusses eussiez eussions eut eux eûmes eût eûtes furent fus fusse fussent fusses fussiez fussions fut fûmes fût fûtes ici il ils j je l la le les leur leurs lui m ma mais me mes moi mon même n ne nos notre nous on ont ou par pas pour qu que quel quelle quelles quels qui s sa sans se sera serai seraient serais serait seras serez seriez serions serons seront ses soi soient sois soit sommes son sont soyez soyons suis sur t ta te tes toi ton tu un une vos votre vous y à étaient étais était étant étiez étions été étée étées étés êtes".split(" ")),e.Pipeline.registerFunction(e.fr.stopWordFilter,"stopWordFilter-fr")}}); \ No newline at end of file diff --git a/430/_static/javascripts/lunr/lunr.hu.js b/430/_static/javascripts/lunr/lunr.hu.js new file mode 100644 index 000000000..56a4b0dc1 --- /dev/null +++ b/430/_static/javascripts/lunr/lunr.hu.js @@ -0,0 +1 @@ +!function(e,n){"function"==typeof define&&define.amd?define(n):"object"==typeof exports?module.exports=n():n()(e.lunr)}(this,function(){return function(e){if(void 0===e)throw new Error("Lunr is not present. Please include / require Lunr before this script.");if(void 0===e.stemmerSupport)throw new Error("Lunr stemmer support is not present. Please include / require Lunr stemmer support before this script.");var p,_,n;e.hu=function(){this.pipeline.reset(),this.pipeline.add(e.hu.trimmer,e.hu.stopWordFilter,e.hu.stemmer),this.searchPipeline&&(this.searchPipeline.reset(),this.searchPipeline.add(e.hu.stemmer))},e.hu.wordCharacters="A-Za-zªºÀ-ÖØ-öø-ʸˠ-ˤᴀ-ᴥᴬ-ᵜᵢ-ᵥᵫ-ᵷᵹ-ᶾḀ-ỿⁱⁿₐ-ₜKÅℲⅎⅠ-ↈⱠ-ⱿꜢ-ꞇꞋ-ꞭꞰ-ꞷꟷ-ꟿꬰ-ꭚꭜ-ꭤff-stA-Za-z",e.hu.trimmer=e.trimmerSupport.generateTrimmer(e.hu.wordCharacters),e.Pipeline.registerFunction(e.hu.trimmer,"trimmer-hu"),e.hu.stemmer=(p=e.stemmerSupport.Among,_=e.stemmerSupport.SnowballProgram,n=new function(){var r,i=[new p("cs",-1,-1),new p("dzs",-1,-1),new p("gy",-1,-1),new p("ly",-1,-1),new p("ny",-1,-1),new p("sz",-1,-1),new p("ty",-1,-1),new p("zs",-1,-1)],n=[new p("á",-1,1),new p("é",-1,2)],a=[new p("bb",-1,-1),new p("cc",-1,-1),new p("dd",-1,-1),new p("ff",-1,-1),new p("gg",-1,-1),new p("jj",-1,-1),new p("kk",-1,-1),new p("ll",-1,-1),new p("mm",-1,-1),new p("nn",-1,-1),new p("pp",-1,-1),new p("rr",-1,-1),new p("ccs",-1,-1),new p("ss",-1,-1),new p("zzs",-1,-1),new p("tt",-1,-1),new p("vv",-1,-1),new p("ggy",-1,-1),new p("lly",-1,-1),new p("nny",-1,-1),new p("tty",-1,-1),new p("ssz",-1,-1),new p("zz",-1,-1)],t=[new p("al",-1,1),new p("el",-1,2)],e=[new p("ba",-1,-1),new p("ra",-1,-1),new p("be",-1,-1),new p("re",-1,-1),new p("ig",-1,-1),new p("nak",-1,-1),new p("nek",-1,-1),new p("val",-1,-1),new p("vel",-1,-1),new p("ul",-1,-1),new p("nál",-1,-1),new p("nél",-1,-1),new p("ból",-1,-1),new p("ról",-1,-1),new p("tól",-1,-1),new p("bõl",-1,-1),new p("rõl",-1,-1),new p("tõl",-1,-1),new p("ül",-1,-1),new p("n",-1,-1),new p("an",19,-1),new p("ban",20,-1),new p("en",19,-1),new p("ben",22,-1),new p("képpen",22,-1),new p("on",19,-1),new p("ön",19,-1),new p("képp",-1,-1),new p("kor",-1,-1),new p("t",-1,-1),new p("at",29,-1),new p("et",29,-1),new p("ként",29,-1),new p("anként",32,-1),new p("enként",32,-1),new p("onként",32,-1),new p("ot",29,-1),new p("ért",29,-1),new p("öt",29,-1),new p("hez",-1,-1),new p("hoz",-1,-1),new p("höz",-1,-1),new p("vá",-1,-1),new p("vé",-1,-1)],s=[new p("án",-1,2),new p("én",-1,1),new p("ánként",-1,3)],c=[new p("stul",-1,2),new p("astul",0,1),new p("ástul",0,3),new p("stül",-1,2),new p("estül",3,1),new p("éstül",3,4)],w=[new p("á",-1,1),new p("é",-1,2)],o=[new p("k",-1,7),new p("ak",0,4),new p("ek",0,6),new p("ok",0,5),new p("ák",0,1),new p("ék",0,2),new p("ök",0,3)],l=[new p("éi",-1,7),new p("áéi",0,6),new p("ééi",0,5),new p("é",-1,9),new p("ké",3,4),new p("aké",4,1),new p("eké",4,1),new p("oké",4,1),new p("áké",4,3),new p("éké",4,2),new p("öké",4,1),new p("éé",3,8)],u=[new p("a",-1,18),new p("ja",0,17),new p("d",-1,16),new p("ad",2,13),new p("ed",2,13),new p("od",2,13),new p("ád",2,14),new p("éd",2,15),new p("öd",2,13),new p("e",-1,18),new p("je",9,17),new p("nk",-1,4),new p("unk",11,1),new p("ánk",11,2),new p("énk",11,3),new p("ünk",11,1),new p("uk",-1,8),new p("juk",16,7),new p("ájuk",17,5),new p("ük",-1,8),new p("jük",19,7),new p("éjük",20,6),new p("m",-1,12),new p("am",22,9),new p("em",22,9),new p("om",22,9),new p("ám",22,10),new p("ém",22,11),new p("o",-1,18),new p("á",-1,19),new p("é",-1,20)],m=[new p("id",-1,10),new p("aid",0,9),new p("jaid",1,6),new p("eid",0,9),new p("jeid",3,6),new p("áid",0,7),new p("éid",0,8),new p("i",-1,15),new p("ai",7,14),new p("jai",8,11),new p("ei",7,14),new p("jei",10,11),new p("ái",7,12),new p("éi",7,13),new p("itek",-1,24),new p("eitek",14,21),new p("jeitek",15,20),new p("éitek",14,23),new p("ik",-1,29),new p("aik",18,26),new p("jaik",19,25),new p("eik",18,26),new p("jeik",21,25),new p("áik",18,27),new p("éik",18,28),new p("ink",-1,20),new p("aink",25,17),new p("jaink",26,16),new p("eink",25,17),new p("jeink",28,16),new p("áink",25,18),new p("éink",25,19),new p("aitok",-1,21),new p("jaitok",32,20),new p("áitok",-1,22),new p("im",-1,5),new p("aim",35,4),new p("jaim",36,1),new p("eim",35,4),new p("jeim",38,1),new p("áim",35,2),new p("éim",35,3)],k=[17,65,16,0,0,0,0,0,0,0,0,0,0,0,0,0,1,17,52,14],f=new _;function b(){return r<=f.cursor}function d(){var e=f.limit-f.cursor;return!!f.find_among_b(a,23)&&(f.cursor=f.limit-e,!0)}function g(){if(f.cursor>f.limit_backward){f.cursor--,f.ket=f.cursor;var e=f.cursor-1;f.limit_backward<=e&&e<=f.limit&&(f.cursor=e,f.bra=e,f.slice_del())}}function h(){f.ket=f.cursor,f.find_among_b(e,44)&&(f.bra=f.cursor,b()&&(f.slice_del(),function(){var e;if(f.ket=f.cursor,(e=f.find_among_b(n,2))&&(f.bra=f.cursor,b()))switch(e){case 1:f.slice_from("a");break;case 2:f.slice_from("e")}}()))}this.setCurrent=function(e){f.setCurrent(e)},this.getCurrent=function(){return f.getCurrent()},this.stem=function(){var e=f.cursor;return function(){var e,n=f.cursor;if(r=f.limit,f.in_grouping(k,97,252))for(;;){if(e=f.cursor,f.out_grouping(k,97,252))return f.cursor=e,f.find_among(i,8)||(f.cursor=e)=f.limit)return r=e;f.cursor++}if(f.cursor=n,f.out_grouping(k,97,252)){for(;!f.in_grouping(k,97,252);){if(f.cursor>=f.limit)return;f.cursor++}r=f.cursor}}(),f.limit_backward=e,f.cursor=f.limit,function(){var e;if(f.ket=f.cursor,(e=f.find_among_b(t,2))&&(f.bra=f.cursor,b())){if((1==e||2==e)&&!d())return;f.slice_del(),g()}}(),f.cursor=f.limit,h(),f.cursor=f.limit,function(){var e;if(f.ket=f.cursor,(e=f.find_among_b(s,3))&&(f.bra=f.cursor,b()))switch(e){case 1:f.slice_from("e");break;case 2:case 3:f.slice_from("a")}}(),f.cursor=f.limit,function(){var e;if(f.ket=f.cursor,(e=f.find_among_b(c,6))&&(f.bra=f.cursor,b()))switch(e){case 1:case 2:f.slice_del();break;case 3:f.slice_from("a");break;case 4:f.slice_from("e")}}(),f.cursor=f.limit,function(){var e;if(f.ket=f.cursor,(e=f.find_among_b(w,2))&&(f.bra=f.cursor,b())){if((1==e||2==e)&&!d())return;f.slice_del(),g()}}(),f.cursor=f.limit,function(){var e;if(f.ket=f.cursor,(e=f.find_among_b(l,12))&&(f.bra=f.cursor,b()))switch(e){case 1:case 4:case 7:case 9:f.slice_del();break;case 2:case 5:case 8:f.slice_from("e");break;case 3:case 6:f.slice_from("a")}}(),f.cursor=f.limit,function(){var e;if(f.ket=f.cursor,(e=f.find_among_b(u,31))&&(f.bra=f.cursor,b()))switch(e){case 1:case 4:case 7:case 8:case 9:case 12:case 13:case 16:case 17:case 18:f.slice_del();break;case 2:case 5:case 10:case 14:case 19:f.slice_from("a");break;case 3:case 6:case 11:case 15:case 20:f.slice_from("e")}}(),f.cursor=f.limit,function(){var e;if(f.ket=f.cursor,(e=f.find_among_b(m,42))&&(f.bra=f.cursor,b()))switch(e){case 1:case 4:case 5:case 6:case 9:case 10:case 11:case 14:case 15:case 16:case 17:case 20:case 21:case 24:case 25:case 26:case 29:f.slice_del();break;case 2:case 7:case 12:case 18:case 22:case 27:f.slice_from("a");break;case 3:case 8:case 13:case 19:case 23:case 28:f.slice_from("e")}}(),f.cursor=f.limit,function(){var e;if(f.ket=f.cursor,(e=f.find_among_b(o,7))&&(f.bra=f.cursor,b()))switch(e){case 1:f.slice_from("a");break;case 2:f.slice_from("e");break;case 3:case 4:case 5:case 6:case 7:f.slice_del()}}(),!0}},function(e){return"function"==typeof e.update?e.update(function(e){return n.setCurrent(e),n.stem(),n.getCurrent()}):(n.setCurrent(e),n.stem(),n.getCurrent())}),e.Pipeline.registerFunction(e.hu.stemmer,"stemmer-hu"),e.hu.stopWordFilter=e.generateStopWordFilter("a abban ahhoz ahogy ahol aki akik akkor alatt amely amelyek amelyekben amelyeket amelyet amelynek ami amikor amit amolyan amíg annak arra arról az azok azon azonban azt aztán azután azzal azért be belül benne bár cikk cikkek cikkeket csak de e ebben eddig egy egyes egyetlen egyik egyre egyéb egész ehhez ekkor el ellen elsõ elég elõ elõször elõtt emilyen ennek erre ez ezek ezen ezt ezzel ezért fel felé hanem hiszen hogy hogyan igen ill ill. illetve ilyen ilyenkor ismét ison itt jobban jó jól kell kellett keressünk keresztül ki kívül között közül legalább legyen lehet lehetett lenne lenni lesz lett maga magát majd majd meg mellett mely melyek mert mi mikor milyen minden mindenki mindent mindig mint mintha mit mivel miért most már más másik még míg nagy nagyobb nagyon ne nekem neki nem nincs néha néhány nélkül olyan ott pedig persze rá s saját sem semmi sok sokat sokkal szemben szerint szinte számára talán tehát teljes tovább továbbá több ugyanis utolsó után utána vagy vagyis vagyok valaki valami valamint való van vannak vele vissza viszont volna volt voltak voltam voltunk által általában át én éppen és így õ õk õket össze úgy új újabb újra".split(" ")),e.Pipeline.registerFunction(e.hu.stopWordFilter,"stopWordFilter-hu")}}); \ No newline at end of file diff --git a/430/_static/javascripts/lunr/lunr.it.js b/430/_static/javascripts/lunr/lunr.it.js new file mode 100644 index 000000000..50dddaa04 --- /dev/null +++ b/430/_static/javascripts/lunr/lunr.it.js @@ -0,0 +1 @@ +!function(e,r){"function"==typeof define&&define.amd?define(r):"object"==typeof exports?module.exports=r():r()(e.lunr)}(this,function(){return function(e){if(void 0===e)throw new Error("Lunr is not present. Please include / require Lunr before this script.");if(void 0===e.stemmerSupport)throw new Error("Lunr stemmer support is not present. Please include / require Lunr stemmer support before this script.");var z,P,r;e.it=function(){this.pipeline.reset(),this.pipeline.add(e.it.trimmer,e.it.stopWordFilter,e.it.stemmer),this.searchPipeline&&(this.searchPipeline.reset(),this.searchPipeline.add(e.it.stemmer))},e.it.wordCharacters="A-Za-zªºÀ-ÖØ-öø-ʸˠ-ˤᴀ-ᴥᴬ-ᵜᵢ-ᵥᵫ-ᵷᵹ-ᶾḀ-ỿⁱⁿₐ-ₜKÅℲⅎⅠ-ↈⱠ-ⱿꜢ-ꞇꞋ-ꞭꞰ-ꞷꟷ-ꟿꬰ-ꭚꭜ-ꭤff-stA-Za-z",e.it.trimmer=e.trimmerSupport.generateTrimmer(e.it.wordCharacters),e.Pipeline.registerFunction(e.it.trimmer,"trimmer-it"),e.it.stemmer=(z=e.stemmerSupport.Among,P=e.stemmerSupport.SnowballProgram,r=new function(){var o,t,s,a=[new z("",-1,7),new z("qu",0,6),new z("á",0,1),new z("é",0,2),new z("í",0,3),new z("ó",0,4),new z("ú",0,5)],u=[new z("",-1,3),new z("I",0,1),new z("U",0,2)],c=[new z("la",-1,-1),new z("cela",0,-1),new z("gliela",0,-1),new z("mela",0,-1),new z("tela",0,-1),new z("vela",0,-1),new z("le",-1,-1),new z("cele",6,-1),new z("gliele",6,-1),new z("mele",6,-1),new z("tele",6,-1),new z("vele",6,-1),new z("ne",-1,-1),new z("cene",12,-1),new z("gliene",12,-1),new z("mene",12,-1),new z("sene",12,-1),new z("tene",12,-1),new z("vene",12,-1),new z("ci",-1,-1),new z("li",-1,-1),new z("celi",20,-1),new z("glieli",20,-1),new z("meli",20,-1),new z("teli",20,-1),new z("veli",20,-1),new z("gli",20,-1),new z("mi",-1,-1),new z("si",-1,-1),new z("ti",-1,-1),new z("vi",-1,-1),new z("lo",-1,-1),new z("celo",31,-1),new z("glielo",31,-1),new z("melo",31,-1),new z("telo",31,-1),new z("velo",31,-1)],w=[new z("ando",-1,1),new z("endo",-1,1),new z("ar",-1,2),new z("er",-1,2),new z("ir",-1,2)],r=[new z("ic",-1,-1),new z("abil",-1,-1),new z("os",-1,-1),new z("iv",-1,1)],n=[new z("ic",-1,1),new z("abil",-1,1),new z("iv",-1,1)],i=[new z("ica",-1,1),new z("logia",-1,3),new z("osa",-1,1),new z("ista",-1,1),new z("iva",-1,9),new z("anza",-1,1),new z("enza",-1,5),new z("ice",-1,1),new z("atrice",7,1),new z("iche",-1,1),new z("logie",-1,3),new z("abile",-1,1),new z("ibile",-1,1),new z("usione",-1,4),new z("azione",-1,2),new z("uzione",-1,4),new z("atore",-1,2),new z("ose",-1,1),new z("ante",-1,1),new z("mente",-1,1),new z("amente",19,7),new z("iste",-1,1),new z("ive",-1,9),new z("anze",-1,1),new z("enze",-1,5),new z("ici",-1,1),new z("atrici",25,1),new z("ichi",-1,1),new z("abili",-1,1),new z("ibili",-1,1),new z("ismi",-1,1),new z("usioni",-1,4),new z("azioni",-1,2),new z("uzioni",-1,4),new z("atori",-1,2),new z("osi",-1,1),new z("anti",-1,1),new z("amenti",-1,6),new z("imenti",-1,6),new z("isti",-1,1),new z("ivi",-1,9),new z("ico",-1,1),new z("ismo",-1,1),new z("oso",-1,1),new z("amento",-1,6),new z("imento",-1,6),new z("ivo",-1,9),new z("ità",-1,8),new z("istà",-1,1),new z("istè",-1,1),new z("istì",-1,1)],l=[new z("isca",-1,1),new z("enda",-1,1),new z("ata",-1,1),new z("ita",-1,1),new z("uta",-1,1),new z("ava",-1,1),new z("eva",-1,1),new z("iva",-1,1),new z("erebbe",-1,1),new z("irebbe",-1,1),new z("isce",-1,1),new z("ende",-1,1),new z("are",-1,1),new z("ere",-1,1),new z("ire",-1,1),new z("asse",-1,1),new z("ate",-1,1),new z("avate",16,1),new z("evate",16,1),new z("ivate",16,1),new z("ete",-1,1),new z("erete",20,1),new z("irete",20,1),new z("ite",-1,1),new z("ereste",-1,1),new z("ireste",-1,1),new z("ute",-1,1),new z("erai",-1,1),new z("irai",-1,1),new z("isci",-1,1),new z("endi",-1,1),new z("erei",-1,1),new z("irei",-1,1),new z("assi",-1,1),new z("ati",-1,1),new z("iti",-1,1),new z("eresti",-1,1),new z("iresti",-1,1),new z("uti",-1,1),new z("avi",-1,1),new z("evi",-1,1),new z("ivi",-1,1),new z("isco",-1,1),new z("ando",-1,1),new z("endo",-1,1),new z("Yamo",-1,1),new z("iamo",-1,1),new z("avamo",-1,1),new z("evamo",-1,1),new z("ivamo",-1,1),new z("eremo",-1,1),new z("iremo",-1,1),new z("assimo",-1,1),new z("ammo",-1,1),new z("emmo",-1,1),new z("eremmo",54,1),new z("iremmo",54,1),new z("immo",-1,1),new z("ano",-1,1),new z("iscano",58,1),new z("avano",58,1),new z("evano",58,1),new z("ivano",58,1),new z("eranno",-1,1),new z("iranno",-1,1),new z("ono",-1,1),new z("iscono",65,1),new z("arono",65,1),new z("erono",65,1),new z("irono",65,1),new z("erebbero",-1,1),new z("irebbero",-1,1),new z("assero",-1,1),new z("essero",-1,1),new z("issero",-1,1),new z("ato",-1,1),new z("ito",-1,1),new z("uto",-1,1),new z("avo",-1,1),new z("evo",-1,1),new z("ivo",-1,1),new z("ar",-1,1),new z("ir",-1,1),new z("erà",-1,1),new z("irà",-1,1),new z("erò",-1,1),new z("irò",-1,1)],m=[17,65,16,0,0,0,0,0,0,0,0,0,0,0,0,128,128,8,2,1],f=[17,65,0,0,0,0,0,0,0,0,0,0,0,0,0,128,128,8,2],v=[17],b=new P;function d(e,r,n){return!(!b.eq_s(1,e)||(b.ket=b.cursor,!b.in_grouping(m,97,249)))&&(b.slice_from(r),b.cursor=n,!0)}function _(e){if(b.cursor=e,!b.in_grouping(m,97,249))return!1;for(;!b.out_grouping(m,97,249);){if(b.cursor>=b.limit)return!1;b.cursor++}return!0}function g(){var e,r=b.cursor;if(!function(){if(b.in_grouping(m,97,249)){var e=b.cursor;if(b.out_grouping(m,97,249)){for(;!b.in_grouping(m,97,249);){if(b.cursor>=b.limit)return _(e);b.cursor++}return!0}return _(e)}return!1}()){if(b.cursor=r,!b.out_grouping(m,97,249))return;if(e=b.cursor,b.out_grouping(m,97,249)){for(;!b.in_grouping(m,97,249);){if(b.cursor>=b.limit)return b.cursor=e,void(b.in_grouping(m,97,249)&&b.cursor=b.limit)return;b.cursor++}s=b.cursor}function p(){for(;!b.in_grouping(m,97,249);){if(b.cursor>=b.limit)return!1;b.cursor++}for(;!b.out_grouping(m,97,249);){if(b.cursor>=b.limit)return!1;b.cursor++}return!0}function k(){return s<=b.cursor}function h(){return o<=b.cursor}function q(){var e;if(b.ket=b.cursor,!(e=b.find_among_b(i,51)))return!1;switch(b.bra=b.cursor,e){case 1:if(!h())return!1;b.slice_del();break;case 2:if(!h())return!1;b.slice_del(),b.ket=b.cursor,b.eq_s_b(2,"ic")&&(b.bra=b.cursor,h()&&b.slice_del());break;case 3:if(!h())return!1;b.slice_from("log");break;case 4:if(!h())return!1;b.slice_from("u");break;case 5:if(!h())return!1;b.slice_from("ente");break;case 6:if(!k())return!1;b.slice_del();break;case 7:if(!(t<=b.cursor))return!1;b.slice_del(),b.ket=b.cursor,(e=b.find_among_b(r,4))&&(b.bra=b.cursor,h()&&(b.slice_del(),1==e&&(b.ket=b.cursor,b.eq_s_b(2,"at")&&(b.bra=b.cursor,h()&&b.slice_del()))));break;case 8:if(!h())return!1;b.slice_del(),b.ket=b.cursor,(e=b.find_among_b(n,3))&&(b.bra=b.cursor,1==e&&h()&&b.slice_del());break;case 9:if(!h())return!1;b.slice_del(),b.ket=b.cursor,b.eq_s_b(2,"at")&&(b.bra=b.cursor,h()&&(b.slice_del(),b.ket=b.cursor,b.eq_s_b(2,"ic")&&(b.bra=b.cursor,h()&&b.slice_del())))}return!0}function C(){var e;e=b.limit-b.cursor,b.ket=b.cursor,b.in_grouping_b(f,97,242)&&(b.bra=b.cursor,k()&&(b.slice_del(),b.ket=b.cursor,b.eq_s_b(1,"i")&&(b.bra=b.cursor,k())))?b.slice_del():b.cursor=b.limit-e,b.ket=b.cursor,b.eq_s_b(1,"h")&&(b.bra=b.cursor,b.in_grouping_b(v,99,103)&&k()&&b.slice_del())}this.setCurrent=function(e){b.setCurrent(e)},this.getCurrent=function(){return b.getCurrent()},this.stem=function(){var e,r,n,i=b.cursor;return function(){for(var e,r,n,i,o=b.cursor;;){if(b.bra=b.cursor,e=b.find_among(a,7))switch(b.ket=b.cursor,e){case 1:b.slice_from("à");continue;case 2:b.slice_from("è");continue;case 3:b.slice_from("ì");continue;case 4:b.slice_from("ò");continue;case 5:b.slice_from("ù");continue;case 6:b.slice_from("qU");continue;case 7:if(b.cursor>=b.limit)break;b.cursor++;continue}break}for(b.cursor=o;;)for(r=b.cursor;;){if(n=b.cursor,b.in_grouping(m,97,249)){if(b.bra=b.cursor,i=b.cursor,d("u","U",n))break;if(b.cursor=i,d("i","I",n))break}if(b.cursor=n,b.cursor>=b.limit)return b.cursor=r;b.cursor++}}(),b.cursor=i,e=b.cursor,s=b.limit,o=t=s,g(),b.cursor=e,p()&&(t=b.cursor,p()&&(o=b.cursor)),b.limit_backward=i,b.cursor=b.limit,function(){var e;if(b.ket=b.cursor,b.find_among_b(c,37)&&(b.bra=b.cursor,(e=b.find_among_b(w,5))&&k()))switch(e){case 1:b.slice_del();break;case 2:b.slice_from("e")}}(),b.cursor=b.limit,q()||(b.cursor=b.limit,b.cursor>=s&&(n=b.limit_backward,b.limit_backward=s,b.ket=b.cursor,(r=b.find_among_b(l,87))&&(b.bra=b.cursor,1==r&&b.slice_del()),b.limit_backward=n)),b.cursor=b.limit,C(),b.cursor=b.limit_backward,function(){for(var e;b.bra=b.cursor,e=b.find_among(u,3);)switch(b.ket=b.cursor,e){case 1:b.slice_from("i");break;case 2:b.slice_from("u");break;case 3:if(b.cursor>=b.limit)return;b.cursor++}}(),!0}},function(e){return"function"==typeof e.update?e.update(function(e){return r.setCurrent(e),r.stem(),r.getCurrent()}):(r.setCurrent(e),r.stem(),r.getCurrent())}),e.Pipeline.registerFunction(e.it.stemmer,"stemmer-it"),e.it.stopWordFilter=e.generateStopWordFilter("a abbia abbiamo abbiano abbiate ad agl agli ai al all alla alle allo anche avemmo avendo avesse avessero avessi avessimo aveste avesti avete aveva avevamo avevano avevate avevi avevo avrai avranno avrebbe avrebbero avrei avremmo avremo avreste avresti avrete avrà avrò avuta avute avuti avuto c che chi ci coi col come con contro cui da dagl dagli dai dal dall dalla dalle dallo degl degli dei del dell della delle dello di dov dove e ebbe ebbero ebbi ed era erano eravamo eravate eri ero essendo faccia facciamo facciano facciate faccio facemmo facendo facesse facessero facessi facessimo faceste facesti faceva facevamo facevano facevate facevi facevo fai fanno farai faranno farebbe farebbero farei faremmo faremo fareste faresti farete farà farò fece fecero feci fosse fossero fossi fossimo foste fosti fu fui fummo furono gli ha hai hanno ho i il in io l la le lei li lo loro lui ma mi mia mie miei mio ne negl negli nei nel nell nella nelle nello noi non nostra nostre nostri nostro o per perché più quale quanta quante quanti quanto quella quelle quelli quello questa queste questi questo sarai saranno sarebbe sarebbero sarei saremmo saremo sareste saresti sarete sarà sarò se sei si sia siamo siano siate siete sono sta stai stando stanno starai staranno starebbe starebbero starei staremmo staremo stareste staresti starete starà starò stava stavamo stavano stavate stavi stavo stemmo stesse stessero stessi stessimo steste stesti stette stettero stetti stia stiamo stiano stiate sto su sua sue sugl sugli sui sul sull sulla sulle sullo suo suoi ti tra tu tua tue tuo tuoi tutti tutto un una uno vi voi vostra vostre vostri vostro è".split(" ")),e.Pipeline.registerFunction(e.it.stopWordFilter,"stopWordFilter-it")}}); \ No newline at end of file diff --git a/430/_static/javascripts/lunr/lunr.ja.js b/430/_static/javascripts/lunr/lunr.ja.js new file mode 100644 index 000000000..69f620250 --- /dev/null +++ b/430/_static/javascripts/lunr/lunr.ja.js @@ -0,0 +1 @@ +!function(e,r){"function"==typeof define&&define.amd?define(r):"object"==typeof exports?module.exports=r():r()(e.lunr)}(this,function(){return function(m){if(void 0===m)throw new Error("Lunr is not present. Please include / require Lunr before this script.");if(void 0===m.stemmerSupport)throw new Error("Lunr stemmer support is not present. Please include / require Lunr stemmer support before this script.");var l="2"==m.version[0];m.ja=function(){this.pipeline.reset(),this.pipeline.add(m.ja.trimmer,m.ja.stopWordFilter,m.ja.stemmer),l?this.tokenizer=m.ja.tokenizer:(m.tokenizer&&(m.tokenizer=m.ja.tokenizer),this.tokenizerFn&&(this.tokenizerFn=m.ja.tokenizer))};var j=new m.TinySegmenter;m.ja.tokenizer=function(e){var r,t,i,n,o,s,p,a,u;if(!arguments.length||null==e||null==e)return[];if(Array.isArray(e))return e.map(function(e){return l?new m.Token(e.toLowerCase()):e.toLowerCase()});for(r=(t=e.toString().toLowerCase().replace(/^\s+/,"")).length-1;0<=r;r--)if(/\S/.test(t.charAt(r))){t=t.substring(0,r+1);break}for(o=[],i=t.length,p=a=0;a<=i;a++)if(s=a-p,t.charAt(a).match(/\s/)||a==i){if(0=_.limit||(_.cursor++,!1)}function w(){for(;!_.in_grouping(m,97,232);){if(_.cursor>=_.limit)return!0;_.cursor++}for(;!_.out_grouping(m,97,232);){if(_.cursor>=_.limit)return!0;_.cursor++}return!1}function b(){return i<=_.cursor}function p(){return e<=_.cursor}function g(){var r=_.limit-_.cursor;_.find_among_b(t,3)&&(_.cursor=_.limit-r,_.ket=_.cursor,_.cursor>_.limit_backward&&(_.cursor--,_.bra=_.cursor,_.slice_del()))}function h(){var r;u=!1,_.ket=_.cursor,_.eq_s_b(1,"e")&&(_.bra=_.cursor,b()&&(r=_.limit-_.cursor,_.out_grouping_b(m,97,232)&&(_.cursor=_.limit-r,_.slice_del(),u=!0,g())))}function k(){var r;b()&&(r=_.limit-_.cursor,_.out_grouping_b(m,97,232)&&(_.cursor=_.limit-r,_.eq_s_b(3,"gem")||(_.cursor=_.limit-r,_.slice_del(),g())))}this.setCurrent=function(r){_.setCurrent(r)},this.getCurrent=function(){return _.getCurrent()},this.stem=function(){var r=_.cursor;return function(){for(var r,e,i,n=_.cursor;;){if(_.bra=_.cursor,r=_.find_among(o,11))switch(_.ket=_.cursor,r){case 1:_.slice_from("a");continue;case 2:_.slice_from("e");continue;case 3:_.slice_from("i");continue;case 4:_.slice_from("o");continue;case 5:_.slice_from("u");continue;case 6:if(_.cursor>=_.limit)break;_.cursor++;continue}break}for(_.cursor=n,_.bra=n,_.eq_s(1,"y")?(_.ket=_.cursor,_.slice_from("Y")):_.cursor=n;;)if(e=_.cursor,_.in_grouping(m,97,232)){if(i=_.cursor,_.bra=i,_.eq_s(1,"i"))_.ket=_.cursor,_.in_grouping(m,97,232)&&(_.slice_from("I"),_.cursor=e);else if(_.cursor=i,_.eq_s(1,"y"))_.ket=_.cursor,_.slice_from("Y"),_.cursor=e;else if(s(e))break}else if(s(e))break}(),_.cursor=r,i=_.limit,e=i,w()||((i=_.cursor)<3&&(i=3),w()||(e=_.cursor)),_.limit_backward=r,_.cursor=_.limit,function(){var r,e,i,n,o,t,s=_.limit-_.cursor;if(_.ket=_.cursor,r=_.find_among_b(c,5))switch(_.bra=_.cursor,r){case 1:b()&&_.slice_from("heid");break;case 2:k();break;case 3:b()&&_.out_grouping_b(f,97,232)&&_.slice_del()}if(_.cursor=_.limit-s,h(),_.cursor=_.limit-s,_.ket=_.cursor,_.eq_s_b(4,"heid")&&(_.bra=_.cursor,p()&&(e=_.limit-_.cursor,_.eq_s_b(1,"c")||(_.cursor=_.limit-e,_.slice_del(),_.ket=_.cursor,_.eq_s_b(2,"en")&&(_.bra=_.cursor,k())))),_.cursor=_.limit-s,_.ket=_.cursor,r=_.find_among_b(a,6))switch(_.bra=_.cursor,r){case 1:if(p()){if(_.slice_del(),i=_.limit-_.cursor,_.ket=_.cursor,_.eq_s_b(2,"ig")&&(_.bra=_.cursor,p()&&(n=_.limit-_.cursor,!_.eq_s_b(1,"e")))){_.cursor=_.limit-n,_.slice_del();break}_.cursor=_.limit-i,g()}break;case 2:p()&&(o=_.limit-_.cursor,_.eq_s_b(1,"e")||(_.cursor=_.limit-o,_.slice_del()));break;case 3:p()&&(_.slice_del(),h());break;case 4:p()&&_.slice_del();break;case 5:p()&&u&&_.slice_del()}_.cursor=_.limit-s,_.out_grouping_b(d,73,232)&&(t=_.limit-_.cursor,_.find_among_b(l,4)&&_.out_grouping_b(m,97,232)&&(_.cursor=_.limit-t,_.ket=_.cursor,_.cursor>_.limit_backward&&(_.cursor--,_.bra=_.cursor,_.slice_del())))}(),_.cursor=_.limit_backward,function(){for(var r;;)if(_.bra=_.cursor,r=_.find_among(n,3))switch(_.ket=_.cursor,r){case 1:_.slice_from("y");break;case 2:_.slice_from("i");break;case 3:if(_.cursor>=_.limit)return;_.cursor++}}(),!0}},function(r){return"function"==typeof r.update?r.update(function(r){return e.setCurrent(r),e.stem(),e.getCurrent()}):(e.setCurrent(r),e.stem(),e.getCurrent())}),r.Pipeline.registerFunction(r.nl.stemmer,"stemmer-nl"),r.nl.stopWordFilter=r.generateStopWordFilter(" aan al alles als altijd andere ben bij daar dan dat de der deze die dit doch doen door dus een eens en er ge geen geweest haar had heb hebben heeft hem het hier hij hoe hun iemand iets ik in is ja je kan kon kunnen maar me meer men met mij mijn moet na naar niet niets nog nu of om omdat onder ons ook op over reeds te tegen toch toen tot u uit uw van veel voor want waren was wat werd wezen wie wil worden wordt zal ze zelf zich zij zijn zo zonder zou".split(" ")),r.Pipeline.registerFunction(r.nl.stopWordFilter,"stopWordFilter-nl")}}); \ No newline at end of file diff --git a/430/_static/javascripts/lunr/lunr.no.js b/430/_static/javascripts/lunr/lunr.no.js new file mode 100644 index 000000000..3d156b9c1 --- /dev/null +++ b/430/_static/javascripts/lunr/lunr.no.js @@ -0,0 +1 @@ +!function(e,r){"function"==typeof define&&define.amd?define(r):"object"==typeof exports?module.exports=r():r()(e.lunr)}(this,function(){return function(e){if(void 0===e)throw new Error("Lunr is not present. Please include / require Lunr before this script.");if(void 0===e.stemmerSupport)throw new Error("Lunr stemmer support is not present. Please include / require Lunr stemmer support before this script.");var r,n,i;e.no=function(){this.pipeline.reset(),this.pipeline.add(e.no.trimmer,e.no.stopWordFilter,e.no.stemmer),this.searchPipeline&&(this.searchPipeline.reset(),this.searchPipeline.add(e.no.stemmer))},e.no.wordCharacters="A-Za-zªºÀ-ÖØ-öø-ʸˠ-ˤᴀ-ᴥᴬ-ᵜᵢ-ᵥᵫ-ᵷᵹ-ᶾḀ-ỿⁱⁿₐ-ₜKÅℲⅎⅠ-ↈⱠ-ⱿꜢ-ꞇꞋ-ꞭꞰ-ꞷꟷ-ꟿꬰ-ꭚꭜ-ꭤff-stA-Za-z",e.no.trimmer=e.trimmerSupport.generateTrimmer(e.no.wordCharacters),e.Pipeline.registerFunction(e.no.trimmer,"trimmer-no"),e.no.stemmer=(r=e.stemmerSupport.Among,n=e.stemmerSupport.SnowballProgram,i=new function(){var o,s,a=[new r("a",-1,1),new r("e",-1,1),new r("ede",1,1),new r("ande",1,1),new r("ende",1,1),new r("ane",1,1),new r("ene",1,1),new r("hetene",6,1),new r("erte",1,3),new r("en",-1,1),new r("heten",9,1),new r("ar",-1,1),new r("er",-1,1),new r("heter",12,1),new r("s",-1,2),new r("as",14,1),new r("es",14,1),new r("edes",16,1),new r("endes",16,1),new r("enes",16,1),new r("hetenes",19,1),new r("ens",14,1),new r("hetens",21,1),new r("ers",14,1),new r("ets",14,1),new r("et",-1,1),new r("het",25,1),new r("ert",-1,3),new r("ast",-1,1)],m=[new r("dt",-1,-1),new r("vt",-1,-1)],l=[new r("leg",-1,1),new r("eleg",0,1),new r("ig",-1,1),new r("eig",2,1),new r("lig",2,1),new r("elig",4,1),new r("els",-1,1),new r("lov",-1,1),new r("elov",7,1),new r("slov",7,1),new r("hetslov",9,1)],u=[17,65,16,1,0,0,0,0,0,0,0,0,0,0,0,0,48,0,128],d=[119,125,149,1],c=new n;this.setCurrent=function(e){c.setCurrent(e)},this.getCurrent=function(){return c.getCurrent()},this.stem=function(){var e,r,n,i,t=c.cursor;return function(){var e,r=c.cursor+3;if(s=c.limit,0<=r||r<=c.limit){for(o=r;;){if(e=c.cursor,c.in_grouping(u,97,248)){c.cursor=e;break}if(e>=c.limit)return;c.cursor=e+1}for(;!c.out_grouping(u,97,248);){if(c.cursor>=c.limit)return;c.cursor++}(s=c.cursor)=s&&(r=c.limit_backward,c.limit_backward=s,c.ket=c.cursor,e=c.find_among_b(a,29),c.limit_backward=r,e))switch(c.bra=c.cursor,e){case 1:c.slice_del();break;case 2:n=c.limit-c.cursor,c.in_grouping_b(d,98,122)?c.slice_del():(c.cursor=c.limit-n,c.eq_s_b(1,"k")&&c.out_grouping_b(u,97,248)&&c.slice_del());break;case 3:c.slice_from("er")}}(),c.cursor=c.limit,r=c.limit-c.cursor,c.cursor>=s&&(e=c.limit_backward,c.limit_backward=s,c.ket=c.cursor,c.find_among_b(m,2)?(c.bra=c.cursor,c.limit_backward=e,c.cursor=c.limit-r,c.cursor>c.limit_backward&&(c.cursor--,c.bra=c.cursor,c.slice_del())):c.limit_backward=e),c.cursor=c.limit,c.cursor>=s&&(i=c.limit_backward,c.limit_backward=s,c.ket=c.cursor,(n=c.find_among_b(l,11))?(c.bra=c.cursor,c.limit_backward=i,1==n&&c.slice_del()):c.limit_backward=i),!0}},function(e){return"function"==typeof e.update?e.update(function(e){return i.setCurrent(e),i.stem(),i.getCurrent()}):(i.setCurrent(e),i.stem(),i.getCurrent())}),e.Pipeline.registerFunction(e.no.stemmer,"stemmer-no"),e.no.stopWordFilter=e.generateStopWordFilter("alle at av bare begge ble blei bli blir blitt både båe da de deg dei deim deira deires dem den denne der dere deres det dette di din disse ditt du dykk dykkar då eg ein eit eitt eller elles en enn er et ett etter for fordi fra før ha hadde han hans har hennar henne hennes her hjå ho hoe honom hoss hossen hun hva hvem hver hvilke hvilken hvis hvor hvordan hvorfor i ikke ikkje ikkje ingen ingi inkje inn inni ja jeg kan kom korleis korso kun kunne kva kvar kvarhelst kven kvi kvifor man mange me med medan meg meget mellom men mi min mine mitt mot mykje ned no noe noen noka noko nokon nokor nokre nå når og også om opp oss over på samme seg selv si si sia sidan siden sin sine sitt sjøl skal skulle slik so som som somme somt så sånn til um upp ut uten var vart varte ved vere verte vi vil ville vore vors vort vår være være vært å".split(" ")),e.Pipeline.registerFunction(e.no.stopWordFilter,"stopWordFilter-no")}}); \ No newline at end of file diff --git a/430/_static/javascripts/lunr/lunr.pt.js b/430/_static/javascripts/lunr/lunr.pt.js new file mode 100644 index 000000000..f50fc9fa6 --- /dev/null +++ b/430/_static/javascripts/lunr/lunr.pt.js @@ -0,0 +1 @@ +!function(e,r){"function"==typeof define&&define.amd?define(r):"object"==typeof exports?module.exports=r():r()(e.lunr)}(this,function(){return function(e){if(void 0===e)throw new Error("Lunr is not present. Please include / require Lunr before this script.");if(void 0===e.stemmerSupport)throw new Error("Lunr stemmer support is not present. Please include / require Lunr stemmer support before this script.");var j,C,r;e.pt=function(){this.pipeline.reset(),this.pipeline.add(e.pt.trimmer,e.pt.stopWordFilter,e.pt.stemmer),this.searchPipeline&&(this.searchPipeline.reset(),this.searchPipeline.add(e.pt.stemmer))},e.pt.wordCharacters="A-Za-zªºÀ-ÖØ-öø-ʸˠ-ˤᴀ-ᴥᴬ-ᵜᵢ-ᵥᵫ-ᵷᵹ-ᶾḀ-ỿⁱⁿₐ-ₜKÅℲⅎⅠ-ↈⱠ-ⱿꜢ-ꞇꞋ-ꞭꞰ-ꞷꟷ-ꟿꬰ-ꭚꭜ-ꭤff-stA-Za-z",e.pt.trimmer=e.trimmerSupport.generateTrimmer(e.pt.wordCharacters),e.Pipeline.registerFunction(e.pt.trimmer,"trimmer-pt"),e.pt.stemmer=(j=e.stemmerSupport.Among,C=e.stemmerSupport.SnowballProgram,r=new function(){var s,n,i,o=[new j("",-1,3),new j("ã",0,1),new j("õ",0,2)],a=[new j("",-1,3),new j("a~",0,1),new j("o~",0,2)],r=[new j("ic",-1,-1),new j("ad",-1,-1),new j("os",-1,-1),new j("iv",-1,1)],t=[new j("ante",-1,1),new j("avel",-1,1),new j("ível",-1,1)],u=[new j("ic",-1,1),new j("abil",-1,1),new j("iv",-1,1)],w=[new j("ica",-1,1),new j("ância",-1,1),new j("ência",-1,4),new j("ira",-1,9),new j("adora",-1,1),new j("osa",-1,1),new j("ista",-1,1),new j("iva",-1,8),new j("eza",-1,1),new j("logía",-1,2),new j("idade",-1,7),new j("ante",-1,1),new j("mente",-1,6),new j("amente",12,5),new j("ável",-1,1),new j("ível",-1,1),new j("ución",-1,3),new j("ico",-1,1),new j("ismo",-1,1),new j("oso",-1,1),new j("amento",-1,1),new j("imento",-1,1),new j("ivo",-1,8),new j("aça~o",-1,1),new j("ador",-1,1),new j("icas",-1,1),new j("ências",-1,4),new j("iras",-1,9),new j("adoras",-1,1),new j("osas",-1,1),new j("istas",-1,1),new j("ivas",-1,8),new j("ezas",-1,1),new j("logías",-1,2),new j("idades",-1,7),new j("uciones",-1,3),new j("adores",-1,1),new j("antes",-1,1),new j("aço~es",-1,1),new j("icos",-1,1),new j("ismos",-1,1),new j("osos",-1,1),new j("amentos",-1,1),new j("imentos",-1,1),new j("ivos",-1,8)],m=[new j("ada",-1,1),new j("ida",-1,1),new j("ia",-1,1),new j("aria",2,1),new j("eria",2,1),new j("iria",2,1),new j("ara",-1,1),new j("era",-1,1),new j("ira",-1,1),new j("ava",-1,1),new j("asse",-1,1),new j("esse",-1,1),new j("isse",-1,1),new j("aste",-1,1),new j("este",-1,1),new j("iste",-1,1),new j("ei",-1,1),new j("arei",16,1),new j("erei",16,1),new j("irei",16,1),new j("am",-1,1),new j("iam",20,1),new j("ariam",21,1),new j("eriam",21,1),new j("iriam",21,1),new j("aram",20,1),new j("eram",20,1),new j("iram",20,1),new j("avam",20,1),new j("em",-1,1),new j("arem",29,1),new j("erem",29,1),new j("irem",29,1),new j("assem",29,1),new j("essem",29,1),new j("issem",29,1),new j("ado",-1,1),new j("ido",-1,1),new j("ando",-1,1),new j("endo",-1,1),new j("indo",-1,1),new j("ara~o",-1,1),new j("era~o",-1,1),new j("ira~o",-1,1),new j("ar",-1,1),new j("er",-1,1),new j("ir",-1,1),new j("as",-1,1),new j("adas",47,1),new j("idas",47,1),new j("ias",47,1),new j("arias",50,1),new j("erias",50,1),new j("irias",50,1),new j("aras",47,1),new j("eras",47,1),new j("iras",47,1),new j("avas",47,1),new j("es",-1,1),new j("ardes",58,1),new j("erdes",58,1),new j("irdes",58,1),new j("ares",58,1),new j("eres",58,1),new j("ires",58,1),new j("asses",58,1),new j("esses",58,1),new j("isses",58,1),new j("astes",58,1),new j("estes",58,1),new j("istes",58,1),new j("is",-1,1),new j("ais",71,1),new j("eis",71,1),new j("areis",73,1),new j("ereis",73,1),new j("ireis",73,1),new j("áreis",73,1),new j("éreis",73,1),new j("íreis",73,1),new j("ásseis",73,1),new j("ésseis",73,1),new j("ísseis",73,1),new j("áveis",73,1),new j("íeis",73,1),new j("aríeis",84,1),new j("eríeis",84,1),new j("iríeis",84,1),new j("ados",-1,1),new j("idos",-1,1),new j("amos",-1,1),new j("áramos",90,1),new j("éramos",90,1),new j("íramos",90,1),new j("ávamos",90,1),new j("íamos",90,1),new j("aríamos",95,1),new j("eríamos",95,1),new j("iríamos",95,1),new j("emos",-1,1),new j("aremos",99,1),new j("eremos",99,1),new j("iremos",99,1),new j("ássemos",99,1),new j("êssemos",99,1),new j("íssemos",99,1),new j("imos",-1,1),new j("armos",-1,1),new j("ermos",-1,1),new j("irmos",-1,1),new j("ámos",-1,1),new j("arás",-1,1),new j("erás",-1,1),new j("irás",-1,1),new j("eu",-1,1),new j("iu",-1,1),new j("ou",-1,1),new j("ará",-1,1),new j("erá",-1,1),new j("irá",-1,1)],c=[new j("a",-1,1),new j("i",-1,1),new j("o",-1,1),new j("os",-1,1),new j("á",-1,1),new j("í",-1,1),new j("ó",-1,1)],l=[new j("e",-1,1),new j("ç",-1,2),new j("é",-1,1),new j("ê",-1,1)],f=[17,65,16,0,0,0,0,0,0,0,0,0,0,0,0,0,3,19,12,2],d=new C;function v(){if(d.out_grouping(f,97,250)){for(;!d.in_grouping(f,97,250);){if(d.cursor>=d.limit)return!0;d.cursor++}return!1}return!0}function p(){var e,r,s=d.cursor;if(d.in_grouping(f,97,250))if(e=d.cursor,v()){if(d.cursor=e,function(){if(d.in_grouping(f,97,250))for(;!d.out_grouping(f,97,250);){if(d.cursor>=d.limit)return!1;d.cursor++}return i=d.cursor,!0}())return}else i=d.cursor;if(d.cursor=s,d.out_grouping(f,97,250)){if(r=d.cursor,v()){if(d.cursor=r,!d.in_grouping(f,97,250)||d.cursor>=d.limit)return;d.cursor++}i=d.cursor}}function _(){for(;!d.in_grouping(f,97,250);){if(d.cursor>=d.limit)return!1;d.cursor++}for(;!d.out_grouping(f,97,250);){if(d.cursor>=d.limit)return!1;d.cursor++}return!0}function h(){return i<=d.cursor}function b(){return s<=d.cursor}function g(){var e;if(d.ket=d.cursor,!(e=d.find_among_b(w,45)))return!1;switch(d.bra=d.cursor,e){case 1:if(!b())return!1;d.slice_del();break;case 2:if(!b())return!1;d.slice_from("log");break;case 3:if(!b())return!1;d.slice_from("u");break;case 4:if(!b())return!1;d.slice_from("ente");break;case 5:if(!(n<=d.cursor))return!1;d.slice_del(),d.ket=d.cursor,(e=d.find_among_b(r,4))&&(d.bra=d.cursor,b()&&(d.slice_del(),1==e&&(d.ket=d.cursor,d.eq_s_b(2,"at")&&(d.bra=d.cursor,b()&&d.slice_del()))));break;case 6:if(!b())return!1;d.slice_del(),d.ket=d.cursor,(e=d.find_among_b(t,3))&&(d.bra=d.cursor,1==e&&b()&&d.slice_del());break;case 7:if(!b())return!1;d.slice_del(),d.ket=d.cursor,(e=d.find_among_b(u,3))&&(d.bra=d.cursor,1==e&&b()&&d.slice_del());break;case 8:if(!b())return!1;d.slice_del(),d.ket=d.cursor,d.eq_s_b(2,"at")&&(d.bra=d.cursor,b()&&d.slice_del());break;case 9:if(!h()||!d.eq_s_b(1,"e"))return!1;d.slice_from("ir")}return!0}function k(e,r){if(d.eq_s_b(1,e)){d.bra=d.cursor;var s=d.limit-d.cursor;if(d.eq_s_b(1,r))return d.cursor=d.limit-s,h()&&d.slice_del(),!1}return!0}function q(){if(!g()&&(d.cursor=d.limit,!function(){var e,r;if(d.cursor>=i){if(r=d.limit_backward,d.limit_backward=i,d.ket=d.cursor,e=d.find_among_b(m,120))return d.bra=d.cursor,1==e&&d.slice_del(),d.limit_backward=r,!0;d.limit_backward=r}return!1}()))return d.cursor=d.limit,d.ket=d.cursor,void((e=d.find_among_b(c,7))&&(d.bra=d.cursor,1==e&&h()&&d.slice_del()));var e;d.cursor=d.limit,d.ket=d.cursor,d.eq_s_b(1,"i")&&(d.bra=d.cursor,d.eq_s_b(1,"c")&&(d.cursor=d.limit,h()&&d.slice_del()))}this.setCurrent=function(e){d.setCurrent(e)},this.getCurrent=function(){return d.getCurrent()},this.stem=function(){var e,r=d.cursor;return function(){for(var e;;){if(d.bra=d.cursor,e=d.find_among(o,3))switch(d.ket=d.cursor,e){case 1:d.slice_from("a~");continue;case 2:d.slice_from("o~");continue;case 3:if(d.cursor>=d.limit)break;d.cursor++;continue}break}}(),d.cursor=r,e=d.cursor,i=d.limit,s=n=i,p(),d.cursor=e,_()&&(n=d.cursor,_()&&(s=d.cursor)),d.limit_backward=r,d.cursor=d.limit,q(),d.cursor=d.limit,function(){var e;if(d.ket=d.cursor,e=d.find_among_b(l,4))switch(d.bra=d.cursor,e){case 1:h()&&(d.slice_del(),d.ket=d.cursor,d.limit,d.cursor,k("u","g")&&k("i","c"));break;case 2:d.slice_from("c")}}(),d.cursor=d.limit_backward,function(){for(var e;;){if(d.bra=d.cursor,e=d.find_among(a,3))switch(d.ket=d.cursor,e){case 1:d.slice_from("ã");continue;case 2:d.slice_from("õ");continue;case 3:if(d.cursor>=d.limit)break;d.cursor++;continue}break}}(),!0}},function(e){return"function"==typeof e.update?e.update(function(e){return r.setCurrent(e),r.stem(),r.getCurrent()}):(r.setCurrent(e),r.stem(),r.getCurrent())}),e.Pipeline.registerFunction(e.pt.stemmer,"stemmer-pt"),e.pt.stopWordFilter=e.generateStopWordFilter("a ao aos aquela aquelas aquele aqueles aquilo as até com como da das de dela delas dele deles depois do dos e ela elas ele eles em entre era eram essa essas esse esses esta estamos estas estava estavam este esteja estejam estejamos estes esteve estive estivemos estiver estivera estiveram estiverem estivermos estivesse estivessem estivéramos estivéssemos estou está estávamos estão eu foi fomos for fora foram forem formos fosse fossem fui fôramos fôssemos haja hajam hajamos havemos hei houve houvemos houver houvera houveram houverei houverem houveremos houveria houveriam houvermos houverá houverão houveríamos houvesse houvessem houvéramos houvéssemos há hão isso isto já lhe lhes mais mas me mesmo meu meus minha minhas muito na nas nem no nos nossa nossas nosso nossos num numa não nós o os ou para pela pelas pelo pelos por qual quando que quem se seja sejam sejamos sem serei seremos seria seriam será serão seríamos seu seus somos sou sua suas são só também te tem temos tenha tenham tenhamos tenho terei teremos teria teriam terá terão teríamos teu teus teve tinha tinham tive tivemos tiver tivera tiveram tiverem tivermos tivesse tivessem tivéramos tivéssemos tu tua tuas tém tínhamos um uma você vocês vos à às éramos".split(" ")),e.Pipeline.registerFunction(e.pt.stopWordFilter,"stopWordFilter-pt")}}); \ No newline at end of file diff --git a/430/_static/javascripts/lunr/lunr.ro.js b/430/_static/javascripts/lunr/lunr.ro.js new file mode 100644 index 000000000..b19627e1b --- /dev/null +++ b/430/_static/javascripts/lunr/lunr.ro.js @@ -0,0 +1 @@ +!function(e,i){"function"==typeof define&&define.amd?define(i):"object"==typeof exports?module.exports=i():i()(e.lunr)}(this,function(){return function(e){if(void 0===e)throw new Error("Lunr is not present. Please include / require Lunr before this script.");if(void 0===e.stemmerSupport)throw new Error("Lunr stemmer support is not present. Please include / require Lunr stemmer support before this script.");var h,z,i;e.ro=function(){this.pipeline.reset(),this.pipeline.add(e.ro.trimmer,e.ro.stopWordFilter,e.ro.stemmer),this.searchPipeline&&(this.searchPipeline.reset(),this.searchPipeline.add(e.ro.stemmer))},e.ro.wordCharacters="A-Za-zªºÀ-ÖØ-öø-ʸˠ-ˤᴀ-ᴥᴬ-ᵜᵢ-ᵥᵫ-ᵷᵹ-ᶾḀ-ỿⁱⁿₐ-ₜKÅℲⅎⅠ-ↈⱠ-ⱿꜢ-ꞇꞋ-ꞭꞰ-ꞷꟷ-ꟿꬰ-ꭚꭜ-ꭤff-stA-Za-z",e.ro.trimmer=e.trimmerSupport.generateTrimmer(e.ro.wordCharacters),e.Pipeline.registerFunction(e.ro.trimmer,"trimmer-ro"),e.ro.stemmer=(h=e.stemmerSupport.Among,z=e.stemmerSupport.SnowballProgram,i=new function(){var r,n,t,a,o=[new h("",-1,3),new h("I",0,1),new h("U",0,2)],s=[new h("ea",-1,3),new h("aţia",-1,7),new h("aua",-1,2),new h("iua",-1,4),new h("aţie",-1,7),new h("ele",-1,3),new h("ile",-1,5),new h("iile",6,4),new h("iei",-1,4),new h("atei",-1,6),new h("ii",-1,4),new h("ului",-1,1),new h("ul",-1,1),new h("elor",-1,3),new h("ilor",-1,4),new h("iilor",14,4)],c=[new h("icala",-1,4),new h("iciva",-1,4),new h("ativa",-1,5),new h("itiva",-1,6),new h("icale",-1,4),new h("aţiune",-1,5),new h("iţiune",-1,6),new h("atoare",-1,5),new h("itoare",-1,6),new h("ătoare",-1,5),new h("icitate",-1,4),new h("abilitate",-1,1),new h("ibilitate",-1,2),new h("ivitate",-1,3),new h("icive",-1,4),new h("ative",-1,5),new h("itive",-1,6),new h("icali",-1,4),new h("atori",-1,5),new h("icatori",18,4),new h("itori",-1,6),new h("ători",-1,5),new h("icitati",-1,4),new h("abilitati",-1,1),new h("ivitati",-1,3),new h("icivi",-1,4),new h("ativi",-1,5),new h("itivi",-1,6),new h("icităi",-1,4),new h("abilităi",-1,1),new h("ivităi",-1,3),new h("icităţi",-1,4),new h("abilităţi",-1,1),new h("ivităţi",-1,3),new h("ical",-1,4),new h("ator",-1,5),new h("icator",35,4),new h("itor",-1,6),new h("ător",-1,5),new h("iciv",-1,4),new h("ativ",-1,5),new h("itiv",-1,6),new h("icală",-1,4),new h("icivă",-1,4),new h("ativă",-1,5),new h("itivă",-1,6)],u=[new h("ica",-1,1),new h("abila",-1,1),new h("ibila",-1,1),new h("oasa",-1,1),new h("ata",-1,1),new h("ita",-1,1),new h("anta",-1,1),new h("ista",-1,3),new h("uta",-1,1),new h("iva",-1,1),new h("ic",-1,1),new h("ice",-1,1),new h("abile",-1,1),new h("ibile",-1,1),new h("isme",-1,3),new h("iune",-1,2),new h("oase",-1,1),new h("ate",-1,1),new h("itate",17,1),new h("ite",-1,1),new h("ante",-1,1),new h("iste",-1,3),new h("ute",-1,1),new h("ive",-1,1),new h("ici",-1,1),new h("abili",-1,1),new h("ibili",-1,1),new h("iuni",-1,2),new h("atori",-1,1),new h("osi",-1,1),new h("ati",-1,1),new h("itati",30,1),new h("iti",-1,1),new h("anti",-1,1),new h("isti",-1,3),new h("uti",-1,1),new h("işti",-1,3),new h("ivi",-1,1),new h("ităi",-1,1),new h("oşi",-1,1),new h("ităţi",-1,1),new h("abil",-1,1),new h("ibil",-1,1),new h("ism",-1,3),new h("ator",-1,1),new h("os",-1,1),new h("at",-1,1),new h("it",-1,1),new h("ant",-1,1),new h("ist",-1,3),new h("ut",-1,1),new h("iv",-1,1),new h("ică",-1,1),new h("abilă",-1,1),new h("ibilă",-1,1),new h("oasă",-1,1),new h("ată",-1,1),new h("ită",-1,1),new h("antă",-1,1),new h("istă",-1,3),new h("ută",-1,1),new h("ivă",-1,1)],w=[new h("ea",-1,1),new h("ia",-1,1),new h("esc",-1,1),new h("ăsc",-1,1),new h("ind",-1,1),new h("ând",-1,1),new h("are",-1,1),new h("ere",-1,1),new h("ire",-1,1),new h("âre",-1,1),new h("se",-1,2),new h("ase",10,1),new h("sese",10,2),new h("ise",10,1),new h("use",10,1),new h("âse",10,1),new h("eşte",-1,1),new h("ăşte",-1,1),new h("eze",-1,1),new h("ai",-1,1),new h("eai",19,1),new h("iai",19,1),new h("sei",-1,2),new h("eşti",-1,1),new h("ăşti",-1,1),new h("ui",-1,1),new h("ezi",-1,1),new h("âi",-1,1),new h("aşi",-1,1),new h("seşi",-1,2),new h("aseşi",29,1),new h("seseşi",29,2),new h("iseşi",29,1),new h("useşi",29,1),new h("âseşi",29,1),new h("işi",-1,1),new h("uşi",-1,1),new h("âşi",-1,1),new h("aţi",-1,2),new h("eaţi",38,1),new h("iaţi",38,1),new h("eţi",-1,2),new h("iţi",-1,2),new h("âţi",-1,2),new h("arăţi",-1,1),new h("serăţi",-1,2),new h("aserăţi",45,1),new h("seserăţi",45,2),new h("iserăţi",45,1),new h("userăţi",45,1),new h("âserăţi",45,1),new h("irăţi",-1,1),new h("urăţi",-1,1),new h("ârăţi",-1,1),new h("am",-1,1),new h("eam",54,1),new h("iam",54,1),new h("em",-1,2),new h("asem",57,1),new h("sesem",57,2),new h("isem",57,1),new h("usem",57,1),new h("âsem",57,1),new h("im",-1,2),new h("âm",-1,2),new h("ăm",-1,2),new h("arăm",65,1),new h("serăm",65,2),new h("aserăm",67,1),new h("seserăm",67,2),new h("iserăm",67,1),new h("userăm",67,1),new h("âserăm",67,1),new h("irăm",65,1),new h("urăm",65,1),new h("ârăm",65,1),new h("au",-1,1),new h("eau",76,1),new h("iau",76,1),new h("indu",-1,1),new h("ându",-1,1),new h("ez",-1,1),new h("ească",-1,1),new h("ară",-1,1),new h("seră",-1,2),new h("aseră",84,1),new h("seseră",84,2),new h("iseră",84,1),new h("useră",84,1),new h("âseră",84,1),new h("iră",-1,1),new h("ură",-1,1),new h("âră",-1,1),new h("ează",-1,1)],i=[new h("a",-1,1),new h("e",-1,1),new h("ie",1,1),new h("i",-1,1),new h("ă",-1,1)],m=[17,65,16,0,0,0,0,0,0,0,0,0,0,0,0,0,2,32,0,0,4],l=new z;function f(e,i){l.eq_s(1,e)&&(l.ket=l.cursor,l.in_grouping(m,97,259)&&l.slice_from(i))}function p(){if(l.out_grouping(m,97,259)){for(;!l.in_grouping(m,97,259);){if(l.cursor>=l.limit)return!0;l.cursor++}return!1}return!0}function d(){var e,i,r=l.cursor;if(l.in_grouping(m,97,259)){if(e=l.cursor,!p())return void(a=l.cursor);if(l.cursor=e,!function(){if(l.in_grouping(m,97,259))for(;!l.out_grouping(m,97,259);){if(l.cursor>=l.limit)return!0;l.cursor++}return!1}())return void(a=l.cursor)}l.cursor=r,l.out_grouping(m,97,259)&&(i=l.cursor,p()&&(l.cursor=i,l.in_grouping(m,97,259)&&l.cursor=l.limit)return!1;l.cursor++}for(;!l.out_grouping(m,97,259);){if(l.cursor>=l.limit)return!1;l.cursor++}return!0}function v(){return t<=l.cursor}function _(){var e,i=l.limit-l.cursor;if(l.ket=l.cursor,(e=l.find_among_b(c,46))&&(l.bra=l.cursor,v())){switch(e){case 1:l.slice_from("abil");break;case 2:l.slice_from("ibil");break;case 3:l.slice_from("iv");break;case 4:l.slice_from("ic");break;case 5:l.slice_from("at");break;case 6:l.slice_from("it")}return r=!0,l.cursor=l.limit-i,!0}return!1}function g(){var e,i;for(r=!1;;)if(i=l.limit-l.cursor,!_()){l.cursor=l.limit-i;break}if(l.ket=l.cursor,(e=l.find_among_b(u,62))&&(l.bra=l.cursor,n<=l.cursor)){switch(e){case 1:l.slice_del();break;case 2:l.eq_s_b(1,"ţ")&&(l.bra=l.cursor,l.slice_from("t"));break;case 3:l.slice_from("ist")}r=!0}}function k(){var e;l.ket=l.cursor,(e=l.find_among_b(i,5))&&(l.bra=l.cursor,a<=l.cursor&&1==e&&l.slice_del())}this.setCurrent=function(e){l.setCurrent(e)},this.getCurrent=function(){return l.getCurrent()},this.stem=function(){var e,i=l.cursor;return function(){for(var e,i;e=l.cursor,l.in_grouping(m,97,259)&&(i=l.cursor,l.bra=i,f("u","U"),l.cursor=i,f("i","I")),l.cursor=e,!(l.cursor>=l.limit);)l.cursor++}(),l.cursor=i,e=l.cursor,a=l.limit,n=t=a,d(),l.cursor=e,b()&&(t=l.cursor,b()&&(n=l.cursor)),l.limit_backward=i,l.cursor=l.limit,function(){var e,i;if(l.ket=l.cursor,(e=l.find_among_b(s,16))&&(l.bra=l.cursor,v()))switch(e){case 1:l.slice_del();break;case 2:l.slice_from("a");break;case 3:l.slice_from("e");break;case 4:l.slice_from("i");break;case 5:i=l.limit-l.cursor,l.eq_s_b(2,"ab")||(l.cursor=l.limit-i,l.slice_from("i"));break;case 6:l.slice_from("at");break;case 7:l.slice_from("aţi")}}(),l.cursor=l.limit,g(),l.cursor=l.limit,r||(l.cursor=l.limit,function(){var e,i,r;if(l.cursor>=a){if(i=l.limit_backward,l.limit_backward=a,l.ket=l.cursor,e=l.find_among_b(w,94))switch(l.bra=l.cursor,e){case 1:if(r=l.limit-l.cursor,!l.out_grouping_b(m,97,259)&&(l.cursor=l.limit-r,!l.eq_s_b(1,"u")))break;case 2:l.slice_del()}l.limit_backward=i}}(),l.cursor=l.limit),k(),l.cursor=l.limit_backward,function(){for(var e;;){if(l.bra=l.cursor,e=l.find_among(o,3))switch(l.ket=l.cursor,e){case 1:l.slice_from("i");continue;case 2:l.slice_from("u");continue;case 3:if(l.cursor>=l.limit)break;l.cursor++;continue}break}}(),!0}},function(e){return"function"==typeof e.update?e.update(function(e){return i.setCurrent(e),i.stem(),i.getCurrent()}):(i.setCurrent(e),i.stem(),i.getCurrent())}),e.Pipeline.registerFunction(e.ro.stemmer,"stemmer-ro"),e.ro.stopWordFilter=e.generateStopWordFilter("acea aceasta această aceea acei aceia acel acela acele acelea acest acesta aceste acestea aceşti aceştia acolo acord acum ai aia aibă aici al ale alea altceva altcineva am ar are asemenea asta astea astăzi asupra au avea avem aveţi azi aş aşadar aţi bine bucur bună ca care caut ce cel ceva chiar cinci cine cineva contra cu cum cumva curând curînd când cât câte câtva câţi cînd cît cîte cîtva cîţi că căci cărei căror cărui către da dacă dar datorită dată dau de deci deja deoarece departe deşi din dinaintea dintr- dintre doi doilea două drept după dă ea ei el ele eram este eu eşti face fata fi fie fiecare fii fim fiu fiţi frumos fără graţie halbă iar ieri la le li lor lui lângă lîngă mai mea mei mele mereu meu mi mie mine mult multă mulţi mulţumesc mâine mîine mă ne nevoie nici nicăieri nimeni nimeri nimic nişte noastre noastră noi noroc nostru nouă noştri nu opt ori oricare orice oricine oricum oricând oricât oricînd oricît oriunde patra patru patrulea pe pentru peste pic poate pot prea prima primul prin puţin puţina puţină până pînă rog sa sale sau se spate spre sub sunt suntem sunteţi sută sînt sîntem sînteţi să săi său ta tale te timp tine toate toată tot totuşi toţi trei treia treilea tu tăi tău un una unde undeva unei uneia unele uneori unii unor unora unu unui unuia unul vi voastre voastră voi vostru vouă voştri vreme vreo vreun vă zece zero zi zice îi îl îmi împotriva în înainte înaintea încotro încât încît între întrucât întrucît îţi ăla ălea ăsta ăstea ăştia şapte şase şi ştiu ţi ţie".split(" ")),e.Pipeline.registerFunction(e.ro.stopWordFilter,"stopWordFilter-ro")}}); \ No newline at end of file diff --git a/430/_static/javascripts/lunr/lunr.ru.js b/430/_static/javascripts/lunr/lunr.ru.js new file mode 100644 index 000000000..ac9924804 --- /dev/null +++ b/430/_static/javascripts/lunr/lunr.ru.js @@ -0,0 +1 @@ +!function(e,n){"function"==typeof define&&define.amd?define(n):"object"==typeof exports?module.exports=n():n()(e.lunr)}(this,function(){return function(e){if(void 0===e)throw new Error("Lunr is not present. Please include / require Lunr before this script.");if(void 0===e.stemmerSupport)throw new Error("Lunr stemmer support is not present. Please include / require Lunr stemmer support before this script.");var h,g,n;e.ru=function(){this.pipeline.reset(),this.pipeline.add(e.ru.trimmer,e.ru.stopWordFilter,e.ru.stemmer),this.searchPipeline&&(this.searchPipeline.reset(),this.searchPipeline.add(e.ru.stemmer))},e.ru.wordCharacters="Ѐ-҄҇-ԯᴫᵸⷠ-ⷿꙀ-ꚟ︮︯",e.ru.trimmer=e.trimmerSupport.generateTrimmer(e.ru.wordCharacters),e.Pipeline.registerFunction(e.ru.trimmer,"trimmer-ru"),e.ru.stemmer=(h=e.stemmerSupport.Among,g=e.stemmerSupport.SnowballProgram,n=new function(){var n,e,r=[new h("в",-1,1),new h("ив",0,2),new h("ыв",0,2),new h("вши",-1,1),new h("ивши",3,2),new h("ывши",3,2),new h("вшись",-1,1),new h("ившись",6,2),new h("ывшись",6,2)],t=[new h("ее",-1,1),new h("ие",-1,1),new h("ое",-1,1),new h("ые",-1,1),new h("ими",-1,1),new h("ыми",-1,1),new h("ей",-1,1),new h("ий",-1,1),new h("ой",-1,1),new h("ый",-1,1),new h("ем",-1,1),new h("им",-1,1),new h("ом",-1,1),new h("ым",-1,1),new h("его",-1,1),new h("ого",-1,1),new h("ему",-1,1),new h("ому",-1,1),new h("их",-1,1),new h("ых",-1,1),new h("ею",-1,1),new h("ою",-1,1),new h("ую",-1,1),new h("юю",-1,1),new h("ая",-1,1),new h("яя",-1,1)],w=[new h("ем",-1,1),new h("нн",-1,1),new h("вш",-1,1),new h("ивш",2,2),new h("ывш",2,2),new h("щ",-1,1),new h("ющ",5,1),new h("ующ",6,2)],i=[new h("сь",-1,1),new h("ся",-1,1)],u=[new h("ла",-1,1),new h("ила",0,2),new h("ыла",0,2),new h("на",-1,1),new h("ена",3,2),new h("ете",-1,1),new h("ите",-1,2),new h("йте",-1,1),new h("ейте",7,2),new h("уйте",7,2),new h("ли",-1,1),new h("или",10,2),new h("ыли",10,2),new h("й",-1,1),new h("ей",13,2),new h("уй",13,2),new h("л",-1,1),new h("ил",16,2),new h("ыл",16,2),new h("ем",-1,1),new h("им",-1,2),new h("ым",-1,2),new h("н",-1,1),new h("ен",22,2),new h("ло",-1,1),new h("ило",24,2),new h("ыло",24,2),new h("но",-1,1),new h("ено",27,2),new h("нно",27,1),new h("ет",-1,1),new h("ует",30,2),new h("ит",-1,2),new h("ыт",-1,2),new h("ют",-1,1),new h("уют",34,2),new h("ят",-1,2),new h("ны",-1,1),new h("ены",37,2),new h("ть",-1,1),new h("ить",39,2),new h("ыть",39,2),new h("ешь",-1,1),new h("ишь",-1,2),new h("ю",-1,2),new h("ую",44,2)],s=[new h("а",-1,1),new h("ев",-1,1),new h("ов",-1,1),new h("е",-1,1),new h("ие",3,1),new h("ье",3,1),new h("и",-1,1),new h("еи",6,1),new h("ии",6,1),new h("ами",6,1),new h("ями",6,1),new h("иями",10,1),new h("й",-1,1),new h("ей",12,1),new h("ией",13,1),new h("ий",12,1),new h("ой",12,1),new h("ам",-1,1),new h("ем",-1,1),new h("ием",18,1),new h("ом",-1,1),new h("ям",-1,1),new h("иям",21,1),new h("о",-1,1),new h("у",-1,1),new h("ах",-1,1),new h("ях",-1,1),new h("иях",26,1),new h("ы",-1,1),new h("ь",-1,1),new h("ю",-1,1),new h("ию",30,1),new h("ью",30,1),new h("я",-1,1),new h("ия",33,1),new h("ья",33,1)],o=[new h("ост",-1,1),new h("ость",-1,1)],c=[new h("ейше",-1,1),new h("н",-1,2),new h("ейш",-1,1),new h("ь",-1,3)],m=[33,65,8,232],l=new g;function f(){for(;!l.in_grouping(m,1072,1103);){if(l.cursor>=l.limit)return!1;l.cursor++}return!0}function a(){for(;!l.out_grouping(m,1072,1103);){if(l.cursor>=l.limit)return!1;l.cursor++}return!0}function p(e,n){var r,t;if(l.ket=l.cursor,r=l.find_among_b(e,n)){switch(l.bra=l.cursor,r){case 1:if(t=l.limit-l.cursor,!l.eq_s_b(1,"а")&&(l.cursor=l.limit-t,!l.eq_s_b(1,"я")))return!1;case 2:l.slice_del()}return!0}return!1}function d(e,n){var r;return l.ket=l.cursor,!!(r=l.find_among_b(e,n))&&(l.bra=l.cursor,1==r&&l.slice_del(),!0)}function _(){return!!d(t,26)&&(p(w,8),!0)}function b(){var e;l.ket=l.cursor,(e=l.find_among_b(o,2))&&(l.bra=l.cursor,n<=l.cursor&&1==e&&l.slice_del())}this.setCurrent=function(e){l.setCurrent(e)},this.getCurrent=function(){return l.getCurrent()},this.stem=function(){return e=l.limit,n=e,f()&&(e=l.cursor,a()&&f()&&a()&&(n=l.cursor)),l.cursor=l.limit,!(l.cursor>3]&1<<(7&s))return this.cursor++,!0}return!1},in_grouping_b:function(r,t,i){if(this.cursor>this.limit_backward){var s=b.charCodeAt(this.cursor-1);if(s<=i&&t<=s&&r[(s-=t)>>3]&1<<(7&s))return this.cursor--,!0}return!1},out_grouping:function(r,t,i){if(this.cursor>3]&1<<(7&s)))return this.cursor++,!0}return!1},out_grouping_b:function(r,t,i){if(this.cursor>this.limit_backward){var s=b.charCodeAt(this.cursor-1);if(i>3]&1<<(7&s)))return this.cursor--,!0}return!1},eq_s:function(r,t){if(this.limit-this.cursor>1),a=0,f=u=(l=r[i]).s_size){if(this.cursor=e+l.s_size,!l.method)return l.result;var m=l.method();if(this.cursor=e+l.s_size,m)return l.result}if((i=l.substring_i)<0)return 0}},find_among_b:function(r,t){for(var i=0,s=t,e=this.cursor,n=this.limit_backward,u=0,o=0,h=!1;;){for(var c=i+(s-i>>1),a=0,f=u=(_=r[i]).s_size){if(this.cursor=e-_.s_size,!_.method)return _.result;var m=_.method();if(this.cursor=e-_.s_size,m)return _.result}if((i=_.substring_i)<0)return 0}},replace_s:function(r,t,i){var s=i.length-(t-r);return b=b.substring(0,r)+i+b.substring(t),this.limit+=s,this.cursor>=t?this.cursor+=s:this.cursor>r&&(this.cursor=r),s},slice_check:function(){if(this.bra<0||this.bra>this.ket||this.ket>this.limit||this.limit>b.length)throw"faulty slice operation"},slice_from:function(r){this.slice_check(),this.replace_s(this.bra,this.ket,r)},slice_del:function(){this.slice_from("")},insert:function(r,t,i){var s=this.replace_s(r,t,i);r<=this.bra&&(this.bra+=s),r<=this.ket&&(this.ket+=s)},slice_to:function(){return this.slice_check(),b.substring(this.bra,this.ket)},eq_v_b:function(r){return this.eq_s_b(r.length,r)}}}},r.trimmerSupport={generateTrimmer:function(r){var t=new RegExp("^[^"+r+"]+"),i=new RegExp("[^"+r+"]+$");return function(r){return"function"==typeof r.update?r.update(function(r){return r.replace(t,"").replace(i,"")}):r.replace(t,"").replace(i,"")}}}}}); \ No newline at end of file diff --git a/430/_static/javascripts/lunr/lunr.sv.js b/430/_static/javascripts/lunr/lunr.sv.js new file mode 100644 index 000000000..6daf5f9d8 --- /dev/null +++ b/430/_static/javascripts/lunr/lunr.sv.js @@ -0,0 +1 @@ +!function(e,r){"function"==typeof define&&define.amd?define(r):"object"==typeof exports?module.exports=r():r()(e.lunr)}(this,function(){return function(e){if(void 0===e)throw new Error("Lunr is not present. Please include / require Lunr before this script.");if(void 0===e.stemmerSupport)throw new Error("Lunr stemmer support is not present. Please include / require Lunr stemmer support before this script.");var r,l,n;e.sv=function(){this.pipeline.reset(),this.pipeline.add(e.sv.trimmer,e.sv.stopWordFilter,e.sv.stemmer),this.searchPipeline&&(this.searchPipeline.reset(),this.searchPipeline.add(e.sv.stemmer))},e.sv.wordCharacters="A-Za-zªºÀ-ÖØ-öø-ʸˠ-ˤᴀ-ᴥᴬ-ᵜᵢ-ᵥᵫ-ᵷᵹ-ᶾḀ-ỿⁱⁿₐ-ₜKÅℲⅎⅠ-ↈⱠ-ⱿꜢ-ꞇꞋ-ꞭꞰ-ꞷꟷ-ꟿꬰ-ꭚꭜ-ꭤff-stA-Za-z",e.sv.trimmer=e.trimmerSupport.generateTrimmer(e.sv.wordCharacters),e.Pipeline.registerFunction(e.sv.trimmer,"trimmer-sv"),e.sv.stemmer=(r=e.stemmerSupport.Among,l=e.stemmerSupport.SnowballProgram,n=new function(){var n,t,i=[new r("a",-1,1),new r("arna",0,1),new r("erna",0,1),new r("heterna",2,1),new r("orna",0,1),new r("ad",-1,1),new r("e",-1,1),new r("ade",6,1),new r("ande",6,1),new r("arne",6,1),new r("are",6,1),new r("aste",6,1),new r("en",-1,1),new r("anden",12,1),new r("aren",12,1),new r("heten",12,1),new r("ern",-1,1),new r("ar",-1,1),new r("er",-1,1),new r("heter",18,1),new r("or",-1,1),new r("s",-1,2),new r("as",21,1),new r("arnas",22,1),new r("ernas",22,1),new r("ornas",22,1),new r("es",21,1),new r("ades",26,1),new r("andes",26,1),new r("ens",21,1),new r("arens",29,1),new r("hetens",29,1),new r("erns",21,1),new r("at",-1,1),new r("andet",-1,1),new r("het",-1,1),new r("ast",-1,1)],s=[new r("dd",-1,-1),new r("gd",-1,-1),new r("nn",-1,-1),new r("dt",-1,-1),new r("gt",-1,-1),new r("kt",-1,-1),new r("tt",-1,-1)],a=[new r("ig",-1,1),new r("lig",0,1),new r("els",-1,1),new r("fullt",-1,3),new r("löst",-1,2)],o=[17,65,16,1,0,0,0,0,0,0,0,0,0,0,0,0,24,0,32],u=[119,127,149],m=new l;this.setCurrent=function(e){m.setCurrent(e)},this.getCurrent=function(){return m.getCurrent()},this.stem=function(){var e,r=m.cursor;return function(){var e,r=m.cursor+3;if(t=m.limit,0<=r||r<=m.limit){for(n=r;;){if(e=m.cursor,m.in_grouping(o,97,246)){m.cursor=e;break}if(m.cursor=e,m.cursor>=m.limit)return;m.cursor++}for(;!m.out_grouping(o,97,246);){if(m.cursor>=m.limit)return;m.cursor++}(t=m.cursor)=t&&(m.limit_backward=t,m.cursor=m.limit,m.ket=m.cursor,e=m.find_among_b(i,37),m.limit_backward=r,e))switch(m.bra=m.cursor,e){case 1:m.slice_del();break;case 2:m.in_grouping_b(u,98,121)&&m.slice_del()}}(),m.cursor=m.limit,e=m.limit_backward,m.cursor>=t&&(m.limit_backward=t,m.cursor=m.limit,m.find_among_b(s,7)&&(m.cursor=m.limit,m.ket=m.cursor,m.cursor>m.limit_backward&&(m.bra=--m.cursor,m.slice_del())),m.limit_backward=e),m.cursor=m.limit,function(){var e,r;if(m.cursor>=t){if(r=m.limit_backward,m.limit_backward=t,m.cursor=m.limit,m.ket=m.cursor,e=m.find_among_b(a,5))switch(m.bra=m.cursor,e){case 1:m.slice_del();break;case 2:m.slice_from("lös");break;case 3:m.slice_from("full")}m.limit_backward=r}}(),!0}},function(e){return"function"==typeof e.update?e.update(function(e){return n.setCurrent(e),n.stem(),n.getCurrent()}):(n.setCurrent(e),n.stem(),n.getCurrent())}),e.Pipeline.registerFunction(e.sv.stemmer,"stemmer-sv"),e.sv.stopWordFilter=e.generateStopWordFilter("alla allt att av blev bli blir blivit de dem den denna deras dess dessa det detta dig din dina ditt du där då efter ej eller en er era ert ett från för ha hade han hans har henne hennes hon honom hur här i icke ingen inom inte jag ju kan kunde man med mellan men mig min mina mitt mot mycket ni nu när någon något några och om oss på samma sedan sig sin sina sitta själv skulle som så sådan sådana sådant till under upp ut utan vad var vara varför varit varje vars vart vem vi vid vilka vilkas vilken vilket vår våra vårt än är åt över".split(" ")),e.Pipeline.registerFunction(e.sv.stopWordFilter,"stopWordFilter-sv")}}); \ No newline at end of file diff --git a/430/_static/javascripts/lunr/lunr.th.js b/430/_static/javascripts/lunr/lunr.th.js new file mode 100644 index 000000000..ee8ef373a --- /dev/null +++ b/430/_static/javascripts/lunr/lunr.th.js @@ -0,0 +1 @@ +!function(e,r){"function"==typeof define&&define.amd?define(r):"object"==typeof exports?module.exports=r():r()(e.lunr)}(this,function(){return function(t){if(void 0===t)throw new Error("Lunr is not present. Please include / require Lunr before this script.");if(void 0===t.stemmerSupport)throw new Error("Lunr stemmer support is not present. Please include / require Lunr stemmer support before this script.");var i="2"==t.version[0];t.th=function(){this.pipeline.reset(),this.pipeline.add(t.th.trimmer),i?this.tokenizer=t.th.tokenizer:(t.tokenizer&&(t.tokenizer=t.th.tokenizer),this.tokenizerFn&&(this.tokenizerFn=t.th.tokenizer))},t.th.wordCharacters="[฀-๿]",t.th.trimmer=t.trimmerSupport.generateTrimmer(t.th.wordCharacters),t.Pipeline.registerFunction(t.th.trimmer,"trimmer-th");var n=t.wordcut;n.init(),t.th.tokenizer=function(e){if(!arguments.length||null==e||null==e)return[];if(Array.isArray(e))return e.map(function(e){return i?new t.Token(e):e});var r=e.toString().replace(/^\s+/,"");return n.cut(r).split("|")}}}); \ No newline at end of file diff --git a/430/_static/javascripts/lunr/lunr.tr.js b/430/_static/javascripts/lunr/lunr.tr.js new file mode 100644 index 000000000..e8fb5a7df --- /dev/null +++ b/430/_static/javascripts/lunr/lunr.tr.js @@ -0,0 +1 @@ +!function(r,i){"function"==typeof define&&define.amd?define(i):"object"==typeof exports?module.exports=i():i()(r.lunr)}(this,function(){return function(r){if(void 0===r)throw new Error("Lunr is not present. Please include / require Lunr before this script.");if(void 0===r.stemmerSupport)throw new Error("Lunr stemmer support is not present. Please include / require Lunr stemmer support before this script.");var mr,dr,i;r.tr=function(){this.pipeline.reset(),this.pipeline.add(r.tr.trimmer,r.tr.stopWordFilter,r.tr.stemmer),this.searchPipeline&&(this.searchPipeline.reset(),this.searchPipeline.add(r.tr.stemmer))},r.tr.wordCharacters="A-Za-zªºÀ-ÖØ-öø-ʸˠ-ˤᴀ-ᴥᴬ-ᵜᵢ-ᵥᵫ-ᵷᵹ-ᶾḀ-ỿⁱⁿₐ-ₜKÅℲⅎⅠ-ↈⱠ-ⱿꜢ-ꞇꞋ-ꞭꞰ-ꞷꟷ-ꟿꬰ-ꭚꭜ-ꭤff-stA-Za-z",r.tr.trimmer=r.trimmerSupport.generateTrimmer(r.tr.wordCharacters),r.Pipeline.registerFunction(r.tr.trimmer,"trimmer-tr"),r.tr.stemmer=(mr=r.stemmerSupport.Among,dr=r.stemmerSupport.SnowballProgram,i=new function(){var t,r=[new mr("m",-1,-1),new mr("n",-1,-1),new mr("miz",-1,-1),new mr("niz",-1,-1),new mr("muz",-1,-1),new mr("nuz",-1,-1),new mr("müz",-1,-1),new mr("nüz",-1,-1),new mr("mız",-1,-1),new mr("nız",-1,-1)],i=[new mr("leri",-1,-1),new mr("ları",-1,-1)],e=[new mr("ni",-1,-1),new mr("nu",-1,-1),new mr("nü",-1,-1),new mr("nı",-1,-1)],n=[new mr("in",-1,-1),new mr("un",-1,-1),new mr("ün",-1,-1),new mr("ın",-1,-1)],u=[new mr("a",-1,-1),new mr("e",-1,-1)],o=[new mr("na",-1,-1),new mr("ne",-1,-1)],s=[new mr("da",-1,-1),new mr("ta",-1,-1),new mr("de",-1,-1),new mr("te",-1,-1)],c=[new mr("nda",-1,-1),new mr("nde",-1,-1)],l=[new mr("dan",-1,-1),new mr("tan",-1,-1),new mr("den",-1,-1),new mr("ten",-1,-1)],a=[new mr("ndan",-1,-1),new mr("nden",-1,-1)],m=[new mr("la",-1,-1),new mr("le",-1,-1)],d=[new mr("ca",-1,-1),new mr("ce",-1,-1)],f=[new mr("im",-1,-1),new mr("um",-1,-1),new mr("üm",-1,-1),new mr("ım",-1,-1)],b=[new mr("sin",-1,-1),new mr("sun",-1,-1),new mr("sün",-1,-1),new mr("sın",-1,-1)],w=[new mr("iz",-1,-1),new mr("uz",-1,-1),new mr("üz",-1,-1),new mr("ız",-1,-1)],_=[new mr("siniz",-1,-1),new mr("sunuz",-1,-1),new mr("sünüz",-1,-1),new mr("sınız",-1,-1)],k=[new mr("lar",-1,-1),new mr("ler",-1,-1)],p=[new mr("niz",-1,-1),new mr("nuz",-1,-1),new mr("nüz",-1,-1),new mr("nız",-1,-1)],g=[new mr("dir",-1,-1),new mr("tir",-1,-1),new mr("dur",-1,-1),new mr("tur",-1,-1),new mr("dür",-1,-1),new mr("tür",-1,-1),new mr("dır",-1,-1),new mr("tır",-1,-1)],y=[new mr("casına",-1,-1),new mr("cesine",-1,-1)],z=[new mr("di",-1,-1),new mr("ti",-1,-1),new mr("dik",-1,-1),new mr("tik",-1,-1),new mr("duk",-1,-1),new mr("tuk",-1,-1),new mr("dük",-1,-1),new mr("tük",-1,-1),new mr("dık",-1,-1),new mr("tık",-1,-1),new mr("dim",-1,-1),new mr("tim",-1,-1),new mr("dum",-1,-1),new mr("tum",-1,-1),new mr("düm",-1,-1),new mr("tüm",-1,-1),new mr("dım",-1,-1),new mr("tım",-1,-1),new mr("din",-1,-1),new mr("tin",-1,-1),new mr("dun",-1,-1),new mr("tun",-1,-1),new mr("dün",-1,-1),new mr("tün",-1,-1),new mr("dın",-1,-1),new mr("tın",-1,-1),new mr("du",-1,-1),new mr("tu",-1,-1),new mr("dü",-1,-1),new mr("tü",-1,-1),new mr("dı",-1,-1),new mr("tı",-1,-1)],h=[new mr("sa",-1,-1),new mr("se",-1,-1),new mr("sak",-1,-1),new mr("sek",-1,-1),new mr("sam",-1,-1),new mr("sem",-1,-1),new mr("san",-1,-1),new mr("sen",-1,-1)],v=[new mr("miş",-1,-1),new mr("muş",-1,-1),new mr("müş",-1,-1),new mr("mış",-1,-1)],q=[new mr("b",-1,1),new mr("c",-1,2),new mr("d",-1,3),new mr("ğ",-1,4)],C=[17,65,16,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,32,8,0,0,0,0,0,0,1],P=[1,16,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,8,0,0,0,0,0,0,1],F=[65],S=[65],W=[["a",[1,64,16,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1],97,305],["e",[17,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,130],101,252],["ı",[1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1],97,305],["i",[17],101,105],["o",F,111,117],["ö",S,246,252],["u",F,111,117]],L=new dr;function x(r,i,e){for(;;){var n=L.limit-L.cursor;if(L.in_grouping_b(r,i,e)){L.cursor=L.limit-n;break}if(L.cursor=L.limit-n,L.cursor<=L.limit_backward)return!1;L.cursor--}return!0}function A(){var r,i;r=L.limit-L.cursor,x(C,97,305);for(var e=0;eL.limit_backward&&(L.cursor--,e=L.limit-L.cursor,i()))?(L.cursor=L.limit-e,!0):(L.cursor=L.limit-n,r()?(L.cursor=L.limit-n,!1):(L.cursor=L.limit-n,!(L.cursor<=L.limit_backward)&&(L.cursor--,!!i()&&(L.cursor=L.limit-n,!0))))}function j(r){return E(r,function(){return L.in_grouping_b(C,97,305)})}function T(){return j(function(){return L.eq_s_b(1,"n")})}function Z(){return j(function(){return L.eq_s_b(1,"y")})}function B(){return L.find_among_b(r,10)&&E(function(){return L.in_grouping_b(P,105,305)},function(){return L.out_grouping_b(C,97,305)})}function D(){return A()&&L.in_grouping_b(P,105,305)&&j(function(){return L.eq_s_b(1,"s")})}function G(){return L.find_among_b(i,2)}function H(){return A()&&L.find_among_b(n,4)&&T()}function I(){return A()&&L.find_among_b(s,4)}function J(){return A()&&L.find_among_b(c,2)}function K(){return A()&&L.find_among_b(f,4)&&Z()}function M(){return A()&&L.find_among_b(b,4)}function N(){return A()&&L.find_among_b(w,4)&&Z()}function O(){return L.find_among_b(_,4)}function Q(){return A()&&L.find_among_b(k,2)}function R(){return A()&&L.find_among_b(g,8)}function U(){return A()&&L.find_among_b(z,32)&&Z()}function V(){return L.find_among_b(h,8)&&Z()}function X(){return A()&&L.find_among_b(v,4)&&Z()}function Y(){var r=L.limit-L.cursor;return!(X()||(L.cursor=L.limit-r,U()||(L.cursor=L.limit-r,V()||(L.cursor=L.limit-r,L.eq_s_b(3,"ken")&&Z()))))}function $(){if(L.find_among_b(y,2)){var r=L.limit-L.cursor;if(O()||(L.cursor=L.limit-r,Q()||(L.cursor=L.limit-r,K()||(L.cursor=L.limit-r,M()||(L.cursor=L.limit-r,N()||(L.cursor=L.limit-r))))),X())return!1}return!0}function rr(){if(!A()||!L.find_among_b(p,4))return!0;var r=L.limit-L.cursor;return!U()&&(L.cursor=L.limit-r,!V())}function ir(){var r,i,e,n=L.limit-L.cursor;if(L.ket=L.cursor,t=!0,Y()&&(L.cursor=L.limit-n,$()&&(L.cursor=L.limit-n,function(){if(Q()){L.bra=L.cursor,L.slice_del();var r=L.limit-L.cursor;return L.ket=L.cursor,R()||(L.cursor=L.limit-r,U()||(L.cursor=L.limit-r,V()||(L.cursor=L.limit-r,X()||(L.cursor=L.limit-r)))),t=!1}return!0}()&&(L.cursor=L.limit-n,rr()&&(L.cursor=L.limit-n,e=L.limit-L.cursor,!(O()||(L.cursor=L.limit-e,N()||(L.cursor=L.limit-e,M()||(L.cursor=L.limit-e,K()))))||(L.bra=L.cursor,L.slice_del(),i=L.limit-L.cursor,L.ket=L.cursor,X()||(L.cursor=L.limit-i),0)))))){if(L.cursor=L.limit-n,!R())return;L.bra=L.cursor,L.slice_del(),L.ket=L.cursor,r=L.limit-L.cursor,O()||(L.cursor=L.limit-r,Q()||(L.cursor=L.limit-r,K()||(L.cursor=L.limit-r,M()||(L.cursor=L.limit-r,N()||(L.cursor=L.limit-r))))),X()||(L.cursor=L.limit-r)}L.bra=L.cursor,L.slice_del()}function er(){var r,i,e,n;if(L.ket=L.cursor,L.eq_s_b(2,"ki")){if(r=L.limit-L.cursor,I())return L.bra=L.cursor,L.slice_del(),i=L.limit-L.cursor,L.ket=L.cursor,Q()?(L.bra=L.cursor,L.slice_del(),er()):(L.cursor=L.limit-i,B()&&(L.bra=L.cursor,L.slice_del(),L.ket=L.cursor,Q()&&(L.bra=L.cursor,L.slice_del(),er()))),!0;if(L.cursor=L.limit-r,H()){if(L.bra=L.cursor,L.slice_del(),L.ket=L.cursor,e=L.limit-L.cursor,G())L.bra=L.cursor,L.slice_del();else{if(L.cursor=L.limit-e,L.ket=L.cursor,!B()&&(L.cursor=L.limit-e,!D()&&(L.cursor=L.limit-e,!er())))return!0;L.bra=L.cursor,L.slice_del(),L.ket=L.cursor,Q()&&(L.bra=L.cursor,L.slice_del(),er())}return!0}if(L.cursor=L.limit-r,J()){if(n=L.limit-L.cursor,G())L.bra=L.cursor,L.slice_del();else if(L.cursor=L.limit-n,D())L.bra=L.cursor,L.slice_del(),L.ket=L.cursor,Q()&&(L.bra=L.cursor,L.slice_del(),er());else if(L.cursor=L.limit-n,!er())return!1;return!0}}return!1}function nr(r){if(L.ket=L.cursor,!J()&&(L.cursor=L.limit-r,!A()||!L.find_among_b(o,2)))return!1;var i=L.limit-L.cursor;if(G())L.bra=L.cursor,L.slice_del();else if(L.cursor=L.limit-i,D())L.bra=L.cursor,L.slice_del(),L.ket=L.cursor,Q()&&(L.bra=L.cursor,L.slice_del(),er());else if(L.cursor=L.limit-i,!er())return!1;return!0}function tr(r){if(L.ket=L.cursor,!(A()&&L.find_among_b(a,2)||(L.cursor=L.limit-r,A()&&L.find_among_b(e,4))))return!1;var i=L.limit-L.cursor;return!(!D()&&(L.cursor=L.limit-i,!G()))&&(L.bra=L.cursor,L.slice_del(),L.ket=L.cursor,Q()&&(L.bra=L.cursor,L.slice_del(),er()),!0)}function ur(){var r,i=L.limit-L.cursor;return L.ket=L.cursor,!!(H()||(L.cursor=L.limit-i,A()&&L.find_among_b(m,2)&&Z()))&&(L.bra=L.cursor,L.slice_del(),r=L.limit-L.cursor,L.ket=L.cursor,!(!Q()||(L.bra=L.cursor,L.slice_del(),!er()))||(L.cursor=L.limit-r,L.ket=L.cursor,(B()||(L.cursor=L.limit-r,D()||(L.cursor=L.limit-r,er())))&&(L.bra=L.cursor,L.slice_del(),L.ket=L.cursor,Q()&&(L.bra=L.cursor,L.slice_del(),er())),!0))}function or(){var r,i,e=L.limit-L.cursor;if(L.ket=L.cursor,!(I()||(L.cursor=L.limit-e,A()&&L.in_grouping_b(P,105,305)&&Z()||(L.cursor=L.limit-e,A()&&L.find_among_b(u,2)&&Z()))))return!1;if(L.bra=L.cursor,L.slice_del(),L.ket=L.cursor,r=L.limit-L.cursor,B())L.bra=L.cursor,L.slice_del(),i=L.limit-L.cursor,L.ket=L.cursor,Q()||(L.cursor=L.limit-i);else if(L.cursor=L.limit-r,!Q())return!0;return L.bra=L.cursor,L.slice_del(),L.ket=L.cursor,er(),!0}function sr(){var r,i,e=L.limit-L.cursor;if(L.ket=L.cursor,Q())return L.bra=L.cursor,L.slice_del(),void er();if(L.cursor=L.limit-e,L.ket=L.cursor,A()&&L.find_among_b(d,2)&&T())if(L.bra=L.cursor,L.slice_del(),r=L.limit-L.cursor,L.ket=L.cursor,G())L.bra=L.cursor,L.slice_del();else{if(L.cursor=L.limit-r,L.ket=L.cursor,!B()&&(L.cursor=L.limit-r,!D())){if(L.cursor=L.limit-r,L.ket=L.cursor,!Q())return;if(L.bra=L.cursor,L.slice_del(),!er())return}L.bra=L.cursor,L.slice_del(),L.ket=L.cursor,Q()&&(L.bra=L.cursor,L.slice_del(),er())}else if(L.cursor=L.limit-e,!nr(e)&&(L.cursor=L.limit-e,!tr(e))){if(L.cursor=L.limit-e,L.ket=L.cursor,A()&&L.find_among_b(l,4))return L.bra=L.cursor,L.slice_del(),L.ket=L.cursor,i=L.limit-L.cursor,void(B()?(L.bra=L.cursor,L.slice_del(),L.ket=L.cursor,Q()&&(L.bra=L.cursor,L.slice_del(),er())):(L.cursor=L.limit-i,Q()?(L.bra=L.cursor,L.slice_del()):L.cursor=L.limit-i,er()));if(L.cursor=L.limit-e,!ur()){if(L.cursor=L.limit-e,G())return L.bra=L.cursor,void L.slice_del();L.cursor=L.limit-e,er()||(L.cursor=L.limit-e,or()||(L.cursor=L.limit-e,L.ket=L.cursor,(B()||(L.cursor=L.limit-e,D()))&&(L.bra=L.cursor,L.slice_del(),L.ket=L.cursor,Q()&&(L.bra=L.cursor,L.slice_del(),er()))))}}}function cr(r,i,e){if(L.cursor=L.limit-r,function(){for(;;){var r=L.limit-L.cursor;if(L.in_grouping_b(C,97,305)){L.cursor=L.limit-r;break}if(L.cursor=L.limit-r,L.cursor<=L.limit_backward)return!1;L.cursor--}return!0}()){var n=L.limit-L.cursor;if(!L.eq_s_b(1,i)&&(L.cursor=L.limit-n,!L.eq_s_b(1,e)))return!0;L.cursor=L.limit-r;var t=L.cursor;return L.insert(L.cursor,L.cursor,e),L.cursor=t,!1}return!0}function lr(r,i,e){for(;!L.eq_s(i,e);){if(L.cursor>=L.limit)return!0;L.cursor++}return i!=L.limit||(L.cursor=r,!1)}function ar(){var r,i,e=L.cursor;return!(!lr(r=L.cursor,2,"ad")||!lr(L.cursor=r,5,"soyad"))&&(L.limit_backward=e,L.cursor=L.limit,i=L.limit-L.cursor,(L.eq_s_b(1,"d")||(L.cursor=L.limit-i,L.eq_s_b(1,"g")))&&cr(i,"a","ı")&&cr(i,"e","i")&&cr(i,"o","u")&&cr(i,"ö","ü"),L.cursor=L.limit,function(){var r;if(L.ket=L.cursor,r=L.find_among_b(q,4))switch(L.bra=L.cursor,r){case 1:L.slice_from("p");break;case 2:L.slice_from("ç");break;case 3:L.slice_from("t");break;case 4:L.slice_from("k")}}(),!0)}this.setCurrent=function(r){L.setCurrent(r)},this.getCurrent=function(){return L.getCurrent()},this.stem=function(){return!!(function(){for(var r,i=L.cursor,e=2;;){for(r=L.cursor;!L.in_grouping(C,97,305);){if(L.cursor>=L.limit)return L.cursor=r,!(0e&&(this._events[n].warned=!0,console.error("(node) warning: possible EventEmitter memory leak detected. %d listeners added. Use emitter.setMaxListeners() to increase limit.",this._events[n].length),"function"==typeof console.trace&&console.trace()));return this},r.prototype.once=function(n,t){if(!a(t))throw TypeError("listener must be a function");var e=!1;function r(){this.removeListener(n,r),e||(e=!0,t.apply(this,arguments))}return r.listener=t,this.on(n,r),this},r.prototype.removeListener=function(n,t){var e,r,i,o;if(!a(t))throw TypeError("listener must be a function");if(!this._events||!this._events[n])return this;if(i=(e=this._events[n]).length,r=-1,e===t||a(e.listener)&&e.listener===t)delete this._events[n],this._events.removeListener&&this.emit("removeListener",n,t);else if(c(e)){for(o=i;0this.maxLength)return i();if(!this.stat&&p(this.cache,o)){var t=this.cache[o];if(Array.isArray(t)&&(t="DIR"),!n||"DIR"===t)return i(null,t);if(n&&"FILE"===t)return i()}var e=this.statCache[o];if(void 0!==e){if(!1===e)return i(null,e);var s=e.isDirectory()?"DIR":"FILE";return n&&"FILE"===s?i():i(null,s,e)}var a=this,c=d("stat\0"+o,function(n,e){{if(e&&e.isSymbolicLink())return u.stat(o,function(n,t){n?a._stat2(r,o,null,e,i):a._stat2(r,o,n,t,i)});a._stat2(r,o,n,e,i)}});c&&u.lstat(o,c)},b.prototype._stat2=function(n,t,e,r,i){if(e)return this.statCache[t]=!1,i();var o="/"===n.slice(-1);if(this.statCache[t]=r,"/"===t.slice(-1)&&!r.isDirectory())return i(null,!1,r);var s=r.isDirectory()?"DIR":"FILE";return this.cache[t]=this.cache[t]||s,o&&"DIR"!==s?i():i(null,s,r)}}).call(this,_("_process"))},{"./common.js":15,"./sync.js":17,_process:24,assert:9,events:14,fs:12,inflight:18,inherits:19,minimatch:20,once:21,path:22,"path-is-absolute":23,util:28}],17:[function(e,r,n){(function(i){(r.exports=n).GlobSync=h;var s=e("fs"),c=e("minimatch"),g=(c.Minimatch,e("./glob.js").Glob,e("util"),e("path")),u=e("assert"),l=e("path-is-absolute"),t=e("./common.js"),o=(t.alphasort,t.alphasorti,t.setopts),a=t.ownProp,f=t.childrenIgnored;function n(n,t){if("function"==typeof t||3===arguments.length)throw new TypeError("callback provided to sync glob\nSee: https://github.com/isaacs/node-glob/issues/167");return new h(n,t).found}function h(n,t){if(!n)throw new Error("must provide pattern");if("function"==typeof t||3===arguments.length)throw new TypeError("callback provided to sync glob\nSee: https://github.com/isaacs/node-glob/issues/167");if(!(this instanceof h))return new h(n,t);if(o(this,n,t),this.noprocess)return this;var e=this.minimatch.set.length;this.matches=new Array(e);for(var r=0;rthis.maxLength)return!1;if(!this.stat&&a(this.cache,t)){var r=this.cache[t];if(Array.isArray(r)&&(r="DIR"),!e||"DIR"===r)return r;if(e&&"FILE"===r)return!1}var i=this.statCache[t];if(!i){var o;try{o=s.lstatSync(t)}catch(n){return!1}if(o.isSymbolicLink())try{i=s.statSync(t)}catch(n){i=o}else i=o}r=(this.statCache[t]=i).isDirectory()?"DIR":"FILE";return this.cache[t]=this.cache[t]||r,(!e||"DIR"===r)&&r},h.prototype._mark=function(n){return t.mark(this,n)},h.prototype._makeAbs=function(n){return t.makeAbs(this,n)}}).call(this,e("_process"))},{"./common.js":15,"./glob.js":16,_process:24,assert:9,fs:12,minimatch:20,path:22,"path-is-absolute":23,util:28}],18:[function(t,r,n){(function(s){var n=t("wrappy"),a=Object.create(null),e=t("once");r.exports=n(function(n,t){return a[n]?(a[n].push(t),null):(a[n]=[t],o=n,e(function n(){var t=a[o],e=t.length,r=function(n){for(var t=n.length,e=[],r=0;re?(t.splice(0,e),s.nextTick(function(){n.apply(null,r)})):delete a[o]}}));var o})}).call(this,t("_process"))},{_process:24,once:21,wrappy:29}],19:[function(n,t,e){"function"==typeof Object.create?t.exports=function(n,t){n.super_=t,n.prototype=Object.create(t.prototype,{constructor:{value:n,enumerable:!1,writable:!0,configurable:!0}})}:t.exports=function(n,t){n.super_=t;var e=function(){};e.prototype=t.prototype,n.prototype=new e,n.prototype.constructor=n}},{}],20:[function(n,t,e){(t.exports=s).Minimatch=i;var u={sep:"/"};try{u=n("path")}catch(n){}var M=s.GLOBSTAR=i.GLOBSTAR={},r=n("brace-expansion"),C={"!":{open:"(?:(?!(?:",close:"))[^/]*?)"},"?":{open:"(?:",close:")?"},"+":{open:"(?:",close:")+"},"*":{open:"(?:",close:")*"},"@":{open:"(?:",close:")"}},P="[^/]",z=P+"*?",B="().*{}+?[]^$\\!".split("").reduce(function(n,t){return n[t]=!0,n},{});var l=/\/+/;function o(t,e){t=t||{},e=e||{};var r={};return Object.keys(e).forEach(function(n){r[n]=e[n]}),Object.keys(t).forEach(function(n){r[n]=t[n]}),r}function s(n,t,e){if("string"!=typeof t)throw new TypeError("glob pattern string required");return e||(e={}),!(!e.nocomment&&"#"===t.charAt(0))&&(""===t.trim()?""===n:new i(t,e).match(n))}function i(n,t){if(!(this instanceof i))return new i(n,t);if("string"!=typeof n)throw new TypeError("glob pattern string required");t||(t={}),n=n.trim(),"/"!==u.sep&&(n=n.split(u.sep).join("/")),this.options=t,this.set=[],this.pattern=n,this.regexp=null,this.negate=!1,this.comment=!1,this.empty=!1,this.make()}function a(n,t){if(t||(t=this instanceof i?this.options:{}),void 0===(n=void 0===n?this.pattern:n))throw new TypeError("undefined pattern");return t.nobrace||!n.match(/\{.*\}/)?[n]:r(n)}s.filter=function(r,i){return i=i||{},function(n,t,e){return s(n,r,i)}},s.defaults=function(r){if(!r||!Object.keys(r).length)return s;var i=s,n=function(n,t,e){return i.minimatch(n,t,o(r,e))};return n.Minimatch=function(n,t){return new i.Minimatch(n,o(r,t))},n},i.defaults=function(n){return n&&Object.keys(n).length?s.defaults(n).Minimatch:i},i.prototype.debug=function(){},i.prototype.make=function(){if(this._made)return;var n=this.pattern,t=this.options;if(!t.nocomment&&"#"===n.charAt(0))return void(this.comment=!0);if(!n)return void(this.empty=!0);this.parseNegate();var e=this.globSet=this.braceExpand();t.debug&&(this.debug=console.error);this.debug(this.pattern,e),e=this.globParts=e.map(function(n){return n.split(l)}),this.debug(this.pattern,e),e=e.map(function(n,t,e){return n.map(this.parse,this)},this),this.debug(this.pattern,e),e=e.filter(function(n){return-1===n.indexOf(!1)}),this.debug(this.pattern,e),this.set=e},i.prototype.parseNegate=function(){var n=this.pattern,t=!1,e=this.options,r=0;if(e.nonegate)return;for(var i=0,o=n.length;i>> no match, partial?",n,f,t,h),f!==s))}if("string"==typeof u?(c=r.nocase?l.toLowerCase()===u.toLowerCase():l===u,this.debug("string match",u,l,c)):(c=l.match(u),this.debug("pattern match",u,l,c)),!c)return!1}if(i===s&&o===a)return!0;if(i===s)return e;if(o===a)return i===s-1&&""===n[i];throw new Error("wtf?")}},{"brace-expansion":11,path:22}],21:[function(n,t,e){var r=n("wrappy");function i(n){var t=function(){return t.called?t.value:(t.called=!0,t.value=n.apply(this,arguments))};return t.called=!1,t}function o(n){var t=function(){if(t.called)throw new Error(t.onceError);return t.called=!0,t.value=n.apply(this,arguments)},e=n.name||"Function wrapped with `once`";return t.onceError=e+" shouldn't be called more than once",t.called=!1,t}t.exports=r(i),t.exports.strict=r(o),i.proto=i(function(){Object.defineProperty(Function.prototype,"once",{value:function(){return i(this)},configurable:!0}),Object.defineProperty(Function.prototype,"onceStrict",{value:function(){return o(this)},configurable:!0})})},{wrappy:29}],22:[function(n,t,u){(function(i){function o(n,t){for(var e=0,r=n.length-1;0<=r;r--){var i=n[r];"."===i?n.splice(r,1):".."===i?(n.splice(r,1),e++):e&&(n.splice(r,1),e--)}if(t)for(;e--;e)n.unshift("..");return n}var t=/^(\/?|)([\s\S]*?)((?:\.{1,2}|[^\/]+?|)(\.[^.\/]*|))(?:[\/]*)$/,s=function(n){return t.exec(n).slice(1)};function a(n,t){if(n.filter)return n.filter(t);for(var e=[],r=0;r":">",'"':""","'":"'","`":"`"},D=d.invert(N),F=function(t){var e=function(n){return t[n]},n="(?:"+d.keys(t).join("|")+")",r=RegExp(n),i=RegExp(n,"g");return function(n){return n=null==n?"":""+n,r.test(n)?n.replace(i,e):n}};d.escape=F(N),d.unescape=F(D),d.result=function(n,t,e){var r=null==n?void 0:n[t];return void 0===r&&(r=e),d.isFunction(r)?r.call(n):r};var M=0;d.uniqueId=function(n){var t=++M+"";return n?n+t:t},d.templateSettings={evaluate:/<%([\s\S]+?)%>/g,interpolate:/<%=([\s\S]+?)%>/g,escape:/<%-([\s\S]+?)%>/g};var C=/(.)^/,P={"'":"'","\\":"\\","\r":"r","\n":"n","\u2028":"u2028","\u2029":"u2029"},z=/\\|'|\r|\n|\u2028|\u2029/g,B=function(n){return"\\"+P[n]};d.template=function(o,n,t){!n&&t&&(n=t),n=d.defaults({},n,d.templateSettings);var e=RegExp([(n.escape||C).source,(n.interpolate||C).source,(n.evaluate||C).source].join("|")+"|$","g"),s=0,a="__p+='";o.replace(e,function(n,t,e,r,i){return a+=o.slice(s,i).replace(z,B),s=i+n.length,t?a+="'+\n((__t=("+t+"))==null?'':_.escape(__t))+\n'":e?a+="'+\n((__t=("+e+"))==null?'':__t)+\n'":r&&(a+="';\n"+r+"\n__p+='"),n}),a+="';\n",n.variable||(a="with(obj||{}){\n"+a+"}\n"),a="var __t,__p='',__j=Array.prototype.join,print=function(){__p+=__j.call(arguments,'');};\n"+a+"return __p;\n";try{var r=new Function(n.variable||"obj","_",a)}catch(n){throw n.source=a,n}var i=function(n){return r.call(this,n,d)},c=n.variable||"obj";return i.source="function("+c+"){\n"+a+"}",i},d.chain=function(n){var t=d(n);return t._chain=!0,t};var U=function(n,t){return n._chain?d(t).chain():t};d.mixin=function(e){d.each(d.functions(e),function(n){var t=d[n]=e[n];d.prototype[n]=function(){var n=[this._wrapped];return i.apply(n,arguments),U(this,t.apply(d,n))}})},d.mixin(d),d.each(["pop","push","reverse","shift","sort","splice","unshift"],function(t){var e=r[t];d.prototype[t]=function(){var n=this._wrapped;return e.apply(n,arguments),"shift"!==t&&"splice"!==t||0!==n.length||delete n[0],U(this,n)}}),d.each(["concat","join","slice"],function(n){var t=r[n];d.prototype[n]=function(){return U(this,t.apply(this._wrapped,arguments))}}),d.prototype.value=function(){return this._wrapped},d.prototype.valueOf=d.prototype.toJSON=d.prototype.value,d.prototype.toString=function(){return""+this._wrapped}}).call(this)},{}],26:[function(n,t,e){arguments[4][19][0].apply(e,arguments)},{dup:19}],27:[function(n,t,e){t.exports=function(n){return n&&"object"==typeof n&&"function"==typeof n.copy&&"function"==typeof n.fill&&"function"==typeof n.readUInt8}},{}],28:[function(h,n,k){(function(r,i){var a=/%[sdj%]/g;k.format=function(n){if(!_(n)){for(var t=[],e=0;e elements + // (i.e., `typeof document.createElement( "object" ) === "function"`). + // We don't want to classify *any* DOM node as a function. + // Support: QtWeb <=3.8.5, WebKit <=534.34, wkhtmltopdf tool <=0.12.5 + // Plus for old WebKit, typeof returns "function" for HTML collections + // (e.g., `typeof document.getElementsByTagName("div") === "function"`). (gh-4756) + return typeof obj === "function" && typeof obj.nodeType !== "number" && + typeof obj.item !== "function"; + }; + + +var isWindow = function isWindow( obj ) { + return obj != null && obj === obj.window; + }; + + +var document = window.document; + + + + var preservedScriptAttributes = { + type: true, + src: true, + nonce: true, + noModule: true + }; + + function DOMEval( code, node, doc ) { + doc = doc || document; + + var i, val, + script = doc.createElement( "script" ); + + script.text = code; + if ( node ) { + for ( i in preservedScriptAttributes ) { + + // Support: Firefox 64+, Edge 18+ + // Some browsers don't support the "nonce" property on scripts. + // On the other hand, just using `getAttribute` is not enough as + // the `nonce` attribute is reset to an empty string whenever it + // becomes browsing-context connected. + // See https://github.com/whatwg/html/issues/2369 + // See https://html.spec.whatwg.org/#nonce-attributes + // The `node.getAttribute` check was added for the sake of + // `jQuery.globalEval` so that it can fake a nonce-containing node + // via an object. + val = node[ i ] || node.getAttribute && node.getAttribute( i ); + if ( val ) { + script.setAttribute( i, val ); + } + } + } + doc.head.appendChild( script ).parentNode.removeChild( script ); + } + + +function toType( obj ) { + if ( obj == null ) { + return obj + ""; + } + + // Support: Android <=2.3 only (functionish RegExp) + return typeof obj === "object" || typeof obj === "function" ? + class2type[ toString.call( obj ) ] || "object" : + typeof obj; +} +/* global Symbol */ +// Defining this global in .eslintrc.json would create a danger of using the global +// unguarded in another place, it seems safer to define global only for this module + + + +var + version = "3.6.0", + + // Define a local copy of jQuery + jQuery = function( selector, context ) { + + // The jQuery object is actually just the init constructor 'enhanced' + // Need init if jQuery is called (just allow error to be thrown if not included) + return new jQuery.fn.init( selector, context ); + }; + +jQuery.fn = jQuery.prototype = { + + // The current version of jQuery being used + jquery: version, + + constructor: jQuery, + + // The default length of a jQuery object is 0 + length: 0, + + toArray: function() { + return slice.call( this ); + }, + + // Get the Nth element in the matched element set OR + // Get the whole matched element set as a clean array + get: function( num ) { + + // Return all the elements in a clean array + if ( num == null ) { + return slice.call( this ); + } + + // Return just the one element from the set + return num < 0 ? this[ num + this.length ] : this[ num ]; + }, + + // Take an array of elements and push it onto the stack + // (returning the new matched element set) + pushStack: function( elems ) { + + // Build a new jQuery matched element set + var ret = jQuery.merge( this.constructor(), elems ); + + // Add the old object onto the stack (as a reference) + ret.prevObject = this; + + // Return the newly-formed element set + return ret; + }, + + // Execute a callback for every element in the matched set. + each: function( callback ) { + return jQuery.each( this, callback ); + }, + + map: function( callback ) { + return this.pushStack( jQuery.map( this, function( elem, i ) { + return callback.call( elem, i, elem ); + } ) ); + }, + + slice: function() { + return this.pushStack( slice.apply( this, arguments ) ); + }, + + first: function() { + return this.eq( 0 ); + }, + + last: function() { + return this.eq( -1 ); + }, + + even: function() { + return this.pushStack( jQuery.grep( this, function( _elem, i ) { + return ( i + 1 ) % 2; + } ) ); + }, + + odd: function() { + return this.pushStack( jQuery.grep( this, function( _elem, i ) { + return i % 2; + } ) ); + }, + + eq: function( i ) { + var len = this.length, + j = +i + ( i < 0 ? len : 0 ); + return this.pushStack( j >= 0 && j < len ? [ this[ j ] ] : [] ); + }, + + end: function() { + return this.prevObject || this.constructor(); + }, + + // For internal use only. + // Behaves like an Array's method, not like a jQuery method. + push: push, + sort: arr.sort, + splice: arr.splice +}; + +jQuery.extend = jQuery.fn.extend = function() { + var options, name, src, copy, copyIsArray, clone, + target = arguments[ 0 ] || {}, + i = 1, + length = arguments.length, + deep = false; + + // Handle a deep copy situation + if ( typeof target === "boolean" ) { + deep = target; + + // Skip the boolean and the target + target = arguments[ i ] || {}; + i++; + } + + // Handle case when target is a string or something (possible in deep copy) + if ( typeof target !== "object" && !isFunction( target ) ) { + target = {}; + } + + // Extend jQuery itself if only one argument is passed + if ( i === length ) { + target = this; + i--; + } + + for ( ; i < length; i++ ) { + + // Only deal with non-null/undefined values + if ( ( options = arguments[ i ] ) != null ) { + + // Extend the base object + for ( name in options ) { + copy = options[ name ]; + + // Prevent Object.prototype pollution + // Prevent never-ending loop + if ( name === "__proto__" || target === copy ) { + continue; + } + + // Recurse if we're merging plain objects or arrays + if ( deep && copy && ( jQuery.isPlainObject( copy ) || + ( copyIsArray = Array.isArray( copy ) ) ) ) { + src = target[ name ]; + + // Ensure proper type for the source value + if ( copyIsArray && !Array.isArray( src ) ) { + clone = []; + } else if ( !copyIsArray && !jQuery.isPlainObject( src ) ) { + clone = {}; + } else { + clone = src; + } + copyIsArray = false; + + // Never move original objects, clone them + target[ name ] = jQuery.extend( deep, clone, copy ); + + // Don't bring in undefined values + } else if ( copy !== undefined ) { + target[ name ] = copy; + } + } + } + } + + // Return the modified object + return target; +}; + +jQuery.extend( { + + // Unique for each copy of jQuery on the page + expando: "jQuery" + ( version + Math.random() ).replace( /\D/g, "" ), + + // Assume jQuery is ready without the ready module + isReady: true, + + error: function( msg ) { + throw new Error( msg ); + }, + + noop: function() {}, + + isPlainObject: function( obj ) { + var proto, Ctor; + + // Detect obvious negatives + // Use toString instead of jQuery.type to catch host objects + if ( !obj || toString.call( obj ) !== "[object Object]" ) { + return false; + } + + proto = getProto( obj ); + + // Objects with no prototype (e.g., `Object.create( null )`) are plain + if ( !proto ) { + return true; + } + + // Objects with prototype are plain iff they were constructed by a global Object function + Ctor = hasOwn.call( proto, "constructor" ) && proto.constructor; + return typeof Ctor === "function" && fnToString.call( Ctor ) === ObjectFunctionString; + }, + + isEmptyObject: function( obj ) { + var name; + + for ( name in obj ) { + return false; + } + return true; + }, + + // Evaluates a script in a provided context; falls back to the global one + // if not specified. + globalEval: function( code, options, doc ) { + DOMEval( code, { nonce: options && options.nonce }, doc ); + }, + + each: function( obj, callback ) { + var length, i = 0; + + if ( isArrayLike( obj ) ) { + length = obj.length; + for ( ; i < length; i++ ) { + if ( callback.call( obj[ i ], i, obj[ i ] ) === false ) { + break; + } + } + } else { + for ( i in obj ) { + if ( callback.call( obj[ i ], i, obj[ i ] ) === false ) { + break; + } + } + } + + return obj; + }, + + // results is for internal usage only + makeArray: function( arr, results ) { + var ret = results || []; + + if ( arr != null ) { + if ( isArrayLike( Object( arr ) ) ) { + jQuery.merge( ret, + typeof arr === "string" ? + [ arr ] : arr + ); + } else { + push.call( ret, arr ); + } + } + + return ret; + }, + + inArray: function( elem, arr, i ) { + return arr == null ? -1 : indexOf.call( arr, elem, i ); + }, + + // Support: Android <=4.0 only, PhantomJS 1 only + // push.apply(_, arraylike) throws on ancient WebKit + merge: function( first, second ) { + var len = +second.length, + j = 0, + i = first.length; + + for ( ; j < len; j++ ) { + first[ i++ ] = second[ j ]; + } + + first.length = i; + + return first; + }, + + grep: function( elems, callback, invert ) { + var callbackInverse, + matches = [], + i = 0, + length = elems.length, + callbackExpect = !invert; + + // Go through the array, only saving the items + // that pass the validator function + for ( ; i < length; i++ ) { + callbackInverse = !callback( elems[ i ], i ); + if ( callbackInverse !== callbackExpect ) { + matches.push( elems[ i ] ); + } + } + + return matches; + }, + + // arg is for internal usage only + map: function( elems, callback, arg ) { + var length, value, + i = 0, + ret = []; + + // Go through the array, translating each of the items to their new values + if ( isArrayLike( elems ) ) { + length = elems.length; + for ( ; i < length; i++ ) { + value = callback( elems[ i ], i, arg ); + + if ( value != null ) { + ret.push( value ); + } + } + + // Go through every key on the object, + } else { + for ( i in elems ) { + value = callback( elems[ i ], i, arg ); + + if ( value != null ) { + ret.push( value ); + } + } + } + + // Flatten any nested arrays + return flat( ret ); + }, + + // A global GUID counter for objects + guid: 1, + + // jQuery.support is not used in Core but other projects attach their + // properties to it so it needs to exist. + support: support +} ); + +if ( typeof Symbol === "function" ) { + jQuery.fn[ Symbol.iterator ] = arr[ Symbol.iterator ]; +} + +// Populate the class2type map +jQuery.each( "Boolean Number String Function Array Date RegExp Object Error Symbol".split( " " ), + function( _i, name ) { + class2type[ "[object " + name + "]" ] = name.toLowerCase(); + } ); + +function isArrayLike( obj ) { + + // Support: real iOS 8.2 only (not reproducible in simulator) + // `in` check used to prevent JIT error (gh-2145) + // hasOwn isn't used here due to false negatives + // regarding Nodelist length in IE + var length = !!obj && "length" in obj && obj.length, + type = toType( obj ); + + if ( isFunction( obj ) || isWindow( obj ) ) { + return false; + } + + return type === "array" || length === 0 || + typeof length === "number" && length > 0 && ( length - 1 ) in obj; +} +var Sizzle = +/*! + * Sizzle CSS Selector Engine v2.3.6 + * https://sizzlejs.com/ + * + * Copyright JS Foundation and other contributors + * Released under the MIT license + * https://js.foundation/ + * + * Date: 2021-02-16 + */ +( function( window ) { +var i, + support, + Expr, + getText, + isXML, + tokenize, + compile, + select, + outermostContext, + sortInput, + hasDuplicate, + + // Local document vars + setDocument, + document, + docElem, + documentIsHTML, + rbuggyQSA, + rbuggyMatches, + matches, + contains, + + // Instance-specific data + expando = "sizzle" + 1 * new Date(), + preferredDoc = window.document, + dirruns = 0, + done = 0, + classCache = createCache(), + tokenCache = createCache(), + compilerCache = createCache(), + nonnativeSelectorCache = createCache(), + sortOrder = function( a, b ) { + if ( a === b ) { + hasDuplicate = true; + } + return 0; + }, + + // Instance methods + hasOwn = ( {} ).hasOwnProperty, + arr = [], + pop = arr.pop, + pushNative = arr.push, + push = arr.push, + slice = arr.slice, + + // Use a stripped-down indexOf as it's faster than native + // https://jsperf.com/thor-indexof-vs-for/5 + indexOf = function( list, elem ) { + var i = 0, + len = list.length; + for ( ; i < len; i++ ) { + if ( list[ i ] === elem ) { + return i; + } + } + return -1; + }, + + booleans = "checked|selected|async|autofocus|autoplay|controls|defer|disabled|hidden|" + + "ismap|loop|multiple|open|readonly|required|scoped", + + // Regular expressions + + // http://www.w3.org/TR/css3-selectors/#whitespace + whitespace = "[\\x20\\t\\r\\n\\f]", + + // https://www.w3.org/TR/css-syntax-3/#ident-token-diagram + identifier = "(?:\\\\[\\da-fA-F]{1,6}" + whitespace + + "?|\\\\[^\\r\\n\\f]|[\\w-]|[^\0-\\x7f])+", + + // Attribute selectors: http://www.w3.org/TR/selectors/#attribute-selectors + attributes = "\\[" + whitespace + "*(" + identifier + ")(?:" + whitespace + + + // Operator (capture 2) + "*([*^$|!~]?=)" + whitespace + + + // "Attribute values must be CSS identifiers [capture 5] + // or strings [capture 3 or capture 4]" + "*(?:'((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\"|(" + identifier + "))|)" + + whitespace + "*\\]", + + pseudos = ":(" + identifier + ")(?:\\((" + + + // To reduce the number of selectors needing tokenize in the preFilter, prefer arguments: + // 1. quoted (capture 3; capture 4 or capture 5) + "('((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\")|" + + + // 2. simple (capture 6) + "((?:\\\\.|[^\\\\()[\\]]|" + attributes + ")*)|" + + + // 3. anything else (capture 2) + ".*" + + ")\\)|)", + + // Leading and non-escaped trailing whitespace, capturing some non-whitespace characters preceding the latter + rwhitespace = new RegExp( whitespace + "+", "g" ), + rtrim = new RegExp( "^" + whitespace + "+|((?:^|[^\\\\])(?:\\\\.)*)" + + whitespace + "+$", "g" ), + + rcomma = new RegExp( "^" + whitespace + "*," + whitespace + "*" ), + rcombinators = new RegExp( "^" + whitespace + "*([>+~]|" + whitespace + ")" + whitespace + + "*" ), + rdescend = new RegExp( whitespace + "|>" ), + + rpseudo = new RegExp( pseudos ), + ridentifier = new RegExp( "^" + identifier + "$" ), + + matchExpr = { + "ID": new RegExp( "^#(" + identifier + ")" ), + "CLASS": new RegExp( "^\\.(" + identifier + ")" ), + "TAG": new RegExp( "^(" + identifier + "|[*])" ), + "ATTR": new RegExp( "^" + attributes ), + "PSEUDO": new RegExp( "^" + pseudos ), + "CHILD": new RegExp( "^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\(" + + whitespace + "*(even|odd|(([+-]|)(\\d*)n|)" + whitespace + "*(?:([+-]|)" + + whitespace + "*(\\d+)|))" + whitespace + "*\\)|)", "i" ), + "bool": new RegExp( "^(?:" + booleans + ")$", "i" ), + + // For use in libraries implementing .is() + // We use this for POS matching in `select` + "needsContext": new RegExp( "^" + whitespace + + "*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\(" + whitespace + + "*((?:-\\d)?\\d*)" + whitespace + "*\\)|)(?=[^-]|$)", "i" ) + }, + + rhtml = /HTML$/i, + rinputs = /^(?:input|select|textarea|button)$/i, + rheader = /^h\d$/i, + + rnative = /^[^{]+\{\s*\[native \w/, + + // Easily-parseable/retrievable ID or TAG or CLASS selectors + rquickExpr = /^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/, + + rsibling = /[+~]/, + + // CSS escapes + // http://www.w3.org/TR/CSS21/syndata.html#escaped-characters + runescape = new RegExp( "\\\\[\\da-fA-F]{1,6}" + whitespace + "?|\\\\([^\\r\\n\\f])", "g" ), + funescape = function( escape, nonHex ) { + var high = "0x" + escape.slice( 1 ) - 0x10000; + + return nonHex ? + + // Strip the backslash prefix from a non-hex escape sequence + nonHex : + + // Replace a hexadecimal escape sequence with the encoded Unicode code point + // Support: IE <=11+ + // For values outside the Basic Multilingual Plane (BMP), manually construct a + // surrogate pair + high < 0 ? + String.fromCharCode( high + 0x10000 ) : + String.fromCharCode( high >> 10 | 0xD800, high & 0x3FF | 0xDC00 ); + }, + + // CSS string/identifier serialization + // https://drafts.csswg.org/cssom/#common-serializing-idioms + rcssescape = /([\0-\x1f\x7f]|^-?\d)|^-$|[^\0-\x1f\x7f-\uFFFF\w-]/g, + fcssescape = function( ch, asCodePoint ) { + if ( asCodePoint ) { + + // U+0000 NULL becomes U+FFFD REPLACEMENT CHARACTER + if ( ch === "\0" ) { + return "\uFFFD"; + } + + // Control characters and (dependent upon position) numbers get escaped as code points + return ch.slice( 0, -1 ) + "\\" + + ch.charCodeAt( ch.length - 1 ).toString( 16 ) + " "; + } + + // Other potentially-special ASCII characters get backslash-escaped + return "\\" + ch; + }, + + // Used for iframes + // See setDocument() + // Removing the function wrapper causes a "Permission Denied" + // error in IE + unloadHandler = function() { + setDocument(); + }, + + inDisabledFieldset = addCombinator( + function( elem ) { + return elem.disabled === true && elem.nodeName.toLowerCase() === "fieldset"; + }, + { dir: "parentNode", next: "legend" } + ); + +// Optimize for push.apply( _, NodeList ) +try { + push.apply( + ( arr = slice.call( preferredDoc.childNodes ) ), + preferredDoc.childNodes + ); + + // Support: Android<4.0 + // Detect silently failing push.apply + // eslint-disable-next-line no-unused-expressions + arr[ preferredDoc.childNodes.length ].nodeType; +} catch ( e ) { + push = { apply: arr.length ? + + // Leverage slice if possible + function( target, els ) { + pushNative.apply( target, slice.call( els ) ); + } : + + // Support: IE<9 + // Otherwise append directly + function( target, els ) { + var j = target.length, + i = 0; + + // Can't trust NodeList.length + while ( ( target[ j++ ] = els[ i++ ] ) ) {} + target.length = j - 1; + } + }; +} + +function Sizzle( selector, context, results, seed ) { + var m, i, elem, nid, match, groups, newSelector, + newContext = context && context.ownerDocument, + + // nodeType defaults to 9, since context defaults to document + nodeType = context ? context.nodeType : 9; + + results = results || []; + + // Return early from calls with invalid selector or context + if ( typeof selector !== "string" || !selector || + nodeType !== 1 && nodeType !== 9 && nodeType !== 11 ) { + + return results; + } + + // Try to shortcut find operations (as opposed to filters) in HTML documents + if ( !seed ) { + setDocument( context ); + context = context || document; + + if ( documentIsHTML ) { + + // If the selector is sufficiently simple, try using a "get*By*" DOM method + // (excepting DocumentFragment context, where the methods don't exist) + if ( nodeType !== 11 && ( match = rquickExpr.exec( selector ) ) ) { + + // ID selector + if ( ( m = match[ 1 ] ) ) { + + // Document context + if ( nodeType === 9 ) { + if ( ( elem = context.getElementById( m ) ) ) { + + // Support: IE, Opera, Webkit + // TODO: identify versions + // getElementById can match elements by name instead of ID + if ( elem.id === m ) { + results.push( elem ); + return results; + } + } else { + return results; + } + + // Element context + } else { + + // Support: IE, Opera, Webkit + // TODO: identify versions + // getElementById can match elements by name instead of ID + if ( newContext && ( elem = newContext.getElementById( m ) ) && + contains( context, elem ) && + elem.id === m ) { + + results.push( elem ); + return results; + } + } + + // Type selector + } else if ( match[ 2 ] ) { + push.apply( results, context.getElementsByTagName( selector ) ); + return results; + + // Class selector + } else if ( ( m = match[ 3 ] ) && support.getElementsByClassName && + context.getElementsByClassName ) { + + push.apply( results, context.getElementsByClassName( m ) ); + return results; + } + } + + // Take advantage of querySelectorAll + if ( support.qsa && + !nonnativeSelectorCache[ selector + " " ] && + ( !rbuggyQSA || !rbuggyQSA.test( selector ) ) && + + // Support: IE 8 only + // Exclude object elements + ( nodeType !== 1 || context.nodeName.toLowerCase() !== "object" ) ) { + + newSelector = selector; + newContext = context; + + // qSA considers elements outside a scoping root when evaluating child or + // descendant combinators, which is not what we want. + // In such cases, we work around the behavior by prefixing every selector in the + // list with an ID selector referencing the scope context. + // The technique has to be used as well when a leading combinator is used + // as such selectors are not recognized by querySelectorAll. + // Thanks to Andrew Dupont for this technique. + if ( nodeType === 1 && + ( rdescend.test( selector ) || rcombinators.test( selector ) ) ) { + + // Expand context for sibling selectors + newContext = rsibling.test( selector ) && testContext( context.parentNode ) || + context; + + // We can use :scope instead of the ID hack if the browser + // supports it & if we're not changing the context. + if ( newContext !== context || !support.scope ) { + + // Capture the context ID, setting it first if necessary + if ( ( nid = context.getAttribute( "id" ) ) ) { + nid = nid.replace( rcssescape, fcssescape ); + } else { + context.setAttribute( "id", ( nid = expando ) ); + } + } + + // Prefix every selector in the list + groups = tokenize( selector ); + i = groups.length; + while ( i-- ) { + groups[ i ] = ( nid ? "#" + nid : ":scope" ) + " " + + toSelector( groups[ i ] ); + } + newSelector = groups.join( "," ); + } + + try { + push.apply( results, + newContext.querySelectorAll( newSelector ) + ); + return results; + } catch ( qsaError ) { + nonnativeSelectorCache( selector, true ); + } finally { + if ( nid === expando ) { + context.removeAttribute( "id" ); + } + } + } + } + } + + // All others + return select( selector.replace( rtrim, "$1" ), context, results, seed ); +} + +/** + * Create key-value caches of limited size + * @returns {function(string, object)} Returns the Object data after storing it on itself with + * property name the (space-suffixed) string and (if the cache is larger than Expr.cacheLength) + * deleting the oldest entry + */ +function createCache() { + var keys = []; + + function cache( key, value ) { + + // Use (key + " ") to avoid collision with native prototype properties (see Issue #157) + if ( keys.push( key + " " ) > Expr.cacheLength ) { + + // Only keep the most recent entries + delete cache[ keys.shift() ]; + } + return ( cache[ key + " " ] = value ); + } + return cache; +} + +/** + * Mark a function for special use by Sizzle + * @param {Function} fn The function to mark + */ +function markFunction( fn ) { + fn[ expando ] = true; + return fn; +} + +/** + * Support testing using an element + * @param {Function} fn Passed the created element and returns a boolean result + */ +function assert( fn ) { + var el = document.createElement( "fieldset" ); + + try { + return !!fn( el ); + } catch ( e ) { + return false; + } finally { + + // Remove from its parent by default + if ( el.parentNode ) { + el.parentNode.removeChild( el ); + } + + // release memory in IE + el = null; + } +} + +/** + * Adds the same handler for all of the specified attrs + * @param {String} attrs Pipe-separated list of attributes + * @param {Function} handler The method that will be applied + */ +function addHandle( attrs, handler ) { + var arr = attrs.split( "|" ), + i = arr.length; + + while ( i-- ) { + Expr.attrHandle[ arr[ i ] ] = handler; + } +} + +/** + * Checks document order of two siblings + * @param {Element} a + * @param {Element} b + * @returns {Number} Returns less than 0 if a precedes b, greater than 0 if a follows b + */ +function siblingCheck( a, b ) { + var cur = b && a, + diff = cur && a.nodeType === 1 && b.nodeType === 1 && + a.sourceIndex - b.sourceIndex; + + // Use IE sourceIndex if available on both nodes + if ( diff ) { + return diff; + } + + // Check if b follows a + if ( cur ) { + while ( ( cur = cur.nextSibling ) ) { + if ( cur === b ) { + return -1; + } + } + } + + return a ? 1 : -1; +} + +/** + * Returns a function to use in pseudos for input types + * @param {String} type + */ +function createInputPseudo( type ) { + return function( elem ) { + var name = elem.nodeName.toLowerCase(); + return name === "input" && elem.type === type; + }; +} + +/** + * Returns a function to use in pseudos for buttons + * @param {String} type + */ +function createButtonPseudo( type ) { + return function( elem ) { + var name = elem.nodeName.toLowerCase(); + return ( name === "input" || name === "button" ) && elem.type === type; + }; +} + +/** + * Returns a function to use in pseudos for :enabled/:disabled + * @param {Boolean} disabled true for :disabled; false for :enabled + */ +function createDisabledPseudo( disabled ) { + + // Known :disabled false positives: fieldset[disabled] > legend:nth-of-type(n+2) :can-disable + return function( elem ) { + + // Only certain elements can match :enabled or :disabled + // https://html.spec.whatwg.org/multipage/scripting.html#selector-enabled + // https://html.spec.whatwg.org/multipage/scripting.html#selector-disabled + if ( "form" in elem ) { + + // Check for inherited disabledness on relevant non-disabled elements: + // * listed form-associated elements in a disabled fieldset + // https://html.spec.whatwg.org/multipage/forms.html#category-listed + // https://html.spec.whatwg.org/multipage/forms.html#concept-fe-disabled + // * option elements in a disabled optgroup + // https://html.spec.whatwg.org/multipage/forms.html#concept-option-disabled + // All such elements have a "form" property. + if ( elem.parentNode && elem.disabled === false ) { + + // Option elements defer to a parent optgroup if present + if ( "label" in elem ) { + if ( "label" in elem.parentNode ) { + return elem.parentNode.disabled === disabled; + } else { + return elem.disabled === disabled; + } + } + + // Support: IE 6 - 11 + // Use the isDisabled shortcut property to check for disabled fieldset ancestors + return elem.isDisabled === disabled || + + // Where there is no isDisabled, check manually + /* jshint -W018 */ + elem.isDisabled !== !disabled && + inDisabledFieldset( elem ) === disabled; + } + + return elem.disabled === disabled; + + // Try to winnow out elements that can't be disabled before trusting the disabled property. + // Some victims get caught in our net (label, legend, menu, track), but it shouldn't + // even exist on them, let alone have a boolean value. + } else if ( "label" in elem ) { + return elem.disabled === disabled; + } + + // Remaining elements are neither :enabled nor :disabled + return false; + }; +} + +/** + * Returns a function to use in pseudos for positionals + * @param {Function} fn + */ +function createPositionalPseudo( fn ) { + return markFunction( function( argument ) { + argument = +argument; + return markFunction( function( seed, matches ) { + var j, + matchIndexes = fn( [], seed.length, argument ), + i = matchIndexes.length; + + // Match elements found at the specified indexes + while ( i-- ) { + if ( seed[ ( j = matchIndexes[ i ] ) ] ) { + seed[ j ] = !( matches[ j ] = seed[ j ] ); + } + } + } ); + } ); +} + +/** + * Checks a node for validity as a Sizzle context + * @param {Element|Object=} context + * @returns {Element|Object|Boolean} The input node if acceptable, otherwise a falsy value + */ +function testContext( context ) { + return context && typeof context.getElementsByTagName !== "undefined" && context; +} + +// Expose support vars for convenience +support = Sizzle.support = {}; + +/** + * Detects XML nodes + * @param {Element|Object} elem An element or a document + * @returns {Boolean} True iff elem is a non-HTML XML node + */ +isXML = Sizzle.isXML = function( elem ) { + var namespace = elem && elem.namespaceURI, + docElem = elem && ( elem.ownerDocument || elem ).documentElement; + + // Support: IE <=8 + // Assume HTML when documentElement doesn't yet exist, such as inside loading iframes + // https://bugs.jquery.com/ticket/4833 + return !rhtml.test( namespace || docElem && docElem.nodeName || "HTML" ); +}; + +/** + * Sets document-related variables once based on the current document + * @param {Element|Object} [doc] An element or document object to use to set the document + * @returns {Object} Returns the current document + */ +setDocument = Sizzle.setDocument = function( node ) { + var hasCompare, subWindow, + doc = node ? node.ownerDocument || node : preferredDoc; + + // Return early if doc is invalid or already selected + // Support: IE 11+, Edge 17 - 18+ + // IE/Edge sometimes throw a "Permission denied" error when strict-comparing + // two documents; shallow comparisons work. + // eslint-disable-next-line eqeqeq + if ( doc == document || doc.nodeType !== 9 || !doc.documentElement ) { + return document; + } + + // Update global variables + document = doc; + docElem = document.documentElement; + documentIsHTML = !isXML( document ); + + // Support: IE 9 - 11+, Edge 12 - 18+ + // Accessing iframe documents after unload throws "permission denied" errors (jQuery #13936) + // Support: IE 11+, Edge 17 - 18+ + // IE/Edge sometimes throw a "Permission denied" error when strict-comparing + // two documents; shallow comparisons work. + // eslint-disable-next-line eqeqeq + if ( preferredDoc != document && + ( subWindow = document.defaultView ) && subWindow.top !== subWindow ) { + + // Support: IE 11, Edge + if ( subWindow.addEventListener ) { + subWindow.addEventListener( "unload", unloadHandler, false ); + + // Support: IE 9 - 10 only + } else if ( subWindow.attachEvent ) { + subWindow.attachEvent( "onunload", unloadHandler ); + } + } + + // Support: IE 8 - 11+, Edge 12 - 18+, Chrome <=16 - 25 only, Firefox <=3.6 - 31 only, + // Safari 4 - 5 only, Opera <=11.6 - 12.x only + // IE/Edge & older browsers don't support the :scope pseudo-class. + // Support: Safari 6.0 only + // Safari 6.0 supports :scope but it's an alias of :root there. + support.scope = assert( function( el ) { + docElem.appendChild( el ).appendChild( document.createElement( "div" ) ); + return typeof el.querySelectorAll !== "undefined" && + !el.querySelectorAll( ":scope fieldset div" ).length; + } ); + + /* Attributes + ---------------------------------------------------------------------- */ + + // Support: IE<8 + // Verify that getAttribute really returns attributes and not properties + // (excepting IE8 booleans) + support.attributes = assert( function( el ) { + el.className = "i"; + return !el.getAttribute( "className" ); + } ); + + /* getElement(s)By* + ---------------------------------------------------------------------- */ + + // Check if getElementsByTagName("*") returns only elements + support.getElementsByTagName = assert( function( el ) { + el.appendChild( document.createComment( "" ) ); + return !el.getElementsByTagName( "*" ).length; + } ); + + // Support: IE<9 + support.getElementsByClassName = rnative.test( document.getElementsByClassName ); + + // Support: IE<10 + // Check if getElementById returns elements by name + // The broken getElementById methods don't pick up programmatically-set names, + // so use a roundabout getElementsByName test + support.getById = assert( function( el ) { + docElem.appendChild( el ).id = expando; + return !document.getElementsByName || !document.getElementsByName( expando ).length; + } ); + + // ID filter and find + if ( support.getById ) { + Expr.filter[ "ID" ] = function( id ) { + var attrId = id.replace( runescape, funescape ); + return function( elem ) { + return elem.getAttribute( "id" ) === attrId; + }; + }; + Expr.find[ "ID" ] = function( id, context ) { + if ( typeof context.getElementById !== "undefined" && documentIsHTML ) { + var elem = context.getElementById( id ); + return elem ? [ elem ] : []; + } + }; + } else { + Expr.filter[ "ID" ] = function( id ) { + var attrId = id.replace( runescape, funescape ); + return function( elem ) { + var node = typeof elem.getAttributeNode !== "undefined" && + elem.getAttributeNode( "id" ); + return node && node.value === attrId; + }; + }; + + // Support: IE 6 - 7 only + // getElementById is not reliable as a find shortcut + Expr.find[ "ID" ] = function( id, context ) { + if ( typeof context.getElementById !== "undefined" && documentIsHTML ) { + var node, i, elems, + elem = context.getElementById( id ); + + if ( elem ) { + + // Verify the id attribute + node = elem.getAttributeNode( "id" ); + if ( node && node.value === id ) { + return [ elem ]; + } + + // Fall back on getElementsByName + elems = context.getElementsByName( id ); + i = 0; + while ( ( elem = elems[ i++ ] ) ) { + node = elem.getAttributeNode( "id" ); + if ( node && node.value === id ) { + return [ elem ]; + } + } + } + + return []; + } + }; + } + + // Tag + Expr.find[ "TAG" ] = support.getElementsByTagName ? + function( tag, context ) { + if ( typeof context.getElementsByTagName !== "undefined" ) { + return context.getElementsByTagName( tag ); + + // DocumentFragment nodes don't have gEBTN + } else if ( support.qsa ) { + return context.querySelectorAll( tag ); + } + } : + + function( tag, context ) { + var elem, + tmp = [], + i = 0, + + // By happy coincidence, a (broken) gEBTN appears on DocumentFragment nodes too + results = context.getElementsByTagName( tag ); + + // Filter out possible comments + if ( tag === "*" ) { + while ( ( elem = results[ i++ ] ) ) { + if ( elem.nodeType === 1 ) { + tmp.push( elem ); + } + } + + return tmp; + } + return results; + }; + + // Class + Expr.find[ "CLASS" ] = support.getElementsByClassName && function( className, context ) { + if ( typeof context.getElementsByClassName !== "undefined" && documentIsHTML ) { + return context.getElementsByClassName( className ); + } + }; + + /* QSA/matchesSelector + ---------------------------------------------------------------------- */ + + // QSA and matchesSelector support + + // matchesSelector(:active) reports false when true (IE9/Opera 11.5) + rbuggyMatches = []; + + // qSa(:focus) reports false when true (Chrome 21) + // We allow this because of a bug in IE8/9 that throws an error + // whenever `document.activeElement` is accessed on an iframe + // So, we allow :focus to pass through QSA all the time to avoid the IE error + // See https://bugs.jquery.com/ticket/13378 + rbuggyQSA = []; + + if ( ( support.qsa = rnative.test( document.querySelectorAll ) ) ) { + + // Build QSA regex + // Regex strategy adopted from Diego Perini + assert( function( el ) { + + var input; + + // Select is set to empty string on purpose + // This is to test IE's treatment of not explicitly + // setting a boolean content attribute, + // since its presence should be enough + // https://bugs.jquery.com/ticket/12359 + docElem.appendChild( el ).innerHTML = "" + + ""; + + // Support: IE8, Opera 11-12.16 + // Nothing should be selected when empty strings follow ^= or $= or *= + // The test attribute must be unknown in Opera but "safe" for WinRT + // https://msdn.microsoft.com/en-us/library/ie/hh465388.aspx#attribute_section + if ( el.querySelectorAll( "[msallowcapture^='']" ).length ) { + rbuggyQSA.push( "[*^$]=" + whitespace + "*(?:''|\"\")" ); + } + + // Support: IE8 + // Boolean attributes and "value" are not treated correctly + if ( !el.querySelectorAll( "[selected]" ).length ) { + rbuggyQSA.push( "\\[" + whitespace + "*(?:value|" + booleans + ")" ); + } + + // Support: Chrome<29, Android<4.4, Safari<7.0+, iOS<7.0+, PhantomJS<1.9.8+ + if ( !el.querySelectorAll( "[id~=" + expando + "-]" ).length ) { + rbuggyQSA.push( "~=" ); + } + + // Support: IE 11+, Edge 15 - 18+ + // IE 11/Edge don't find elements on a `[name='']` query in some cases. + // Adding a temporary attribute to the document before the selection works + // around the issue. + // Interestingly, IE 10 & older don't seem to have the issue. + input = document.createElement( "input" ); + input.setAttribute( "name", "" ); + el.appendChild( input ); + if ( !el.querySelectorAll( "[name='']" ).length ) { + rbuggyQSA.push( "\\[" + whitespace + "*name" + whitespace + "*=" + + whitespace + "*(?:''|\"\")" ); + } + + // Webkit/Opera - :checked should return selected option elements + // http://www.w3.org/TR/2011/REC-css3-selectors-20110929/#checked + // IE8 throws error here and will not see later tests + if ( !el.querySelectorAll( ":checked" ).length ) { + rbuggyQSA.push( ":checked" ); + } + + // Support: Safari 8+, iOS 8+ + // https://bugs.webkit.org/show_bug.cgi?id=136851 + // In-page `selector#id sibling-combinator selector` fails + if ( !el.querySelectorAll( "a#" + expando + "+*" ).length ) { + rbuggyQSA.push( ".#.+[+~]" ); + } + + // Support: Firefox <=3.6 - 5 only + // Old Firefox doesn't throw on a badly-escaped identifier. + el.querySelectorAll( "\\\f" ); + rbuggyQSA.push( "[\\r\\n\\f]" ); + } ); + + assert( function( el ) { + el.innerHTML = "" + + ""; + + // Support: Windows 8 Native Apps + // The type and name attributes are restricted during .innerHTML assignment + var input = document.createElement( "input" ); + input.setAttribute( "type", "hidden" ); + el.appendChild( input ).setAttribute( "name", "D" ); + + // Support: IE8 + // Enforce case-sensitivity of name attribute + if ( el.querySelectorAll( "[name=d]" ).length ) { + rbuggyQSA.push( "name" + whitespace + "*[*^$|!~]?=" ); + } + + // FF 3.5 - :enabled/:disabled and hidden elements (hidden elements are still enabled) + // IE8 throws error here and will not see later tests + if ( el.querySelectorAll( ":enabled" ).length !== 2 ) { + rbuggyQSA.push( ":enabled", ":disabled" ); + } + + // Support: IE9-11+ + // IE's :disabled selector does not pick up the children of disabled fieldsets + docElem.appendChild( el ).disabled = true; + if ( el.querySelectorAll( ":disabled" ).length !== 2 ) { + rbuggyQSA.push( ":enabled", ":disabled" ); + } + + // Support: Opera 10 - 11 only + // Opera 10-11 does not throw on post-comma invalid pseudos + el.querySelectorAll( "*,:x" ); + rbuggyQSA.push( ",.*:" ); + } ); + } + + if ( ( support.matchesSelector = rnative.test( ( matches = docElem.matches || + docElem.webkitMatchesSelector || + docElem.mozMatchesSelector || + docElem.oMatchesSelector || + docElem.msMatchesSelector ) ) ) ) { + + assert( function( el ) { + + // Check to see if it's possible to do matchesSelector + // on a disconnected node (IE 9) + support.disconnectedMatch = matches.call( el, "*" ); + + // This should fail with an exception + // Gecko does not error, returns false instead + matches.call( el, "[s!='']:x" ); + rbuggyMatches.push( "!=", pseudos ); + } ); + } + + rbuggyQSA = rbuggyQSA.length && new RegExp( rbuggyQSA.join( "|" ) ); + rbuggyMatches = rbuggyMatches.length && new RegExp( rbuggyMatches.join( "|" ) ); + + /* Contains + ---------------------------------------------------------------------- */ + hasCompare = rnative.test( docElem.compareDocumentPosition ); + + // Element contains another + // Purposefully self-exclusive + // As in, an element does not contain itself + contains = hasCompare || rnative.test( docElem.contains ) ? + function( a, b ) { + var adown = a.nodeType === 9 ? a.documentElement : a, + bup = b && b.parentNode; + return a === bup || !!( bup && bup.nodeType === 1 && ( + adown.contains ? + adown.contains( bup ) : + a.compareDocumentPosition && a.compareDocumentPosition( bup ) & 16 + ) ); + } : + function( a, b ) { + if ( b ) { + while ( ( b = b.parentNode ) ) { + if ( b === a ) { + return true; + } + } + } + return false; + }; + + /* Sorting + ---------------------------------------------------------------------- */ + + // Document order sorting + sortOrder = hasCompare ? + function( a, b ) { + + // Flag for duplicate removal + if ( a === b ) { + hasDuplicate = true; + return 0; + } + + // Sort on method existence if only one input has compareDocumentPosition + var compare = !a.compareDocumentPosition - !b.compareDocumentPosition; + if ( compare ) { + return compare; + } + + // Calculate position if both inputs belong to the same document + // Support: IE 11+, Edge 17 - 18+ + // IE/Edge sometimes throw a "Permission denied" error when strict-comparing + // two documents; shallow comparisons work. + // eslint-disable-next-line eqeqeq + compare = ( a.ownerDocument || a ) == ( b.ownerDocument || b ) ? + a.compareDocumentPosition( b ) : + + // Otherwise we know they are disconnected + 1; + + // Disconnected nodes + if ( compare & 1 || + ( !support.sortDetached && b.compareDocumentPosition( a ) === compare ) ) { + + // Choose the first element that is related to our preferred document + // Support: IE 11+, Edge 17 - 18+ + // IE/Edge sometimes throw a "Permission denied" error when strict-comparing + // two documents; shallow comparisons work. + // eslint-disable-next-line eqeqeq + if ( a == document || a.ownerDocument == preferredDoc && + contains( preferredDoc, a ) ) { + return -1; + } + + // Support: IE 11+, Edge 17 - 18+ + // IE/Edge sometimes throw a "Permission denied" error when strict-comparing + // two documents; shallow comparisons work. + // eslint-disable-next-line eqeqeq + if ( b == document || b.ownerDocument == preferredDoc && + contains( preferredDoc, b ) ) { + return 1; + } + + // Maintain original order + return sortInput ? + ( indexOf( sortInput, a ) - indexOf( sortInput, b ) ) : + 0; + } + + return compare & 4 ? -1 : 1; + } : + function( a, b ) { + + // Exit early if the nodes are identical + if ( a === b ) { + hasDuplicate = true; + return 0; + } + + var cur, + i = 0, + aup = a.parentNode, + bup = b.parentNode, + ap = [ a ], + bp = [ b ]; + + // Parentless nodes are either documents or disconnected + if ( !aup || !bup ) { + + // Support: IE 11+, Edge 17 - 18+ + // IE/Edge sometimes throw a "Permission denied" error when strict-comparing + // two documents; shallow comparisons work. + /* eslint-disable eqeqeq */ + return a == document ? -1 : + b == document ? 1 : + /* eslint-enable eqeqeq */ + aup ? -1 : + bup ? 1 : + sortInput ? + ( indexOf( sortInput, a ) - indexOf( sortInput, b ) ) : + 0; + + // If the nodes are siblings, we can do a quick check + } else if ( aup === bup ) { + return siblingCheck( a, b ); + } + + // Otherwise we need full lists of their ancestors for comparison + cur = a; + while ( ( cur = cur.parentNode ) ) { + ap.unshift( cur ); + } + cur = b; + while ( ( cur = cur.parentNode ) ) { + bp.unshift( cur ); + } + + // Walk down the tree looking for a discrepancy + while ( ap[ i ] === bp[ i ] ) { + i++; + } + + return i ? + + // Do a sibling check if the nodes have a common ancestor + siblingCheck( ap[ i ], bp[ i ] ) : + + // Otherwise nodes in our document sort first + // Support: IE 11+, Edge 17 - 18+ + // IE/Edge sometimes throw a "Permission denied" error when strict-comparing + // two documents; shallow comparisons work. + /* eslint-disable eqeqeq */ + ap[ i ] == preferredDoc ? -1 : + bp[ i ] == preferredDoc ? 1 : + /* eslint-enable eqeqeq */ + 0; + }; + + return document; +}; + +Sizzle.matches = function( expr, elements ) { + return Sizzle( expr, null, null, elements ); +}; + +Sizzle.matchesSelector = function( elem, expr ) { + setDocument( elem ); + + if ( support.matchesSelector && documentIsHTML && + !nonnativeSelectorCache[ expr + " " ] && + ( !rbuggyMatches || !rbuggyMatches.test( expr ) ) && + ( !rbuggyQSA || !rbuggyQSA.test( expr ) ) ) { + + try { + var ret = matches.call( elem, expr ); + + // IE 9's matchesSelector returns false on disconnected nodes + if ( ret || support.disconnectedMatch || + + // As well, disconnected nodes are said to be in a document + // fragment in IE 9 + elem.document && elem.document.nodeType !== 11 ) { + return ret; + } + } catch ( e ) { + nonnativeSelectorCache( expr, true ); + } + } + + return Sizzle( expr, document, null, [ elem ] ).length > 0; +}; + +Sizzle.contains = function( context, elem ) { + + // Set document vars if needed + // Support: IE 11+, Edge 17 - 18+ + // IE/Edge sometimes throw a "Permission denied" error when strict-comparing + // two documents; shallow comparisons work. + // eslint-disable-next-line eqeqeq + if ( ( context.ownerDocument || context ) != document ) { + setDocument( context ); + } + return contains( context, elem ); +}; + +Sizzle.attr = function( elem, name ) { + + // Set document vars if needed + // Support: IE 11+, Edge 17 - 18+ + // IE/Edge sometimes throw a "Permission denied" error when strict-comparing + // two documents; shallow comparisons work. + // eslint-disable-next-line eqeqeq + if ( ( elem.ownerDocument || elem ) != document ) { + setDocument( elem ); + } + + var fn = Expr.attrHandle[ name.toLowerCase() ], + + // Don't get fooled by Object.prototype properties (jQuery #13807) + val = fn && hasOwn.call( Expr.attrHandle, name.toLowerCase() ) ? + fn( elem, name, !documentIsHTML ) : + undefined; + + return val !== undefined ? + val : + support.attributes || !documentIsHTML ? + elem.getAttribute( name ) : + ( val = elem.getAttributeNode( name ) ) && val.specified ? + val.value : + null; +}; + +Sizzle.escape = function( sel ) { + return ( sel + "" ).replace( rcssescape, fcssescape ); +}; + +Sizzle.error = function( msg ) { + throw new Error( "Syntax error, unrecognized expression: " + msg ); +}; + +/** + * Document sorting and removing duplicates + * @param {ArrayLike} results + */ +Sizzle.uniqueSort = function( results ) { + var elem, + duplicates = [], + j = 0, + i = 0; + + // Unless we *know* we can detect duplicates, assume their presence + hasDuplicate = !support.detectDuplicates; + sortInput = !support.sortStable && results.slice( 0 ); + results.sort( sortOrder ); + + if ( hasDuplicate ) { + while ( ( elem = results[ i++ ] ) ) { + if ( elem === results[ i ] ) { + j = duplicates.push( i ); + } + } + while ( j-- ) { + results.splice( duplicates[ j ], 1 ); + } + } + + // Clear input after sorting to release objects + // See https://github.com/jquery/sizzle/pull/225 + sortInput = null; + + return results; +}; + +/** + * Utility function for retrieving the text value of an array of DOM nodes + * @param {Array|Element} elem + */ +getText = Sizzle.getText = function( elem ) { + var node, + ret = "", + i = 0, + nodeType = elem.nodeType; + + if ( !nodeType ) { + + // If no nodeType, this is expected to be an array + while ( ( node = elem[ i++ ] ) ) { + + // Do not traverse comment nodes + ret += getText( node ); + } + } else if ( nodeType === 1 || nodeType === 9 || nodeType === 11 ) { + + // Use textContent for elements + // innerText usage removed for consistency of new lines (jQuery #11153) + if ( typeof elem.textContent === "string" ) { + return elem.textContent; + } else { + + // Traverse its children + for ( elem = elem.firstChild; elem; elem = elem.nextSibling ) { + ret += getText( elem ); + } + } + } else if ( nodeType === 3 || nodeType === 4 ) { + return elem.nodeValue; + } + + // Do not include comment or processing instruction nodes + + return ret; +}; + +Expr = Sizzle.selectors = { + + // Can be adjusted by the user + cacheLength: 50, + + createPseudo: markFunction, + + match: matchExpr, + + attrHandle: {}, + + find: {}, + + relative: { + ">": { dir: "parentNode", first: true }, + " ": { dir: "parentNode" }, + "+": { dir: "previousSibling", first: true }, + "~": { dir: "previousSibling" } + }, + + preFilter: { + "ATTR": function( match ) { + match[ 1 ] = match[ 1 ].replace( runescape, funescape ); + + // Move the given value to match[3] whether quoted or unquoted + match[ 3 ] = ( match[ 3 ] || match[ 4 ] || + match[ 5 ] || "" ).replace( runescape, funescape ); + + if ( match[ 2 ] === "~=" ) { + match[ 3 ] = " " + match[ 3 ] + " "; + } + + return match.slice( 0, 4 ); + }, + + "CHILD": function( match ) { + + /* matches from matchExpr["CHILD"] + 1 type (only|nth|...) + 2 what (child|of-type) + 3 argument (even|odd|\d*|\d*n([+-]\d+)?|...) + 4 xn-component of xn+y argument ([+-]?\d*n|) + 5 sign of xn-component + 6 x of xn-component + 7 sign of y-component + 8 y of y-component + */ + match[ 1 ] = match[ 1 ].toLowerCase(); + + if ( match[ 1 ].slice( 0, 3 ) === "nth" ) { + + // nth-* requires argument + if ( !match[ 3 ] ) { + Sizzle.error( match[ 0 ] ); + } + + // numeric x and y parameters for Expr.filter.CHILD + // remember that false/true cast respectively to 0/1 + match[ 4 ] = +( match[ 4 ] ? + match[ 5 ] + ( match[ 6 ] || 1 ) : + 2 * ( match[ 3 ] === "even" || match[ 3 ] === "odd" ) ); + match[ 5 ] = +( ( match[ 7 ] + match[ 8 ] ) || match[ 3 ] === "odd" ); + + // other types prohibit arguments + } else if ( match[ 3 ] ) { + Sizzle.error( match[ 0 ] ); + } + + return match; + }, + + "PSEUDO": function( match ) { + var excess, + unquoted = !match[ 6 ] && match[ 2 ]; + + if ( matchExpr[ "CHILD" ].test( match[ 0 ] ) ) { + return null; + } + + // Accept quoted arguments as-is + if ( match[ 3 ] ) { + match[ 2 ] = match[ 4 ] || match[ 5 ] || ""; + + // Strip excess characters from unquoted arguments + } else if ( unquoted && rpseudo.test( unquoted ) && + + // Get excess from tokenize (recursively) + ( excess = tokenize( unquoted, true ) ) && + + // advance to the next closing parenthesis + ( excess = unquoted.indexOf( ")", unquoted.length - excess ) - unquoted.length ) ) { + + // excess is a negative index + match[ 0 ] = match[ 0 ].slice( 0, excess ); + match[ 2 ] = unquoted.slice( 0, excess ); + } + + // Return only captures needed by the pseudo filter method (type and argument) + return match.slice( 0, 3 ); + } + }, + + filter: { + + "TAG": function( nodeNameSelector ) { + var nodeName = nodeNameSelector.replace( runescape, funescape ).toLowerCase(); + return nodeNameSelector === "*" ? + function() { + return true; + } : + function( elem ) { + return elem.nodeName && elem.nodeName.toLowerCase() === nodeName; + }; + }, + + "CLASS": function( className ) { + var pattern = classCache[ className + " " ]; + + return pattern || + ( pattern = new RegExp( "(^|" + whitespace + + ")" + className + "(" + whitespace + "|$)" ) ) && classCache( + className, function( elem ) { + return pattern.test( + typeof elem.className === "string" && elem.className || + typeof elem.getAttribute !== "undefined" && + elem.getAttribute( "class" ) || + "" + ); + } ); + }, + + "ATTR": function( name, operator, check ) { + return function( elem ) { + var result = Sizzle.attr( elem, name ); + + if ( result == null ) { + return operator === "!="; + } + if ( !operator ) { + return true; + } + + result += ""; + + /* eslint-disable max-len */ + + return operator === "=" ? result === check : + operator === "!=" ? result !== check : + operator === "^=" ? check && result.indexOf( check ) === 0 : + operator === "*=" ? check && result.indexOf( check ) > -1 : + operator === "$=" ? check && result.slice( -check.length ) === check : + operator === "~=" ? ( " " + result.replace( rwhitespace, " " ) + " " ).indexOf( check ) > -1 : + operator === "|=" ? result === check || result.slice( 0, check.length + 1 ) === check + "-" : + false; + /* eslint-enable max-len */ + + }; + }, + + "CHILD": function( type, what, _argument, first, last ) { + var simple = type.slice( 0, 3 ) !== "nth", + forward = type.slice( -4 ) !== "last", + ofType = what === "of-type"; + + return first === 1 && last === 0 ? + + // Shortcut for :nth-*(n) + function( elem ) { + return !!elem.parentNode; + } : + + function( elem, _context, xml ) { + var cache, uniqueCache, outerCache, node, nodeIndex, start, + dir = simple !== forward ? "nextSibling" : "previousSibling", + parent = elem.parentNode, + name = ofType && elem.nodeName.toLowerCase(), + useCache = !xml && !ofType, + diff = false; + + if ( parent ) { + + // :(first|last|only)-(child|of-type) + if ( simple ) { + while ( dir ) { + node = elem; + while ( ( node = node[ dir ] ) ) { + if ( ofType ? + node.nodeName.toLowerCase() === name : + node.nodeType === 1 ) { + + return false; + } + } + + // Reverse direction for :only-* (if we haven't yet done so) + start = dir = type === "only" && !start && "nextSibling"; + } + return true; + } + + start = [ forward ? parent.firstChild : parent.lastChild ]; + + // non-xml :nth-child(...) stores cache data on `parent` + if ( forward && useCache ) { + + // Seek `elem` from a previously-cached index + + // ...in a gzip-friendly way + node = parent; + outerCache = node[ expando ] || ( node[ expando ] = {} ); + + // Support: IE <9 only + // Defend against cloned attroperties (jQuery gh-1709) + uniqueCache = outerCache[ node.uniqueID ] || + ( outerCache[ node.uniqueID ] = {} ); + + cache = uniqueCache[ type ] || []; + nodeIndex = cache[ 0 ] === dirruns && cache[ 1 ]; + diff = nodeIndex && cache[ 2 ]; + node = nodeIndex && parent.childNodes[ nodeIndex ]; + + while ( ( node = ++nodeIndex && node && node[ dir ] || + + // Fallback to seeking `elem` from the start + ( diff = nodeIndex = 0 ) || start.pop() ) ) { + + // When found, cache indexes on `parent` and break + if ( node.nodeType === 1 && ++diff && node === elem ) { + uniqueCache[ type ] = [ dirruns, nodeIndex, diff ]; + break; + } + } + + } else { + + // Use previously-cached element index if available + if ( useCache ) { + + // ...in a gzip-friendly way + node = elem; + outerCache = node[ expando ] || ( node[ expando ] = {} ); + + // Support: IE <9 only + // Defend against cloned attroperties (jQuery gh-1709) + uniqueCache = outerCache[ node.uniqueID ] || + ( outerCache[ node.uniqueID ] = {} ); + + cache = uniqueCache[ type ] || []; + nodeIndex = cache[ 0 ] === dirruns && cache[ 1 ]; + diff = nodeIndex; + } + + // xml :nth-child(...) + // or :nth-last-child(...) or :nth(-last)?-of-type(...) + if ( diff === false ) { + + // Use the same loop as above to seek `elem` from the start + while ( ( node = ++nodeIndex && node && node[ dir ] || + ( diff = nodeIndex = 0 ) || start.pop() ) ) { + + if ( ( ofType ? + node.nodeName.toLowerCase() === name : + node.nodeType === 1 ) && + ++diff ) { + + // Cache the index of each encountered element + if ( useCache ) { + outerCache = node[ expando ] || + ( node[ expando ] = {} ); + + // Support: IE <9 only + // Defend against cloned attroperties (jQuery gh-1709) + uniqueCache = outerCache[ node.uniqueID ] || + ( outerCache[ node.uniqueID ] = {} ); + + uniqueCache[ type ] = [ dirruns, diff ]; + } + + if ( node === elem ) { + break; + } + } + } + } + } + + // Incorporate the offset, then check against cycle size + diff -= last; + return diff === first || ( diff % first === 0 && diff / first >= 0 ); + } + }; + }, + + "PSEUDO": function( pseudo, argument ) { + + // pseudo-class names are case-insensitive + // http://www.w3.org/TR/selectors/#pseudo-classes + // Prioritize by case sensitivity in case custom pseudos are added with uppercase letters + // Remember that setFilters inherits from pseudos + var args, + fn = Expr.pseudos[ pseudo ] || Expr.setFilters[ pseudo.toLowerCase() ] || + Sizzle.error( "unsupported pseudo: " + pseudo ); + + // The user may use createPseudo to indicate that + // arguments are needed to create the filter function + // just as Sizzle does + if ( fn[ expando ] ) { + return fn( argument ); + } + + // But maintain support for old signatures + if ( fn.length > 1 ) { + args = [ pseudo, pseudo, "", argument ]; + return Expr.setFilters.hasOwnProperty( pseudo.toLowerCase() ) ? + markFunction( function( seed, matches ) { + var idx, + matched = fn( seed, argument ), + i = matched.length; + while ( i-- ) { + idx = indexOf( seed, matched[ i ] ); + seed[ idx ] = !( matches[ idx ] = matched[ i ] ); + } + } ) : + function( elem ) { + return fn( elem, 0, args ); + }; + } + + return fn; + } + }, + + pseudos: { + + // Potentially complex pseudos + "not": markFunction( function( selector ) { + + // Trim the selector passed to compile + // to avoid treating leading and trailing + // spaces as combinators + var input = [], + results = [], + matcher = compile( selector.replace( rtrim, "$1" ) ); + + return matcher[ expando ] ? + markFunction( function( seed, matches, _context, xml ) { + var elem, + unmatched = matcher( seed, null, xml, [] ), + i = seed.length; + + // Match elements unmatched by `matcher` + while ( i-- ) { + if ( ( elem = unmatched[ i ] ) ) { + seed[ i ] = !( matches[ i ] = elem ); + } + } + } ) : + function( elem, _context, xml ) { + input[ 0 ] = elem; + matcher( input, null, xml, results ); + + // Don't keep the element (issue #299) + input[ 0 ] = null; + return !results.pop(); + }; + } ), + + "has": markFunction( function( selector ) { + return function( elem ) { + return Sizzle( selector, elem ).length > 0; + }; + } ), + + "contains": markFunction( function( text ) { + text = text.replace( runescape, funescape ); + return function( elem ) { + return ( elem.textContent || getText( elem ) ).indexOf( text ) > -1; + }; + } ), + + // "Whether an element is represented by a :lang() selector + // is based solely on the element's language value + // being equal to the identifier C, + // or beginning with the identifier C immediately followed by "-". + // The matching of C against the element's language value is performed case-insensitively. + // The identifier C does not have to be a valid language name." + // http://www.w3.org/TR/selectors/#lang-pseudo + "lang": markFunction( function( lang ) { + + // lang value must be a valid identifier + if ( !ridentifier.test( lang || "" ) ) { + Sizzle.error( "unsupported lang: " + lang ); + } + lang = lang.replace( runescape, funescape ).toLowerCase(); + return function( elem ) { + var elemLang; + do { + if ( ( elemLang = documentIsHTML ? + elem.lang : + elem.getAttribute( "xml:lang" ) || elem.getAttribute( "lang" ) ) ) { + + elemLang = elemLang.toLowerCase(); + return elemLang === lang || elemLang.indexOf( lang + "-" ) === 0; + } + } while ( ( elem = elem.parentNode ) && elem.nodeType === 1 ); + return false; + }; + } ), + + // Miscellaneous + "target": function( elem ) { + var hash = window.location && window.location.hash; + return hash && hash.slice( 1 ) === elem.id; + }, + + "root": function( elem ) { + return elem === docElem; + }, + + "focus": function( elem ) { + return elem === document.activeElement && + ( !document.hasFocus || document.hasFocus() ) && + !!( elem.type || elem.href || ~elem.tabIndex ); + }, + + // Boolean properties + "enabled": createDisabledPseudo( false ), + "disabled": createDisabledPseudo( true ), + + "checked": function( elem ) { + + // In CSS3, :checked should return both checked and selected elements + // http://www.w3.org/TR/2011/REC-css3-selectors-20110929/#checked + var nodeName = elem.nodeName.toLowerCase(); + return ( nodeName === "input" && !!elem.checked ) || + ( nodeName === "option" && !!elem.selected ); + }, + + "selected": function( elem ) { + + // Accessing this property makes selected-by-default + // options in Safari work properly + if ( elem.parentNode ) { + // eslint-disable-next-line no-unused-expressions + elem.parentNode.selectedIndex; + } + + return elem.selected === true; + }, + + // Contents + "empty": function( elem ) { + + // http://www.w3.org/TR/selectors/#empty-pseudo + // :empty is negated by element (1) or content nodes (text: 3; cdata: 4; entity ref: 5), + // but not by others (comment: 8; processing instruction: 7; etc.) + // nodeType < 6 works because attributes (2) do not appear as children + for ( elem = elem.firstChild; elem; elem = elem.nextSibling ) { + if ( elem.nodeType < 6 ) { + return false; + } + } + return true; + }, + + "parent": function( elem ) { + return !Expr.pseudos[ "empty" ]( elem ); + }, + + // Element/input types + "header": function( elem ) { + return rheader.test( elem.nodeName ); + }, + + "input": function( elem ) { + return rinputs.test( elem.nodeName ); + }, + + "button": function( elem ) { + var name = elem.nodeName.toLowerCase(); + return name === "input" && elem.type === "button" || name === "button"; + }, + + "text": function( elem ) { + var attr; + return elem.nodeName.toLowerCase() === "input" && + elem.type === "text" && + + // Support: IE<8 + // New HTML5 attribute values (e.g., "search") appear with elem.type === "text" + ( ( attr = elem.getAttribute( "type" ) ) == null || + attr.toLowerCase() === "text" ); + }, + + // Position-in-collection + "first": createPositionalPseudo( function() { + return [ 0 ]; + } ), + + "last": createPositionalPseudo( function( _matchIndexes, length ) { + return [ length - 1 ]; + } ), + + "eq": createPositionalPseudo( function( _matchIndexes, length, argument ) { + return [ argument < 0 ? argument + length : argument ]; + } ), + + "even": createPositionalPseudo( function( matchIndexes, length ) { + var i = 0; + for ( ; i < length; i += 2 ) { + matchIndexes.push( i ); + } + return matchIndexes; + } ), + + "odd": createPositionalPseudo( function( matchIndexes, length ) { + var i = 1; + for ( ; i < length; i += 2 ) { + matchIndexes.push( i ); + } + return matchIndexes; + } ), + + "lt": createPositionalPseudo( function( matchIndexes, length, argument ) { + var i = argument < 0 ? + argument + length : + argument > length ? + length : + argument; + for ( ; --i >= 0; ) { + matchIndexes.push( i ); + } + return matchIndexes; + } ), + + "gt": createPositionalPseudo( function( matchIndexes, length, argument ) { + var i = argument < 0 ? argument + length : argument; + for ( ; ++i < length; ) { + matchIndexes.push( i ); + } + return matchIndexes; + } ) + } +}; + +Expr.pseudos[ "nth" ] = Expr.pseudos[ "eq" ]; + +// Add button/input type pseudos +for ( i in { radio: true, checkbox: true, file: true, password: true, image: true } ) { + Expr.pseudos[ i ] = createInputPseudo( i ); +} +for ( i in { submit: true, reset: true } ) { + Expr.pseudos[ i ] = createButtonPseudo( i ); +} + +// Easy API for creating new setFilters +function setFilters() {} +setFilters.prototype = Expr.filters = Expr.pseudos; +Expr.setFilters = new setFilters(); + +tokenize = Sizzle.tokenize = function( selector, parseOnly ) { + var matched, match, tokens, type, + soFar, groups, preFilters, + cached = tokenCache[ selector + " " ]; + + if ( cached ) { + return parseOnly ? 0 : cached.slice( 0 ); + } + + soFar = selector; + groups = []; + preFilters = Expr.preFilter; + + while ( soFar ) { + + // Comma and first run + if ( !matched || ( match = rcomma.exec( soFar ) ) ) { + if ( match ) { + + // Don't consume trailing commas as valid + soFar = soFar.slice( match[ 0 ].length ) || soFar; + } + groups.push( ( tokens = [] ) ); + } + + matched = false; + + // Combinators + if ( ( match = rcombinators.exec( soFar ) ) ) { + matched = match.shift(); + tokens.push( { + value: matched, + + // Cast descendant combinators to space + type: match[ 0 ].replace( rtrim, " " ) + } ); + soFar = soFar.slice( matched.length ); + } + + // Filters + for ( type in Expr.filter ) { + if ( ( match = matchExpr[ type ].exec( soFar ) ) && ( !preFilters[ type ] || + ( match = preFilters[ type ]( match ) ) ) ) { + matched = match.shift(); + tokens.push( { + value: matched, + type: type, + matches: match + } ); + soFar = soFar.slice( matched.length ); + } + } + + if ( !matched ) { + break; + } + } + + // Return the length of the invalid excess + // if we're just parsing + // Otherwise, throw an error or return tokens + return parseOnly ? + soFar.length : + soFar ? + Sizzle.error( selector ) : + + // Cache the tokens + tokenCache( selector, groups ).slice( 0 ); +}; + +function toSelector( tokens ) { + var i = 0, + len = tokens.length, + selector = ""; + for ( ; i < len; i++ ) { + selector += tokens[ i ].value; + } + return selector; +} + +function addCombinator( matcher, combinator, base ) { + var dir = combinator.dir, + skip = combinator.next, + key = skip || dir, + checkNonElements = base && key === "parentNode", + doneName = done++; + + return combinator.first ? + + // Check against closest ancestor/preceding element + function( elem, context, xml ) { + while ( ( elem = elem[ dir ] ) ) { + if ( elem.nodeType === 1 || checkNonElements ) { + return matcher( elem, context, xml ); + } + } + return false; + } : + + // Check against all ancestor/preceding elements + function( elem, context, xml ) { + var oldCache, uniqueCache, outerCache, + newCache = [ dirruns, doneName ]; + + // We can't set arbitrary data on XML nodes, so they don't benefit from combinator caching + if ( xml ) { + while ( ( elem = elem[ dir ] ) ) { + if ( elem.nodeType === 1 || checkNonElements ) { + if ( matcher( elem, context, xml ) ) { + return true; + } + } + } + } else { + while ( ( elem = elem[ dir ] ) ) { + if ( elem.nodeType === 1 || checkNonElements ) { + outerCache = elem[ expando ] || ( elem[ expando ] = {} ); + + // Support: IE <9 only + // Defend against cloned attroperties (jQuery gh-1709) + uniqueCache = outerCache[ elem.uniqueID ] || + ( outerCache[ elem.uniqueID ] = {} ); + + if ( skip && skip === elem.nodeName.toLowerCase() ) { + elem = elem[ dir ] || elem; + } else if ( ( oldCache = uniqueCache[ key ] ) && + oldCache[ 0 ] === dirruns && oldCache[ 1 ] === doneName ) { + + // Assign to newCache so results back-propagate to previous elements + return ( newCache[ 2 ] = oldCache[ 2 ] ); + } else { + + // Reuse newcache so results back-propagate to previous elements + uniqueCache[ key ] = newCache; + + // A match means we're done; a fail means we have to keep checking + if ( ( newCache[ 2 ] = matcher( elem, context, xml ) ) ) { + return true; + } + } + } + } + } + return false; + }; +} + +function elementMatcher( matchers ) { + return matchers.length > 1 ? + function( elem, context, xml ) { + var i = matchers.length; + while ( i-- ) { + if ( !matchers[ i ]( elem, context, xml ) ) { + return false; + } + } + return true; + } : + matchers[ 0 ]; +} + +function multipleContexts( selector, contexts, results ) { + var i = 0, + len = contexts.length; + for ( ; i < len; i++ ) { + Sizzle( selector, contexts[ i ], results ); + } + return results; +} + +function condense( unmatched, map, filter, context, xml ) { + var elem, + newUnmatched = [], + i = 0, + len = unmatched.length, + mapped = map != null; + + for ( ; i < len; i++ ) { + if ( ( elem = unmatched[ i ] ) ) { + if ( !filter || filter( elem, context, xml ) ) { + newUnmatched.push( elem ); + if ( mapped ) { + map.push( i ); + } + } + } + } + + return newUnmatched; +} + +function setMatcher( preFilter, selector, matcher, postFilter, postFinder, postSelector ) { + if ( postFilter && !postFilter[ expando ] ) { + postFilter = setMatcher( postFilter ); + } + if ( postFinder && !postFinder[ expando ] ) { + postFinder = setMatcher( postFinder, postSelector ); + } + return markFunction( function( seed, results, context, xml ) { + var temp, i, elem, + preMap = [], + postMap = [], + preexisting = results.length, + + // Get initial elements from seed or context + elems = seed || multipleContexts( + selector || "*", + context.nodeType ? [ context ] : context, + [] + ), + + // Prefilter to get matcher input, preserving a map for seed-results synchronization + matcherIn = preFilter && ( seed || !selector ) ? + condense( elems, preMap, preFilter, context, xml ) : + elems, + + matcherOut = matcher ? + + // If we have a postFinder, or filtered seed, or non-seed postFilter or preexisting results, + postFinder || ( seed ? preFilter : preexisting || postFilter ) ? + + // ...intermediate processing is necessary + [] : + + // ...otherwise use results directly + results : + matcherIn; + + // Find primary matches + if ( matcher ) { + matcher( matcherIn, matcherOut, context, xml ); + } + + // Apply postFilter + if ( postFilter ) { + temp = condense( matcherOut, postMap ); + postFilter( temp, [], context, xml ); + + // Un-match failing elements by moving them back to matcherIn + i = temp.length; + while ( i-- ) { + if ( ( elem = temp[ i ] ) ) { + matcherOut[ postMap[ i ] ] = !( matcherIn[ postMap[ i ] ] = elem ); + } + } + } + + if ( seed ) { + if ( postFinder || preFilter ) { + if ( postFinder ) { + + // Get the final matcherOut by condensing this intermediate into postFinder contexts + temp = []; + i = matcherOut.length; + while ( i-- ) { + if ( ( elem = matcherOut[ i ] ) ) { + + // Restore matcherIn since elem is not yet a final match + temp.push( ( matcherIn[ i ] = elem ) ); + } + } + postFinder( null, ( matcherOut = [] ), temp, xml ); + } + + // Move matched elements from seed to results to keep them synchronized + i = matcherOut.length; + while ( i-- ) { + if ( ( elem = matcherOut[ i ] ) && + ( temp = postFinder ? indexOf( seed, elem ) : preMap[ i ] ) > -1 ) { + + seed[ temp ] = !( results[ temp ] = elem ); + } + } + } + + // Add elements to results, through postFinder if defined + } else { + matcherOut = condense( + matcherOut === results ? + matcherOut.splice( preexisting, matcherOut.length ) : + matcherOut + ); + if ( postFinder ) { + postFinder( null, results, matcherOut, xml ); + } else { + push.apply( results, matcherOut ); + } + } + } ); +} + +function matcherFromTokens( tokens ) { + var checkContext, matcher, j, + len = tokens.length, + leadingRelative = Expr.relative[ tokens[ 0 ].type ], + implicitRelative = leadingRelative || Expr.relative[ " " ], + i = leadingRelative ? 1 : 0, + + // The foundational matcher ensures that elements are reachable from top-level context(s) + matchContext = addCombinator( function( elem ) { + return elem === checkContext; + }, implicitRelative, true ), + matchAnyContext = addCombinator( function( elem ) { + return indexOf( checkContext, elem ) > -1; + }, implicitRelative, true ), + matchers = [ function( elem, context, xml ) { + var ret = ( !leadingRelative && ( xml || context !== outermostContext ) ) || ( + ( checkContext = context ).nodeType ? + matchContext( elem, context, xml ) : + matchAnyContext( elem, context, xml ) ); + + // Avoid hanging onto element (issue #299) + checkContext = null; + return ret; + } ]; + + for ( ; i < len; i++ ) { + if ( ( matcher = Expr.relative[ tokens[ i ].type ] ) ) { + matchers = [ addCombinator( elementMatcher( matchers ), matcher ) ]; + } else { + matcher = Expr.filter[ tokens[ i ].type ].apply( null, tokens[ i ].matches ); + + // Return special upon seeing a positional matcher + if ( matcher[ expando ] ) { + + // Find the next relative operator (if any) for proper handling + j = ++i; + for ( ; j < len; j++ ) { + if ( Expr.relative[ tokens[ j ].type ] ) { + break; + } + } + return setMatcher( + i > 1 && elementMatcher( matchers ), + i > 1 && toSelector( + + // If the preceding token was a descendant combinator, insert an implicit any-element `*` + tokens + .slice( 0, i - 1 ) + .concat( { value: tokens[ i - 2 ].type === " " ? "*" : "" } ) + ).replace( rtrim, "$1" ), + matcher, + i < j && matcherFromTokens( tokens.slice( i, j ) ), + j < len && matcherFromTokens( ( tokens = tokens.slice( j ) ) ), + j < len && toSelector( tokens ) + ); + } + matchers.push( matcher ); + } + } + + return elementMatcher( matchers ); +} + +function matcherFromGroupMatchers( elementMatchers, setMatchers ) { + var bySet = setMatchers.length > 0, + byElement = elementMatchers.length > 0, + superMatcher = function( seed, context, xml, results, outermost ) { + var elem, j, matcher, + matchedCount = 0, + i = "0", + unmatched = seed && [], + setMatched = [], + contextBackup = outermostContext, + + // We must always have either seed elements or outermost context + elems = seed || byElement && Expr.find[ "TAG" ]( "*", outermost ), + + // Use integer dirruns iff this is the outermost matcher + dirrunsUnique = ( dirruns += contextBackup == null ? 1 : Math.random() || 0.1 ), + len = elems.length; + + if ( outermost ) { + + // Support: IE 11+, Edge 17 - 18+ + // IE/Edge sometimes throw a "Permission denied" error when strict-comparing + // two documents; shallow comparisons work. + // eslint-disable-next-line eqeqeq + outermostContext = context == document || context || outermost; + } + + // Add elements passing elementMatchers directly to results + // Support: IE<9, Safari + // Tolerate NodeList properties (IE: "length"; Safari: ) matching elements by id + for ( ; i !== len && ( elem = elems[ i ] ) != null; i++ ) { + if ( byElement && elem ) { + j = 0; + + // Support: IE 11+, Edge 17 - 18+ + // IE/Edge sometimes throw a "Permission denied" error when strict-comparing + // two documents; shallow comparisons work. + // eslint-disable-next-line eqeqeq + if ( !context && elem.ownerDocument != document ) { + setDocument( elem ); + xml = !documentIsHTML; + } + while ( ( matcher = elementMatchers[ j++ ] ) ) { + if ( matcher( elem, context || document, xml ) ) { + results.push( elem ); + break; + } + } + if ( outermost ) { + dirruns = dirrunsUnique; + } + } + + // Track unmatched elements for set filters + if ( bySet ) { + + // They will have gone through all possible matchers + if ( ( elem = !matcher && elem ) ) { + matchedCount--; + } + + // Lengthen the array for every element, matched or not + if ( seed ) { + unmatched.push( elem ); + } + } + } + + // `i` is now the count of elements visited above, and adding it to `matchedCount` + // makes the latter nonnegative. + matchedCount += i; + + // Apply set filters to unmatched elements + // NOTE: This can be skipped if there are no unmatched elements (i.e., `matchedCount` + // equals `i`), unless we didn't visit _any_ elements in the above loop because we have + // no element matchers and no seed. + // Incrementing an initially-string "0" `i` allows `i` to remain a string only in that + // case, which will result in a "00" `matchedCount` that differs from `i` but is also + // numerically zero. + if ( bySet && i !== matchedCount ) { + j = 0; + while ( ( matcher = setMatchers[ j++ ] ) ) { + matcher( unmatched, setMatched, context, xml ); + } + + if ( seed ) { + + // Reintegrate element matches to eliminate the need for sorting + if ( matchedCount > 0 ) { + while ( i-- ) { + if ( !( unmatched[ i ] || setMatched[ i ] ) ) { + setMatched[ i ] = pop.call( results ); + } + } + } + + // Discard index placeholder values to get only actual matches + setMatched = condense( setMatched ); + } + + // Add matches to results + push.apply( results, setMatched ); + + // Seedless set matches succeeding multiple successful matchers stipulate sorting + if ( outermost && !seed && setMatched.length > 0 && + ( matchedCount + setMatchers.length ) > 1 ) { + + Sizzle.uniqueSort( results ); + } + } + + // Override manipulation of globals by nested matchers + if ( outermost ) { + dirruns = dirrunsUnique; + outermostContext = contextBackup; + } + + return unmatched; + }; + + return bySet ? + markFunction( superMatcher ) : + superMatcher; +} + +compile = Sizzle.compile = function( selector, match /* Internal Use Only */ ) { + var i, + setMatchers = [], + elementMatchers = [], + cached = compilerCache[ selector + " " ]; + + if ( !cached ) { + + // Generate a function of recursive functions that can be used to check each element + if ( !match ) { + match = tokenize( selector ); + } + i = match.length; + while ( i-- ) { + cached = matcherFromTokens( match[ i ] ); + if ( cached[ expando ] ) { + setMatchers.push( cached ); + } else { + elementMatchers.push( cached ); + } + } + + // Cache the compiled function + cached = compilerCache( + selector, + matcherFromGroupMatchers( elementMatchers, setMatchers ) + ); + + // Save selector and tokenization + cached.selector = selector; + } + return cached; +}; + +/** + * A low-level selection function that works with Sizzle's compiled + * selector functions + * @param {String|Function} selector A selector or a pre-compiled + * selector function built with Sizzle.compile + * @param {Element} context + * @param {Array} [results] + * @param {Array} [seed] A set of elements to match against + */ +select = Sizzle.select = function( selector, context, results, seed ) { + var i, tokens, token, type, find, + compiled = typeof selector === "function" && selector, + match = !seed && tokenize( ( selector = compiled.selector || selector ) ); + + results = results || []; + + // Try to minimize operations if there is only one selector in the list and no seed + // (the latter of which guarantees us context) + if ( match.length === 1 ) { + + // Reduce context if the leading compound selector is an ID + tokens = match[ 0 ] = match[ 0 ].slice( 0 ); + if ( tokens.length > 2 && ( token = tokens[ 0 ] ).type === "ID" && + context.nodeType === 9 && documentIsHTML && Expr.relative[ tokens[ 1 ].type ] ) { + + context = ( Expr.find[ "ID" ]( token.matches[ 0 ] + .replace( runescape, funescape ), context ) || [] )[ 0 ]; + if ( !context ) { + return results; + + // Precompiled matchers will still verify ancestry, so step up a level + } else if ( compiled ) { + context = context.parentNode; + } + + selector = selector.slice( tokens.shift().value.length ); + } + + // Fetch a seed set for right-to-left matching + i = matchExpr[ "needsContext" ].test( selector ) ? 0 : tokens.length; + while ( i-- ) { + token = tokens[ i ]; + + // Abort if we hit a combinator + if ( Expr.relative[ ( type = token.type ) ] ) { + break; + } + if ( ( find = Expr.find[ type ] ) ) { + + // Search, expanding context for leading sibling combinators + if ( ( seed = find( + token.matches[ 0 ].replace( runescape, funescape ), + rsibling.test( tokens[ 0 ].type ) && testContext( context.parentNode ) || + context + ) ) ) { + + // If seed is empty or no tokens remain, we can return early + tokens.splice( i, 1 ); + selector = seed.length && toSelector( tokens ); + if ( !selector ) { + push.apply( results, seed ); + return results; + } + + break; + } + } + } + } + + // Compile and execute a filtering function if one is not provided + // Provide `match` to avoid retokenization if we modified the selector above + ( compiled || compile( selector, match ) )( + seed, + context, + !documentIsHTML, + results, + !context || rsibling.test( selector ) && testContext( context.parentNode ) || context + ); + return results; +}; + +// One-time assignments + +// Sort stability +support.sortStable = expando.split( "" ).sort( sortOrder ).join( "" ) === expando; + +// Support: Chrome 14-35+ +// Always assume duplicates if they aren't passed to the comparison function +support.detectDuplicates = !!hasDuplicate; + +// Initialize against the default document +setDocument(); + +// Support: Webkit<537.32 - Safari 6.0.3/Chrome 25 (fixed in Chrome 27) +// Detached nodes confoundingly follow *each other* +support.sortDetached = assert( function( el ) { + + // Should return 1, but returns 4 (following) + return el.compareDocumentPosition( document.createElement( "fieldset" ) ) & 1; +} ); + +// Support: IE<8 +// Prevent attribute/property "interpolation" +// https://msdn.microsoft.com/en-us/library/ms536429%28VS.85%29.aspx +if ( !assert( function( el ) { + el.innerHTML = ""; + return el.firstChild.getAttribute( "href" ) === "#"; +} ) ) { + addHandle( "type|href|height|width", function( elem, name, isXML ) { + if ( !isXML ) { + return elem.getAttribute( name, name.toLowerCase() === "type" ? 1 : 2 ); + } + } ); +} + +// Support: IE<9 +// Use defaultValue in place of getAttribute("value") +if ( !support.attributes || !assert( function( el ) { + el.innerHTML = ""; + el.firstChild.setAttribute( "value", "" ); + return el.firstChild.getAttribute( "value" ) === ""; +} ) ) { + addHandle( "value", function( elem, _name, isXML ) { + if ( !isXML && elem.nodeName.toLowerCase() === "input" ) { + return elem.defaultValue; + } + } ); +} + +// Support: IE<9 +// Use getAttributeNode to fetch booleans when getAttribute lies +if ( !assert( function( el ) { + return el.getAttribute( "disabled" ) == null; +} ) ) { + addHandle( booleans, function( elem, name, isXML ) { + var val; + if ( !isXML ) { + return elem[ name ] === true ? name.toLowerCase() : + ( val = elem.getAttributeNode( name ) ) && val.specified ? + val.value : + null; + } + } ); +} + +return Sizzle; + +} )( window ); + + + +jQuery.find = Sizzle; +jQuery.expr = Sizzle.selectors; + +// Deprecated +jQuery.expr[ ":" ] = jQuery.expr.pseudos; +jQuery.uniqueSort = jQuery.unique = Sizzle.uniqueSort; +jQuery.text = Sizzle.getText; +jQuery.isXMLDoc = Sizzle.isXML; +jQuery.contains = Sizzle.contains; +jQuery.escapeSelector = Sizzle.escape; + + + + +var dir = function( elem, dir, until ) { + var matched = [], + truncate = until !== undefined; + + while ( ( elem = elem[ dir ] ) && elem.nodeType !== 9 ) { + if ( elem.nodeType === 1 ) { + if ( truncate && jQuery( elem ).is( until ) ) { + break; + } + matched.push( elem ); + } + } + return matched; +}; + + +var siblings = function( n, elem ) { + var matched = []; + + for ( ; n; n = n.nextSibling ) { + if ( n.nodeType === 1 && n !== elem ) { + matched.push( n ); + } + } + + return matched; +}; + + +var rneedsContext = jQuery.expr.match.needsContext; + + + +function nodeName( elem, name ) { + + return elem.nodeName && elem.nodeName.toLowerCase() === name.toLowerCase(); + +} +var rsingleTag = ( /^<([a-z][^\/\0>:\x20\t\r\n\f]*)[\x20\t\r\n\f]*\/?>(?:<\/\1>|)$/i ); + + + +// Implement the identical functionality for filter and not +function winnow( elements, qualifier, not ) { + if ( isFunction( qualifier ) ) { + return jQuery.grep( elements, function( elem, i ) { + return !!qualifier.call( elem, i, elem ) !== not; + } ); + } + + // Single element + if ( qualifier.nodeType ) { + return jQuery.grep( elements, function( elem ) { + return ( elem === qualifier ) !== not; + } ); + } + + // Arraylike of elements (jQuery, arguments, Array) + if ( typeof qualifier !== "string" ) { + return jQuery.grep( elements, function( elem ) { + return ( indexOf.call( qualifier, elem ) > -1 ) !== not; + } ); + } + + // Filtered directly for both simple and complex selectors + return jQuery.filter( qualifier, elements, not ); +} + +jQuery.filter = function( expr, elems, not ) { + var elem = elems[ 0 ]; + + if ( not ) { + expr = ":not(" + expr + ")"; + } + + if ( elems.length === 1 && elem.nodeType === 1 ) { + return jQuery.find.matchesSelector( elem, expr ) ? [ elem ] : []; + } + + return jQuery.find.matches( expr, jQuery.grep( elems, function( elem ) { + return elem.nodeType === 1; + } ) ); +}; + +jQuery.fn.extend( { + find: function( selector ) { + var i, ret, + len = this.length, + self = this; + + if ( typeof selector !== "string" ) { + return this.pushStack( jQuery( selector ).filter( function() { + for ( i = 0; i < len; i++ ) { + if ( jQuery.contains( self[ i ], this ) ) { + return true; + } + } + } ) ); + } + + ret = this.pushStack( [] ); + + for ( i = 0; i < len; i++ ) { + jQuery.find( selector, self[ i ], ret ); + } + + return len > 1 ? jQuery.uniqueSort( ret ) : ret; + }, + filter: function( selector ) { + return this.pushStack( winnow( this, selector || [], false ) ); + }, + not: function( selector ) { + return this.pushStack( winnow( this, selector || [], true ) ); + }, + is: function( selector ) { + return !!winnow( + this, + + // If this is a positional/relative selector, check membership in the returned set + // so $("p:first").is("p:last") won't return true for a doc with two "p". + typeof selector === "string" && rneedsContext.test( selector ) ? + jQuery( selector ) : + selector || [], + false + ).length; + } +} ); + + +// Initialize a jQuery object + + +// A central reference to the root jQuery(document) +var rootjQuery, + + // A simple way to check for HTML strings + // Prioritize #id over to avoid XSS via location.hash (#9521) + // Strict HTML recognition (#11290: must start with <) + // Shortcut simple #id case for speed + rquickExpr = /^(?:\s*(<[\w\W]+>)[^>]*|#([\w-]+))$/, + + init = jQuery.fn.init = function( selector, context, root ) { + var match, elem; + + // HANDLE: $(""), $(null), $(undefined), $(false) + if ( !selector ) { + return this; + } + + // Method init() accepts an alternate rootjQuery + // so migrate can support jQuery.sub (gh-2101) + root = root || rootjQuery; + + // Handle HTML strings + if ( typeof selector === "string" ) { + if ( selector[ 0 ] === "<" && + selector[ selector.length - 1 ] === ">" && + selector.length >= 3 ) { + + // Assume that strings that start and end with <> are HTML and skip the regex check + match = [ null, selector, null ]; + + } else { + match = rquickExpr.exec( selector ); + } + + // Match html or make sure no context is specified for #id + if ( match && ( match[ 1 ] || !context ) ) { + + // HANDLE: $(html) -> $(array) + if ( match[ 1 ] ) { + context = context instanceof jQuery ? context[ 0 ] : context; + + // Option to run scripts is true for back-compat + // Intentionally let the error be thrown if parseHTML is not present + jQuery.merge( this, jQuery.parseHTML( + match[ 1 ], + context && context.nodeType ? context.ownerDocument || context : document, + true + ) ); + + // HANDLE: $(html, props) + if ( rsingleTag.test( match[ 1 ] ) && jQuery.isPlainObject( context ) ) { + for ( match in context ) { + + // Properties of context are called as methods if possible + if ( isFunction( this[ match ] ) ) { + this[ match ]( context[ match ] ); + + // ...and otherwise set as attributes + } else { + this.attr( match, context[ match ] ); + } + } + } + + return this; + + // HANDLE: $(#id) + } else { + elem = document.getElementById( match[ 2 ] ); + + if ( elem ) { + + // Inject the element directly into the jQuery object + this[ 0 ] = elem; + this.length = 1; + } + return this; + } + + // HANDLE: $(expr, $(...)) + } else if ( !context || context.jquery ) { + return ( context || root ).find( selector ); + + // HANDLE: $(expr, context) + // (which is just equivalent to: $(context).find(expr) + } else { + return this.constructor( context ).find( selector ); + } + + // HANDLE: $(DOMElement) + } else if ( selector.nodeType ) { + this[ 0 ] = selector; + this.length = 1; + return this; + + // HANDLE: $(function) + // Shortcut for document ready + } else if ( isFunction( selector ) ) { + return root.ready !== undefined ? + root.ready( selector ) : + + // Execute immediately if ready is not present + selector( jQuery ); + } + + return jQuery.makeArray( selector, this ); + }; + +// Give the init function the jQuery prototype for later instantiation +init.prototype = jQuery.fn; + +// Initialize central reference +rootjQuery = jQuery( document ); + + +var rparentsprev = /^(?:parents|prev(?:Until|All))/, + + // Methods guaranteed to produce a unique set when starting from a unique set + guaranteedUnique = { + children: true, + contents: true, + next: true, + prev: true + }; + +jQuery.fn.extend( { + has: function( target ) { + var targets = jQuery( target, this ), + l = targets.length; + + return this.filter( function() { + var i = 0; + for ( ; i < l; i++ ) { + if ( jQuery.contains( this, targets[ i ] ) ) { + return true; + } + } + } ); + }, + + closest: function( selectors, context ) { + var cur, + i = 0, + l = this.length, + matched = [], + targets = typeof selectors !== "string" && jQuery( selectors ); + + // Positional selectors never match, since there's no _selection_ context + if ( !rneedsContext.test( selectors ) ) { + for ( ; i < l; i++ ) { + for ( cur = this[ i ]; cur && cur !== context; cur = cur.parentNode ) { + + // Always skip document fragments + if ( cur.nodeType < 11 && ( targets ? + targets.index( cur ) > -1 : + + // Don't pass non-elements to Sizzle + cur.nodeType === 1 && + jQuery.find.matchesSelector( cur, selectors ) ) ) { + + matched.push( cur ); + break; + } + } + } + } + + return this.pushStack( matched.length > 1 ? jQuery.uniqueSort( matched ) : matched ); + }, + + // Determine the position of an element within the set + index: function( elem ) { + + // No argument, return index in parent + if ( !elem ) { + return ( this[ 0 ] && this[ 0 ].parentNode ) ? this.first().prevAll().length : -1; + } + + // Index in selector + if ( typeof elem === "string" ) { + return indexOf.call( jQuery( elem ), this[ 0 ] ); + } + + // Locate the position of the desired element + return indexOf.call( this, + + // If it receives a jQuery object, the first element is used + elem.jquery ? elem[ 0 ] : elem + ); + }, + + add: function( selector, context ) { + return this.pushStack( + jQuery.uniqueSort( + jQuery.merge( this.get(), jQuery( selector, context ) ) + ) + ); + }, + + addBack: function( selector ) { + return this.add( selector == null ? + this.prevObject : this.prevObject.filter( selector ) + ); + } +} ); + +function sibling( cur, dir ) { + while ( ( cur = cur[ dir ] ) && cur.nodeType !== 1 ) {} + return cur; +} + +jQuery.each( { + parent: function( elem ) { + var parent = elem.parentNode; + return parent && parent.nodeType !== 11 ? parent : null; + }, + parents: function( elem ) { + return dir( elem, "parentNode" ); + }, + parentsUntil: function( elem, _i, until ) { + return dir( elem, "parentNode", until ); + }, + next: function( elem ) { + return sibling( elem, "nextSibling" ); + }, + prev: function( elem ) { + return sibling( elem, "previousSibling" ); + }, + nextAll: function( elem ) { + return dir( elem, "nextSibling" ); + }, + prevAll: function( elem ) { + return dir( elem, "previousSibling" ); + }, + nextUntil: function( elem, _i, until ) { + return dir( elem, "nextSibling", until ); + }, + prevUntil: function( elem, _i, until ) { + return dir( elem, "previousSibling", until ); + }, + siblings: function( elem ) { + return siblings( ( elem.parentNode || {} ).firstChild, elem ); + }, + children: function( elem ) { + return siblings( elem.firstChild ); + }, + contents: function( elem ) { + if ( elem.contentDocument != null && + + // Support: IE 11+ + // elements with no `data` attribute has an object + // `contentDocument` with a `null` prototype. + getProto( elem.contentDocument ) ) { + + return elem.contentDocument; + } + + // Support: IE 9 - 11 only, iOS 7 only, Android Browser <=4.3 only + // Treat the template element as a regular one in browsers that + // don't support it. + if ( nodeName( elem, "template" ) ) { + elem = elem.content || elem; + } + + return jQuery.merge( [], elem.childNodes ); + } +}, function( name, fn ) { + jQuery.fn[ name ] = function( until, selector ) { + var matched = jQuery.map( this, fn, until ); + + if ( name.slice( -5 ) !== "Until" ) { + selector = until; + } + + if ( selector && typeof selector === "string" ) { + matched = jQuery.filter( selector, matched ); + } + + if ( this.length > 1 ) { + + // Remove duplicates + if ( !guaranteedUnique[ name ] ) { + jQuery.uniqueSort( matched ); + } + + // Reverse order for parents* and prev-derivatives + if ( rparentsprev.test( name ) ) { + matched.reverse(); + } + } + + return this.pushStack( matched ); + }; +} ); +var rnothtmlwhite = ( /[^\x20\t\r\n\f]+/g ); + + + +// Convert String-formatted options into Object-formatted ones +function createOptions( options ) { + var object = {}; + jQuery.each( options.match( rnothtmlwhite ) || [], function( _, flag ) { + object[ flag ] = true; + } ); + return object; +} + +/* + * Create a callback list using the following parameters: + * + * options: an optional list of space-separated options that will change how + * the callback list behaves or a more traditional option object + * + * By default a callback list will act like an event callback list and can be + * "fired" multiple times. + * + * Possible options: + * + * once: will ensure the callback list can only be fired once (like a Deferred) + * + * memory: will keep track of previous values and will call any callback added + * after the list has been fired right away with the latest "memorized" + * values (like a Deferred) + * + * unique: will ensure a callback can only be added once (no duplicate in the list) + * + * stopOnFalse: interrupt callings when a callback returns false + * + */ +jQuery.Callbacks = function( options ) { + + // Convert options from String-formatted to Object-formatted if needed + // (we check in cache first) + options = typeof options === "string" ? + createOptions( options ) : + jQuery.extend( {}, options ); + + var // Flag to know if list is currently firing + firing, + + // Last fire value for non-forgettable lists + memory, + + // Flag to know if list was already fired + fired, + + // Flag to prevent firing + locked, + + // Actual callback list + list = [], + + // Queue of execution data for repeatable lists + queue = [], + + // Index of currently firing callback (modified by add/remove as needed) + firingIndex = -1, + + // Fire callbacks + fire = function() { + + // Enforce single-firing + locked = locked || options.once; + + // Execute callbacks for all pending executions, + // respecting firingIndex overrides and runtime changes + fired = firing = true; + for ( ; queue.length; firingIndex = -1 ) { + memory = queue.shift(); + while ( ++firingIndex < list.length ) { + + // Run callback and check for early termination + if ( list[ firingIndex ].apply( memory[ 0 ], memory[ 1 ] ) === false && + options.stopOnFalse ) { + + // Jump to end and forget the data so .add doesn't re-fire + firingIndex = list.length; + memory = false; + } + } + } + + // Forget the data if we're done with it + if ( !options.memory ) { + memory = false; + } + + firing = false; + + // Clean up if we're done firing for good + if ( locked ) { + + // Keep an empty list if we have data for future add calls + if ( memory ) { + list = []; + + // Otherwise, this object is spent + } else { + list = ""; + } + } + }, + + // Actual Callbacks object + self = { + + // Add a callback or a collection of callbacks to the list + add: function() { + if ( list ) { + + // If we have memory from a past run, we should fire after adding + if ( memory && !firing ) { + firingIndex = list.length - 1; + queue.push( memory ); + } + + ( function add( args ) { + jQuery.each( args, function( _, arg ) { + if ( isFunction( arg ) ) { + if ( !options.unique || !self.has( arg ) ) { + list.push( arg ); + } + } else if ( arg && arg.length && toType( arg ) !== "string" ) { + + // Inspect recursively + add( arg ); + } + } ); + } )( arguments ); + + if ( memory && !firing ) { + fire(); + } + } + return this; + }, + + // Remove a callback from the list + remove: function() { + jQuery.each( arguments, function( _, arg ) { + var index; + while ( ( index = jQuery.inArray( arg, list, index ) ) > -1 ) { + list.splice( index, 1 ); + + // Handle firing indexes + if ( index <= firingIndex ) { + firingIndex--; + } + } + } ); + return this; + }, + + // Check if a given callback is in the list. + // If no argument is given, return whether or not list has callbacks attached. + has: function( fn ) { + return fn ? + jQuery.inArray( fn, list ) > -1 : + list.length > 0; + }, + + // Remove all callbacks from the list + empty: function() { + if ( list ) { + list = []; + } + return this; + }, + + // Disable .fire and .add + // Abort any current/pending executions + // Clear all callbacks and values + disable: function() { + locked = queue = []; + list = memory = ""; + return this; + }, + disabled: function() { + return !list; + }, + + // Disable .fire + // Also disable .add unless we have memory (since it would have no effect) + // Abort any pending executions + lock: function() { + locked = queue = []; + if ( !memory && !firing ) { + list = memory = ""; + } + return this; + }, + locked: function() { + return !!locked; + }, + + // Call all callbacks with the given context and arguments + fireWith: function( context, args ) { + if ( !locked ) { + args = args || []; + args = [ context, args.slice ? args.slice() : args ]; + queue.push( args ); + if ( !firing ) { + fire(); + } + } + return this; + }, + + // Call all the callbacks with the given arguments + fire: function() { + self.fireWith( this, arguments ); + return this; + }, + + // To know if the callbacks have already been called at least once + fired: function() { + return !!fired; + } + }; + + return self; +}; + + +function Identity( v ) { + return v; +} +function Thrower( ex ) { + throw ex; +} + +function adoptValue( value, resolve, reject, noValue ) { + var method; + + try { + + // Check for promise aspect first to privilege synchronous behavior + if ( value && isFunction( ( method = value.promise ) ) ) { + method.call( value ).done( resolve ).fail( reject ); + + // Other thenables + } else if ( value && isFunction( ( method = value.then ) ) ) { + method.call( value, resolve, reject ); + + // Other non-thenables + } else { + + // Control `resolve` arguments by letting Array#slice cast boolean `noValue` to integer: + // * false: [ value ].slice( 0 ) => resolve( value ) + // * true: [ value ].slice( 1 ) => resolve() + resolve.apply( undefined, [ value ].slice( noValue ) ); + } + + // For Promises/A+, convert exceptions into rejections + // Since jQuery.when doesn't unwrap thenables, we can skip the extra checks appearing in + // Deferred#then to conditionally suppress rejection. + } catch ( value ) { + + // Support: Android 4.0 only + // Strict mode functions invoked without .call/.apply get global-object context + reject.apply( undefined, [ value ] ); + } +} + +jQuery.extend( { + + Deferred: function( func ) { + var tuples = [ + + // action, add listener, callbacks, + // ... .then handlers, argument index, [final state] + [ "notify", "progress", jQuery.Callbacks( "memory" ), + jQuery.Callbacks( "memory" ), 2 ], + [ "resolve", "done", jQuery.Callbacks( "once memory" ), + jQuery.Callbacks( "once memory" ), 0, "resolved" ], + [ "reject", "fail", jQuery.Callbacks( "once memory" ), + jQuery.Callbacks( "once memory" ), 1, "rejected" ] + ], + state = "pending", + promise = { + state: function() { + return state; + }, + always: function() { + deferred.done( arguments ).fail( arguments ); + return this; + }, + "catch": function( fn ) { + return promise.then( null, fn ); + }, + + // Keep pipe for back-compat + pipe: function( /* fnDone, fnFail, fnProgress */ ) { + var fns = arguments; + + return jQuery.Deferred( function( newDefer ) { + jQuery.each( tuples, function( _i, tuple ) { + + // Map tuples (progress, done, fail) to arguments (done, fail, progress) + var fn = isFunction( fns[ tuple[ 4 ] ] ) && fns[ tuple[ 4 ] ]; + + // deferred.progress(function() { bind to newDefer or newDefer.notify }) + // deferred.done(function() { bind to newDefer or newDefer.resolve }) + // deferred.fail(function() { bind to newDefer or newDefer.reject }) + deferred[ tuple[ 1 ] ]( function() { + var returned = fn && fn.apply( this, arguments ); + if ( returned && isFunction( returned.promise ) ) { + returned.promise() + .progress( newDefer.notify ) + .done( newDefer.resolve ) + .fail( newDefer.reject ); + } else { + newDefer[ tuple[ 0 ] + "With" ]( + this, + fn ? [ returned ] : arguments + ); + } + } ); + } ); + fns = null; + } ).promise(); + }, + then: function( onFulfilled, onRejected, onProgress ) { + var maxDepth = 0; + function resolve( depth, deferred, handler, special ) { + return function() { + var that = this, + args = arguments, + mightThrow = function() { + var returned, then; + + // Support: Promises/A+ section 2.3.3.3.3 + // https://promisesaplus.com/#point-59 + // Ignore double-resolution attempts + if ( depth < maxDepth ) { + return; + } + + returned = handler.apply( that, args ); + + // Support: Promises/A+ section 2.3.1 + // https://promisesaplus.com/#point-48 + if ( returned === deferred.promise() ) { + throw new TypeError( "Thenable self-resolution" ); + } + + // Support: Promises/A+ sections 2.3.3.1, 3.5 + // https://promisesaplus.com/#point-54 + // https://promisesaplus.com/#point-75 + // Retrieve `then` only once + then = returned && + + // Support: Promises/A+ section 2.3.4 + // https://promisesaplus.com/#point-64 + // Only check objects and functions for thenability + ( typeof returned === "object" || + typeof returned === "function" ) && + returned.then; + + // Handle a returned thenable + if ( isFunction( then ) ) { + + // Special processors (notify) just wait for resolution + if ( special ) { + then.call( + returned, + resolve( maxDepth, deferred, Identity, special ), + resolve( maxDepth, deferred, Thrower, special ) + ); + + // Normal processors (resolve) also hook into progress + } else { + + // ...and disregard older resolution values + maxDepth++; + + then.call( + returned, + resolve( maxDepth, deferred, Identity, special ), + resolve( maxDepth, deferred, Thrower, special ), + resolve( maxDepth, deferred, Identity, + deferred.notifyWith ) + ); + } + + // Handle all other returned values + } else { + + // Only substitute handlers pass on context + // and multiple values (non-spec behavior) + if ( handler !== Identity ) { + that = undefined; + args = [ returned ]; + } + + // Process the value(s) + // Default process is resolve + ( special || deferred.resolveWith )( that, args ); + } + }, + + // Only normal processors (resolve) catch and reject exceptions + process = special ? + mightThrow : + function() { + try { + mightThrow(); + } catch ( e ) { + + if ( jQuery.Deferred.exceptionHook ) { + jQuery.Deferred.exceptionHook( e, + process.stackTrace ); + } + + // Support: Promises/A+ section 2.3.3.3.4.1 + // https://promisesaplus.com/#point-61 + // Ignore post-resolution exceptions + if ( depth + 1 >= maxDepth ) { + + // Only substitute handlers pass on context + // and multiple values (non-spec behavior) + if ( handler !== Thrower ) { + that = undefined; + args = [ e ]; + } + + deferred.rejectWith( that, args ); + } + } + }; + + // Support: Promises/A+ section 2.3.3.3.1 + // https://promisesaplus.com/#point-57 + // Re-resolve promises immediately to dodge false rejection from + // subsequent errors + if ( depth ) { + process(); + } else { + + // Call an optional hook to record the stack, in case of exception + // since it's otherwise lost when execution goes async + if ( jQuery.Deferred.getStackHook ) { + process.stackTrace = jQuery.Deferred.getStackHook(); + } + window.setTimeout( process ); + } + }; + } + + return jQuery.Deferred( function( newDefer ) { + + // progress_handlers.add( ... ) + tuples[ 0 ][ 3 ].add( + resolve( + 0, + newDefer, + isFunction( onProgress ) ? + onProgress : + Identity, + newDefer.notifyWith + ) + ); + + // fulfilled_handlers.add( ... ) + tuples[ 1 ][ 3 ].add( + resolve( + 0, + newDefer, + isFunction( onFulfilled ) ? + onFulfilled : + Identity + ) + ); + + // rejected_handlers.add( ... ) + tuples[ 2 ][ 3 ].add( + resolve( + 0, + newDefer, + isFunction( onRejected ) ? + onRejected : + Thrower + ) + ); + } ).promise(); + }, + + // Get a promise for this deferred + // If obj is provided, the promise aspect is added to the object + promise: function( obj ) { + return obj != null ? jQuery.extend( obj, promise ) : promise; + } + }, + deferred = {}; + + // Add list-specific methods + jQuery.each( tuples, function( i, tuple ) { + var list = tuple[ 2 ], + stateString = tuple[ 5 ]; + + // promise.progress = list.add + // promise.done = list.add + // promise.fail = list.add + promise[ tuple[ 1 ] ] = list.add; + + // Handle state + if ( stateString ) { + list.add( + function() { + + // state = "resolved" (i.e., fulfilled) + // state = "rejected" + state = stateString; + }, + + // rejected_callbacks.disable + // fulfilled_callbacks.disable + tuples[ 3 - i ][ 2 ].disable, + + // rejected_handlers.disable + // fulfilled_handlers.disable + tuples[ 3 - i ][ 3 ].disable, + + // progress_callbacks.lock + tuples[ 0 ][ 2 ].lock, + + // progress_handlers.lock + tuples[ 0 ][ 3 ].lock + ); + } + + // progress_handlers.fire + // fulfilled_handlers.fire + // rejected_handlers.fire + list.add( tuple[ 3 ].fire ); + + // deferred.notify = function() { deferred.notifyWith(...) } + // deferred.resolve = function() { deferred.resolveWith(...) } + // deferred.reject = function() { deferred.rejectWith(...) } + deferred[ tuple[ 0 ] ] = function() { + deferred[ tuple[ 0 ] + "With" ]( this === deferred ? undefined : this, arguments ); + return this; + }; + + // deferred.notifyWith = list.fireWith + // deferred.resolveWith = list.fireWith + // deferred.rejectWith = list.fireWith + deferred[ tuple[ 0 ] + "With" ] = list.fireWith; + } ); + + // Make the deferred a promise + promise.promise( deferred ); + + // Call given func if any + if ( func ) { + func.call( deferred, deferred ); + } + + // All done! + return deferred; + }, + + // Deferred helper + when: function( singleValue ) { + var + + // count of uncompleted subordinates + remaining = arguments.length, + + // count of unprocessed arguments + i = remaining, + + // subordinate fulfillment data + resolveContexts = Array( i ), + resolveValues = slice.call( arguments ), + + // the primary Deferred + primary = jQuery.Deferred(), + + // subordinate callback factory + updateFunc = function( i ) { + return function( value ) { + resolveContexts[ i ] = this; + resolveValues[ i ] = arguments.length > 1 ? slice.call( arguments ) : value; + if ( !( --remaining ) ) { + primary.resolveWith( resolveContexts, resolveValues ); + } + }; + }; + + // Single- and empty arguments are adopted like Promise.resolve + if ( remaining <= 1 ) { + adoptValue( singleValue, primary.done( updateFunc( i ) ).resolve, primary.reject, + !remaining ); + + // Use .then() to unwrap secondary thenables (cf. gh-3000) + if ( primary.state() === "pending" || + isFunction( resolveValues[ i ] && resolveValues[ i ].then ) ) { + + return primary.then(); + } + } + + // Multiple arguments are aggregated like Promise.all array elements + while ( i-- ) { + adoptValue( resolveValues[ i ], updateFunc( i ), primary.reject ); + } + + return primary.promise(); + } +} ); + + +// These usually indicate a programmer mistake during development, +// warn about them ASAP rather than swallowing them by default. +var rerrorNames = /^(Eval|Internal|Range|Reference|Syntax|Type|URI)Error$/; + +jQuery.Deferred.exceptionHook = function( error, stack ) { + + // Support: IE 8 - 9 only + // Console exists when dev tools are open, which can happen at any time + if ( window.console && window.console.warn && error && rerrorNames.test( error.name ) ) { + window.console.warn( "jQuery.Deferred exception: " + error.message, error.stack, stack ); + } +}; + + + + +jQuery.readyException = function( error ) { + window.setTimeout( function() { + throw error; + } ); +}; + + + + +// The deferred used on DOM ready +var readyList = jQuery.Deferred(); + +jQuery.fn.ready = function( fn ) { + + readyList + .then( fn ) + + // Wrap jQuery.readyException in a function so that the lookup + // happens at the time of error handling instead of callback + // registration. + .catch( function( error ) { + jQuery.readyException( error ); + } ); + + return this; +}; + +jQuery.extend( { + + // Is the DOM ready to be used? Set to true once it occurs. + isReady: false, + + // A counter to track how many items to wait for before + // the ready event fires. See #6781 + readyWait: 1, + + // Handle when the DOM is ready + ready: function( wait ) { + + // Abort if there are pending holds or we're already ready + if ( wait === true ? --jQuery.readyWait : jQuery.isReady ) { + return; + } + + // Remember that the DOM is ready + jQuery.isReady = true; + + // If a normal DOM Ready event fired, decrement, and wait if need be + if ( wait !== true && --jQuery.readyWait > 0 ) { + return; + } + + // If there are functions bound, to execute + readyList.resolveWith( document, [ jQuery ] ); + } +} ); + +jQuery.ready.then = readyList.then; + +// The ready event handler and self cleanup method +function completed() { + document.removeEventListener( "DOMContentLoaded", completed ); + window.removeEventListener( "load", completed ); + jQuery.ready(); +} + +// Catch cases where $(document).ready() is called +// after the browser event has already occurred. +// Support: IE <=9 - 10 only +// Older IE sometimes signals "interactive" too soon +if ( document.readyState === "complete" || + ( document.readyState !== "loading" && !document.documentElement.doScroll ) ) { + + // Handle it asynchronously to allow scripts the opportunity to delay ready + window.setTimeout( jQuery.ready ); + +} else { + + // Use the handy event callback + document.addEventListener( "DOMContentLoaded", completed ); + + // A fallback to window.onload, that will always work + window.addEventListener( "load", completed ); +} + + + + +// Multifunctional method to get and set values of a collection +// The value/s can optionally be executed if it's a function +var access = function( elems, fn, key, value, chainable, emptyGet, raw ) { + var i = 0, + len = elems.length, + bulk = key == null; + + // Sets many values + if ( toType( key ) === "object" ) { + chainable = true; + for ( i in key ) { + access( elems, fn, i, key[ i ], true, emptyGet, raw ); + } + + // Sets one value + } else if ( value !== undefined ) { + chainable = true; + + if ( !isFunction( value ) ) { + raw = true; + } + + if ( bulk ) { + + // Bulk operations run against the entire set + if ( raw ) { + fn.call( elems, value ); + fn = null; + + // ...except when executing function values + } else { + bulk = fn; + fn = function( elem, _key, value ) { + return bulk.call( jQuery( elem ), value ); + }; + } + } + + if ( fn ) { + for ( ; i < len; i++ ) { + fn( + elems[ i ], key, raw ? + value : + value.call( elems[ i ], i, fn( elems[ i ], key ) ) + ); + } + } + } + + if ( chainable ) { + return elems; + } + + // Gets + if ( bulk ) { + return fn.call( elems ); + } + + return len ? fn( elems[ 0 ], key ) : emptyGet; +}; + + +// Matches dashed string for camelizing +var rmsPrefix = /^-ms-/, + rdashAlpha = /-([a-z])/g; + +// Used by camelCase as callback to replace() +function fcamelCase( _all, letter ) { + return letter.toUpperCase(); +} + +// Convert dashed to camelCase; used by the css and data modules +// Support: IE <=9 - 11, Edge 12 - 15 +// Microsoft forgot to hump their vendor prefix (#9572) +function camelCase( string ) { + return string.replace( rmsPrefix, "ms-" ).replace( rdashAlpha, fcamelCase ); +} +var acceptData = function( owner ) { + + // Accepts only: + // - Node + // - Node.ELEMENT_NODE + // - Node.DOCUMENT_NODE + // - Object + // - Any + return owner.nodeType === 1 || owner.nodeType === 9 || !( +owner.nodeType ); +}; + + + + +function Data() { + this.expando = jQuery.expando + Data.uid++; +} + +Data.uid = 1; + +Data.prototype = { + + cache: function( owner ) { + + // Check if the owner object already has a cache + var value = owner[ this.expando ]; + + // If not, create one + if ( !value ) { + value = {}; + + // We can accept data for non-element nodes in modern browsers, + // but we should not, see #8335. + // Always return an empty object. + if ( acceptData( owner ) ) { + + // If it is a node unlikely to be stringify-ed or looped over + // use plain assignment + if ( owner.nodeType ) { + owner[ this.expando ] = value; + + // Otherwise secure it in a non-enumerable property + // configurable must be true to allow the property to be + // deleted when data is removed + } else { + Object.defineProperty( owner, this.expando, { + value: value, + configurable: true + } ); + } + } + } + + return value; + }, + set: function( owner, data, value ) { + var prop, + cache = this.cache( owner ); + + // Handle: [ owner, key, value ] args + // Always use camelCase key (gh-2257) + if ( typeof data === "string" ) { + cache[ camelCase( data ) ] = value; + + // Handle: [ owner, { properties } ] args + } else { + + // Copy the properties one-by-one to the cache object + for ( prop in data ) { + cache[ camelCase( prop ) ] = data[ prop ]; + } + } + return cache; + }, + get: function( owner, key ) { + return key === undefined ? + this.cache( owner ) : + + // Always use camelCase key (gh-2257) + owner[ this.expando ] && owner[ this.expando ][ camelCase( key ) ]; + }, + access: function( owner, key, value ) { + + // In cases where either: + // + // 1. No key was specified + // 2. A string key was specified, but no value provided + // + // Take the "read" path and allow the get method to determine + // which value to return, respectively either: + // + // 1. The entire cache object + // 2. The data stored at the key + // + if ( key === undefined || + ( ( key && typeof key === "string" ) && value === undefined ) ) { + + return this.get( owner, key ); + } + + // When the key is not a string, or both a key and value + // are specified, set or extend (existing objects) with either: + // + // 1. An object of properties + // 2. A key and value + // + this.set( owner, key, value ); + + // Since the "set" path can have two possible entry points + // return the expected data based on which path was taken[*] + return value !== undefined ? value : key; + }, + remove: function( owner, key ) { + var i, + cache = owner[ this.expando ]; + + if ( cache === undefined ) { + return; + } + + if ( key !== undefined ) { + + // Support array or space separated string of keys + if ( Array.isArray( key ) ) { + + // If key is an array of keys... + // We always set camelCase keys, so remove that. + key = key.map( camelCase ); + } else { + key = camelCase( key ); + + // If a key with the spaces exists, use it. + // Otherwise, create an array by matching non-whitespace + key = key in cache ? + [ key ] : + ( key.match( rnothtmlwhite ) || [] ); + } + + i = key.length; + + while ( i-- ) { + delete cache[ key[ i ] ]; + } + } + + // Remove the expando if there's no more data + if ( key === undefined || jQuery.isEmptyObject( cache ) ) { + + // Support: Chrome <=35 - 45 + // Webkit & Blink performance suffers when deleting properties + // from DOM nodes, so set to undefined instead + // https://bugs.chromium.org/p/chromium/issues/detail?id=378607 (bug restricted) + if ( owner.nodeType ) { + owner[ this.expando ] = undefined; + } else { + delete owner[ this.expando ]; + } + } + }, + hasData: function( owner ) { + var cache = owner[ this.expando ]; + return cache !== undefined && !jQuery.isEmptyObject( cache ); + } +}; +var dataPriv = new Data(); + +var dataUser = new Data(); + + + +// Implementation Summary +// +// 1. Enforce API surface and semantic compatibility with 1.9.x branch +// 2. Improve the module's maintainability by reducing the storage +// paths to a single mechanism. +// 3. Use the same single mechanism to support "private" and "user" data. +// 4. _Never_ expose "private" data to user code (TODO: Drop _data, _removeData) +// 5. Avoid exposing implementation details on user objects (eg. expando properties) +// 6. Provide a clear path for implementation upgrade to WeakMap in 2014 + +var rbrace = /^(?:\{[\w\W]*\}|\[[\w\W]*\])$/, + rmultiDash = /[A-Z]/g; + +function getData( data ) { + if ( data === "true" ) { + return true; + } + + if ( data === "false" ) { + return false; + } + + if ( data === "null" ) { + return null; + } + + // Only convert to a number if it doesn't change the string + if ( data === +data + "" ) { + return +data; + } + + if ( rbrace.test( data ) ) { + return JSON.parse( data ); + } + + return data; +} + +function dataAttr( elem, key, data ) { + var name; + + // If nothing was found internally, try to fetch any + // data from the HTML5 data-* attribute + if ( data === undefined && elem.nodeType === 1 ) { + name = "data-" + key.replace( rmultiDash, "-$&" ).toLowerCase(); + data = elem.getAttribute( name ); + + if ( typeof data === "string" ) { + try { + data = getData( data ); + } catch ( e ) {} + + // Make sure we set the data so it isn't changed later + dataUser.set( elem, key, data ); + } else { + data = undefined; + } + } + return data; +} + +jQuery.extend( { + hasData: function( elem ) { + return dataUser.hasData( elem ) || dataPriv.hasData( elem ); + }, + + data: function( elem, name, data ) { + return dataUser.access( elem, name, data ); + }, + + removeData: function( elem, name ) { + dataUser.remove( elem, name ); + }, + + // TODO: Now that all calls to _data and _removeData have been replaced + // with direct calls to dataPriv methods, these can be deprecated. + _data: function( elem, name, data ) { + return dataPriv.access( elem, name, data ); + }, + + _removeData: function( elem, name ) { + dataPriv.remove( elem, name ); + } +} ); + +jQuery.fn.extend( { + data: function( key, value ) { + var i, name, data, + elem = this[ 0 ], + attrs = elem && elem.attributes; + + // Gets all values + if ( key === undefined ) { + if ( this.length ) { + data = dataUser.get( elem ); + + if ( elem.nodeType === 1 && !dataPriv.get( elem, "hasDataAttrs" ) ) { + i = attrs.length; + while ( i-- ) { + + // Support: IE 11 only + // The attrs elements can be null (#14894) + if ( attrs[ i ] ) { + name = attrs[ i ].name; + if ( name.indexOf( "data-" ) === 0 ) { + name = camelCase( name.slice( 5 ) ); + dataAttr( elem, name, data[ name ] ); + } + } + } + dataPriv.set( elem, "hasDataAttrs", true ); + } + } + + return data; + } + + // Sets multiple values + if ( typeof key === "object" ) { + return this.each( function() { + dataUser.set( this, key ); + } ); + } + + return access( this, function( value ) { + var data; + + // The calling jQuery object (element matches) is not empty + // (and therefore has an element appears at this[ 0 ]) and the + // `value` parameter was not undefined. An empty jQuery object + // will result in `undefined` for elem = this[ 0 ] which will + // throw an exception if an attempt to read a data cache is made. + if ( elem && value === undefined ) { + + // Attempt to get data from the cache + // The key will always be camelCased in Data + data = dataUser.get( elem, key ); + if ( data !== undefined ) { + return data; + } + + // Attempt to "discover" the data in + // HTML5 custom data-* attrs + data = dataAttr( elem, key ); + if ( data !== undefined ) { + return data; + } + + // We tried really hard, but the data doesn't exist. + return; + } + + // Set the data... + this.each( function() { + + // We always store the camelCased key + dataUser.set( this, key, value ); + } ); + }, null, value, arguments.length > 1, null, true ); + }, + + removeData: function( key ) { + return this.each( function() { + dataUser.remove( this, key ); + } ); + } +} ); + + +jQuery.extend( { + queue: function( elem, type, data ) { + var queue; + + if ( elem ) { + type = ( type || "fx" ) + "queue"; + queue = dataPriv.get( elem, type ); + + // Speed up dequeue by getting out quickly if this is just a lookup + if ( data ) { + if ( !queue || Array.isArray( data ) ) { + queue = dataPriv.access( elem, type, jQuery.makeArray( data ) ); + } else { + queue.push( data ); + } + } + return queue || []; + } + }, + + dequeue: function( elem, type ) { + type = type || "fx"; + + var queue = jQuery.queue( elem, type ), + startLength = queue.length, + fn = queue.shift(), + hooks = jQuery._queueHooks( elem, type ), + next = function() { + jQuery.dequeue( elem, type ); + }; + + // If the fx queue is dequeued, always remove the progress sentinel + if ( fn === "inprogress" ) { + fn = queue.shift(); + startLength--; + } + + if ( fn ) { + + // Add a progress sentinel to prevent the fx queue from being + // automatically dequeued + if ( type === "fx" ) { + queue.unshift( "inprogress" ); + } + + // Clear up the last queue stop function + delete hooks.stop; + fn.call( elem, next, hooks ); + } + + if ( !startLength && hooks ) { + hooks.empty.fire(); + } + }, + + // Not public - generate a queueHooks object, or return the current one + _queueHooks: function( elem, type ) { + var key = type + "queueHooks"; + return dataPriv.get( elem, key ) || dataPriv.access( elem, key, { + empty: jQuery.Callbacks( "once memory" ).add( function() { + dataPriv.remove( elem, [ type + "queue", key ] ); + } ) + } ); + } +} ); + +jQuery.fn.extend( { + queue: function( type, data ) { + var setter = 2; + + if ( typeof type !== "string" ) { + data = type; + type = "fx"; + setter--; + } + + if ( arguments.length < setter ) { + return jQuery.queue( this[ 0 ], type ); + } + + return data === undefined ? + this : + this.each( function() { + var queue = jQuery.queue( this, type, data ); + + // Ensure a hooks for this queue + jQuery._queueHooks( this, type ); + + if ( type === "fx" && queue[ 0 ] !== "inprogress" ) { + jQuery.dequeue( this, type ); + } + } ); + }, + dequeue: function( type ) { + return this.each( function() { + jQuery.dequeue( this, type ); + } ); + }, + clearQueue: function( type ) { + return this.queue( type || "fx", [] ); + }, + + // Get a promise resolved when queues of a certain type + // are emptied (fx is the type by default) + promise: function( type, obj ) { + var tmp, + count = 1, + defer = jQuery.Deferred(), + elements = this, + i = this.length, + resolve = function() { + if ( !( --count ) ) { + defer.resolveWith( elements, [ elements ] ); + } + }; + + if ( typeof type !== "string" ) { + obj = type; + type = undefined; + } + type = type || "fx"; + + while ( i-- ) { + tmp = dataPriv.get( elements[ i ], type + "queueHooks" ); + if ( tmp && tmp.empty ) { + count++; + tmp.empty.add( resolve ); + } + } + resolve(); + return defer.promise( obj ); + } +} ); +var pnum = ( /[+-]?(?:\d*\.|)\d+(?:[eE][+-]?\d+|)/ ).source; + +var rcssNum = new RegExp( "^(?:([+-])=|)(" + pnum + ")([a-z%]*)$", "i" ); + + +var cssExpand = [ "Top", "Right", "Bottom", "Left" ]; + +var documentElement = document.documentElement; + + + + var isAttached = function( elem ) { + return jQuery.contains( elem.ownerDocument, elem ); + }, + composed = { composed: true }; + + // Support: IE 9 - 11+, Edge 12 - 18+, iOS 10.0 - 10.2 only + // Check attachment across shadow DOM boundaries when possible (gh-3504) + // Support: iOS 10.0-10.2 only + // Early iOS 10 versions support `attachShadow` but not `getRootNode`, + // leading to errors. We need to check for `getRootNode`. + if ( documentElement.getRootNode ) { + isAttached = function( elem ) { + return jQuery.contains( elem.ownerDocument, elem ) || + elem.getRootNode( composed ) === elem.ownerDocument; + }; + } +var isHiddenWithinTree = function( elem, el ) { + + // isHiddenWithinTree might be called from jQuery#filter function; + // in that case, element will be second argument + elem = el || elem; + + // Inline style trumps all + return elem.style.display === "none" || + elem.style.display === "" && + + // Otherwise, check computed style + // Support: Firefox <=43 - 45 + // Disconnected elements can have computed display: none, so first confirm that elem is + // in the document. + isAttached( elem ) && + + jQuery.css( elem, "display" ) === "none"; + }; + + + +function adjustCSS( elem, prop, valueParts, tween ) { + var adjusted, scale, + maxIterations = 20, + currentValue = tween ? + function() { + return tween.cur(); + } : + function() { + return jQuery.css( elem, prop, "" ); + }, + initial = currentValue(), + unit = valueParts && valueParts[ 3 ] || ( jQuery.cssNumber[ prop ] ? "" : "px" ), + + // Starting value computation is required for potential unit mismatches + initialInUnit = elem.nodeType && + ( jQuery.cssNumber[ prop ] || unit !== "px" && +initial ) && + rcssNum.exec( jQuery.css( elem, prop ) ); + + if ( initialInUnit && initialInUnit[ 3 ] !== unit ) { + + // Support: Firefox <=54 + // Halve the iteration target value to prevent interference from CSS upper bounds (gh-2144) + initial = initial / 2; + + // Trust units reported by jQuery.css + unit = unit || initialInUnit[ 3 ]; + + // Iteratively approximate from a nonzero starting point + initialInUnit = +initial || 1; + + while ( maxIterations-- ) { + + // Evaluate and update our best guess (doubling guesses that zero out). + // Finish if the scale equals or crosses 1 (making the old*new product non-positive). + jQuery.style( elem, prop, initialInUnit + unit ); + if ( ( 1 - scale ) * ( 1 - ( scale = currentValue() / initial || 0.5 ) ) <= 0 ) { + maxIterations = 0; + } + initialInUnit = initialInUnit / scale; + + } + + initialInUnit = initialInUnit * 2; + jQuery.style( elem, prop, initialInUnit + unit ); + + // Make sure we update the tween properties later on + valueParts = valueParts || []; + } + + if ( valueParts ) { + initialInUnit = +initialInUnit || +initial || 0; + + // Apply relative offset (+=/-=) if specified + adjusted = valueParts[ 1 ] ? + initialInUnit + ( valueParts[ 1 ] + 1 ) * valueParts[ 2 ] : + +valueParts[ 2 ]; + if ( tween ) { + tween.unit = unit; + tween.start = initialInUnit; + tween.end = adjusted; + } + } + return adjusted; +} + + +var defaultDisplayMap = {}; + +function getDefaultDisplay( elem ) { + var temp, + doc = elem.ownerDocument, + nodeName = elem.nodeName, + display = defaultDisplayMap[ nodeName ]; + + if ( display ) { + return display; + } + + temp = doc.body.appendChild( doc.createElement( nodeName ) ); + display = jQuery.css( temp, "display" ); + + temp.parentNode.removeChild( temp ); + + if ( display === "none" ) { + display = "block"; + } + defaultDisplayMap[ nodeName ] = display; + + return display; +} + +function showHide( elements, show ) { + var display, elem, + values = [], + index = 0, + length = elements.length; + + // Determine new display value for elements that need to change + for ( ; index < length; index++ ) { + elem = elements[ index ]; + if ( !elem.style ) { + continue; + } + + display = elem.style.display; + if ( show ) { + + // Since we force visibility upon cascade-hidden elements, an immediate (and slow) + // check is required in this first loop unless we have a nonempty display value (either + // inline or about-to-be-restored) + if ( display === "none" ) { + values[ index ] = dataPriv.get( elem, "display" ) || null; + if ( !values[ index ] ) { + elem.style.display = ""; + } + } + if ( elem.style.display === "" && isHiddenWithinTree( elem ) ) { + values[ index ] = getDefaultDisplay( elem ); + } + } else { + if ( display !== "none" ) { + values[ index ] = "none"; + + // Remember what we're overwriting + dataPriv.set( elem, "display", display ); + } + } + } + + // Set the display of the elements in a second loop to avoid constant reflow + for ( index = 0; index < length; index++ ) { + if ( values[ index ] != null ) { + elements[ index ].style.display = values[ index ]; + } + } + + return elements; +} + +jQuery.fn.extend( { + show: function() { + return showHide( this, true ); + }, + hide: function() { + return showHide( this ); + }, + toggle: function( state ) { + if ( typeof state === "boolean" ) { + return state ? this.show() : this.hide(); + } + + return this.each( function() { + if ( isHiddenWithinTree( this ) ) { + jQuery( this ).show(); + } else { + jQuery( this ).hide(); + } + } ); + } +} ); +var rcheckableType = ( /^(?:checkbox|radio)$/i ); + +var rtagName = ( /<([a-z][^\/\0>\x20\t\r\n\f]*)/i ); + +var rscriptType = ( /^$|^module$|\/(?:java|ecma)script/i ); + + + +( function() { + var fragment = document.createDocumentFragment(), + div = fragment.appendChild( document.createElement( "div" ) ), + input = document.createElement( "input" ); + + // Support: Android 4.0 - 4.3 only + // Check state lost if the name is set (#11217) + // Support: Windows Web Apps (WWA) + // `name` and `type` must use .setAttribute for WWA (#14901) + input.setAttribute( "type", "radio" ); + input.setAttribute( "checked", "checked" ); + input.setAttribute( "name", "t" ); + + div.appendChild( input ); + + // Support: Android <=4.1 only + // Older WebKit doesn't clone checked state correctly in fragments + support.checkClone = div.cloneNode( true ).cloneNode( true ).lastChild.checked; + + // Support: IE <=11 only + // Make sure textarea (and checkbox) defaultValue is properly cloned + div.innerHTML = ""; + support.noCloneChecked = !!div.cloneNode( true ).lastChild.defaultValue; + + // Support: IE <=9 only + // IE <=9 replaces "; + support.option = !!div.lastChild; +} )(); + + +// We have to close these tags to support XHTML (#13200) +var wrapMap = { + + // XHTML parsers do not magically insert elements in the + // same way that tag soup parsers do. So we cannot shorten + // this by omitting
or other required elements. + thead: [ 1, "
", "
" ], + col: [ 2, "", "
" ], + tr: [ 2, "", "
" ], + td: [ 3, "", "
" ], + + _default: [ 0, "", "" ] +}; + +wrapMap.tbody = wrapMap.tfoot = wrapMap.colgroup = wrapMap.caption = wrapMap.thead; +wrapMap.th = wrapMap.td; + +// Support: IE <=9 only +if ( !support.option ) { + wrapMap.optgroup = wrapMap.option = [ 1, "" ]; +} + + +function getAll( context, tag ) { + + // Support: IE <=9 - 11 only + // Use typeof to avoid zero-argument method invocation on host objects (#15151) + var ret; + + if ( typeof context.getElementsByTagName !== "undefined" ) { + ret = context.getElementsByTagName( tag || "*" ); + + } else if ( typeof context.querySelectorAll !== "undefined" ) { + ret = context.querySelectorAll( tag || "*" ); + + } else { + ret = []; + } + + if ( tag === undefined || tag && nodeName( context, tag ) ) { + return jQuery.merge( [ context ], ret ); + } + + return ret; +} + + +// Mark scripts as having already been evaluated +function setGlobalEval( elems, refElements ) { + var i = 0, + l = elems.length; + + for ( ; i < l; i++ ) { + dataPriv.set( + elems[ i ], + "globalEval", + !refElements || dataPriv.get( refElements[ i ], "globalEval" ) + ); + } +} + + +var rhtml = /<|&#?\w+;/; + +function buildFragment( elems, context, scripts, selection, ignored ) { + var elem, tmp, tag, wrap, attached, j, + fragment = context.createDocumentFragment(), + nodes = [], + i = 0, + l = elems.length; + + for ( ; i < l; i++ ) { + elem = elems[ i ]; + + if ( elem || elem === 0 ) { + + // Add nodes directly + if ( toType( elem ) === "object" ) { + + // Support: Android <=4.0 only, PhantomJS 1 only + // push.apply(_, arraylike) throws on ancient WebKit + jQuery.merge( nodes, elem.nodeType ? [ elem ] : elem ); + + // Convert non-html into a text node + } else if ( !rhtml.test( elem ) ) { + nodes.push( context.createTextNode( elem ) ); + + // Convert html into DOM nodes + } else { + tmp = tmp || fragment.appendChild( context.createElement( "div" ) ); + + // Deserialize a standard representation + tag = ( rtagName.exec( elem ) || [ "", "" ] )[ 1 ].toLowerCase(); + wrap = wrapMap[ tag ] || wrapMap._default; + tmp.innerHTML = wrap[ 1 ] + jQuery.htmlPrefilter( elem ) + wrap[ 2 ]; + + // Descend through wrappers to the right content + j = wrap[ 0 ]; + while ( j-- ) { + tmp = tmp.lastChild; + } + + // Support: Android <=4.0 only, PhantomJS 1 only + // push.apply(_, arraylike) throws on ancient WebKit + jQuery.merge( nodes, tmp.childNodes ); + + // Remember the top-level container + tmp = fragment.firstChild; + + // Ensure the created nodes are orphaned (#12392) + tmp.textContent = ""; + } + } + } + + // Remove wrapper from fragment + fragment.textContent = ""; + + i = 0; + while ( ( elem = nodes[ i++ ] ) ) { + + // Skip elements already in the context collection (trac-4087) + if ( selection && jQuery.inArray( elem, selection ) > -1 ) { + if ( ignored ) { + ignored.push( elem ); + } + continue; + } + + attached = isAttached( elem ); + + // Append to fragment + tmp = getAll( fragment.appendChild( elem ), "script" ); + + // Preserve script evaluation history + if ( attached ) { + setGlobalEval( tmp ); + } + + // Capture executables + if ( scripts ) { + j = 0; + while ( ( elem = tmp[ j++ ] ) ) { + if ( rscriptType.test( elem.type || "" ) ) { + scripts.push( elem ); + } + } + } + } + + return fragment; +} + + +var rtypenamespace = /^([^.]*)(?:\.(.+)|)/; + +function returnTrue() { + return true; +} + +function returnFalse() { + return false; +} + +// Support: IE <=9 - 11+ +// focus() and blur() are asynchronous, except when they are no-op. +// So expect focus to be synchronous when the element is already active, +// and blur to be synchronous when the element is not already active. +// (focus and blur are always synchronous in other supported browsers, +// this just defines when we can count on it). +function expectSync( elem, type ) { + return ( elem === safeActiveElement() ) === ( type === "focus" ); +} + +// Support: IE <=9 only +// Accessing document.activeElement can throw unexpectedly +// https://bugs.jquery.com/ticket/13393 +function safeActiveElement() { + try { + return document.activeElement; + } catch ( err ) { } +} + +function on( elem, types, selector, data, fn, one ) { + var origFn, type; + + // Types can be a map of types/handlers + if ( typeof types === "object" ) { + + // ( types-Object, selector, data ) + if ( typeof selector !== "string" ) { + + // ( types-Object, data ) + data = data || selector; + selector = undefined; + } + for ( type in types ) { + on( elem, type, selector, data, types[ type ], one ); + } + return elem; + } + + if ( data == null && fn == null ) { + + // ( types, fn ) + fn = selector; + data = selector = undefined; + } else if ( fn == null ) { + if ( typeof selector === "string" ) { + + // ( types, selector, fn ) + fn = data; + data = undefined; + } else { + + // ( types, data, fn ) + fn = data; + data = selector; + selector = undefined; + } + } + if ( fn === false ) { + fn = returnFalse; + } else if ( !fn ) { + return elem; + } + + if ( one === 1 ) { + origFn = fn; + fn = function( event ) { + + // Can use an empty set, since event contains the info + jQuery().off( event ); + return origFn.apply( this, arguments ); + }; + + // Use same guid so caller can remove using origFn + fn.guid = origFn.guid || ( origFn.guid = jQuery.guid++ ); + } + return elem.each( function() { + jQuery.event.add( this, types, fn, data, selector ); + } ); +} + +/* + * Helper functions for managing events -- not part of the public interface. + * Props to Dean Edwards' addEvent library for many of the ideas. + */ +jQuery.event = { + + global: {}, + + add: function( elem, types, handler, data, selector ) { + + var handleObjIn, eventHandle, tmp, + events, t, handleObj, + special, handlers, type, namespaces, origType, + elemData = dataPriv.get( elem ); + + // Only attach events to objects that accept data + if ( !acceptData( elem ) ) { + return; + } + + // Caller can pass in an object of custom data in lieu of the handler + if ( handler.handler ) { + handleObjIn = handler; + handler = handleObjIn.handler; + selector = handleObjIn.selector; + } + + // Ensure that invalid selectors throw exceptions at attach time + // Evaluate against documentElement in case elem is a non-element node (e.g., document) + if ( selector ) { + jQuery.find.matchesSelector( documentElement, selector ); + } + + // Make sure that the handler has a unique ID, used to find/remove it later + if ( !handler.guid ) { + handler.guid = jQuery.guid++; + } + + // Init the element's event structure and main handler, if this is the first + if ( !( events = elemData.events ) ) { + events = elemData.events = Object.create( null ); + } + if ( !( eventHandle = elemData.handle ) ) { + eventHandle = elemData.handle = function( e ) { + + // Discard the second event of a jQuery.event.trigger() and + // when an event is called after a page has unloaded + return typeof jQuery !== "undefined" && jQuery.event.triggered !== e.type ? + jQuery.event.dispatch.apply( elem, arguments ) : undefined; + }; + } + + // Handle multiple events separated by a space + types = ( types || "" ).match( rnothtmlwhite ) || [ "" ]; + t = types.length; + while ( t-- ) { + tmp = rtypenamespace.exec( types[ t ] ) || []; + type = origType = tmp[ 1 ]; + namespaces = ( tmp[ 2 ] || "" ).split( "." ).sort(); + + // There *must* be a type, no attaching namespace-only handlers + if ( !type ) { + continue; + } + + // If event changes its type, use the special event handlers for the changed type + special = jQuery.event.special[ type ] || {}; + + // If selector defined, determine special event api type, otherwise given type + type = ( selector ? special.delegateType : special.bindType ) || type; + + // Update special based on newly reset type + special = jQuery.event.special[ type ] || {}; + + // handleObj is passed to all event handlers + handleObj = jQuery.extend( { + type: type, + origType: origType, + data: data, + handler: handler, + guid: handler.guid, + selector: selector, + needsContext: selector && jQuery.expr.match.needsContext.test( selector ), + namespace: namespaces.join( "." ) + }, handleObjIn ); + + // Init the event handler queue if we're the first + if ( !( handlers = events[ type ] ) ) { + handlers = events[ type ] = []; + handlers.delegateCount = 0; + + // Only use addEventListener if the special events handler returns false + if ( !special.setup || + special.setup.call( elem, data, namespaces, eventHandle ) === false ) { + + if ( elem.addEventListener ) { + elem.addEventListener( type, eventHandle ); + } + } + } + + if ( special.add ) { + special.add.call( elem, handleObj ); + + if ( !handleObj.handler.guid ) { + handleObj.handler.guid = handler.guid; + } + } + + // Add to the element's handler list, delegates in front + if ( selector ) { + handlers.splice( handlers.delegateCount++, 0, handleObj ); + } else { + handlers.push( handleObj ); + } + + // Keep track of which events have ever been used, for event optimization + jQuery.event.global[ type ] = true; + } + + }, + + // Detach an event or set of events from an element + remove: function( elem, types, handler, selector, mappedTypes ) { + + var j, origCount, tmp, + events, t, handleObj, + special, handlers, type, namespaces, origType, + elemData = dataPriv.hasData( elem ) && dataPriv.get( elem ); + + if ( !elemData || !( events = elemData.events ) ) { + return; + } + + // Once for each type.namespace in types; type may be omitted + types = ( types || "" ).match( rnothtmlwhite ) || [ "" ]; + t = types.length; + while ( t-- ) { + tmp = rtypenamespace.exec( types[ t ] ) || []; + type = origType = tmp[ 1 ]; + namespaces = ( tmp[ 2 ] || "" ).split( "." ).sort(); + + // Unbind all events (on this namespace, if provided) for the element + if ( !type ) { + for ( type in events ) { + jQuery.event.remove( elem, type + types[ t ], handler, selector, true ); + } + continue; + } + + special = jQuery.event.special[ type ] || {}; + type = ( selector ? special.delegateType : special.bindType ) || type; + handlers = events[ type ] || []; + tmp = tmp[ 2 ] && + new RegExp( "(^|\\.)" + namespaces.join( "\\.(?:.*\\.|)" ) + "(\\.|$)" ); + + // Remove matching events + origCount = j = handlers.length; + while ( j-- ) { + handleObj = handlers[ j ]; + + if ( ( mappedTypes || origType === handleObj.origType ) && + ( !handler || handler.guid === handleObj.guid ) && + ( !tmp || tmp.test( handleObj.namespace ) ) && + ( !selector || selector === handleObj.selector || + selector === "**" && handleObj.selector ) ) { + handlers.splice( j, 1 ); + + if ( handleObj.selector ) { + handlers.delegateCount--; + } + if ( special.remove ) { + special.remove.call( elem, handleObj ); + } + } + } + + // Remove generic event handler if we removed something and no more handlers exist + // (avoids potential for endless recursion during removal of special event handlers) + if ( origCount && !handlers.length ) { + if ( !special.teardown || + special.teardown.call( elem, namespaces, elemData.handle ) === false ) { + + jQuery.removeEvent( elem, type, elemData.handle ); + } + + delete events[ type ]; + } + } + + // Remove data and the expando if it's no longer used + if ( jQuery.isEmptyObject( events ) ) { + dataPriv.remove( elem, "handle events" ); + } + }, + + dispatch: function( nativeEvent ) { + + var i, j, ret, matched, handleObj, handlerQueue, + args = new Array( arguments.length ), + + // Make a writable jQuery.Event from the native event object + event = jQuery.event.fix( nativeEvent ), + + handlers = ( + dataPriv.get( this, "events" ) || Object.create( null ) + )[ event.type ] || [], + special = jQuery.event.special[ event.type ] || {}; + + // Use the fix-ed jQuery.Event rather than the (read-only) native event + args[ 0 ] = event; + + for ( i = 1; i < arguments.length; i++ ) { + args[ i ] = arguments[ i ]; + } + + event.delegateTarget = this; + + // Call the preDispatch hook for the mapped type, and let it bail if desired + if ( special.preDispatch && special.preDispatch.call( this, event ) === false ) { + return; + } + + // Determine handlers + handlerQueue = jQuery.event.handlers.call( this, event, handlers ); + + // Run delegates first; they may want to stop propagation beneath us + i = 0; + while ( ( matched = handlerQueue[ i++ ] ) && !event.isPropagationStopped() ) { + event.currentTarget = matched.elem; + + j = 0; + while ( ( handleObj = matched.handlers[ j++ ] ) && + !event.isImmediatePropagationStopped() ) { + + // If the event is namespaced, then each handler is only invoked if it is + // specially universal or its namespaces are a superset of the event's. + if ( !event.rnamespace || handleObj.namespace === false || + event.rnamespace.test( handleObj.namespace ) ) { + + event.handleObj = handleObj; + event.data = handleObj.data; + + ret = ( ( jQuery.event.special[ handleObj.origType ] || {} ).handle || + handleObj.handler ).apply( matched.elem, args ); + + if ( ret !== undefined ) { + if ( ( event.result = ret ) === false ) { + event.preventDefault(); + event.stopPropagation(); + } + } + } + } + } + + // Call the postDispatch hook for the mapped type + if ( special.postDispatch ) { + special.postDispatch.call( this, event ); + } + + return event.result; + }, + + handlers: function( event, handlers ) { + var i, handleObj, sel, matchedHandlers, matchedSelectors, + handlerQueue = [], + delegateCount = handlers.delegateCount, + cur = event.target; + + // Find delegate handlers + if ( delegateCount && + + // Support: IE <=9 + // Black-hole SVG instance trees (trac-13180) + cur.nodeType && + + // Support: Firefox <=42 + // Suppress spec-violating clicks indicating a non-primary pointer button (trac-3861) + // https://www.w3.org/TR/DOM-Level-3-Events/#event-type-click + // Support: IE 11 only + // ...but not arrow key "clicks" of radio inputs, which can have `button` -1 (gh-2343) + !( event.type === "click" && event.button >= 1 ) ) { + + for ( ; cur !== this; cur = cur.parentNode || this ) { + + // Don't check non-elements (#13208) + // Don't process clicks on disabled elements (#6911, #8165, #11382, #11764) + if ( cur.nodeType === 1 && !( event.type === "click" && cur.disabled === true ) ) { + matchedHandlers = []; + matchedSelectors = {}; + for ( i = 0; i < delegateCount; i++ ) { + handleObj = handlers[ i ]; + + // Don't conflict with Object.prototype properties (#13203) + sel = handleObj.selector + " "; + + if ( matchedSelectors[ sel ] === undefined ) { + matchedSelectors[ sel ] = handleObj.needsContext ? + jQuery( sel, this ).index( cur ) > -1 : + jQuery.find( sel, this, null, [ cur ] ).length; + } + if ( matchedSelectors[ sel ] ) { + matchedHandlers.push( handleObj ); + } + } + if ( matchedHandlers.length ) { + handlerQueue.push( { elem: cur, handlers: matchedHandlers } ); + } + } + } + } + + // Add the remaining (directly-bound) handlers + cur = this; + if ( delegateCount < handlers.length ) { + handlerQueue.push( { elem: cur, handlers: handlers.slice( delegateCount ) } ); + } + + return handlerQueue; + }, + + addProp: function( name, hook ) { + Object.defineProperty( jQuery.Event.prototype, name, { + enumerable: true, + configurable: true, + + get: isFunction( hook ) ? + function() { + if ( this.originalEvent ) { + return hook( this.originalEvent ); + } + } : + function() { + if ( this.originalEvent ) { + return this.originalEvent[ name ]; + } + }, + + set: function( value ) { + Object.defineProperty( this, name, { + enumerable: true, + configurable: true, + writable: true, + value: value + } ); + } + } ); + }, + + fix: function( originalEvent ) { + return originalEvent[ jQuery.expando ] ? + originalEvent : + new jQuery.Event( originalEvent ); + }, + + special: { + load: { + + // Prevent triggered image.load events from bubbling to window.load + noBubble: true + }, + click: { + + // Utilize native event to ensure correct state for checkable inputs + setup: function( data ) { + + // For mutual compressibility with _default, replace `this` access with a local var. + // `|| data` is dead code meant only to preserve the variable through minification. + var el = this || data; + + // Claim the first handler + if ( rcheckableType.test( el.type ) && + el.click && nodeName( el, "input" ) ) { + + // dataPriv.set( el, "click", ... ) + leverageNative( el, "click", returnTrue ); + } + + // Return false to allow normal processing in the caller + return false; + }, + trigger: function( data ) { + + // For mutual compressibility with _default, replace `this` access with a local var. + // `|| data` is dead code meant only to preserve the variable through minification. + var el = this || data; + + // Force setup before triggering a click + if ( rcheckableType.test( el.type ) && + el.click && nodeName( el, "input" ) ) { + + leverageNative( el, "click" ); + } + + // Return non-false to allow normal event-path propagation + return true; + }, + + // For cross-browser consistency, suppress native .click() on links + // Also prevent it if we're currently inside a leveraged native-event stack + _default: function( event ) { + var target = event.target; + return rcheckableType.test( target.type ) && + target.click && nodeName( target, "input" ) && + dataPriv.get( target, "click" ) || + nodeName( target, "a" ); + } + }, + + beforeunload: { + postDispatch: function( event ) { + + // Support: Firefox 20+ + // Firefox doesn't alert if the returnValue field is not set. + if ( event.result !== undefined && event.originalEvent ) { + event.originalEvent.returnValue = event.result; + } + } + } + } +}; + +// Ensure the presence of an event listener that handles manually-triggered +// synthetic events by interrupting progress until reinvoked in response to +// *native* events that it fires directly, ensuring that state changes have +// already occurred before other listeners are invoked. +function leverageNative( el, type, expectSync ) { + + // Missing expectSync indicates a trigger call, which must force setup through jQuery.event.add + if ( !expectSync ) { + if ( dataPriv.get( el, type ) === undefined ) { + jQuery.event.add( el, type, returnTrue ); + } + return; + } + + // Register the controller as a special universal handler for all event namespaces + dataPriv.set( el, type, false ); + jQuery.event.add( el, type, { + namespace: false, + handler: function( event ) { + var notAsync, result, + saved = dataPriv.get( this, type ); + + if ( ( event.isTrigger & 1 ) && this[ type ] ) { + + // Interrupt processing of the outer synthetic .trigger()ed event + // Saved data should be false in such cases, but might be a leftover capture object + // from an async native handler (gh-4350) + if ( !saved.length ) { + + // Store arguments for use when handling the inner native event + // There will always be at least one argument (an event object), so this array + // will not be confused with a leftover capture object. + saved = slice.call( arguments ); + dataPriv.set( this, type, saved ); + + // Trigger the native event and capture its result + // Support: IE <=9 - 11+ + // focus() and blur() are asynchronous + notAsync = expectSync( this, type ); + this[ type ](); + result = dataPriv.get( this, type ); + if ( saved !== result || notAsync ) { + dataPriv.set( this, type, false ); + } else { + result = {}; + } + if ( saved !== result ) { + + // Cancel the outer synthetic event + event.stopImmediatePropagation(); + event.preventDefault(); + + // Support: Chrome 86+ + // In Chrome, if an element having a focusout handler is blurred by + // clicking outside of it, it invokes the handler synchronously. If + // that handler calls `.remove()` on the element, the data is cleared, + // leaving `result` undefined. We need to guard against this. + return result && result.value; + } + + // If this is an inner synthetic event for an event with a bubbling surrogate + // (focus or blur), assume that the surrogate already propagated from triggering the + // native event and prevent that from happening again here. + // This technically gets the ordering wrong w.r.t. to `.trigger()` (in which the + // bubbling surrogate propagates *after* the non-bubbling base), but that seems + // less bad than duplication. + } else if ( ( jQuery.event.special[ type ] || {} ).delegateType ) { + event.stopPropagation(); + } + + // If this is a native event triggered above, everything is now in order + // Fire an inner synthetic event with the original arguments + } else if ( saved.length ) { + + // ...and capture the result + dataPriv.set( this, type, { + value: jQuery.event.trigger( + + // Support: IE <=9 - 11+ + // Extend with the prototype to reset the above stopImmediatePropagation() + jQuery.extend( saved[ 0 ], jQuery.Event.prototype ), + saved.slice( 1 ), + this + ) + } ); + + // Abort handling of the native event + event.stopImmediatePropagation(); + } + } + } ); +} + +jQuery.removeEvent = function( elem, type, handle ) { + + // This "if" is needed for plain objects + if ( elem.removeEventListener ) { + elem.removeEventListener( type, handle ); + } +}; + +jQuery.Event = function( src, props ) { + + // Allow instantiation without the 'new' keyword + if ( !( this instanceof jQuery.Event ) ) { + return new jQuery.Event( src, props ); + } + + // Event object + if ( src && src.type ) { + this.originalEvent = src; + this.type = src.type; + + // Events bubbling up the document may have been marked as prevented + // by a handler lower down the tree; reflect the correct value. + this.isDefaultPrevented = src.defaultPrevented || + src.defaultPrevented === undefined && + + // Support: Android <=2.3 only + src.returnValue === false ? + returnTrue : + returnFalse; + + // Create target properties + // Support: Safari <=6 - 7 only + // Target should not be a text node (#504, #13143) + this.target = ( src.target && src.target.nodeType === 3 ) ? + src.target.parentNode : + src.target; + + this.currentTarget = src.currentTarget; + this.relatedTarget = src.relatedTarget; + + // Event type + } else { + this.type = src; + } + + // Put explicitly provided properties onto the event object + if ( props ) { + jQuery.extend( this, props ); + } + + // Create a timestamp if incoming event doesn't have one + this.timeStamp = src && src.timeStamp || Date.now(); + + // Mark it as fixed + this[ jQuery.expando ] = true; +}; + +// jQuery.Event is based on DOM3 Events as specified by the ECMAScript Language Binding +// https://www.w3.org/TR/2003/WD-DOM-Level-3-Events-20030331/ecma-script-binding.html +jQuery.Event.prototype = { + constructor: jQuery.Event, + isDefaultPrevented: returnFalse, + isPropagationStopped: returnFalse, + isImmediatePropagationStopped: returnFalse, + isSimulated: false, + + preventDefault: function() { + var e = this.originalEvent; + + this.isDefaultPrevented = returnTrue; + + if ( e && !this.isSimulated ) { + e.preventDefault(); + } + }, + stopPropagation: function() { + var e = this.originalEvent; + + this.isPropagationStopped = returnTrue; + + if ( e && !this.isSimulated ) { + e.stopPropagation(); + } + }, + stopImmediatePropagation: function() { + var e = this.originalEvent; + + this.isImmediatePropagationStopped = returnTrue; + + if ( e && !this.isSimulated ) { + e.stopImmediatePropagation(); + } + + this.stopPropagation(); + } +}; + +// Includes all common event props including KeyEvent and MouseEvent specific props +jQuery.each( { + altKey: true, + bubbles: true, + cancelable: true, + changedTouches: true, + ctrlKey: true, + detail: true, + eventPhase: true, + metaKey: true, + pageX: true, + pageY: true, + shiftKey: true, + view: true, + "char": true, + code: true, + charCode: true, + key: true, + keyCode: true, + button: true, + buttons: true, + clientX: true, + clientY: true, + offsetX: true, + offsetY: true, + pointerId: true, + pointerType: true, + screenX: true, + screenY: true, + targetTouches: true, + toElement: true, + touches: true, + which: true +}, jQuery.event.addProp ); + +jQuery.each( { focus: "focusin", blur: "focusout" }, function( type, delegateType ) { + jQuery.event.special[ type ] = { + + // Utilize native event if possible so blur/focus sequence is correct + setup: function() { + + // Claim the first handler + // dataPriv.set( this, "focus", ... ) + // dataPriv.set( this, "blur", ... ) + leverageNative( this, type, expectSync ); + + // Return false to allow normal processing in the caller + return false; + }, + trigger: function() { + + // Force setup before trigger + leverageNative( this, type ); + + // Return non-false to allow normal event-path propagation + return true; + }, + + // Suppress native focus or blur as it's already being fired + // in leverageNative. + _default: function() { + return true; + }, + + delegateType: delegateType + }; +} ); + +// Create mouseenter/leave events using mouseover/out and event-time checks +// so that event delegation works in jQuery. +// Do the same for pointerenter/pointerleave and pointerover/pointerout +// +// Support: Safari 7 only +// Safari sends mouseenter too often; see: +// https://bugs.chromium.org/p/chromium/issues/detail?id=470258 +// for the description of the bug (it existed in older Chrome versions as well). +jQuery.each( { + mouseenter: "mouseover", + mouseleave: "mouseout", + pointerenter: "pointerover", + pointerleave: "pointerout" +}, function( orig, fix ) { + jQuery.event.special[ orig ] = { + delegateType: fix, + bindType: fix, + + handle: function( event ) { + var ret, + target = this, + related = event.relatedTarget, + handleObj = event.handleObj; + + // For mouseenter/leave call the handler if related is outside the target. + // NB: No relatedTarget if the mouse left/entered the browser window + if ( !related || ( related !== target && !jQuery.contains( target, related ) ) ) { + event.type = handleObj.origType; + ret = handleObj.handler.apply( this, arguments ); + event.type = fix; + } + return ret; + } + }; +} ); + +jQuery.fn.extend( { + + on: function( types, selector, data, fn ) { + return on( this, types, selector, data, fn ); + }, + one: function( types, selector, data, fn ) { + return on( this, types, selector, data, fn, 1 ); + }, + off: function( types, selector, fn ) { + var handleObj, type; + if ( types && types.preventDefault && types.handleObj ) { + + // ( event ) dispatched jQuery.Event + handleObj = types.handleObj; + jQuery( types.delegateTarget ).off( + handleObj.namespace ? + handleObj.origType + "." + handleObj.namespace : + handleObj.origType, + handleObj.selector, + handleObj.handler + ); + return this; + } + if ( typeof types === "object" ) { + + // ( types-object [, selector] ) + for ( type in types ) { + this.off( type, selector, types[ type ] ); + } + return this; + } + if ( selector === false || typeof selector === "function" ) { + + // ( types [, fn] ) + fn = selector; + selector = undefined; + } + if ( fn === false ) { + fn = returnFalse; + } + return this.each( function() { + jQuery.event.remove( this, types, fn, selector ); + } ); + } +} ); + + +var + + // Support: IE <=10 - 11, Edge 12 - 13 only + // In IE/Edge using regex groups here causes severe slowdowns. + // See https://connect.microsoft.com/IE/feedback/details/1736512/ + rnoInnerhtml = /\s*$/g; + +// Prefer a tbody over its parent table for containing new rows +function manipulationTarget( elem, content ) { + if ( nodeName( elem, "table" ) && + nodeName( content.nodeType !== 11 ? content : content.firstChild, "tr" ) ) { + + return jQuery( elem ).children( "tbody" )[ 0 ] || elem; + } + + return elem; +} + +// Replace/restore the type attribute of script elements for safe DOM manipulation +function disableScript( elem ) { + elem.type = ( elem.getAttribute( "type" ) !== null ) + "/" + elem.type; + return elem; +} +function restoreScript( elem ) { + if ( ( elem.type || "" ).slice( 0, 5 ) === "true/" ) { + elem.type = elem.type.slice( 5 ); + } else { + elem.removeAttribute( "type" ); + } + + return elem; +} + +function cloneCopyEvent( src, dest ) { + var i, l, type, pdataOld, udataOld, udataCur, events; + + if ( dest.nodeType !== 1 ) { + return; + } + + // 1. Copy private data: events, handlers, etc. + if ( dataPriv.hasData( src ) ) { + pdataOld = dataPriv.get( src ); + events = pdataOld.events; + + if ( events ) { + dataPriv.remove( dest, "handle events" ); + + for ( type in events ) { + for ( i = 0, l = events[ type ].length; i < l; i++ ) { + jQuery.event.add( dest, type, events[ type ][ i ] ); + } + } + } + } + + // 2. Copy user data + if ( dataUser.hasData( src ) ) { + udataOld = dataUser.access( src ); + udataCur = jQuery.extend( {}, udataOld ); + + dataUser.set( dest, udataCur ); + } +} + +// Fix IE bugs, see support tests +function fixInput( src, dest ) { + var nodeName = dest.nodeName.toLowerCase(); + + // Fails to persist the checked state of a cloned checkbox or radio button. + if ( nodeName === "input" && rcheckableType.test( src.type ) ) { + dest.checked = src.checked; + + // Fails to return the selected option to the default selected state when cloning options + } else if ( nodeName === "input" || nodeName === "textarea" ) { + dest.defaultValue = src.defaultValue; + } +} + +function domManip( collection, args, callback, ignored ) { + + // Flatten any nested arrays + args = flat( args ); + + var fragment, first, scripts, hasScripts, node, doc, + i = 0, + l = collection.length, + iNoClone = l - 1, + value = args[ 0 ], + valueIsFunction = isFunction( value ); + + // We can't cloneNode fragments that contain checked, in WebKit + if ( valueIsFunction || + ( l > 1 && typeof value === "string" && + !support.checkClone && rchecked.test( value ) ) ) { + return collection.each( function( index ) { + var self = collection.eq( index ); + if ( valueIsFunction ) { + args[ 0 ] = value.call( this, index, self.html() ); + } + domManip( self, args, callback, ignored ); + } ); + } + + if ( l ) { + fragment = buildFragment( args, collection[ 0 ].ownerDocument, false, collection, ignored ); + first = fragment.firstChild; + + if ( fragment.childNodes.length === 1 ) { + fragment = first; + } + + // Require either new content or an interest in ignored elements to invoke the callback + if ( first || ignored ) { + scripts = jQuery.map( getAll( fragment, "script" ), disableScript ); + hasScripts = scripts.length; + + // Use the original fragment for the last item + // instead of the first because it can end up + // being emptied incorrectly in certain situations (#8070). + for ( ; i < l; i++ ) { + node = fragment; + + if ( i !== iNoClone ) { + node = jQuery.clone( node, true, true ); + + // Keep references to cloned scripts for later restoration + if ( hasScripts ) { + + // Support: Android <=4.0 only, PhantomJS 1 only + // push.apply(_, arraylike) throws on ancient WebKit + jQuery.merge( scripts, getAll( node, "script" ) ); + } + } + + callback.call( collection[ i ], node, i ); + } + + if ( hasScripts ) { + doc = scripts[ scripts.length - 1 ].ownerDocument; + + // Reenable scripts + jQuery.map( scripts, restoreScript ); + + // Evaluate executable scripts on first document insertion + for ( i = 0; i < hasScripts; i++ ) { + node = scripts[ i ]; + if ( rscriptType.test( node.type || "" ) && + !dataPriv.access( node, "globalEval" ) && + jQuery.contains( doc, node ) ) { + + if ( node.src && ( node.type || "" ).toLowerCase() !== "module" ) { + + // Optional AJAX dependency, but won't run scripts if not present + if ( jQuery._evalUrl && !node.noModule ) { + jQuery._evalUrl( node.src, { + nonce: node.nonce || node.getAttribute( "nonce" ) + }, doc ); + } + } else { + DOMEval( node.textContent.replace( rcleanScript, "" ), node, doc ); + } + } + } + } + } + } + + return collection; +} + +function remove( elem, selector, keepData ) { + var node, + nodes = selector ? jQuery.filter( selector, elem ) : elem, + i = 0; + + for ( ; ( node = nodes[ i ] ) != null; i++ ) { + if ( !keepData && node.nodeType === 1 ) { + jQuery.cleanData( getAll( node ) ); + } + + if ( node.parentNode ) { + if ( keepData && isAttached( node ) ) { + setGlobalEval( getAll( node, "script" ) ); + } + node.parentNode.removeChild( node ); + } + } + + return elem; +} + +jQuery.extend( { + htmlPrefilter: function( html ) { + return html; + }, + + clone: function( elem, dataAndEvents, deepDataAndEvents ) { + var i, l, srcElements, destElements, + clone = elem.cloneNode( true ), + inPage = isAttached( elem ); + + // Fix IE cloning issues + if ( !support.noCloneChecked && ( elem.nodeType === 1 || elem.nodeType === 11 ) && + !jQuery.isXMLDoc( elem ) ) { + + // We eschew Sizzle here for performance reasons: https://jsperf.com/getall-vs-sizzle/2 + destElements = getAll( clone ); + srcElements = getAll( elem ); + + for ( i = 0, l = srcElements.length; i < l; i++ ) { + fixInput( srcElements[ i ], destElements[ i ] ); + } + } + + // Copy the events from the original to the clone + if ( dataAndEvents ) { + if ( deepDataAndEvents ) { + srcElements = srcElements || getAll( elem ); + destElements = destElements || getAll( clone ); + + for ( i = 0, l = srcElements.length; i < l; i++ ) { + cloneCopyEvent( srcElements[ i ], destElements[ i ] ); + } + } else { + cloneCopyEvent( elem, clone ); + } + } + + // Preserve script evaluation history + destElements = getAll( clone, "script" ); + if ( destElements.length > 0 ) { + setGlobalEval( destElements, !inPage && getAll( elem, "script" ) ); + } + + // Return the cloned set + return clone; + }, + + cleanData: function( elems ) { + var data, elem, type, + special = jQuery.event.special, + i = 0; + + for ( ; ( elem = elems[ i ] ) !== undefined; i++ ) { + if ( acceptData( elem ) ) { + if ( ( data = elem[ dataPriv.expando ] ) ) { + if ( data.events ) { + for ( type in data.events ) { + if ( special[ type ] ) { + jQuery.event.remove( elem, type ); + + // This is a shortcut to avoid jQuery.event.remove's overhead + } else { + jQuery.removeEvent( elem, type, data.handle ); + } + } + } + + // Support: Chrome <=35 - 45+ + // Assign undefined instead of using delete, see Data#remove + elem[ dataPriv.expando ] = undefined; + } + if ( elem[ dataUser.expando ] ) { + + // Support: Chrome <=35 - 45+ + // Assign undefined instead of using delete, see Data#remove + elem[ dataUser.expando ] = undefined; + } + } + } + } +} ); + +jQuery.fn.extend( { + detach: function( selector ) { + return remove( this, selector, true ); + }, + + remove: function( selector ) { + return remove( this, selector ); + }, + + text: function( value ) { + return access( this, function( value ) { + return value === undefined ? + jQuery.text( this ) : + this.empty().each( function() { + if ( this.nodeType === 1 || this.nodeType === 11 || this.nodeType === 9 ) { + this.textContent = value; + } + } ); + }, null, value, arguments.length ); + }, + + append: function() { + return domManip( this, arguments, function( elem ) { + if ( this.nodeType === 1 || this.nodeType === 11 || this.nodeType === 9 ) { + var target = manipulationTarget( this, elem ); + target.appendChild( elem ); + } + } ); + }, + + prepend: function() { + return domManip( this, arguments, function( elem ) { + if ( this.nodeType === 1 || this.nodeType === 11 || this.nodeType === 9 ) { + var target = manipulationTarget( this, elem ); + target.insertBefore( elem, target.firstChild ); + } + } ); + }, + + before: function() { + return domManip( this, arguments, function( elem ) { + if ( this.parentNode ) { + this.parentNode.insertBefore( elem, this ); + } + } ); + }, + + after: function() { + return domManip( this, arguments, function( elem ) { + if ( this.parentNode ) { + this.parentNode.insertBefore( elem, this.nextSibling ); + } + } ); + }, + + empty: function() { + var elem, + i = 0; + + for ( ; ( elem = this[ i ] ) != null; i++ ) { + if ( elem.nodeType === 1 ) { + + // Prevent memory leaks + jQuery.cleanData( getAll( elem, false ) ); + + // Remove any remaining nodes + elem.textContent = ""; + } + } + + return this; + }, + + clone: function( dataAndEvents, deepDataAndEvents ) { + dataAndEvents = dataAndEvents == null ? false : dataAndEvents; + deepDataAndEvents = deepDataAndEvents == null ? dataAndEvents : deepDataAndEvents; + + return this.map( function() { + return jQuery.clone( this, dataAndEvents, deepDataAndEvents ); + } ); + }, + + html: function( value ) { + return access( this, function( value ) { + var elem = this[ 0 ] || {}, + i = 0, + l = this.length; + + if ( value === undefined && elem.nodeType === 1 ) { + return elem.innerHTML; + } + + // See if we can take a shortcut and just use innerHTML + if ( typeof value === "string" && !rnoInnerhtml.test( value ) && + !wrapMap[ ( rtagName.exec( value ) || [ "", "" ] )[ 1 ].toLowerCase() ] ) { + + value = jQuery.htmlPrefilter( value ); + + try { + for ( ; i < l; i++ ) { + elem = this[ i ] || {}; + + // Remove element nodes and prevent memory leaks + if ( elem.nodeType === 1 ) { + jQuery.cleanData( getAll( elem, false ) ); + elem.innerHTML = value; + } + } + + elem = 0; + + // If using innerHTML throws an exception, use the fallback method + } catch ( e ) {} + } + + if ( elem ) { + this.empty().append( value ); + } + }, null, value, arguments.length ); + }, + + replaceWith: function() { + var ignored = []; + + // Make the changes, replacing each non-ignored context element with the new content + return domManip( this, arguments, function( elem ) { + var parent = this.parentNode; + + if ( jQuery.inArray( this, ignored ) < 0 ) { + jQuery.cleanData( getAll( this ) ); + if ( parent ) { + parent.replaceChild( elem, this ); + } + } + + // Force callback invocation + }, ignored ); + } +} ); + +jQuery.each( { + appendTo: "append", + prependTo: "prepend", + insertBefore: "before", + insertAfter: "after", + replaceAll: "replaceWith" +}, function( name, original ) { + jQuery.fn[ name ] = function( selector ) { + var elems, + ret = [], + insert = jQuery( selector ), + last = insert.length - 1, + i = 0; + + for ( ; i <= last; i++ ) { + elems = i === last ? this : this.clone( true ); + jQuery( insert[ i ] )[ original ]( elems ); + + // Support: Android <=4.0 only, PhantomJS 1 only + // .get() because push.apply(_, arraylike) throws on ancient WebKit + push.apply( ret, elems.get() ); + } + + return this.pushStack( ret ); + }; +} ); +var rnumnonpx = new RegExp( "^(" + pnum + ")(?!px)[a-z%]+$", "i" ); + +var getStyles = function( elem ) { + + // Support: IE <=11 only, Firefox <=30 (#15098, #14150) + // IE throws on elements created in popups + // FF meanwhile throws on frame elements through "defaultView.getComputedStyle" + var view = elem.ownerDocument.defaultView; + + if ( !view || !view.opener ) { + view = window; + } + + return view.getComputedStyle( elem ); + }; + +var swap = function( elem, options, callback ) { + var ret, name, + old = {}; + + // Remember the old values, and insert the new ones + for ( name in options ) { + old[ name ] = elem.style[ name ]; + elem.style[ name ] = options[ name ]; + } + + ret = callback.call( elem ); + + // Revert the old values + for ( name in options ) { + elem.style[ name ] = old[ name ]; + } + + return ret; +}; + + +var rboxStyle = new RegExp( cssExpand.join( "|" ), "i" ); + + + +( function() { + + // Executing both pixelPosition & boxSizingReliable tests require only one layout + // so they're executed at the same time to save the second computation. + function computeStyleTests() { + + // This is a singleton, we need to execute it only once + if ( !div ) { + return; + } + + container.style.cssText = "position:absolute;left:-11111px;width:60px;" + + "margin-top:1px;padding:0;border:0"; + div.style.cssText = + "position:relative;display:block;box-sizing:border-box;overflow:scroll;" + + "margin:auto;border:1px;padding:1px;" + + "width:60%;top:1%"; + documentElement.appendChild( container ).appendChild( div ); + + var divStyle = window.getComputedStyle( div ); + pixelPositionVal = divStyle.top !== "1%"; + + // Support: Android 4.0 - 4.3 only, Firefox <=3 - 44 + reliableMarginLeftVal = roundPixelMeasures( divStyle.marginLeft ) === 12; + + // Support: Android 4.0 - 4.3 only, Safari <=9.1 - 10.1, iOS <=7.0 - 9.3 + // Some styles come back with percentage values, even though they shouldn't + div.style.right = "60%"; + pixelBoxStylesVal = roundPixelMeasures( divStyle.right ) === 36; + + // Support: IE 9 - 11 only + // Detect misreporting of content dimensions for box-sizing:border-box elements + boxSizingReliableVal = roundPixelMeasures( divStyle.width ) === 36; + + // Support: IE 9 only + // Detect overflow:scroll screwiness (gh-3699) + // Support: Chrome <=64 + // Don't get tricked when zoom affects offsetWidth (gh-4029) + div.style.position = "absolute"; + scrollboxSizeVal = roundPixelMeasures( div.offsetWidth / 3 ) === 12; + + documentElement.removeChild( container ); + + // Nullify the div so it wouldn't be stored in the memory and + // it will also be a sign that checks already performed + div = null; + } + + function roundPixelMeasures( measure ) { + return Math.round( parseFloat( measure ) ); + } + + var pixelPositionVal, boxSizingReliableVal, scrollboxSizeVal, pixelBoxStylesVal, + reliableTrDimensionsVal, reliableMarginLeftVal, + container = document.createElement( "div" ), + div = document.createElement( "div" ); + + // Finish early in limited (non-browser) environments + if ( !div.style ) { + return; + } + + // Support: IE <=9 - 11 only + // Style of cloned element affects source element cloned (#8908) + div.style.backgroundClip = "content-box"; + div.cloneNode( true ).style.backgroundClip = ""; + support.clearCloneStyle = div.style.backgroundClip === "content-box"; + + jQuery.extend( support, { + boxSizingReliable: function() { + computeStyleTests(); + return boxSizingReliableVal; + }, + pixelBoxStyles: function() { + computeStyleTests(); + return pixelBoxStylesVal; + }, + pixelPosition: function() { + computeStyleTests(); + return pixelPositionVal; + }, + reliableMarginLeft: function() { + computeStyleTests(); + return reliableMarginLeftVal; + }, + scrollboxSize: function() { + computeStyleTests(); + return scrollboxSizeVal; + }, + + // Support: IE 9 - 11+, Edge 15 - 18+ + // IE/Edge misreport `getComputedStyle` of table rows with width/height + // set in CSS while `offset*` properties report correct values. + // Behavior in IE 9 is more subtle than in newer versions & it passes + // some versions of this test; make sure not to make it pass there! + // + // Support: Firefox 70+ + // Only Firefox includes border widths + // in computed dimensions. (gh-4529) + reliableTrDimensions: function() { + var table, tr, trChild, trStyle; + if ( reliableTrDimensionsVal == null ) { + table = document.createElement( "table" ); + tr = document.createElement( "tr" ); + trChild = document.createElement( "div" ); + + table.style.cssText = "position:absolute;left:-11111px;border-collapse:separate"; + tr.style.cssText = "border:1px solid"; + + // Support: Chrome 86+ + // Height set through cssText does not get applied. + // Computed height then comes back as 0. + tr.style.height = "1px"; + trChild.style.height = "9px"; + + // Support: Android 8 Chrome 86+ + // In our bodyBackground.html iframe, + // display for all div elements is set to "inline", + // which causes a problem only in Android 8 Chrome 86. + // Ensuring the div is display: block + // gets around this issue. + trChild.style.display = "block"; + + documentElement + .appendChild( table ) + .appendChild( tr ) + .appendChild( trChild ); + + trStyle = window.getComputedStyle( tr ); + reliableTrDimensionsVal = ( parseInt( trStyle.height, 10 ) + + parseInt( trStyle.borderTopWidth, 10 ) + + parseInt( trStyle.borderBottomWidth, 10 ) ) === tr.offsetHeight; + + documentElement.removeChild( table ); + } + return reliableTrDimensionsVal; + } + } ); +} )(); + + +function curCSS( elem, name, computed ) { + var width, minWidth, maxWidth, ret, + + // Support: Firefox 51+ + // Retrieving style before computed somehow + // fixes an issue with getting wrong values + // on detached elements + style = elem.style; + + computed = computed || getStyles( elem ); + + // getPropertyValue is needed for: + // .css('filter') (IE 9 only, #12537) + // .css('--customProperty) (#3144) + if ( computed ) { + ret = computed.getPropertyValue( name ) || computed[ name ]; + + if ( ret === "" && !isAttached( elem ) ) { + ret = jQuery.style( elem, name ); + } + + // A tribute to the "awesome hack by Dean Edwards" + // Android Browser returns percentage for some values, + // but width seems to be reliably pixels. + // This is against the CSSOM draft spec: + // https://drafts.csswg.org/cssom/#resolved-values + if ( !support.pixelBoxStyles() && rnumnonpx.test( ret ) && rboxStyle.test( name ) ) { + + // Remember the original values + width = style.width; + minWidth = style.minWidth; + maxWidth = style.maxWidth; + + // Put in the new values to get a computed value out + style.minWidth = style.maxWidth = style.width = ret; + ret = computed.width; + + // Revert the changed values + style.width = width; + style.minWidth = minWidth; + style.maxWidth = maxWidth; + } + } + + return ret !== undefined ? + + // Support: IE <=9 - 11 only + // IE returns zIndex value as an integer. + ret + "" : + ret; +} + + +function addGetHookIf( conditionFn, hookFn ) { + + // Define the hook, we'll check on the first run if it's really needed. + return { + get: function() { + if ( conditionFn() ) { + + // Hook not needed (or it's not possible to use it due + // to missing dependency), remove it. + delete this.get; + return; + } + + // Hook needed; redefine it so that the support test is not executed again. + return ( this.get = hookFn ).apply( this, arguments ); + } + }; +} + + +var cssPrefixes = [ "Webkit", "Moz", "ms" ], + emptyStyle = document.createElement( "div" ).style, + vendorProps = {}; + +// Return a vendor-prefixed property or undefined +function vendorPropName( name ) { + + // Check for vendor prefixed names + var capName = name[ 0 ].toUpperCase() + name.slice( 1 ), + i = cssPrefixes.length; + + while ( i-- ) { + name = cssPrefixes[ i ] + capName; + if ( name in emptyStyle ) { + return name; + } + } +} + +// Return a potentially-mapped jQuery.cssProps or vendor prefixed property +function finalPropName( name ) { + var final = jQuery.cssProps[ name ] || vendorProps[ name ]; + + if ( final ) { + return final; + } + if ( name in emptyStyle ) { + return name; + } + return vendorProps[ name ] = vendorPropName( name ) || name; +} + + +var + + // Swappable if display is none or starts with table + // except "table", "table-cell", or "table-caption" + // See here for display values: https://developer.mozilla.org/en-US/docs/CSS/display + rdisplayswap = /^(none|table(?!-c[ea]).+)/, + rcustomProp = /^--/, + cssShow = { position: "absolute", visibility: "hidden", display: "block" }, + cssNormalTransform = { + letterSpacing: "0", + fontWeight: "400" + }; + +function setPositiveNumber( _elem, value, subtract ) { + + // Any relative (+/-) values have already been + // normalized at this point + var matches = rcssNum.exec( value ); + return matches ? + + // Guard against undefined "subtract", e.g., when used as in cssHooks + Math.max( 0, matches[ 2 ] - ( subtract || 0 ) ) + ( matches[ 3 ] || "px" ) : + value; +} + +function boxModelAdjustment( elem, dimension, box, isBorderBox, styles, computedVal ) { + var i = dimension === "width" ? 1 : 0, + extra = 0, + delta = 0; + + // Adjustment may not be necessary + if ( box === ( isBorderBox ? "border" : "content" ) ) { + return 0; + } + + for ( ; i < 4; i += 2 ) { + + // Both box models exclude margin + if ( box === "margin" ) { + delta += jQuery.css( elem, box + cssExpand[ i ], true, styles ); + } + + // If we get here with a content-box, we're seeking "padding" or "border" or "margin" + if ( !isBorderBox ) { + + // Add padding + delta += jQuery.css( elem, "padding" + cssExpand[ i ], true, styles ); + + // For "border" or "margin", add border + if ( box !== "padding" ) { + delta += jQuery.css( elem, "border" + cssExpand[ i ] + "Width", true, styles ); + + // But still keep track of it otherwise + } else { + extra += jQuery.css( elem, "border" + cssExpand[ i ] + "Width", true, styles ); + } + + // If we get here with a border-box (content + padding + border), we're seeking "content" or + // "padding" or "margin" + } else { + + // For "content", subtract padding + if ( box === "content" ) { + delta -= jQuery.css( elem, "padding" + cssExpand[ i ], true, styles ); + } + + // For "content" or "padding", subtract border + if ( box !== "margin" ) { + delta -= jQuery.css( elem, "border" + cssExpand[ i ] + "Width", true, styles ); + } + } + } + + // Account for positive content-box scroll gutter when requested by providing computedVal + if ( !isBorderBox && computedVal >= 0 ) { + + // offsetWidth/offsetHeight is a rounded sum of content, padding, scroll gutter, and border + // Assuming integer scroll gutter, subtract the rest and round down + delta += Math.max( 0, Math.ceil( + elem[ "offset" + dimension[ 0 ].toUpperCase() + dimension.slice( 1 ) ] - + computedVal - + delta - + extra - + 0.5 + + // If offsetWidth/offsetHeight is unknown, then we can't determine content-box scroll gutter + // Use an explicit zero to avoid NaN (gh-3964) + ) ) || 0; + } + + return delta; +} + +function getWidthOrHeight( elem, dimension, extra ) { + + // Start with computed style + var styles = getStyles( elem ), + + // To avoid forcing a reflow, only fetch boxSizing if we need it (gh-4322). + // Fake content-box until we know it's needed to know the true value. + boxSizingNeeded = !support.boxSizingReliable() || extra, + isBorderBox = boxSizingNeeded && + jQuery.css( elem, "boxSizing", false, styles ) === "border-box", + valueIsBorderBox = isBorderBox, + + val = curCSS( elem, dimension, styles ), + offsetProp = "offset" + dimension[ 0 ].toUpperCase() + dimension.slice( 1 ); + + // Support: Firefox <=54 + // Return a confounding non-pixel value or feign ignorance, as appropriate. + if ( rnumnonpx.test( val ) ) { + if ( !extra ) { + return val; + } + val = "auto"; + } + + + // Support: IE 9 - 11 only + // Use offsetWidth/offsetHeight for when box sizing is unreliable. + // In those cases, the computed value can be trusted to be border-box. + if ( ( !support.boxSizingReliable() && isBorderBox || + + // Support: IE 10 - 11+, Edge 15 - 18+ + // IE/Edge misreport `getComputedStyle` of table rows with width/height + // set in CSS while `offset*` properties report correct values. + // Interestingly, in some cases IE 9 doesn't suffer from this issue. + !support.reliableTrDimensions() && nodeName( elem, "tr" ) || + + // Fall back to offsetWidth/offsetHeight when value is "auto" + // This happens for inline elements with no explicit setting (gh-3571) + val === "auto" || + + // Support: Android <=4.1 - 4.3 only + // Also use offsetWidth/offsetHeight for misreported inline dimensions (gh-3602) + !parseFloat( val ) && jQuery.css( elem, "display", false, styles ) === "inline" ) && + + // Make sure the element is visible & connected + elem.getClientRects().length ) { + + isBorderBox = jQuery.css( elem, "boxSizing", false, styles ) === "border-box"; + + // Where available, offsetWidth/offsetHeight approximate border box dimensions. + // Where not available (e.g., SVG), assume unreliable box-sizing and interpret the + // retrieved value as a content box dimension. + valueIsBorderBox = offsetProp in elem; + if ( valueIsBorderBox ) { + val = elem[ offsetProp ]; + } + } + + // Normalize "" and auto + val = parseFloat( val ) || 0; + + // Adjust for the element's box model + return ( val + + boxModelAdjustment( + elem, + dimension, + extra || ( isBorderBox ? "border" : "content" ), + valueIsBorderBox, + styles, + + // Provide the current computed size to request scroll gutter calculation (gh-3589) + val + ) + ) + "px"; +} + +jQuery.extend( { + + // Add in style property hooks for overriding the default + // behavior of getting and setting a style property + cssHooks: { + opacity: { + get: function( elem, computed ) { + if ( computed ) { + + // We should always get a number back from opacity + var ret = curCSS( elem, "opacity" ); + return ret === "" ? "1" : ret; + } + } + } + }, + + // Don't automatically add "px" to these possibly-unitless properties + cssNumber: { + "animationIterationCount": true, + "columnCount": true, + "fillOpacity": true, + "flexGrow": true, + "flexShrink": true, + "fontWeight": true, + "gridArea": true, + "gridColumn": true, + "gridColumnEnd": true, + "gridColumnStart": true, + "gridRow": true, + "gridRowEnd": true, + "gridRowStart": true, + "lineHeight": true, + "opacity": true, + "order": true, + "orphans": true, + "widows": true, + "zIndex": true, + "zoom": true + }, + + // Add in properties whose names you wish to fix before + // setting or getting the value + cssProps: {}, + + // Get and set the style property on a DOM Node + style: function( elem, name, value, extra ) { + + // Don't set styles on text and comment nodes + if ( !elem || elem.nodeType === 3 || elem.nodeType === 8 || !elem.style ) { + return; + } + + // Make sure that we're working with the right name + var ret, type, hooks, + origName = camelCase( name ), + isCustomProp = rcustomProp.test( name ), + style = elem.style; + + // Make sure that we're working with the right name. We don't + // want to query the value if it is a CSS custom property + // since they are user-defined. + if ( !isCustomProp ) { + name = finalPropName( origName ); + } + + // Gets hook for the prefixed version, then unprefixed version + hooks = jQuery.cssHooks[ name ] || jQuery.cssHooks[ origName ]; + + // Check if we're setting a value + if ( value !== undefined ) { + type = typeof value; + + // Convert "+=" or "-=" to relative numbers (#7345) + if ( type === "string" && ( ret = rcssNum.exec( value ) ) && ret[ 1 ] ) { + value = adjustCSS( elem, name, ret ); + + // Fixes bug #9237 + type = "number"; + } + + // Make sure that null and NaN values aren't set (#7116) + if ( value == null || value !== value ) { + return; + } + + // If a number was passed in, add the unit (except for certain CSS properties) + // The isCustomProp check can be removed in jQuery 4.0 when we only auto-append + // "px" to a few hardcoded values. + if ( type === "number" && !isCustomProp ) { + value += ret && ret[ 3 ] || ( jQuery.cssNumber[ origName ] ? "" : "px" ); + } + + // background-* props affect original clone's values + if ( !support.clearCloneStyle && value === "" && name.indexOf( "background" ) === 0 ) { + style[ name ] = "inherit"; + } + + // If a hook was provided, use that value, otherwise just set the specified value + if ( !hooks || !( "set" in hooks ) || + ( value = hooks.set( elem, value, extra ) ) !== undefined ) { + + if ( isCustomProp ) { + style.setProperty( name, value ); + } else { + style[ name ] = value; + } + } + + } else { + + // If a hook was provided get the non-computed value from there + if ( hooks && "get" in hooks && + ( ret = hooks.get( elem, false, extra ) ) !== undefined ) { + + return ret; + } + + // Otherwise just get the value from the style object + return style[ name ]; + } + }, + + css: function( elem, name, extra, styles ) { + var val, num, hooks, + origName = camelCase( name ), + isCustomProp = rcustomProp.test( name ); + + // Make sure that we're working with the right name. We don't + // want to modify the value if it is a CSS custom property + // since they are user-defined. + if ( !isCustomProp ) { + name = finalPropName( origName ); + } + + // Try prefixed name followed by the unprefixed name + hooks = jQuery.cssHooks[ name ] || jQuery.cssHooks[ origName ]; + + // If a hook was provided get the computed value from there + if ( hooks && "get" in hooks ) { + val = hooks.get( elem, true, extra ); + } + + // Otherwise, if a way to get the computed value exists, use that + if ( val === undefined ) { + val = curCSS( elem, name, styles ); + } + + // Convert "normal" to computed value + if ( val === "normal" && name in cssNormalTransform ) { + val = cssNormalTransform[ name ]; + } + + // Make numeric if forced or a qualifier was provided and val looks numeric + if ( extra === "" || extra ) { + num = parseFloat( val ); + return extra === true || isFinite( num ) ? num || 0 : val; + } + + return val; + } +} ); + +jQuery.each( [ "height", "width" ], function( _i, dimension ) { + jQuery.cssHooks[ dimension ] = { + get: function( elem, computed, extra ) { + if ( computed ) { + + // Certain elements can have dimension info if we invisibly show them + // but it must have a current display style that would benefit + return rdisplayswap.test( jQuery.css( elem, "display" ) ) && + + // Support: Safari 8+ + // Table columns in Safari have non-zero offsetWidth & zero + // getBoundingClientRect().width unless display is changed. + // Support: IE <=11 only + // Running getBoundingClientRect on a disconnected node + // in IE throws an error. + ( !elem.getClientRects().length || !elem.getBoundingClientRect().width ) ? + swap( elem, cssShow, function() { + return getWidthOrHeight( elem, dimension, extra ); + } ) : + getWidthOrHeight( elem, dimension, extra ); + } + }, + + set: function( elem, value, extra ) { + var matches, + styles = getStyles( elem ), + + // Only read styles.position if the test has a chance to fail + // to avoid forcing a reflow. + scrollboxSizeBuggy = !support.scrollboxSize() && + styles.position === "absolute", + + // To avoid forcing a reflow, only fetch boxSizing if we need it (gh-3991) + boxSizingNeeded = scrollboxSizeBuggy || extra, + isBorderBox = boxSizingNeeded && + jQuery.css( elem, "boxSizing", false, styles ) === "border-box", + subtract = extra ? + boxModelAdjustment( + elem, + dimension, + extra, + isBorderBox, + styles + ) : + 0; + + // Account for unreliable border-box dimensions by comparing offset* to computed and + // faking a content-box to get border and padding (gh-3699) + if ( isBorderBox && scrollboxSizeBuggy ) { + subtract -= Math.ceil( + elem[ "offset" + dimension[ 0 ].toUpperCase() + dimension.slice( 1 ) ] - + parseFloat( styles[ dimension ] ) - + boxModelAdjustment( elem, dimension, "border", false, styles ) - + 0.5 + ); + } + + // Convert to pixels if value adjustment is needed + if ( subtract && ( matches = rcssNum.exec( value ) ) && + ( matches[ 3 ] || "px" ) !== "px" ) { + + elem.style[ dimension ] = value; + value = jQuery.css( elem, dimension ); + } + + return setPositiveNumber( elem, value, subtract ); + } + }; +} ); + +jQuery.cssHooks.marginLeft = addGetHookIf( support.reliableMarginLeft, + function( elem, computed ) { + if ( computed ) { + return ( parseFloat( curCSS( elem, "marginLeft" ) ) || + elem.getBoundingClientRect().left - + swap( elem, { marginLeft: 0 }, function() { + return elem.getBoundingClientRect().left; + } ) + ) + "px"; + } + } +); + +// These hooks are used by animate to expand properties +jQuery.each( { + margin: "", + padding: "", + border: "Width" +}, function( prefix, suffix ) { + jQuery.cssHooks[ prefix + suffix ] = { + expand: function( value ) { + var i = 0, + expanded = {}, + + // Assumes a single number if not a string + parts = typeof value === "string" ? value.split( " " ) : [ value ]; + + for ( ; i < 4; i++ ) { + expanded[ prefix + cssExpand[ i ] + suffix ] = + parts[ i ] || parts[ i - 2 ] || parts[ 0 ]; + } + + return expanded; + } + }; + + if ( prefix !== "margin" ) { + jQuery.cssHooks[ prefix + suffix ].set = setPositiveNumber; + } +} ); + +jQuery.fn.extend( { + css: function( name, value ) { + return access( this, function( elem, name, value ) { + var styles, len, + map = {}, + i = 0; + + if ( Array.isArray( name ) ) { + styles = getStyles( elem ); + len = name.length; + + for ( ; i < len; i++ ) { + map[ name[ i ] ] = jQuery.css( elem, name[ i ], false, styles ); + } + + return map; + } + + return value !== undefined ? + jQuery.style( elem, name, value ) : + jQuery.css( elem, name ); + }, name, value, arguments.length > 1 ); + } +} ); + + +function Tween( elem, options, prop, end, easing ) { + return new Tween.prototype.init( elem, options, prop, end, easing ); +} +jQuery.Tween = Tween; + +Tween.prototype = { + constructor: Tween, + init: function( elem, options, prop, end, easing, unit ) { + this.elem = elem; + this.prop = prop; + this.easing = easing || jQuery.easing._default; + this.options = options; + this.start = this.now = this.cur(); + this.end = end; + this.unit = unit || ( jQuery.cssNumber[ prop ] ? "" : "px" ); + }, + cur: function() { + var hooks = Tween.propHooks[ this.prop ]; + + return hooks && hooks.get ? + hooks.get( this ) : + Tween.propHooks._default.get( this ); + }, + run: function( percent ) { + var eased, + hooks = Tween.propHooks[ this.prop ]; + + if ( this.options.duration ) { + this.pos = eased = jQuery.easing[ this.easing ]( + percent, this.options.duration * percent, 0, 1, this.options.duration + ); + } else { + this.pos = eased = percent; + } + this.now = ( this.end - this.start ) * eased + this.start; + + if ( this.options.step ) { + this.options.step.call( this.elem, this.now, this ); + } + + if ( hooks && hooks.set ) { + hooks.set( this ); + } else { + Tween.propHooks._default.set( this ); + } + return this; + } +}; + +Tween.prototype.init.prototype = Tween.prototype; + +Tween.propHooks = { + _default: { + get: function( tween ) { + var result; + + // Use a property on the element directly when it is not a DOM element, + // or when there is no matching style property that exists. + if ( tween.elem.nodeType !== 1 || + tween.elem[ tween.prop ] != null && tween.elem.style[ tween.prop ] == null ) { + return tween.elem[ tween.prop ]; + } + + // Passing an empty string as a 3rd parameter to .css will automatically + // attempt a parseFloat and fallback to a string if the parse fails. + // Simple values such as "10px" are parsed to Float; + // complex values such as "rotate(1rad)" are returned as-is. + result = jQuery.css( tween.elem, tween.prop, "" ); + + // Empty strings, null, undefined and "auto" are converted to 0. + return !result || result === "auto" ? 0 : result; + }, + set: function( tween ) { + + // Use step hook for back compat. + // Use cssHook if its there. + // Use .style if available and use plain properties where available. + if ( jQuery.fx.step[ tween.prop ] ) { + jQuery.fx.step[ tween.prop ]( tween ); + } else if ( tween.elem.nodeType === 1 && ( + jQuery.cssHooks[ tween.prop ] || + tween.elem.style[ finalPropName( tween.prop ) ] != null ) ) { + jQuery.style( tween.elem, tween.prop, tween.now + tween.unit ); + } else { + tween.elem[ tween.prop ] = tween.now; + } + } + } +}; + +// Support: IE <=9 only +// Panic based approach to setting things on disconnected nodes +Tween.propHooks.scrollTop = Tween.propHooks.scrollLeft = { + set: function( tween ) { + if ( tween.elem.nodeType && tween.elem.parentNode ) { + tween.elem[ tween.prop ] = tween.now; + } + } +}; + +jQuery.easing = { + linear: function( p ) { + return p; + }, + swing: function( p ) { + return 0.5 - Math.cos( p * Math.PI ) / 2; + }, + _default: "swing" +}; + +jQuery.fx = Tween.prototype.init; + +// Back compat <1.8 extension point +jQuery.fx.step = {}; + + + + +var + fxNow, inProgress, + rfxtypes = /^(?:toggle|show|hide)$/, + rrun = /queueHooks$/; + +function schedule() { + if ( inProgress ) { + if ( document.hidden === false && window.requestAnimationFrame ) { + window.requestAnimationFrame( schedule ); + } else { + window.setTimeout( schedule, jQuery.fx.interval ); + } + + jQuery.fx.tick(); + } +} + +// Animations created synchronously will run synchronously +function createFxNow() { + window.setTimeout( function() { + fxNow = undefined; + } ); + return ( fxNow = Date.now() ); +} + +// Generate parameters to create a standard animation +function genFx( type, includeWidth ) { + var which, + i = 0, + attrs = { height: type }; + + // If we include width, step value is 1 to do all cssExpand values, + // otherwise step value is 2 to skip over Left and Right + includeWidth = includeWidth ? 1 : 0; + for ( ; i < 4; i += 2 - includeWidth ) { + which = cssExpand[ i ]; + attrs[ "margin" + which ] = attrs[ "padding" + which ] = type; + } + + if ( includeWidth ) { + attrs.opacity = attrs.width = type; + } + + return attrs; +} + +function createTween( value, prop, animation ) { + var tween, + collection = ( Animation.tweeners[ prop ] || [] ).concat( Animation.tweeners[ "*" ] ), + index = 0, + length = collection.length; + for ( ; index < length; index++ ) { + if ( ( tween = collection[ index ].call( animation, prop, value ) ) ) { + + // We're done with this property + return tween; + } + } +} + +function defaultPrefilter( elem, props, opts ) { + var prop, value, toggle, hooks, oldfire, propTween, restoreDisplay, display, + isBox = "width" in props || "height" in props, + anim = this, + orig = {}, + style = elem.style, + hidden = elem.nodeType && isHiddenWithinTree( elem ), + dataShow = dataPriv.get( elem, "fxshow" ); + + // Queue-skipping animations hijack the fx hooks + if ( !opts.queue ) { + hooks = jQuery._queueHooks( elem, "fx" ); + if ( hooks.unqueued == null ) { + hooks.unqueued = 0; + oldfire = hooks.empty.fire; + hooks.empty.fire = function() { + if ( !hooks.unqueued ) { + oldfire(); + } + }; + } + hooks.unqueued++; + + anim.always( function() { + + // Ensure the complete handler is called before this completes + anim.always( function() { + hooks.unqueued--; + if ( !jQuery.queue( elem, "fx" ).length ) { + hooks.empty.fire(); + } + } ); + } ); + } + + // Detect show/hide animations + for ( prop in props ) { + value = props[ prop ]; + if ( rfxtypes.test( value ) ) { + delete props[ prop ]; + toggle = toggle || value === "toggle"; + if ( value === ( hidden ? "hide" : "show" ) ) { + + // Pretend to be hidden if this is a "show" and + // there is still data from a stopped show/hide + if ( value === "show" && dataShow && dataShow[ prop ] !== undefined ) { + hidden = true; + + // Ignore all other no-op show/hide data + } else { + continue; + } + } + orig[ prop ] = dataShow && dataShow[ prop ] || jQuery.style( elem, prop ); + } + } + + // Bail out if this is a no-op like .hide().hide() + propTween = !jQuery.isEmptyObject( props ); + if ( !propTween && jQuery.isEmptyObject( orig ) ) { + return; + } + + // Restrict "overflow" and "display" styles during box animations + if ( isBox && elem.nodeType === 1 ) { + + // Support: IE <=9 - 11, Edge 12 - 15 + // Record all 3 overflow attributes because IE does not infer the shorthand + // from identically-valued overflowX and overflowY and Edge just mirrors + // the overflowX value there. + opts.overflow = [ style.overflow, style.overflowX, style.overflowY ]; + + // Identify a display type, preferring old show/hide data over the CSS cascade + restoreDisplay = dataShow && dataShow.display; + if ( restoreDisplay == null ) { + restoreDisplay = dataPriv.get( elem, "display" ); + } + display = jQuery.css( elem, "display" ); + if ( display === "none" ) { + if ( restoreDisplay ) { + display = restoreDisplay; + } else { + + // Get nonempty value(s) by temporarily forcing visibility + showHide( [ elem ], true ); + restoreDisplay = elem.style.display || restoreDisplay; + display = jQuery.css( elem, "display" ); + showHide( [ elem ] ); + } + } + + // Animate inline elements as inline-block + if ( display === "inline" || display === "inline-block" && restoreDisplay != null ) { + if ( jQuery.css( elem, "float" ) === "none" ) { + + // Restore the original display value at the end of pure show/hide animations + if ( !propTween ) { + anim.done( function() { + style.display = restoreDisplay; + } ); + if ( restoreDisplay == null ) { + display = style.display; + restoreDisplay = display === "none" ? "" : display; + } + } + style.display = "inline-block"; + } + } + } + + if ( opts.overflow ) { + style.overflow = "hidden"; + anim.always( function() { + style.overflow = opts.overflow[ 0 ]; + style.overflowX = opts.overflow[ 1 ]; + style.overflowY = opts.overflow[ 2 ]; + } ); + } + + // Implement show/hide animations + propTween = false; + for ( prop in orig ) { + + // General show/hide setup for this element animation + if ( !propTween ) { + if ( dataShow ) { + if ( "hidden" in dataShow ) { + hidden = dataShow.hidden; + } + } else { + dataShow = dataPriv.access( elem, "fxshow", { display: restoreDisplay } ); + } + + // Store hidden/visible for toggle so `.stop().toggle()` "reverses" + if ( toggle ) { + dataShow.hidden = !hidden; + } + + // Show elements before animating them + if ( hidden ) { + showHide( [ elem ], true ); + } + + /* eslint-disable no-loop-func */ + + anim.done( function() { + + /* eslint-enable no-loop-func */ + + // The final step of a "hide" animation is actually hiding the element + if ( !hidden ) { + showHide( [ elem ] ); + } + dataPriv.remove( elem, "fxshow" ); + for ( prop in orig ) { + jQuery.style( elem, prop, orig[ prop ] ); + } + } ); + } + + // Per-property setup + propTween = createTween( hidden ? dataShow[ prop ] : 0, prop, anim ); + if ( !( prop in dataShow ) ) { + dataShow[ prop ] = propTween.start; + if ( hidden ) { + propTween.end = propTween.start; + propTween.start = 0; + } + } + } +} + +function propFilter( props, specialEasing ) { + var index, name, easing, value, hooks; + + // camelCase, specialEasing and expand cssHook pass + for ( index in props ) { + name = camelCase( index ); + easing = specialEasing[ name ]; + value = props[ index ]; + if ( Array.isArray( value ) ) { + easing = value[ 1 ]; + value = props[ index ] = value[ 0 ]; + } + + if ( index !== name ) { + props[ name ] = value; + delete props[ index ]; + } + + hooks = jQuery.cssHooks[ name ]; + if ( hooks && "expand" in hooks ) { + value = hooks.expand( value ); + delete props[ name ]; + + // Not quite $.extend, this won't overwrite existing keys. + // Reusing 'index' because we have the correct "name" + for ( index in value ) { + if ( !( index in props ) ) { + props[ index ] = value[ index ]; + specialEasing[ index ] = easing; + } + } + } else { + specialEasing[ name ] = easing; + } + } +} + +function Animation( elem, properties, options ) { + var result, + stopped, + index = 0, + length = Animation.prefilters.length, + deferred = jQuery.Deferred().always( function() { + + // Don't match elem in the :animated selector + delete tick.elem; + } ), + tick = function() { + if ( stopped ) { + return false; + } + var currentTime = fxNow || createFxNow(), + remaining = Math.max( 0, animation.startTime + animation.duration - currentTime ), + + // Support: Android 2.3 only + // Archaic crash bug won't allow us to use `1 - ( 0.5 || 0 )` (#12497) + temp = remaining / animation.duration || 0, + percent = 1 - temp, + index = 0, + length = animation.tweens.length; + + for ( ; index < length; index++ ) { + animation.tweens[ index ].run( percent ); + } + + deferred.notifyWith( elem, [ animation, percent, remaining ] ); + + // If there's more to do, yield + if ( percent < 1 && length ) { + return remaining; + } + + // If this was an empty animation, synthesize a final progress notification + if ( !length ) { + deferred.notifyWith( elem, [ animation, 1, 0 ] ); + } + + // Resolve the animation and report its conclusion + deferred.resolveWith( elem, [ animation ] ); + return false; + }, + animation = deferred.promise( { + elem: elem, + props: jQuery.extend( {}, properties ), + opts: jQuery.extend( true, { + specialEasing: {}, + easing: jQuery.easing._default + }, options ), + originalProperties: properties, + originalOptions: options, + startTime: fxNow || createFxNow(), + duration: options.duration, + tweens: [], + createTween: function( prop, end ) { + var tween = jQuery.Tween( elem, animation.opts, prop, end, + animation.opts.specialEasing[ prop ] || animation.opts.easing ); + animation.tweens.push( tween ); + return tween; + }, + stop: function( gotoEnd ) { + var index = 0, + + // If we are going to the end, we want to run all the tweens + // otherwise we skip this part + length = gotoEnd ? animation.tweens.length : 0; + if ( stopped ) { + return this; + } + stopped = true; + for ( ; index < length; index++ ) { + animation.tweens[ index ].run( 1 ); + } + + // Resolve when we played the last frame; otherwise, reject + if ( gotoEnd ) { + deferred.notifyWith( elem, [ animation, 1, 0 ] ); + deferred.resolveWith( elem, [ animation, gotoEnd ] ); + } else { + deferred.rejectWith( elem, [ animation, gotoEnd ] ); + } + return this; + } + } ), + props = animation.props; + + propFilter( props, animation.opts.specialEasing ); + + for ( ; index < length; index++ ) { + result = Animation.prefilters[ index ].call( animation, elem, props, animation.opts ); + if ( result ) { + if ( isFunction( result.stop ) ) { + jQuery._queueHooks( animation.elem, animation.opts.queue ).stop = + result.stop.bind( result ); + } + return result; + } + } + + jQuery.map( props, createTween, animation ); + + if ( isFunction( animation.opts.start ) ) { + animation.opts.start.call( elem, animation ); + } + + // Attach callbacks from options + animation + .progress( animation.opts.progress ) + .done( animation.opts.done, animation.opts.complete ) + .fail( animation.opts.fail ) + .always( animation.opts.always ); + + jQuery.fx.timer( + jQuery.extend( tick, { + elem: elem, + anim: animation, + queue: animation.opts.queue + } ) + ); + + return animation; +} + +jQuery.Animation = jQuery.extend( Animation, { + + tweeners: { + "*": [ function( prop, value ) { + var tween = this.createTween( prop, value ); + adjustCSS( tween.elem, prop, rcssNum.exec( value ), tween ); + return tween; + } ] + }, + + tweener: function( props, callback ) { + if ( isFunction( props ) ) { + callback = props; + props = [ "*" ]; + } else { + props = props.match( rnothtmlwhite ); + } + + var prop, + index = 0, + length = props.length; + + for ( ; index < length; index++ ) { + prop = props[ index ]; + Animation.tweeners[ prop ] = Animation.tweeners[ prop ] || []; + Animation.tweeners[ prop ].unshift( callback ); + } + }, + + prefilters: [ defaultPrefilter ], + + prefilter: function( callback, prepend ) { + if ( prepend ) { + Animation.prefilters.unshift( callback ); + } else { + Animation.prefilters.push( callback ); + } + } +} ); + +jQuery.speed = function( speed, easing, fn ) { + var opt = speed && typeof speed === "object" ? jQuery.extend( {}, speed ) : { + complete: fn || !fn && easing || + isFunction( speed ) && speed, + duration: speed, + easing: fn && easing || easing && !isFunction( easing ) && easing + }; + + // Go to the end state if fx are off + if ( jQuery.fx.off ) { + opt.duration = 0; + + } else { + if ( typeof opt.duration !== "number" ) { + if ( opt.duration in jQuery.fx.speeds ) { + opt.duration = jQuery.fx.speeds[ opt.duration ]; + + } else { + opt.duration = jQuery.fx.speeds._default; + } + } + } + + // Normalize opt.queue - true/undefined/null -> "fx" + if ( opt.queue == null || opt.queue === true ) { + opt.queue = "fx"; + } + + // Queueing + opt.old = opt.complete; + + opt.complete = function() { + if ( isFunction( opt.old ) ) { + opt.old.call( this ); + } + + if ( opt.queue ) { + jQuery.dequeue( this, opt.queue ); + } + }; + + return opt; +}; + +jQuery.fn.extend( { + fadeTo: function( speed, to, easing, callback ) { + + // Show any hidden elements after setting opacity to 0 + return this.filter( isHiddenWithinTree ).css( "opacity", 0 ).show() + + // Animate to the value specified + .end().animate( { opacity: to }, speed, easing, callback ); + }, + animate: function( prop, speed, easing, callback ) { + var empty = jQuery.isEmptyObject( prop ), + optall = jQuery.speed( speed, easing, callback ), + doAnimation = function() { + + // Operate on a copy of prop so per-property easing won't be lost + var anim = Animation( this, jQuery.extend( {}, prop ), optall ); + + // Empty animations, or finishing resolves immediately + if ( empty || dataPriv.get( this, "finish" ) ) { + anim.stop( true ); + } + }; + + doAnimation.finish = doAnimation; + + return empty || optall.queue === false ? + this.each( doAnimation ) : + this.queue( optall.queue, doAnimation ); + }, + stop: function( type, clearQueue, gotoEnd ) { + var stopQueue = function( hooks ) { + var stop = hooks.stop; + delete hooks.stop; + stop( gotoEnd ); + }; + + if ( typeof type !== "string" ) { + gotoEnd = clearQueue; + clearQueue = type; + type = undefined; + } + if ( clearQueue ) { + this.queue( type || "fx", [] ); + } + + return this.each( function() { + var dequeue = true, + index = type != null && type + "queueHooks", + timers = jQuery.timers, + data = dataPriv.get( this ); + + if ( index ) { + if ( data[ index ] && data[ index ].stop ) { + stopQueue( data[ index ] ); + } + } else { + for ( index in data ) { + if ( data[ index ] && data[ index ].stop && rrun.test( index ) ) { + stopQueue( data[ index ] ); + } + } + } + + for ( index = timers.length; index--; ) { + if ( timers[ index ].elem === this && + ( type == null || timers[ index ].queue === type ) ) { + + timers[ index ].anim.stop( gotoEnd ); + dequeue = false; + timers.splice( index, 1 ); + } + } + + // Start the next in the queue if the last step wasn't forced. + // Timers currently will call their complete callbacks, which + // will dequeue but only if they were gotoEnd. + if ( dequeue || !gotoEnd ) { + jQuery.dequeue( this, type ); + } + } ); + }, + finish: function( type ) { + if ( type !== false ) { + type = type || "fx"; + } + return this.each( function() { + var index, + data = dataPriv.get( this ), + queue = data[ type + "queue" ], + hooks = data[ type + "queueHooks" ], + timers = jQuery.timers, + length = queue ? queue.length : 0; + + // Enable finishing flag on private data + data.finish = true; + + // Empty the queue first + jQuery.queue( this, type, [] ); + + if ( hooks && hooks.stop ) { + hooks.stop.call( this, true ); + } + + // Look for any active animations, and finish them + for ( index = timers.length; index--; ) { + if ( timers[ index ].elem === this && timers[ index ].queue === type ) { + timers[ index ].anim.stop( true ); + timers.splice( index, 1 ); + } + } + + // Look for any animations in the old queue and finish them + for ( index = 0; index < length; index++ ) { + if ( queue[ index ] && queue[ index ].finish ) { + queue[ index ].finish.call( this ); + } + } + + // Turn off finishing flag + delete data.finish; + } ); + } +} ); + +jQuery.each( [ "toggle", "show", "hide" ], function( _i, name ) { + var cssFn = jQuery.fn[ name ]; + jQuery.fn[ name ] = function( speed, easing, callback ) { + return speed == null || typeof speed === "boolean" ? + cssFn.apply( this, arguments ) : + this.animate( genFx( name, true ), speed, easing, callback ); + }; +} ); + +// Generate shortcuts for custom animations +jQuery.each( { + slideDown: genFx( "show" ), + slideUp: genFx( "hide" ), + slideToggle: genFx( "toggle" ), + fadeIn: { opacity: "show" }, + fadeOut: { opacity: "hide" }, + fadeToggle: { opacity: "toggle" } +}, function( name, props ) { + jQuery.fn[ name ] = function( speed, easing, callback ) { + return this.animate( props, speed, easing, callback ); + }; +} ); + +jQuery.timers = []; +jQuery.fx.tick = function() { + var timer, + i = 0, + timers = jQuery.timers; + + fxNow = Date.now(); + + for ( ; i < timers.length; i++ ) { + timer = timers[ i ]; + + // Run the timer and safely remove it when done (allowing for external removal) + if ( !timer() && timers[ i ] === timer ) { + timers.splice( i--, 1 ); + } + } + + if ( !timers.length ) { + jQuery.fx.stop(); + } + fxNow = undefined; +}; + +jQuery.fx.timer = function( timer ) { + jQuery.timers.push( timer ); + jQuery.fx.start(); +}; + +jQuery.fx.interval = 13; +jQuery.fx.start = function() { + if ( inProgress ) { + return; + } + + inProgress = true; + schedule(); +}; + +jQuery.fx.stop = function() { + inProgress = null; +}; + +jQuery.fx.speeds = { + slow: 600, + fast: 200, + + // Default speed + _default: 400 +}; + + +// Based off of the plugin by Clint Helfers, with permission. +// https://web.archive.org/web/20100324014747/http://blindsignals.com/index.php/2009/07/jquery-delay/ +jQuery.fn.delay = function( time, type ) { + time = jQuery.fx ? jQuery.fx.speeds[ time ] || time : time; + type = type || "fx"; + + return this.queue( type, function( next, hooks ) { + var timeout = window.setTimeout( next, time ); + hooks.stop = function() { + window.clearTimeout( timeout ); + }; + } ); +}; + + +( function() { + var input = document.createElement( "input" ), + select = document.createElement( "select" ), + opt = select.appendChild( document.createElement( "option" ) ); + + input.type = "checkbox"; + + // Support: Android <=4.3 only + // Default value for a checkbox should be "on" + support.checkOn = input.value !== ""; + + // Support: IE <=11 only + // Must access selectedIndex to make default options select + support.optSelected = opt.selected; + + // Support: IE <=11 only + // An input loses its value after becoming a radio + input = document.createElement( "input" ); + input.value = "t"; + input.type = "radio"; + support.radioValue = input.value === "t"; +} )(); + + +var boolHook, + attrHandle = jQuery.expr.attrHandle; + +jQuery.fn.extend( { + attr: function( name, value ) { + return access( this, jQuery.attr, name, value, arguments.length > 1 ); + }, + + removeAttr: function( name ) { + return this.each( function() { + jQuery.removeAttr( this, name ); + } ); + } +} ); + +jQuery.extend( { + attr: function( elem, name, value ) { + var ret, hooks, + nType = elem.nodeType; + + // Don't get/set attributes on text, comment and attribute nodes + if ( nType === 3 || nType === 8 || nType === 2 ) { + return; + } + + // Fallback to prop when attributes are not supported + if ( typeof elem.getAttribute === "undefined" ) { + return jQuery.prop( elem, name, value ); + } + + // Attribute hooks are determined by the lowercase version + // Grab necessary hook if one is defined + if ( nType !== 1 || !jQuery.isXMLDoc( elem ) ) { + hooks = jQuery.attrHooks[ name.toLowerCase() ] || + ( jQuery.expr.match.bool.test( name ) ? boolHook : undefined ); + } + + if ( value !== undefined ) { + if ( value === null ) { + jQuery.removeAttr( elem, name ); + return; + } + + if ( hooks && "set" in hooks && + ( ret = hooks.set( elem, value, name ) ) !== undefined ) { + return ret; + } + + elem.setAttribute( name, value + "" ); + return value; + } + + if ( hooks && "get" in hooks && ( ret = hooks.get( elem, name ) ) !== null ) { + return ret; + } + + ret = jQuery.find.attr( elem, name ); + + // Non-existent attributes return null, we normalize to undefined + return ret == null ? undefined : ret; + }, + + attrHooks: { + type: { + set: function( elem, value ) { + if ( !support.radioValue && value === "radio" && + nodeName( elem, "input" ) ) { + var val = elem.value; + elem.setAttribute( "type", value ); + if ( val ) { + elem.value = val; + } + return value; + } + } + } + }, + + removeAttr: function( elem, value ) { + var name, + i = 0, + + // Attribute names can contain non-HTML whitespace characters + // https://html.spec.whatwg.org/multipage/syntax.html#attributes-2 + attrNames = value && value.match( rnothtmlwhite ); + + if ( attrNames && elem.nodeType === 1 ) { + while ( ( name = attrNames[ i++ ] ) ) { + elem.removeAttribute( name ); + } + } + } +} ); + +// Hooks for boolean attributes +boolHook = { + set: function( elem, value, name ) { + if ( value === false ) { + + // Remove boolean attributes when set to false + jQuery.removeAttr( elem, name ); + } else { + elem.setAttribute( name, name ); + } + return name; + } +}; + +jQuery.each( jQuery.expr.match.bool.source.match( /\w+/g ), function( _i, name ) { + var getter = attrHandle[ name ] || jQuery.find.attr; + + attrHandle[ name ] = function( elem, name, isXML ) { + var ret, handle, + lowercaseName = name.toLowerCase(); + + if ( !isXML ) { + + // Avoid an infinite loop by temporarily removing this function from the getter + handle = attrHandle[ lowercaseName ]; + attrHandle[ lowercaseName ] = ret; + ret = getter( elem, name, isXML ) != null ? + lowercaseName : + null; + attrHandle[ lowercaseName ] = handle; + } + return ret; + }; +} ); + + + + +var rfocusable = /^(?:input|select|textarea|button)$/i, + rclickable = /^(?:a|area)$/i; + +jQuery.fn.extend( { + prop: function( name, value ) { + return access( this, jQuery.prop, name, value, arguments.length > 1 ); + }, + + removeProp: function( name ) { + return this.each( function() { + delete this[ jQuery.propFix[ name ] || name ]; + } ); + } +} ); + +jQuery.extend( { + prop: function( elem, name, value ) { + var ret, hooks, + nType = elem.nodeType; + + // Don't get/set properties on text, comment and attribute nodes + if ( nType === 3 || nType === 8 || nType === 2 ) { + return; + } + + if ( nType !== 1 || !jQuery.isXMLDoc( elem ) ) { + + // Fix name and attach hooks + name = jQuery.propFix[ name ] || name; + hooks = jQuery.propHooks[ name ]; + } + + if ( value !== undefined ) { + if ( hooks && "set" in hooks && + ( ret = hooks.set( elem, value, name ) ) !== undefined ) { + return ret; + } + + return ( elem[ name ] = value ); + } + + if ( hooks && "get" in hooks && ( ret = hooks.get( elem, name ) ) !== null ) { + return ret; + } + + return elem[ name ]; + }, + + propHooks: { + tabIndex: { + get: function( elem ) { + + // Support: IE <=9 - 11 only + // elem.tabIndex doesn't always return the + // correct value when it hasn't been explicitly set + // https://web.archive.org/web/20141116233347/http://fluidproject.org/blog/2008/01/09/getting-setting-and-removing-tabindex-values-with-javascript/ + // Use proper attribute retrieval(#12072) + var tabindex = jQuery.find.attr( elem, "tabindex" ); + + if ( tabindex ) { + return parseInt( tabindex, 10 ); + } + + if ( + rfocusable.test( elem.nodeName ) || + rclickable.test( elem.nodeName ) && + elem.href + ) { + return 0; + } + + return -1; + } + } + }, + + propFix: { + "for": "htmlFor", + "class": "className" + } +} ); + +// Support: IE <=11 only +// Accessing the selectedIndex property +// forces the browser to respect setting selected +// on the option +// The getter ensures a default option is selected +// when in an optgroup +// eslint rule "no-unused-expressions" is disabled for this code +// since it considers such accessions noop +if ( !support.optSelected ) { + jQuery.propHooks.selected = { + get: function( elem ) { + + /* eslint no-unused-expressions: "off" */ + + var parent = elem.parentNode; + if ( parent && parent.parentNode ) { + parent.parentNode.selectedIndex; + } + return null; + }, + set: function( elem ) { + + /* eslint no-unused-expressions: "off" */ + + var parent = elem.parentNode; + if ( parent ) { + parent.selectedIndex; + + if ( parent.parentNode ) { + parent.parentNode.selectedIndex; + } + } + } + }; +} + +jQuery.each( [ + "tabIndex", + "readOnly", + "maxLength", + "cellSpacing", + "cellPadding", + "rowSpan", + "colSpan", + "useMap", + "frameBorder", + "contentEditable" +], function() { + jQuery.propFix[ this.toLowerCase() ] = this; +} ); + + + + + // Strip and collapse whitespace according to HTML spec + // https://infra.spec.whatwg.org/#strip-and-collapse-ascii-whitespace + function stripAndCollapse( value ) { + var tokens = value.match( rnothtmlwhite ) || []; + return tokens.join( " " ); + } + + +function getClass( elem ) { + return elem.getAttribute && elem.getAttribute( "class" ) || ""; +} + +function classesToArray( value ) { + if ( Array.isArray( value ) ) { + return value; + } + if ( typeof value === "string" ) { + return value.match( rnothtmlwhite ) || []; + } + return []; +} + +jQuery.fn.extend( { + addClass: function( value ) { + var classes, elem, cur, curValue, clazz, j, finalValue, + i = 0; + + if ( isFunction( value ) ) { + return this.each( function( j ) { + jQuery( this ).addClass( value.call( this, j, getClass( this ) ) ); + } ); + } + + classes = classesToArray( value ); + + if ( classes.length ) { + while ( ( elem = this[ i++ ] ) ) { + curValue = getClass( elem ); + cur = elem.nodeType === 1 && ( " " + stripAndCollapse( curValue ) + " " ); + + if ( cur ) { + j = 0; + while ( ( clazz = classes[ j++ ] ) ) { + if ( cur.indexOf( " " + clazz + " " ) < 0 ) { + cur += clazz + " "; + } + } + + // Only assign if different to avoid unneeded rendering. + finalValue = stripAndCollapse( cur ); + if ( curValue !== finalValue ) { + elem.setAttribute( "class", finalValue ); + } + } + } + } + + return this; + }, + + removeClass: function( value ) { + var classes, elem, cur, curValue, clazz, j, finalValue, + i = 0; + + if ( isFunction( value ) ) { + return this.each( function( j ) { + jQuery( this ).removeClass( value.call( this, j, getClass( this ) ) ); + } ); + } + + if ( !arguments.length ) { + return this.attr( "class", "" ); + } + + classes = classesToArray( value ); + + if ( classes.length ) { + while ( ( elem = this[ i++ ] ) ) { + curValue = getClass( elem ); + + // This expression is here for better compressibility (see addClass) + cur = elem.nodeType === 1 && ( " " + stripAndCollapse( curValue ) + " " ); + + if ( cur ) { + j = 0; + while ( ( clazz = classes[ j++ ] ) ) { + + // Remove *all* instances + while ( cur.indexOf( " " + clazz + " " ) > -1 ) { + cur = cur.replace( " " + clazz + " ", " " ); + } + } + + // Only assign if different to avoid unneeded rendering. + finalValue = stripAndCollapse( cur ); + if ( curValue !== finalValue ) { + elem.setAttribute( "class", finalValue ); + } + } + } + } + + return this; + }, + + toggleClass: function( value, stateVal ) { + var type = typeof value, + isValidValue = type === "string" || Array.isArray( value ); + + if ( typeof stateVal === "boolean" && isValidValue ) { + return stateVal ? this.addClass( value ) : this.removeClass( value ); + } + + if ( isFunction( value ) ) { + return this.each( function( i ) { + jQuery( this ).toggleClass( + value.call( this, i, getClass( this ), stateVal ), + stateVal + ); + } ); + } + + return this.each( function() { + var className, i, self, classNames; + + if ( isValidValue ) { + + // Toggle individual class names + i = 0; + self = jQuery( this ); + classNames = classesToArray( value ); + + while ( ( className = classNames[ i++ ] ) ) { + + // Check each className given, space separated list + if ( self.hasClass( className ) ) { + self.removeClass( className ); + } else { + self.addClass( className ); + } + } + + // Toggle whole class name + } else if ( value === undefined || type === "boolean" ) { + className = getClass( this ); + if ( className ) { + + // Store className if set + dataPriv.set( this, "__className__", className ); + } + + // If the element has a class name or if we're passed `false`, + // then remove the whole classname (if there was one, the above saved it). + // Otherwise bring back whatever was previously saved (if anything), + // falling back to the empty string if nothing was stored. + if ( this.setAttribute ) { + this.setAttribute( "class", + className || value === false ? + "" : + dataPriv.get( this, "__className__" ) || "" + ); + } + } + } ); + }, + + hasClass: function( selector ) { + var className, elem, + i = 0; + + className = " " + selector + " "; + while ( ( elem = this[ i++ ] ) ) { + if ( elem.nodeType === 1 && + ( " " + stripAndCollapse( getClass( elem ) ) + " " ).indexOf( className ) > -1 ) { + return true; + } + } + + return false; + } +} ); + + + + +var rreturn = /\r/g; + +jQuery.fn.extend( { + val: function( value ) { + var hooks, ret, valueIsFunction, + elem = this[ 0 ]; + + if ( !arguments.length ) { + if ( elem ) { + hooks = jQuery.valHooks[ elem.type ] || + jQuery.valHooks[ elem.nodeName.toLowerCase() ]; + + if ( hooks && + "get" in hooks && + ( ret = hooks.get( elem, "value" ) ) !== undefined + ) { + return ret; + } + + ret = elem.value; + + // Handle most common string cases + if ( typeof ret === "string" ) { + return ret.replace( rreturn, "" ); + } + + // Handle cases where value is null/undef or number + return ret == null ? "" : ret; + } + + return; + } + + valueIsFunction = isFunction( value ); + + return this.each( function( i ) { + var val; + + if ( this.nodeType !== 1 ) { + return; + } + + if ( valueIsFunction ) { + val = value.call( this, i, jQuery( this ).val() ); + } else { + val = value; + } + + // Treat null/undefined as ""; convert numbers to string + if ( val == null ) { + val = ""; + + } else if ( typeof val === "number" ) { + val += ""; + + } else if ( Array.isArray( val ) ) { + val = jQuery.map( val, function( value ) { + return value == null ? "" : value + ""; + } ); + } + + hooks = jQuery.valHooks[ this.type ] || jQuery.valHooks[ this.nodeName.toLowerCase() ]; + + // If set returns undefined, fall back to normal setting + if ( !hooks || !( "set" in hooks ) || hooks.set( this, val, "value" ) === undefined ) { + this.value = val; + } + } ); + } +} ); + +jQuery.extend( { + valHooks: { + option: { + get: function( elem ) { + + var val = jQuery.find.attr( elem, "value" ); + return val != null ? + val : + + // Support: IE <=10 - 11 only + // option.text throws exceptions (#14686, #14858) + // Strip and collapse whitespace + // https://html.spec.whatwg.org/#strip-and-collapse-whitespace + stripAndCollapse( jQuery.text( elem ) ); + } + }, + select: { + get: function( elem ) { + var value, option, i, + options = elem.options, + index = elem.selectedIndex, + one = elem.type === "select-one", + values = one ? null : [], + max = one ? index + 1 : options.length; + + if ( index < 0 ) { + i = max; + + } else { + i = one ? index : 0; + } + + // Loop through all the selected options + for ( ; i < max; i++ ) { + option = options[ i ]; + + // Support: IE <=9 only + // IE8-9 doesn't update selected after form reset (#2551) + if ( ( option.selected || i === index ) && + + // Don't return options that are disabled or in a disabled optgroup + !option.disabled && + ( !option.parentNode.disabled || + !nodeName( option.parentNode, "optgroup" ) ) ) { + + // Get the specific value for the option + value = jQuery( option ).val(); + + // We don't need an array for one selects + if ( one ) { + return value; + } + + // Multi-Selects return an array + values.push( value ); + } + } + + return values; + }, + + set: function( elem, value ) { + var optionSet, option, + options = elem.options, + values = jQuery.makeArray( value ), + i = options.length; + + while ( i-- ) { + option = options[ i ]; + + /* eslint-disable no-cond-assign */ + + if ( option.selected = + jQuery.inArray( jQuery.valHooks.option.get( option ), values ) > -1 + ) { + optionSet = true; + } + + /* eslint-enable no-cond-assign */ + } + + // Force browsers to behave consistently when non-matching value is set + if ( !optionSet ) { + elem.selectedIndex = -1; + } + return values; + } + } + } +} ); + +// Radios and checkboxes getter/setter +jQuery.each( [ "radio", "checkbox" ], function() { + jQuery.valHooks[ this ] = { + set: function( elem, value ) { + if ( Array.isArray( value ) ) { + return ( elem.checked = jQuery.inArray( jQuery( elem ).val(), value ) > -1 ); + } + } + }; + if ( !support.checkOn ) { + jQuery.valHooks[ this ].get = function( elem ) { + return elem.getAttribute( "value" ) === null ? "on" : elem.value; + }; + } +} ); + + + + +// Return jQuery for attributes-only inclusion + + +support.focusin = "onfocusin" in window; + + +var rfocusMorph = /^(?:focusinfocus|focusoutblur)$/, + stopPropagationCallback = function( e ) { + e.stopPropagation(); + }; + +jQuery.extend( jQuery.event, { + + trigger: function( event, data, elem, onlyHandlers ) { + + var i, cur, tmp, bubbleType, ontype, handle, special, lastElement, + eventPath = [ elem || document ], + type = hasOwn.call( event, "type" ) ? event.type : event, + namespaces = hasOwn.call( event, "namespace" ) ? event.namespace.split( "." ) : []; + + cur = lastElement = tmp = elem = elem || document; + + // Don't do events on text and comment nodes + if ( elem.nodeType === 3 || elem.nodeType === 8 ) { + return; + } + + // focus/blur morphs to focusin/out; ensure we're not firing them right now + if ( rfocusMorph.test( type + jQuery.event.triggered ) ) { + return; + } + + if ( type.indexOf( "." ) > -1 ) { + + // Namespaced trigger; create a regexp to match event type in handle() + namespaces = type.split( "." ); + type = namespaces.shift(); + namespaces.sort(); + } + ontype = type.indexOf( ":" ) < 0 && "on" + type; + + // Caller can pass in a jQuery.Event object, Object, or just an event type string + event = event[ jQuery.expando ] ? + event : + new jQuery.Event( type, typeof event === "object" && event ); + + // Trigger bitmask: & 1 for native handlers; & 2 for jQuery (always true) + event.isTrigger = onlyHandlers ? 2 : 3; + event.namespace = namespaces.join( "." ); + event.rnamespace = event.namespace ? + new RegExp( "(^|\\.)" + namespaces.join( "\\.(?:.*\\.|)" ) + "(\\.|$)" ) : + null; + + // Clean up the event in case it is being reused + event.result = undefined; + if ( !event.target ) { + event.target = elem; + } + + // Clone any incoming data and prepend the event, creating the handler arg list + data = data == null ? + [ event ] : + jQuery.makeArray( data, [ event ] ); + + // Allow special events to draw outside the lines + special = jQuery.event.special[ type ] || {}; + if ( !onlyHandlers && special.trigger && special.trigger.apply( elem, data ) === false ) { + return; + } + + // Determine event propagation path in advance, per W3C events spec (#9951) + // Bubble up to document, then to window; watch for a global ownerDocument var (#9724) + if ( !onlyHandlers && !special.noBubble && !isWindow( elem ) ) { + + bubbleType = special.delegateType || type; + if ( !rfocusMorph.test( bubbleType + type ) ) { + cur = cur.parentNode; + } + for ( ; cur; cur = cur.parentNode ) { + eventPath.push( cur ); + tmp = cur; + } + + // Only add window if we got to document (e.g., not plain obj or detached DOM) + if ( tmp === ( elem.ownerDocument || document ) ) { + eventPath.push( tmp.defaultView || tmp.parentWindow || window ); + } + } + + // Fire handlers on the event path + i = 0; + while ( ( cur = eventPath[ i++ ] ) && !event.isPropagationStopped() ) { + lastElement = cur; + event.type = i > 1 ? + bubbleType : + special.bindType || type; + + // jQuery handler + handle = ( dataPriv.get( cur, "events" ) || Object.create( null ) )[ event.type ] && + dataPriv.get( cur, "handle" ); + if ( handle ) { + handle.apply( cur, data ); + } + + // Native handler + handle = ontype && cur[ ontype ]; + if ( handle && handle.apply && acceptData( cur ) ) { + event.result = handle.apply( cur, data ); + if ( event.result === false ) { + event.preventDefault(); + } + } + } + event.type = type; + + // If nobody prevented the default action, do it now + if ( !onlyHandlers && !event.isDefaultPrevented() ) { + + if ( ( !special._default || + special._default.apply( eventPath.pop(), data ) === false ) && + acceptData( elem ) ) { + + // Call a native DOM method on the target with the same name as the event. + // Don't do default actions on window, that's where global variables be (#6170) + if ( ontype && isFunction( elem[ type ] ) && !isWindow( elem ) ) { + + // Don't re-trigger an onFOO event when we call its FOO() method + tmp = elem[ ontype ]; + + if ( tmp ) { + elem[ ontype ] = null; + } + + // Prevent re-triggering of the same event, since we already bubbled it above + jQuery.event.triggered = type; + + if ( event.isPropagationStopped() ) { + lastElement.addEventListener( type, stopPropagationCallback ); + } + + elem[ type ](); + + if ( event.isPropagationStopped() ) { + lastElement.removeEventListener( type, stopPropagationCallback ); + } + + jQuery.event.triggered = undefined; + + if ( tmp ) { + elem[ ontype ] = tmp; + } + } + } + } + + return event.result; + }, + + // Piggyback on a donor event to simulate a different one + // Used only for `focus(in | out)` events + simulate: function( type, elem, event ) { + var e = jQuery.extend( + new jQuery.Event(), + event, + { + type: type, + isSimulated: true + } + ); + + jQuery.event.trigger( e, null, elem ); + } + +} ); + +jQuery.fn.extend( { + + trigger: function( type, data ) { + return this.each( function() { + jQuery.event.trigger( type, data, this ); + } ); + }, + triggerHandler: function( type, data ) { + var elem = this[ 0 ]; + if ( elem ) { + return jQuery.event.trigger( type, data, elem, true ); + } + } +} ); + + +// Support: Firefox <=44 +// Firefox doesn't have focus(in | out) events +// Related ticket - https://bugzilla.mozilla.org/show_bug.cgi?id=687787 +// +// Support: Chrome <=48 - 49, Safari <=9.0 - 9.1 +// focus(in | out) events fire after focus & blur events, +// which is spec violation - http://www.w3.org/TR/DOM-Level-3-Events/#events-focusevent-event-order +// Related ticket - https://bugs.chromium.org/p/chromium/issues/detail?id=449857 +if ( !support.focusin ) { + jQuery.each( { focus: "focusin", blur: "focusout" }, function( orig, fix ) { + + // Attach a single capturing handler on the document while someone wants focusin/focusout + var handler = function( event ) { + jQuery.event.simulate( fix, event.target, jQuery.event.fix( event ) ); + }; + + jQuery.event.special[ fix ] = { + setup: function() { + + // Handle: regular nodes (via `this.ownerDocument`), window + // (via `this.document`) & document (via `this`). + var doc = this.ownerDocument || this.document || this, + attaches = dataPriv.access( doc, fix ); + + if ( !attaches ) { + doc.addEventListener( orig, handler, true ); + } + dataPriv.access( doc, fix, ( attaches || 0 ) + 1 ); + }, + teardown: function() { + var doc = this.ownerDocument || this.document || this, + attaches = dataPriv.access( doc, fix ) - 1; + + if ( !attaches ) { + doc.removeEventListener( orig, handler, true ); + dataPriv.remove( doc, fix ); + + } else { + dataPriv.access( doc, fix, attaches ); + } + } + }; + } ); +} +var location = window.location; + +var nonce = { guid: Date.now() }; + +var rquery = ( /\?/ ); + + + +// Cross-browser xml parsing +jQuery.parseXML = function( data ) { + var xml, parserErrorElem; + if ( !data || typeof data !== "string" ) { + return null; + } + + // Support: IE 9 - 11 only + // IE throws on parseFromString with invalid input. + try { + xml = ( new window.DOMParser() ).parseFromString( data, "text/xml" ); + } catch ( e ) {} + + parserErrorElem = xml && xml.getElementsByTagName( "parsererror" )[ 0 ]; + if ( !xml || parserErrorElem ) { + jQuery.error( "Invalid XML: " + ( + parserErrorElem ? + jQuery.map( parserErrorElem.childNodes, function( el ) { + return el.textContent; + } ).join( "\n" ) : + data + ) ); + } + return xml; +}; + + +var + rbracket = /\[\]$/, + rCRLF = /\r?\n/g, + rsubmitterTypes = /^(?:submit|button|image|reset|file)$/i, + rsubmittable = /^(?:input|select|textarea|keygen)/i; + +function buildParams( prefix, obj, traditional, add ) { + var name; + + if ( Array.isArray( obj ) ) { + + // Serialize array item. + jQuery.each( obj, function( i, v ) { + if ( traditional || rbracket.test( prefix ) ) { + + // Treat each array item as a scalar. + add( prefix, v ); + + } else { + + // Item is non-scalar (array or object), encode its numeric index. + buildParams( + prefix + "[" + ( typeof v === "object" && v != null ? i : "" ) + "]", + v, + traditional, + add + ); + } + } ); + + } else if ( !traditional && toType( obj ) === "object" ) { + + // Serialize object item. + for ( name in obj ) { + buildParams( prefix + "[" + name + "]", obj[ name ], traditional, add ); + } + + } else { + + // Serialize scalar item. + add( prefix, obj ); + } +} + +// Serialize an array of form elements or a set of +// key/values into a query string +jQuery.param = function( a, traditional ) { + var prefix, + s = [], + add = function( key, valueOrFunction ) { + + // If value is a function, invoke it and use its return value + var value = isFunction( valueOrFunction ) ? + valueOrFunction() : + valueOrFunction; + + s[ s.length ] = encodeURIComponent( key ) + "=" + + encodeURIComponent( value == null ? "" : value ); + }; + + if ( a == null ) { + return ""; + } + + // If an array was passed in, assume that it is an array of form elements. + if ( Array.isArray( a ) || ( a.jquery && !jQuery.isPlainObject( a ) ) ) { + + // Serialize the form elements + jQuery.each( a, function() { + add( this.name, this.value ); + } ); + + } else { + + // If traditional, encode the "old" way (the way 1.3.2 or older + // did it), otherwise encode params recursively. + for ( prefix in a ) { + buildParams( prefix, a[ prefix ], traditional, add ); + } + } + + // Return the resulting serialization + return s.join( "&" ); +}; + +jQuery.fn.extend( { + serialize: function() { + return jQuery.param( this.serializeArray() ); + }, + serializeArray: function() { + return this.map( function() { + + // Can add propHook for "elements" to filter or add form elements + var elements = jQuery.prop( this, "elements" ); + return elements ? jQuery.makeArray( elements ) : this; + } ).filter( function() { + var type = this.type; + + // Use .is( ":disabled" ) so that fieldset[disabled] works + return this.name && !jQuery( this ).is( ":disabled" ) && + rsubmittable.test( this.nodeName ) && !rsubmitterTypes.test( type ) && + ( this.checked || !rcheckableType.test( type ) ); + } ).map( function( _i, elem ) { + var val = jQuery( this ).val(); + + if ( val == null ) { + return null; + } + + if ( Array.isArray( val ) ) { + return jQuery.map( val, function( val ) { + return { name: elem.name, value: val.replace( rCRLF, "\r\n" ) }; + } ); + } + + return { name: elem.name, value: val.replace( rCRLF, "\r\n" ) }; + } ).get(); + } +} ); + + +var + r20 = /%20/g, + rhash = /#.*$/, + rantiCache = /([?&])_=[^&]*/, + rheaders = /^(.*?):[ \t]*([^\r\n]*)$/mg, + + // #7653, #8125, #8152: local protocol detection + rlocalProtocol = /^(?:about|app|app-storage|.+-extension|file|res|widget):$/, + rnoContent = /^(?:GET|HEAD)$/, + rprotocol = /^\/\//, + + /* Prefilters + * 1) They are useful to introduce custom dataTypes (see ajax/jsonp.js for an example) + * 2) These are called: + * - BEFORE asking for a transport + * - AFTER param serialization (s.data is a string if s.processData is true) + * 3) key is the dataType + * 4) the catchall symbol "*" can be used + * 5) execution will start with transport dataType and THEN continue down to "*" if needed + */ + prefilters = {}, + + /* Transports bindings + * 1) key is the dataType + * 2) the catchall symbol "*" can be used + * 3) selection will start with transport dataType and THEN go to "*" if needed + */ + transports = {}, + + // Avoid comment-prolog char sequence (#10098); must appease lint and evade compression + allTypes = "*/".concat( "*" ), + + // Anchor tag for parsing the document origin + originAnchor = document.createElement( "a" ); + +originAnchor.href = location.href; + +// Base "constructor" for jQuery.ajaxPrefilter and jQuery.ajaxTransport +function addToPrefiltersOrTransports( structure ) { + + // dataTypeExpression is optional and defaults to "*" + return function( dataTypeExpression, func ) { + + if ( typeof dataTypeExpression !== "string" ) { + func = dataTypeExpression; + dataTypeExpression = "*"; + } + + var dataType, + i = 0, + dataTypes = dataTypeExpression.toLowerCase().match( rnothtmlwhite ) || []; + + if ( isFunction( func ) ) { + + // For each dataType in the dataTypeExpression + while ( ( dataType = dataTypes[ i++ ] ) ) { + + // Prepend if requested + if ( dataType[ 0 ] === "+" ) { + dataType = dataType.slice( 1 ) || "*"; + ( structure[ dataType ] = structure[ dataType ] || [] ).unshift( func ); + + // Otherwise append + } else { + ( structure[ dataType ] = structure[ dataType ] || [] ).push( func ); + } + } + } + }; +} + +// Base inspection function for prefilters and transports +function inspectPrefiltersOrTransports( structure, options, originalOptions, jqXHR ) { + + var inspected = {}, + seekingTransport = ( structure === transports ); + + function inspect( dataType ) { + var selected; + inspected[ dataType ] = true; + jQuery.each( structure[ dataType ] || [], function( _, prefilterOrFactory ) { + var dataTypeOrTransport = prefilterOrFactory( options, originalOptions, jqXHR ); + if ( typeof dataTypeOrTransport === "string" && + !seekingTransport && !inspected[ dataTypeOrTransport ] ) { + + options.dataTypes.unshift( dataTypeOrTransport ); + inspect( dataTypeOrTransport ); + return false; + } else if ( seekingTransport ) { + return !( selected = dataTypeOrTransport ); + } + } ); + return selected; + } + + return inspect( options.dataTypes[ 0 ] ) || !inspected[ "*" ] && inspect( "*" ); +} + +// A special extend for ajax options +// that takes "flat" options (not to be deep extended) +// Fixes #9887 +function ajaxExtend( target, src ) { + var key, deep, + flatOptions = jQuery.ajaxSettings.flatOptions || {}; + + for ( key in src ) { + if ( src[ key ] !== undefined ) { + ( flatOptions[ key ] ? target : ( deep || ( deep = {} ) ) )[ key ] = src[ key ]; + } + } + if ( deep ) { + jQuery.extend( true, target, deep ); + } + + return target; +} + +/* Handles responses to an ajax request: + * - finds the right dataType (mediates between content-type and expected dataType) + * - returns the corresponding response + */ +function ajaxHandleResponses( s, jqXHR, responses ) { + + var ct, type, finalDataType, firstDataType, + contents = s.contents, + dataTypes = s.dataTypes; + + // Remove auto dataType and get content-type in the process + while ( dataTypes[ 0 ] === "*" ) { + dataTypes.shift(); + if ( ct === undefined ) { + ct = s.mimeType || jqXHR.getResponseHeader( "Content-Type" ); + } + } + + // Check if we're dealing with a known content-type + if ( ct ) { + for ( type in contents ) { + if ( contents[ type ] && contents[ type ].test( ct ) ) { + dataTypes.unshift( type ); + break; + } + } + } + + // Check to see if we have a response for the expected dataType + if ( dataTypes[ 0 ] in responses ) { + finalDataType = dataTypes[ 0 ]; + } else { + + // Try convertible dataTypes + for ( type in responses ) { + if ( !dataTypes[ 0 ] || s.converters[ type + " " + dataTypes[ 0 ] ] ) { + finalDataType = type; + break; + } + if ( !firstDataType ) { + firstDataType = type; + } + } + + // Or just use first one + finalDataType = finalDataType || firstDataType; + } + + // If we found a dataType + // We add the dataType to the list if needed + // and return the corresponding response + if ( finalDataType ) { + if ( finalDataType !== dataTypes[ 0 ] ) { + dataTypes.unshift( finalDataType ); + } + return responses[ finalDataType ]; + } +} + +/* Chain conversions given the request and the original response + * Also sets the responseXXX fields on the jqXHR instance + */ +function ajaxConvert( s, response, jqXHR, isSuccess ) { + var conv2, current, conv, tmp, prev, + converters = {}, + + // Work with a copy of dataTypes in case we need to modify it for conversion + dataTypes = s.dataTypes.slice(); + + // Create converters map with lowercased keys + if ( dataTypes[ 1 ] ) { + for ( conv in s.converters ) { + converters[ conv.toLowerCase() ] = s.converters[ conv ]; + } + } + + current = dataTypes.shift(); + + // Convert to each sequential dataType + while ( current ) { + + if ( s.responseFields[ current ] ) { + jqXHR[ s.responseFields[ current ] ] = response; + } + + // Apply the dataFilter if provided + if ( !prev && isSuccess && s.dataFilter ) { + response = s.dataFilter( response, s.dataType ); + } + + prev = current; + current = dataTypes.shift(); + + if ( current ) { + + // There's only work to do if current dataType is non-auto + if ( current === "*" ) { + + current = prev; + + // Convert response if prev dataType is non-auto and differs from current + } else if ( prev !== "*" && prev !== current ) { + + // Seek a direct converter + conv = converters[ prev + " " + current ] || converters[ "* " + current ]; + + // If none found, seek a pair + if ( !conv ) { + for ( conv2 in converters ) { + + // If conv2 outputs current + tmp = conv2.split( " " ); + if ( tmp[ 1 ] === current ) { + + // If prev can be converted to accepted input + conv = converters[ prev + " " + tmp[ 0 ] ] || + converters[ "* " + tmp[ 0 ] ]; + if ( conv ) { + + // Condense equivalence converters + if ( conv === true ) { + conv = converters[ conv2 ]; + + // Otherwise, insert the intermediate dataType + } else if ( converters[ conv2 ] !== true ) { + current = tmp[ 0 ]; + dataTypes.unshift( tmp[ 1 ] ); + } + break; + } + } + } + } + + // Apply converter (if not an equivalence) + if ( conv !== true ) { + + // Unless errors are allowed to bubble, catch and return them + if ( conv && s.throws ) { + response = conv( response ); + } else { + try { + response = conv( response ); + } catch ( e ) { + return { + state: "parsererror", + error: conv ? e : "No conversion from " + prev + " to " + current + }; + } + } + } + } + } + } + + return { state: "success", data: response }; +} + +jQuery.extend( { + + // Counter for holding the number of active queries + active: 0, + + // Last-Modified header cache for next request + lastModified: {}, + etag: {}, + + ajaxSettings: { + url: location.href, + type: "GET", + isLocal: rlocalProtocol.test( location.protocol ), + global: true, + processData: true, + async: true, + contentType: "application/x-www-form-urlencoded; charset=UTF-8", + + /* + timeout: 0, + data: null, + dataType: null, + username: null, + password: null, + cache: null, + throws: false, + traditional: false, + headers: {}, + */ + + accepts: { + "*": allTypes, + text: "text/plain", + html: "text/html", + xml: "application/xml, text/xml", + json: "application/json, text/javascript" + }, + + contents: { + xml: /\bxml\b/, + html: /\bhtml/, + json: /\bjson\b/ + }, + + responseFields: { + xml: "responseXML", + text: "responseText", + json: "responseJSON" + }, + + // Data converters + // Keys separate source (or catchall "*") and destination types with a single space + converters: { + + // Convert anything to text + "* text": String, + + // Text to html (true = no transformation) + "text html": true, + + // Evaluate text as a json expression + "text json": JSON.parse, + + // Parse text as xml + "text xml": jQuery.parseXML + }, + + // For options that shouldn't be deep extended: + // you can add your own custom options here if + // and when you create one that shouldn't be + // deep extended (see ajaxExtend) + flatOptions: { + url: true, + context: true + } + }, + + // Creates a full fledged settings object into target + // with both ajaxSettings and settings fields. + // If target is omitted, writes into ajaxSettings. + ajaxSetup: function( target, settings ) { + return settings ? + + // Building a settings object + ajaxExtend( ajaxExtend( target, jQuery.ajaxSettings ), settings ) : + + // Extending ajaxSettings + ajaxExtend( jQuery.ajaxSettings, target ); + }, + + ajaxPrefilter: addToPrefiltersOrTransports( prefilters ), + ajaxTransport: addToPrefiltersOrTransports( transports ), + + // Main method + ajax: function( url, options ) { + + // If url is an object, simulate pre-1.5 signature + if ( typeof url === "object" ) { + options = url; + url = undefined; + } + + // Force options to be an object + options = options || {}; + + var transport, + + // URL without anti-cache param + cacheURL, + + // Response headers + responseHeadersString, + responseHeaders, + + // timeout handle + timeoutTimer, + + // Url cleanup var + urlAnchor, + + // Request state (becomes false upon send and true upon completion) + completed, + + // To know if global events are to be dispatched + fireGlobals, + + // Loop variable + i, + + // uncached part of the url + uncached, + + // Create the final options object + s = jQuery.ajaxSetup( {}, options ), + + // Callbacks context + callbackContext = s.context || s, + + // Context for global events is callbackContext if it is a DOM node or jQuery collection + globalEventContext = s.context && + ( callbackContext.nodeType || callbackContext.jquery ) ? + jQuery( callbackContext ) : + jQuery.event, + + // Deferreds + deferred = jQuery.Deferred(), + completeDeferred = jQuery.Callbacks( "once memory" ), + + // Status-dependent callbacks + statusCode = s.statusCode || {}, + + // Headers (they are sent all at once) + requestHeaders = {}, + requestHeadersNames = {}, + + // Default abort message + strAbort = "canceled", + + // Fake xhr + jqXHR = { + readyState: 0, + + // Builds headers hashtable if needed + getResponseHeader: function( key ) { + var match; + if ( completed ) { + if ( !responseHeaders ) { + responseHeaders = {}; + while ( ( match = rheaders.exec( responseHeadersString ) ) ) { + responseHeaders[ match[ 1 ].toLowerCase() + " " ] = + ( responseHeaders[ match[ 1 ].toLowerCase() + " " ] || [] ) + .concat( match[ 2 ] ); + } + } + match = responseHeaders[ key.toLowerCase() + " " ]; + } + return match == null ? null : match.join( ", " ); + }, + + // Raw string + getAllResponseHeaders: function() { + return completed ? responseHeadersString : null; + }, + + // Caches the header + setRequestHeader: function( name, value ) { + if ( completed == null ) { + name = requestHeadersNames[ name.toLowerCase() ] = + requestHeadersNames[ name.toLowerCase() ] || name; + requestHeaders[ name ] = value; + } + return this; + }, + + // Overrides response content-type header + overrideMimeType: function( type ) { + if ( completed == null ) { + s.mimeType = type; + } + return this; + }, + + // Status-dependent callbacks + statusCode: function( map ) { + var code; + if ( map ) { + if ( completed ) { + + // Execute the appropriate callbacks + jqXHR.always( map[ jqXHR.status ] ); + } else { + + // Lazy-add the new callbacks in a way that preserves old ones + for ( code in map ) { + statusCode[ code ] = [ statusCode[ code ], map[ code ] ]; + } + } + } + return this; + }, + + // Cancel the request + abort: function( statusText ) { + var finalText = statusText || strAbort; + if ( transport ) { + transport.abort( finalText ); + } + done( 0, finalText ); + return this; + } + }; + + // Attach deferreds + deferred.promise( jqXHR ); + + // Add protocol if not provided (prefilters might expect it) + // Handle falsy url in the settings object (#10093: consistency with old signature) + // We also use the url parameter if available + s.url = ( ( url || s.url || location.href ) + "" ) + .replace( rprotocol, location.protocol + "//" ); + + // Alias method option to type as per ticket #12004 + s.type = options.method || options.type || s.method || s.type; + + // Extract dataTypes list + s.dataTypes = ( s.dataType || "*" ).toLowerCase().match( rnothtmlwhite ) || [ "" ]; + + // A cross-domain request is in order when the origin doesn't match the current origin. + if ( s.crossDomain == null ) { + urlAnchor = document.createElement( "a" ); + + // Support: IE <=8 - 11, Edge 12 - 15 + // IE throws exception on accessing the href property if url is malformed, + // e.g. http://example.com:80x/ + try { + urlAnchor.href = s.url; + + // Support: IE <=8 - 11 only + // Anchor's host property isn't correctly set when s.url is relative + urlAnchor.href = urlAnchor.href; + s.crossDomain = originAnchor.protocol + "//" + originAnchor.host !== + urlAnchor.protocol + "//" + urlAnchor.host; + } catch ( e ) { + + // If there is an error parsing the URL, assume it is crossDomain, + // it can be rejected by the transport if it is invalid + s.crossDomain = true; + } + } + + // Convert data if not already a string + if ( s.data && s.processData && typeof s.data !== "string" ) { + s.data = jQuery.param( s.data, s.traditional ); + } + + // Apply prefilters + inspectPrefiltersOrTransports( prefilters, s, options, jqXHR ); + + // If request was aborted inside a prefilter, stop there + if ( completed ) { + return jqXHR; + } + + // We can fire global events as of now if asked to + // Don't fire events if jQuery.event is undefined in an AMD-usage scenario (#15118) + fireGlobals = jQuery.event && s.global; + + // Watch for a new set of requests + if ( fireGlobals && jQuery.active++ === 0 ) { + jQuery.event.trigger( "ajaxStart" ); + } + + // Uppercase the type + s.type = s.type.toUpperCase(); + + // Determine if request has content + s.hasContent = !rnoContent.test( s.type ); + + // Save the URL in case we're toying with the If-Modified-Since + // and/or If-None-Match header later on + // Remove hash to simplify url manipulation + cacheURL = s.url.replace( rhash, "" ); + + // More options handling for requests with no content + if ( !s.hasContent ) { + + // Remember the hash so we can put it back + uncached = s.url.slice( cacheURL.length ); + + // If data is available and should be processed, append data to url + if ( s.data && ( s.processData || typeof s.data === "string" ) ) { + cacheURL += ( rquery.test( cacheURL ) ? "&" : "?" ) + s.data; + + // #9682: remove data so that it's not used in an eventual retry + delete s.data; + } + + // Add or update anti-cache param if needed + if ( s.cache === false ) { + cacheURL = cacheURL.replace( rantiCache, "$1" ); + uncached = ( rquery.test( cacheURL ) ? "&" : "?" ) + "_=" + ( nonce.guid++ ) + + uncached; + } + + // Put hash and anti-cache on the URL that will be requested (gh-1732) + s.url = cacheURL + uncached; + + // Change '%20' to '+' if this is encoded form body content (gh-2658) + } else if ( s.data && s.processData && + ( s.contentType || "" ).indexOf( "application/x-www-form-urlencoded" ) === 0 ) { + s.data = s.data.replace( r20, "+" ); + } + + // Set the If-Modified-Since and/or If-None-Match header, if in ifModified mode. + if ( s.ifModified ) { + if ( jQuery.lastModified[ cacheURL ] ) { + jqXHR.setRequestHeader( "If-Modified-Since", jQuery.lastModified[ cacheURL ] ); + } + if ( jQuery.etag[ cacheURL ] ) { + jqXHR.setRequestHeader( "If-None-Match", jQuery.etag[ cacheURL ] ); + } + } + + // Set the correct header, if data is being sent + if ( s.data && s.hasContent && s.contentType !== false || options.contentType ) { + jqXHR.setRequestHeader( "Content-Type", s.contentType ); + } + + // Set the Accepts header for the server, depending on the dataType + jqXHR.setRequestHeader( + "Accept", + s.dataTypes[ 0 ] && s.accepts[ s.dataTypes[ 0 ] ] ? + s.accepts[ s.dataTypes[ 0 ] ] + + ( s.dataTypes[ 0 ] !== "*" ? ", " + allTypes + "; q=0.01" : "" ) : + s.accepts[ "*" ] + ); + + // Check for headers option + for ( i in s.headers ) { + jqXHR.setRequestHeader( i, s.headers[ i ] ); + } + + // Allow custom headers/mimetypes and early abort + if ( s.beforeSend && + ( s.beforeSend.call( callbackContext, jqXHR, s ) === false || completed ) ) { + + // Abort if not done already and return + return jqXHR.abort(); + } + + // Aborting is no longer a cancellation + strAbort = "abort"; + + // Install callbacks on deferreds + completeDeferred.add( s.complete ); + jqXHR.done( s.success ); + jqXHR.fail( s.error ); + + // Get transport + transport = inspectPrefiltersOrTransports( transports, s, options, jqXHR ); + + // If no transport, we auto-abort + if ( !transport ) { + done( -1, "No Transport" ); + } else { + jqXHR.readyState = 1; + + // Send global event + if ( fireGlobals ) { + globalEventContext.trigger( "ajaxSend", [ jqXHR, s ] ); + } + + // If request was aborted inside ajaxSend, stop there + if ( completed ) { + return jqXHR; + } + + // Timeout + if ( s.async && s.timeout > 0 ) { + timeoutTimer = window.setTimeout( function() { + jqXHR.abort( "timeout" ); + }, s.timeout ); + } + + try { + completed = false; + transport.send( requestHeaders, done ); + } catch ( e ) { + + // Rethrow post-completion exceptions + if ( completed ) { + throw e; + } + + // Propagate others as results + done( -1, e ); + } + } + + // Callback for when everything is done + function done( status, nativeStatusText, responses, headers ) { + var isSuccess, success, error, response, modified, + statusText = nativeStatusText; + + // Ignore repeat invocations + if ( completed ) { + return; + } + + completed = true; + + // Clear timeout if it exists + if ( timeoutTimer ) { + window.clearTimeout( timeoutTimer ); + } + + // Dereference transport for early garbage collection + // (no matter how long the jqXHR object will be used) + transport = undefined; + + // Cache response headers + responseHeadersString = headers || ""; + + // Set readyState + jqXHR.readyState = status > 0 ? 4 : 0; + + // Determine if successful + isSuccess = status >= 200 && status < 300 || status === 304; + + // Get response data + if ( responses ) { + response = ajaxHandleResponses( s, jqXHR, responses ); + } + + // Use a noop converter for missing script but not if jsonp + if ( !isSuccess && + jQuery.inArray( "script", s.dataTypes ) > -1 && + jQuery.inArray( "json", s.dataTypes ) < 0 ) { + s.converters[ "text script" ] = function() {}; + } + + // Convert no matter what (that way responseXXX fields are always set) + response = ajaxConvert( s, response, jqXHR, isSuccess ); + + // If successful, handle type chaining + if ( isSuccess ) { + + // Set the If-Modified-Since and/or If-None-Match header, if in ifModified mode. + if ( s.ifModified ) { + modified = jqXHR.getResponseHeader( "Last-Modified" ); + if ( modified ) { + jQuery.lastModified[ cacheURL ] = modified; + } + modified = jqXHR.getResponseHeader( "etag" ); + if ( modified ) { + jQuery.etag[ cacheURL ] = modified; + } + } + + // if no content + if ( status === 204 || s.type === "HEAD" ) { + statusText = "nocontent"; + + // if not modified + } else if ( status === 304 ) { + statusText = "notmodified"; + + // If we have data, let's convert it + } else { + statusText = response.state; + success = response.data; + error = response.error; + isSuccess = !error; + } + } else { + + // Extract error from statusText and normalize for non-aborts + error = statusText; + if ( status || !statusText ) { + statusText = "error"; + if ( status < 0 ) { + status = 0; + } + } + } + + // Set data for the fake xhr object + jqXHR.status = status; + jqXHR.statusText = ( nativeStatusText || statusText ) + ""; + + // Success/Error + if ( isSuccess ) { + deferred.resolveWith( callbackContext, [ success, statusText, jqXHR ] ); + } else { + deferred.rejectWith( callbackContext, [ jqXHR, statusText, error ] ); + } + + // Status-dependent callbacks + jqXHR.statusCode( statusCode ); + statusCode = undefined; + + if ( fireGlobals ) { + globalEventContext.trigger( isSuccess ? "ajaxSuccess" : "ajaxError", + [ jqXHR, s, isSuccess ? success : error ] ); + } + + // Complete + completeDeferred.fireWith( callbackContext, [ jqXHR, statusText ] ); + + if ( fireGlobals ) { + globalEventContext.trigger( "ajaxComplete", [ jqXHR, s ] ); + + // Handle the global AJAX counter + if ( !( --jQuery.active ) ) { + jQuery.event.trigger( "ajaxStop" ); + } + } + } + + return jqXHR; + }, + + getJSON: function( url, data, callback ) { + return jQuery.get( url, data, callback, "json" ); + }, + + getScript: function( url, callback ) { + return jQuery.get( url, undefined, callback, "script" ); + } +} ); + +jQuery.each( [ "get", "post" ], function( _i, method ) { + jQuery[ method ] = function( url, data, callback, type ) { + + // Shift arguments if data argument was omitted + if ( isFunction( data ) ) { + type = type || callback; + callback = data; + data = undefined; + } + + // The url can be an options object (which then must have .url) + return jQuery.ajax( jQuery.extend( { + url: url, + type: method, + dataType: type, + data: data, + success: callback + }, jQuery.isPlainObject( url ) && url ) ); + }; +} ); + +jQuery.ajaxPrefilter( function( s ) { + var i; + for ( i in s.headers ) { + if ( i.toLowerCase() === "content-type" ) { + s.contentType = s.headers[ i ] || ""; + } + } +} ); + + +jQuery._evalUrl = function( url, options, doc ) { + return jQuery.ajax( { + url: url, + + // Make this explicit, since user can override this through ajaxSetup (#11264) + type: "GET", + dataType: "script", + cache: true, + async: false, + global: false, + + // Only evaluate the response if it is successful (gh-4126) + // dataFilter is not invoked for failure responses, so using it instead + // of the default converter is kludgy but it works. + converters: { + "text script": function() {} + }, + dataFilter: function( response ) { + jQuery.globalEval( response, options, doc ); + } + } ); +}; + + +jQuery.fn.extend( { + wrapAll: function( html ) { + var wrap; + + if ( this[ 0 ] ) { + if ( isFunction( html ) ) { + html = html.call( this[ 0 ] ); + } + + // The elements to wrap the target around + wrap = jQuery( html, this[ 0 ].ownerDocument ).eq( 0 ).clone( true ); + + if ( this[ 0 ].parentNode ) { + wrap.insertBefore( this[ 0 ] ); + } + + wrap.map( function() { + var elem = this; + + while ( elem.firstElementChild ) { + elem = elem.firstElementChild; + } + + return elem; + } ).append( this ); + } + + return this; + }, + + wrapInner: function( html ) { + if ( isFunction( html ) ) { + return this.each( function( i ) { + jQuery( this ).wrapInner( html.call( this, i ) ); + } ); + } + + return this.each( function() { + var self = jQuery( this ), + contents = self.contents(); + + if ( contents.length ) { + contents.wrapAll( html ); + + } else { + self.append( html ); + } + } ); + }, + + wrap: function( html ) { + var htmlIsFunction = isFunction( html ); + + return this.each( function( i ) { + jQuery( this ).wrapAll( htmlIsFunction ? html.call( this, i ) : html ); + } ); + }, + + unwrap: function( selector ) { + this.parent( selector ).not( "body" ).each( function() { + jQuery( this ).replaceWith( this.childNodes ); + } ); + return this; + } +} ); + + +jQuery.expr.pseudos.hidden = function( elem ) { + return !jQuery.expr.pseudos.visible( elem ); +}; +jQuery.expr.pseudos.visible = function( elem ) { + return !!( elem.offsetWidth || elem.offsetHeight || elem.getClientRects().length ); +}; + + + + +jQuery.ajaxSettings.xhr = function() { + try { + return new window.XMLHttpRequest(); + } catch ( e ) {} +}; + +var xhrSuccessStatus = { + + // File protocol always yields status code 0, assume 200 + 0: 200, + + // Support: IE <=9 only + // #1450: sometimes IE returns 1223 when it should be 204 + 1223: 204 + }, + xhrSupported = jQuery.ajaxSettings.xhr(); + +support.cors = !!xhrSupported && ( "withCredentials" in xhrSupported ); +support.ajax = xhrSupported = !!xhrSupported; + +jQuery.ajaxTransport( function( options ) { + var callback, errorCallback; + + // Cross domain only allowed if supported through XMLHttpRequest + if ( support.cors || xhrSupported && !options.crossDomain ) { + return { + send: function( headers, complete ) { + var i, + xhr = options.xhr(); + + xhr.open( + options.type, + options.url, + options.async, + options.username, + options.password + ); + + // Apply custom fields if provided + if ( options.xhrFields ) { + for ( i in options.xhrFields ) { + xhr[ i ] = options.xhrFields[ i ]; + } + } + + // Override mime type if needed + if ( options.mimeType && xhr.overrideMimeType ) { + xhr.overrideMimeType( options.mimeType ); + } + + // X-Requested-With header + // For cross-domain requests, seeing as conditions for a preflight are + // akin to a jigsaw puzzle, we simply never set it to be sure. + // (it can always be set on a per-request basis or even using ajaxSetup) + // For same-domain requests, won't change header if already provided. + if ( !options.crossDomain && !headers[ "X-Requested-With" ] ) { + headers[ "X-Requested-With" ] = "XMLHttpRequest"; + } + + // Set headers + for ( i in headers ) { + xhr.setRequestHeader( i, headers[ i ] ); + } + + // Callback + callback = function( type ) { + return function() { + if ( callback ) { + callback = errorCallback = xhr.onload = + xhr.onerror = xhr.onabort = xhr.ontimeout = + xhr.onreadystatechange = null; + + if ( type === "abort" ) { + xhr.abort(); + } else if ( type === "error" ) { + + // Support: IE <=9 only + // On a manual native abort, IE9 throws + // errors on any property access that is not readyState + if ( typeof xhr.status !== "number" ) { + complete( 0, "error" ); + } else { + complete( + + // File: protocol always yields status 0; see #8605, #14207 + xhr.status, + xhr.statusText + ); + } + } else { + complete( + xhrSuccessStatus[ xhr.status ] || xhr.status, + xhr.statusText, + + // Support: IE <=9 only + // IE9 has no XHR2 but throws on binary (trac-11426) + // For XHR2 non-text, let the caller handle it (gh-2498) + ( xhr.responseType || "text" ) !== "text" || + typeof xhr.responseText !== "string" ? + { binary: xhr.response } : + { text: xhr.responseText }, + xhr.getAllResponseHeaders() + ); + } + } + }; + }; + + // Listen to events + xhr.onload = callback(); + errorCallback = xhr.onerror = xhr.ontimeout = callback( "error" ); + + // Support: IE 9 only + // Use onreadystatechange to replace onabort + // to handle uncaught aborts + if ( xhr.onabort !== undefined ) { + xhr.onabort = errorCallback; + } else { + xhr.onreadystatechange = function() { + + // Check readyState before timeout as it changes + if ( xhr.readyState === 4 ) { + + // Allow onerror to be called first, + // but that will not handle a native abort + // Also, save errorCallback to a variable + // as xhr.onerror cannot be accessed + window.setTimeout( function() { + if ( callback ) { + errorCallback(); + } + } ); + } + }; + } + + // Create the abort callback + callback = callback( "abort" ); + + try { + + // Do send the request (this may raise an exception) + xhr.send( options.hasContent && options.data || null ); + } catch ( e ) { + + // #14683: Only rethrow if this hasn't been notified as an error yet + if ( callback ) { + throw e; + } + } + }, + + abort: function() { + if ( callback ) { + callback(); + } + } + }; + } +} ); + + + + +// Prevent auto-execution of scripts when no explicit dataType was provided (See gh-2432) +jQuery.ajaxPrefilter( function( s ) { + if ( s.crossDomain ) { + s.contents.script = false; + } +} ); + +// Install script dataType +jQuery.ajaxSetup( { + accepts: { + script: "text/javascript, application/javascript, " + + "application/ecmascript, application/x-ecmascript" + }, + contents: { + script: /\b(?:java|ecma)script\b/ + }, + converters: { + "text script": function( text ) { + jQuery.globalEval( text ); + return text; + } + } +} ); + +// Handle cache's special case and crossDomain +jQuery.ajaxPrefilter( "script", function( s ) { + if ( s.cache === undefined ) { + s.cache = false; + } + if ( s.crossDomain ) { + s.type = "GET"; + } +} ); + +// Bind script tag hack transport +jQuery.ajaxTransport( "script", function( s ) { + + // This transport only deals with cross domain or forced-by-attrs requests + if ( s.crossDomain || s.scriptAttrs ) { + var script, callback; + return { + send: function( _, complete ) { + script = jQuery( " + + + + Administration — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ + +
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/dist-sort.html b/430/admin/dist-sort.html new file mode 100644 index 000000000..662313808 --- /dev/null +++ b/430/admin/dist-sort.html @@ -0,0 +1,513 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Distributed sort — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Distributed sort#

+

Distributed sort allows to sort data, which exceeds query.max-memory-per-node. +Distributed sort is enabled via the distributed_sort session property, or +distributed-sort configuration property set in +etc/config.properties of the coordinator. Distributed sort is enabled by +default.

+

When distributed sort is enabled, the sort operator executes in parallel on multiple +nodes in the cluster. Partially sorted data from each Trino worker node is then streamed +to a single worker node for a final merge. This technique allows to utilize memory of multiple +Trino worker nodes for sorting. The primary purpose of distributed sort is to allow for sorting +of data sets which don’t normally fit into single node memory. Performance improvement +can be expected, but it won’t scale linearly with the number of nodes, since the +data needs to be merged by a single node.

+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/dynamic-filtering.html b/430/admin/dynamic-filtering.html new file mode 100644 index 000000000..3638e6b1e --- /dev/null +++ b/430/admin/dynamic-filtering.html @@ -0,0 +1,761 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Dynamic filtering — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Dynamic filtering#

+

Dynamic filtering optimizations significantly improve the performance of queries +with selective joins by avoiding reading of data that would be filtered by join condition.

+

Consider the following query which captures a common pattern of a fact table store_sales +joined with a filtered dimension table date_dim:

+
+

SELECT count(*) +FROM store_sales +JOIN date_dim ON store_sales.ss_sold_date_sk = date_dim.d_date_sk +WHERE d_following_holiday=’Y’ AND d_year = 2000;

+
+

Without dynamic filtering, Trino pushes predicates for the dimension table to the +table scan on date_dim, and it scans all the data in the fact table since there +are no filters on store_sales in the query. The join operator ends up throwing away +most of the probe-side rows as the join criteria is highly selective.

+

When dynamic filtering is enabled, Trino collects candidate values for join condition +from the processed dimension table on the right side of join. In the case of broadcast joins, +the runtime predicates generated from this collection are pushed into the local table scan +on the left side of the join running on the same worker.

+

Additionally, these runtime predicates are communicated to the coordinator over the network +so that dynamic filtering can also be performed on the coordinator during enumeration of +table scan splits.

+

For example, in the case of the Hive connector, dynamic filters are used +to skip loading of partitions which don’t match the join criteria. +This is known as dynamic partition pruning.

+

After completing the collection of dynamic filters, the coordinator also distributes them +to worker nodes over the network for partitioned joins. This allows push down of dynamic +filters from partitioned joins into the table scans on the left side of that join. +Distribution of dynamic filters from the coordinator to workers is enabled by default. +It can be disabled by setting either the enable-coordinator-dynamic-filters-distribution +configuration property, or the session property +enable_coordinator_dynamic_filters_distribution to false.

+

The results of dynamic filtering optimization can include the following benefits:

+
    +
  • improved overall query performance

  • +
  • reduced network traffic between Trino and the data source

  • +
  • reduced load on the remote data source

  • +
+

Dynamic filtering is enabled by default. It can be disabled by setting either the +enable-dynamic-filtering configuration property, or the session property +enable_dynamic_filtering to false.

+

Support for push down of dynamic filters is specific to each connector, +and the relevant underlying database or storage system. The documentation for +specific connectors with support for dynamic filtering includes further details, +for example the Hive connector +or the Memory connector.

+
+

Analysis and confirmation#

+

Dynamic filtering depends on a number of factors:

+
    +
  • Planner support for dynamic filtering for a given join operation in Trino. +Currently inner and right joins with =, <, <=, >, >= or +IS NOT DISTINCT FROM join conditions, and +semi-joins with IN conditions are supported.

  • +
  • Connector support for utilizing dynamic filters pushed into the table scan at runtime. +For example, the Hive connector can push dynamic filters into ORC and Parquet readers +to perform stripe or row-group pruning.

  • +
  • Connector support for utilizing dynamic filters at the splits enumeration stage.

  • +
  • Size of right (build) side of the join.

  • +
+

You can take a closer look at the EXPLAIN plan of the query +to analyze if the planner is adding dynamic filters to a specific query’s plan. +For example, the explain plan for the above query can be obtained by running +the following statement:

+
EXPLAIN
+SELECT count(*)
+FROM store_sales
+JOIN date_dim ON store_sales.ss_sold_date_sk = date_dim.d_date_sk
+WHERE d_following_holiday='Y' AND d_year = 2000;
+
+
+

The explain plan for this query shows dynamicFilterAssignments in the +InnerJoin node with dynamic filter df_370 collected from build symbol d_date_sk. +You can also see the dynamicFilter predicate as part of the Hive ScanFilterProject +operator where df_370 is associated with probe symbol ss_sold_date_sk. +This shows you that the planner is successful in pushing dynamic filters +down to the connector in the query plan.

+
...
+
+Fragment 1 [SOURCE]
+    Output layout: [count_3]
+    Output partitioning: SINGLE []
+    Aggregate(PARTIAL)
+    │   Layout: [count_3:bigint]
+    │   count_3 := count(*)
+    └─ InnerJoin[(""ss_sold_date_sk"" = ""d_date_sk"")][$hashvalue, $hashvalue_4]
+       │   Layout: []
+       │   Estimates: {rows: 0 (0B), cpu: 0, memory: 0B, network: 0B}
+       │   Distribution: REPLICATED
+       │   dynamicFilterAssignments = {d_date_sk -> #df_370}
+       ├─ ScanFilterProject[table = hive:default:store_sales, grouped = false, filterPredicate = true, dynamicFilters = {""ss_sold_date_sk"" = #df_370}]
+       │      Layout: [ss_sold_date_sk:bigint, $hashvalue:bigint]
+       │      Estimates: {rows: 0 (0B), cpu: 0, memory: 0B, network: 0B}/{rows: 0 (0B), cpu: 0, memory: 0B, network: 0B}/{rows: 0 (0B), cpu: 0, memory: 0B, network: 0B}
+       │      $hashvalue := combine_hash(bigint '0', COALESCE(""$operator$hash_code""(""ss_sold_date_sk""), 0))
+       │      ss_sold_date_sk := ss_sold_date_sk:bigint:REGULAR
+       └─ LocalExchange[HASH][$hashvalue_4] (""d_date_sk"")
+          │   Layout: [d_date_sk:bigint, $hashvalue_4:bigint]
+          │   Estimates: {rows: 0 (0B), cpu: 0, memory: 0B, network: 0B}
+          └─ RemoteSource[2]
+                 Layout: [d_date_sk:bigint, $hashvalue_5:bigint]
+
+Fragment 2 [SOURCE]
+    Output layout: [d_date_sk, $hashvalue_6]
+    Output partitioning: BROADCAST []
+    ScanFilterProject[table = hive:default:date_dim, grouped = false, filterPredicate = ((""d_following_holiday"" = CAST('Y' AS char(1))) AND (""d_year"" = 2000))]
+        Layout: [d_date_sk:bigint, $hashvalue_6:bigint]
+        Estimates: {rows: 0 (0B), cpu: 0, memory: 0B, network: 0B}/{rows: 0 (0B), cpu: 0, memory: 0B, network: 0B}/{rows: 0 (0B), cpu: 0, memory: 0B, network: 0B}
+        $hashvalue_6 := combine_hash(bigint '0', COALESCE(""$operator$hash_code""(""d_date_sk""), 0))
+        d_following_holiday := d_following_holiday:char(1):REGULAR
+        d_date_sk := d_date_sk:bigint:REGULAR
+        d_year := d_year:int:REGULAR
+
+
+

During execution of a query with dynamic filters, Trino populates statistics +about dynamic filters in the QueryInfo JSON available through the +Web UI. +In the queryStats section, statistics about dynamic filters collected +by the coordinator can be found in the dynamicFiltersStats structure.

+
"dynamicFiltersStats" : {
+      "dynamicFilterDomainStats" : [ {
+        "dynamicFilterId" : "df_370",
+        "simplifiedDomain" : "[ SortedRangeSet[type=bigint, ranges=3, {[2451546], ..., [2451905]}] ]",
+        "collectionDuration" : "2.34s"
+      } ],
+      "lazyDynamicFilters" : 1,
+      "replicatedDynamicFilters" : 1,
+      "totalDynamicFilters" : 1,
+      "dynamicFiltersCompleted" : 1
+}
+
+
+

Push down of dynamic filters into a table scan on the worker nodes can be +verified by looking at the operator statistics for that table scan. +dynamicFilterSplitsProcessed records the number of splits +processed after a dynamic filter is pushed down to the table scan.

+
"operatorType" : "ScanFilterAndProjectOperator",
+"totalDrivers" : 1,
+"addInputCalls" : 762,
+"addInputWall" : "0.00ns",
+"addInputCpu" : "0.00ns",
+"physicalInputDataSize" : "0B",
+"physicalInputPositions" : 28800991,
+"inputPositions" : 28800991,
+"dynamicFilterSplitsProcessed" : 1,
+
+
+

Dynamic filters are reported as a part of the +EXPLAIN ANALYZE plan in the statistics for +ScanFilterProject nodes.

+
...
+
+ └─ InnerJoin[("ss_sold_date_sk" = "d_date_sk")][$hashvalue, $hashvalue_4]
+    │   Layout: []
+    │   Estimates: {rows: 11859 (0B), cpu: 8.84M, memory: 3.19kB, network: 3.19kB}
+    │   CPU: 78.00ms (30.00%), Scheduled: 295.00ms (47.05%), Output: 296 rows (0B)
+    │   Left (probe) Input avg.: 120527.00 rows, Input std.dev.: 0.00%
+    │   Right (build) Input avg.: 0.19 rows, Input std.dev.: 208.17%
+    │   Distribution: REPLICATED
+    │   dynamicFilterAssignments = {d_date_sk -> #df_370}
+    ├─ ScanFilterProject[table = hive:default:store_sales, grouped = false, filterPredicate = true, dynamicFilters = {"ss_sold_date_sk" = #df_370}]
+    │      Layout: [ss_sold_date_sk:bigint, $hashvalue:bigint]
+    │      Estimates: {rows: 120527 (2.03MB), cpu: 1017.64k, memory: 0B, network: 0B}/{rows: 120527 (2.03MB), cpu: 1.99M, memory: 0B, network: 0B}/{rows: 120527 (2.03MB), cpu: 4.02M, memory: 0B, network: 0B}
+    │      CPU: 49.00ms (18.85%), Scheduled: 123.00ms (19.62%), Output: 120527 rows (2.07MB)
+    │      Input avg.: 120527.00 rows, Input std.dev.: 0.00%
+    │      $hashvalue := combine_hash(bigint '0', COALESCE("$operator$hash_code"("ss_sold_date_sk"), 0))
+    │      ss_sold_date_sk := ss_sold_date_sk:bigint:REGULAR
+    │      Input: 120527 rows (1.03MB), Filtered: 0.00%
+    │      Dynamic filters:
+    │          - df_370, [ SortedRangeSet[type=bigint, ranges=3, {[2451546], ..., [2451905]}] ], collection time=2.34s
+    |
+...
+
+
+
+
+

Dynamic filter collection thresholds#

+

In order for dynamic filtering to work, the smaller dimension table +needs to be chosen as a join’s build side. The cost-based optimizer can automatically +do this using table statistics provided by connectors. Therefore, it is recommended +to keep table statistics up to date and rely on the +CBO to correctly choose the smaller table on the build side of join.

+

Collection of values of the join key columns from the build side for +dynamic filtering may incur additional CPU overhead during query execution. +Therefore, to limit the overhead of collecting dynamic filters +to the cases where the join operator is likely to be selective, +Trino defines thresholds on the size of dynamic filters collected from build side tasks. +Collection of dynamic filters for joins with large build sides can be enabled +using the enable-large-dynamic-filters configuration property or the +enable_large_dynamic_filters session property.

+

When large dynamic filters are enabled, limits on the size of dynamic filters can +be configured using the configuration properties +dynamic-filtering.large.max-distinct-values-per-driver, +dynamic-filtering.large.max-size-per-driver , +dynamic-filtering.large.range-row-limit-per-driver, +dynamic-filtering.large-partitioned.max-distinct-values-per-driver, +dynamic-filtering.large-partitioned.max-size-per-driver and +dynamic-filtering.large-partitioned.range-row-limit-per-driver.

+

Similarly, limits for dynamic filters when enable-large-dynamic-filters +is not enabled can be configured using configuration properties like +dynamic-filtering.small.max-distinct-values-per-driver, +dynamic-filtering.small.max-size-per-driver , +dynamic-filtering.small.range-row-limit-per-driver, +dynamic-filtering.small-partitioned.max-distinct-values-per-driver, +dynamic-filtering.small-partitioned.max-size-per-driver and +dynamic-filtering.small-partitioned.range-row-limit-per-driver.

+

The dynamic-filtering.large.* and dynamic-filtering.small.* limits are applied +when dynamic filters are collected before build side is partitioned on join +keys (when broadcast join is chosen or when fault tolerant execution is enabled). The +dynamic-filtering.large-partitioned.* and dynamic-filtering.small-partitioned.* +limits are applied when dynamic filters are collected after build side is partitioned +on join keys (when partitioned join is chosen and fault tolerant execution is disabled).

+

The properties based on max-distinct-values-per-driver and max-size-per-driver +define thresholds for the size up to which dynamic filters are collected in a +distinct values data structure. When the build side exceeds these thresholds, +Trino switches to collecting min and max values per column to reduce overhead. +This min-max filter has much lower granularity than the distinct values filter. +However, it may still be beneficial in filtering some data from the probe side, +especially when a range of values is selected from the build side of the join. +The limits for min-max filters collection are defined by the properties +based on range-row-limit-per-driver.

+
+
+

Dimension tables layout#

+

Dynamic filtering works best for dimension tables where +table keys are correlated with columns.

+

For example, a date dimension key column should be correlated with a date column, +so the table keys monotonically increase with date values. +An address dimension key can be composed of other columns such as +COUNTRY-STATE-ZIP-ADDRESS_ID with an example value of US-NY-10001-1234. +This usage allows dynamic filtering to succeed even with a large number +of selected rows from the dimension table.

+
+
+

Limitations#

+
    +
  • Min-max dynamic filter collection is not supported for DOUBLE, REAL and unorderable data types.

  • +
  • Dynamic filtering is not supported for DOUBLE and REAL data types when using IS NOT DISTINCT FROM predicate.

  • +
  • Dynamic filtering is supported when the join key contains a cast from the build key type to the +probe key type. Dynamic filtering is also supported in limited scenarios when there is an implicit +cast from the probe key type to the build key type. For example, dynamic filtering is supported when +the build side key is of DOUBLE type and the probe side key is of REAL or INTEGER type.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/event-listeners-http.html b/430/admin/event-listeners-http.html new file mode 100644 index 000000000..353b04857 --- /dev/null +++ b/430/admin/event-listeners-http.html @@ -0,0 +1,643 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + HTTP event listener — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

HTTP event listener#

+

The HTTP event listener plugin allows streaming of query events, encoded in +JSON format, to an external service for further processing, by POSTing them +to a specified URI.

+
+

Rationale#

+

This event listener is a simple first step into better understanding the usage +of a datalake using query events provided by Trino. These can provide CPU and memory +usage metrics, what data is being accessed with resolution down to specific columns, +and metadata about the query processing.

+

Running the capture system separate from Trino reduces the performance impact and +avoids downtime for non-client-facing changes.

+
+
+

Requirements#

+

You need to perform the following steps:

+
    +
  • Provide an HTTP/S service that accepts POST events with a JSON body.

  • +
  • Configure http-event-listener.connect-ingest-uri in the event listener properties file +with the URI of the service.

  • +
  • Detail the events to send in the Configuration section.

  • +
+
+
+

Configuration#

+

To configure the HTTP event listener plugin, create an event listener properties +file in etc named http-event-listener.properties with the following contents +as an example:

+
event-listener.name=http
+http-event-listener.log-created=true
+http-event-listener.connect-ingest-uri=<your ingest URI>
+
+
+

And set add etc/http-event-listener.properties to event-listener.config-files +in Config properties:

+
event-listener.config-files=etc/http-event-listener.properties,...
+
+
+
+

Configuration properties#

+ +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

Default

http-event-listener.log-created

Enable the plugin to log QueryCreatedEvent events

false

http-event-listener.log-completed

Enable the plugin to log QueryCompletedEvent events

false

http-event-listener.log-split

Enable the plugin to log SplitCompletedEvent events

false

http-event-listener.connect-ingest-uri

The URI that the plugin will POST events to

None. See the requirements section.

http-event-listener.connect-http-headers

List of custom HTTP headers to be sent along with the events. See +Custom HTTP headers for more details

Empty

http-event-listener.connect-retry-count

The number of retries on server error. A server is considered to be +in an error state when the response code is 500 or higher

0

http-event-listener.connect-retry-delay

Duration for which to delay between attempts to send a request

1s

http-event-listener.connect-backoff-base

The base used for exponential backoff when retrying on server error. +The formula used to calculate the delay is +attemptDelay = retryDelay * backoffBase^{attemptCount}. +Attempt count starts from 0. Leave this empty or set to 1 to disable +exponential backoff and keep constant delays

2

http-event-listener.connect-max-delay

The upper bound of a delay between 2 retries. This should be +used with exponential backoff.

1m

http-event-listener.*

Pass configuration onto the HTTP client

+
+
+

Custom HTTP headers#

+

Providing custom HTTP headers is a useful mechanism for sending metadata along with +event messages.

+

Providing headers follows the pattern of key:value pairs separated by commas:

+
http-event-listener.connect-http-headers="Header-Name-1:header value 1,Header-Value-2:header value 2,..."
+
+
+

If you need to use a comma(,) or colon(:) in a header name or value, +escape it using a backslash (\).

+

Keep in mind that these are static, so they can not carry information +taken from the event itself.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/event-listeners-mysql.html b/430/admin/event-listeners-mysql.html new file mode 100644 index 000000000..64d17cfba --- /dev/null +++ b/430/admin/event-listeners-mysql.html @@ -0,0 +1,591 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + MySQL event listener — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

MySQL event listener#

+

The MySQL event listener plugin allows streaming of query events to an external +MySQL database. The query history in the database can then be accessed directly +in MySQL or via Trino in a catalog using the MySQL connector.

+
+

Rationale#

+

This event listener is a first step to store the query history of your Trino +cluster. The query events can provide CPU and memory usage metrics, what data is +being accessed with resolution down to specific columns, and metadata about the +query processing.

+

Running the capture system separate from Trino reduces the performance impact +and avoids downtime for non-client-facing changes.

+
+
+

Requirements#

+

You need to perform the following steps:

+
    +
  • Create a MySQL database.

  • +
  • Determine the JDBC connection URL for the database.

  • +
  • Ensure network access from the Trino coordinator to MySQL is available. +Port 3306 is the default port.

  • +
+
+
+

Configuration#

+

To configure the MySQL event listener plugin, create an event listener properties +file in etc named mysql-event-listener.properties with the following contents +as an example:

+
event-listener.name=mysql
+mysql-event-listener.db.url=jdbc:mysql://example.net:3306
+
+
+

The mysql-event-listener.db.url defines the connection to a MySQL database +available at the domain example.net on port 3306. You can pass further +parameters to the MySQL JDBC driver. The supported parameters for the URL are +documented in the MySQL Developer +Guide.

+

And set event-listener.config-files to etc/mysql-event-listener.properties +in Config properties:

+
event-listener.config-files=etc/mysql-event-listener.properties
+
+
+

If another event listener is already configured, add the new value +etc/mysql-event-listener.properties with a separating comma.

+

After this configuration and successful start of the Trino cluster, the table +trino_queries is created in the MySQL database. From then on, any query +processing event is captured by the event listener and a new row is inserted +into the table. The table includes many columns, such as query identifier, query +string, user, catalog, and others with information about the query processing.

+
+

Configuration properties#

+ ++++ + + + + + + + + + + +

Property name

Description

mysql-event-listener.db.url

JDBC connection URL to the database including credentials

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/fault-tolerant-execution.html b/430/admin/fault-tolerant-execution.html new file mode 100644 index 000000000..c62add296 --- /dev/null +++ b/430/admin/fault-tolerant-execution.html @@ -0,0 +1,1247 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Fault-tolerant execution — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Fault-tolerant execution#

+

By default, if a Trino node lacks the resources to execute a task or +otherwise fails during query execution, the query fails and must be run again +manually. The longer the runtime of a query, the more likely it is to be +susceptible to such failures.

+

Fault-tolerant execution is a mechanism in Trino that enables a cluster to +mitigate query failures by retrying queries or their component tasks in +the event of failure. With fault-tolerant execution enabled, intermediate +exchange data is spooled and can be re-used by another worker in the event of a +worker outage or other fault during query execution.

+
+

Note

+

Fault tolerance does not apply to broken queries or other user error. For +example, Trino does not spend resources retrying a query that fails because +its SQL cannot be parsed.

+

For a step-by-step guide explaining how to configure a Trino cluster with +fault-tolerant execution to improve query processing resilience, read +Improve query processing resilience.

+
+
+

Configuration#

+

Fault-tolerant execution is disabled by default. To enable the feature, set the +retry-policy configuration property to either QUERY or TASK +depending on the desired retry policy.

+
retry-policy=QUERY
+
+
+
+

Warning

+

Setting retry-policy may cause queries to fail with connectors that do not +explicitly support fault-tolerant execution, resulting in a “This connector +does not support query retries” error message.

+

Support for fault-tolerant execution of SQL statements varies on a +per-connector basis, with more details in the documentation for each +connector. The following connectors support fault-tolerant execution:

+ +
+

The following configuration properties control the behavior of fault-tolerant +execution on a Trino cluster:

+ + +++++ + + + + + + + + + + + + + + + + + + + + +
Fault-tolerant execution configuration properties#

Property name

Description

Default value

retry-policy

Configures what is retried in the event of failure, either QUERY to retry +the whole query, or TASK to retry tasks individually if they fail. See +retry policy for more information.

NONE

exchange.deduplication-buffer-size

Data size of the coordinator’s in-memory buffer used +by fault-tolerant execution to store output of query +stages. If this buffer is filled during query +execution, the query fails with a “Task descriptor storage capacity has been +exceeded” error message unless an exchange manager +is configured.

32MB

exchange.compression-enabled

Enable compression of spooling data. Setting to true is recommended +when using an exchange manager.

false

+
+
+

Retry policy#

+

The retry-policy configuration property designates whether Trino retries +entire queries or a query’s individual tasks in the event of failure.

+
+

QUERY#

+

A QUERY retry policy instructs Trino to automatically retry a query in the +event of an error occuring on a worker node. A QUERY retry policy is +recommended when the majority of the Trino cluster’s workload consists of many +small queries.

+

By default Trino does not implement fault tolerance for queries whose result set +exceeds 32MB in size, such as SELECT statements that return a very +large data set to the user. This limit can be increased by modifying the +exchange.deduplication-buffer-size configuration property to be greater than +the default value of 32MB, but this results in higher memory usage on the +coordinator.

+

To enable fault-tolerant execution on queries with a larger result set, it is +strongly recommended to configure an exchange manager that utilizes external storage for spooled data and +therefore allows for storage of spilled data beyond the in-memory buffer size.

+
+
+

TASK#

+

A TASK retry policy instructs Trino to retry individual query tasks in the event of failure. You must configure an +exchange manager to use the task retry policy. +This policy is recommended when executing large batch queries, as the cluster +can more efficiently retry smaller tasks within the query rather than retry the +whole query.

+

When a cluster is configured with a TASK retry policy, some relevant +configuration properties have their default values changed to follow best +practices for a fault-tolerant cluster. However, this automatic change does not +affect clusters that have these properties manually configured. If you have +any of the following properties configured in the config.properties file on +a cluster with a TASK retry policy, it is strongly recommended to make the +following changes:

+
    +
  • Set the task.low-memory-killer.policy +query management property to +total-reservation-on-blocked-nodes, or queries may +need to be manually killed if the cluster runs out of memory.

  • +
  • Set the query.low-memory-killer.delay +query management property to +0s so the cluster immediately unblocks nodes that run out of memory.

  • +
  • Modify the query.remote-task.max-error-duration +query management property +to adjust how long Trino allows a remote task to try reconnecting before +considering it lost and rescheduling.

  • +
+
+

Note

+

A TASK retry policy is best suited for large batch queries, but this +policy can result in higher latency for short-running queries executed in high +volume. As a best practice, it is recommended to run a dedicated cluster +with a TASK retry policy for large batch queries, separate from another +cluster that handles short queries.

+
+
+
+
+

Advanced configuration#

+

You can further configure fault-tolerant execution with the following +configuration properties. The default values for these properties should work +for most deployments, but you can change these values for testing or +troubleshooting purposes.

+
+

Retry limits#

+

The following configuration properties control the thresholds at which +queries/tasks are no longer retried in the event of repeated failures:

+ + ++++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Fault tolerance retry limit configuration properties#

Property name

Description

Default value

Retry policy

query-retry-attempts

Maximum number of times Trino may attempt to retry a query before declaring +the query as failed.

4

Only QUERY

task-retry-attempts-per-task

Maximum number of times Trino may attempt to retry a single task before +declaring the query as failed.

4

Only TASK

retry-initial-delay

Minimum time that a failed query or task must wait +before it is retried. May be overridden with the retry_initial_delay +session property.

10s

QUERY and TASK

retry-max-delay

Maximum :ref:time <prop-type-duration> that a failed query or task must +wait before it is retried. Wait time is increased on each subsequent +failure. May be overridden with the retry_max_delay session +property.

1m

QUERY and TASK

retry-delay-scale-factor

Factor by which retry delay is increased on each query or task failure. May +be overridden with the retry_delay_scale_factor session +property.

2.0

QUERY and TASK

+
+
+

Task sizing#

+

With a TASK retry policy, it is important to manage the amount of data +processed in each task. If tasks are too small, the management of task +coordination can take more processing time and resources than executing the task +itself. If tasks are too large, then a single task may require more resources +than are available on any one node and therefore prevent the query from +completing.

+

Trino supports limited automatic task sizing. If issues are occurring +during fault-tolerant task execution, you can configure the following +configuration properties to manually control task sizing. These configuration +properties only apply to a TASK retry policy.

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Task sizing configuration properties#

Property name

Description

Default value

fault-tolerant-execution-standard-split-size

Standard split data size +processed by tasks that read data from source tables. Value is interpreted +with split weight taken into account. If the weight of splits produced by a +catalog denotes that they are lighter or heavier than “standard” split, then +the number of splits processed by a single task is adjusted accordingly.

+

May be overridden for the current session with the +fault_tolerant_execution_standard_split_size session +property.

+

64MB

fault-tolerant-execution-max-task-split-count

Maximum number of splits processed by a single task. +This value is not split weight-adjusted and serves as protection against +situations where catalogs report an incorrect split weight.

+

May be overridden for the current session with the +fault_tolerant_execution_max_task_split_count session +property.

+

256

fault-tolerant-execution-arbitrary-distribution-compute-task-target-size-growth-period

The number of tasks created for any given non-writer stage of arbitrary +distribution before task size is increased.

64

fault-tolerant-execution-arbitrary-distribution-compute-task-target-size-growth-factor

Growth factor for adaptive sizing of non-writer tasks of arbitrary +distribution for fault-tolerant execution. Lower bound is 1.0. For every +task size increase, new task target size is old task target size multiplied +by this growth factor.

1.26

fault-tolerant-execution-arbitrary-distribution-compute-task-target-size-min

Initial/minimum target input data size for non-writer +tasks of arbitrary distribution of fault-tolerant execution.

512MB

fault-tolerant-execution-arbitrary-distribution-compute-task-target-size-max

Maximum target input data size for each non-writer +task of arbitrary distribution of fault-tolerant execution.

50GB

fault-tolerant-execution-arbitrary-distribution-write-task-target-size-growth-period

The number of tasks created for any given writer stage of arbitrary +distribution before task size is increased.

64

fault-tolerant-execution-arbitrary-distribution-write-task-target-size-growth-factor

Growth factor for adaptive sizing of writer tasks of arbitrary distribution +for fault-tolerant execution. Lower bound is 1.0. For every task size +increase, new task target size is old task target size multiplied by this +growth factor.

1.26

fault-tolerant-execution-arbitrary-distribution-write-task-target-size-min

Initial/minimum target input data size for writer +tasks of arbitrary distribution of fault-tolerant execution.

4GB

fault-tolerant-execution-arbitrary-distribution-write-task-target-size-max

Maximum target input data size for writer tasks of +arbitrary distribution of fault-tolerant execution.

50GB

fault-tolerant-execution-hash-distribution-compute-task-target-size

Target input data size for non-writer tasks of hash +distribution of fault-tolerant execution.

512MB

fault-tolerant-execution-hash-distribution-write-task-target-size

Target input data size of writer tasks of hash +distribution of fault-tolerant execution.

4GB

fault-tolerant-execution-hash-distribution-write-task-target-max-count

Soft upper bound on number of writer tasks in a stage of hash distribution +of fault-tolerant execution.

2000

+
+
+

Node allocation#

+

With a TASK retry policy, nodes are allocated to tasks based on available +memory and estimated memory usage. If task failure occurs due to exceeding +available memory on a node, the task is restarted with a request to allocate the +full node for its execution.

+

The initial task memory-requirements estimation is static and configured with +the fault-tolerant-task-memory configuration property. This property only +applies to a TASK retry policy.

+ + +++++ + + + + + + + + + + + + +
Node allocation configuration properties#

Property name

Description

Default value

fault-tolerant-execution-task-memory

Initial task memory data size estimation +used for bin-packing when allocating nodes for tasks. May be overridden +for the current session with the +fault_tolerant_execution_task_memory +session property.

5GB

+
+
+

Other tuning#

+

The following additional configuration property can be used to manage +fault-tolerant execution:

+ + ++++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Other fault-tolerant execution configuration properties#

Property name

Description

Default value

Retry policy

fault-tolerant-execution-task-descriptor-storage-max-memory

Maximum data size of memory to be used to +store task descriptors for fault tolerant queries on coordinator. Extra +memory is needed to be able to reschedule tasks in case of a failure.

(JVM heap size * 0.15)

Only TASK

fault-tolerant-execution-max-partition-count

Maximum number of partitions to use for distributed joins and aggregations, +similar in function to the query.max-hash-partition-count query +management property. It is not +recommended to increase this property value above the default of 50, which +may result in instability and poor performance. May be overridden for the +current session with the fault_tolerant_execution_max_partition_count +session property.

50

Only TASK

fault-tolerant-execution-min-partition-count

Minimum number of partitions to use for distributed joins and aggregations, +similar in function to the query.min-hash-partition-count query +management property. May be overridden +for the current session with the +fault_tolerant_execution_min_partition_count session +property.

4

Only TASK

fault-tolerant-execution-min-partition-count-for-write

Minimum number of partitions to use for distributed joins and aggregations +in write queries, similar in function to the +query.min-hash-partition-count-for-write query management +property. May be overridden for the +current session with the +fault_tolerant_execution_min_partition_count_for_write session +property.

50

Only TASK

max-tasks-waiting-for-node-per-stage

Allow for up to configured number of tasks to wait for node allocation +per stage, before pausing scheduling for other tasks from this stage.

5

Only TASK

+
+
+
+

Exchange manager#

+

Exchange spooling is responsible for storing and managing spooled data for +fault-tolerant execution. You can configure a filesystem-based exchange manager +that stores spooled data in a specified location, such as AWS S3 and S3-compatible systems, Azure Blob Storage, Google Cloud Storage, +or HDFS.

+
+

Configuration#

+

To configure an exchange manager, create a new +etc/exchange-manager.properties configuration file on the coordinator and +all worker nodes. In this file, set the exchange-manager.name configuration +property to filesystem or hdfs, and set additional configuration properties as needed +for your storage solution.

+

The following table lists the available configuration properties for +exchange-manager.properties, their default values, and which filesystem(s) +the property may be configured for:

+ + ++++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Exchange manager configuration properties#

Property name

Description

Default value

Supported filesystem

exchange.base-directories

Comma-separated list of URI locations that the exchange manager uses to +store spooling data.

Any

exchange.sink-buffer-pool-min-size

The minimum buffer pool size for an exchange sink. The larger the buffer +pool size, the larger the write parallelism and memory usage.

10

Any

exchange.sink-buffers-per-partition

The number of buffers per partition in the buffer pool. The larger the +buffer pool size, the larger the write parallelism and memory usage.

2

Any

exchange.sink-max-file-size

Max data size of files written by exchange sinks.

1GB

Any

exchange.source-concurrent-readers

Number of concurrent readers to read from spooling storage. The larger the +number of concurrent readers, the larger the read parallelism and memory +usage.

4

Any

exchange.s3.aws-access-key

AWS access key to use. Required for a connection to AWS S3 and GCS, can be +ignored for other S3 storage systems.

AWS S3, GCS

exchange.s3.aws-secret-key

AWS secret key to use. Required for a connection to AWS S3 and GCS, can be +ignored for other S3 storage systems.

AWS S3, GCS

exchange.s3.iam-role

IAM role to assume.

AWS S3, GCS

exchange.s3.external-id

External ID for the IAM role trust policy.

AWS S3, GCS

exchange.s3.region

Region of the S3 bucket.

AWS S3, GCS

exchange.s3.endpoint

S3 storage endpoint server if using an S3-compatible storage system that +is not AWS. If using AWS S3, this can be ignored. If using GCS, set it +to https://storage.googleapis.com.

Any S3-compatible storage

exchange.s3.max-error-retries

Maximum number of times the exchange manager’s S3 client should retry +a request.

10

Any S3-compatible storage

exchange.s3.path-style-access

Enables using path-style access +for all requests to S3.

false

Any S3-compatible storage

exchange.s3.upload.part-size

Part data size for S3 multi-part upload.

5MB

Any S3-compatible storage

exchange.gcs.json-key-file-path

Path to the JSON file that contains your Google Cloud Platform service +account key. Not to be set together with exchange.gcs.json-key

GCS

exchange.gcs.json-key

Your Google Cloud Platform service account key in JSON format. Not to be set +together with exchange.gcs.json-key-file-path

GCS

exchange.azure.connection-string

Connection string used to access the spooling container.

Azure Blob Storage

exchange.azure.block-size

Block data size for Azure block blob parallel upload.

4MB

Azure Blob Storage

exchange.azure.max-error-retries

Maximum number of times the exchange manager’s Azure client should +retry a request.

10

Azure Blob Storage

exchange.hdfs.block-size

Block data size for HDFS storage.

4MB

HDFS

hdfs.config.resources

Comma-separated list of paths to HDFS configuration files, for example +/etc/hdfs-site.xml. The files must exist on all nodes in the Trino +cluster.

HDFS

+

It is recommended to set the exchange.compression-enabled property to +true in the cluster’s config.properties file, to reduce the exchange +manager’s overall I/O load. It is also recommended to configure a bucket +lifecycle rule to automatically expire abandoned objects in the event of a node +crash.

+
+

AWS S3#

+

The following example exchange-manager.properties configuration specifies an +AWS S3 bucket as the spooling storage destination. Note that the destination +does not have to be in AWS, but can be any S3-compatible storage system.

+
exchange-manager.name=filesystem
+exchange.base-directories=s3://exchange-spooling-bucket
+exchange.s3.region=us-west-1
+exchange.s3.aws-access-key=example-access-key
+exchange.s3.aws-secret-key=example-secret-key
+
+
+

You can configure multiple S3 buckets for the exchange manager to distribute +spooled data across buckets, reducing the I/O load on any one bucket. If a query +fails with the error message +“software.amazon.awssdk.services.s3.model.S3Exception: Please reduce your +request rate”, this indicates that the workload is I/O intensive, and you should +specify multiple S3 buckets in exchange.base-directories to balance the +load:

+
exchange.base-directories=s3://exchange-spooling-bucket-1,s3://exchange-spooling-bucket-2
+
+
+
+
+

Azure Blob Storage#

+

The following example exchange-manager.properties configuration specifies an +Azure Blob Storage container as the spooling storage destination. You must use +Azure Blob Storage, not Azure Data Lake Storage or any other hierarchical +storage option in Azure.

+
exchange-manager.name=filesystem
+exchange.base-directories=abfs://container_name@account_name.dfs.core.windows.net
+exchange.azure.connection-string=connection-string
+
+
+
+
+

Google Cloud Storage#

+

To enable exchange spooling on GCS in Trino, change the request endpoint to the +https://storage.googleapis.com Google storage URI, and configure your AWS +access/secret keys to use the GCS HMAC keys. If you deploy Trino on GCP, you +must either create a service account with access to your spooling bucket or +configure the key path to your GCS credential file.

+

For more information on GCS’s S3 compatibility, refer to the Google Cloud +documentation on S3 migration.

+

The following example exchange-manager.properties configuration specifies a +GCS bucket as the spooling storage destination.

+
exchange-manager.name=filesystem
+exchange.base-directories=gs://exchange-spooling-bucket
+exchange.s3.region=us-west-1
+exchange.s3.aws-access-key=example-access-key
+exchange.s3.aws-secret-key=example-secret-key
+exchange.s3.endpoint=https://storage.googleapis.com
+exchange.gcs.json-key-file-path=/path/to/gcs_keyfile.json
+
+
+
+
+

HDFS#

+

The following exchange-manager.properties configuration example specifies HDFS +as the spooling storage destination.

+
exchange-manager.name=hdfs
+exchange.base-directories=hadoop-master:9000/exchange-spooling-directory
+hdfs.config.resources=/usr/lib/hadoop/etc/hadoop/core-site.xml
+
+
+
+
+

Local filesystem storage#

+

The following example exchange-manager.properties configuration specifies a +local directory, /tmp/trino-exchange-manager, as the spooling storage +destination.

+
+

Note

+

It is only recommended to use a local filesystem for exchange in standalone, +non-production clusters. A local directory can only be used for exchange in +a distributed cluster if the exchange directory is shared and accessible +from all worker nodes.

+
+
exchange-manager.name=filesystem
+exchange.base-directories=/tmp/trino-exchange-manager
+
+
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/graceful-shutdown.html b/430/admin/graceful-shutdown.html new file mode 100644 index 000000000..9bef4e88c --- /dev/null +++ b/430/admin/graceful-shutdown.html @@ -0,0 +1,548 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Graceful shutdown — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Graceful shutdown#

+

Trino has a graceful shutdown API that can be used exclusively on workers in +order to ensure that they terminate without affecting running queries, given a +sufficient grace period.

+

You can invoke the API with a HTTP PUT request:

+
curl -v -X PUT -d '"SHUTTING_DOWN"' -H "Content-type: application/json" \
+    http://worker:8081/v1/info/state
+
+
+

A successful invocation is logged with a Shutdown requested message at +INFO level in the worker server log.

+

Keep the following aspects in mind:

+
    +
  • If your cluster is secure, you need to provide a basic-authorization header, +or satisfy whatever other security you have enabled.

  • +
  • If you have TLS/HTTPS enabled, you have to ensure the worker certificate is +CA signed, or trusted by the server calling the shut down endpoint. +Otherwise, you can make the call --insecure, but that isn’t recommended.

  • +
  • The default System access control does not allow +graceful shutdowns. You can use the allow-all system access control, or +configure system information rules with the file system access +control. These configuration must be present on all workers.

  • +
+
+

Shutdown behavior#

+

Once the API is called, the worker performs the following steps:

+
    +
  • Go into SHUTTING_DOWN state.

  • +
  • +
    Sleep for shutdown.grace-period, which defaults to 2 minutes.
      +
    • After this, the coordinator is aware of the shutdown and stops sending +tasks to the worker.

    • +
    +
    +
    +
  • +
  • Block until all active tasks are complete.

  • +
  • Sleep for the grace period again in order to ensure the coordinator sees +all tasks are complete.

  • +
  • Shutdown the application.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/jmx.html b/430/admin/jmx.html new file mode 100644 index 000000000..316ce6221 --- /dev/null +++ b/430/admin/jmx.html @@ -0,0 +1,593 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Monitoring with JMX — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Monitoring with JMX#

+

Trino exposes a large number of different metrics via the Java Management Extensions (JMX).

+

You have to enable JMX by setting the ports used by the RMI registry and server +in the config.properties file:

+
jmx.rmiregistry.port=9080
+jmx.rmiserver.port=9081
+
+
+
    +
  • jmx.rmiregistry.port: +Specifies the port for the JMX RMI registry. JMX clients should connect to this port.

  • +
  • jmx.rmiserver.port: +Specifies the port for the JMX RMI server. Trino exports many metrics, +that are useful for monitoring via JMX.

  • +
+

Additionally configure a Java system property in the +jvm.config with the RMI server port:

+
-Dcom.sun.management.jmxremote.rmi.port=9081
+
+
+

JConsole (supplied with the JDK), VisualVM, and +many other tools can be used to access the metrics in a client application. +Many monitoring solutions support JMX. You can also use the +JMX connector and query the metrics using SQL.

+

Many of these JMX metrics are a complex metric object such as a CounterStat +that has a collection of related metrics. For example, InputPositions has +InputPositions.TotalCount, InputPositions.OneMinute.Count, and so on.

+

A small subset of the available metrics are described below.

+
+

JVM#

+
    +
  • Heap size: java.lang:type=Memory:HeapMemoryUsage.used

  • +
  • Thread count: java.lang:type=Threading:ThreadCount

  • +
+
+
+

Trino cluster and nodes#

+
    +
  • Active nodes: +trino.failuredetector:name=HeartbeatFailureDetector:ActiveCount

  • +
  • Free memory (general pool): +trino.memory:type=ClusterMemoryPool:name=general:FreeDistributedBytes

  • +
  • Cumulative count (since Trino started) of queries that ran out of memory and were killed: +trino.memory:name=ClusterMemoryManager:QueriesKilledDueToOutOfMemory

  • +
+
+
+

Trino queries#

+
    +
  • Active queries currently executing or queued: trino.execution:name=QueryManager:RunningQueries

  • +
  • Queries started: trino.execution:name=QueryManager:StartedQueries.FiveMinute.Count

  • +
  • Failed queries from last 5 min (all): trino.execution:name=QueryManager:FailedQueries.FiveMinute.Count

  • +
  • Failed queries from last 5 min (internal): trino.execution:name=QueryManager:InternalFailures.FiveMinute.Count

  • +
  • Failed queries from last 5 min (external): trino.execution:name=QueryManager:ExternalFailures.FiveMinute.Count

  • +
  • Failed queries (user): trino.execution:name=QueryManager:UserErrorFailures.FiveMinute.Count

  • +
  • Execution latency (P50): trino.execution:name=QueryManager:ExecutionTime.FiveMinutes.P50

  • +
  • Input data rate (P90): trino.execution:name=QueryManager:WallInputBytesRate.FiveMinutes.P90

  • +
+
+
+

Trino tasks#

+
    +
  • Input data bytes: trino.execution:name=SqlTaskManager:InputDataSize.FiveMinute.Count

  • +
  • Input rows: trino.execution:name=SqlTaskManager:InputPositions.FiveMinute.Count

  • +
+
+
+

Connectors#

+

Many connectors provide their own metrics. The metric names typically start with +trino.plugin.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/properties-exchange.html b/430/admin/properties-exchange.html new file mode 100644 index 000000000..c6cfd6734 --- /dev/null +++ b/430/admin/properties-exchange.html @@ -0,0 +1,724 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Exchange properties — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Exchange properties#

+

Exchanges transfer data between Trino nodes for different stages of +a query. Adjusting these properties may help to resolve inter-node +communication issues or improve network utilization.

+
+

exchange.client-threads#

+
    +
  • Type: integer

  • +
  • Minimum value: 1

  • +
  • Default value: 25

  • +
+

Number of threads used by exchange clients to fetch data from other Trino +nodes. A higher value can improve performance for large clusters or clusters +with very high concurrency, but excessively high values may cause a drop +in performance due to context switches and additional memory usage.

+
+
+

exchange.concurrent-request-multiplier#

+
    +
  • Type: integer

  • +
  • Minimum value: 1

  • +
  • Default value: 3

  • +
+

Multiplier determining the number of concurrent requests relative to +available buffer memory. The maximum number of requests is determined +using a heuristic of the number of clients that can fit into available +buffer space, based on average buffer usage per request times this +multiplier. For example, with an exchange.max-buffer-size of 32 MB +and 20 MB already used and average size per request being 2MB, +the maximum number of clients is +multiplier * ((32MB - 20MB) / 2MB) = multiplier * 6. Tuning this +value adjusts the heuristic, which may increase concurrency and improve +network utilization.

+
+
+

exchange.data-integrity-verification#

+
    +
  • Type: string

  • +
  • Allowed values: NONE, ABORT, RETRY

  • +
  • Default value: ABORT

  • +
+

Configure the resulting behavior of data integrity issues. By default, +ABORT causes queries to be aborted when data integrity issues are +detected as part of the built-in verification. Setting the property to +NONE disables the verification. RETRY causes the data exchange to be +repeated when integrity issues are detected.

+
+
+

exchange.max-buffer-size#

+ +

Size of buffer in the exchange client that holds data fetched from other +nodes before it is processed. A larger buffer can increase network +throughput for larger clusters, and thus decrease query processing time, +but reduces the amount of memory available for other usages.

+
+
+

exchange.max-response-size#

+
    +
  • Type: data size

  • +
  • Minimum value: 1MB

  • +
  • Default value: 16MB

  • +
+

Maximum size of a response returned from an exchange request. The response +is placed in the exchange client buffer, which is shared across all +concurrent requests for the exchange.

+

Increasing the value may improve network throughput, if there is high +latency. Decreasing the value may improve query performance for large +clusters as it reduces skew, due to the exchange client buffer holding +responses for more tasks, rather than hold more data from fewer tasks.

+
+
+

sink.max-buffer-size#

+ +

Output buffer size for task data that is waiting to be pulled by upstream +tasks. If the task output is hash partitioned, then the buffer is +shared across all of the partitioned consumers. Increasing this value may +improve network throughput for data transferred between stages, if the +network has high latency, or if there are many nodes in the cluster.

+
+
+

sink.max-broadcast-buffer-size#

+
    +
  • Type data size

  • +
  • Default value: 200MB

  • +
+

Broadcast output buffer size for task data that is waiting to be pulled by +upstream tasks. The broadcast buffer is used to store and transfer build side +data for replicated joins. If the buffer is too small, it prevents scaling of +join probe side tasks, when new nodes are added to the cluster.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/properties-general.html b/430/admin/properties-general.html new file mode 100644 index 000000000..fa234785e --- /dev/null +++ b/430/admin/properties-general.html @@ -0,0 +1,690 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + General properties — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

General properties#

+
+

join-distribution-type#

+
    +
  • Type: string

  • +
  • Allowed values: AUTOMATIC, PARTITIONED, BROADCAST

  • +
  • Default value: AUTOMATIC

  • +
  • Session property: join_distribution_type

  • +
+

The type of distributed join to use. When set to PARTITIONED, Trino +uses hash distributed joins. When set to BROADCAST, it broadcasts the +right table to all nodes in the cluster that have data from the left table. +Partitioned joins require redistributing both tables using a hash of the join key. +This can be slower, sometimes substantially, than broadcast joins, but allows much +larger joins. In particular broadcast joins are faster, if the right table is +much smaller than the left. However, broadcast joins require that the tables on the right +side of the join after filtering fit in memory on each node, whereas distributed joins +only need to fit in distributed memory across all nodes. When set to AUTOMATIC, +Trino makes a cost based decision as to which distribution type is optimal. +It considers switching the left and right inputs to the join. In AUTOMATIC +mode, Trino defaults to hash distributed joins if no cost could be computed, such as if +the tables do not have statistics.

+
+
+

redistribute-writes#

+
    +
  • Type: boolean

  • +
  • Default value: true

  • +
  • Session property: redistribute_writes

  • +
+

This property enables redistribution of data before writing. This can +eliminate the performance impact of data skew when writing by hashing it +across nodes in the cluster. It can be disabled, when it is known that the +output data set is not skewed, in order to avoid the overhead of hashing and +redistributing all the data across the network.

+
+
+

protocol.v1.alternate-header-name#

+

Type: string

+

The 351 release of Trino changes the HTTP client protocol headers to start with +X-Trino-. Clients for versions 350 and lower expect the HTTP headers to +start with X-Presto-, while newer clients expect X-Trino-. You can support these +older clients by setting this property to Presto.

+

The preferred approach to migrating from versions earlier than 351 is to update +all clients together with the release, or immediately afterwards, and then +remove usage of this property.

+

Ensure to use this only as a temporary measure to assist in your migration +efforts.

+
+
+

protocol.v1.prepared-statement-compression.length-threshold#

+
    +
  • Type: integer

  • +
  • Default value: 2048

  • +
+

Prepared statements that are submitted to Trino for processing, and are longer +than the value of this property, are compressed for transport via the HTTP +header to improve handling, and to avoid failures due to hitting HTTP header +size limits.

+
+
+

protocol.v1.prepared-statement-compression.min-gain#

+
    +
  • Type: integer

  • +
  • Default value: 512

  • +
+

Prepared statement compression is not applied if the size gain is less than the +configured value. Smaller statements do not benefit from compression, and are +left uncompressed.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/properties-http-client.html b/430/admin/properties-http-client.html new file mode 100644 index 000000000..2da11083b --- /dev/null +++ b/430/admin/properties-http-client.html @@ -0,0 +1,914 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + HTTP client properties — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

HTTP client properties#

+

HTTP client properties allow you to configure the connection from Trino to +external services using HTTP.

+

The following properties can be used after adding the specific prefix to the +property. For example, for OAuth 2.0 authentication, you can enable HTTP for +interactions with the external OAuth 2.0 provider by adding the prefix +oauth2-jwk to the http-client.connect-timeout property, and increasing +the connection timeout to ten seconds by setting the value to 10:

+
oauth2-jwk.http-client.connect-timeout=10s
+
+
+

The following prefixes are supported:

+ +
+

General properties#

+
+

http-client.connect-timeout#

+
    +
  • Type: duration

  • +
  • Default value: 5s

  • +
  • Minimum value: 0ms

  • +
+

Timeout value for establishing the connection to the external service.

+
+
+

http-client.max-connections#

+
    +
  • Type: integer

  • +
  • Default value: 200

  • +
+

Maximum connections allowed to the service.

+
+
+

http-client.request-timeout#

+
    +
  • Type: duration

  • +
  • Default value: 5m

  • +
  • Minimum value: 0ms

  • +
+

Timeout value for the overall request.

+
+
+
+

TLS and security properties#

+
+

http-client.https.excluded-cipher#

+ +

A comma-separated list of regexes for the names of cipher algorithms to exclude.

+
+
+

http-client.https.included-cipher#

+ +

A comma-separated list of regexes for the names of the cipher algorithms to use.

+
+
+

http-client.https.hostname-verification#

+
    +
  • Type: boolean

  • +
  • Default value: true

  • +
+

Verify that the server hostname matches the server DNS name in the +SubjectAlternativeName (SAN) field of the certificate.

+
+
+

http-client.key-store-password#

+ +

Password for the keystore.

+
+
+

http-client.key-store-path#

+ +

File path on the server to the keystore file.

+
+
+

http-client.secure-random-algorithm#

+ +

Set the secure random algorithm for the connection. The default varies by +operating system. Algorithms are specified according to standard algorithm name +documentation.

+

Possible types include NativePRNG, NativePRNGBlocking, +NativePRNGNonBlocking, PKCS11, and SHA1PRNG.

+
+
+

http-client.trust-store-password#

+ +

Password for the truststore.

+
+
+

http-client.trust-store-path#

+ +

File path on the server to the truststore file.

+
+
+
+

Proxy properties#

+
+

http-client.http-proxy#

+ +

Host and port for an HTTP proxy with the format example.net:8080.

+
+
+

http-client.http-proxy.secure#

+
    +
  • Type: boolean

  • +
  • Default value: false

  • +
+

Enable HTTPS for the proxy.

+
+
+

http-client.socks-proxy#

+ +

Host and port for a SOCKS proxy.

+
+
+
+

Request logging#

+
+

http-client.log.compression-enabled#

+
    +
  • Type: boolean

  • +
  • Default value: true

  • +
+

Enable log file compression. The client uses the .gz format for log files.

+
+
+

http-client.log.enabled#

+
    +
  • Type: boolean

  • +
  • Default value: false

  • +
+

Enable logging of HTTP requests.

+
+
+

http-client.log.flush-interval#

+ +

Frequency of flushing the log data to disk.

+
+
+

http-client.log.max-history#

+
    +
  • Type: integer

  • +
  • Default value: 15

  • +
+

Retention limit of log files in days. Files older than the max-history are +deleted when the HTTP client creates files for new logging periods.

+
+
+

http-client.log.max-size#

+ +

Maximum total size of all log files on disk.

+
+
+

http-client.log.path#

+
    +
  • Type: string

  • +
  • Default value: var/log/

  • +
+

Sets the path of the log files. All log files are named http-client.log, and +have the prefix of the specific HTTP client added. For example, +jwk-http-client.log.

+
+
+

http-client.log.queue-size#

+
    +
  • Type: integer

  • +
  • Default value: 10000

  • +
  • Minimum value: 1

  • +
+

Size of the HTTP client logging queue.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/properties-logging.html b/430/admin/properties-logging.html new file mode 100644 index 000000000..b0a983881 --- /dev/null +++ b/430/admin/properties-logging.html @@ -0,0 +1,752 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Logging properties — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Logging properties#

+
+

log.annotation-file#

+ +

An optional properties file that contains annotations to be included with +each log message. This can be used to include machine-specific or +environment-specific information into logs which are centrally aggregated. +The annotation values can contain references to environment variables.

+
environment=production
+host=${ENV:HOSTNAME}
+
+
+
+
+

log.format#

+
    +
  • Type: string

  • +
  • Default value: TEXT

  • +
+

The file format for log records. Can be set to either TEXT or JSON. When +set to JSON, the log record is formatted as a JSON object, one record per +line. Any newlines in the field values, such as exception stack traces, are +escaped as normal in the JSON object. This allows for capturing and indexing +exceptions as singular fields in a logging search system.

+
+
+

log.path#

+ +

The path to the log file used by Trino. The path is relative to the data +directory, configured to var/log/server.log by the launcher script as +detailed in Running Trino. Alternatively, you can write logs to separate +the process (typically running next to Trino as a sidecar process) via the TCP +protocol by using a log path of the format tcp://host:port.

+
+
+

log.max-size#

+ +

The maximum file size for the general application log file.

+
+
+

log.max-total-size#

+ +

The maximum file size for all general application log files combined.

+
+
+

log.compression#

+
    +
  • Type: string

  • +
  • Default value: GZIP

  • +
+

The compression format for rotated log files. Can be set to either GZIP or NONE. When +set to NONE, compression is disabled.

+
+
+

http-server.log.enabled#

+
    +
  • Type: boolean

  • +
  • Default value: true

  • +
+

Flag to enable or disable logging for the HTTP server.

+
+
+

http-server.log.compression.enabled#

+
    +
  • Type: boolean

  • +
  • Default value: true

  • +
+

Flag to enable or disable compression of the log files of the HTTP server.

+
+
+

http-server.log.path#

+
    +
  • Type: string

  • +
  • Default value: var/log/http-request.log

  • +
+

The path to the log file used by the HTTP server. The path is relative to +the data directory, configured by the launcher script as detailed in +Running Trino.

+
+
+

http-server.log.max-history#

+
    +
  • Type: integer

  • +
  • Default value: 15

  • +
+

The maximum number of log files for the HTTP server to use, before +log rotation replaces old content.

+
+
+

http-server.log.max-size#

+
    +
  • Type: data size

  • +
  • Default value: unlimited

  • +
+

The maximum file size for the log file of the HTTP server. Defaults to +unlimited, setting a data size value limits the file size +to that value.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/properties-node-scheduler.html b/430/admin/properties-node-scheduler.html new file mode 100644 index 000000000..9218ec1db --- /dev/null +++ b/430/admin/properties-node-scheduler.html @@ -0,0 +1,876 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Node scheduler properties — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Node scheduler properties#

+
+

node-scheduler.include-coordinator#

+
    +
  • Type: boolean

  • +
  • Default value: true

  • +
+

Allows scheduling work on the coordinator so that a single machine can function +as both coordinator and worker. For large clusters, processing work on the +coordinator can negatively impact query performance because the machine’s +resources are not available for the critical coordinator tasks of scheduling, +managing, and monitoring query execution.

+
+

Splits#

+
+
+
+

node-scheduler.max-splits-per-node#

+
    +
  • Type: integer

  • +
  • Default value: 100

  • +
+

The target value for the total number of splits that can be running for +each worker node, assuming all splits have the standard split weight.

+

Using a higher value is recommended, if queries are submitted in large batches +(e.g., running a large group of reports periodically), or for connectors that +produce many splits that complete quickly but do not support assigning split +weight values to express that to the split scheduler. Increasing this value may +improve query latency, by ensuring that the workers have enough splits to keep +them fully utilized.

+

When connectors do support weight based split scheduling, the number of splits +assigned will depend on the weight of the individual splits. If splits are +small, more of them are allowed to be assigned to each worker to compensate.

+

Setting this too high wastes memory and may result in lower performance +due to splits not being balanced across workers. Ideally, it should be set +such that there is always at least one split waiting to be processed, but +not higher.

+
+
+

node-scheduler.min-pending-splits-per-task#

+
    +
  • Type: integer

  • +
  • Default value: 10

  • +
+

The minimum number of outstanding splits with the standard split weight guaranteed to be scheduled on a node (even when the node +is already at the limit for total number of splits) for a single task given the task has remaining splits to process. +Allowing a minimum number of splits per stage is required to prevent starvation and deadlocks.

+

This value must be smaller or equal than max-adjusted-pending-splits-per-task and +node-scheduler.max-splits-per-node, is usually increased for the same reasons, +and has similar drawbacks if set too high.

+
+
+

node-scheduler.max-adjusted-pending-splits-per-task#

+
    +
  • Type: integer

  • +
  • Default value: 2000

  • +
+

The maximum number of outstanding splits with the standard split weight guaranteed to be scheduled on a node (even when the node +is already at the limit for total number of splits) for a single task given the task has remaining splits to process. +Split queue size is adjusted dynamically during split scheduling and cannot exceed node-scheduler.max-adjusted-pending-splits-per-task. +Split queue size per task will be adjusted upward if node processes splits faster than it receives them.

+

Usually increased for the same reasons as node-scheduler.max-splits-per-node, with smaller drawbacks +if set too high.

+
+

Note

+

Only applies for uniform scheduler policy.

+
+
+
+

node-scheduler.max-unacknowledged-splits-per-task#

+
    +
  • Type: integer

  • +
  • Default value: 2000

  • +
+

Maximum number of splits that are either queued on the coordinator, but not yet sent or confirmed to have been received by +the worker. This limit enforcement takes precedence over other existing split limit configurations +like node-scheduler.max-splits-per-node or node-scheduler.max-adjusted-pending-splits-per-task +and is designed to prevent large task update requests that might cause a query to fail.

+
+
+

node-scheduler.min-candidates#

+
    +
  • Type: integer

  • +
  • Minimum value: 1

  • +
  • Default value: 10

  • +
+

The minimum number of candidate nodes that are evaluated by the +node scheduler when choosing the target node for a split. Setting +this value too low may prevent splits from being properly balanced +across all worker nodes. Setting it too high may increase query +latency and increase CPU usage on the coordinator.

+
+
+

node-scheduler.policy#

+
    +
  • Type: string

  • +
  • Allowed values: uniform, topology

  • +
  • Default value: uniform

  • +
+

Sets the node scheduler policy to use when scheduling splits. uniform attempts +to schedule splits on the host where the data is located, while maintaining a uniform +distribution across all hosts. topology tries to schedule splits according to +the topology distance between nodes and splits. It is recommended to use uniform +for clusters where distributed storage runs on the same nodes as Trino workers.

+
+

Network topology#

+
+
+
+

node-scheduler.network-topology.segments#

+
    +
  • Type: string

  • +
  • Default value: machine

  • +
+

A comma-separated string describing the meaning of each segment of a network location. +For example, setting region,rack,machine means a network location contains three segments.

+
+
+

node-scheduler.network-topology.type#

+
    +
  • Type: string

  • +
  • Allowed values: flat, file, subnet

  • +
  • Default value: flat

  • +
+

Sets the network topology type. To use this option, node-scheduler.policy +must be set to topology.

+
    +
  • flat: the topology has only one segment, with one value for each machine.

  • +
  • file: the topology is loaded from a file using the properties +node-scheduler.network-topology.file and +node-scheduler.network-topology.refresh-period described in the +following sections.

  • +
  • subnet: the topology is derived based on subnet configuration provided +through properties node-scheduler.network-topology.subnet.cidr-prefix-lengths +and node-scheduler.network-topology.subnet.ip-address-protocol described +in the following sections.

  • +
+
+

File based network topology#

+
+
+
+

node-scheduler.network-topology.file#

+ +

Load the network topology from a file. To use this option, node-scheduler.network-topology.type +must be set to file. Each line contains a mapping between a host name and a +network location, separated by whitespace. Network location must begin with a leading +/ and segments are separated by a /.

+
192.168.0.1 /region1/rack1/machine1
+192.168.0.2 /region1/rack1/machine2
+hdfs01.example.com /region2/rack2/machine3
+
+
+
+
+

node-scheduler.network-topology.refresh-period#

+
    +
  • Type: duration

  • +
  • Minimum value: 1ms

  • +
  • Default value: 5m

  • +
+

Controls how often the network topology file is reloaded. To use this option, +node-scheduler.network-topology.type must be set to file.

+
+

Subnet based network topology#

+
+
+
+

node-scheduler.network-topology.subnet.ip-address-protocol#

+
    +
  • Type: string

  • +
  • Allowed values: IPv4, IPv6

  • +
  • Default value: IPv4

  • +
+

Sets the IP address protocol to be used for computing subnet based +topology. To use this option, node-scheduler.network-topology.type must +be set to subnet.

+
+
+

node-scheduler.network-topology.subnet.cidr-prefix-lengths#

+

A comma-separated list of integer values defining CIDR prefix +lengths for subnet masks. The prefix lengths must be in increasing order. The +maximum prefix length values for IPv4 and IPv6 protocols are 32 and 128 +respectively. To use this option, node-scheduler.network-topology.type must +be set to subnet.

+

For example, the value 24,25,27 for this property with IPv4 protocol means +that masks applied on the IP address to compute location segments are +255.255.255.0, 255.255.255.128 and 255.255.255.224. So the segments +created for an address 192.168.0.172 are [192.168.0.0, 192.168.0.128, 192.168.0.160, 192.168.0.172].

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/properties-optimizer.html b/430/admin/properties-optimizer.html new file mode 100644 index 000000000..1c712a228 --- /dev/null +++ b/430/admin/properties-optimizer.html @@ -0,0 +1,953 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Optimizer properties — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Optimizer properties#

+
+

optimizer.dictionary-aggregation#

+
    +
  • Type: boolean

  • +
  • Default value: false

  • +
  • Session property: dictionary_aggregation

  • +
+

Enables optimization for aggregations on dictionaries.

+
+
+

optimizer.optimize-hash-generation#

+
    +
  • Type: boolean

  • +
  • Default value: true

  • +
  • Session property: `optimize_hash_generation

  • +
+

Compute hash codes for distribution, joins, and aggregations early during execution, +allowing result to be shared between operations later in the query. This can reduce +CPU usage by avoiding computing the same hash multiple times, but at the cost of +additional network transfer for the hashes. In most cases it decreases overall +query processing time.

+

It is often helpful to disable this property, when using EXPLAIN in order +to make the query plan easier to read.

+
+
+

optimizer.optimize-metadata-queries#

+
    +
  • Type: boolean

  • +
  • Default value: false

  • +
+

Enable optimization of some aggregations by using values that are stored as metadata. +This allows Trino to execute some simple queries in constant time. Currently, this +optimization applies to max, min and approx_distinct of partition +keys and other aggregation insensitive to the cardinality of the input,including +DISTINCT aggregates. Using this may speed up some queries significantly.

+

The main drawback is that it can produce incorrect results, if the connector returns +partition keys for partitions that have no rows. In particular, the Hive connector +can return empty partitions, if they were created by other systems. Trino cannot +create them.

+
+
+

optimizer.mark-distinct-strategy#

+
    +
  • Type: string

  • +
  • Allowed values: AUTOMATIC, ALWAYS, NONE

  • +
  • Default value: AUTOMATIC

  • +
  • Session property: mark_distinct_strategy

  • +
+

The mark distinct strategy to use for distinct aggregations. NONE does not use +MarkDistinct operator. ALWAYS uses MarkDistinct for multiple distinct +aggregations or for mix of distinct and non-distinct aggregations. +AUTOMATIC limits the use of MarkDistinct only for cases with limited +concurrency (global or small cardinality aggregations), where direct distinct +aggregation implementation cannot utilize CPU efficiently.

+
+
+

optimizer.push-aggregation-through-outer-join#

+
    +
  • Type: boolean

  • +
  • Default value: true

  • +
  • Session property: push_aggregation_through_join

  • +
+

When an aggregation is above an outer join and all columns from the outer side of the join +are in the grouping clause, the aggregation is pushed below the outer join. This optimization +is particularly useful for correlated scalar subqueries, which get rewritten to an aggregation +over an outer join. For example:

+
SELECT * FROM item i
+    WHERE i.i_current_price > (
+        SELECT AVG(j.i_current_price) FROM item j
+            WHERE i.i_category = j.i_category);
+
+
+

Enabling this optimization can substantially speed up queries by reducing the +amount of data that needs to be processed by the join. However, it may slow down +some queries that have very selective joins.

+
+
+

optimizer.push-table-write-through-union#

+
    +
  • Type: boolean

  • +
  • Default value: true

  • +
  • Session property: push_table_write_through_union

  • +
+

Parallelize writes when using UNION ALL in queries that write data. This improves the +speed of writing output tables in UNION ALL queries, because these writes do not require +additional synchronization when collecting results. Enabling this optimization can improve +UNION ALL speed, when write speed is not yet saturated. However, it may slow down queries +in an already heavily loaded system.

+
+
+

optimizer.join-reordering-strategy#

+
    +
  • Type: string

  • +
  • Allowed values: AUTOMATIC, ELIMINATE_CROSS_JOINS, NONE

  • +
  • Default value: AUTOMATIC

  • +
  • Session property: join_reordering_strategy

  • +
+

The join reordering strategy to use. NONE maintains the order the tables are listed in the +query. ELIMINATE_CROSS_JOINS reorders joins to eliminate cross joins, where possible, and +otherwise maintains the original query order. When reordering joins, it also strives to maintain the +original table order as much as possible. AUTOMATIC enumerates possible orders, and uses +statistics-based cost estimation to determine the least cost order. If stats are not available, or if +for any reason a cost could not be computed, the ELIMINATE_CROSS_JOINS strategy is used.

+
+
+

optimizer.max-reordered-joins#

+
    +
  • Type: integer

  • +
  • Default value: 9

  • +
+

When optimizer.join-reordering-strategy is set to cost-based, this property determines +the maximum number of joins that can be reordered at once.

+
+

Warning

+

The number of possible join orders scales factorially with the number of +relations, so increasing this value can cause serious performance issues.

+
+
+
+

optimizer.optimize-duplicate-insensitive-joins#

+
    +
  • Type: boolean

  • +
  • Default value: true

  • +
+

Reduces number of rows produced by joins when optimizer detects that duplicated +join output rows can be skipped.

+
+
+

optimizer.use-exact-partitioning#

+
    +
  • Type: boolean

  • +
  • Default value: false

  • +
+

Re-partition data unless the partitioning of the upstream +stage exactly matches what the downstream stage +expects. This can also be specified using the use_exact_partitioning session +property.

+
+
+

optimizer.use-table-scan-node-partitioning#

+
    +
  • Type: boolean

  • +
  • Default value: true

  • +
+

Use connector provided table node partitioning when reading tables. +For example, table node partitioning corresponds to Hive table buckets. +When set to true and minimal partition to task ratio is matched or exceeded, +each table partition is read by a separate worker. The minimal ratio is defined in +optimizer.table-scan-node-partitioning-min-bucket-to-task-ratio.

+

Partition reader assignments are distributed across workers for +parallel processing. Use of table scan node partitioning can improve +query performance by reducing query complexity. For example, +cluster wide data reshuffling might not be needed when processing an aggregation query. +However, query parallelism might be reduced when partition count is +low compared to number of workers.

+
+
+

optimizer.table-scan-node-partitioning-min-bucket-to-task-ratio#

+
    +
  • Type: double

  • +
  • Default value: 0.5

  • +
+

Specifies minimal bucket to task ratio that has to be matched or exceeded in order +to use table scan node partitioning. When the table bucket count is small +compared to the number of workers, then the table scan is distributed across +all workers for improved parallelism.

+
+
+

optimizer.colocated-joins-enabled#

+
    +
  • Type: boolean

  • +
  • Default value: true

  • +
  • Session property: colocated_join

  • +
+

Use co-located joins when both sides of a join have the same table partitioning on the join keys +and the conditions for optimizer.use-table-scan-node-partitioning are met. +For example, a join on bucketed Hive tables with matching bucketing schemes can +avoid exchanging data between workers using a co-located join to improve query performance.

+
+
+

optimizer.filter-conjunction-independence-factor#

+
    +
  • Type: double

  • +
  • Default value: 0.75

  • +
  • Min allowed value: 0

  • +
  • Max allowed value: 1

  • +
+

Scales the strength of independence assumption for estimating the selectivity of +the conjunction of multiple predicates. Lower values for this property will produce +more conservative estimates by assuming a greater degree of correlation between the +columns of the predicates in a conjunction. A value of 0 results in the +optimizer assuming that the columns of the predicates are fully correlated and only +the most selective predicate drives the selectivity of a conjunction of predicates.

+
+
+

optimizer.join-multi-clause-independence-factor#

+
    +
  • Type: double

  • +
  • Default value: 0.25

  • +
  • Min allowed value: 0

  • +
  • Max allowed value: 1

  • +
+

Scales the strength of independence assumption for estimating the output of a +multi-clause join. Lower values for this property will produce more +conservative estimates by assuming a greater degree of correlation between the +columns of the clauses in a join. A value of 0 results in the optimizer +assuming that the columns of the join clauses are fully correlated and only +the most selective clause drives the selectivity of the join.

+
+
+

optimizer.non-estimatable-predicate-approximation.enabled#

+
    +
  • Type: boolean

  • +
  • Default value: true

  • +
+

Enables approximation of the output row count of filters whose costs cannot be +accurately estimated even with complete statistics. This allows the optimizer to +produce more efficient plans in the presence of filters which were previously +not estimated.

+
+
+

optimizer.join-partitioned-build-min-row-count#

+
    +
  • Type: integer

  • +
  • Default value: 1000000

  • +
  • Min allowed value: 0

  • +
+

The minimum number of join build side rows required to use partitioned join lookup. +If the build side of a join is estimated to be smaller than the configured threshold, +single threaded join lookup is used to improve join performance. +A value of 0 disables this optimization.

+
+
+

optimizer.min-input-size-per-task#

+
    +
  • Type: data size

  • +
  • Default value: 5GB

  • +
  • Min allowed value: 0MB

  • +
  • Session property: min_input_size_per_task

  • +
+

The minimum input size required per task. This will help optimizer to determine hash +partition count for joins and aggregations. Limiting hash partition count for small queries +increases concurrency on large clusters where multiple small queries are running concurrently. +The estimated value will always be between min_hash_partition_count and +max_hash_partition_count session property. +A value of 0MB disables this optimization.

+
+
+

optimizer.min-input-rows-per-task#

+
    +
  • Type: integer

  • +
  • Default value: 10000000

  • +
  • Min allowed value: 0

  • +
  • Session property: min_input_rows_per_task

  • +
+

The minimum number of input rows required per task. This will help optimizer to determine hash +partition count for joins and aggregations. Limiting hash partition count for small queries +increases concurrency on large clusters where multiple small queries are running concurrently. +The estimated value will always be between min_hash_partition_count and +max_hash_partition_count session property. +A value of 0 disables this optimization.

+
+
+

optimizer.use-cost-based-partitioning#

+
    +
  • Type: boolean

  • +
  • Default value: true

  • +
  • Session property: use_cost_based_partitioning

  • +
+

When enabled the cost based optimizer is used to determine if repartitioning the output of an +already partitioned stage is necessary.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/properties-query-management.html b/430/admin/properties-query-management.html new file mode 100644 index 000000000..da7d19b37 --- /dev/null +++ b/430/admin/properties-query-management.html @@ -0,0 +1,1005 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Query management properties — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Query management properties#

+
+

query.client.timeout#

+ +

Configures how long the cluster runs without contact from the client +application, such as the CLI, before it abandons and cancels its work.

+
+
+

query.execution-policy#

+
    +
  • Type: string

  • +
  • Default value: phased

  • +
  • Session property: execution_policy

  • +
+

Configures the algorithm to organize the processing of all of the +stages of a query. You can use the following execution policies:

+
    +
  • phased schedules stages in a sequence to avoid blockages because of +inter-stage dependencies. This policy maximizes cluster resource utilization +and provides the lowest query wall time.

  • +
  • all-at-once schedules all of the stages of a query at one time. As a +result, cluster resource utilization is initially high, but inter-stage +dependencies typically prevent full processing and cause longer queue times +which increases the query wall time overall.

  • +
+
+
+

query.determine-partition-count-for-write-enabled#

+
    +
  • Type: boolean

  • +
  • Default value: false

  • +
  • Session property: determine_partition_count_for_write_enabled

  • +
+

Enables determining the number of partitions based on amount of data read and processed by the +query for write queries.

+
+
+

query.max-hash-partition-count#

+
    +
  • Type: integer

  • +
  • Default value: 100

  • +
  • Session property: max_hash_partition_count

  • +
+

The maximum number of partitions to use for processing distributed operations, such as +joins, aggregations, partitioned window functions and others.

+
+
+

query.min-hash-partition-count#

+
    +
  • Type: integer

  • +
  • Default value: 4

  • +
  • Session property: min_hash_partition_count

  • +
+

The minimum number of partitions to use for processing distributed operations, such as +joins, aggregations, partitioned window functions and others.

+
+
+

query.min-hash-partition-count-for-write#

+
    +
  • Type: integer

  • +
  • Default value: 50

  • +
  • Session property: min_hash_partition_count_for_writre

  • +
+

The minimum number of partitions to use for processing distributed operations in write queries, +such as joins, aggregations, partitioned window functions and others.

+
+
+

query.max-writer-tasks-count#

+
    +
  • Type: integer

  • +
  • Default value: 100

  • +
  • Session property: max_writer_tasks_count

  • +
+

The maximum number of tasks that will take part in writing data during +INSERT, CREATE TABLE AS SELECT and EXECUTE queries. +The limit is only applicable when redistribute-writes or scale-writers is be enabled.

+
+
+

query.low-memory-killer.policy#

+
    +
  • Type: string

  • +
  • Default value: total-reservation-on-blocked-nodes

  • +
+

Configures the behavior to handle killing running queries in the event of low +memory availability. Supports the following values:

+
    +
  • none - Do not kill any queries in the event of low memory.

  • +
  • total-reservation - Kill the query currently using the most total memory.

  • +
  • total-reservation-on-blocked-nodes - Kill the query currently using the +most memory specifically on nodes that are now out of memory.

  • +
+
+

Note

+

Only applies for queries with task level retries disabled (retry-policy set to NONE or QUERY)

+
+
+
+

task.low-memory-killer.policy#

+
    +
  • Type: string

  • +
  • Default value: total-reservation-on-blocked-nodes

  • +
+

Configures the behavior to handle killing running tasks in the event of low +memory availability. Supports the following values:

+
    +
  • none - Do not kill any tasks in the event of low memory.

  • +
  • total-reservation-on-blocked-nodes - Kill the tasks which are part of the queries +which has task retries enabled and are currently using the most memory specifically +on nodes that are now out of memory.

  • +
  • least-waste - Kill the tasks which are part of the queries +which has task retries enabled and use significant amount of memory on nodes +which are now out of memory. This policy avoids killing tasks which are already +executing for a long time, so significant amount of work is not wasted.

  • +
+
+

Note

+

Only applies for queries with task level retries enabled (retry-policy=TASK)

+
+
+
+

query.low-memory-killer.delay#

+ +

The amount of time a query is allowed to recover between running out of memory +and being killed, if query.low-memory-killer.policy or +task.low-memory-killer.policy is set to value differnt than none.

+
+
+

query.max-execution-time#

+
    +
  • Type: duration

  • +
  • Default value: 100d

  • +
  • Session property: query_max_execution_time

  • +
+

The maximum allowed time for a query to be actively executing on the +cluster, before it is terminated. Compared to the run time below, execution +time does not include analysis, query planning or wait times in a queue.

+
+
+

query.max-length#

+
    +
  • Type: integer

  • +
  • Default value: 1,000,000

  • +
  • Maximum value: 1,000,000,000

  • +
+

The maximum number of characters allowed for the SQL query text. Longer queries +are not processed, and terminated with error QUERY_TEXT_TOO_LARGE.

+
+
+

query.max-planning-time#

+
    +
  • Type: duration

  • +
  • Default value: 10m

  • +
  • Session property: query_max_planning_time

  • +
+

The maximum allowed time for a query to be actively planning the execution. +After this period the coordinator will make its best effort to stop the +query. Note that some operations in planning phase are not easily cancellable +and may not terminate immediately.

+
+
+

query.max-run-time#

+
    +
  • Type: duration

  • +
  • Default value: 100d

  • +
  • Session property: query_max_run_time

  • +
+

The maximum allowed time for a query to be processed on the cluster, before +it is terminated. The time includes time for analysis and planning, but also +time spend in a queue waiting, so essentially this is the time allowed for a +query to exist since creation.

+
+
+

query.max-scan-physical-bytes#

+
    +
  • Type: data size

  • +
  • Session property: query_max_scan_physical_bytes

  • +
+

The maximum number of bytes that can be scanned by a query during its execution. +When this limit is reached, query processing is terminated to prevent excessive +resource usage.

+
+
+

query.max-stage-count#

+
    +
  • Type: integer

  • +
  • Default value: 150

  • +
  • Minimum value: 1

  • +
+

The maximum number of stages allowed to be generated per query. If a query +generates more stages than this it will get killed with error +QUERY_HAS_TOO_MANY_STAGES.

+
+

Warning

+

Setting this to a high value can cause queries with large number of +stages to introduce instability in the cluster causing unrelated queries +to get killed with REMOTE_TASK_ERROR and the message +Max requests queued per destination exceeded for HttpDestination ...

+
+
+
+

query.max-history#

+
    +
  • Type: integer

  • +
  • Default value: 100

  • +
+

The maximum number of queries to keep in the query history to provide +statistics and other information. If this amount is reached, queries are +removed based on age.

+
+
+

query.min-expire-age#

+ +

The minimal age of a query in the history before it is expired. An expired +query is removed from the query history buffer and no longer available in +the Web UI.

+
+
+

query.remote-task.enable-adaptive-request-size#

+
    +
  • Type: boolean

  • +
  • Default value: true

  • +
  • Session property: query_remote_task_enable_adaptive_request_size

  • +
+

Enables dynamically splitting up server requests sent by tasks, which can +prevent out-of-memory errors for large schemas. The default settings are +optimized for typical usage and should only be modified by advanced users +working with extremely large tables.

+
+
+

query.remote-task.guaranteed-splits-per-task#

+
    +
  • Type: integer

  • +
  • Default value: 3

  • +
  • Session property: query_remote_task_guaranteed_splits_per_task

  • +
+

The minimum number of splits that should be assigned to each remote task to +ensure that each task has a minimum amount of work to perform. Requires +query.remote-task.enable-adaptive-request-size to be enabled.

+
+
+

query.remote-task.max-error-duration#

+ +

Timeout value for remote tasks that fail to communicate with the coordinator. If +the coordinator is unable to receive updates from a remote task before this +value is reached, the coordinator treats the task as failed.

+
+
+

query.remote-task.max-request-size#

+
    +
  • Type: data size

  • +
  • Default value: 8MB

  • +
  • Session property: query_remote_task_max_request_size

  • +
+

The maximum size of a single request made by a remote task. Requires +query.remote-task.enable-adaptive-request-size to be enabled.

+
+
+

query.remote-task.request-size-headroom#

+
    +
  • Type: data size

  • +
  • Default value: 2MB

  • +
  • Session property: query_remote_task_request_size_headroom

  • +
+

Determines the amount of headroom that should be allocated beyond the size of +the request data. Requires query.remote-task.enable-adaptive-request-size to +be enabled.

+
+
+

query.info-url-template#

+
    +
  • Type: string

  • +
  • Default value: (URL of the query info page on the coordinator)

  • +
+

Configure redirection of clients to an alternative location for query +information. The URL must contain a query id placeholder ${QUERY_ID}.

+

For example https://example.com/query/${QUERY_ID}.

+

The ${QUERY_ID} gets replaced with the actual query’s id.

+
+
+

retry-policy#

+
    +
  • Type: string

  • +
  • Default value: NONE

  • +
+

The retry policy to use for +Fault-tolerant execution. Supports the following values:

+
    +
  • NONE - Disable fault-tolerant execution.

  • +
  • TASK - Retry individual tasks within a query in the event of failure. +Requires configuration of an exchange manager.

  • +
  • QUERY - Retry the whole query in the event of failure.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/properties-regexp-function.html b/430/admin/properties-regexp-function.html new file mode 100644 index 000000000..9d81a6676 --- /dev/null +++ b/430/admin/properties-regexp-function.html @@ -0,0 +1,656 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Regular expression function properties — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Regular expression function properties#

+

These properties allow tuning the Regular expression functions.

+
+

regex-library#

+
    +
  • Type: string

  • +
  • Allowed values: JONI, RE2J

  • +
  • Default value: JONI

  • +
+

Which library to use for regular expression functions. +JONI is generally faster for common usage, but can require exponential +time for certain expression patterns. RE2J uses a different algorithm, +which guarantees linear time, but is often slower.

+
+
+

re2j.dfa-states-limit#

+
    +
  • Type: integer

  • +
  • Minimum value: 2

  • +
  • Default value: 2147483647

  • +
+

The maximum number of states to use when RE2J builds the fast, +but potentially memory intensive, deterministic finite automaton (DFA) +for regular expression matching. If the limit is reached, RE2J falls +back to the algorithm that uses the slower, but less memory intensive +non-deterministic finite automaton (NFA). Decreasing this value decreases the +maximum memory footprint of a regular expression search at the cost of speed.

+
+
+

re2j.dfa-retries#

+
    +
  • Type: integer

  • +
  • Minimum value: 0

  • +
  • Default value: 5

  • +
+

The number of times that RE2J retries the DFA algorithm, when +it reaches a states limit before using the slower, but less memory +intensive NFA algorithm, for all future inputs for that search. If hitting the +limit for a given input row is likely to be an outlier, you want to be able +to process subsequent rows using the faster DFA algorithm. If you are likely +to hit the limit on matches for subsequent rows as well, you want to use the +correct algorithm from the beginning so as not to waste time and resources. +The more rows you are processing, the larger this value should be.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/properties-resource-management.html b/430/admin/properties-resource-management.html new file mode 100644 index 000000000..e06a05f21 --- /dev/null +++ b/430/admin/properties-resource-management.html @@ -0,0 +1,723 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Resource management properties — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Resource management properties#

+
+

query.max-cpu-time#

+
    +
  • Type: duration

  • +
  • Default value: 1_000_000_000d

  • +
+

This is the max amount of CPU time that a query can use across the entire +cluster. Queries that exceed this limit are killed.

+
+
+

query.max-memory-per-node#

+
    +
  • Type: data size

  • +
  • Default value: (JVM max memory * 0.3)

  • +
+

This is the max amount of user memory a query can use on a worker. +User memory is allocated during execution for things that are directly +attributable to, or controllable by, a user query. For example, memory used +by the hash tables built during execution, memory used during sorting, etc. +When the user memory allocation of a query on any worker hits this limit, +it is killed.

+
+

Warning

+

The sum of query.max-memory-per-node and +memory.heap-headroom-per-node must be less than the +maximum heap size in the JVM on the node. See JVM config.

+
+
+

Note

+

Does not apply for queries with task level retries enabled (retry-policy=TASK)

+
+
+
+

query.max-memory#

+ +

This is the max amount of user memory a query can use across the entire cluster. +User memory is allocated during execution for things that are directly +attributable to, or controllable by, a user query. For example, memory used +by the hash tables built during execution, memory used during sorting, etc. +When the user memory allocation of a query across all workers hits this limit +it is killed.

+
+

Warning

+

query.max-total-memory must be greater than +query.max-memory.

+
+
+

Note

+

Does not apply for queries with task level retries enabled (retry-policy=TASK)

+
+
+
+

query.max-total-memory#

+
    +
  • Type: data size

  • +
  • Default value: (query.max-memory * 2)

  • +
+

This is the max amount of memory a query can use across the entire cluster, +including revocable memory. When the memory allocated by a query across all +workers hits this limit it is killed. The value of query.max-total-memory +must be greater than query.max-memory.

+
+

Warning

+

query.max-total-memory must be greater than +query.max-memory.

+
+
+

Note

+

Does not apply for queries with task level retries enabled (retry-policy=TASK)

+
+
+
+

memory.heap-headroom-per-node#

+
    +
  • Type: data size

  • +
  • Default value: (JVM max memory * 0.3)

  • +
+

This is the amount of memory set aside as headroom/buffer in the JVM heap +for allocations that are not tracked by Trino.

+
+

Warning

+

The sum of query.max-memory-per-node and +memory.heap-headroom-per-node must be less than the +maximum heap size in the JVM on the node. See JVM config.

+
+
+
+

exchange.deduplication-buffer-size#

+ +

Size of the buffer used for spooled data during +Fault-tolerant execution.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/properties-spilling.html b/430/admin/properties-spilling.html new file mode 100644 index 000000000..f15b59214 --- /dev/null +++ b/430/admin/properties-spilling.html @@ -0,0 +1,723 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Spilling properties — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Spilling properties#

+

These properties control Spill to disk.

+
+

spill-enabled#

+
    +
  • Type: boolean

  • +
  • Default value: false

  • +
  • Session property: spill_enabled

  • +
+

Try spilling memory to disk to avoid exceeding memory limits for the query.

+

Spilling works by offloading memory to disk. This process can allow a query with a large memory +footprint to pass at the cost of slower execution times. Spilling is supported for +aggregations, joins (inner and outer), sorting, and window functions. This property does not +reduce memory usage required for other join types.

+
+
+

spiller-spill-path#

+
    +
  • Type: string

  • +
  • No default value. Must be set when spilling is enabled

  • +
+

Directory where spilled content is written. It can be a comma separated +list to spill simultaneously to multiple directories, which helps to utilize +multiple drives installed in the system.

+

It is not recommended to spill to system drives. Most importantly, do not spill +to the drive on which the JVM logs are written, as disk overutilization might +cause JVM to pause for lengthy periods, causing queries to fail.

+
+
+

spiller-max-used-space-threshold#

+
    +
  • Type: double

  • +
  • Default value: 0.9

  • +
+

If disk space usage ratio of a given spill path is above this threshold, +this spill path is not eligible for spilling.

+
+
+

spiller-threads#

+
    +
  • Type: integer

  • +
  • Default value: 4

  • +
+

Number of spiller threads. Increase this value if the default is not able +to saturate the underlying spilling device (for example, when using RAID).

+
+
+

max-spill-per-node#

+ +

Max spill space to be used by all queries on a single node.

+
+
+

query-max-spill-per-node#

+ +

Max spill space to be used by a single query on a single node.

+
+
+

aggregation-operator-unspill-memory-limit#

+ +

Limit for memory used for unspilling a single aggregation operator instance.

+
+
+

spill-compression-enabled#

+
    +
  • Type: boolean

  • +
  • Default value: false

  • +
+

Enables data compression for pages spilled to disk.

+
+
+

spill-encryption-enabled#

+
    +
  • Type: boolean

  • +
  • Default value: false

  • +
+

Enables using a randomly generated secret key (per spill file) to encrypt and decrypt +data spilled to disk.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/properties-task.html b/430/admin/properties-task.html new file mode 100644 index 000000000..fcf0f7832 --- /dev/null +++ b/430/admin/properties-task.html @@ -0,0 +1,858 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Task properties — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Task properties#

+
+

task.concurrency#

+
    +
  • Type: integer

  • +
  • Restrictions: Must be a power of two

  • +
  • Default value: The number of physical CPUs of the node, with a minimum value of 2 and a maximum of 32

  • +
  • Session property: task_concurrency

  • +
+

Default local concurrency for parallel operators, such as joins and aggregations. +This value should be adjusted up or down based on the query concurrency and worker +resource utilization. Lower values are better for clusters that run many queries +concurrently, because the cluster is already utilized by all the running +queries, so adding more concurrency results in slow downs due to context +switching and other overhead. Higher values are better for clusters that only run +one or a few queries at a time.

+
+
+

task.http-response-threads#

+
    +
  • Type: integer

  • +
  • Minimum value: 1

  • +
  • Default value: 100

  • +
+

Maximum number of threads that may be created to handle HTTP responses. Threads are +created on demand and are cleaned up when idle, thus there is no overhead to a large +value, if the number of requests to be handled is small. More threads may be helpful +on clusters with a high number of concurrent queries, or on clusters with hundreds +or thousands of workers.

+
+
+

task.http-timeout-threads#

+
    +
  • Type: integer

  • +
  • Minimum value: 1

  • +
  • Default value: 3

  • +
+

Number of threads used to handle timeouts when generating HTTP responses. This value +should be increased if all the threads are frequently in use. This can be monitored +via the trino.server:name=AsyncHttpExecutionMBean:TimeoutExecutor +JMX object. If ActiveCount is always the same as PoolSize, increase the +number of threads.

+
+
+

task.info-update-interval#

+
    +
  • Type: duration

  • +
  • Minimum value: 1ms

  • +
  • Maximum value: 10s

  • +
  • Default value: 3s

  • +
+

Controls staleness of task information, which is used in scheduling. Larger values +can reduce coordinator CPU load, but may result in suboptimal split scheduling.

+
+
+

task.max-drivers-per-task#

+
    +
  • Type: integer

  • +
  • Minimum value: 1

  • +
  • Default Value: 2147483647

  • +
+

Controls the maximum number of drivers a task runs concurrently. Setting this value +reduces the likelihood that a task uses too many drivers and can improve concurrent query +performance. This can lead to resource waste if it runs too few concurrent queries.

+
+
+

task.max-partial-aggregation-memory#

+ +

Maximum size of partial aggregation results for distributed aggregations. Increasing this +value can result in less network transfer and lower CPU utilization, by allowing more +groups to be kept locally before being flushed, at the cost of additional memory usage.

+
+
+

task.max-worker-threads#

+
    +
  • Type: integer

  • +
  • Default value: (Node CPUs * 2)

  • +
+

Sets the number of threads used by workers to process splits. Increasing this number +can improve throughput, if worker CPU utilization is low and all the threads are in use, +but it causes increased heap space usage. Setting the value too high may cause a drop +in performance due to a context switching. The number of active threads is available +via the RunningSplits property of the +trino.execution.executor:name=TaskExecutor.RunningSplits JMX object.

+
+
+

task.min-drivers#

+
    +
  • Type: integer

  • +
  • Default value: (task.max-worker-threads * 2)

  • +
+

The target number of running leaf splits on a worker. This is a minimum value because +each leaf task is guaranteed at least 3 running splits. Non-leaf tasks are also +guaranteed to run in order to prevent deadlocks. A lower value may improve responsiveness +for new tasks, but can result in underutilized resources. A higher value can increase +resource utilization, but uses additional memory.

+
+
+

task.min-drivers-per-task#

+
    +
  • Type: integer

  • +
  • Minimum value: 1

  • +
  • Default Value: 3

  • +
+

The minimum number of drivers guaranteed to run concurrently for a single task given +the task has remaining splits to process.

+
+
+

task.scale-writers.enabled#

+ +
+
+

task.min-writer-count#

+
    +
  • Type: integer

  • +
  • Default value: 1

  • +
  • Session property: task_min_writer_count

  • +
+

The number of concurrent writer threads per worker per query when +preferred partitioning and +task writer scaling are not used. Increasing this value may +increase write speed, especially when a query is not I/O bound and can take advantage of +additional CPU for parallel writes.

+

Some connectors can be bottlenecked on the CPU when writing due to compression or other factors. +Setting this too high may cause the cluster to become overloaded due to excessive resource +utilization. Especially when the engine is inserting into a partitioned table without using +preferred partitioning. In such case, each writer thread +could write to all partitions. This can lead to out of memory error since writing to a partition +allocates a certain amount of memory for buffering.

+
+
+

task.max-writer-count#

+
    +
  • Type: integer

  • +
  • Restrictions: Must be a power of two

  • +
  • Default value: The number of physical CPUs of the node, with a minimum value of 2 and a maximum of 64

  • +
  • Session property: task_max_writer_count

  • +
+

The number of concurrent writer threads per worker per query when either +task writer scaling or +preferred partitioning is used. Increasing this value may +increase write speed, especially when a query is not I/O bound and can take advantage of additional +CPU for parallel writes. Some connectors can be bottlenecked on CPU when writing due to compression +or other factors. Setting this too high may cause the cluster to become overloaded due to excessive +resource utilization.

+
+
+

task.interrupt-stuck-split-tasks-enabled#

+
    +
  • Type: boolean

  • +
  • Default value: true

  • +
+

Enables Trino detecting and failing tasks containing splits that have been stuck. Can be +specified by task.interrupt-stuck-split-tasks-timeout and +task.interrupt-stuck-split-tasks-detection-interval. Only applies to threads that +are blocked by the third-party Joni regular expression library.

+
+
+

task.interrupt-stuck-split-tasks-warning-threshold#

+
    +
  • Type: duration

  • +
  • Minimum value: 1m

  • +
  • Default value: 10m

  • +
+

Print out call stacks at /v1/maxActiveSplits endpoint and generate JMX metrics +for splits running longer than the threshold.

+
+
+

task.interrupt-stuck-split-tasks-timeout#

+
    +
  • Type: duration

  • +
  • Minimum value: 3m

  • +
  • Default value: 10m

  • +
+

The length of time Trino waits for a blocked split processing thread before failing the +task. Only applies to threads that are blocked by the third-party Joni regular +expression library.

+
+
+

task.interrupt-stuck-split-tasks-detection-interval#

+
    +
  • Type: duration

  • +
  • Minimum value: 1m

  • +
  • Default value: 2m

  • +
+

The interval of Trino checks for splits that have processing time exceeding +task.interrupt-stuck-split-tasks-timeout. Only applies to threads that are blocked +by the third-party Joni regular expression library.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/properties-web-interface.html b/430/admin/properties-web-interface.html new file mode 100644 index 000000000..3d0e8b819 --- /dev/null +++ b/430/admin/properties-web-interface.html @@ -0,0 +1,669 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Web UI properties — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Web UI properties#

+

The following properties can be used to configure the Web UI.

+
+

web-ui.authentication.type#

+
    +
  • Type: string

  • +
  • Allowed values: FORM, FIXED, CERTIFICATE, KERBEROS, JWT, OAUTH2

  • +
  • Default value: FORM

  • +
+

The authentication mechanism to allow user access to the Web UI. See +Web UI Authentication.

+
+
+

web-ui.enabled#

+
    +
  • Type: boolean

  • +
  • Default value: true

  • +
+

This property controls whether or not the Web UI is available.

+
+
+

web-ui.shared-secret#

+
    +
  • Type: string

  • +
  • Default value: randomly generated unless set

  • +
+

The shared secret is used to generate authentication cookies for users of +the Web UI. If not set to a static value, any coordinator restart generates +a new random value, which in turn invalidates the session of any currently +logged in Web UI user.

+
+
+

web-ui.session-timeout#

+ +

The duration how long a user can be logged into the Web UI, before the +session times out, which forces an automatic log-out.

+
+
+

web-ui.user#

+
    +
  • Type: string

  • +
  • Default value: None

  • +
+

The username automatically used for authentication to the Web UI with the fixed +authentication type. See Web UI Authentication.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/properties-write-partitioning.html b/430/admin/properties-write-partitioning.html new file mode 100644 index 000000000..576e2b397 --- /dev/null +++ b/430/admin/properties-write-partitioning.html @@ -0,0 +1,618 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Write partitioning properties — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Write partitioning properties#

+
+

use-preferred-write-partitioning#

+
    +
  • Type: boolean

  • +
  • Default value: true

  • +
  • Session property: use_preferred_write_partitioning

  • +
+

Enable preferred write partitioning. When set to true, each partition is +written by a separate writer. For some connectors such as the Hive connector, +only a single new file is written per partition, instead of multiple files. +Partition writer assignments are distributed across worker nodes for parallel +processing.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/properties-writer-scaling.html b/430/admin/properties-writer-scaling.html new file mode 100644 index 000000000..4487b81bc --- /dev/null +++ b/430/admin/properties-writer-scaling.html @@ -0,0 +1,655 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Writer scaling properties — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Writer scaling properties#

+

Writer scaling allows Trino to dynamically scale out the number of writer tasks +rather than allocating a fixed number of tasks. Additional tasks are added when +the average amount of physical data per writer is above a minimum threshold, but +only if the query is bottlenecked on writing.

+

Writer scaling is useful with connectors like Hive that produce one or more +files per writer – reducing the number of writers results in a larger average +file size. However, writer scaling can have a small impact on query wall time +due to the decreased writer parallelism while the writer count ramps up to match +the needs of the query.

+
+

scale-writers#

+
    +
  • Type: boolean

  • +
  • Default value: true

  • +
  • Session property: scale_writers

  • +
+

Enable writer scaling by dynamically increasing the number of writer tasks on +the cluster.

+
+
+

task.scale-writers.enabled#

+
    +
  • Type: boolean

  • +
  • Default value: true

  • +
  • Session property: task_scale_writers_enabled

  • +
+

Enable scaling the number of concurrent writers within a task. The maximum +writer count per task for scaling is task.max-writer-count. Additional +writers are added only when the average amount of uncompressed data processed +per writer is above the minimum threshold of writer-scaling-min-data-processed +and query is bottlenecked on writing.

+
+
+

writer-scaling-min-data-processed#

+
    +
  • Type: data size

  • +
  • Default value: 100MB

  • +
  • Session property: writer_scaling_min_data_processed

  • +
+

The minimum amount of uncompressed data that must be processed by a writer +before another writer can be added.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/properties.html b/430/admin/properties.html new file mode 100644 index 000000000..b410414e5 --- /dev/null +++ b/430/admin/properties.html @@ -0,0 +1,727 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Properties reference — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Properties reference#

+

This section describes the most important configuration properties and (where +applicable) their corresponding session properties, that may be used to tune Trino or alter its +behavior when required. Unless specified otherwise, configuration properties +must be set on the coordinator and all worker nodes.

+

The following pages are not a complete list of all configuration and +session properties available in Trino, and do not include any connector-specific +catalog configuration properties. For more information on catalog configuration +properties, refer to the connector documentation.

+ +
+

Property value types#

+

Trino configuration properties support different value types with their own +allowed values and syntax. Additional limitations apply on a per-property basis, +and disallowed values result in a validation error.

+
+

boolean#

+

The properties of type boolean support two values, true or false.

+
+
+

data size#

+

The properties of type data size support values that describe an amount of +data, measured in byte-based units. These units are incremented in multiples of +1024, so one megabyte is 1024 kilobytes, one kilobyte is 1024 bytes, and so on. +For example, the value 6GB describes six gigabytes, which is +(6 * 1024 * 1024 * 1024) = 6442450944 bytes.

+

The data size type supports the following units:

+
    +
  • B: Bytes

  • +
  • kB: Kilobytes

  • +
  • MB: Megabytes

  • +
  • GB: Gigabytes

  • +
  • TB: Terabytes

  • +
  • PB: Petabytes

  • +
+
+
+

double#

+

The properties of type double support numerical values including decimals, +such as 1.6. double type values can be negative, if supported by the +specific property.

+
+
+

duration#

+

The properties of type duration support values describing an +amount of time, using the syntax of a non-negative number followed by a time +unit. For example, the value 7m describes seven minutes.

+

The duration type supports the following units:

+
    +
  • ns: Nanoseconds

  • +
  • us: Microseconds

  • +
  • ms: Milliseconds

  • +
  • s: Seconds

  • +
  • m: Minutes

  • +
  • h: Hours

  • +
  • d: Days

  • +
+

A duration of 0 is treated as zero regardless of the unit that follows. +For example, 0s and 0m both mean the same thing.

+

Properties of type duration also support decimal values, such as 2.25d. +These are handled as a fractional value of the specified unit. For example, the +value 1.5m equals one and a half minutes, or 90 seconds.

+
+
+

integer#

+

The properties of type integer support whole numeric values, such as 5 +and 1000. Negative values are supported as well, for example -7. +integer type values must be whole numbers, decimal values such as 2.5 +are not supported.

+

Some integer type properties enforce their own minimum and maximum values.

+
+
+

string#

+

The properties of type string support a set of values that consist of a +sequence of characters. Allowed values are defined on a property-by-property +basis, refer to the specific property for its supported and default values.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/resource-groups.html b/430/admin/resource-groups.html new file mode 100644 index 000000000..2ef90a66d --- /dev/null +++ b/430/admin/resource-groups.html @@ -0,0 +1,1008 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Resource groups — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Resource groups#

+

Resource groups place limits on resource usage, and can enforce queueing policies on +queries that run within them, or divide their resources among sub-groups. A query +belongs to a single resource group, and consumes resources from that group (and its ancestors). +Except for the limit on queued queries, when a resource group runs out of a resource +it does not cause running queries to fail; instead new queries become queued. +A resource group may have sub-groups or may accept queries, but may not do both.

+

The resource groups and associated selection rules are configured by a manager, which is pluggable.

+

You can use a file-based or a database-based resource group manager:

+
    +
  • Add a file etc/resource-groups.properties

  • +
  • Set the resource-groups.configuration-manager property to file or db

  • +
  • Add further configuration properties for the desired manager.

  • +
+
+

File resource group manager#

+

The file resource group manager reads a JSON configuration file, specified with +resource-groups.config-file:

+
resource-groups.configuration-manager=file
+resource-groups.config-file=etc/resource-groups.json
+
+
+

The path to the JSON file can be an absolute path, or a path relative to the Trino +data directory. The JSON file only needs to be present on the coordinator.

+
+
+

Database resource group manager#

+

The database resource group manager loads the configuration from a relational database. The +supported databases are MySQL, PostgreSQL, and Oracle.

+
resource-groups.configuration-manager=db
+resource-groups.config-db-url=jdbc:mysql://localhost:3306/resource_groups
+resource-groups.config-db-user=username
+resource-groups.config-db-password=password
+
+
+

The resource group configuration must be populated through tables +resource_groups_global_properties, resource_groups, and +selectors. If any of the tables do not exist when Trino starts, they +will be created automatically.

+

The rules in the selectors table are processed in descending order of the +values in the priority field.

+

The resource_groups table also contains an environment field which is +matched with the value contained in the node.environment property in +Node properties. This allows the resource group configuration for different +Trino clusters to be stored in the same database if required.

+

The configuration is reloaded from the database every second, and the changes +are reflected automatically for incoming queries.

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Database resource group manager properties#

Property name

Description

Default value

resource-groups.config-db-url

Database URL to load configuration from.

none

resource-groups.config-db-user

Database user to connect with.

none

resource-groups.config-db-password

Password for database user to connect with.

none

resource-groups.max-refresh-interval

The maximum time period for which the cluster will continue to accept +queries after refresh failures, causing configuration to become stale.

1h

resource-groups.refresh-interval

How often the cluster reloads from the database

1s

resource-groups.exact-match-selector-enabled

Setting this flag enables usage of an additional +exact_match_source_selectors table to configure resource group selection +rules defined exact name based matches for source, environment and query +type. By default, the rules are only loaded from the selectors table, with +a regex-based filter for source, among other filters.

false

+
+
+

Resource group properties#

+
    +
  • name (required): name of the group. May be a template (see below).

  • +
  • maxQueued (required): maximum number of queued queries. Once this limit is reached +new queries are rejected.

  • +
  • softConcurrencyLimit (optional): number of concurrently running queries after which +new queries will only run if all peer resource groups below their soft limits are ineligible +or if all eligible peers are above soft limits.

  • +
  • hardConcurrencyLimit (required): maximum number of running queries.

  • +
  • softMemoryLimit (required): maximum amount of distributed memory this +group may use, before new queries become queued. May be specified as +an absolute value (i.e. 1GB) or as a percentage (i.e. 10%) of the cluster’s memory.

  • +
  • softCpuLimit (optional): maximum amount of CPU time this +group may use in a period (see cpuQuotaPeriod), before a penalty is applied to +the maximum number of running queries. hardCpuLimit must also be specified.

  • +
  • hardCpuLimit (optional): maximum amount of CPU time this +group may use in a period.

  • +
  • schedulingPolicy (optional): specifies how queued queries are selected to run, +and how sub-groups become eligible to start their queries. May be one of three values:

    +
      +
    • fair (default): queued queries are processed first-in-first-out, and sub-groups +must take turns starting new queries, if they have any queued.

    • +
    • weighted_fair: sub-groups are selected based on their schedulingWeight and the number of +queries they are already running concurrently. The expected share of running queries for a +sub-group is computed based on the weights for all currently eligible sub-groups. The sub-group +with the least concurrency relative to its share is selected to start the next query.

    • +
    • weighted: queued queries are selected stochastically in proportion to their priority, +specified via the query_priority session property. Sub groups are selected +to start new queries in proportion to their schedulingWeight.

    • +
    • query_priority: all sub-groups must also be configured with query_priority. +Queued queries are selected strictly according to their priority.

    • +
    +
  • +
  • schedulingWeight (optional): weight of this sub-group used in weighted +and the weighted_fair scheduling policy. Defaults to 1. See +Scheduling weight example.

  • +
  • jmxExport (optional): If true, group statistics are exported to JMX for monitoring. +Defaults to false.

  • +
  • subGroups (optional): list of sub-groups.

  • +
+
+

Scheduling weight example#

+

Schedule weighting is a method of assigning a priority to a resource. Sub-groups +with a higher scheduling weight are given higher priority. For example, to +ensure timely execution of scheduled pipelines queries, weight them higher than +adhoc queries.

+

In the following example, pipeline queries are weighted with a value of 350, +which is higher than the adhoc queries that have a scheduling weight of 150. +This means that approximately 70% (350 out of 500 queries) of your queries come +from the pipeline sub-group, and 30% (150 out of 500 queries) come from the adhoc +sub-group in a given timeframe. Alternatively, if you set each sub-group value to +1, the weight of the queries for the pipeline and adhoc sub-groups are split +evenly and each receive 50% of the queries in a given timeframe.

+
{
+  {
+    "name": "pipeline",
+    "schedulingWeight": 350,
+  },
+  {
+    "name": "adhoc",
+    "schedulingWeight": 150
+  }
+}
+
+
+
+
+
+

Selector rules#

+
    +
  • user (optional): regex to match against user name.

  • +
  • userGroup (optional): regex to match against every user group the user belongs to.

  • +
  • source (optional): regex to match against source string.

  • +
  • queryType (optional): string to match against the type of the query submitted:

    +
      +
    • SELECT: SELECT queries.

    • +
    • EXPLAIN: EXPLAIN queries (but not EXPLAIN ANALYZE).

    • +
    • DESCRIBE: DESCRIBE, DESCRIBE INPUT, DESCRIBE OUTPUT, and SHOW queries.

    • +
    • INSERT: INSERT, CREATE TABLE AS, and REFRESH MATERIALIZED VIEW queries.

    • +
    • UPDATE: UPDATE queries.

    • +
    • DELETE: DELETE queries.

    • +
    • ANALYZE: ANALYZE queries.

    • +
    • DATA_DEFINITION: Queries that alter/create/drop the metadata of schemas/tables/views, +and that manage prepared statements, privileges, sessions, and transactions.

    • +
    +
  • +
  • clientTags (optional): list of tags. To match, every tag in this list must be in the list of +client-provided tags associated with the query.

  • +
  • group (required): the group these queries will run in.

  • +
+

Selectors are processed sequentially and the first one that matches will be used.

+
+
+

Global properties#

+
    +
  • cpuQuotaPeriod (optional): the period in which cpu quotas are enforced.

  • +
+
+
+

Providing selector properties#

+

The source name can be set as follows:

+
    +
  • CLI: use the --source option.

  • +
  • JDBC driver when used in client apps: add the source property to the +connection configuration and set the value when using a Java application that +uses the JDBC Driver.

  • +
  • JDBC driver used with Java programs: add a property with the key source +and the value on the Connection instance as shown in the example.

  • +
+

Client tags can be set as follows:

+
    +
  • CLI: use the --client-tags option.

  • +
  • JDBC driver when used in client apps: add the clientTags property to the +connection configuration and set the value when using a Java application that +uses the JDBC Driver.

  • +
  • JDBC driver used with Java programs: add a property with the key +clientTags and the value on the Connection instance as shown in +the example.

  • +
+
+
+

Example#

+

In the example configuration below, there are several resource groups, some of which are templates. +Templates allow administrators to construct resource group trees dynamically. For example, in +the pipeline_${USER} group, ${USER} is expanded to the name of the user that submitted +the query. ${SOURCE} is also supported, which is expanded to the source that submitted the +query. You may also use custom named variables in the source and user regular expressions.

+

There are four selectors, that define which queries run in which resource group:

+
    +
  • The first selector matches queries from bob and places them in the admin group.

  • +
  • The second selector matches queries from admin user group and places them in the admin group.

  • +
  • The third selector matches all data definition (DDL) queries from a source name that includes pipeline +and places them in the global.data_definition group. This could help reduce queue times for this +class of queries, since they are expected to be fast.

  • +
  • The fourth selector matches queries from a source name that includes pipeline, and places them in a +dynamically-created per-user pipeline group under the global.pipeline group.

  • +
  • The fifth selector matches queries that come from BI tools which have a source matching the regular +expression jdbc#(?<toolname>.*) and have client provided tags that are a superset of hipri. +These are placed in a dynamically-created sub-group under the global.adhoc group. +The dynamic sub-groups are created based on the values of named variables toolname and user. +The values are derived from the source regular expression and the query user respectively. +Consider a query with a source jdbc#powerfulbi, user kayla, and client tags hipri and fast. +This query is routed to the global.adhoc.bi-powerfulbi.kayla resource group.

  • +
  • The last selector is a catch-all, which places all queries that have not yet been matched into a per-user +adhoc group.

  • +
+

Together, these selectors implement the following policy:

+
    +
  • The user bob and any user belonging to user group admin +is an admin and can run up to 50 concurrent queries. +Queries will be run based on user-provided priority.

  • +
+

For the remaining users:

+
    +
  • No more than 100 total queries may run concurrently.

  • +
  • Up to 5 concurrent DDL queries with a source pipeline can run. Queries are run in FIFO order.

  • +
  • Non-DDL queries will run under the global.pipeline group, with a total concurrency of 45, and a per-user +concurrency of 5. Queries are run in FIFO order.

  • +
  • For BI tools, each tool can run up to 10 concurrent queries, and each user can run up to 3. If the total demand +exceeds the limit of 10, the user with the fewest running queries gets the next concurrency slot. This policy +results in fairness when under contention.

  • +
  • All remaining queries are placed into a per-user group under global.adhoc.other that behaves similarly.

  • +
+
+

File resource group manager#

+
{
+  "rootGroups": [
+    {
+      "name": "global",
+      "softMemoryLimit": "80%",
+      "hardConcurrencyLimit": 100,
+      "maxQueued": 1000,
+      "schedulingPolicy": "weighted",
+      "jmxExport": true,
+      "subGroups": [
+        {
+          "name": "data_definition",
+          "softMemoryLimit": "10%",
+          "hardConcurrencyLimit": 5,
+          "maxQueued": 100,
+          "schedulingWeight": 1
+        },
+        {
+          "name": "adhoc",
+          "softMemoryLimit": "10%",
+          "hardConcurrencyLimit": 50,
+          "maxQueued": 1,
+          "schedulingWeight": 10,
+          "subGroups": [
+            {
+              "name": "other",
+              "softMemoryLimit": "10%",
+              "hardConcurrencyLimit": 2,
+              "maxQueued": 1,
+              "schedulingWeight": 10,
+              "schedulingPolicy": "weighted_fair",
+              "subGroups": [
+                {
+                  "name": "${USER}",
+                  "softMemoryLimit": "10%",
+                  "hardConcurrencyLimit": 1,
+                  "maxQueued": 100
+                }
+              ]
+            },
+            {
+              "name": "bi-${toolname}",
+              "softMemoryLimit": "10%",
+              "hardConcurrencyLimit": 10,
+              "maxQueued": 100,
+              "schedulingWeight": 10,
+              "schedulingPolicy": "weighted_fair",
+              "subGroups": [
+                {
+                  "name": "${USER}",
+                  "softMemoryLimit": "10%",
+                  "hardConcurrencyLimit": 3,
+                  "maxQueued": 10
+                }
+              ]
+            }
+          ]
+        },
+        {
+          "name": "pipeline",
+          "softMemoryLimit": "80%",
+          "hardConcurrencyLimit": 45,
+          "maxQueued": 100,
+          "schedulingWeight": 1,
+          "jmxExport": true,
+          "subGroups": [
+            {
+              "name": "pipeline_${USER}",
+              "softMemoryLimit": "50%",
+              "hardConcurrencyLimit": 5,
+              "maxQueued": 100
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "name": "admin",
+      "softMemoryLimit": "100%",
+      "hardConcurrencyLimit": 50,
+      "maxQueued": 100,
+      "schedulingPolicy": "query_priority",
+      "jmxExport": true
+    }
+  ],
+  "selectors": [
+    {
+      "user": "bob",
+      "group": "admin"
+    },
+    {
+      "userGroup": "admin",
+      "group": "admin"
+    },
+    {
+      "source": ".*pipeline.*",
+      "queryType": "DATA_DEFINITION",
+      "group": "global.data_definition"
+    },
+    {
+      "source": ".*pipeline.*",
+      "group": "global.pipeline.pipeline_${USER}"
+    },
+    {
+      "source": "jdbc#(?<toolname>.*)",
+      "clientTags": ["hipri"],
+      "group": "global.adhoc.bi-${toolname}.${USER}"
+    },
+    {
+      "group": "global.adhoc.other.${USER}"
+    }
+  ],
+  "cpuQuotaPeriod": "1h"
+}
+
+
+
+
+

Database resource group manager#

+

This example is for a MySQL database.

+
-- global properties
+INSERT INTO resource_groups_global_properties (name, value) VALUES ('cpu_quota_period', '1h');
+
+-- Every row in resource_groups table indicates a resource group.
+-- The enviroment name is 'test_environment', make sure it matches `node.environment` in your cluster.
+-- The parent-child relationship is indicated by the ID in 'parent' column.
+
+-- create a root group 'global' with NULL parent
+INSERT INTO resource_groups (name, soft_memory_limit, hard_concurrency_limit, max_queued, scheduling_policy, jmx_export, environment) VALUES ('global', '80%', 100, 1000, 'weighted', true, 'test_environment');
+
+-- get ID of 'global' group
+SELECT resource_group_id FROM resource_groups WHERE name = 'global';  -- 1
+-- create two new groups with 'global' as parent
+INSERT INTO resource_groups (name, soft_memory_limit, hard_concurrency_limit, max_queued, scheduling_weight, environment, parent) VALUES ('data_definition', '10%', 5, 100, 1, 'test_environment', 1);
+INSERT INTO resource_groups (name, soft_memory_limit, hard_concurrency_limit, max_queued, scheduling_weight, environment, parent) VALUES ('adhoc', '10%', 50, 1, 10, 'test_environment', 1);
+
+-- get ID of 'adhoc' group
+SELECT resource_group_id FROM resource_groups WHERE name = 'adhoc';   -- 3
+-- create 'other' group with 'adhoc' as parent
+INSERT INTO resource_groups (name, soft_memory_limit, hard_concurrency_limit, max_queued, scheduling_weight, scheduling_policy, environment, parent) VALUES ('other', '10%', 2, 1, 10, 'weighted_fair', 'test_environment', 3);
+
+-- get ID of 'other' group
+SELECT resource_group_id FROM resource_groups WHERE name = 'other';  -- 4
+-- create '${USER}' group with 'other' as parent.
+INSERT INTO resource_groups (name, soft_memory_limit, hard_concurrency_limit, max_queued, environment, parent) VALUES ('${USER}', '10%', 1, 100, 'test_environment', 4);
+
+-- create 'bi-${toolname}' group with 'adhoc' as parent
+INSERT INTO resource_groups (name, soft_memory_limit, hard_concurrency_limit, max_queued, scheduling_weight, scheduling_policy, environment, parent) VALUES ('bi-${toolname}', '10%', 10, 100, 10, 'weighted_fair', 'test_environment', 3);
+
+-- get ID of 'bi-${toolname}' group
+SELECT resource_group_id FROM resource_groups WHERE name = 'bi-${toolname}';  -- 6
+-- create '${USER}' group with 'bi-${toolname}' as parent. This indicates
+-- nested group 'global.adhoc.bi-${toolname}.${USER}', and will have a
+-- different ID than 'global.adhoc.other.${USER}' created above.
+INSERT INTO resource_groups (name, soft_memory_limit, hard_concurrency_limit, max_queued,  environment, parent) VALUES ('${USER}', '10%', 3, 10, 'test_environment', 6);
+
+-- create 'pipeline' group with 'global' as parent
+INSERT INTO resource_groups (name, soft_memory_limit, hard_concurrency_limit, max_queued, scheduling_weight, jmx_export, environment, parent) VALUES ('pipeline', '80%', 45, 100, 1, true, 'test_environment', 1);
+
+-- get ID of 'pipeline' group
+SELECT resource_group_id FROM resource_groups WHERE name = 'pipeline'; -- 8
+-- create 'pipeline_${USER}' group with 'pipeline' as parent
+INSERT INTO resource_groups (name, soft_memory_limit, hard_concurrency_limit, max_queued,  environment, parent) VALUES ('pipeline_${USER}', '50%', 5, 100, 'test_environment', 8);
+
+-- create a root group 'admin' with NULL parent
+INSERT INTO resource_groups (name, soft_memory_limit, hard_concurrency_limit, max_queued, scheduling_policy, environment, jmx_export) VALUES ('admin', '100%', 50, 100, 'query_priority', 'test_environment', true);
+
+
+-- Selectors
+
+-- use ID of 'admin' resource group for selector
+INSERT INTO selectors (resource_group_id, user_regex, priority) VALUES ((SELECT resource_group_id FROM resource_groups WHERE name = 'admin'), 'bob', 6);
+
+-- use ID of 'admin' resource group for selector
+INSERT INTO selectors (resource_group_id, user_group_regex, priority) VALUES ((SELECT resource_group_id FROM resource_groups WHERE name = 'admin'), 'admin', 5);
+
+-- use ID of 'global.data_definition' resource group for selector
+INSERT INTO selectors (resource_group_id, source_regex, query_type, priority) VALUES ((SELECT resource_group_id FROM resource_groups WHERE name = 'data_definition'), '.*pipeline.*', 'DATA_DEFINITION', 4);
+
+-- use ID of 'global.pipeline.pipeline_${USER}' resource group for selector
+INSERT INTO selectors (resource_group_id, source_regex, priority) VALUES ((SELECT resource_group_id FROM resource_groups WHERE name = 'pipeline_${USER}'), '.*pipeline.*', 3);
+
+-- get ID of 'global.adhoc.bi-${toolname}.${USER}' resource group by disambiguating group name using parent ID
+SELECT A.resource_group_id self_id, B.resource_group_id parent_id, concat(B.name, '.', A.name) name_with_parent
+FROM resource_groups A JOIN resource_groups B ON A.parent = B.resource_group_id
+WHERE A.name = '${USER}' AND B.name = 'bi-${toolname}';
+--  7 |         6 | bi-${toolname}.${USER}
+INSERT INTO selectors (resource_group_id, source_regex, client_tags, priority) VALUES (7, 'jdbc#(?<toolname>.*)', '["hipri"]', 2);
+
+-- get ID of 'global.adhoc.other.${USER}' resource group for by disambiguating group name using parent ID
+SELECT A.resource_group_id self_id, B.resource_group_id parent_id, concat(B.name, '.', A.name) name_with_parent
+FROM resource_groups A JOIN resource_groups B ON A.parent = B.resource_group_id
+WHERE A.name = '${USER}' AND B.name = 'other';
+-- |       5 |         4 | other.${USER}    |
+INSERT INTO selectors (resource_group_id, priority) VALUES (5, 1);
+
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/session-property-managers.html b/430/admin/session-property-managers.html new file mode 100644 index 000000000..398b5e27a --- /dev/null +++ b/430/admin/session-property-managers.html @@ -0,0 +1,591 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Session property managers — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Session property managers#

+

Administrators can add session properties to control the behavior for subsets of their workload. +These properties are defaults, and can be overridden by users, if authorized to do so. Session +properties can be used to control resource usage, enable or disable features, and change query +characteristics. Session property managers are pluggable.

+

Add an etc/session-property-config.properties file with the following contents to enable +the built-in manager, that reads a JSON config file:

+
session-property-config.configuration-manager=file
+session-property-manager.config-file=etc/session-property-config.json
+
+
+

Change the value of session-property-manager.config-file to point to a JSON config file, +which can be an absolute path, or a path relative to the Trino data directory.

+

This configuration file consists of a list of match rules, each of which specify a list of +conditions that the query must meet, and a list of session properties that should be applied +by default. All matching rules contribute to constructing a list of session properties. Rules +are applied in the order they are specified. Rules specified later in the file override values +for properties that have been previously encountered.

+
+

Match rules#

+
    +
  • user (optional): regex to match against user name.

  • +
  • source (optional): regex to match against source string.

  • +
  • +
    queryType (optional): string to match against the type of the query submitted:
      +
    • DATA_DEFINITION: Queries that alter/create/drop the metadata of schemas/tables/views, and that manage +prepared statements, privileges, sessions, and transactions.

    • +
    • DELETE: DELETE queries.

    • +
    • DESCRIBE: DESCRIBE, DESCRIBE INPUT, DESCRIBE OUTPUT, and SHOW queries.

    • +
    • EXPLAIN: EXPLAIN queries.

    • +
    • INSERT: INSERT and CREATE TABLE AS queries.

    • +
    • SELECT: SELECT queries.

    • +
    +
    +
    +
  • +
  • clientTags (optional): list of tags. To match, every tag in this list must be in the list of +client-provided tags associated with the query.

  • +
  • group (optional): regex to match against the fully qualified name of the resource group the query is +routed to.

  • +
  • sessionProperties: map with string keys and values. Each entry is a system or catalog property name and +corresponding value. Values must be specified as strings, no matter the actual data type.

  • +
+
+
+

Example#

+

Consider the following set of requirements:

+
    +
  • All queries running under the global resource group must have an execution time limit of 8 hours.

  • +
  • All interactive queries are routed to sub-groups under the global.interactive group, and have an execution time +limit of 1 hour (tighter than the constraint on global).

  • +
  • All ETL queries (tagged with ‘etl’) are routed to sub-groups under the global.pipeline group, and must be +configured with certain properties to control writer behavior and a hive catalog property.

  • +
+

These requirements can be expressed with the following rules:

+
[
+  {
+    "group": "global.*",
+    "sessionProperties": {
+      "query_max_execution_time": "8h"
+    }
+  },
+  {
+    "group": "global.interactive.*",
+    "sessionProperties": {
+      "query_max_execution_time": "1h"
+    }
+  },
+  {
+    "group": "global.pipeline.*",
+    "clientTags": ["etl"],
+    "sessionProperties": {
+      "scale_writers": "true",
+      "writer_min_size": "1GB",
+      "hive.insert_existing_partitions_behavior": "overwrite"
+    }
+  }
+]
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/spill.html b/430/admin/spill.html new file mode 100644 index 000000000..11ff465f2 --- /dev/null +++ b/430/admin/spill.html @@ -0,0 +1,660 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Spill to disk — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Spill to disk#

+
+

Overview#

+

In the case of memory intensive operations, Trino allows offloading +intermediate operation results to disk. The goal of this mechanism is to +enable execution of queries that require amounts of memory exceeding per query +or per node limits.

+

The mechanism is similar to OS level page swapping. However, it is +implemented on the application level to address specific needs of Trino.

+

Properties related to spilling are described in Spilling properties.

+
+
+

Memory management and spill#

+

By default, Trino kills queries, if the memory requested by the query execution +exceeds session properties query_max_memory or +query_max_memory_per_node. This mechanism ensures fairness in allocation +of memory to queries, and prevents deadlock caused by memory allocation. +It is efficient when there is a lot of small queries in the cluster, but +leads to killing large queries that don’t stay within the limits.

+

To overcome this inefficiency, the concept of revocable memory was introduced. A +query can request memory that does not count toward the limits, but this memory +can be revoked by the memory manager at any time. When memory is revoked, the +query runner spills intermediate data from memory to disk and continues to +process it later.

+

In practice, when the cluster is idle, and all memory is available, a memory +intensive query may use all of the memory in the cluster. On the other hand, +when the cluster does not have much free memory, the same query may be forced to +use disk as storage for intermediate data. A query, that is forced to spill to +disk, may have a longer execution time by orders of magnitude than a query that +runs completely in memory.

+

Please note that enabling spill-to-disk does not guarantee execution of all +memory intensive queries. It is still possible that the query runner fails +to divide intermediate data into chunks small enough so that every chunk fits into +memory, leading to Out of memory errors while loading the data from disk.

+
+
+

Spill disk space#

+

Spilling intermediate results to disk, and retrieving them back, is expensive +in terms of IO operations. Thus, queries that use spill likely become +throttled by disk. To increase query performance, it is recommended to +provide multiple paths on separate local devices for spill (property +spiller-spill-path in Spilling properties).

+

The system drive should not be used for spilling, especially not to the drive where the JVM +is running and writing logs. Doing so may lead to cluster instability. Additionally, +it is recommended to monitor the disk saturation of the configured spill paths.

+

Trino treats spill paths as independent disks (see JBOD), so +there is no need to use RAID for spill.

+
+
+

Spill compression#

+

When spill compression is enabled (spill-compression-enabled property in +Spilling properties), spilled pages are compressed, before being +written to disk. Enabling this feature can reduce disk IO at the cost +of extra CPU load to compress and decompress spilled pages.

+
+
+

Spill encryption#

+

When spill encryption is enabled (spill-encryption-enabled property in +Spilling properties), spill contents are encrypted with a randomly generated +(per spill file) secret key. Enabling this increases CPU load and reduces throughput +of spilling to disk, but can protect spilled data from being recovered from spill files. +Consider reducing the value of memory-revoking-threshold when spill +encryption is enabled, to account for the increase in latency of spilling.

+
+
+

Supported operations#

+

Not all operations support spilling to disk, and each handles spilling +differently. Currently, the mechanism is implemented for the following +operations.

+
+

Joins#

+

During the join operation, one of the tables being joined is stored in memory. +This table is called the build table. The rows from the other table stream +through and are passed onto the next operation, if they match rows in the build +table. The most memory-intensive part of the join is this build table.

+

When the task concurrency is greater than one, the build table is partitioned. +The number of partitions is equal to the value of the task.concurrency +configuration parameter (see Task properties).

+

When the build table is partitioned, the spill-to-disk mechanism can decrease +the peak memory usage needed by the join operation. When a query approaches the +memory limit, a subset of the partitions of the build table gets spilled to disk, +along with rows from the other table that fall into those same partitions. The +number of partitions, that get spilled, influences the amount of disk space needed.

+

Afterward, the spilled partitions are read back one-by-one to finish the join +operation.

+

With this mechanism, the peak memory used by the join operator can be decreased +to the size of the largest build table partition. Assuming no data skew, this +is 1 / task.concurrency times the size of the whole build table.

+
+
+

Aggregations#

+

Aggregation functions perform an operation on a group of values and return one +value. If the number of groups you’re aggregating over is large, a significant +amount of memory may be needed. When spill-to-disk is enabled, if there is not +enough memory, intermediate cumulated aggregation results are written to disk. +They are loaded back and merged with a lower memory footprint.

+
+
+

Order by#

+

If your trying to sort a larger amount of data, a significant amount of memory +may be needed. When spill to disk for order by is enabled, if there is not enough +memory, intermediate sorted results are written to disk. They are loaded back and +merged with a lower memory footprint.

+
+
+

Window functions#

+

Window functions perform an operator over a window of rows, and return one value +for each row. If this window of rows is large, a significant amount of memory may +be needed. When spill to disk for window functions is enabled, if there is not enough +memory, intermediate results are written to disk. They are loaded back and merged +when memory is available. There is a current limitation that spill does not work +in all cases, such as when a single window is very large.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/tuning.html b/430/admin/tuning.html new file mode 100644 index 000000000..719a362c5 --- /dev/null +++ b/430/admin/tuning.html @@ -0,0 +1,524 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Tuning Trino — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Tuning Trino#

+

The default Trino settings should work well for most workloads. The following +information may help you, if your cluster is facing a specific performance problem.

+
+

Config properties#

+

See Properties reference.

+
+
+

JVM settings#

+

The following can be helpful for diagnosing garbage collection (GC) issues:

+
-Xlog:gc*,safepoint::time,level,tags,tid
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/admin/web-interface.html b/430/admin/web-interface.html new file mode 100644 index 000000000..c16b0a076 --- /dev/null +++ b/430/admin/web-interface.html @@ -0,0 +1,613 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Web UI — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Web UI#

+

Trino provides a web-based user interface (UI) for monitoring a Trino cluster +and managing queries. The Web UI is accessible on the coordinator via +HTTP or HTTPS, using the corresponding port number specified in the coordinator +Config properties. It can be configured with Web UI properties.

+

The Web UI can be disabled entirely with the web-ui.enabled property.

+
+

Authentication#

+

The Web UI requires users to authenticate. If Trino is not configured to require +authentication, then any username can be used, and no password is required or +allowed. Typically, users login with the same username that they use for +running queries.

+

If no system access control is installed, then all users are able to view and kill +any query. This can be restricted by using query rules with the +System access control. Users always have permission to view +or kill their own queries.

+
+

Password authentication#

+

Typically, a password-based authentication method +such as LDAP or password file +is used to secure both the Trino server and the Web UI. When the Trino server +is configured to use a password authenticator, the Web UI authentication type +is automatically set to FORM. In this case, the Web UI displays a login form +that accepts a username and password.

+
+
+

Fixed user authentication#

+

If you require the Web UI to be accessible without authentication, you can set a fixed +username that will be used for all Web UI access by setting the authentication type to +FIXED and setting the username with the web-ui.user configuration property. +If there is a system access control installed, this user must have permission to view +(and possibly to kill) queries.

+
+
+

Other authentication types#

+

The following Web UI authentication types are also supported:

+ +

For these authentication types, the username is defined by User mapping.

+
+
+
+

User interface overview#

+

The main page has a list of queries along with information like unique query ID, query text, +query state, percentage completed, username and source from which this query originated. +The currently running queries are at the top of the page, followed by the most recently +completed or failed queries.

+

The possible query states are as follows:

+
    +
  • QUEUED – Query has been accepted and is awaiting execution.

  • +
  • PLANNING – Query is being planned.

  • +
  • STARTING – Query execution is being started.

  • +
  • RUNNING – Query has at least one running task.

  • +
  • BLOCKED – Query is blocked and is waiting for resources (buffer space, memory, splits, etc.).

  • +
  • FINISHING – Query is finishing (e.g. commit for autocommit queries).

  • +
  • FINISHED – Query has finished executing and all output has been consumed.

  • +
  • FAILED – Query execution failed.

  • +
+

The BLOCKED state is normal, but if it is persistent, it should be investigated. +It has many potential causes: insufficient memory or splits, disk or network I/O bottlenecks, data skew +(all the data goes to a few workers), a lack of parallelism (only a few workers available), or computationally +expensive stages of the query following a given stage. Additionally, a query can be in +the BLOCKED state if a client is not processing the data fast enough (common with “SELECT *” queries).

+

For more detailed information about a query, simply click the query ID link. +The query detail page has a summary section, graphical representation of various stages of the +query and a list of tasks. Each task ID can be clicked to get more information about that task.

+

The summary section has a button to kill the currently running query. There are two visualizations +available in the summary section: task execution and timeline. The full JSON document containing +information and statistics about the query is available by clicking the JSON link. These visualizations +and other statistics can be used to analyze where time is being spent for a query.

+
+
+

Configuring query history#

+

The following configuration properties affect how query history is collected for display in the Web UI:

+
    +
  • query.min-expire-age

  • +
  • query.max-history

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/appendix.html b/430/appendix.html new file mode 100644 index 000000000..033fe4c48 --- /dev/null +++ b/430/appendix.html @@ -0,0 +1,430 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Appendix — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+ +
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/appendix/from-hive.html b/430/appendix/from-hive.html new file mode 100644 index 000000000..117dd6035 --- /dev/null +++ b/430/appendix/from-hive.html @@ -0,0 +1,636 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Migrating from Hive — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ + + + +
+
+ +
+

Migrating from Hive#

+

Trino uses ANSI SQL syntax and semantics, whereas Hive uses a language similar +to SQL called HiveQL which is loosely modeled after MySQL (which itself has many +differences from ANSI SQL).

+
+

Use subscript for accessing a dynamic index of an array instead of a udf#

+

The subscript operator in SQL supports full expressions, unlike Hive (which only supports constants). Therefore you can write queries like:

+
SELECT my_array[CARDINALITY(my_array)] as last_element
+FROM ...
+
+
+
+
+

Avoid out of bounds access of arrays#

+

Accessing out of bounds elements of an array will result in an exception. You can avoid this with an if as follows:

+
SELECT IF(CARDINALITY(my_array) >= 3, my_array[3], NULL)
+FROM ...
+
+
+
+
+

Use ANSI SQL syntax for arrays#

+

Arrays are indexed starting from 1, not from 0:

+
SELECT my_array[1] AS first_element
+FROM ...
+
+
+

Construct arrays with ANSI syntax:

+
SELECT ARRAY[1, 2, 3] AS my_array
+
+
+
+
+

Use ANSI SQL syntax for identifiers and strings#

+

Strings are delimited with single quotes and identifiers are quoted with double quotes, not backquotes:

+
SELECT name AS "User Name"
+FROM "7day_active"
+WHERE name = 'foo'
+
+
+
+
+

Quote identifiers that start with numbers#

+

Identifiers that start with numbers are not legal in ANSI SQL and must be quoted using double quotes:

+
SELECT *
+FROM "7day_active"
+
+
+
+
+

Use the standard string concatenation operator#

+

Use the ANSI SQL string concatenation operator:

+
SELECT a || b || c
+FROM ...
+
+
+
+
+

Use standard types for CAST targets#

+

The following standard types are supported for CAST targets:

+
SELECT
+  CAST(x AS varchar)
+, CAST(x AS bigint)
+, CAST(x AS double)
+, CAST(x AS boolean)
+FROM ...
+
+
+

In particular, use VARCHAR instead of STRING.

+
+
+

Use CAST when dividing integers#

+

Trino follows the standard behavior of performing integer division when dividing two integers. For example, dividing 7 by 2 will result in 3, not 3.5. +To perform floating point division on two integers, cast one of them to a double:

+
SELECT CAST(5 AS DOUBLE) / 2
+
+
+
+
+

Use WITH for complex expressions or queries#

+

When you want to re-use a complex output expression as a filter, use either an inline subquery or factor it out using the WITH clause:

+
WITH a AS (
+  SELECT substr(name, 1, 3) x
+  FROM ...
+)
+SELECT *
+FROM a
+WHERE x = 'foo'
+
+
+
+
+

Use UNNEST to expand arrays and maps#

+

Trino supports UNNEST for expanding arrays and maps. +Use UNNEST instead of LATERAL VIEW explode().

+

Hive query:

+
SELECT student, score
+FROM tests
+LATERAL VIEW explode(scores) t AS score;
+
+
+

Trino query:

+
SELECT student, score
+FROM tests
+CROSS JOIN UNNEST(scores) AS t (score);
+
+
+
+
+

Use ANSI SQL syntax for date and time INTERVAL expressions#

+

Trino supports the ANSI SQL style INTERVAL expressions that differs from the implementation used in Hive.

+
    +
  • The INTERVAL keyword is required and is not optional.

  • +
  • Date and time units must be singular. For example day and not days.

  • +
  • Values must be quoted.

  • +
+

Hive query:

+
SELECT cast('2000-08-19' as date) + 14 days;
+
+
+

Equivalent Trino query:

+
SELECT cast('2000-08-19' as date) + INTERVAL '14' day;
+
+
+
+
+

Caution with datediff#

+

The Hive datediff function returns the difference between the two dates in +days and is declared as:

+
datediff(string enddate, string startdate)  -> integer
+
+
+

The equivalent Trino function date_diff +uses a reverse order for the two date parameters and requires a unit. This has +to be taken into account when migrating:

+

Hive query:

+
datediff(enddate, startdate)
+
+
+

Trino query:

+
date_diff('day', startdate, enddate)
+
+
+
+
+

Overwriting data on insert#

+

By default, INSERT queries are not allowed to overwrite existing data. You +can use the catalog session property insert_existing_partitions_behavior to +allow overwrites. Prepend the name of the catalog using the Hive connector, for +example hdfs, and set the property in the session before you run the insert +query:

+
SET SESSION hdfs.insert_existing_partitions_behavior = 'OVERWRITE';
+INSERT INTO hdfs.schema.table ...
+
+
+

The resulting behavior is equivalent to using INSERT OVERWRITE in Hive.

+

Insert overwrite operation is not supported by Trino when the table is stored on +encrypted HDFS, when the table is unpartitioned or table is transactional.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/appendix/legal-notices.html b/430/appendix/legal-notices.html new file mode 100644 index 000000000..b86371981 --- /dev/null +++ b/430/appendix/legal-notices.html @@ -0,0 +1,498 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Legal notices — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ + + + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/client.html b/430/client.html new file mode 100644 index 000000000..c90c549f3 --- /dev/null +++ b/430/client.html @@ -0,0 +1,437 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Clients — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Clients#

+

A client is used to send queries to Trino and receive results, or otherwise +interact with Trino and the connected data sources.

+

Some clients, such as the command line interface, can +provide a user interface directly. Clients like the JDBC driver, provide a mechanism for other tools to connect to Trino.

+

The following clients are available:

+ +

In addition, the community provides numerous other clients for platforms such as Python, and these +can in turn be used to connect applications using these platforms.

+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/client/cli.html b/430/client/cli.html new file mode 100644 index 000000000..5f3cee714 --- /dev/null +++ b/430/client/cli.html @@ -0,0 +1,1191 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command line interface — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Command line interface#

+

The Trino CLI provides a terminal-based, interactive shell for running +queries. The CLI is a +self-executing +JAR file, which means it acts like a normal UNIX executable.

+
+

Requirements#

+

The CLI requires a Java virtual machine available on the path. +It can be used with Java version 8 and higher.

+

The CLI uses the Trino client REST API over +HTTP/HTTPS to communicate with the coordinator on the cluster.

+

The CLI version should be identical to the version of the Trino cluster, or +newer. Older versions typically work, but only a subset is regularly tested. +Versions before 350 are not supported.

+
+
+

Installation#

+

Download trino-cli-430-executable.jar, rename it to trino, make it executable with +chmod +x, and run it to show the version of the CLI:

+
./trino --version
+
+
+

Run the CLI with --help or -h to see all available options.

+

Windows users, and users unable to execute the preceeding steps, can use the +equivalent java command with the -jar option to run the CLI, and show +the version:

+
java -jar trino-cli-*-executable.jar --version
+
+
+

The syntax can be used for the examples in the following sections. In addition, +using the java command allows you to add configuration options for the Java +runtime with the -D syntax. You can use this for debugging and +troubleshooting, such as when specifying additional Kerberos debug options.

+
+
+

Running the CLI#

+

The minimal command to start the CLI in interactive mode specifies the URL of +the coordinator in the Trino cluster:

+
./trino http://trino.example.com:8080
+
+
+

If successful, you will get a prompt to execute commands. Use the help +command to see a list of supported commands. Use the clear command to clear +the terminal. To stop and exit the CLI, run exit or quit.:

+
trino> help
+
+Supported commands:
+QUIT
+EXIT
+CLEAR
+EXPLAIN [ ( option [, ...] ) ] <query>
+    options: FORMAT { TEXT | GRAPHVIZ | JSON }
+            TYPE { LOGICAL | DISTRIBUTED | VALIDATE | IO }
+DESCRIBE <table>
+SHOW COLUMNS FROM <table>
+SHOW FUNCTIONS
+SHOW CATALOGS [LIKE <pattern>]
+SHOW SCHEMAS [FROM <catalog>] [LIKE <pattern>]
+SHOW TABLES [FROM <schema>] [LIKE <pattern>]
+USE [<catalog>.]<schema>
+
+
+

You can now run SQL statements. After processing, the CLI will show results and +statistics.

+
trino> SELECT count(*) FROM tpch.tiny.nation;
+
+_col0
+-------
+    25
+(1 row)
+
+Query 20220324_213359_00007_w6hbk, FINISHED, 1 node
+Splits: 13 total, 13 done (100.00%)
+2.92 [25 rows, 0B] [8 rows/s, 0B/s]
+
+
+

As part of starting the CLI, you can set the default catalog and schema. This +allows you to query tables directly without specifying catalog and schema.

+
./trino http://trino.example.com:8080/tpch/tiny
+
+trino:tiny> SHOW TABLES;
+
+  Table
+----------
+customer
+lineitem
+nation
+orders
+part
+partsupp
+region
+supplier
+(8 rows)
+
+
+

You can also set the default catalog and schema with the USE +statement.

+
trino> USE tpch.tiny;
+USE
+trino:tiny>
+
+
+

Many other options are available to further configure the CLI in interactive +mode:

+ ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Option

Description

--catalog

Sets the default catalog. You can change the default catalog and schema with +USE.

--client-info

Adds arbitrary text as extra information about the client.

--client-request-timeout

Sets the duration for query processing, after which, the client request is +terminated. Defaults to 2m.

--client-tags

Adds extra tags information about the client and the CLI user. Separate +multiple tags with commas. The tags can be used as input for +Resource groups.

--debug

Enables display of debug information during CLI usage for +Troubleshooting. Displays more information about query +processing statistics.

--disable-auto-suggestion

Disables autocomplete suggestions.

--disable-compression

Disables compression of query results.

--editing-mode

Sets key bindings in the CLI to be compatible with VI or +EMACS editors. Defaults to EMACS.

--http-proxy

Configures the URL of the HTTP proxy to connect to Trino.

--history-file

Path to the history file. Defaults to ~/.trino_history.

--network-logging

Configures the level of detail provided for network logging of the CLI. +Defaults to NONE, other options are BASIC, HEADERS, or BODY.

--output-format-interactive=<format>

Specify the format to use for printing query results. +Defaults to ALIGNED.

--pager=<pager>

Path to the pager program used to display the query results. Set to an empty +value to completely disable pagination. Defaults to less with a carefully +selected set of options.

--no-progress

Do not show query processing progress.

--password

Prompts for a password. Use if your Trino server requires password +authentication. You can set the TRINO_PASSWORD environment variable with +the password value to avoid the prompt. For more information, see +Username and password authentication.

--schema

Sets the default schema. You can change the default catalog and schema +with USE.

--server

The HTTP/HTTPS address and port of the Trino coordinator. The port must be +set to the port the Trino coordinator is listening for connections on. Trino +server location defaults to http://localhost:8080. Can only be set if URL +is not specified.

--session

Sets one or more session properties. +Property can be used multiple times with the format +session_property_name=value.

--socks-proxy

Configures the URL of the SOCKS proxy to connect to Trino.

--source

Specifies the name of the application or source connecting to Trino. +Defaults to trino-cli. The value can be used as input for +Resource groups.

--timezone

Sets the time zone for the session using the time zone name. Defaults to +the timezone set on your workstation.

--user

Sets the username for Username and password authentication. Defaults to your +operating system username. You can override the default username, if your +cluster uses a different username or authentication mechanism.

+

Most of the options can also be set as parameters in the URL. This means +a JDBC URL can be used in the CLI after removing the jdbc: prefix. +However, the same parameter may not be specified using both methods. +See the JDBC driver parameter reference +to find out URL parameter names. For example:

+
./trino 'https://trino.example.com?SSL=true&SSLVerification=FULL&clientInfo=extra'
+
+
+
+
+

TLS/HTTPS#

+

Trino is typically available with an HTTPS URL. This means that all network +traffic between the CLI and Trino uses TLS. TLS configuration is common, since it is a requirement for any authentication.

+

Use the HTTPS URL to connect to the server:

+
./trino https://trino.example.com
+
+
+

The recommended TLS implementation is to use a globally trusted certificate. In +this case, no other options are necessary, since the JVM running the CLI +recognizes these certificates.

+

Use the options from the following table to further configure TLS and +certificate usage:

+ ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Option

Description

--insecure

Skip certificate validation when connecting with TLS/HTTPS (should only be +used for debugging).

--keystore-path

The location of the Java Keystore file that contains the certificate of the +server to connect with TLS.

--keystore-password

The password for the keystore. This must match the password you specified +when creating the keystore.

--keystore-type

Determined by the keystore file format. The default keystore type is JKS. +This advanced option is only necessary if you use a custom Java Cryptography +Architecture (JCA) provider implementation.

--truststore-password

The password for the truststore. This must match the password you specified +when creating the truststore.

--truststore-path

The location of the Java truststore file that will be used to secure TLS.

--truststore-type

Determined by the truststore file format. The default keystore type is JKS. +This advanced option is only necessary if you use a custom Java Cryptography +Architecture (JCA) provider implementation.

--use-system-truststore

Verify the server certificate using the system truststore of the operating +system. Windows and macOS are supported. For other operating systems, the +default Java truststore is used. The truststore type can be overridden using +--truststore-type.

+
+
+

Authentication#

+

The Trino CLI supports many Authentication types detailed in +the following sections:

+
+

Username and password authentication#

+

Username and password authentication is typically configured in a cluster using +the PASSWORD authentication type, +for example with LDAP authentication or Password file authentication.

+

The following code example connects to the server, establishes your user name, +and prompts the CLI for your password:

+
./trino https://trino.example.com --user=exampleusername --password
+
+
+

Alternatively, set the password as the value of the TRINO_PASSWORD +environment variable. Typically use single quotes to avoid problems with +special characters such as $:

+
export TRINO_PASSWORD='LongSecurePassword123!@#'
+
+
+

If the TRINO_PASSWORD environment variable is set, you are not prompted +to provide a password to connect with the CLI.

+
./trino https://trino.example.com --user=exampleusername --password
+
+
+
+
+

External authentication - SSO#

+

Use the --external-authentication option for browser-based SSO +authentication, as detailed in OAuth 2.0 authentication. With this configuration, +the CLI displays a URL that you must open in a web browser for authentication.

+

The detailed behavior is as follows:

+
    +
  • Start the CLI with the --external-authentication option and execute a +query.

  • +
  • The CLI starts and connects to Trino.

  • +
  • A message appears in the CLI directing you to open a browser with a specified +URL when the first query is submitted.

  • +
  • Open the URL in a browser and follow through the authentication process.

  • +
  • The CLI automatically receives a token.

  • +
  • When successfully authenticated in the browser, the CLI proceeds to execute +the query.

  • +
  • Further queries in the CLI session do not require additional logins while the +authentication token remains valid. Token expiration depends on the external +authentication type configuration.

  • +
  • Expired tokens force you to log in again.

  • +
+
+
+

Certificate authentication#

+

Use the following CLI arguments to connect to a cluster that uses +certificate authentication.

+ + ++++ + + + + + + + + + + + + + +
CLI options for certificate authentication#

Option

Description

--keystore-path=<path>

Absolute or relative path to a PEM or +JKS file, which must contain a certificate +that is trusted by the Trino cluster you are connecting to.

--keystore-password=<password>

Only required if the keystore has a password.

+

The truststore related options are independent of client certificate +authentication with the CLI; instead, they control the client’s trust of the +server’s certificate.

+
+
+

JWT authentication#

+

To access a Trino cluster configured to use JWT authentication, use the +--access-token=<token> option to pass a JWT to the server.

+
+
+

Kerberos authentication#

+

The Trino CLI can connect to a Trino cluster that has Kerberos authentication +enabled.

+

Invoking the CLI with Kerberos support enabled requires a number of additional +command line options. You also need the Kerberos configuration files for your user on the machine running the CLI. The +simplest way to invoke the CLI is with a wrapper script:

+
#!/bin/bash
+
+./trino \
+  --server https://trino.example.com \
+  --krb5-config-path /etc/krb5.conf \
+  --krb5-principal someuser@EXAMPLE.COM \
+  --krb5-keytab-path /home/someuser/someuser.keytab \
+  --krb5-remote-service-name trino
+
+
+

When using Kerberos authentication, access to the Trino coordinator must be +through TLS and HTTPS.

+

The following table lists the available options for Kerberos authentication:

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
CLI options for Kerberos authentication#

Option

Description

--krb5-config-path

Path to Kerberos configuration files.

--krb5-credential-cache-path

Kerberos credential cache path.

--krb5-disable-remote-service-hostname-canonicalization

Disable service hostname canonicalization using the DNS reverse lookup.

--krb5-keytab-path

The location of the keytab that can be used to authenticate the principal +specified by --krb5-principal.

--krb5-principal

The principal to use when authenticating to the coordinator.

--krb5-remote-service-name

Trino coordinator Kerberos service name.

--krb5-service-principal-pattern

Remote kerberos service principal pattern. Defaults to ${SERVICE}@${HOST}.

+
+

Additional Kerberos debugging information#

+

You can enable additional Kerberos debugging information for the Trino CLI +process by passing -Dsun.security.krb5.debug=true, +-Dtrino.client.debugKerberos=true, and +-Djava.security.debug=gssloginconfig,configfile,configparser,logincontext +as a JVM argument when starting the CLI process:

+
java \
+  -Dsun.security.krb5.debug=true \
+  -Djava.security.debug=gssloginconfig,configfile,configparser,logincontext \
+  -Dtrino.client.debugKerberos=true \
+  -jar trino-cli-*-executable.jar \
+  --server https://trino.example.com \
+  --krb5-config-path /etc/krb5.conf \
+  --krb5-principal someuser@EXAMPLE.COM \
+  --krb5-keytab-path /home/someuser/someuser.keytab \
+  --krb5-remote-service-name trino
+
+
+

For help with interpreting Kerberos debugging messages, see additional resources.

+
+
+
+
+

Pagination#

+

By default, the results of queries are paginated using the less program +which is configured with a carefully selected set of options. This behavior +can be overridden by setting the --pager option or +the TRINO_PAGER environment variable to the name of a different program +such as more or pspg, +or it can be set to an empty value to completely disable pagination.

+
+
+

History#

+

The CLI keeps a history of your previously used commands. You can access your +history by scrolling or searching. Use the up and down arrows to scroll and +Control+S and Control+R to search. To execute a query again, +press Enter.

+

By default, you can locate the Trino history file in ~/.trino_history. +Use the --history-file option or the `TRINO_HISTORY_FILE environment variable +to change the default.

+
+

Auto suggestion#

+

The CLI generates autocomplete suggestions based on command history.

+

Press to accept the suggestion and replace the current command line +buffer. Press Ctrl+ (Option+ on Mac) to accept only the next +keyword. Continue typing to reject the suggestion.

+
+
+
+

Configuration file#

+

The CLI can read default values for all options from a file. It uses the first +file found from the ordered list of locations:

+
    +
  • File path set as value of the TRINO_CONFIG environment variable.

  • +
  • .trino_config in the current users home directory.

  • +
  • $XDG_CONFIG_HOME/trino/config.

  • +
+

For example, you could create separate configuration files with different +authentication options, like kerberos-cli.properties and ldap-cli.properties. +Assuming they’re located in the current directory, you can set the +TRINO_CONFIG environment variable for a single invocation of the CLI by +adding it before the trino command:

+
TRINO_CONFIG=kerberos-cli.properties trino https://first-cluster.example.com:8443
+TRINO_CONFIG=ldap-cli.properties trino https://second-cluster.example.com:8443
+
+
+

In the preceding example, the default configuration files are not used.

+

You can use all supported options without the -- prefix in the configuration +properties file. Options that normally don’t take an argument are boolean, so +set them to either true or false. For example:

+
output-format-interactive=AUTO
+timezone=Europe/Warsaw
+user=trino-client
+network-logging=BASIC
+krb5-disable-remote-service-hostname-canonicalization=true
+
+
+
+
+

Batch mode#

+

Running the Trino CLI with the --execute, --file, or passing queries to +the standard input uses the batch (non-interactive) mode. In this mode +the CLI does not report progress, and exits after processing the supplied +queries. Results are printed in CSV format by default. You can configure +other formats and redirect the output to a file.

+

The following options are available to further configure the CLI in batch +mode:

+ ++++ + + + + + + + + + + + + + + + + + + + + + + +

Option

Description

--execute=<execute>

Execute specified statements and exit.

-f, --file=<file>

Execute statements from file and exit.

--ignore-errors

Continue processing in batch mode when an error occurs. Default is to exit +immediately.

--output-format=<format>

Specify the format to use for printing query results. +Defaults to CSV.

--progress

Show query progress in batch mode. It does not affect the output, which, for +example can be safely redirected to a file.

+
+

Examples#

+

Consider the following command run as shown, or with the +--output-format=CSV option, which is the default for non-interactive usage:

+
trino --execute 'SELECT nationkey, name, regionkey FROM tpch.sf1.nation LIMIT 3'
+
+
+

The output is as follows:

+
"0","ALGERIA","0"
+"1","ARGENTINA","1"
+"2","BRAZIL","1"
+
+
+

The output with the --output-format=JSON option:

+
{"nationkey":0,"name":"ALGERIA","regionkey":0}
+{"nationkey":1,"name":"ARGENTINA","regionkey":1}
+{"nationkey":2,"name":"BRAZIL","regionkey":1}
+
+
+

The output with the --output-format=ALIGNED option, which is the default +for interactive usage:

+
nationkey |   name    | regionkey
+----------+-----------+----------
+        0 | ALGERIA   |         0
+        1 | ARGENTINA |         1
+        2 | BRAZIL    |         1
+
+
+

The output with the --output-format=VERTICAL option:

+
-[ RECORD 1 ]--------
+nationkey | 0
+name      | ALGERIA
+regionkey | 0
+-[ RECORD 2 ]--------
+nationkey | 1
+name      | ARGENTINA
+regionkey | 1
+-[ RECORD 3 ]--------
+nationkey | 2
+name      | BRAZIL
+regionkey | 1
+
+
+

The preceding command with --output-format=NULL produces no output. +However, if you have an error in the query, such as incorrectly using +region instead of regionkey, the command has an exit status of 1 +and displays an error message (which is unaffected by the output format):

+
Query 20200707_170726_00030_2iup9 failed: line 1:25: Column 'region' cannot be resolved
+SELECT nationkey, name, region FROM tpch.sf1.nation LIMIT 3
+
+
+
+
+
+

Output formats#

+

The Trino CLI provides the options --output-format +and --output-format-interactive to control how the output is displayed. +The available options shown in the following table must be entered +in uppercase. The default value is ALIGNED in interactive mode, +and CSV in non-interactive mode.

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Output format options#

Option

Description

CSV

Comma-separated values, each value quoted. No header row.

CSV_HEADER

Comma-separated values, quoted with header row.

CSV_UNQUOTED

Comma-separated values without quotes.

CSV_HEADER_UNQUOTED

Comma-separated values with header row but no quotes.

TSV

Tab-separated values.

TSV_HEADER

Tab-separated values with header row.

JSON

Output rows emitted as JSON objects with name-value pairs.

ALIGNED

Output emitted as an ASCII character table with values.

VERTICAL

Output emitted as record-oriented top-down lines, one per value.

AUTO

Same as ALIGNED if output would fit the current terminal width, +and VERTICAL otherwise.

MARKDOWN

Output emitted as a Markdown table.

NULL

Suppresses normal query results. This can be useful during development to +test a query’s shell return code or to see whether it results in error +messages.

+
+
+

Troubleshooting#

+

If something goes wrong, you see an error message:

+
$ trino
+trino> select count(*) from tpch.tiny.nations;
+Query 20200804_201646_00003_f5f6c failed: line 1:22: Table 'tpch.tiny.nations' does not exist
+select count(*) from tpch.tiny.nations
+
+
+

To view debug information, including the stack trace for failures, use the +--debug option:

+
$ trino --debug
+trino> select count(*) from tpch.tiny.nations;
+Query 20200804_201629_00002_f5f6c failed: line 1:22: Table 'tpch.tiny.nations' does not exist
+io.trino.spi.TrinoException: line 1:22: Table 'tpch.tiny.nations' does not exist
+at io.trino.sql.analyzer.SemanticExceptions.semanticException(SemanticExceptions.java:48)
+at io.trino.sql.analyzer.SemanticExceptions.semanticException(SemanticExceptions.java:43)
+...
+at java.base/java.lang.Thread.run(Thread.java:834)
+select count(*) from tpch.tiny.nations
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/client/jdbc.html b/430/client/jdbc.html new file mode 100644 index 000000000..7d820c9e5 --- /dev/null +++ b/430/client/jdbc.html @@ -0,0 +1,719 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + JDBC driver — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

JDBC driver#

+

The Trino JDBC driver allows +users to access Trino using Java-based applications, and other non-Java +applications running in a JVM. Both desktop and server-side applications, such +as those used for reporting and database development, use the JDBC driver.

+
+

Requirements#

+

The Trino JDBC driver has the following requirements:

+
    +
  • Java version 8 or higher.

  • +
  • All users that connect to Trino with the JDBC driver must be granted access to +query tables in the system.jdbc schema.

  • +
+

The JDBC driver version should be identical to the version of the Trino cluster, +or newer. Older versions typically work, but only a subset is regularly tested. +Versions before 350 are not supported.

+
+
+

Installing#

+

Download trino-jdbc-430.jar and add it to the classpath of your Java application.

+

The driver is also available from Maven Central:

+
<dependency>
+    <groupId>io.trino</groupId>
+    <artifactId>trino-jdbc</artifactId>
+    <version>430</version>
+</dependency>
+

We recommend using the latest version of the JDBC driver. A list of all +available versions can be found in the Maven Central Repository. Navigate to the +directory for the desired version, and select the trino-jdbc-xxx.jar file +to download, where xxx is the version number.

+

Once downloaded, you must add the JAR file to a directory in the classpath +of users on systems where they will access Trino.

+

After you have downloaded the JDBC driver and added it to your +classpath, you’ll typically need to restart your application in order to +recognize the new driver. Then, depending on your application, you +may need to manually register and configure the driver.

+

The CLI uses the HTTP protocol and the +Trino client REST API to communicate +with Trino.

+
+
+

Registering and configuring the driver#

+

Drivers are commonly loaded automatically by applications once they are added to +its classpath. If your application does not, such as is the case for some +GUI-based SQL editors, read this section. The steps to register the JDBC driver +in a UI or on the command line depend upon the specific application you are +using. Please check your application’s documentation.

+

Once registered, you must also configure the connection information as described +in the following section.

+
+
+

Connecting#

+

When your driver is loaded, registered and configured, you are ready to connect +to Trino from your application. The following JDBC URL formats are supported:

+
jdbc:trino://host:port
+jdbc:trino://host:port/catalog
+jdbc:trino://host:port/catalog/schema
+
+
+

The following is an example of a JDBC URL used to create a connection:

+
jdbc:trino://example.net:8080/hive/sales
+
+
+

This example JDBC URL locates a Trino instance running on port 8080 on +example.net, with the catalog hive and the schema sales defined.

+
+

Note

+

Typically, the JDBC driver classname is configured automatically by your +client. If it is not, use io.trino.jdbc.TrinoDriver wherever a driver +classname is required.

+
+
+
+

Connection parameters#

+

The driver supports various parameters that may be set as URL parameters, +or as properties passed to DriverManager. Both of the following +examples are equivalent:

+
// properties
+String url = "jdbc:trino://example.net:8080/hive/sales";
+Properties properties = new Properties();
+properties.setProperty("user", "test");
+properties.setProperty("password", "secret");
+properties.setProperty("SSL", "true");
+Connection connection = DriverManager.getConnection(url, properties);
+
+// URL parameters
+String url = "jdbc:trino://example.net:8443/hive/sales?user=test&password=secret&SSL=true";
+Connection connection = DriverManager.getConnection(url);
+
+
+

These methods may be mixed; some parameters may be specified in the URL, +while others are specified using properties. However, the same parameter +may not be specified using both methods.

+
+
+

Parameter reference#

+ ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Name

Description

user

Username to use for authentication and authorization.

password

Password to use for LDAP authentication.

sessionUser

Session username override, used for impersonation.

socksProxy

SOCKS proxy host and port. Example: localhost:1080

httpProxy

HTTP proxy host and port. Example: localhost:8888

clientInfo

Extra information about the client.

clientTags

Client tags for selecting resource groups. Example: abc,xyz

traceToken

Trace token for correlating requests across systems.

source

Source name for the Trino query. This parameter should be used in preference +to ApplicationName. Thus, it takes precedence over ApplicationName +and/or applicationNamePrefix.

applicationNamePrefix

Prefix to append to any specified ApplicationName client info property, +which is used to set the source name for the Trino query if the source +parameter has not been set. If neither this property nor ApplicationName +or source are set, the source name for the query is trino-jdbc.

accessToken

JWT access token for token based authentication.

SSL

Set true to specify using TLS/HTTPS for connections.

SSLVerification

The method of TLS verification. There are three modes: FULL +(default), CA and NONE. For FULL, the normal TLS verification +is performed. For CA, only the CA is verified but hostname mismatch +is allowed. For NONE, there is no verification.

SSLKeyStorePath

Use only when connecting to a Trino cluster that has certificate +authentication enabled. Specifies the path to a +PEM or JKS file, which must +contain a certificate that is trusted by the Trino cluster you connect to.

SSLKeyStorePassword

The password for the KeyStore, if any.

SSLKeyStoreType

The type of the KeyStore. The default type is provided by the Java +keystore.type security property or jks if none exists.

SSLTrustStorePath

The location of the Java TrustStore file to use to validate HTTPS server +certificates.

SSLTrustStorePassword

The password for the TrustStore.

SSLTrustStoreType

The type of the TrustStore. The default type is provided by the Java +keystore.type security property or jks if none exists.

SSLUseSystemTrustStore

Set true to automatically use the system TrustStore based on the operating +system. The supported OSes are Windows and macOS. For Windows, the +Windows-ROOT TrustStore is selected. For macOS, the KeychainStore +TrustStore is selected. For other OSes, the default Java TrustStore is +loaded. The TrustStore specification can be overridden using +SSLTrustStoreType.

hostnameInCertificate

Expected hostname in the certificate presented by the Trino server. Only +applicable with full SSL verification enabled.

KerberosRemoteServiceName

Trino coordinator Kerberos service name. This parameter is required for +Kerberos authentication.

KerberosPrincipal

The principal to use when authenticating to the Trino coordinator.

KerberosUseCanonicalHostname

Use the canonical hostname of the Trino coordinator for the Kerberos service +principal by first resolving the hostname to an IP address and then doing a +reverse DNS lookup for that IP address. This is enabled by default.

KerberosServicePrincipalPattern

Trino coordinator Kerberos service principal pattern. The default is +${SERVICE}@${HOST}. ${SERVICE} is replaced with the value of +KerberosRemoteServiceName and ${HOST} is replaced with the hostname of +the coordinator (after canonicalization if enabled).

KerberosConfigPath

Kerberos configuration file.

KerberosKeytabPath

Kerberos keytab file.

KerberosCredentialCachePath

Kerberos credential cache.

KerberosDelegation

Set to true to use the token from an existing Kerberos context. This +allows client to use Kerberos authentication without passing the Keytab or +credential cache. Defaults to false.

extraCredentials

Extra credentials for connecting to external services, specified as a list +of key-value pairs. For example, foo:bar;abc:xyz creates the credential +named abc with value xyz and the credential named foo with value +bar.

roles

Authorization roles to use for catalogs, specified as a list of key-value +pairs for the catalog and role. For example, catalog1:roleA;catalog2:roleB +sets roleA for catalog1 and roleB for catalog2.

sessionProperties

Session properties to set for the system and for catalogs, specified as a +list of key-value pairs. For example, abc:xyz;example.foo:bar sets the +system property abc to the value xyz and the foo property for catalog +example to the value bar.

externalAuthentication

Set to true if you want to use external authentication via +OAuth 2.0 authentication. Use a local web browser to authenticate with an +identity provider (IdP) that has been configured for the Trino coordinator.

externalAuthenticationTokenCache

Allows the sharing of external authentication tokens between different +connections for the same authenticated user until the cache is invalidated, +such as when a client is restarted or when the classloader reloads the JDBC +driver. This is disabled by default, with a value of NONE. To enable, set +the value to MEMORY. If the JDBC driver is used in a shared mode by +different users, the first registered token is stored and authenticates all +users.

disableCompression

Whether compression should be enabled.

assumeLiteralUnderscoreInMetadataCallsForNonConformingClients

When enabled, the name patterns passed to DatabaseMetaData methods are +treated as underscores. You can use this as a workaround for applications +that do not escape schema or table names when passing them to +DatabaseMetaData methods as schema or table name patterns. :::

timezone

Sets the time zone for the session using the time zone +passed. +Defaults to the timezone of the JVM running the JDBC driver.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector.html b/430/connector.html new file mode 100644 index 000000000..8a83a03d3 --- /dev/null +++ b/430/connector.html @@ -0,0 +1,704 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Connectors — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+ +
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/accumulo.html b/430/connector/accumulo.html new file mode 100644 index 000000000..02527d6a4 --- /dev/null +++ b/430/connector/accumulo.html @@ -0,0 +1,1580 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Accumulo connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Accumulo connector#

+

The Accumulo connector supports reading and writing data from +Apache Accumulo. +Please read this page thoroughly to understand the capabilities and features of the connector.

+
+

Installing the iterator dependency#

+

The Accumulo connector uses custom Accumulo iterators in +order to push various information in SQL predicate clauses to Accumulo for +server-side filtering, known as predicate pushdown. In order +for the server-side iterators to work, you need to add the trino-accumulo-iterators +JAR file to Accumulo’s lib/ext directory on each TabletServer node.

+
# For each TabletServer node:
+scp $TRINO_HOME/plugins/accumulo/trino-accumulo-iterators-*.jar [tabletserver_address]:$ACCUMULO_HOME/lib/ext
+
+# TabletServer should pick up new JAR files in ext directory, but may require restart
+
+
+
+
+

Requirements#

+

To connect to Accumulo, you need:

+
    +
  • Accumulo versions 1.x starting with 1.7.4. Versions 2.x are not supported.

  • +
  • Network access from the Trino coordinator and workers to the Accumulo +Zookeeper server. Port 2181 is the default port.

  • +
+
+
+

Connector configuration#

+

Create etc/catalog/example.properties to mount the accumulo connector as +the example catalog, with the following connector properties as appropriate +for your setup:

+
connector.name=accumulo
+accumulo.instance=xxx
+accumulo.zookeepers=xxx
+accumulo.username=username
+accumulo.password=password
+
+
+

Replace the accumulo.xxx properties as required.

+
+
+

Configuration variables#

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Default value

Required

Description

accumulo.instance

(none)

Yes

Name of the Accumulo instance

accumulo.zookeepers

(none)

Yes

ZooKeeper connect string

accumulo.username

(none)

Yes

Accumulo user for Trino

accumulo.password

(none)

Yes

Accumulo password for user

accumulo.zookeeper.metadata.root

/trino-accumulo

No

Root znode for storing metadata. Only relevant if using default Metadata Manager

accumulo.cardinality.cache.size

100000

No

Sets the size of the index cardinality cache

accumulo.cardinality.cache.expire.duration

5m

No

Sets the expiration duration of the cardinality cache.

+
+
+

Usage#

+

Simply begin using SQL to create a new table in Accumulo to begin +working with data. By default, the first column of the table definition +is set to the Accumulo row ID. This should be the primary key of your +table, and keep in mind that any INSERT statements containing the same +row ID is effectively an UPDATE as far as Accumulo is concerned, as any +previous data in the cell is overwritten. The row ID can be +any valid Trino datatype. If the first column is not your primary key, you +can set the row ID column using the row_id table property within the WITH +clause of your table definition.

+

Simply issue a CREATE TABLE statement to create a new Trino/Accumulo table:

+
CREATE TABLE example_schema.scientists (
+  recordkey VARCHAR,
+  name VARCHAR,
+  age BIGINT,
+  birthday DATE
+);
+
+
+
DESCRIBE example_schema.scientists;
+
+
+
  Column   |  Type   | Extra |                      Comment
+-----------+---------+-------+---------------------------------------------------
+ recordkey | varchar |       | Accumulo row ID
+ name      | varchar |       | Accumulo column name:name. Indexed: false
+ age       | bigint  |       | Accumulo column age:age. Indexed: false
+ birthday  | date    |       | Accumulo column birthday:birthday. Indexed: false
+
+
+

This command creates a new Accumulo table with the recordkey column +as the Accumulo row ID. The name, age, and birthday columns are mapped to +auto-generated column family and qualifier values (which, in practice, +are both identical to the Trino column name).

+

When creating a table using SQL, you can optionally specify a +column_mapping table property. The value of this property is a +comma-delimited list of triples, Trino column : Accumulo column +family : accumulo column qualifier, with one triple for every +non-row ID column. This sets the mapping of the Trino column name to +the corresponding Accumulo column family and column qualifier.

+

If you don’t specify the column_mapping table property, then the +connector auto-generates column names (respecting any configured locality groups). +Auto-generation of column names is only available for internal tables, so if your +table is external you must specify the column_mapping property.

+

For a full list of table properties, see Table Properties.

+

For example:

+
CREATE TABLE example_schema.scientists (
+  recordkey VARCHAR,
+  name VARCHAR,
+  age BIGINT,
+  birthday DATE
+)
+WITH (
+  column_mapping = 'name:metadata:name,age:metadata:age,birthday:metadata:date'
+);
+
+
+
DESCRIBE example_schema.scientists;
+
+
+
  Column   |  Type   | Extra |                    Comment
+-----------+---------+-------+-----------------------------------------------
+ recordkey | varchar |       | Accumulo row ID
+ name      | varchar |       | Accumulo column metadata:name. Indexed: false
+ age       | bigint  |       | Accumulo column metadata:age. Indexed: false
+ birthday  | date    |       | Accumulo column metadata:date. Indexed: false
+
+
+

You can then issue INSERT statements to put data into Accumulo.

+
+

Note

+

While issuing INSERT statements is convenient, +this method of loading data into Accumulo is low-throughput. You want +to use the Accumulo APIs to write Mutations directly to the tables. +See the section on Loading Data for more details.

+
+
INSERT INTO example_schema.scientists VALUES
+('row1', 'Grace Hopper', 109, DATE '1906-12-09' ),
+('row2', 'Alan Turing', 103, DATE '1912-06-23' );
+
+
+
SELECT * FROM example_schema.scientists;
+
+
+
 recordkey |     name     | age |  birthday
+-----------+--------------+-----+------------
+ row1      | Grace Hopper | 109 | 1906-12-09
+ row2      | Alan Turing  | 103 | 1912-06-23
+(2 rows)
+
+
+

As you’d expect, rows inserted into Accumulo via the shell or +programmatically will also show up when queried. (The Accumulo shell +thinks “-5321” is an option and not a number… so we’ll just make TBL a +little younger.)

+
$ accumulo shell -u root -p secret
+root@default> table example_schema.scientists
+root@default example_schema.scientists> insert row3 metadata name "Tim Berners-Lee"
+root@default example_schema.scientists> insert row3 metadata age 60
+root@default example_schema.scientists> insert row3 metadata date 5321
+
+
+
SELECT * FROM example_schema.scientists;
+
+
+
 recordkey |      name       | age |  birthday
+-----------+-----------------+-----+------------
+ row1      | Grace Hopper    | 109 | 1906-12-09
+ row2      | Alan Turing     | 103 | 1912-06-23
+ row3      | Tim Berners-Lee |  60 | 1984-07-27
+(3 rows)
+
+
+

You can also drop tables using DROP TABLE. This command drops both +metadata and the tables. See the below section on External +Tables for more details on internal and external +tables.

+
DROP TABLE example_schema.scientists;
+
+
+
+
+

Indexing columns#

+

Internally, the connector creates an Accumulo Range and packs it in +a split. This split gets passed to a Trino Worker to read the data from +the Range via a BatchScanner. When issuing a query that results +in a full table scan, each Trino Worker gets a single Range that +maps to a single tablet of the table. When issuing a query with a +predicate (i.e. WHERE x = 10 clause), Trino passes the values +within the predicate (10) to the connector so it can use this +information to scan less data. When the Accumulo row ID is used as part +of the predicate clause, this narrows down the Range lookup to quickly +retrieve a subset of data from Accumulo.

+

But what about the other columns? If you’re frequently querying on +non-row ID columns, you should consider using the indexing +feature built into the Accumulo connector. This feature can drastically +reduce query runtime when selecting a handful of values from the table, +and the heavy lifting is done for you when loading data via Trino +INSERT statements. Keep in mind writing data to Accumulo via +INSERT does not have high throughput.

+

To enable indexing, add the index_columns table property and specify +a comma-delimited list of Trino column names you wish to index (we use the +string serializer here to help with this example – you +should be using the default lexicoder serializer).

+
CREATE TABLE example_schema.scientists (
+  recordkey VARCHAR,
+  name VARCHAR,
+  age BIGINT,
+  birthday DATE
+)
+WITH (
+  serializer = 'string',
+  index_columns='name,age,birthday'
+);
+
+
+

After creating the table, we see there are an additional two Accumulo +tables to store the index and metrics.

+
root@default> tables
+accumulo.metadata
+accumulo.root
+example_schema.scientists
+example_schema.scientists_idx
+example_schema.scientists_idx_metrics
+trace
+
+
+

After inserting data, we can look at the index table and see there are +indexed values for the name, age, and birthday columns. The connector +queries this index table

+
INSERT INTO example_schema.scientists VALUES
+('row1', 'Grace Hopper', 109, DATE '1906-12-09'),
+('row2', 'Alan Turing', 103, DATE '1912-06-23');
+
+
+
root@default> scan -t example_schema.scientists_idx
+-21011 metadata_date:row2 []
+-23034 metadata_date:row1 []
+103 metadata_age:row2 []
+109 metadata_age:row1 []
+Alan Turing metadata_name:row2 []
+Grace Hopper metadata_name:row1 []
+
+
+

When issuing a query with a WHERE clause against indexed columns, +the connector searches the index table for all row IDs that contain the +value within the predicate. These row IDs are bundled into a Trino +split as single-value Range objects, the number of row IDs per split +is controlled by the value of accumulo.index_rows_per_split, and +passed to a Trino worker to be configured in the BatchScanner which +scans the data table.

+
SELECT * FROM example_schema.scientists WHERE age = 109;
+
+
+
 recordkey |     name     | age |  birthday
+-----------+--------------+-----+------------
+ row1      | Grace Hopper | 109 | 1906-12-09
+(1 row)
+
+
+
+
+

Loading data#

+

The Accumulo connector supports loading data via INSERT statements, however +this method tends to be low-throughput and should not be relied on when +throughput is a concern.

+
+
+

External tables#

+

By default, the tables created using SQL statements via Trino are +internal tables, that is both the Trino table metadata and the +Accumulo tables are managed by Trino. When you create an internal +table, the Accumulo table is created as well. You receive an error +if the Accumulo table already exists. When an internal table is dropped +via Trino, the Accumulo table, and any index tables, are dropped as +well.

+

To change this behavior, set the external property to true when +issuing the CREATE statement. This makes the table an external +table, and a DROP TABLE command only deletes the metadata +associated with the table. If the Accumulo tables do not already exist, +they are created by the connector.

+

Creating an external table will set any configured locality groups as well +as the iterators on the index and metrics tables, if the table is indexed. +In short, the only difference between an external table and an internal table, +is that the connector deletes the Accumulo tables when a DROP TABLE command +is issued.

+

External tables can be a bit more difficult to work with, as the data is stored +in an expected format. If the data is not stored correctly, then you’re +gonna have a bad time. Users must provide a column_mapping property +when creating the table. This creates the mapping of Trino column name +to the column family/qualifier for the cell of the table. The value of the +cell is stored in the Value of the Accumulo key/value pair. By default, +this value is expected to be serialized using Accumulo’s lexicoder API. +If you are storing values as strings, you can specify a different serializer +using the serializer property of the table. See the section on +Table Properties for more information.

+

Next, we create the Trino external table.

+
CREATE TABLE external_table (
+  a VARCHAR,
+  b BIGINT,
+  c DATE
+)
+WITH (
+  column_mapping = 'a:md:a,b:md:b,c:md:c',
+  external = true,
+  index_columns = 'b,c',
+  locality_groups = 'foo:b,c'
+);
+
+
+

After creating the table, usage of the table continues as usual:

+
INSERT INTO external_table VALUES
+('1', 1, DATE '2015-03-06'),
+('2', 2, DATE '2015-03-07');
+
+
+
SELECT * FROM external_table;
+
+
+
 a | b |     c
+---+---+------------
+ 1 | 1 | 2015-03-06
+ 2 | 2 | 2015-03-06
+(2 rows)
+
+
+
DROP TABLE external_table;
+
+
+

After dropping the table, the table still exists in Accumulo because it is external.

+
root@default> tables
+accumulo.metadata
+accumulo.root
+external_table
+external_table_idx
+external_table_idx_metrics
+trace
+
+
+

If we wanted to add a new column to the table, we can create the table again and specify a new column. +Any existing rows in the table have a value of NULL. This command re-configures the Accumulo +tables, setting the locality groups and iterator configuration.

+
CREATE TABLE external_table (
+  a VARCHAR,
+  b BIGINT,
+  c DATE,
+  d INTEGER
+)
+WITH (
+  column_mapping = 'a:md:a,b:md:b,c:md:c,d:md:d',
+  external = true,
+  index_columns = 'b,c,d',
+  locality_groups = 'foo:b,c,d'
+);
+
+SELECT * FROM external_table;
+
+
+
 a | b |     c      |  d
+---+---+------------+------
+ 1 | 1 | 2015-03-06 | NULL
+ 2 | 2 | 2015-03-07 | NULL
+(2 rows)
+
+
+
+
+

Table properties#

+

Table property usage example:

+
CREATE TABLE example_schema.scientists (
+  recordkey VARCHAR,
+  name VARCHAR,
+  age BIGINT,
+  birthday DATE
+)
+WITH (
+  column_mapping = 'name:metadata:name,age:metadata:age,birthday:metadata:date',
+  index_columns = 'name,age'
+);
+
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Default value

Description

column_mapping

(generated)

Comma-delimited list of column metadata: col_name:col_family:col_qualifier,[...]. Required for external tables. Not setting this property results in auto-generated column names.

index_columns

(none)

A comma-delimited list of Trino columns that are indexed in this table’s corresponding index table

external

false

If true, Trino will only do metadata operations for the table. Otherwise, Trino will create and drop Accumulo tables where appropriate.

locality_groups

(none)

List of locality groups to set on the Accumulo table. Only valid on internal tables. String format is locality group name, colon, comma delimited list of column families in the group. Groups are delimited by pipes. Example: group1:famA,famB,famC|group2:famD,famE,famF|etc...

row_id

(first column)

Trino column name that maps to the Accumulo row ID.

serializer

default

Serializer for Accumulo data encodings. Can either be default, string, lexicoder or a Java class name. Default is default, i.e. the value from AccumuloRowSerializer.getDefault(), i.e. lexicoder.

scan_auths

(user auths)

Scan-time authorizations set on the batch scanner.

+
+
+

Session properties#

+

You can change the default value of a session property by using SET SESSION. +Note that session properties are prefixed with the catalog name:

+
SET SESSION example.column_filter_optimizations_enabled = false;
+
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Default value

Description

optimize_locality_enabled

true

Set to true to enable data locality for non-indexed scans

optimize_split_ranges_enabled

true

Set to true to split non-indexed queries by tablet splits. Should generally be true.

optimize_index_enabled

true

Set to true to enable usage of the secondary index on query

index_rows_per_split

10000

The number of Accumulo row IDs that are packed into a single Trino split

index_threshold

0.2

The ratio between number of rows to be scanned based on the index over the total number of rows. If the ratio is below this threshold, the index will be used.

index_lowest_cardinality_threshold

0.01

The threshold where the column with the lowest cardinality will be used instead of computing an intersection of ranges in the index. Secondary index must be enabled

index_metrics_enabled

true

Set to true to enable usage of the metrics table to optimize usage of the index

scan_username

(config)

User to impersonate when scanning the tables. This property trumps the scan_auths table property

index_short_circuit_cardinality_fetch

true

Short circuit the retrieval of index metrics once any column is less than the lowest cardinality threshold

index_cardinality_cache_polling_duration

10ms

Sets the cardinality cache polling duration for short circuit retrieval of index metrics

+
+
+

Adding columns#

+

Adding a new column to an existing table cannot be done today via +ALTER TABLE [table] ADD COLUMN [name] [type] because of the additional +metadata required for the columns to work; the column family, qualifier, +and if the column is indexed.

+
+
+

Serializers#

+

The Trino connector for Accumulo has a pluggable serializer framework +for handling I/O between Trino and Accumulo. This enables end-users the +ability to programmatically serialized and deserialize their special data +formats within Accumulo, while abstracting away the complexity of the +connector itself.

+

There are two types of serializers currently available; a string +serializer that treats values as Java String, and a lexicoder +serializer that leverages Accumulo’s Lexicoder API to store values. The +default serializer is the lexicoder serializer, as this serializer +does not require expensive conversion operations back and forth between +String objects and the Trino types – the cell’s value is encoded as a +byte array.

+

Additionally, the lexicoder serializer does proper lexigraphical ordering of +numerical types like BIGINT or TIMESTAMP. This is essential for the connector +to properly leverage the secondary index when querying for data.

+

You can change the default the serializer by specifying the +serializer table property, using either default (which is +lexicoder), string or lexicoder for the built-in types, or +you could provide your own implementation by extending +AccumuloRowSerializer, adding it to the Trino CLASSPATH, and +specifying the fully-qualified Java class name in the connector configuration.

+
CREATE TABLE example_schema.scientists (
+  recordkey VARCHAR,
+  name VARCHAR,
+  age BIGINT,
+  birthday DATE
+)
+WITH (
+  column_mapping = 'name:metadata:name,age:metadata:age,birthday:metadata:date',
+  serializer = 'default'
+);
+
+
+
INSERT INTO example_schema.scientists VALUES
+('row1', 'Grace Hopper', 109, DATE '1906-12-09' ),
+('row2', 'Alan Turing', 103, DATE '1912-06-23' );
+
+
+
root@default> scan -t example_schema.scientists
+row1 metadata:age []    \x08\x80\x00\x00\x00\x00\x00\x00m
+row1 metadata:date []    \x08\x7F\xFF\xFF\xFF\xFF\xFF\xA6\x06
+row1 metadata:name []    Grace Hopper
+row2 metadata:age []    \x08\x80\x00\x00\x00\x00\x00\x00g
+row2 metadata:date []    \x08\x7F\xFF\xFF\xFF\xFF\xFF\xAD\xED
+row2 metadata:name []    Alan Turing
+
+
+
CREATE TABLE example_schema.stringy_scientists (
+  recordkey VARCHAR,
+  name VARCHAR,
+  age BIGINT,
+  birthday DATE
+)
+WITH (
+  column_mapping = 'name:metadata:name,age:metadata:age,birthday:metadata:date',
+  serializer = 'string'
+);
+
+
+
INSERT INTO example_schema.stringy_scientists VALUES
+('row1', 'Grace Hopper', 109, DATE '1906-12-09' ),
+('row2', 'Alan Turing', 103, DATE '1912-06-23' );
+
+
+
root@default> scan -t example_schema.stringy_scientists
+row1 metadata:age []    109
+row1 metadata:date []    -23034
+row1 metadata:name []    Grace Hopper
+row2 metadata:age []    103
+row2 metadata:date []    -21011
+row2 metadata:name []    Alan Turing
+
+
+
CREATE TABLE example_schema.custom_scientists (
+  recordkey VARCHAR,
+  name VARCHAR,
+  age BIGINT,
+  birthday DATE
+)
+WITH (
+  column_mapping = 'name:metadata:name,age:metadata:age,birthday:metadata:date',
+  serializer = 'my.serializer.package.MySerializer'
+);
+
+
+
+
+

Metadata management#

+

Metadata for the Trino/Accumulo tables is stored in ZooKeeper. You can, +and should, issue SQL statements in Trino to create and drop tables. +This is the easiest method of creating the metadata required to make the +connector work. It is best to not mess with the metadata, but here are +the details of how it is stored.

+

A root node in ZooKeeper holds all the mappings, and the format is as +follows:

+
/metadata-root/schema/table
+
+
+

Where metadata-root is the value of zookeeper.metadata.root in +the config file (default is /trino-accumulo), schema is the +Trino schema (which is identical to the Accumulo namespace name), and +table is the Trino table name (again, identical to Accumulo name). +The data of the table ZooKeeper node is a serialized +AccumuloTable Java object (which resides in the connector code). +This table contains the schema (namespace) name, table name, column +definitions, the serializer to use for the table, and any additional +table properties.

+

If you have a need to programmatically manipulate the ZooKeeper metadata +for Accumulo, take a look at +io.trino.plugin.accumulo.metadata.ZooKeeperMetadataManager for some +Java code to simplify the process.

+
+
+

Converting table from internal to external#

+

If your table is internal, you can convert it to an external table by deleting +the corresponding znode in ZooKeeper, effectively making the table no longer exist as +far as Trino is concerned. Then, create the table again using the same DDL, but adding the +external = true table property.

+

For example:

+

1. We’re starting with an internal table foo.bar that was created with the below DDL. +If you have not previously defined a table property for column_mapping (like this example), +be sure to describe the table before deleting the metadata. We need the column mappings +when creating the external table.

+
CREATE TABLE foo.bar (a VARCHAR, b BIGINT, c DATE)
+WITH (
+    index_columns = 'b,c'
+);
+
+
+
DESCRIBE foo.bar;
+
+
+
 Column |  Type   | Extra |               Comment
+--------+---------+-------+-------------------------------------
+ a      | varchar |       | Accumulo row ID
+ b      | bigint  |       | Accumulo column b:b. Indexed: true
+ c      | date    |       | Accumulo column c:c. Indexed: true
+
+
+

2. Using the ZooKeeper CLI, delete the corresponding znode. Note this uses the default ZooKeeper +metadata root of /trino-accumulo

+
$ zkCli.sh
+[zk: localhost:2181(CONNECTED) 1] delete /trino-accumulo/foo/bar
+
+
+

3. Re-create the table using the same DDL as before, but adding the external=true property. +Note that if you had not previously defined the column_mapping, you need to add the property +to the new DDL (external tables require this property to be set). The column mappings are in +the output of the DESCRIBE statement.

+
CREATE TABLE foo.bar (
+  a VARCHAR,
+  b BIGINT,
+  c DATE
+)
+WITH (
+  column_mapping = 'a:a:a,b:b:b,c:c:c',
+  index_columns = 'b,c',
+  external = true
+);
+
+
+
+
+

Type mapping#

+

Because Trino and Accumulo each support types that the other does not, this +connector modifies some types when reading or writing data. Data types may not +map the same way in both directions between Trino and the data source. Refer to +the following sections for type mapping in each direction.

+
+

Accumulo type to Trino type mapping#

+

The connector maps Accumulo types to the corresponding Trino types following +this table:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Accumulo type to Trino type mapping#

Accumulo type

Trino type

Notes

BOOLEAN

BOOLEAN

TINYINT

TINYINT

SMALLINT

SMALLINT

INTEGER

INTEGER

BIGINT

BIGINT

REAL

REAL

DOUBLE

DOUBLE

VARCHAR(n)

VARCHAR(n)

VARBINARY

VARBINARY

DATE

DATE

TIME(n)

TIME(n)

TIMESTAMP(n)

TIMESTAMP(n)

+

No other types are supported

+
+
+

Trino type to Accumulo type mapping#

+

The connector maps Trino types to the corresponding Trino type to Accumulo type +mapping types following this table:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Trino type to Accumulo type mapping#

Trino type

Accumulo type

Notes

BOOLEAN

BOOLEAN

TINYINT

TINYINT

Trino only supports writing values belonging to [0, 127]

SMALLINT

SMALLINT

INTEGER

INTEGER

BIGINT

BIGINT

REAL

REAL

DOUBLE

DOUBLE

VARCHAR(n)

VARCHAR(n)

VARBINARY

VARBINARY

DATE

DATE

TIME(n)

TIME(n)

TIMESTAMP(n)

TIMESTAMP(n)

+

No other types are supported

+
+
+
+

SQL support#

+

The connector provides read and write access to data and metadata in +the Accumulo database. In addition to the globally available and read operation +statements, the connector supports the following features:

+ +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/atop.html b/430/connector/atop.html new file mode 100644 index 000000000..03b4b993d --- /dev/null +++ b/430/connector/atop.html @@ -0,0 +1,857 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Atop connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+ +
+
+ +
+
+ +
+

Atop connector#

+

The Atop connector supports reading disk utilization statistics from the Atop +(Advanced System and Process Monitor) Linux server performance analysis tool.

+
+

Requirements#

+

In order to use this connector, the host on which the Trino worker is running +needs to have the atop tool installed locally.

+
+
+

Connector configuration#

+

The connector can read disk utilization statistics on the Trino cluster. +Create a catalog properties file that specifies the Atop connector by +setting the connector.name to atop.

+

For example, create the file etc/catalog/example.properties with the +following connector properties as appropriate for your setup:

+
connector.name=atop
+atop.executable-path=/usr/bin/atop
+
+
+
+
+

Configuration properties#

+ ++++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Default value

Required

Description

atop.concurrent-readers-per-node

1

Yes

The number of concurrent read operations allowed per node.

atop.executable-path

(none)

Yes

The file path on the local file system for the atop utility.

atop.executable-read-timeout

1ms

Yes

The timeout when reading from the atop process.

atop.max-history-days

30

Yes

The maximum number of days in the past to take into account for statistics.

atop.security

ALLOW_ALL

Yes

The access control for the connector.

atop.time-zone

System default

Yes

The time zone identifier in which the atop data is collected. Generally the timezone of the host. +Sample time zone identifiers: Europe/Vienna, +0100, UTC.

+
+
+

Usage#

+

The Atop connector provides a default schema.

+

The tables exposed by this connector can be retrieved by running SHOW TABLES:

+
SHOW TABLES FROM example.default;
+
+
+
  Table
+---------
+ disks
+ reboots
+(2 rows)
+
+
+

The disks table offers disk utilization statistics recorded on the Trino node.

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Disks columns#

Name

Type

Description

host_ip

VARCHAR

Trino worker IP

start_time

TIMESTAMP(3) WITH TIME ZONE

Interval start time for the statistics

end_time

TIMESTAMP(3) WITH TIME ZONE

Interval end time for the statistics

device_name

VARCHAR

Logical volume/hard disk name

utilization_percent

DOUBLE

The percentage of time the unit was busy handling requests

io_time

INTERVAL DAY TO SECOND

Time spent for I/O

read_requests

BIGINT

Number of reads issued

sectors_read

BIGINT

Number of sectors transferred for reads

write_requests

BIGINT

Number of writes issued

sectors_written

BIGINT

Number of sectors transferred for write

+

The reboots table offers information about the system reboots performed on the Trino node.

+ + +++++ + + + + + + + + + + + + + + + + +
Reboots columns#

Name

Type

Description

host_ip

VARCHAR

Trino worker IP

power_on_time

TIMESTAMP(3) WITH TIME ZONE

The boot/reboot timestamp

+
+
+

SQL support#

+

The connector provides globally available and +read operation statements to access system and process monitor +information on your Trino nodes.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/bigquery.html b/430/connector/bigquery.html new file mode 100644 index 000000000..d63be8e10 --- /dev/null +++ b/430/connector/bigquery.html @@ -0,0 +1,1219 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + BigQuery connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

BigQuery connector#

+

The BigQuery connector allows querying the data stored in BigQuery. This can be used to join data between +different systems like BigQuery and Hive. The connector uses the BigQuery +Storage API to +read the data from the tables.

+
+

BigQuery Storage API#

+

The Storage API streams data in parallel directly from BigQuery via gRPC without +using Google Cloud Storage as an intermediary. +It has a number of advantages over using the previous export-based read flow +that should generally lead to better read performance:

+
+
Direct Streaming

It does not leave any temporary files in Google Cloud Storage. Rows are read +directly from BigQuery servers using an Avro wire format.

+
+
Column Filtering

The new API allows column filtering to only read the data you are interested in. +Backed by a columnar datastore, +it can efficiently stream data without reading all columns.

+
+
Dynamic Sharding

The API rebalances records between readers until they all complete. This means +that all Map phases will finish nearly concurrently. See this blog article on +how dynamic sharding is similarly used in Google Cloud Dataflow.

+
+
+
+
+

Requirements#

+

To connect to BigQuery, you need:

+
    +
  • To enable the BigQuery Storage Read API.

  • +
  • Network access from your Trino coordinator and workers to the +Google Cloud API service endpoint. This endpoint uses HTTPS, or port 443.

  • +
  • To configure BigQuery so that the Trino coordinator and workers have permissions +in BigQuery.

  • +
  • To set up authentication. Your authentiation options differ depending on whether +you are using Dataproc/Google Compute Engine (GCE) or not.

    +

    On Dataproc/GCE the authentication is done from the machine’s role.

    +

    Outside Dataproc/GCE you have 3 options:

    +
      +
    • Use a service account JSON key and GOOGLE_APPLICATION_CREDENTIALS as +described in the Google Cloud authentication getting started guide.

    • +
    • Set bigquery.credentials-key in the catalog properties file. It should +contain the contents of the JSON file, encoded using base64.

    • +
    • Set bigquery.credentials-file in the catalog properties file. It should +point to the location of the JSON file.

    • +
    +
  • +
+
+
+

Configuration#

+

To configure the BigQuery connector, create a catalog properties file in +etc/catalog named example.properties, to mount the BigQuery connector as +the example catalog. Create the file with the following contents, replacing +the connection properties as appropriate for your setup:

+
connector.name=bigquery
+bigquery.project-id=<your Google Cloud Platform project id>
+
+
+
+

Multiple GCP projects#

+

The BigQuery connector can only access a single GCP project.Thus, if you have +data in multiple GCP projects, You need to create several catalogs, each +pointing to a different GCP project. For example, if you have two GCP projects, +one for the sales and one for analytics, you can create two properties files in +etc/catalog named sales.properties and analytics.properties, both +having connector.name=bigquery but with different project-id. This will +create the two catalogs, sales and analytics respectively.

+
+
+

Configuring partitioning#

+

By default the connector creates one partition per 400MB in the table being +read (before filtering). This should roughly correspond to the maximum number +of readers supported by the BigQuery Storage API. This can be configured +explicitly with the bigquery.parallelism property. BigQuery may limit the +number of partitions based on server constraints.

+
+
+

Arrow serialization support#

+

This is an experimental feature which introduces support for using Apache Arrow +as the serialization format when reading from BigQuery. Please note there are +a few caveats:

+
    +
  • Using Apache Arrow serialization is disabled by default. In order to enable +it, set the bigquery.experimental.arrow-serialization.enabled +configuration property to true and add +--add-opens=java.base/java.nio=ALL-UNNAMED to the Trino +JVM config.

  • +
+
+
+

Reading from views#

+

The connector has a preliminary support for reading from BigQuery views. Please note there are +a few caveats:

+
    +
  • Reading from views is disabled by default. In order to enable it, set the +bigquery.views-enabled configuration property to true.

  • +
  • BigQuery views are not materialized by default, which means that the +connector needs to materialize them before it can read them. This process +affects the read performance.

  • +
  • The materialization process can also incur additional costs to your BigQuery bill.

  • +
  • By default, the materialized views are created in the same project and +dataset. Those can be configured by the optional bigquery.view-materialization-project +and bigquery.view-materialization-dataset properties, respectively. The +service account must have write permission to the project and the dataset in +order to materialize the view.

  • +
+
+
+

Configuration properties#

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property

Description

Default

bigquery.project-id

The Google Cloud Project ID where the data reside

Taken from the service account

bigquery.parent-project-id

The project ID Google Cloud Project to bill for the export

Taken from the service account

bigquery.parallelism

The number of partitions to split the data into

The number of executors

bigquery.views-enabled

Enables the connector to read from views and not only tables. Please read this section before enabling this feature.

false

bigquery.view-expire-duration

Expire duration for the materialized view.

24h

bigquery.view-materialization-project

The project where the materialized view is going to be created

The view’s project

bigquery.view-materialization-dataset

The dataset where the materialized view is going to be created

The view’s dataset

bigquery.skip-view-materialization

Use REST API to access views instead of Storage API. BigQuery BIGNUMERIC and TIMESTAMP types are unsupported.

false

bigquery.views-cache-ttl

Duration for which the materialization of a view will be cached and reused. Set to 0ms to disable the cache.

15m

bigquery.metadata.cache-ttl

Duration for which metadata retrieved from BigQuery is cached and reused. Set to 0ms to disable the cache.

0ms

bigquery.max-read-rows-retries

The number of retries in case of retryable server issues

3

bigquery.credentials-key

The base64 encoded credentials key

None. See the requirements section.

bigquery.credentials-file

The path to the JSON credentials file

None. See the requirements section.

bigquery.case-insensitive-name-matching

Match dataset and table names case-insensitively

false

bigquery.query-results-cache.enabled

Enable query results cache

false

bigquery.experimental.arrow-serialization.enabled

Enable using Apache Arrow serialization when reading data from BigQuery. Please read this section before enabling this feature.

false

bigquery.rpc-proxy.enabled

Use a proxy for communication with BigQuery.

false

bigquery.rpc-proxy.uri

Proxy URI to use if connecting through a proxy.

bigquery.rpc-proxy.username

Proxy user name to use if connecting through a proxy.

bigquery.rpc-proxy.password

Proxy password to use if connecting through a proxy.

bigquery.rpc-proxy.keystore-path

Keystore containing client certificates to present to proxy if connecting through a proxy. Only required if proxy uses mutual TLS.

bigquery.rpc-proxy.keystore-password

Password of the keystore specified by bigquery.rpc-proxy.keystore-path.

bigquery.rpc-proxy.truststore-path

Truststore containing certificates of the proxy server if connecting through a proxy.

bigquery.rpc-proxy.truststore-password

Password of the truststore specified by bigquery.rpc-proxy.truststore-path.

+
+
+
+

Type mapping#

+

Because Trino and BigQuery each support types that the other does not, this +connector modifies some types when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction.

+
+

BigQuery type to Trino type mapping#

+

The connector maps BigQuery types to the corresponding Trino types according +to the following table:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
BigQuery type to Trino type mapping#

BigQuery type

Trino type

Notes

BOOLEAN

BOOLEAN

INT64

BIGINT

INT, SMALLINT, INTEGER, BIGINT, TINYINT, and +BYTEINT are aliases for INT64 in BigQuery.

FLOAT64

DOUBLE

NUMERIC

DECIMAL(P,S)

The default precision and scale of NUMERIC is (38, 9).

BIGNUMERIC

DECIMAL(P,S)

Precision > 38 is not supported. The default precision and scale of +BIGNUMERIC is (77, 38).

DATE

DATE

DATETIME

TIMESTAMP(6)

STRING

VARCHAR

BYTES

VARBINARY

TIME

TIME(6)

TIMESTAMP

TIMESTAMP(6) WITH TIME ZONE

Time zone is UTC

GEOGRAPHY

VARCHAR

In Well-known text (WKT) format

ARRAY

ARRAY

RECORD

ROW

+

No other types are supported.

+
+
+

Trino type to BigQuery type mapping#

+

The connector maps Trino types to the corresponding BigQuery types according +to the following table:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Trino type to BigQuery type mapping#

Trino type

BigQuery type

Notes

BOOLEAN

BOOLEAN

VARBINARY

BYTES

DATE

DATE

DOUBLE

FLOAT

BIGINT

INT64

INT, SMALLINT, INTEGER, BIGINT, TINYINT, and +BYTEINT are aliases for INT64 in BigQuery.

DECIMAL(P,S)

NUMERIC

The default precision and scale of NUMERIC is (38, 9).

VARCHAR

STRING

TIMESTAMP(6)

DATETIME

+

No other types are supported.

+
+
+
+

System tables#

+

For each Trino table which maps to BigQuery view there exists a system table +which exposes BigQuery view definition. Given a BigQuery view example_view +you can send query SELECT * example_view$view_definition to see the SQL +which defines view in BigQuery.

+
+
+

Special columns#

+

In addition to the defined columns, the BigQuery connector exposes +partition information in a number of hidden columns:

+
    +
  • $partition_date: Equivalent to _PARTITIONDATE pseudo-column in BigQuery

  • +
  • $partition_time: Equivalent to _PARTITIONTIME pseudo-column in BigQuery

  • +
+

You can use these columns in your SQL statements like any other column. They +can be selected directly, or used in conditional statements. For example, you +can inspect the partition date and time for each record:

+
SELECT *, "$partition_date", "$partition_time"
+FROM example.web.page_views;
+
+
+

Retrieve all records stored in the partition _PARTITIONDATE = '2022-04-07':

+
SELECT *
+FROM example.web.page_views
+WHERE "$partition_date" = date '2022-04-07';
+
+
+
+

Note

+

Two special partitions __NULL__ and __UNPARTITIONED__ are not supported.

+
+
+
+

SQL support#

+

The connector provides read and write access to data and metadata in the +BigQuery database. In addition to the +globally available and +read operation statements, the connector supports +the following features:

+ +
+
+

Fault-tolerant execution support#

+

The connector supports Fault-tolerant execution of query +processing. Read and write operations are both supported with any retry policy.

+
+
+

Table functions#

+

The connector provides specific table functions to +access BigQuery.

+
+

query(varchar) -> table#

+

The query function allows you to query the underlying BigQuery directly. It +requires syntax native to BigQuery, because the full query is pushed down and +processed by BigQuery. This can be useful for accessing native features which are +not available in Trino or for improving query performance in situations where +running a query natively may be faster.

+

The native query passed to the underlying data source is required to return a +table as a result set. Only the data source performs validation or security +checks for these queries using its own configuration. Trino does not perform +these tasks. Only use passthrough queries to read data.

+

For example, query the example catalog and group and concatenate all +employee IDs by manager ID:

+
SELECT
+  *
+FROM
+  TABLE(
+    example.system.query(
+      query => 'SELECT
+        manager_id, STRING_AGG(employee_id)
+      FROM
+        company.employees
+      GROUP BY
+        manager_id'
+    )
+  );
+
+
+
+

Note

+

The query engine does not preserve the order of the results of this +function. If the passed query contains an ORDER BY clause, the +function result may not be ordered as expected.

+
+
+
+
+

FAQ#

+
+

What is the Pricing for the Storage API?#

+

See the BigQuery pricing documentation.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/blackhole.html b/430/connector/blackhole.html new file mode 100644 index 000000000..7a90b602e --- /dev/null +++ b/430/connector/blackhole.html @@ -0,0 +1,788 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Black Hole connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Black Hole connector#

+

Primarily Black Hole connector is designed for high performance testing of +other components. It works like the /dev/null device on Unix-like +operating systems for data writing and like /dev/null or /dev/zero +for data reading. However, it also has some other features that allow testing Trino +in a more controlled manner. Metadata for any tables created via this connector +is kept in memory on the coordinator and discarded when Trino restarts. +Created tables are by default always empty, and any data written to them +is ignored and data reads return no rows.

+

During table creation, a desired rows number can be specified. +In such cases, writes behave in the same way, but reads +always return the specified number of some constant rows. +You shouldn’t rely on the content of such rows.

+
+

Configuration#

+

Create etc/catalog/example.properties to mount the blackhole connector +as the example catalog, with the following contents:

+
connector.name=blackhole
+
+
+
+
+

Examples#

+

Create a table using the blackhole connector:

+
CREATE TABLE example.test.nation AS
+SELECT * from tpch.tiny.nation;
+
+
+

Insert data into a table in the blackhole connector:

+
INSERT INTO example.test.nation
+SELECT * FROM tpch.tiny.nation;
+
+
+

Select from the blackhole connector:

+
SELECT count(*) FROM example.test.nation;
+
+
+

The above query always returns zero.

+

Create a table with a constant number of rows (500 * 1000 * 2000):

+
CREATE TABLE example.test.nation (
+  nationkey BIGINT,
+  name VARCHAR
+)
+WITH (
+  split_count = 500,
+  pages_per_split = 1000,
+  rows_per_page = 2000
+);
+
+
+

Now query it:

+
SELECT count(*) FROM example.test.nation;
+
+
+

The above query returns 1,000,000,000.

+

Length of variable length columns can be controlled using the field_length +table property (default value is equal to 16):

+
CREATE TABLE example.test.nation (
+  nationkey BIGINT,
+  name VARCHAR
+)
+WITH (
+  split_count = 500,
+  pages_per_split = 1000,
+  rows_per_page = 2000,
+  field_length = 100
+);
+
+
+

The consuming and producing rate can be slowed down +using the page_processing_delay table property. +Setting this property to 5s leads to a 5 second +delay before consuming or producing a new page:

+
CREATE TABLE example.test.delay (
+  dummy BIGINT
+)
+WITH (
+  split_count = 1,
+  pages_per_split = 1,
+  rows_per_page = 1,
+  page_processing_delay = '5s'
+);
+
+
+
+
+

SQL support#

+

The connector provides globally available, +read operation, and supports the following +additional features:

+ +
+

Note

+

The connector discards all written data. While read operations are supported, +they return rows with all NULL values, with the number of rows controlled +via table properties.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/cassandra.html b/430/connector/cassandra.html new file mode 100644 index 000000000..cc0bf1666 --- /dev/null +++ b/430/connector/cassandra.html @@ -0,0 +1,1199 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Cassandra connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Cassandra connector#

+

The Cassandra connector allows querying data stored in +Apache Cassandra.

+
+

Requirements#

+

To connect to Cassandra, you need:

+
    +
  • Cassandra version 3.0 or higher.

  • +
  • Network access from the Trino coordinator and workers to Cassandra. +Port 9042 is the default port.

  • +
+
+
+

Configuration#

+

To configure the Cassandra connector, create a catalog properties file +etc/catalog/example.properties with the following contents, replacing +host1,host2 with a comma-separated list of the Cassandra nodes, used to +discovery the cluster topology:

+
connector.name=cassandra
+cassandra.contact-points=host1,host2
+cassandra.load-policy.dc-aware.local-dc=datacenter1
+
+
+

You also need to set cassandra.native-protocol-port, if your +Cassandra nodes are not using the default port 9042.

+
+

Multiple Cassandra clusters#

+

You can have as many catalogs as you need, so if you have additional +Cassandra clusters, simply add another properties file to etc/catalog +with a different name, making sure it ends in .properties. For +example, if you name the property file sales.properties, Trino +creates a catalog named sales using the configured connector.

+
+
+
+

Configuration properties#

+

The following configuration properties are available:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

cassandra.contact-points

Comma-separated list of hosts in a Cassandra cluster. The Cassandra driver uses these contact points to discover cluster topology. At least one Cassandra host is required.

cassandra.native-protocol-port

The Cassandra server port running the native client protocol, defaults to 9042.

cassandra.consistency-level

Consistency levels in Cassandra refer to the level of consistency to be used for both read and write operations. More information about consistency levels can be found in the Cassandra consistency documentation. This property defaults to a consistency level of ONE. Possible values include ALL, EACH_QUORUM, QUORUM, LOCAL_QUORUM, ONE, TWO, THREE, LOCAL_ONE, ANY, SERIAL, LOCAL_SERIAL.

cassandra.allow-drop-table

Enables DROP TABLE operations. Defaults to false.

cassandra.username

Username used for authentication to the Cassandra cluster. This is a global setting used for all connections, regardless of the user connected to Trino.

cassandra.password

Password used for authentication to the Cassandra cluster. This is a global setting used for all connections, regardless of the user connected to Trino.

cassandra.protocol-version

It is possible to override the protocol version for older Cassandra clusters. By default, the value corresponds to the default protocol version used in the underlying Cassandra java driver. Possible values include V3, V4, V5, V6.

+
+

Note

+

If authorization is enabled, cassandra.username must have enough permissions to perform SELECT queries on +the system.size_estimates table.

+
+

The following advanced configuration properties are available:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

cassandra.fetch-size

Number of rows fetched at a time in a Cassandra query.

cassandra.partition-size-for-batch-select

Number of partitions batched together into a single select for a single partion key column table.

cassandra.split-size

Number of keys per split when querying Cassandra.

cassandra.splits-per-node

Number of splits per node. By default, the values from the system.size_estimates table are used. Only override when connecting to Cassandra versions < 2.1.5, which lacks the system.size_estimates table.

cassandra.batch-size

Maximum number of statements to execute in one batch.

cassandra.client.read-timeout

Maximum time the Cassandra driver waits for an answer to a query from one Cassandra node. Note that the underlying Cassandra driver may retry a query against more than one node in the event of a read timeout. Increasing this may help with queries that use an index.

cassandra.client.connect-timeout

Maximum time the Cassandra driver waits to establish a connection to a Cassandra node. Increasing this may help with heavily loaded Cassandra clusters.

cassandra.client.so-linger

Number of seconds to linger on close if unsent data is queued. If set to zero, the socket will be closed immediately. When this option is non-zero, a socket lingers that many seconds for an acknowledgement that all data was written to a peer. This option can be used to avoid consuming sockets on a Cassandra server by immediately closing connections when they are no longer needed.

cassandra.retry-policy

Policy used to retry failed requests to Cassandra. This property defaults to DEFAULT. Using BACKOFF may help when queries fail with “not enough replicas”. The other possible values are DOWNGRADING_CONSISTENCY and FALLTHROUGH.

cassandra.load-policy.use-dc-aware

Set to true if the load balancing policy requires a local datacenter, defaults to true.

cassandra.load-policy.dc-aware.local-dc

The name of the datacenter considered “local”.

cassandra.load-policy.dc-aware.used-hosts-per-remote-dc

Uses the provided number of host per remote datacenter as failover for the local hosts for DefaultLoadBalancingPolicy.

cassandra.load-policy.dc-aware.allow-remote-dc-for-local

Set to true to allow to use hosts of remote datacenter for local consistency level.

cassandra.no-host-available-retry-timeout

Retry timeout for AllNodesFailedException, defaults to 1m.

cassandra.speculative-execution.limit

The number of speculative executions. This is disabled by default.

cassandra.speculative-execution.delay

The delay between each speculative execution, defaults to 500ms.

cassandra.tls.enabled

Whether TLS security is enabled, defaults to false.

cassandra.tls.keystore-path

Path to the PEM or JKS key store file.

cassandra.tls.truststore-path

Path to the PEM or JKS trust store file.

cassandra.tls.keystore-password

Password for the key store.

cassandra.tls.truststore-password

Password for the trust store.

+
+
+

Querying Cassandra tables#

+

The users table is an example Cassandra table from the Cassandra +Getting Started guide. It can be created along with the example_keyspace +keyspace using Cassandra’s cqlsh (CQL interactive terminal):

+
cqlsh> CREATE KEYSPACE example_keyspace
+   ... WITH REPLICATION = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 };
+cqlsh> USE example_keyspace;
+cqlsh:example_keyspace> CREATE TABLE users (
+              ...   user_id int PRIMARY KEY,
+              ...   fname text,
+              ...   lname text
+              ... );
+
+
+

This table can be described in Trino:

+
DESCRIBE example.example_keyspace.users;
+
+
+
 Column  |  Type   | Extra | Comment
+---------+---------+-------+---------
+ user_id | bigint  |       |
+ fname   | varchar |       |
+ lname   | varchar |       |
+(3 rows)
+
+
+

This table can then be queried in Trino:

+
SELECT * FROM example.example_keyspace.users;
+
+
+
+
+

Type mapping#

+

Because Trino and Cassandra each support types that the other does not, this +connector modifies some types when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction.

+
+

Cassandra type to Trino type mapping#

+

The connector maps Cassandra types to the corresponding Trino types according to +the following table:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Cassandra type to Trino type mapping#

Cassandra type

Trino type

Notes

BOOLEAN

BOOLEAN

TINYINT

TINYINT

SMALLINT

SMALLINT

INT

INTEGER

BIGINT

BIGINT

FLOAT

REAL

DOUBLE

DOUBLE

DECIMAL

DOUBLE

ASCII

VARCHAR

US-ASCII character string

TEXT

VARCHAR

UTF-8 encoded string

VARCHAR

VARCHAR

UTF-8 encoded string

VARINT

VARCHAR

Arbitrary-precision integer

BLOB

VARBINARY

DATE

DATE

TIME

TIME(9)

TIMESTAMP

TIMESTAMP(3) WITH TIME ZONE

LIST<?>

VARCHAR

MAP<?, ?>

VARCHAR

SET<?>

VARCHAR

TUPLE

ROW with anonymous fields

UDT

ROW with field names

INET

IPADDRESS

UUID

UUID

TIMEUUID

UUID

+

No other types are supported.

+
+
+

Trino type to Cassandra type mapping#

+

The connector maps Trino types to the corresponding Cassandra types according to +the following table:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Trino type to Cassandra type mapping#

Trino type

Cassandra type

Notes

BOOLEAN

BOOLEAN

TINYINT

TINYINT

SMALLINT

SMALLINT

INTEGER

INT

BIGINT

BIGINT

REAL

FLOAT

DOUBLE

DOUBLE

VARCHAR

TEXT

DATE

DATE

TIMESTAMP(3) WITH TIME ZONE

TIMESTAMP

IPADDRESS

INET

UUID

UUID

+

No other types are supported.

+
+
+
+

Partition key types#

+

Partition keys can only be of the following types:

+
    +
  • ASCII

  • +
  • TEXT

  • +
  • VARCHAR

  • +
  • BIGINT

  • +
  • BOOLEAN

  • +
  • DOUBLE

  • +
  • INET

  • +
  • INT

  • +
  • FLOAT

  • +
  • DECIMAL

  • +
  • TIMESTAMP

  • +
  • UUID

  • +
  • TIMEUUID

  • +
+
+
+

Limitations#

+
    +
  • Queries without filters containing the partition key result in fetching all partitions. +This causes a full scan of the entire data set, and is therefore much slower compared to a similar +query with a partition key as a filter.

  • +
  • IN list filters are only allowed on index (that is, partition key or clustering key) columns.

  • +
  • Range (< or > and BETWEEN) filters can be applied only to the partition keys.

  • +
+
+
+

SQL support#

+

The connector provides read and write access to data and metadata in +the Cassandra database. In addition to the globally available and read operation +statements, the connector supports the following features:

+ +
+
+

Table functions#

+

The connector provides specific table functions to +access Cassandra. +.. _cassandra-query-function:

+
+

query(varchar) -> table#

+

The query function allows you to query the underlying Cassandra directly. It +requires syntax native to Cassandra, because the full query is pushed down and +processed by Cassandra. This can be useful for accessing native features which are +not available in Trino or for improving query performance in situations where +running a query natively may be faster.

+
+

Note

+

The query engine does not preserve the order of the results of this +function. If the passed query contains an ORDER BY clause, the +function result may not be ordered as expected.

+
+

As a simple example, to select an entire table:

+
SELECT
+  *
+FROM
+  TABLE(
+    example.system.query(
+      query => 'SELECT
+        *
+      FROM
+        tpch.nation'
+    )
+  );
+
+
+
+
+

DROP TABLE#

+

By default, DROP TABLE operations are disabled on Cassandra catalogs. To +enable DROP TABLE, set the cassandra.allow-drop-table catalog +configuration property to true:

+
cassandra.allow-drop-table=true
+
+
+
+
+

SQL delete limitation#

+

DELETE is only supported if the WHERE clause matches entire partitions.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/clickhouse.html b/430/connector/clickhouse.html new file mode 100644 index 000000000..be3bc7750 --- /dev/null +++ b/430/connector/clickhouse.html @@ -0,0 +1,1468 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ClickHouse connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

ClickHouse connector#

+

The ClickHouse connector allows querying tables in an external +ClickHouse server. This can be used to +query data in the databases on that server, or combine it with other data +from different catalogs accessing ClickHouse or any other supported data source.

+
+

Requirements#

+

To connect to a ClickHouse server, you need:

+
    +
  • ClickHouse (version 21.8 or higher) or Altinity (version 20.8 or higher).

  • +
  • Network access from the Trino coordinator and workers to the ClickHouse +server. Port 8123 is the default port.

  • +
+
+
+

Configuration#

+

The connector can query a ClickHouse server. Create a catalog properties file +that specifies the ClickHouse connector by setting the connector.name to +clickhouse.

+

For example, create the file etc/catalog/example.properties. Replace the +connection properties as appropriate for your setup:

+
connector.name=clickhouse
+connection-url=jdbc:clickhouse://host1:8123/
+connection-user=exampleuser
+connection-password=examplepassword
+
+
+

The connection-url defines the connection information and parameters to pass +to the ClickHouse JDBC driver. The supported parameters for the URL are +available in the ClickHouse JDBC driver configuration.

+

The connection-user and connection-password are typically required and +determine the user credentials for the connection, often a service user. You can +use secrets to avoid actual values in the catalog +properties files.

+
+

Connection security#

+

If you have TLS configured with a globally-trusted certificate installed on your +data source, you can enable TLS between your cluster and the data +source by appending a parameter to the JDBC connection string set in the +connection-url catalog configuration property.

+

For example, with version 2.6.4 of the ClickHouse JDBC driver, enable TLS by +appending the ssl=true parameter to the connection-url configuration +property:

+
connection-url=jdbc:clickhouse://host1:8443/?ssl=true
+
+
+

For more information on TLS configuration options, see the Clickhouse JDBC +driver documentation

+
+
+

Data source authentication#

+

The connector can provide credentials for the data source connection +in multiple ways:

+
    +
  • inline, in the connector configuration file

  • +
  • in a separate properties file

  • +
  • in a key store file

  • +
  • as extra credentials set when connecting to Trino

  • +
+

You can use secrets to avoid storing sensitive +values in the catalog properties files.

+

The following table describes configuration properties +for connection credentials:

+ ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

credential-provider.type

Type of the credential provider. Must be one of INLINE, FILE, or +KEYSTORE; defaults to INLINE.

connection-user

Connection user name.

connection-password

Connection password.

user-credential-name

Name of the extra credentials property, whose value to use as the user +name. See extraCredentials in Parameter +reference.

password-credential-name

Name of the extra credentials property, whose value to use as the +password.

connection-credential-file

Location of the properties file where credentials are present. It must +contain the connection-user and connection-password properties.

keystore-file-path

The location of the Java Keystore file, from which to read credentials.

keystore-type

File format of the keystore file, for example JKS or PEM.

keystore-password

Password for the key store.

keystore-user-credential-name

Name of the key store entity to use as the user name.

keystore-user-credential-password

Password for the user name key store entity.

keystore-password-credential-name

Name of the key store entity to use as the password.

keystore-password-credential-password

Password for the password key store entity.

+
+
+

Multiple ClickHouse servers#

+

If you have multiple ClickHouse servers you need to configure one +catalog for each server. To add another catalog:

+
    +
  • Add another properties file to etc/catalog

  • +
  • Save it with a different name that ends in .properties

  • +
+

For example, if you name the property file sales.properties, Trino uses the +configured connector to create a catalog named sales.

+
+
+

General configuration properties#

+

The following table describes general catalog configuration properties for the +connector:

+ +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

Default value

case-insensitive-name-matching

Support case insensitive schema and table names.

false

case-insensitive-name-matching.cache-ttl

This value should be a duration.

1m

case-insensitive-name-matching.config-file

Path to a name mapping configuration file in JSON format that allows +Trino to disambiguate between schemas and tables with similar names in +different cases.

null

case-insensitive-name-matching.config-file.refresh-period

Frequency with which Trino checks the name matching configuration file +for changes. This value should be a duration.

(refresh disabled)

metadata.cache-ttl

The duration for which metadata, including +table and column statistics, is cached.

0s (caching disabled)

metadata.cache-missing

Cache the fact that metadata, including table and column statistics, is +not available

false

metadata.cache-maximum-size

Maximum number of objects stored in the metadata cache

10000

write.batch-size

Maximum number of statements in a batched execution. +Do not change this setting from the default. Non-default values may +negatively impact performance.

1000

dynamic-filtering.enabled

Push down dynamic filters into JDBC queries

true

dynamic-filtering.wait-timeout

Maximum duration for which Trino will wait for dynamic +filters to be collected from the build side of joins before starting a +JDBC query. Using a large timeout can potentially result in more detailed +dynamic filters. However, it can also increase latency for some queries.

20s

+
+
+

Appending query metadata#

+

The optional parameter query.comment-format allows you to configure a SQL +comment that is sent to the datasource with each query. The format of this +comment can contain any characters and the following metadata:

+
    +
  • $QUERY_ID: The identifier of the query.

  • +
  • $USER: The name of the user who submits the query to Trino.

  • +
  • $SOURCE: The identifier of the client tool used to submit the query, for +example trino-cli.

  • +
  • $TRACE_TOKEN: The trace token configured with the client tool.

  • +
+

The comment can provide more context about the query. This additional +information is available in the logs of the datasource. To include environment +variables from the Trino cluster with the comment , use the +${ENV:VARIABLE-NAME} syntax.

+

The following example sets a simple comment that identifies each query sent by +Trino:

+
query.comment-format=Query sent by Trino.
+
+
+

With this configuration, a query such as SELECT * FROM example_table; is +sent to the datasource with the comment appended:

+
SELECT * FROM example_table; /*Query sent by Trino.*/
+
+
+

The following example improves on the preceding example by using metadata:

+
query.comment-format=Query $QUERY_ID sent by user $USER from Trino.
+
+
+

If Jane sent the query with the query identifier +20230622_180528_00000_bkizg, the following comment string is sent to the +datasource:

+
SELECT * FROM example_table; /*Query 20230622_180528_00000_bkizg sent by user Jane from Trino.*/
+
+
+
+

Note

+

Certain JDBC driver settings and logging configurations might cause the +comment to be removed.

+
+
+
+

Domain compaction threshold#

+

Pushing down a large list of predicates to the data source can compromise +performance. Trino compacts large predicates into a simpler range predicate +by default to ensure a balance between performance and predicate pushdown. +If necessary, the threshold for this compaction can be increased to improve +performance when the data source is capable of taking advantage of large +predicates. Increasing this threshold may improve pushdown of large +dynamic filters. +The domain-compaction-threshold catalog configuration property or the +domain_compaction_threshold catalog session property can be used to adjust the default value of +1000 for this threshold.

+
+
+

Procedures#

+
    +
  • system.flush_metadata_cache()

    +

    Flush JDBC metadata caches. For example, the following system call +flushes the metadata caches for all schemas in the example catalog

    +
    USE example.example_schema;
    +CALL system.flush_metadata_cache();
    +
    +
    +
  • +
+
+
+

Case insensitive matching#

+

When case-insensitive-name-matching is set to true, Trino +is able to query non-lowercase schemas and tables by maintaining a mapping of +the lowercase name to the actual name in the remote system. However, if two +schemas and/or tables have names that differ only in case (such as “customers” +and “Customers”) then Trino fails to query them due to ambiguity.

+

In these cases, use the case-insensitive-name-matching.config-file catalog +configuration property to specify a configuration file that maps these remote +schemas/tables to their respective Trino schemas/tables:

+
{
+  "schemas": [
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "mapping": "case_insensitive_1"
+    },
+    {
+      "remoteSchema": "cASEsENSITIVEnAME",
+      "mapping": "case_insensitive_2"
+    }],
+  "tables": [
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "remoteTable": "tablex",
+      "mapping": "table_1"
+    },
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "remoteTable": "TABLEX",
+      "mapping": "table_2"
+    }]
+}
+
+
+

Queries against one of the tables or schemes defined in the mapping +attributes are run against the corresponding remote entity. For example, a query +against tables in the case_insensitive_1 schema is forwarded to the +CaseSensitiveName schema and a query against case_insensitive_2 is forwarded +to the cASEsENSITIVEnAME schema.

+

At the table mapping level, a query on case_insensitive_1.table_1 as +configured above is forwarded to CaseSensitiveName.tablex, and a query on +case_insensitive_1.table_2 is forwarded to CaseSensitiveName.TABLEX.

+

By default, when a change is made to the mapping configuration file, Trino must +be restarted to load the changes. Optionally, you can set the +case-insensitive-name-mapping.refresh-period to have Trino refresh the +properties without requiring a restart:

+
case-insensitive-name-mapping.refresh-period=30s
+
+
+
+
+

Non-transactional INSERT#

+

The connector supports adding rows using INSERT statements. +By default, data insertion is performed by writing data to a temporary table. +You can skip this step to improve performance and write directly to the target +table. Set the insert.non-transactional-insert.enabled catalog property +or the corresponding non_transactional_insert catalog session property to +true.

+

Note that with this property enabled, data can be corrupted in rare cases where +exceptions occur during the insert operation. With transactions disabled, no +rollback can be performed.

+
+
+
+

Querying ClickHouse#

+

The ClickHouse connector provides a schema for every ClickHouse database. +Run SHOW SCHEMAS to see the available ClickHouse databases:

+
SHOW SCHEMAS FROM example;
+
+
+

If you have a ClickHouse database named web, run SHOW TABLES to view the +tables in this database:

+
SHOW TABLES FROM example.web;
+
+
+

Run DESCRIBE or SHOW COLUMNS to list the columns in the clicks table +in the web databases:

+
DESCRIBE example.web.clicks;
+SHOW COLUMNS FROM example.web.clicks;
+
+
+

Run SELECT to access the clicks table in the web database:

+
SELECT * FROM example.web.clicks;
+
+
+
+

Note

+

If you used a different name for your catalog properties file, use +that catalog name instead of example in the above examples.

+
+
+
+

Table properties#

+

Table property usage example:

+
CREATE TABLE default.trino_ck (
+  id int NOT NULL,
+  birthday DATE NOT NULL,
+  name VARCHAR,
+  age BIGINT,
+  logdate DATE NOT NULL
+)
+WITH (
+  engine = 'MergeTree',
+  order_by = ARRAY['id', 'birthday'],
+  partition_by = ARRAY['toYYYYMM(logdate)'],
+  primary_key = ARRAY['id'],
+  sample_by = 'id'
+);
+
+
+

The following are supported ClickHouse table properties from https://clickhouse.tech/docs/en/engines/table-engines/mergetree-family/mergetree/

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Default value

Description

engine

Log

Name and parameters of the engine.

order_by

(none)

Array of columns or expressions to concatenate to create the sorting key. Required if engine is MergeTree.

partition_by

(none)

Array of columns or expressions to use as nested partition keys. Optional.

primary_key

(none)

Array of columns or expressions to concatenate to create the primary key. Optional.

sample_by

(none)

An expression to use for sampling. Optional.

+

Currently the connector only supports Log and MergeTree table engines +in create table statement. ReplicatedMergeTree engine is not yet supported.

+
+
+

Type mapping#

+

Because Trino and ClickHouse each support types that the other does not, this +connector modifies some types when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction.

+
+

ClickHouse type to Trino type mapping#

+

The connector maps ClickHouse types to the corresponding Trino types according +to the following table:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
ClickHouse type to Trino type mapping#

ClickHouse type

Trino type

Notes

Int8

TINYINT

TINYINT, BOOL, BOOLEAN, and INT1 are aliases of Int8

Int16

SMALLINT

SMALLINT and INT2 are aliases of Int16

Int32

INTEGER

INT, INT4, and INTEGER are aliases of Int32

Int64

BIGINT

BIGINT is an alias of Int64

UInt8

SMALLINT

UInt16

INTEGER

UInt32

BIGINT

UInt64

DECIMAL(20,0)

Float32

REAL

FLOAT is an alias of Float32

Float64

DOUBLE

DOUBLE is an alias of Float64

Decimal

DECIMAL

FixedString

VARBINARY

Enabling clickhouse.map-string-as-varchar config property changes the +mapping to VARCHAR

String

VARBINARY

Enabling clickhouse.map-string-as-varchar config property changes the +mapping to VARCHAR

Date

DATE

DateTime[(timezone)]

TIMESTAMP(0) [WITH TIME ZONE]

IPv4

IPADDRESS

IPv6

IPADDRESS

Enum8

VARCHAR

Enum16

VARCHAR

UUID

UUID

+

No other types are supported.

+
+
+

Trino type to ClickHouse type mapping#

+

The connector maps Trino types to the corresponding ClickHouse types according +to the following table:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Trino type to ClickHouse type mapping#

Trino type

ClickHouse type

Notes

BOOLEAN

UInt8

TINYINT

Int8

TINYINT, BOOL, BOOLEAN, and INT1 are aliases of Int8

SMALLINT

Int16

SMALLINT and INT2 are aliases of Int16

INTEGER

Int32

INT, INT4, and INTEGER are aliases of Int32

BIGINT

Int64

BIGINT is an alias of Int64

REAL

Float32

FLOAT is an alias of Float32

DOUBLE

Float64

DOUBLE is an alias of Float64

DECIMAL(p,s)

Decimal(p,s)

VARCHAR

String

CHAR

String

VARBINARY

String

Enabling clickhouse.map-string-as-varchar config property changes the +mapping to VARCHAR

DATE

Date

TIMESTAMP(0)

DateTime

UUID

UUID

+

No other types are supported.

+
+
+

Type mapping configuration properties#

+

The following properties can be used to configure how data types from the +connected data source are mapped to Trino data types and how the metadata is +cached in Trino.

+ +++++ + + + + + + + + + + + + + + + + +

Property name

Description

Default value

unsupported-type-handling

Configure how unsupported column data types are handled:

+
    +
  • IGNORE, column is not accessible.

  • +
  • CONVERT_TO_VARCHAR, column is converted to unbounded VARCHAR.

  • +
+

The respective catalog session property is unsupported_type_handling.

+

IGNORE

jdbc-types-mapped-to-varchar

Allow forced mapping of comma separated lists of data types to convert to +unbounded VARCHAR

+
+
+
+

SQL support#

+

The connector provides read and write access to data and metadata in +a ClickHouse catalog. In addition to the globally available and read operation +statements, the connector supports the following features:

+ +
+

ALTER SCHEMA#

+

The connector supports renaming a schema with the ALTER SCHEMA RENAME +statement. ALTER SCHEMA SET AUTHORIZATION is not supported.

+
+
+
+

Performance#

+

The connector includes a number of performance improvements, detailed in the +following sections.

+
+

Pushdown#

+

The connector supports pushdown for a number of operations:

+ +

Aggregate pushdown for the following functions:

+ +
+

Note

+

The connector performs pushdown where performance may be improved, but in +order to preserve correctness an operation may not be pushed down. When +pushdown of an operation may result in better performance but risks +correctness, the connector prioritizes correctness.

+
+
+

Predicate pushdown support#

+

The connector does not support pushdown of any predicates on columns with +textual types like CHAR or VARCHAR. +This ensures correctness of results since the data source may compare strings +case-insensitively.

+

In the following example, the predicate is not pushed down for either query +since name is a column of type VARCHAR:

+
SELECT * FROM nation WHERE name > 'CANADA';
+SELECT * FROM nation WHERE name = 'CANADA';
+
+
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/delta-lake.html b/430/connector/delta-lake.html new file mode 100644 index 000000000..2e838dc76 --- /dev/null +++ b/430/connector/delta-lake.html @@ -0,0 +1,2066 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Delta Lake connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Delta Lake connector#

+

The Delta Lake connector allows querying data stored in the Delta Lake format, including Databricks Delta Lake. The connector can natively +read the Delta Lake transaction log and thus detect when external systems change +data.

+
+

Requirements#

+

To connect to Databricks Delta Lake, you need:

+
    +
  • Tables written by Databricks Runtime 7.3 LTS, 9.1 LTS, 10.4 LTS, 11.3 LTS, +12.2 LTS and 13.3 LTS are supported.

  • +
  • Deployments using AWS, HDFS, Azure Storage, and Google Cloud Storage (GCS) are +fully supported.

  • +
  • Network access from the coordinator and workers to the Delta Lake storage.

  • +
  • Access to the Hive metastore service (HMS) of Delta Lake or a separate HMS, +or a Glue metastore.

  • +
  • Network access to the HMS from the coordinator and workers. Port 9083 is the +default port for the Thrift protocol used by the HMS.

  • +
  • Data files stored in the Parquet file format. These can be configured using +file format configuration properties per +catalog.

  • +
+
+
+

General configuration#

+

To configure the Delta Lake connector, create a catalog properties file +etc/catalog/example.properties that references the delta_lake +connector and defines a metastore. You must configure a metastore for table +metadata. If you are using a Hive metastore, +hive.metastore.uri must be configured:

+
connector.name=delta_lake
+hive.metastore.uri=thrift://example.net:9083
+
+
+

If you are using AWS Glue as your metastore, you +must instead set hive.metastore to glue:

+
connector.name=delta_lake
+hive.metastore=glue
+
+
+

Each metastore type has specific configuration properties along with +general metastore configuration properties.

+

The connector recognizes Delta Lake tables created in the metastore by the Databricks +runtime. If non-Delta Lake tables are present in the metastore as well, they are not +visible to the connector.

+

To configure access to S3 and S3-compatible storage, Azure storage, and others, +consult the appropriate section of the Hive documentation:

+ +
+

Delta Lake general configuration properties#

+

The following configuration properties are all using reasonable, tested default +values. Typical usage does not require you to configure them.

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Delta Lake configuration properties#

Property name

Description

Default

delta.metadata.cache-ttl

Frequency of checks for metadata updates equivalent to transactions to +update the metadata cache specified in duration.

5m

delta.metadata.cache-size

The maximum number of Delta table metadata entries to cache.

1000

delta.metadata.live-files.cache-size

Amount of memory allocated for caching information about files. Must +be specified in data size values such as 64MB. +Default is calculated to 10% of the maximum memory allocated to the JVM.

delta.metadata.live-files.cache-ttl

Caching duration for active files that correspond to the Delta Lake +tables.

30m

delta.compression-codec

The compression codec to be used when writing new data files. +Possible values are:

+
    +
  • NONE

  • +
  • SNAPPY

  • +
  • ZSTD

  • +
  • GZIP

  • +
+

The equivalent catalog session property is compression_codec.

+

SNAPPY

delta.max-partitions-per-writer

Maximum number of partitions per writer.

100

delta.hide-non-delta-lake-tables

Hide information about tables that are not managed by Delta Lake. Hiding +only applies to tables with the metadata managed in a Glue catalog, and +does not apply to usage with a Hive metastore service.

false

delta.enable-non-concurrent-writes

Enable write support for all +supported file systems. Specifically, take note of the warning about +concurrency and checkpoints.

false

delta.default-checkpoint-writing-interval

Default integer count to write transaction log checkpoint entries. If +the value is set to N, then checkpoints are written after every Nth +statement performing table writes. The value can be overridden for a +specific table with the checkpoint_interval table property.

10

delta.hive-catalog-name

Name of the catalog to which SELECT queries are redirected when a +Hive table is detected.

delta.checkpoint-row-statistics-writing.enabled

Enable writing row statistics to checkpoint files.

true

delta.dynamic-filtering.wait-timeout

Duration to wait for completion of dynamic filtering during split generation. +The equivalent catalog session property is +dynamic_filtering_wait_timeout.

delta.table-statistics-enabled

Enables Table statistics for +performance improvements. The equivalent catalog session property +is statistics_enabled.

true

delta.extended-statistics.enabled

Enable statistics collection with ANALYZE and +use of extended statistics. The equivalent catalog session property +is extended_statistics_enabled.

true

delta.extended-statistics.collect-on-write

Enable collection of extended statistics for write operations. +The equivalent catalog session property is +extended_statistics_collect_on_write.

true

delta.per-transaction-metastore-cache-maximum-size

Maximum number of metastore data objects per transaction in +the Hive metastore cache.

1000

delta.delete-schema-locations-fallback

Whether schema locations are deleted when Trino can’t +determine whether they contain external files.

false

delta.parquet.time-zone

Time zone for Parquet read and write.

JVM default

delta.target-max-file-size

Target maximum size of written files; the actual size could be larger. +The equivalent catalog session property is target_max_file_size.

1GB

delta.unique-table-location

Use randomized, unique table locations.

true

delta.register-table-procedure.enabled

Enable to allow users to call the register_table procedure.

false

delta.vacuum.min-retention

Minimum retention threshold for the files taken into account +for removal by the VACUUM procedure. +The equivalent catalog session property is +vacuum_min_retention.

7 DAYS

+
+
+

Catalog session properties#

+

The following table describes catalog session properties supported by the Delta Lake connector:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Catalog session properties#

Property name

Description

Default

parquet_max_read_block_size

The maximum block size used when reading Parquet files.

16MB

parquet_writer_block_size

The maximum block size created by the Parquet writer.

128MB

parquet_writer_page_size

The maximum page size created by the Parquet writer.

1MB

parquet_writer_batch_size

Maximum number of rows processed by the Parquet writer in a batch.

10000

projection_pushdown_enabled

Read only projected fields from row columns while performing SELECT queries

true

+
+
+
+

Type mapping#

+

Because Trino and Delta Lake each support types that the other does not, this +connector modifies some types when reading or +writing data. Data types might not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction.

+

See the Delta Transaction Log specification +for more information about supported data types in the Delta Lake table format +specification.

+
+

Delta Lake to Trino type mapping#

+

The connector maps Delta Lake types to the corresponding Trino types following +this table:

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Delta Lake to Trino type mapping#

Delta Lake type

Trino type

BOOLEAN

BOOLEAN

INTEGER

INTEGER

BYTE

TINYINT

SHORT

SMALLINT

LONG

BIGINT

FLOAT

REAL

DOUBLE

DOUBLE

DECIMAL(p,s)

DECIMAL(p,s)

STRING

VARCHAR

BINARY

VARBINARY

DATE

DATE

TIMESTAMPNTZ (TIMESTAMP_NTZ)

TIMESTAMP(6)

TIMESTAMP

TIMESTAMP(3) WITH TIME ZONE

ARRAY

ARRAY

MAP

MAP

STRUCT(...)

ROW(...)

+

No other types are supported.

+
+
+

Trino to Delta Lake type mapping#

+

The connector maps Trino types to the corresponding Delta Lake types following +this table:

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Trino to Delta Lake type mapping#

Trino type

Delta Lake type

BOOLEAN

BOOLEAN

INTEGER

INTEGER

TINYINT

BYTE

SMALLINT

SHORT

BIGINT

LONG

REAL

FLOAT

DOUBLE

DOUBLE

DECIMAL(p,s)

DECIMAL(p,s)

VARCHAR

STRING

VARBINARY

BINARY

DATE

DATE

TIMESTAMP

TIMESTAMPNTZ (TIMESTAMP_NTZ)

TIMESTAMP(3) WITH TIME ZONE

TIMESTAMP

ARRAY

ARRAY

MAP

MAP

ROW(...)

STRUCT(...)

+

No other types are supported.

+
+
+
+

Security#

+

The Delta Lake connector allows you to choose one of several means of providing +authorization at the catalog level. You can select a different type of +authorization check in different Delta Lake catalog files.

+
+

Authorization checks#

+

Enable authorization checks for the connector by setting the delta.security +property in the catalog properties file. This property must be one of the +security values in the following table:

+ + ++++ + + + + + + + + + + + + + + + + + + + +
Delta Lake security values#

Property value

Description

ALLOW_ALL (default value)

No authorization checks are enforced.

SYSTEM

The connector relies on system-level access control.

READ_ONLY

Operations that read data or metadata, such as SELECT are +permitted. No operations that write data or metadata, such as +CREATE TABLE, INSERT, or DELETE are +allowed.

FILE

Authorization checks are enforced using a catalog-level access control +configuration file whose path is specified in the security.config-file +catalog configuration property. See +Catalog-level access control files for information on the +authorization configuration file.

+
+
+
+

SQL support#

+

The connector provides read and write access to data and metadata in +Delta Lake. In addition to the globally available and read operation +statements, the connector supports the following features:

+ +
+

Procedures#

+

Use the CALL statement to perform data manipulation or +administrative tasks. Procedures are available in the system schema of each +catalog. The following code snippet displays how to call the +example_procedure in the examplecatalog catalog:

+
CALL examplecatalog.system.example_procedure()
+
+
+
+

Register table#

+

The connector can register table into the metastore with existing transaction +logs and data files.

+

The system.register_table procedure allows the caller to register an +existing Delta Lake table in the metastore, using its existing transaction logs +and data files:

+
CALL example.system.register_table(schema_name => 'testdb', table_name => 'customer_orders', table_location => 's3://my-bucket/a/path')
+
+
+

To prevent unauthorized users from accessing data, this procedure is disabled by +default. The procedure is enabled only when +delta.register-table-procedure.enabled is set to true.

+
+
+

Unregister table#

+

The connector can unregister existing Delta Lake tables from the metastore.

+

The procedure system.unregister_table allows the caller to unregister an +existing Delta Lake table from the metastores without deleting the data:

+
CALL example.system.unregister_table(schema_name => 'testdb', table_name => 'customer_orders')
+
+
+
+
+

Flush metadata cache#

+
    +
  • system.flush_metadata_cache()

    +

    Flushes all metadata caches.

    +
  • +
  • system.flush_metadata_cache(schema_name => ..., table_name => ...)

    +

    Flushes metadata cache entries of a specific table. +Procedure requires passing named parameters.

    +
  • +
+
+
+

VACUUM#

+

The VACUUM procedure removes all old files that are not in the transaction +log, as well as files that are not needed to read table snapshots newer than the +current time minus the retention period defined by the retention period +parameter.

+

Users with INSERT and DELETE permissions on a table can run VACUUM +as follows:

+
CALL example.system.vacuum('exampleschemaname', 'exampletablename', '7d');
+
+
+

All parameters are required and must be presented in the following order:

+
    +
  • Schema name

  • +
  • Table name

  • +
  • Retention period

  • +
+

The delta.vacuum.min-retention configuration property provides a safety +measure to ensure that files are retained as expected. The minimum value for +this property is 0s. There is a minimum retention session property as well, +vacuum_min_retention.

+
+
+
+

Data management#

+

You can use the connector to INSERT, DELETE, +UPDATE, and MERGE data in Delta Lake tables.

+

Write operations are supported for tables stored on the following systems:

+
    +
  • Azure ADLS Gen2, Google Cloud Storage

    +

    Writes to the Azure ADLS Gen2 and Google Cloud Storage are +enabled by default. Trino detects write collisions on these storage systems +when writing from multiple Trino clusters, or from other query engines.

    +
  • +
  • S3 and S3-compatible storage

    +

    Writes to Amazon S3 and S3-compatible storage must be enabled +with the delta.enable-non-concurrent-writes property. Writes to S3 can +safely be made from multiple Trino clusters; however, write collisions are not +detected when writing concurrently from other Delta Lake engines. You must +make sure that no concurrent data modifications are run to avoid data +corruption.

    +
  • +
+
+
+

Schema and table management#

+

The Schema and table management functionality includes support for:

+ +

The connector supports creating schemas. You can create a schema with or without +a specified location.

+

You can create a schema with the CREATE SCHEMA statement and the +location schema property. Tables in this schema are located in a +subdirectory under the schema location. Data files for tables in this schema +using the default location are cleaned up if the table is dropped:

+
CREATE SCHEMA example.example_schema
+WITH (location = 's3://my-bucket/a/path');
+
+
+

Optionally, the location can be omitted. Tables in this schema must have a +location included when you create them. The data files for these tables are not +removed if the table is dropped:

+
CREATE SCHEMA example.example_schema;
+
+
+

When Delta Lake tables exist in storage but not in the metastore, Trino can be +used to register the tables:

+
CREATE TABLE example.default.example_table (
+  dummy BIGINT
+)
+WITH (
+  location = '...'
+)
+
+
+

Columns listed in the DDL, such as dummy in the preceding example, are +ignored. The table schema is read from the transaction log instead. If the +schema is changed by an external system, Trino automatically uses the new +schema.

+
+

Warning

+

Using CREATE TABLE with an existing table content is deprecated, instead +use the system.register_table procedure. The CREATE TABLE ... WITH (location=...) syntax can be temporarily re-enabled using the +delta.legacy-create-table-with-existing-location.enabled catalog +configuration property or +legacy_create_table_with_existing_location_enabled catalog session +property.

+
+

If the specified location does not already contain a Delta table, the connector +automatically writes the initial transaction log entries and registers the table +in the metastore. As a result, any Databricks engine can write to the table:

+
CREATE TABLE example.default.new_table (id BIGINT, address VARCHAR);
+
+
+

The Delta Lake connector also supports creating tables using the CREATE TABLE AS syntax.

+

The connector supports the following ALTER TABLE statements.

+
+

ALTER TABLE EXECUTE#

+

The connector supports the following commands for use with ALTER TABLE EXECUTE.

+
+
optimize#
+

The optimize command is used for rewriting the content of the specified +table so that it is merged into fewer but larger files. If the table is +partitioned, the data compaction acts separately on each partition selected for +optimization. This operation improves read performance.

+

All files with a size below the optional file_size_threshold parameter +(default value for the threshold is 100MB) are merged:

+
ALTER TABLE test_table EXECUTE optimize
+
+
+

The following statement merges files in a table that are +under 10 megabytes in size:

+
ALTER TABLE test_table EXECUTE optimize(file_size_threshold => '10MB')
+
+
+

You can use a WHERE clause with the columns used to partition the table +to filter which partitions are optimized:

+
ALTER TABLE test_partitioned_table EXECUTE optimize
+WHERE partition_key = 1
+
+
+
+
+
+

ALTER TABLE RENAME TO#

+

The connector only supports the ALTER TABLE RENAME TO statement when met with +one of the following conditions:

+
    +
  • The table type is external.

  • +
  • The table is backed by a metastore that does not perform object storage +operations, for example, AWS Glue or Thrift.

  • +
+
+
+

Table properties#

+

The following table properties are available for use:

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + +
Delta Lake table properties#

Property name

Description

location

File system location URI for the table.

partitioned_by

Set partition columns.

checkpoint_interval

Set the checkpoint interval in number of table writes.

change_data_feed_enabled

Enables storing change data feed entries.

column_mapping_mode

Column mapping mode. Possible values are:

+
    +
  • ID

  • +
  • NAME

  • +
  • NONE

  • +
+

Defaults to NONE.

+
+

The following example uses all available table properties:

+
CREATE TABLE example.default.example_partitioned_table
+WITH (
+  location = 's3://my-bucket/a/path',
+  partitioned_by = ARRAY['regionkey'],
+  checkpoint_interval = 5,
+  change_data_feed_enabled = false,
+  column_mapping_mode = 'name'
+)
+AS SELECT name, comment, regionkey FROM tpch.tiny.nation;
+
+
+
+
+

Metadata tables#

+

The connector exposes several metadata tables for each Delta Lake table. +These metadata tables contain information about the internal structure +of the Delta Lake table. You can query each metadata table by appending the +metadata table name to the table name:

+
SELECT * FROM "test_table$history"
+
+
+
+
$history table#
+

The $history table provides a log of the metadata changes performed on +the Delta Lake table.

+

You can retrieve the changelog of the Delta Lake table test_table +by using the following query:

+
SELECT * FROM "test_table$history"
+
+
+
 version |               timestamp               | user_id | user_name |  operation   |         operation_parameters          |                 cluster_id      | read_version |  isolation_level  | is_blind_append
+---------+---------------------------------------+---------+-----------+--------------+---------------------------------------+---------------------------------+--------------+-------------------+----------------
+       2 | 2023-01-19 07:40:54.684 Europe/Vienna | trino   | trino     | WRITE        | {queryId=20230119_064054_00008_4vq5t} | trino-406-trino-coordinator     |            2 | WriteSerializable | true
+       1 | 2023-01-19 07:40:41.373 Europe/Vienna | trino   | trino     | ADD COLUMNS  | {queryId=20230119_064041_00007_4vq5t} | trino-406-trino-coordinator     |            0 | WriteSerializable | true
+       0 | 2023-01-19 07:40:10.497 Europe/Vienna | trino   | trino     | CREATE TABLE | {queryId=20230119_064010_00005_4vq5t} | trino-406-trino-coordinator     |            0 | WriteSerializable | true
+
+
+

The output of the query has the following history columns:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
History columns#

Name

Type

Description

version

BIGINT

The version of the table corresponding to the operation

timestamp

TIMESTAMP(3) WITH TIME ZONE

The time when the table version became active

user_id

VARCHAR

The identifier for the user which performed the operation

user_name

VARCHAR

The username for the user which performed the operation

operation

VARCHAR

The name of the operation performed on the table

operation_parameters

map(VARCHAR, VARCHAR)

Parameters of the operation

cluster_id

VARCHAR

The ID of the cluster which ran the operation

read_version

BIGINT

The version of the table which was read in order to perform the operation

isolation_level

VARCHAR

The level of isolation used to perform the operation

is_blind_append

BOOLEAN

Whether or not the operation appended data

+
+
+
$properties table#
+

The $properties table provides access to Delta Lake table configuration, +table features and table properties. The table rows are key/value pairs.

+

You can retrieve the properties of the Delta +table test_table by using the following query:

+
SELECT * FROM "test_table$properties"
+
+
+
 key                        | value           |
+----------------------------+-----------------+
+delta.minReaderVersion      | 1               |
+delta.minWriterVersion      | 4               |
+delta.columnMapping.mode    | name            |
+delta.feature.columnMapping | supported       |
+
+
+
+
+
+

Metadata columns#

+

In addition to the defined columns, the Delta Lake connector automatically +exposes metadata in a number of hidden columns in each table. You can use these +columns in your SQL statements like any other column, e.g., they can be selected +directly or used in conditional statements.

+
    +
  • +
    $path

    Full file system path name of the file for this row.

    +
    +
    +
  • +
  • +
    $file_modified_time

    Date and time of the last modification of the file for this row.

    +
    +
    +
  • +
  • +
    $file_size

    Size of the file for this row.

    +
    +
    +
  • +
+
+
+
+
+

Fault-tolerant execution support#

+

The connector supports Fault-tolerant execution of query +processing. Read and write operations are both supported with any retry policy.

+
+
+

Table functions#

+

The connector provides the following table functions:

+
+

table_changes#

+

Allows reading Change Data Feed (CDF) entries to expose row-level changes +between two versions of a Delta Lake table. When the change_data_feed_enabled +table property is set to true on a specific Delta Lake table, +the connector records change events for all data changes on the table. +This is how these changes can be read:

+
SELECT
+  *
+FROM
+  TABLE(
+    system.table_changes(
+      schema_name => 'test_schema',
+      table_name => 'tableName',
+      since_version => 0
+    )
+  );
+
+
+

schema_name - type VARCHAR, required, name of the schema for which the function is called

+

table_name - type VARCHAR, required, name of the table for which the function is called

+

since_version - type BIGINT, optional, version from which changes are shown, exclusive

+

In addition to returning the columns present in the table, the function +returns the following values for each change event:

+
    +
  • +
    _change_type

    Gives the type of change that occurred. Possible values are insert, +delete, update_preimage and update_postimage.

    +
    +
    +
  • +
  • +
    _commit_version

    Shows the table version for which the change occurred.

    +
    +
    +
  • +
  • +
    _commit_timestamp

    Represents the timestamp for the commit in which the specified change happened.

    +
    +
    +
  • +
+

This is how it would be normally used:

+

Create table:

+
CREATE TABLE test_schema.pages (page_url VARCHAR, domain VARCHAR, views INTEGER)
+    WITH (change_data_feed_enabled = true);
+
+
+

Insert data:

+
INSERT INTO test_schema.pages
+    VALUES
+        ('url1', 'domain1', 1),
+        ('url2', 'domain2', 2),
+        ('url3', 'domain1', 3);
+INSERT INTO test_schema.pages
+    VALUES
+        ('url4', 'domain1', 400),
+        ('url5', 'domain2', 500),
+        ('url6', 'domain3', 2);
+
+
+

Update data:

+
UPDATE test_schema.pages
+    SET domain = 'domain4'
+    WHERE views = 2;
+
+
+

Select changes:

+
SELECT
+  *
+FROM
+  TABLE(
+    system.table_changes(
+      schema_name => 'test_schema',
+      table_name => 'pages',
+      since_version => 1
+    )
+  )
+ORDER BY _commit_version ASC;
+
+
+

The preceding sequence of SQL statements returns the following result:

+
page_url    |     domain     |    views    |    _change_type     |    _commit_version    |    _commit_timestamp
+url4        |     domain1    |    400      |    insert           |     2                 |    2023-03-10T21:22:23.000+0000
+url5        |     domain2    |    500      |    insert           |     2                 |    2023-03-10T21:22:23.000+0000
+url6        |     domain3    |    2        |    insert           |     2                 |    2023-03-10T21:22:23.000+0000
+url2        |     domain2    |    2        |    update_preimage  |     3                 |    2023-03-10T22:23:24.000+0000
+url2        |     domain4    |    2        |    update_postimage |     3                 |    2023-03-10T22:23:24.000+0000
+url6        |     domain3    |    2        |    update_preimage  |     3                 |    2023-03-10T22:23:24.000+0000
+url6        |     domain4    |    2        |    update_postimage |     3                 |    2023-03-10T22:23:24.000+0000
+
+
+

The output shows what changes happen in which version. +For example in version 3 two rows were modified, first one changed from +('url2', 'domain2', 2) into ('url2', 'domain4', 2) and the second from +('url6', 'domain2', 2) into ('url6', 'domain4', 2).

+

If since_version is not provided the function produces change events +starting from when the table was created.

+
SELECT
+  *
+FROM
+  TABLE(
+    system.table_changes(
+      schema_name => 'test_schema',
+      table_name => 'pages'
+    )
+  )
+ORDER BY _commit_version ASC;
+
+
+

The preceding SQL statement returns the following result:

+
page_url    |     domain     |    views    |    _change_type     |    _commit_version    |    _commit_timestamp
+url1        |     domain1    |    1        |    insert           |     1                 |    2023-03-10T20:21:22.000+0000
+url2        |     domain2    |    2        |    insert           |     1                 |    2023-03-10T20:21:22.000+0000
+url3        |     domain1    |    3        |    insert           |     1                 |    2023-03-10T20:21:22.000+0000
+url4        |     domain1    |    400      |    insert           |     2                 |    2023-03-10T21:22:23.000+0000
+url5        |     domain2    |    500      |    insert           |     2                 |    2023-03-10T21:22:23.000+0000
+url6        |     domain3    |    2        |    insert           |     2                 |    2023-03-10T21:22:23.000+0000
+url2        |     domain2    |    2        |    update_preimage  |     3                 |    2023-03-10T22:23:24.000+0000
+url2        |     domain4    |    2        |    update_postimage |     3                 |    2023-03-10T22:23:24.000+0000
+url6        |     domain3    |    2        |    update_preimage  |     3                 |    2023-03-10T22:23:24.000+0000
+url6        |     domain4    |    2        |    update_postimage |     3                 |    2023-03-10T22:23:24.000+0000
+
+
+

You can see changes that occurred at version 1 as three inserts. They are +not visible in the previous statement when since_version value was set to 1.

+
+
+
+

Performance#

+

The connector includes a number of performance improvements detailed in the +following sections:

+ +
+

Table statistics#

+

Use ANALYZE statements in Trino to populate data size and +number of distinct values (NDV) extended table statistics in Delta Lake. +The minimum value, maximum value, value count, and null value count +statistics are computed on the fly out of the transaction log of the +Delta Lake table. The cost-based optimizer then uses these statistics to improve +query performance.

+

Extended statistics enable a broader set of optimizations, including join +reordering. The controlling catalog property delta.table-statistics-enabled +is enabled by default. The equivalent catalog session property is statistics_enabled.

+

Each ANALYZE statement updates the table statistics incrementally, so only +the data changed since the last ANALYZE is counted. The table statistics are +not automatically updated by write operations such as INSERT, UPDATE, +and DELETE. You must manually run ANALYZE again to update the table +statistics.

+

To collect statistics for a table, execute the following statement:

+
ANALYZE table_schema.table_name;
+
+
+

To recalculate from scratch the statistics for the table use additional parameter mode:

+
+

ANALYZE table_schema.table_name WITH(mode = ‘full_refresh’);

+
+

There are two modes available full_refresh and incremental. +The procedure use incremental by default.

+

To gain the most benefit from cost-based optimizations, run periodic ANALYZE +statements on every large table that is frequently queried.

+
+

Fine-tuning#

+

The files_modified_after property is useful if you want to run the +ANALYZE statement on a table that was previously analyzed. You can use it to +limit the amount of data used to generate the table statistics:

+
ANALYZE example_table WITH(files_modified_after = TIMESTAMP '2021-08-23
+16:43:01.321 Z')
+
+
+

As a result, only files newer than the specified time stamp are used in the +analysis.

+

You can also specify a set or subset of columns to analyze using the columns +property:

+
ANALYZE example_table WITH(columns = ARRAY['nationkey', 'regionkey'])
+
+
+

To run ANALYZE with columns more than once, the next ANALYZE must +run on the same set or a subset of the original columns used.

+

To broaden the set of columns, drop the statistics and reanalyze the table.

+
+
+

Disable and drop extended statistics#

+

You can disable extended statistics with the catalog configuration property +delta.extended-statistics.enabled set to false. Alternatively, you can +disable it for a session, with the catalog session property extended_statistics_enabled set to false.

+

If a table is changed with many delete and update operation, calling ANALYZE +does not result in accurate statistics. To correct the statistics, you have to +drop the extended statistics and analyze the table again.

+

Use the system.drop_extended_stats procedure in the catalog to drop the +extended statistics for a specified table in a specified schema:

+
CALL example.system.drop_extended_stats('example_schema', 'example_table')
+
+
+
+
+
+

Memory usage#

+

The Delta Lake connector is memory intensive and the amount of required memory +grows with the size of Delta Lake transaction logs of any accessed tables. It is +important to take that into account when provisioning the coordinator.

+

You must decrease memory usage by keeping the number of active data files in +the table low by regularly running OPTIMIZE and VACUUM in Delta Lake.

+
+

Memory monitoring#

+

When using the Delta Lake connector, you must monitor memory usage on the +coordinator. Specifically, monitor JVM heap utilization using standard tools as +part of routine operation of the cluster.

+

A good proxy for memory usage is the cache utilization of Delta Lake caches. It +is exposed by the connector with the +plugin.deltalake.transactionlog:name=<catalog-name>,type=transactionlogaccess +JMX bean.

+

You can access it with any standard monitoring software with JMX support, or use +the JMX connector with the following query:

+
SELECT * FROM jmx.current."*.plugin.deltalake.transactionlog:name=<catalog-name>,type=transactionlogaccess"
+
+
+

Following is an example result:

+
datafilemetadatacachestats.hitrate      | 0.97
+datafilemetadatacachestats.missrate     | 0.03
+datafilemetadatacachestats.requestcount | 3232
+metadatacachestats.hitrate              | 0.98
+metadatacachestats.missrate             | 0.02
+metadatacachestats.requestcount         | 6783
+node                                    | trino-master
+object_name                             | io.trino.plugin.deltalake.transactionlog:type=TransactionLogAccess,name=delta
+
+
+

In a healthy system, both datafilemetadatacachestats.hitrate and +metadatacachestats.hitrate are close to 1.0.

+
+
+
+

Table redirection#

+

Trino offers the possibility to transparently redirect operations on an existing +table to the appropriate catalog based on the format of the table and catalog configuration.

+

In the context of connectors which depend on a metastore service +(for example, Hive connector, Iceberg connector and Delta Lake connector), +the metastore (Hive metastore service, AWS Glue Data Catalog) +can be used to accustom tables with different table formats. +Therefore, a metastore database can hold a variety of tables with different table formats.

+

As a concrete example, let’s use the following +simple scenario which makes use of table redirection:

+
USE example.example_schema;
+
+EXPLAIN SELECT * FROM example_table;
+
+
+
                               Query Plan
+-------------------------------------------------------------------------
+Fragment 0 [SOURCE]
+     ...
+     Output[columnNames = [...]]
+     │   ...
+     └─ TableScan[table = another_catalog:example_schema:example_table]
+            ...
+
+
+

The output of the EXPLAIN statement points out the actual +catalog which is handling the SELECT query over the table example_table.

+

The table redirection functionality works also when using +fully qualified names for the tables:

+
EXPLAIN SELECT * FROM example.example_schema.example_table;
+
+
+
                               Query Plan
+-------------------------------------------------------------------------
+Fragment 0 [SOURCE]
+     ...
+     Output[columnNames = [...]]
+     │   ...
+     └─ TableScan[table = another_catalog:example_schema:example_table]
+            ...
+
+
+

Trino offers table redirection support for the following operations:

+ +

Trino does not offer view redirection support.

+

The connector supports redirection from Delta Lake tables to Hive tables +with the delta.hive-catalog-name catalog configuration property.

+
+
+

Performance tuning configuration properties#

+

The following table describes performance tuning catalog properties specific to +the Delta Lake connector.

+
+

Warning

+

Performance tuning configuration properties are considered expert-level +features. Altering these properties from their default values is likely to +cause instability and performance degradation. It is strongly suggested that +you use them only to address non-trivial performance issues, and that you +keep a backup of the original values if you change them.

+
+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Delta Lake performance tuning configuration properties#

Property name

Description

Default

delta.domain-compaction-threshold

Minimum size of query predicates above which Trino compacts the +predicates. Pushing a large list of predicates down to the data source +can compromise performance. For optimization in that situation, Trino +can compact the large predicates. If necessary, adjust the threshold to +ensure a balance between performance and predicate pushdown.

1000

delta.max-outstanding-splits

The target number of buffered splits for each table scan in a query, +before the scheduler tries to pause.

1000

delta.max-splits-per-second

Sets the maximum number of splits used per second to access underlying +storage. Reduce this number if your limit is routinely exceeded, based +on your filesystem limits. This is set to the absolute maximum value, +which results in Trino maximizing the parallelization of data access +by default. Attempting to set it higher results in Trino not being +able to start.

Integer.MAX_VALUE

delta.max-initial-splits

For each query, the coordinator assigns file sections to read first +at the initial-split-size until the max-initial-splits is +reached. Then it starts issuing reads of the max-split-size size.

200

delta.max-initial-split-size

Sets the initial data size for a single read section +assigned to a worker until max-initial-splits have been processed. +You can also use the corresponding catalog session property +<catalog-name>.max_initial_split_size.

32MB

delta.max-split-size

Sets the largest data size for a single read section +assigned to a worker after max-initial-splits have been processed. +You can also use the corresponding catalog session property +<catalog-name>.max_split_size.

64MB

delta.minimum-assigned-split-weight

A decimal value in the range (0, 1] used as a minimum for weights +assigned to each split. A low value might improve performance on tables +with small files. A higher value might improve performance for queries +with highly skewed aggregations or joins.

0.05

delta.projection-pushdown-enabled

Read only projected fields from row columns while performing SELECT queries

true

delta.query-partition-filter-required

Set to true to force a query to use a partition filter. You can use +the query_partition_filter_required catalog session property for +temporary, catalog specific use.

false

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/druid.html b/430/connector/druid.html new file mode 100644 index 000000000..815ee513f --- /dev/null +++ b/430/connector/druid.html @@ -0,0 +1,1162 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Druid connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Druid connector#

+

The Druid connector allows querying an Apache Druid +database from Trino.

+
+

Requirements#

+

To connect to Druid, you need:

+
    +
  • Druid version 0.18.0 or higher.

  • +
  • Network access from the Trino coordinator and workers to your Druid broker. +Port 8082 is the default port.

  • +
+
+
+

Configuration#

+

Create a catalog properties file that specifies the Druid connector by setting +the connector.name to druid and configuring the connection-url with +the JDBC string to connect to Druid.

+

For example, to access a database as example, create the file +etc/catalog/example.properties. Replace BROKER:8082 with the correct +host and port of your Druid broker.

+
connector.name=druid
+connection-url=jdbc:avatica:remote:url=http://BROKER:8082/druid/v2/sql/avatica/
+
+
+

You can add authentication details to connect to a Druid deployment that is +secured by basic authentication by updating the URL and adding credentials:

+
connection-url=jdbc:avatica:remote:url=http://BROKER:port/druid/v2/sql/avatica/;authentication=BASIC
+connection-user=root
+connection-password=secret
+
+
+

Now you can access your Druid database in Trino with the example catalog +name from the properties file.

+

The connection-user and connection-password are typically required and +determine the user credentials for the connection, often a service user. You can +use secrets to avoid actual values in the catalog +properties files.

+
+

Data source authentication#

+

The connector can provide credentials for the data source connection +in multiple ways:

+
    +
  • inline, in the connector configuration file

  • +
  • in a separate properties file

  • +
  • in a key store file

  • +
  • as extra credentials set when connecting to Trino

  • +
+

You can use secrets to avoid storing sensitive +values in the catalog properties files.

+

The following table describes configuration properties +for connection credentials:

+ ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

credential-provider.type

Type of the credential provider. Must be one of INLINE, FILE, or +KEYSTORE; defaults to INLINE.

connection-user

Connection user name.

connection-password

Connection password.

user-credential-name

Name of the extra credentials property, whose value to use as the user +name. See extraCredentials in Parameter +reference.

password-credential-name

Name of the extra credentials property, whose value to use as the +password.

connection-credential-file

Location of the properties file where credentials are present. It must +contain the connection-user and connection-password properties.

keystore-file-path

The location of the Java Keystore file, from which to read credentials.

keystore-type

File format of the keystore file, for example JKS or PEM.

keystore-password

Password for the key store.

keystore-user-credential-name

Name of the key store entity to use as the user name.

keystore-user-credential-password

Password for the user name key store entity.

keystore-password-credential-name

Name of the key store entity to use as the password.

keystore-password-credential-password

Password for the password key store entity.

+
+
+

General configuration properties#

+

The following table describes general catalog configuration properties for the +connector:

+ +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

Default value

case-insensitive-name-matching

Support case insensitive schema and table names.

false

case-insensitive-name-matching.cache-ttl

This value should be a duration.

1m

case-insensitive-name-matching.config-file

Path to a name mapping configuration file in JSON format that allows +Trino to disambiguate between schemas and tables with similar names in +different cases.

null

case-insensitive-name-matching.config-file.refresh-period

Frequency with which Trino checks the name matching configuration file +for changes. This value should be a duration.

(refresh disabled)

metadata.cache-ttl

The duration for which metadata, including +table and column statistics, is cached.

0s (caching disabled)

metadata.cache-missing

Cache the fact that metadata, including table and column statistics, is +not available

false

metadata.cache-maximum-size

Maximum number of objects stored in the metadata cache

10000

write.batch-size

Maximum number of statements in a batched execution. +Do not change this setting from the default. Non-default values may +negatively impact performance.

1000

dynamic-filtering.enabled

Push down dynamic filters into JDBC queries

true

dynamic-filtering.wait-timeout

Maximum duration for which Trino will wait for dynamic +filters to be collected from the build side of joins before starting a +JDBC query. Using a large timeout can potentially result in more detailed +dynamic filters. However, it can also increase latency for some queries.

20s

+
+
+

Appending query metadata#

+

The optional parameter query.comment-format allows you to configure a SQL +comment that is sent to the datasource with each query. The format of this +comment can contain any characters and the following metadata:

+
    +
  • $QUERY_ID: The identifier of the query.

  • +
  • $USER: The name of the user who submits the query to Trino.

  • +
  • $SOURCE: The identifier of the client tool used to submit the query, for +example trino-cli.

  • +
  • $TRACE_TOKEN: The trace token configured with the client tool.

  • +
+

The comment can provide more context about the query. This additional +information is available in the logs of the datasource. To include environment +variables from the Trino cluster with the comment , use the +${ENV:VARIABLE-NAME} syntax.

+

The following example sets a simple comment that identifies each query sent by +Trino:

+
query.comment-format=Query sent by Trino.
+
+
+

With this configuration, a query such as SELECT * FROM example_table; is +sent to the datasource with the comment appended:

+
SELECT * FROM example_table; /*Query sent by Trino.*/
+
+
+

The following example improves on the preceding example by using metadata:

+
query.comment-format=Query $QUERY_ID sent by user $USER from Trino.
+
+
+

If Jane sent the query with the query identifier +20230622_180528_00000_bkizg, the following comment string is sent to the +datasource:

+
SELECT * FROM example_table; /*Query 20230622_180528_00000_bkizg sent by user Jane from Trino.*/
+
+
+
+

Note

+

Certain JDBC driver settings and logging configurations might cause the +comment to be removed.

+
+
+
+

Domain compaction threshold#

+

Pushing down a large list of predicates to the data source can compromise +performance. Trino compacts large predicates into a simpler range predicate +by default to ensure a balance between performance and predicate pushdown. +If necessary, the threshold for this compaction can be increased to improve +performance when the data source is capable of taking advantage of large +predicates. Increasing this threshold may improve pushdown of large +dynamic filters. +The domain-compaction-threshold catalog configuration property or the +domain_compaction_threshold catalog session property can be used to adjust the default value of +32 for this threshold.

+
+
+

Procedures#

+
    +
  • system.flush_metadata_cache()

    +

    Flush JDBC metadata caches. For example, the following system call +flushes the metadata caches for all schemas in the example catalog

    +
    USE example.example_schema;
    +CALL system.flush_metadata_cache();
    +
    +
    +
  • +
+
+
+

Case insensitive matching#

+

When case-insensitive-name-matching is set to true, Trino +is able to query non-lowercase schemas and tables by maintaining a mapping of +the lowercase name to the actual name in the remote system. However, if two +schemas and/or tables have names that differ only in case (such as “customers” +and “Customers”) then Trino fails to query them due to ambiguity.

+

In these cases, use the case-insensitive-name-matching.config-file catalog +configuration property to specify a configuration file that maps these remote +schemas/tables to their respective Trino schemas/tables:

+
{
+  "schemas": [
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "mapping": "case_insensitive_1"
+    },
+    {
+      "remoteSchema": "cASEsENSITIVEnAME",
+      "mapping": "case_insensitive_2"
+    }],
+  "tables": [
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "remoteTable": "tablex",
+      "mapping": "table_1"
+    },
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "remoteTable": "TABLEX",
+      "mapping": "table_2"
+    }]
+}
+
+
+

Queries against one of the tables or schemes defined in the mapping +attributes are run against the corresponding remote entity. For example, a query +against tables in the case_insensitive_1 schema is forwarded to the +CaseSensitiveName schema and a query against case_insensitive_2 is forwarded +to the cASEsENSITIVEnAME schema.

+

At the table mapping level, a query on case_insensitive_1.table_1 as +configured above is forwarded to CaseSensitiveName.tablex, and a query on +case_insensitive_1.table_2 is forwarded to CaseSensitiveName.TABLEX.

+

By default, when a change is made to the mapping configuration file, Trino must +be restarted to load the changes. Optionally, you can set the +case-insensitive-name-mapping.refresh-period to have Trino refresh the +properties without requiring a restart:

+
case-insensitive-name-mapping.refresh-period=30s
+
+
+
+
+
+

Type mapping#

+

Because Trino and Druid each support types that the other does not, this +connector modifies some types when reading data.

+
+

Druid type to Trino type mapping#

+

The connector maps Druid types to the corresponding Trino types according to the +following table:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Druid type to Trino type mapping#

Druid type

Trino type

Notes

STRING

VARCHAR

FLOAT

REAL

DOUBLE

DOUBLE

LONG

BIGINT

Except for the special _time column, which is mapped to TIMESTAMP.

TIMESTAMP

TIMESTAMP

Only applicable to the special _time column.

+

No other data types are supported.

+

Druid does not have a real NULL value for any data type. By +default, Druid treats NULL as the default value for a data type. For +example, LONG would be 0, DOUBLE would be 0.0, STRING would +be an empty string '', and so forth.

+
+
+

Type mapping configuration properties#

+

The following properties can be used to configure how data types from the +connected data source are mapped to Trino data types and how the metadata is +cached in Trino.

+ +++++ + + + + + + + + + + + + + + + + +

Property name

Description

Default value

unsupported-type-handling

Configure how unsupported column data types are handled:

+
    +
  • IGNORE, column is not accessible.

  • +
  • CONVERT_TO_VARCHAR, column is converted to unbounded VARCHAR.

  • +
+

The respective catalog session property is unsupported_type_handling.

+

IGNORE

jdbc-types-mapped-to-varchar

Allow forced mapping of comma separated lists of data types to convert to +unbounded VARCHAR

+
+
+
+

SQL support#

+

The connector provides globally available and +read operation statements to access data and +metadata in the Druid database.

+
+
+

Table functions#

+

The connector provides specific table functions to +access Druid.

+
+

query(varchar) -> table#

+

The query function allows you to query the underlying database directly. It +requires syntax native to Druid, because the full query is pushed down and +processed in Druid. This can be useful for accessing native features which are +not available in Trino or for improving query performance in situations where +running a query natively may be faster.

+

The native query passed to the underlying data source is required to return a +table as a result set. Only the data source performs validation or security +checks for these queries using its own configuration. Trino does not perform +these tasks. Only use passthrough queries to read data.

+

As an example, query the example catalog and use STRING_TO_MV and +MV_LENGTH from Druid SQL’s multi-value string functions +to split and then count the number of comma-separated values in a column:

+
SELECT
+  num_reports
+FROM
+  TABLE(
+    example.system.query(
+      query => 'SELECT
+        MV_LENGTH(
+          STRING_TO_MV(direct_reports, ",")
+        ) AS num_reports
+      FROM company.managers'
+    )
+  );
+
+
+
+

Note

+

The query engine does not preserve the order of the results of this +function. If the passed query contains an ORDER BY clause, the +function result may not be ordered as expected.

+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/elasticsearch.html b/430/connector/elasticsearch.html new file mode 100644 index 000000000..ce4666050 --- /dev/null +++ b/430/connector/elasticsearch.html @@ -0,0 +1,1278 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Elasticsearch connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Elasticsearch connector#

+

The Elasticsearch Connector allows access to Elasticsearch data from Trino. +This document describes how to setup the Elasticsearch Connector to run SQL queries against Elasticsearch.

+
+

Note

+

Elasticsearch (6.6.0 or later) or OpenSearch (1.1.0 or later) is required.

+
+
+

Configuration#

+

To configure the Elasticsearch connector, create a catalog properties file +etc/catalog/example.properties with the following contents, replacing the +properties as appropriate for your setup:

+
connector.name=elasticsearch
+elasticsearch.host=localhost
+elasticsearch.port=9200
+elasticsearch.default-schema-name=default
+
+
+
+

Configuration properties#

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Elasticsearch configuration properties#

Property name

Description

Default

elasticsearch.host

The comma-separated list of host names for the Elasticsearch node to +connect to. This property is required.

elasticsearch.port

Port of the Elasticsearch node to connect to.

9200

elasticsearch.default-schema-name

The schema that contains all tables defined without a qualifying schema +name.

default

elasticsearch.scroll-size

Sets the maximum number of hits that can be returned with each +Elasticsearch scroll request.

1000

elasticsearch.scroll-timeout

Amount of time Elasticsearch keeps the +search context +alive for scroll requests.

1m

elasticsearch.request-timeout

Timeout value for all Elasticsearch requests.

10s

elasticsearch.connect-timeout

Timeout value for all Elasticsearch connection attempts.

1s

elasticsearch.backoff-init-delay

The minimum duration between backpressure retry attempts for a single +request to Elasticsearch. Setting it too low might overwhelm an already +struggling ES cluster.

500ms

elasticsearch.backoff-max-delay

The maximum duration between backpressure retry attempts for a single +request to Elasticsearch.

20s

elasticsearch.max-retry-time

The maximum duration across all retry attempts for a single request to +Elasticsearch.

20s

elasticsearch.node-refresh-interval

How often the list of available Elasticsearch nodes is refreshed.

1m

elasticsearch.ignore-publish-address

Disables using the address published by Elasticsearch to connect for +queries.

+
+
+
+

TLS security#

+

The Elasticsearch connector provides additional security options to support +Elasticsearch clusters that have been configured to use TLS.

+

If your cluster has globally-trusted certificates, you should only need to +enable TLS. If you require custom configuration for certificates, the connector +supports key stores and trust stores in PEM or Java Key Store (JKS) format.

+

The allowed configuration values are:

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + +
TLS Security Properties#

Property name

Description

elasticsearch.tls.enabled

Enables TLS security.

elasticsearch.tls.keystore-path

The path to the PEM or +JKS key store.

elasticsearch.tls.truststore-path

The path to PEM or +JKS trust store.

elasticsearch.tls.keystore-password

The key password for the key store specified by +elasticsearch.tls.keystore-path.

elasticsearch.tls.truststore-password

The key password for the trust store specified by +elasticsearch.tls.truststore-path.

+
+
+

Type mapping#

+

Because Trino and Elasticsearch each support types that the other does not, this +connector maps some types when reading data.

+
+

Elasticsearch type to Trino type mapping#

+

The connector maps Elasticsearch types to the corresponding Trino types +according to the following table:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Elasticsearch type to Trino type mapping#

Elasticsearch type

Trino type

Notes

BOOLEAN

BOOLEAN

DOUBLE

DOUBLE

FLOAT

REAL

BYTE

TINYINT

SHORT

SMALLINT

INTEGER

INTEGER

LONG

BIGINT

KEYWORD

VARCHAR

TEXT

VARCHAR

DATE

TIMESTAMP

For more information, see Date types.

IPADDRESS

IP

+

No other types are supported.

+
+
+

Array types#

+

Fields in Elasticsearch can contain zero or more values +, but there is no dedicated array type. To indicate a field contains an array, it can be annotated in a Trino-specific structure in +the _meta section of the index mapping.

+

For example, you can have an Elasticsearch index that contains documents with the following structure:

+
{
+    "array_string_field": ["trino","the","lean","machine-ohs"],
+    "long_field": 314159265359,
+    "id_field": "564e6982-88ee-4498-aa98-df9e3f6b6109",
+    "timestamp_field": "1987-09-17T06:22:48.000Z",
+    "object_field": {
+        "array_int_field": [86,75,309],
+        "int_field": 2
+    }
+}
+
+
+

The array fields of this structure can be defined by using the following command to add the field +property definition to the _meta.trino property of the target index mapping.

+
curl --request PUT \
+    --url localhost:9200/doc/_mapping \
+    --header 'content-type: application/json' \
+    --data '
+{
+    "_meta": {
+        "trino":{
+            "array_string_field":{
+                "isArray":true
+            },
+            "object_field":{
+                "array_int_field":{
+                    "isArray":true
+                }
+            },
+        }
+    }
+}'
+
+
+
+

Note

+

It is not allowed to use asRawJson and isArray flags simultaneously for the same column.

+
+
+
+

Date types#

+

Elasticsearch supports a wide array of date formats including +built-in date formats and also custom date formats. +The Elasticsearch connector supports only the default date type. All other +date formats including built-in date formats and custom date formats are +not supported. Dates with the format property are ignored.

+
+
+

Raw JSON transform#

+

There are many occurrences where documents in Elasticsearch have more complex +structures that are not represented in the mapping. For example, a single +keyword field can have widely different content including a single +keyword value, an array, or a multidimensional keyword array with any +level of nesting.

+
curl --request PUT \
+    --url localhost:9200/doc/_mapping \
+    --header 'content-type: application/json' \
+    --data '
+{
+    "properties": {
+        "array_string_field":{
+            "type": "keyword"
+        }
+    }
+}'
+
+
+

Notice for the array_string_field that all the following documents are legal +for Elasticsearch. See the Elasticsearch array documentation +for more details.

+
[
+    {
+        "array_string_field": "trino"
+    },
+    {
+        "array_string_field": ["trino","is","the","besto"]
+    },
+    {
+        "array_string_field": ["trino",["is","the","besto"]]
+    },
+    {
+        "array_string_field": ["trino",["is",["the","besto"]]]
+    }
+]
+
+
+

Further, Elasticsearch supports types, such as +dense_vector, +that are not supported in Trino. New types are constantly emerging which can +cause parsing exceptions for users that use of these types in Elasticsearch. To +manage all of these scenarios, you can transform fields to raw JSON by +annotating it in a Trino-specific structure in the _meta +section of the index mapping. This indicates to Trino that the field, and all +nested fields beneath, need to be cast to a VARCHAR field that contains +the raw JSON content. These fields can be defined by using the following command +to add the field property definition to the _meta.presto property of the +target index mapping.

+
curl --request PUT \
+    --url localhost:9200/doc/_mapping \
+    --header 'content-type: application/json' \
+    --data '
+{
+    "_meta": {
+        "presto":{
+            "array_string_field":{
+                "asRawJson":true
+            }
+        }
+    }
+}'
+
+
+

This preceding configurations causes Trino to return the array_string_field +field as a VARCHAR containing raw JSON. You can parse these fields with the +built-in JSON functions.

+
+

Note

+

It is not allowed to use asRawJson and isArray flags simultaneously for the same column.

+
+
+
+
+

Special columns#

+

The following hidden columns are available:

+ + + + + + + + + + + + + + + + + +

Column

Description

_id

The Elasticsearch document ID

_score

The document score returned by the Elasticsearch query

_source

The source of the original document

+
+
+

Full text queries#

+

Trino SQL queries can be combined with Elasticsearch queries by providing the full text query +as part of the table name, separated by a colon. For example:

+
SELECT * FROM "tweets: +trino SQL^2"
+
+
+
+
+

Predicate push down#

+

The connector supports predicate push down of below data types:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Elasticsearch

Trino

Supports

binary

VARBINARY

NO

boolean

BOOLEAN

YES

double

DOUBLE

YES

float

REAL

YES

byte

TINYINT

YES

short

SMALLINT

YES

integer

INTEGER

YES

long

BIGINT

YES

keyword

VARCHAR

YES

text

VARCHAR

NO

date

TIMESTAMP

YES

ip

IPADDRESS

NO

(all others)

(unsupported)

(unsupported)

+
+
+

AWS authorization#

+

To enable AWS authorization using IAM policies, the elasticsearch.security option needs to be set to AWS. +Additionally, the following options need to be configured appropriately:

+ + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

elasticsearch.aws.region

AWS region or the Elasticsearch endpoint. This option is required.

elasticsearch.aws.access-key

AWS access key to use to connect to the Elasticsearch domain. If not set, the Default AWS Credentials Provider chain will be used.

elasticsearch.aws.secret-key

AWS secret key to use to connect to the Elasticsearch domain. If not set, the Default AWS Credentials Provider chain will be used.

elasticsearch.aws.iam-role

Optional ARN of an IAM Role to assume to connect to the Elasticsearch domain. Note: the configured IAM user has to be able to assume this role.

elasticsearch.aws.external-id

Optional external ID to pass while assuming an AWS IAM Role.

+
+
+

Password authentication#

+

To enable password authentication, the elasticsearch.security option needs to be set to PASSWORD. +Additionally the following options need to be configured appropriately:

+ + + + + + + + + + + + + + +

Property name

Description

elasticsearch.auth.user

User name to use to connect to Elasticsearch.

elasticsearch.auth.password

Password to use to connect to Elasticsearch.

+
+
+

SQL support#

+

The connector provides globally available and +read operation statements to access data and +metadata in the Elasticsearch catalog.

+
+
+

Table functions#

+

The connector provides specific table functions to +access Elasticsearch.

+
+

raw_query(varchar) -> table#

+

The raw_query function allows you to query the underlying database directly. +This function requires Elastic Query DSL +syntax, because the full query is pushed down and processed in Elasticsearch. +This can be useful for accessing native features which are not available in +Trino or for improving query performance in situations where running a query +natively may be faster.

+

The native query passed to the underlying data source is required to return a +table as a result set. Only the data source performs validation or security +checks for these queries using its own configuration. Trino does not perform +these tasks. Only use passthrough queries to read data.

+

The raw_query function requires three parameters:

+
    +
  • schema: The schema in the catalog that the query is to be executed on.

  • +
  • index: The index in Elasticsearch to be searched.

  • +
  • query: The query to be executed, written in Elastic Query DSL.

  • +
+

Once executed, the query returns a single row containing the resulting JSON +payload returned by Elasticsearch.

+

For example, query the example catalog and use the raw_query table +function to search for documents in the orders index where the country name +is ALGERIA:

+
SELECT
+  *
+FROM
+  TABLE(
+    example.system.raw_query(
+      schema => 'sales',
+      index => 'orders',
+      query => '{
+        "query": {
+          "match": {
+            "name": "ALGERIA"
+          }
+        }
+      }'
+    )
+  );
+
+
+
+

Note

+

The query engine does not preserve the order of the results of this +function. If the passed query contains an ORDER BY clause, the +function result may not be ordered as expected.

+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/googlesheets.html b/430/connector/googlesheets.html new file mode 100644 index 000000000..ea9ccd4ea --- /dev/null +++ b/430/connector/googlesheets.html @@ -0,0 +1,900 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Google Sheets connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Google Sheets connector#

+

The Google Sheets connector allows reading and writing Google Sheets spreadsheets as tables in Trino.

+
+

Configuration#

+

Create etc/catalog/example.properties to mount the Google Sheets connector +as the example catalog, with the following contents:

+
connector.name=gsheets
+gsheets.credentials-path=/path/to/google-sheets-credentials.json
+gsheets.metadata-sheet-id=exampleId
+
+
+
+
+

Configuration properties#

+

The following configuration properties are available:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

gsheets.credentials-path

Path to the Google API JSON key file

gsheets.credentials-key

The base64 encoded credentials key

gsheets.metadata-sheet-id

Sheet ID of the spreadsheet, that contains the table mapping

gsheets.max-data-cache-size

Maximum number of spreadsheets to cache, defaults to 1000

gsheets.data-cache-ttl

How long to cache spreadsheet data or metadata, defaults to 5m

gsheets.connection-timeout

Timeout when connection to Google Sheets API, defaults to 20s

gsheets.read-timeout

Timeout when reading from Google Sheets API, defaults to 20s

gsheets.write-timeout

Timeout when writing to Google Sheets API, defaults to 20s

+
+
+

Credentials#

+

The connector requires credentials in order to access the Google Sheets API.

+
    +
  1. Open the Google Sheets API +page and click the Enable button. This takes you to the API manager page.

  2. +
  3. Select a project using the drop down menu at the top of the page. +Create a new project, if you do not already have one.

  4. +
  5. Choose Credentials in the left panel.

  6. +
  7. Click Manage service accounts, then create a service account for the connector. +On the Create key step, create and download a key in JSON format.

  8. +
+

The key file needs to be available on the Trino coordinator and workers. +Set the gsheets.credentials-path configuration property to point to this file. +The exact name of the file does not matter – it can be named anything.

+

Alternatively, set the gsheets.credentials-key configuration property. +It should contain the contents of the JSON file, encoded using base64.

+
+
+

Metadata sheet#

+

The metadata sheet is used to map table names to sheet IDs. +Create a new metadata sheet. The first row must be a header row +containing the following columns in this order:

+
    +
  • Table Name

  • +
  • Sheet ID

  • +
  • Owner (optional)

  • +
  • Notes (optional)

  • +
+

See this example sheet +as a reference.

+

The metadata sheet must be shared with the service account user, +the one for which the key credentials file was created. Click the Share +button to share the sheet with the email address of the service account.

+

Set the gsheets.metadata-sheet-id configuration property to the ID of this sheet.

+
+
+

Querying sheets#

+

The service account user must have access to the sheet in order for Trino +to query it. Click the Share button to share the sheet with the email +address of the service account.

+

The sheet needs to be mapped to a Trino table name. Specify a table name +(column A) and the sheet ID (column B) in the metadata sheet. To refer +to a specific range in the sheet, add the range after the sheet ID, separated +with #. If a range is not provided, the connector loads only 10,000 rows by default from +the first tab in the sheet.

+

The first row of the provided sheet range is used as the header and will determine the column +names of the Trino table. +For more details on sheet range syntax see the google sheets docs.

+
+
+

Writing to sheets#

+

The same way sheets can be queried, they can also be written by appending data to existing sheets. +In this case the service account user must also have Editor permissions on the sheet.

+

After data is written to a table, the table contents are removed from the cache +described in API usage limits. If the table is accessed +immediately after the write, querying the Google Sheets API may not reflect the +change yet. In that case the old version of the table is read and cached for the +configured amount of time, and it might take some time for the written changes +to propagate properly.

+

Keep in mind that the Google Sheets API has usage limits, that limit the speed of inserting data. +If you run into timeouts you can increase timeout times to avoid 503: The service is currently unavailable errors.

+
+
+

API usage limits#

+

The Google Sheets API has usage limits, +that may impact the usage of this connector. Increasing the cache duration and/or size +may prevent the limit from being reached. Running queries on the information_schema.columns +table without a schema and table name filter may lead to hitting the limit, as this requires +fetching the sheet data for every table, unless it is already cached.

+
+
+

Type mapping#

+

Because Trino and Google Sheets each support types that the other does not, this +connector modifies some types when reading data.

+
+

Google Sheets type to Trino type mapping#

+

The connector maps Google Sheets types to the corresponding Trino types +following this table:

+ + ++++ + + + + + + + + + + +
Google Sheets type to Trino type mapping#

Google Sheets type

Trino type

TEXT

VARCHAR

+

No other types are supported.

+
+
+
+

SQL support#

+

In addition to the globally available and read operation statements, +this connector supports the following features:

+ +
+
+

Table functions#

+

The connector provides specific Table functions to access Google Sheets.

+
+

sheet(id, range) -> table#

+

The sheet function allows you to query a Google Sheet directly without +specifying it as a named table in the metadata sheet.

+

For example, for a catalog named ‘example’:

+
SELECT *
+FROM
+  TABLE(example.system.sheet(
+      id => 'googleSheetIdHere'));
+
+
+

A sheet range or named range can be provided as an optional range argument. +The default sheet range is $1:$10000 if one is not provided:

+
SELECT *
+FROM
+  TABLE(example.system.sheet(
+      id => 'googleSheetIdHere',
+      range => 'TabName!A1:B4'));
+
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/hive-alluxio.html b/430/connector/hive-alluxio.html new file mode 100644 index 000000000..26f3343b8 --- /dev/null +++ b/430/connector/hive-alluxio.html @@ -0,0 +1,744 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Hive connector with Alluxio — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Hive connector with Alluxio#

+

The Hive connector can read and write tables stored in the Alluxio Data Orchestration +System, +leveraging Alluxio’s distributed block-level read/write caching functionality. +The tables must be created in the Hive metastore with the alluxio:// +location prefix (see Running Apache Hive with Alluxio +for details and examples).

+

Trino queries will then transparently retrieve and cache files or objects from +a variety of disparate storage systems including HDFS and S3.

+
+

Setting up Alluxio with Trino#

+

For information on how to setup, configure, and use Alluxio, refer to Alluxio’s +documentation on using their platform with Trino.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/hive-azure.html b/430/connector/hive-azure.html new file mode 100644 index 000000000..bfe702d24 --- /dev/null +++ b/430/connector/hive-azure.html @@ -0,0 +1,1043 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Hive connector with Azure Storage — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Hive connector with Azure Storage#

+

The Hive connector can be configured to use Azure Data Lake Storage (Gen2). Trino +supports Azure Blob File System (ABFS) to access data in ADLS Gen2.

+

Trino also supports ADLS Gen1 +and Windows Azure Storage Blob driver (WASB), but we recommend migrating to +ADLS Gen2, +as ADLS Gen1 and WASB are legacy options that will be removed in the future. +Learn more from the official documentation.

+
+

Hive connector configuration for Azure Storage credentials#

+

To configure Trino to use the Azure Storage credentials, set the following +configuration properties in the catalog properties file. It is best to use this +type of configuration if the primary storage account is linked to the cluster.

+

The specific configuration depends on the type of storage and uses the +properties from the following sections in the catalog properties file.

+

For more complex use cases, such as configuring multiple secondary storage +accounts using Hadoop’s core-site.xml, see the +Advanced configuration options.

+
+

ADLS Gen2 / ABFS storage#

+

To connect to ABFS storage, you may either use the storage account’s access +key, or a service principal. Do not use both sets of properties at the +same time.

+ + ++++ + + + + + + + + + + + + + +
ABFS Access Key#

Property name

Description

hive.azure.abfs-storage-account

The name of the ADLS Gen2 storage account

hive.azure.abfs-access-key

The decrypted access key for the ADLS Gen2 storage account

+ + ++++ + + + + + + + + + + + + + + + + +
ABFS Service Principal OAuth#

Property name

Description

hive.azure.abfs.oauth.endpoint

The service principal / application’s OAuth 2.0 token endpoint (v1).

hive.azure.abfs.oauth.client-id

The service principal’s client/application ID.

hive.azure.abfs.oauth.secret

A client secret for the service principal.

+

When using a service principal, it must have the Storage Blob Data Owner, +Contributor, or Reader role on the storage account you are using, depending on +which operations you would like to use.

+
+
+

ADLS Gen1 (legacy)#

+

While it is advised to migrate to ADLS Gen2 whenever possible, if you still +choose to use ADLS Gen1 you need to include the following properties in your +catalog configuration.

+
+

Note

+

Credentials for the filesystem can be configured using ClientCredential +type. To authenticate with ADLS Gen1 you must create a new application +secret for your ADLS Gen1 account’s App Registration, and save this value +because you won’t able to retrieve the key later. Refer to the Azure +documentation +for details.

+
+ + ++++ + + + + + + + + + + + + + + + + + + + +
ADLS properties#

Property name

Description

hive.azure.adl-client-id

Client (Application) ID from the App Registrations for your storage +account

hive.azure.adl-credential

Value of the new client (application) secret created

hive.azure.adl-refresh-url

OAuth 2.0 token endpoint url

hive.azure.adl-proxy-host

Proxy host and port in host:port format. Use this property to connect +to an ADLS endpoint via a SOCKS proxy.

+
+
+

WASB storage (legacy)#

+ + ++++ + + + + + + + + + + + + + +
WASB properties#

Property name

Description

hive.azure.wasb-storage-account

Storage account name of Azure Blob Storage

hive.azure.wasb-access-key

The decrypted access key for the Azure Blob Storage

+
+
+

Advanced configuration#

+

All of the configuration properties for the Azure storage driver are stored in +the Hadoop core-site.xml configuration file. When there are secondary +storage accounts involved, we recommend configuring Trino using a +core-site.xml containing the appropriate credentials for each account.

+

The path to the file must be configured in the catalog properties file:

+
hive.config.resources=<path_to_hadoop_core-site.xml>
+
+
+

One way to find your account key is to ask for the connection string for the +storage account. The abfsexample.dfs.core.windows.net account refers to the +storage account. The connection string contains the account key:

+
az storage account  show-connection-string --name abfswales1
+{
+  "connectionString": "DefaultEndpointsProtocol=https;EndpointSuffix=core.windows.net;AccountName=abfsexample;AccountKey=examplekey..."
+}
+
+
+

When you have the account access key, you can add it to your core-site.xml +or Java cryptography extension (JCEKS) file. Alternatively, you can have your +cluster management tool to set the option +fs.azure.account.key.STORAGE-ACCOUNT to the account key value:

+
<property>
+  <name>fs.azure.account.key.abfsexample.dfs.core.windows.net</name>
+  <value>examplekey...</value>
+</property>
+
+
+

For more information, see Hadoop Azure Support: ABFS.

+
+
+
+

Accessing Azure Storage data#

+
+

URI scheme to reference data#

+

Consistent with other FileSystem implementations within Hadoop, the Azure +Standard Blob and Azure Data Lake Storage Gen2 (ABFS) drivers define their own +URI scheme so that resources (directories and files) may be distinctly +addressed. You can access both primary and secondary storage accounts linked to +the cluster with the same URI scheme. Following are example URIs for the +different systems.

+

ABFS URI:

+
abfs[s]://<file_system>@<account_name>.dfs.core.windows.net/<path>/<path>/<file_name>
+
+
+

ADLS Gen1 URI:

+
adl://<data_lake_storage_gen1_name>.azuredatalakestore.net/<path>/<file_name>
+
+
+

Azure Standard Blob URI:

+
wasb[s]://<container>@<account_name>.blob.core.windows.net/<path>/<path>/<file_name>
+
+
+
+
+

Querying Azure Storage#

+

You can query tables already configured in your Hive metastore used in your Hive +catalog. To access Azure Storage data that is not yet mapped in the Hive +metastore, you need to provide the schema of the data, the file format, and the +data location.

+

For example, if you have ORC or Parquet files in an ABFS file_system, you +need to execute a query:

+
-- select schema in which the table is to be defined, must already exist
+USE hive.default;
+
+-- create table
+CREATE TABLE orders (
+     orderkey BIGINT,
+     custkey BIGINT,
+     orderstatus VARCHAR(1),
+     totalprice DOUBLE,
+     orderdate DATE,
+     orderpriority VARCHAR(15),
+     clerk VARCHAR(15),
+     shippriority INTEGER,
+     comment VARCHAR(79)
+) WITH (
+     external_location = 'abfs[s]://<file_system>@<account_name>.dfs.core.windows.net/<path>/<path>/',
+     format = 'ORC' -- or 'PARQUET'
+);
+
+
+

Now you can query the newly mapped table:

+
SELECT * FROM orders;
+
+
+
+
+
+

Writing data#

+
+

Prerequisites#

+

Before you attempt to write data to Azure Storage, make sure you have configured +everything necessary to read data from the storage.

+
+
+

Create a write schema#

+

If the Hive metastore contains schema(s) mapped to Azure storage filesystems, +you can use them to write data to Azure storage.

+

If you don’t want to use existing schemas, or there are no appropriate schemas +in the Hive metastore, you need to create a new one:

+
CREATE SCHEMA hive.abfs_export
+WITH (location = 'abfs[s]://file_system@account_name.dfs.core.windows.net/<path>');
+
+
+
+
+

Write data to Azure Storage#

+

Once you have a schema pointing to a location where you want to write the data, +you can issue a CREATE TABLE AS statement and select your desired file +format. The data will be written to one or more files within the +abfs[s]://file_system@account_name.dfs.core.windows.net/<path>/my_table +namespace. Example:

+
CREATE TABLE hive.abfs_export.orders_abfs
+WITH (format = 'ORC')
+AS SELECT * FROM tpch.sf1.orders;
+
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/hive-caching.html b/430/connector/hive-caching.html new file mode 100644 index 000000000..c418d9460 --- /dev/null +++ b/430/connector/hive-caching.html @@ -0,0 +1,936 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Hive connector storage caching — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Hive connector storage caching#

+

Querying object storage with the Hive connector is a +very common use case for Trino. It often involves the transfer of large amounts +of data. The objects are retrieved from HDFS, or any other supported object +storage, by multiple workers and processed on these workers. Repeated queries +with different parameters, or even different queries from different users, often +access, and therefore transfer, the same objects.

+
+

Benefits#

+

Enabling caching can result in significant benefits:

+

Reduced load on object storage

+

Every retrieved and cached object avoids repeated retrieval from the storage in +subsequent queries. As a result the object storage system does not have to +provide the object again and again.

+

For example, if your query accesses 100MB of objects from the storage, the first +time the query runs 100MB are downloaded and cached. Any following query uses +these objects. If your users run another 100 queries accessing the same objects, +your storage system does not have to do any significant work. Without caching it +has to provide the same objects again and again, resulting in 10GB of total +storage to serve.

+

This reduced load on the object storage can also impact the sizing, and +therefore the cost, of the object storage system.

+

Increased query performance

+

Caching can provide significant performance benefits, by avoiding the repeated +network transfers and instead accessing copies of the objects from a local +cache. Performance gains are more significant if the performance of directly +accessing the object storage is low compared to accessing the cache.

+

For example, if you access object storage in a different network, different data +center or even different cloud-provider region query performance is slow. Adding +caching using fast, local storage has a significant impact and makes your +queries much faster.

+

On the other hand, if your object storage is already running at very high +performance for I/O and network access, and your local cache storage is at +similar speeds, or even slower, performance benefits can be minimal.

+

Reduced query costs

+

A result of the reduced load on the object storage, mentioned earlier, is +significantly reduced network traffic. Network traffic however is a considerable +cost factor in an setup, specifically also when hosted in public cloud provider +systems.

+
+
+

Architecture#

+

Caching can operate in two modes. The async mode provides the queried data +directly and caches any objects asynchronously afterwards. Async is the default +and recommended mode. The query doesn’t pay the cost of warming up the cache. +The cache is populated in the background and the query bypasses the cache if the +cache is not already populated. Any following queries requesting the cached +objects are served directly from the cache.

+

The other mode is a read-through cache. In this mode, if an object is not found +in the cache, it is read from the storage, placed in the cache, and then provided +to the requesting query. In read-through mode, the query always reads from cache +and must wait for the cache to be populated.

+

In both modes, objects are cached on local storage of each worker. Workers can +request cached objects from other workers to avoid requests from the object +storage.

+

The cache chunks are 1MB in size and are well suited for ORC or Parquet file +formats.

+
+
+

Configuration#

+

The caching feature is part of the Hive connector and +can be activated in the catalog properties file:

+
connector.name=hive
+hive.cache.enabled=true
+hive.cache.location=/opt/hive-cache
+
+
+

The cache operates on the coordinator and all workers accessing the object +storage. The used networking ports for the managing BookKeeper and the data +transfer, by default 8898 and 8899, need to be available.

+

To use caching on multiple catalogs, you need to configure different caching +directories and different BookKeeper and data-transfer ports.

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Cache Configuration Parameters#

Property

Description

Default

hive.cache.enabled

Toggle to enable or disable caching

false

hive.cache.location

Required directory location to use for the cache storage on each worker. +Separate multiple directories, which can be mount points for separate +drives, with commas. More tips can be found in the recommendations. Example: +hive.cache.location=/var/lib/trino/cache1,/var/lib/trino/cache2

hive.cache.data-transfer-port

The TCP/IP port used to transfer data managed by the cache.

8898

hive.cache.bookkeeper-port

The TCP/IP port used by the BookKeeper managing the cache.

8899

hive.cache.read-mode

Operational mode for the cache as described earlier in the architecture +section. async and read-through are the supported modes.

async

hive.cache.ttl

Time to live for objects in the cache. Objects, which have not been +requested for the TTL value, are removed from the cache.

7d

hive.cache.disk-usage-percentage

Percentage of disk space used for cached data.

80

+
+
+

Recommendations#

+

The speed of the local cache storage is crucial to the performance of the cache. +The most common and cost efficient approach is to attach high performance SSD +disk or equivalents. Fast cache performance can be also be achieved with a RAM +disk used as in-memory.

+

In all cases, you should avoid using the root partition and disk of the node and +instead attach at multiple dedicated storage devices for the cache on each node. +The cache uses the disk up to a configurable percentage. Storage should be local +on each coordinator and worker node. The directory needs to exist before Trino +starts. We recommend using multiple devices to improve performance of the cache.

+

The capacity of the attached storage devices should be about 20-30% larger than +the size of the queried object storage workload. For example, your current query +workload typically accesses partitions in your HDFS storage that encapsulate +data for the last 3 months. The overall size of these partitions is currently at +1TB. As a result your cache drives have to have a total capacity of 1.2 TB or +more.

+

Your deployment method for Trino decides how to create the directory for +caching. Typically you need to connect a fast storage system, like an SSD drive, +and ensure that is it mounted on the configured path. Kubernetes, CFT and other +systems allow this via volumes.

+
+
+

Object storage systems#

+

The following object storage systems are tested:

+ +
+
+

Metrics#

+

In order to verify how caching works on your system you can take multiple +approaches:

+
    +
  • Inspect the disk usage on the cache storage drives on all nodes

  • +
  • Query the metrics of the caching system exposed by JMX

  • +
+

The implementation of the cache exposes a number of metrics via JMX. You can +inspect these and other metrics directly in Trino with the JMX connector or in external tools.

+

Basic caching statistics for the catalog are available in the +jmx.current."rubix:catalog=<catalog_name>,name=stats" table. +The table jmx.current."rubix:catalog=<catalog_name>,type=detailed,name=stats +contains more detailed statistics.

+

The following example query returns the average cache hit ratio for the hive catalog:

+
SELECT avg(cache_hit)
+FROM jmx.current."rubix:catalog=hive,name=stats"
+WHERE NOT is_nan(cache_hit);
+
+
+
+
+

Limitations#

+

Caching does not support user impersonation and cannot be used with HDFS secured by Kerberos. +It does not take any user-specific access rights to the object storage into account. +The cached objects are simply transparent binary blobs to the caching system and full +access to all content is available.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/hive-cos.html b/430/connector/hive-cos.html new file mode 100644 index 000000000..54587a278 --- /dev/null +++ b/430/connector/hive-cos.html @@ -0,0 +1,819 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Hive connector with IBM Cloud Object Storage — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+ +
+ +
+
+ +
+

Hive connector with IBM Cloud Object Storage#

+

Configure the Hive connector to support IBM Cloud Object Storage COS access.

+
+

Configuration#

+

To use COS, you need to configure a catalog file to use the Hive +connector. For example, create a file etc/ibmcos.properties and +specify the path to the COS service config file with the +hive.cos.service-config property.

+
connector.name=hive
+hive.cos.service-config=etc/cos-service.properties
+
+
+

The service configuration file contains the access and secret keys, as well as +the endpoints for one or multiple COS services:

+
service1.access-key=<your-access-key1>
+service1.secret-key=<your-secret-key1>
+service1.endpoint=<endpoint1>
+service2.access-key=<your-access-key2>
+service2.secret-key=<your-secret-key2>
+service2.endpoint=<endpoint2>
+
+
+

The endpoints property is optional. service1 and service2 are +placeholders for unique COS service names. The granularity for providing access +credentials is at the COS service level.

+

To use IBM COS service, specify the service name, for example: service1 in +the COS path. The general URI path pattern is +cos://<bucket>.<service>/object(s).

+
cos://example-bucket.service1/orders_tiny
+
+
+

Trino translates the COS path, and uses the service1 endpoint and +credentials from cos-service.properties to access +cos://example-bucket.service1/object.

+

The Hive Metastore (HMS) does not support the IBM COS filesystem, by default. +The Stocator library is a possible +solution to this problem. Download the Stocator JAR, +and place it in Hadoop PATH. The Stocator IBM COS configuration +should be placed in core-site.xml. For example:

+
<property>
+        <name>fs.stocator.scheme.list</name>
+        <value>cos</value>
+</property>
+<property>
+        <name>fs.cos.impl</name>
+        <value>com.ibm.stocator.fs.ObjectStoreFileSystem</value>
+</property>
+<property>
+        <name>fs.stocator.cos.impl</name>
+        <value>com.ibm.stocator.fs.cos.COSAPIClient</value>
+</property>
+<property>
+        <name>fs.stocator.cos.scheme</name>
+        <value>cos</value>
+</property>
+<property>
+    <name>fs.cos.service1.endpoint</name>
+    <value>http://s3.eu-de.cloud-object-storage.appdomain.cloud</value>
+</property>
+<property>
+    <name>fs.cos.service1.access.key</name>
+    <value>access-key</value>
+</property>
+<property>
+    <name>fs.cos.service1.secret.key</name>
+    <value>secret-key</value>
+</property>
+
+
+
+
+

Alternative configuration using S3 compatibility#

+

Use the S3 properties for the Hive connector in the catalog file. If only one +IBM Cloud Object Storage endpoint is used, then the configuration can be +simplified:

+
hive.s3.endpoint=http://s3.eu-de.cloud-object-storage.appdomain.cloud
+hive.s3.aws-access-key=access-key
+hive.s3.aws-secret-key=secret-key
+
+
+

Use s3 protocol instead of cos for the table location:

+
s3://example-bucket/object/
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/hive-gcs-tutorial.html b/430/connector/hive-gcs-tutorial.html new file mode 100644 index 000000000..f435cab04 --- /dev/null +++ b/430/connector/hive-gcs-tutorial.html @@ -0,0 +1,816 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Google Cloud Storage — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Google Cloud Storage#

+

Object storage connectors can access +Google Cloud Storage data using the +gs:// URI prefix.

+
+

Requirements#

+

To use Google Cloud Storage with non-anonymous access objects, you need:

+ +
+
+

Configuration#

+

The use of Google Cloud Storage as a storage location for an object storage +catalog requires setting a configuration property that defines the +authentication method for any non-anonymous access object. Access methods cannot +be combined.

+

The default root path used by the gs:\\ prefix is set in the catalog by the +contents of the specified key file, or the key file used to create the OAuth +token.

+ + ++++ + + + + + + + + + + + + + +
Google Cloud Storage configuration properties#

Property Name

Description

hive.gcs.json-key-file-path

JSON key file used to authenticate your Google Cloud service account +with Google Cloud Storage.

hive.gcs.use-access-token

Use client-provided OAuth token to access Google Cloud Storage.

+

The following uses the Delta Lake connector in an example of a minimal +configuration file for an object storage catalog using a JSON key file:

+
connector.name=delta_lake
+hive.metastore.uri=thrift://example.net:9083
+hive.gcs.json-key-file-path=${ENV:GCP_CREDENTIALS_FILE_PATH}
+
+
+
+
+

General usage#

+

Create a schema to use if one does not already exist, as in the following +example:

+
CREATE SCHEMA storage_catalog.sales_data_in_gcs WITH (location = 'gs://example_location');
+
+
+

Once you have created a schema, you can create tables in the schema, as in the +following example:

+
CREATE TABLE storage_catalog.sales_data_in_gcs.orders (
+    orderkey BIGINT,
+    custkey BIGINT,
+    orderstatus VARCHAR(1),
+    totalprice DOUBLE,
+    orderdate DATE,
+    orderpriority VARCHAR(15),
+    clerk VARCHAR(15),
+    shippriority INTEGER,
+    comment VARCHAR(79)
+);
+
+
+

This statement creates the folder gs://sales_data_in_gcs/orders in the root +folder defined in the JSON key file.

+

Your table is now ready to populate with data using INSERT statements. +Alternatively, you can use CREATE TABLE AS statements to create and +populate the table in a single statement.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/hive-s3.html b/430/connector/hive-s3.html new file mode 100644 index 000000000..277230d70 --- /dev/null +++ b/430/connector/hive-s3.html @@ -0,0 +1,1135 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Hive connector with Amazon S3 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Hive connector with Amazon S3#

+

The Hive connector can read and write tables that are stored in +Amazon S3 or S3-compatible systems. +This is accomplished by having a table or database location that +uses an S3 prefix, rather than an HDFS prefix.

+

Trino uses its own S3 filesystem for the URI prefixes +s3://, s3n:// and s3a://.

+
+

S3 configuration properties#

+ ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

hive.s3.aws-access-key

Default AWS access key to use.

hive.s3.aws-secret-key

Default AWS secret key to use.

hive.s3.iam-role

IAM role to assume.

hive.s3.external-id

External ID for the IAM role trust policy.

hive.s3.endpoint

The S3 storage endpoint server. This can be used to connect to an +S3-compatible storage system instead of AWS. When using v4 signatures, +it is recommended to set this to the AWS region-specific endpoint +(e.g., http[s]://s3.<AWS-region>.amazonaws.com).

hive.s3.region

Optional property to force the S3 client to connect to the specified +region only.

hive.s3.storage-class

The S3 storage class to use when writing the data. Currently only +STANDARD and INTELLIGENT_TIERING storage classes are supported. +Default storage class is STANDARD

hive.s3.signer-type

Specify a different signer type for S3-compatible storage. +Example: S3SignerType for v2 signer type

hive.s3.signer-class

Specify a different signer class for S3-compatible storage.

hive.s3.path-style-access

Use path-style access for all requests to the S3-compatible storage. +This is for S3-compatible storage that doesn’t support +virtual-hosted-style access, defaults to false.

hive.s3.staging-directory

Local staging directory for data written to S3. This defaults to the +Java temporary directory specified by the JVM system property +java.io.tmpdir.

hive.s3.pin-client-to-current-region

Pin S3 requests to the same region as the EC2 instance where Trino is +running, defaults to false.

hive.s3.ssl.enabled

Use HTTPS to communicate with the S3 API, defaults to true.

hive.s3.sse.enabled

Use S3 server-side encryption, defaults to false.

hive.s3.sse.type

The type of key management for S3 server-side encryption. Use S3 +for S3 managed or KMS for KMS-managed keys, defaults to S3.

hive.s3.sse.kms-key-id

The KMS Key ID to use for S3 server-side encryption with KMS-managed +keys. If not set, the default key is used.

hive.s3.kms-key-id

If set, use S3 client-side encryption and use the AWS KMS to store +encryption keys and use the value of this property as the KMS Key ID for +newly created objects.

hive.s3.encryption-materials-provider

If set, use S3 client-side encryption and use the value of this property +as the fully qualified name of a Java class which implements the AWS +SDK’s EncryptionMaterialsProvider interface. If the class also +implements Configurable from the Hadoop API, the Hadoop +configuration will be passed in after the object has been created.

hive.s3.upload-acl-type

Canned ACL to use while uploading files to S3, defaults to PRIVATE. +If the files are to be uploaded to an S3 bucket owned by a different AWS +user, the canned ACL has to be set to one of the following: +AUTHENTICATED_READ, AWS_EXEC_READ, BUCKET_OWNER_FULL_CONTROL, +BUCKET_OWNER_READ, LOG_DELIVERY_WRITE, PUBLIC_READ, +PUBLIC_READ_WRITE. Refer to the AWS canned ACL +guide to understand each option’s definition.

hive.s3.skip-glacier-objects

Ignore Glacier objects rather than failing the query. This skips data +that may be expected to be part of the table or partition. Defaults to +false.

hive.s3.streaming.enabled

Use S3 multipart upload API to upload file in streaming way, without +staging file to be created in the local file system.

hive.s3.streaming.part-size

The part size for S3 streaming upload. Defaults to 16MB.

hive.s3.proxy.host

Proxy host to use if connecting through a proxy.

hive.s3.proxy.port

Proxy port to use if connecting through a proxy.

hive.s3.proxy.protocol

Proxy protocol. HTTP or HTTPS , defaults to HTTPS.

hive.s3.proxy.non-proxy-hosts

Hosts list to access without going through the proxy.

hive.s3.proxy.username

Proxy user name to use if connecting through a proxy.

hive.s3.proxy.password

Proxy password to use if connecting through a proxy.

hive.s3.proxy.preemptive-basic-auth

Whether to attempt to authenticate preemptively against proxy when using +base authorization, defaults to false.

hive.s3.sts.endpoint

Optional override for the sts endpoint given that IAM role based +authentication via sts is used.

hive.s3.sts.region

Optional override for the sts region given that IAM role based +authentication via sts is used.

+
+
+

S3 credentials#

+

If you are running Trino on Amazon EC2, using EMR or another facility, +it is recommended that you use IAM Roles for EC2 to govern access to S3. +To enable this, your EC2 instances need to be assigned an IAM Role which +grants appropriate access to the data stored in the S3 bucket(s) you wish +to use. It is also possible to configure an IAM role with hive.s3.iam-role +that is used for accessing any S3 bucket. This is much cleaner than +setting AWS access and secret keys in the hive.s3.aws-access-key +and hive.s3.aws-secret-key settings, and also allows EC2 to automatically +rotate credentials on a regular basis without any additional work on your part.

+
+
+

Custom S3 credentials provider#

+

You can configure a custom S3 credentials provider by setting the configuration +property trino.s3.credentials-provider to the fully qualified class name of +a custom AWS credentials provider implementation. The property must be set in +the Hadoop configuration files referenced by the hive.config.resources Hive +connector property.

+

The class must implement the +AWSCredentialsProvider +interface and provide a two-argument constructor that takes a +java.net.URI and a Hadoop org.apache.hadoop.conf.Configuration +as arguments. A custom credentials provider can be used to provide +temporary credentials from STS (using STSSessionCredentialsProvider), +IAM role-based credentials (using STSAssumeRoleSessionCredentialsProvider), +or credentials for a specific use case (e.g., bucket/user specific credentials).

+
+
+

S3 security mapping#

+

Trino supports flexible security mapping for S3, allowing for separate +credentials or IAM roles for specific users or buckets/paths. The IAM role +for a specific query can be selected from a list of allowed roles by providing +it as an extra credential.

+

Each security mapping entry may specify one or more match criteria. If multiple +criteria are specified, all criteria must match. Available match criteria:

+
    +
  • user: Regular expression to match against username. Example: alice|bob

  • +
  • group: Regular expression to match against any of the groups that the user +belongs to. Example: finance|sales

  • +
  • prefix: S3 URL prefix. It can specify an entire bucket or a path within a +bucket. The URL must start with s3:// but will also match s3a or s3n. +Example: s3://bucket-name/abc/xyz/

  • +
+

The security mapping must provide one or more configuration settings:

+
    +
  • accessKey and secretKey: AWS access key and secret key. This overrides +any globally configured credentials, such as access key or instance credentials.

  • +
  • iamRole: IAM role to use if no user provided role is specified as an +extra credential. This overrides any globally configured IAM role. This role +is allowed to be specified as an extra credential, although specifying it +explicitly has no effect, as it would be used anyway.

  • +
  • roleSessionName: Optional role session name to use with iamRole. This can only +be used when iamRole is specified. If roleSessionName includes the string +${USER}, then the ${USER} portion of the string will be replaced with the +current session’s username. If roleSessionName is not specified, it defaults +to trino-session.

  • +
  • allowedIamRoles: IAM roles that are allowed to be specified as an extra +credential. This is useful because a particular AWS account may have permissions +to use many roles, but a specific user should only be allowed to use a subset +of those roles.

  • +
  • kmsKeyId: ID of KMS-managed key to be used for client-side encryption.

  • +
  • allowedKmsKeyIds: KMS-managed key IDs that are allowed to be specified as an extra +credential. If list cotains “*”, then any key can be specified via extra credential.

  • +
+

The security mapping entries are processed in the order listed in the configuration +JSON. More specific mappings should thus be specified before less specific mappings. +For example, the mapping list might have URL prefix s3://abc/xyz/ followed by +s3://abc/ to allow different configuration for a specific path within a bucket +than for other paths within the bucket. You can set default configuration by not +including any match criteria for the last entry in the list.

+

In addition to the rules above, the default mapping can contain the optional +useClusterDefault boolean property with the following behavior:

+
    +
  • false - (is set by default) property is ignored.

  • +
  • true - This causes the default cluster role to be used as a fallback option. +It can not be used with the following configuration properties:

    +
      +
    • accessKey

    • +
    • secretKey

    • +
    • iamRole

    • +
    • allowedIamRoles

    • +
    +
  • +
+

If no mapping entry matches and no default is configured, the access is denied.

+

The configuration JSON can either be retrieved from a file or REST-endpoint specified via +hive.s3.security-mapping.config-file.

+

Example JSON configuration:

+
{
+  "mappings": [
+    {
+      "prefix": "s3://bucket-name/abc/",
+      "iamRole": "arn:aws:iam::123456789101:role/test_path"
+    },
+    {
+      "user": "bob|charlie",
+      "iamRole": "arn:aws:iam::123456789101:role/test_default",
+      "allowedIamRoles": [
+        "arn:aws:iam::123456789101:role/test1",
+        "arn:aws:iam::123456789101:role/test2",
+        "arn:aws:iam::123456789101:role/test3"
+      ]
+    },
+    {
+      "prefix": "s3://special-bucket/",
+      "accessKey": "AKIAxxxaccess",
+      "secretKey": "iXbXxxxsecret"
+    },
+    {
+      "prefix": "s3://encrypted-bucket/",
+      "kmsKeyId": "kmsKey_10",
+    },
+    {
+      "user": "test.*",
+      "iamRole": "arn:aws:iam::123456789101:role/test_users"
+    },
+    {
+      "group": "finance",
+      "iamRole": "arn:aws:iam::123456789101:role/finance_users"
+    },
+    {
+      "iamRole": "arn:aws:iam::123456789101:role/default"
+    }
+  ]
+}
+
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

hive.s3.security-mapping.config-file

The JSON configuration file or REST-endpoint URI containing security mappings.

hive.s3.security-mapping.json-pointer

A JSON pointer (RFC 6901) to mappings inside the JSON retrieved from the config file or REST-endpont. The whole document (“”) by default.

hive.s3.security-mapping.iam-role-credential-name

The name of the extra credential used to provide the IAM role.

hive.s3.security-mapping.kms-key-id-credential-name

The name of the extra credential used to provide the KMS-managed key ID.

hive.s3.security-mapping.refresh-period

How often to refresh the security mapping configuration.

hive.s3.security-mapping.colon-replacement

The character or characters to be used in place of the colon (:) character when specifying an IAM role name as an extra credential. Any instances of this replacement value in the extra credential value will be converted to a colon. Choose a value that is not used in any of your IAM ARNs.

+
+
+

Tuning properties#

+

The following tuning properties affect the behavior of the client +used by the Trino S3 filesystem when communicating with S3. +Most of these parameters affect settings on the ClientConfiguration +object associated with the AmazonS3Client.

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

Default

hive.s3.max-error-retries

Maximum number of error retries, set on the S3 client.

10

hive.s3.max-client-retries

Maximum number of read attempts to retry.

5

hive.s3.max-backoff-time

Use exponential backoff starting at 1 second up to this maximum value when communicating with S3.

10 minutes

hive.s3.max-retry-time

Maximum time to retry communicating with S3.

10 minutes

hive.s3.connect-timeout

TCP connect timeout.

5 seconds

hive.s3.connect-ttl

TCP connect TTL, which affects connection reusage.

Connections do not expire.

hive.s3.socket-timeout

TCP socket read timeout.

5 seconds

hive.s3.max-connections

Maximum number of simultaneous open connections to S3.

500

hive.s3.multipart.min-file-size

Minimum file size before multi-part upload to S3 is used.

16 MB

hive.s3.multipart.min-part-size

Minimum multi-part upload part size.

5 MB

+
+
+

S3 data encryption#

+

Trino supports reading and writing encrypted data in S3 using both +server-side encryption with S3 managed keys and client-side encryption using +either the Amazon KMS or a software plugin to manage AES encryption keys.

+

With S3 server-side encryption, +called SSE-S3 in the Amazon documentation, the S3 infrastructure takes care of all encryption and decryption +work. One exception is SSL to the client, assuming you have hive.s3.ssl.enabled set to true. +S3 also manages all the encryption keys for you. To enable this, set hive.s3.sse.enabled to true.

+

With S3 client-side encryption, +S3 stores encrypted data and the encryption keys are managed outside of the S3 infrastructure. Data is encrypted +and decrypted by Trino instead of in the S3 infrastructure. In this case, encryption keys can be managed +either by using the AWS KMS, or your own key management system. To use the AWS KMS for key management, set +hive.s3.kms-key-id to the UUID of a KMS key. Your AWS credentials or EC2 IAM role will need to be +granted permission to use the given key as well.

+

To use a custom encryption key management system, set hive.s3.encryption-materials-provider to the +fully qualified name of a class which implements the +EncryptionMaterialsProvider +interface from the AWS Java SDK. This class has to be accessible to the Hive Connector through the +classpath and must be able to communicate with your custom key management system. If this class also implements +the org.apache.hadoop.conf.Configurable interface from the Hadoop Java API, then the Hadoop configuration +is passed in after the object instance is created, and before it is asked to provision or retrieve any +encryption keys.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/hive-security.html b/430/connector/hive-security.html new file mode 100644 index 000000000..6e596115b --- /dev/null +++ b/430/connector/hive-security.html @@ -0,0 +1,1217 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Hive connector security configuration — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Hive connector security configuration#

+
+

Overview#

+

The Hive connector supports both authentication and authorization.

+

Trino can impersonate the end user who is running a query. In the case of a +user running a query from the command line interface, the end user is the +username associated with the Trino CLI process or argument to the optional +--user option.

+

Authentication can be configured with or without user impersonation on +Kerberized Hadoop clusters.

+
+
+

Requirements#

+

End user authentication limited to Kerberized Hadoop clusters. Authentication +user impersonation is available for both Kerberized and non-Kerberized clusters.

+

You must ensure that you meet the Kerberos, user impersonation and keytab +requirements described in this section that apply to your configuration.

+
+

Kerberos#

+

In order to use the Hive connector with a Hadoop cluster that uses kerberos +authentication, you must configure the connector to work with two services on +the Hadoop cluster:

+
    +
  • The Hive metastore Thrift service

  • +
  • The Hadoop Distributed File System (HDFS)

  • +
+

Access to these services by the Hive connector is configured in the properties +file that contains the general Hive connector configuration.

+

Kerberos authentication by ticket cache is not yet supported.

+
+

Note

+

If your krb5.conf location is different from /etc/krb5.conf you +must set it explicitly using the java.security.krb5.conf JVM property +in jvm.config file.

+

Example: -Djava.security.krb5.conf=/example/path/krb5.conf.

+
+
+

Warning

+

Access to the Trino coordinator must be secured e.g., using Kerberos or +password authentication, when using Kerberos authentication to Hadoop services. +Failure to secure access to the Trino coordinator could result in unauthorized +access to sensitive data on the Hadoop cluster. Refer to Security for +further information.

+

See Kerberos authentication for information on setting up Kerberos authentication.

+
+
+

Keytab files#

+

Keytab files contain encryption keys that are used to authenticate principals +to the Kerberos KDC. These encryption keys +must be stored securely; you must take the same precautions to protect them +that you take to protect ssh private keys.

+

In particular, access to keytab files must be limited to only the accounts +that must use them to authenticate. In practice, this is the user that +the Trino process runs as. The ownership and permissions on keytab files +must be set to prevent other users from reading or modifying the files.

+

Keytab files must be distributed to every node running Trino. Under common +deployment situations, the Hive connector configuration is the same on all +nodes. This means that the keytab needs to be in the same location on every +node.

+

You must ensure that the keytab files have the correct permissions on every +node after distributing them.

+
+
+
+

Impersonation in Hadoop#

+

In order to use impersonation, the Hadoop cluster must be +configured to allow the user or principal that Trino is running as to +impersonate the users who log in to Trino. Impersonation in Hadoop is +configured in the file core-site.xml. A complete description of the +configuration options can be found in the Hadoop documentation.

+
+
+
+

Authentication#

+

The default security configuration of the Hive connector does not use +authentication when connecting to a Hadoop cluster. All queries are executed as +the user who runs the Trino process, regardless of which user submits the +query.

+

The Hive connector provides additional security options to support Hadoop +clusters that have been configured to use Kerberos.

+

When accessing HDFS, Trino can +impersonate the end user who is running the +query. This can be used with HDFS permissions and ACLs to provide additional security for data.

+
+

Hive metastore Thrift service authentication#

+

In a Kerberized Hadoop cluster, Trino connects to the Hive metastore Thrift +service using SASL and +authenticates using Kerberos. Kerberos authentication for the metastore is +configured in the connector’s properties file using the following optional +properties:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Hive metastore Thrift service authentication properties#

Property value

Description

Default

hive.metastore.authentication.type

Hive metastore authentication type. One of NONE or KERBEROS. When +using the default value of NONE, Kerberos authentication is disabled, +and no other properties must be configured.

+

When set to KERBEROS the Hive connector connects to the Hive metastore +Thrift service using SASL and authenticate using Kerberos.

+

NONE

hive.metastore.thrift.impersonation.enabled

Enable Hive metastore end user impersonation. See +KERBEROS authentication with impersonation for more information.

false

hive.metastore.service.principal

The Kerberos principal of the Hive metastore service. The coordinator +uses this to authenticate the Hive metastore.

+

The _HOST placeholder can be used in this property value. When +connecting to the Hive metastore, the Hive connector substitutes in the +hostname of the metastore server it is connecting to. This is useful +if the metastore runs on multiple hosts.

+

Example: hive/hive-server-host@EXAMPLE.COM or +hive/_HOST@EXAMPLE.COM.

+

hive.metastore.client.principal

The Kerberos principal that Trino uses when connecting to the Hive +metastore service.

+

Example: trino/trino-server-node@EXAMPLE.COM or +trino/_HOST@EXAMPLE.COM.

+

The _HOST placeholder can be used in this property value. When +connecting to the Hive metastore, the Hive connector substitutes in the +hostname of the worker node Trino is running on. This is useful if +each worker node has its own Kerberos principal.

+

Unless KERBEROS authentication with impersonation is enabled, +the principal specified by hive.metastore.client.principal must have +sufficient privileges to remove files and directories within the +hive/warehouse directory.

+

Warning: If the principal does have sufficient permissions, only the +metadata is removed, and the data continues to consume disk space. This +occurs because the Hive metastore is responsible for deleting the +internal table data. When the metastore is configured to use Kerberos +authentication, all of the HDFS operations performed by the metastore are +impersonated. Errors deleting data are silently ignored.

+

hive.metastore.client.keytab

The path to the keytab file that contains a key for the principal +specified by hive.metastore.client.principal. This file must be +readable by the operating system user running Trino.

+
+

Configuration examples#

+

The following sections describe the configuration properties and values needed +for the various authentication configurations needed to use the Hive metastore +Thrift service with the Hive connector.

+
+
Default NONE authentication without impersonation#
+
hive.metastore.authentication.type=NONE
+
+
+

The default authentication type for the Hive metastore is NONE. When the +authentication type is NONE, Trino connects to an unsecured Hive +metastore. Kerberos is not used.

+
+
+
KERBEROS authentication with impersonation#
+
hive.metastore.authentication.type=KERBEROS
+hive.metastore.thrift.impersonation.enabled=true
+hive.metastore.service.principal=hive/hive-metastore-host.example.com@EXAMPLE.COM
+hive.metastore.client.principal=trino@EXAMPLE.COM
+hive.metastore.client.keytab=/etc/trino/hive.keytab
+
+
+

When the authentication type for the Hive metastore Thrift service is +KERBEROS, Trino connects as the Kerberos principal specified by the +property hive.metastore.client.principal. Trino authenticates this +principal using the keytab specified by the hive.metastore.client.keytab +property, and verifies that the identity of the metastore matches +hive.metastore.service.principal.

+

When using KERBEROS Metastore authentication with impersonation, the +principal specified by the hive.metastore.client.principal property must be +allowed to impersonate the current Trino user, as discussed in the section +Impersonation in Hadoop.

+

Keytab files must be distributed to every node in the cluster that runs Trino.

+

Additional Information About Keytab Files.

+
+
+
+
+

HDFS authentication#

+

In a Kerberized Hadoop cluster, Trino authenticates to HDFS using Kerberos. +Kerberos authentication for HDFS is configured in the connector’s properties +file using the following optional properties:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
HDFS authentication properties#

Property value

Description

Default

hive.hdfs.authentication.type

HDFS authentication type; one of NONE or KERBEROS. When using the +default value of NONE, Kerberos authentication is disabled, and no +other properties must be configured.

+

When set to KERBEROS, the Hive connector authenticates to HDFS using +Kerberos.

+

NONE

hive.hdfs.impersonation.enabled

Enable HDFS end-user impersonation. Impersonating the end user can provide +additional security when accessing HDFS if HDFS permissions or ACLs are +used.

+

HDFS Permissions and ACLs are explained in the HDFS Permissions Guide.

+

false

hive.hdfs.trino.principal

The Kerberos principal Trino uses when connecting to HDFS.

+

Example: trino-hdfs-superuser/trino-server-node@EXAMPLE.COM or +trino-hdfs-superuser/_HOST@EXAMPLE.COM.

+

The _HOST placeholder can be used in this property value. When +connecting to HDFS, the Hive connector substitutes in the hostname of the +worker node Trino is running on. This is useful if each worker node +has its own Kerberos principal.

+

hive.hdfs.trino.keytab

The path to the keytab file that contains a key for the principal +specified by hive.hdfs.trino.principal. This file must be readable by +the operating system user running Trino.

hive.hdfs.wire-encryption.enabled

Enable HDFS wire encryption. In a Kerberized Hadoop cluster that uses HDFS +wire encryption, this must be set to true to enable Trino to access +HDFS. Note that using wire encryption may impact query execution +performance.

+
+

Configuration examples#

+

The following sections describe the configuration properties and values needed +for the various authentication configurations with HDFS and the Hive connector.

+
+
Default NONE authentication without impersonation#
+
hive.hdfs.authentication.type=NONE
+
+
+

The default authentication type for HDFS is NONE. When the authentication +type is NONE, Trino connects to HDFS using Hadoop’s simple authentication +mechanism. Kerberos is not used.

+
+
+
NONE authentication with impersonation#
+
hive.hdfs.authentication.type=NONE
+hive.hdfs.impersonation.enabled=true
+
+
+

When using NONE authentication with impersonation, Trino impersonates +the user who is running the query when accessing HDFS. The user Trino is +running as must be allowed to impersonate this user, as discussed in the +section Impersonation in Hadoop. Kerberos is not used.

+
+
+
KERBEROS authentication without impersonation#
+
hive.hdfs.authentication.type=KERBEROS
+hive.hdfs.trino.principal=hdfs@EXAMPLE.COM
+hive.hdfs.trino.keytab=/etc/trino/hdfs.keytab
+
+
+

When the authentication type is KERBEROS, Trino accesses HDFS as the +principal specified by the hive.hdfs.trino.principal property. Trino +authenticates this principal using the keytab specified by the +hive.hdfs.trino.keytab keytab.

+

Keytab files must be distributed to every node in the cluster that runs Trino.

+

Additional Information About Keytab Files.

+
+
+
KERBEROS authentication with impersonation#
+
hive.hdfs.authentication.type=KERBEROS
+hive.hdfs.impersonation.enabled=true
+hive.hdfs.trino.principal=trino@EXAMPLE.COM
+hive.hdfs.trino.keytab=/etc/trino/hdfs.keytab
+
+
+

When using KERBEROS authentication with impersonation, Trino impersonates +the user who is running the query when accessing HDFS. The principal +specified by the hive.hdfs.trino.principal property must be allowed to +impersonate the current Trino user, as discussed in the section +Impersonation in Hadoop. Trino authenticates +hive.hdfs.trino.principal using the keytab specified by +hive.hdfs.trino.keytab.

+

Keytab files must be distributed to every node in the cluster that runs Trino.

+

Additional Information About Keytab Files.

+
+
+
+
+
+

Authorization#

+

You can enable authorization checks for the Hive connector by setting +the hive.security property in the Hive catalog properties file. This +property must be one of the following values:

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + +
hive.security property values#

Property value

Description

legacy (default value)

Few authorization checks are enforced, thus allowing most operations. The +config properties hive.allow-drop-table, hive.allow-rename-table, +hive.allow-add-column, hive.allow-drop-column and +hive.allow-rename-column are used.

read-only

Operations that read data or metadata, such as SELECT, are permitted, +but none of the operations that write data or metadata, such as +CREATE, INSERT or DELETE, are allowed.

file

Authorization checks are enforced using a catalog-level access control +configuration file whose path is specified in the security.config-file +catalog configuration property. See +Catalog-level access control files for details.

sql-standard

Users are permitted to perform the operations as long as they have the +required privileges as per the SQL standard. In this mode, Trino enforces +the authorization checks for queries based on the privileges defined in +Hive metastore. To alter these privileges, use the GRANT and +REVOKE commands.

+

See the SQL standard based authorization section for details.

+

allow-all

No authorization checks are enforced.

+
+

SQL standard based authorization#

+

When sql-standard security is enabled, Trino enforces the same SQL +standard-based authorization as Hive does.

+

Since Trino’s ROLE syntax support matches the SQL standard, and +Hive does not exactly follow the SQL standard, there are the following +limitations and differences:

+
    +
  • CREATE ROLE role WITH ADMIN is not supported.

  • +
  • The admin role must be enabled to execute CREATE ROLE, DROP ROLE or CREATE SCHEMA.

  • +
  • GRANT role TO user GRANTED BY someone is not supported.

  • +
  • REVOKE role FROM user GRANTED BY someone is not supported.

  • +
  • By default, all a user’s roles, except admin, are enabled in a new user session.

  • +
  • One particular role can be selected by executing SET ROLE role.

  • +
  • SET ROLE ALL enables all of a user’s roles except admin.

  • +
  • The admin role must be enabled explicitly by executing SET ROLE admin.

  • +
  • GRANT privilege ON SCHEMA schema is not supported. Schema ownership can be changed with ALTER SCHEMA schema SET AUTHORIZATION user

  • +
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/hive.html b/430/connector/hive.html new file mode 100644 index 000000000..b9d5dcdb3 --- /dev/null +++ b/430/connector/hive.html @@ -0,0 +1,2300 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Hive connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Hive connector#

+
+
+

The Hive connector allows querying data stored in an +Apache Hive +data warehouse. Hive is a combination of three components:

+
    +
  • Data files in varying formats, that are typically stored in the +Hadoop Distributed File System (HDFS) or in object storage systems +such as Amazon S3.

  • +
  • Metadata about how the data files are mapped to schemas and tables. This +metadata is stored in a database, such as MySQL, and is accessed via the Hive +metastore service.

  • +
  • A query language called HiveQL. This query language is executed on a +distributed computing framework such as MapReduce or Tez.

  • +
+

Trino only uses the first two components: the data and the metadata. +It does not use HiveQL or any part of Hive’s execution environment.

+
+

Requirements#

+

The Hive connector requires a +Hive metastore service (HMS), or a compatible +implementation of the Hive metastore, such as +AWS Glue.

+

Apache Hadoop HDFS 2.x and 3.x are supported.

+

Many distributed storage systems including HDFS, +Amazon S3 or S3-compatible systems, +Google Cloud Storage, +Azure Storage, and +IBM Cloud Object Storage can be queried with the Hive +connector.

+

The coordinator and all workers must have network access to the Hive metastore +and the storage system. Hive metastore access with the Thrift protocol defaults +to using port 9083.

+

Data files must be in a supported file format. Some file formats can be +configured using file format configuration properties per catalog:

+
    +
  • ORC

  • +
  • Parquet

  • +
  • Avro

  • +
  • RCText (RCFile using ColumnarSerDe)

  • +
  • RCBinary (RCFile using LazyBinaryColumnarSerDe)

  • +
  • SequenceFile

  • +
  • JSON (using org.apache.hive.hcatalog.data.JsonSerDe)

  • +
  • CSV (using org.apache.hadoop.hive.serde2.OpenCSVSerde)

  • +
  • TextFile

  • +
+
+
+

General configuration#

+

To configure the Hive connector, create a catalog properties file +etc/catalog/example.properties that references the hive +connector and defines a metastore. You must configure a metastore for table +metadata. If you are using a Hive metastore, +hive.metastore.uri must be configured:

+
connector.name=hive
+hive.metastore.uri=thrift://example.net:9083
+
+
+

If you are using AWS Glue as your metastore, you +must instead set hive.metastore to glue:

+
connector.name=hive
+hive.metastore=glue
+
+
+

Each metastore type has specific configuration properties along with +general metastore configuration properties.

+
+

Multiple Hive clusters#

+

You can have as many catalogs as you need, so if you have additional +Hive clusters, simply add another properties file to etc/catalog +with a different name, making sure it ends in .properties. For +example, if you name the property file sales.properties, Trino +creates a catalog named sales using the configured connector.

+
+
+

HDFS configuration#

+

For basic setups, Trino configures the HDFS client automatically and +does not require any configuration files. In some cases, such as when using +federated HDFS or NameNode high availability, it is necessary to specify +additional HDFS client options in order to access your HDFS cluster. To do so, +add the hive.config.resources property to reference your HDFS config files:

+
hive.config.resources=/etc/hadoop/conf/core-site.xml,/etc/hadoop/conf/hdfs-site.xml
+
+
+

Only specify additional configuration files if necessary for your setup. +We recommend reducing the configuration files to have the minimum +set of required properties, as additional properties may cause problems.

+

The configuration files must exist on all Trino nodes. If you are +referencing existing Hadoop config files, make sure to copy them to +any Trino nodes that are not running Hadoop.

+
+
+

HDFS username and permissions#

+

Before running any CREATE TABLE or CREATE TABLE AS statements +for Hive tables in Trino, you must check that the user Trino is +using to access HDFS has access to the Hive warehouse directory. The Hive +warehouse directory is specified by the configuration variable +hive.metastore.warehouse.dir in hive-site.xml, and the default +value is /user/hive/warehouse.

+

When not using Kerberos with HDFS, Trino accesses HDFS using the +OS user of the Trino process. For example, if Trino is running as +nobody, it accesses HDFS as nobody. You can override this +username by setting the HADOOP_USER_NAME system property in the +Trino JVM config, replacing hdfs_user with the +appropriate username:

+
-DHADOOP_USER_NAME=hdfs_user
+
+
+

The hive user generally works, since Hive is often started with +the hive user and this user has access to the Hive warehouse.

+

Whenever you change the user Trino is using to access HDFS, remove +/tmp/presto-* on HDFS, as the new user may not have access to +the existing temporary directories.

+
+
+

Hive general configuration properties#

+

The following table lists general configuration properties for the Hive +connector. There are additional sets of configuration properties throughout the +Hive connector documentation.

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Hive general configuration properties#

Property Name

Description

Default

hive.config.resources

An optional comma-separated list of HDFS configuration files. These +files must exist on the machines running Trino. Only specify this if +absolutely necessary to access HDFS. Example: /etc/hdfs-site.xml

hive.recursive-directories

Enable reading data from subdirectories of table or partition locations. +If disabled, subdirectories are ignored. This is equivalent to the +hive.mapred.supports.subdirectories property in Hive.

false

hive.ignore-absent-partitions

Ignore partitions when the file system location does not exist rather +than failing the query. This skips data that may be expected to be part +of the table.

false

hive.storage-format

The default file format used when creating new tables.

ORC

hive.compression-codec

The compression codec to use when writing files. Possible values are +NONE, SNAPPY, LZ4, ZSTD, or GZIP.

GZIP

hive.force-local-scheduling

Force splits to be scheduled on the same node as the Hadoop DataNode +process serving the split data. This is useful for installations where +Trino is collocated with every DataNode.

false

hive.respect-table-format

Should new partitions be written using the existing table format or the +default Trino format?

true

hive.immutable-partitions

Can new data be inserted into existing partitions? If true then +setting hive.insert-existing-partitions-behavior to APPEND is +not allowed. This also affects the insert_existing_partitions_behavior +session property in the same way.

false

hive.insert-existing-partitions-behavior

What happens when data is inserted into an existing partition? Possible +values are

+
+
    +
  • APPEND - appends data to existing partitions

  • +
  • OVERWRITE - overwrites existing partitions

  • +
  • ERROR - modifying existing partitions is not allowed

  • +
+
+

APPEND

hive.target-max-file-size

Best effort maximum size of new files.

1GB

hive.create-empty-bucket-files

Should empty files be created for buckets that have no data?

false

hive.validate-bucketing

Enables validation that data is in the correct bucket when reading +bucketed tables.

true

hive.partition-statistics-sample-size

Specifies the number of partitions to analyze when computing table +statistics.

100

hive.max-partitions-per-writers

Maximum number of partitions per writer.

100

hive.max-partitions-for-eager-load

The maximum number of partitions for a single table scan to load eagerly +on the coordinator. Certain optimizations are not possible without eager +loading.

100,000

hive.max-partitions-per-scan

Maximum number of partitions for a single table scan.

1,000,000

hive.dfs.replication

Hadoop file system replication factor.

hive.security

See Hive connector security configuration.

security.config-file

Path of config file to use when hive.security=file. See +Catalog-level access control files for details.

hive.non-managed-table-writes-enabled

Enable writes to non-managed (external) Hive tables.

false

hive.non-managed-table-creates-enabled

Enable creating non-managed (external) Hive tables.

true

hive.collect-column-statistics-on-write

Enables automatic column level statistics collection on write. See +Table Statistics for details.

true

hive.file-status-cache-tables

Cache directory listing for specific tables. Examples:

+
+
    +
  • fruit.apple,fruit.orange to cache listings only for tables +apple and orange in schema fruit

  • +
  • fruit.*,vegetable.* to cache listings for all tables +in schemas fruit and vegetable

  • +
  • * to cache listings for all tables in all schemas

  • +
+
+

hive.file-status-cache.max-retained-size

Maximum retained size of cached file status entries.

1GB

hive.file-status-cache-expire-time

How long a cached directory listing is considered valid.

1m

hive.per-transaction-file-status-cache.max-retained-size

Maximum retained size of all entries in per transaction file status cache. +Retained size limit is shared across all running queries.

100MB

hive.rcfile.time-zone

Adjusts binary encoded timestamp values to a specific time zone. For +Hive 3.1+, this must be set to UTC.

JVM default

hive.timestamp-precision

Specifies the precision to use for Hive columns of type TIMESTAMP. +Possible values are MILLISECONDS, MICROSECONDS and NANOSECONDS. +Values with higher precision than configured are rounded.

MILLISECONDS

hive.temporary-staging-directory-enabled

Controls whether the temporary staging directory configured at +hive.temporary-staging-directory-path is used for write +operations. Temporary staging directory is never used for writes to +non-sorted tables on S3, encrypted HDFS or external location. Writes to +sorted tables will utilize this path for staging temporary files during +sorting operation. When disabled, the target storage will be used for +staging while writing sorted tables which can be inefficient when +writing to object stores like S3.

true

hive.temporary-staging-directory-path

Controls the location of temporary staging directory that is used for +write operations. The ${USER} placeholder can be used to use a +different location for each user.

/tmp/presto-${USER}

hive.hive-views.enabled

Enable translation for Hive views.

false

hive.hive-views.legacy-translation

Use the legacy algorithm to translate Hive views. +You can use the hive_views_legacy_translation catalog session +property for temporary, catalog specific use.

false

hive.parallel-partitioned-bucketed-writes

Improve parallelism of partitioned and bucketed table writes. When +disabled, the number of writing threads is limited to number of buckets.

true

hive.fs.new-directory-permissions

Controls the permissions set on new directories created for tables. It +must be either ‘skip’ or an octal number, with a leading 0. If set to +‘skip’, permissions of newly created directories will not be set by +Trino.

0777

hive.fs.cache.max-size

Maximum number of cached file system objects.

1000

hive.query-partition-filter-required

Set to true to force a query to use a partition filter. You can use +the query_partition_filter_required catalog session property for +temporary, catalog specific use.

false

hive.table-statistics-enabled

Enables Table statistics. The equivalent +catalog session property is +statistics_enabled for session specific use. Set to false to +disable statistics. Disabling statistics means that +Cost-based optimizations can not make smart decisions +about the query plan.

true

hive.auto-purge

Set the default value for the auto_purge table property for managed +tables. See the Table properties for more information on +auto_purge.

false

hive.partition-projection-enabled

Enables Athena partition projection support

false

hive.max-partition-drops-per-query

Maximum number of partitions to drop in a single query.

100,000

hive.single-statement-writes

Enables auto-commit for all writes. This can be used to disallow +multi-statement write transactions.

false

+
+
+
+

Storage#

+

The Hive connector supports the following storage options:

+ +

The Hive connector also supports storage caching.

+
+
+

Security#

+

Please see the Hive connector security configuration section for information on the +security options available for the Hive connector.

+
+
+

SQL support#

+

The connector provides read access and write access to data and metadata in the +configured object storage system and metadata stores:

+ +

Refer to the migration guide for practical advice +on migrating from Hive to Trino.

+

The following sections provide Hive-specific information regarding SQL support.

+
+

Basic usage examples#

+

The examples shown here work on Google Cloud Storage by replacing s3:// with +gs://.

+

Create a new Hive table named page_views in the web schema +that is stored using the ORC file format, partitioned by date and +country, and bucketed by user into 50 buckets. Note that Hive +requires the partition columns to be the last columns in the table:

+
CREATE TABLE example.web.page_views (
+  view_time TIMESTAMP,
+  user_id BIGINT,
+  page_url VARCHAR,
+  ds DATE,
+  country VARCHAR
+)
+WITH (
+  format = 'ORC',
+  partitioned_by = ARRAY['ds', 'country'],
+  bucketed_by = ARRAY['user_id'],
+  bucket_count = 50
+)
+
+
+

Create a new Hive schema named web that stores tables in an +S3 bucket named my-bucket:

+
CREATE SCHEMA example.web
+WITH (location = 's3://my-bucket/')
+
+
+

Drop a schema:

+
DROP SCHEMA example.web
+
+
+

Drop a partition from the page_views table:

+
DELETE FROM example.web.page_views
+WHERE ds = DATE '2016-08-09'
+  AND country = 'US'
+
+
+

Query the page_views table:

+
SELECT * FROM example.web.page_views
+
+
+

List the partitions of the page_views table:

+
SELECT * FROM example.web."page_views$partitions"
+
+
+

Create an external Hive table named request_logs that points at +existing data in S3:

+
CREATE TABLE example.web.request_logs (
+  request_time TIMESTAMP,
+  url VARCHAR,
+  ip VARCHAR,
+  user_agent VARCHAR
+)
+WITH (
+  format = 'TEXTFILE',
+  external_location = 's3://my-bucket/data/logs/'
+)
+
+
+

Collect statistics for the request_logs table:

+
ANALYZE example.web.request_logs;
+
+
+

Drop the external table request_logs. This only drops the metadata +for the table. The referenced data directory is not deleted:

+
DROP TABLE example.web.request_logs
+
+
+
    +
  • CREATE TABLE AS can be used to create transactional tables in ORC format like this:

    +
    CREATE TABLE <name>
    +WITH (
    +    format='ORC',
    +    transactional=true
    +)
    +AS <query>
    +
    +
    +
  • +
+

Add an empty partition to the page_views table:

+
CALL system.create_empty_partition(
+    schema_name => 'web',
+    table_name => 'page_views',
+    partition_columns => ARRAY['ds', 'country'],
+    partition_values => ARRAY['2016-08-09', 'US']);
+
+
+

Drop stats for a partition of the page_views table:

+
CALL system.drop_stats(
+    schema_name => 'web',
+    table_name => 'page_views',
+    partition_values => ARRAY[ARRAY['2016-08-09', 'US']]);
+
+
+
+
+

Procedures#

+

Use the CALL statement to perform data manipulation or +administrative tasks. Procedures must include a qualified catalog name, if your +Hive catalog is called web:

+
CALL web.system.example_procedure()
+
+
+

The following procedures are available:

+
    +
  • system.create_empty_partition(schema_name, table_name, partition_columns, partition_values)

    +

    Create an empty partition in the specified table.

    +
  • +
  • system.sync_partition_metadata(schema_name, table_name, mode, case_sensitive)

    +

    Check and update partitions list in metastore. There are three modes available:

    +
      +
    • ADD : add any partitions that exist on the file system, but not in the metastore.

    • +
    • DROP: drop any partitions that exist in the metastore, but not on the file system.

    • +
    • FULL: perform both ADD and DROP.

    • +
    +

    The case_sensitive argument is optional. The default value is true for compatibility +with Hive’s MSCK REPAIR TABLE behavior, which expects the partition column names in +file system paths to use lowercase (e.g. col_x=SomeValue). Partitions on the file system +not conforming to this convention are ignored, unless the argument is set to false.

    +
  • +
  • system.drop_stats(schema_name, table_name, partition_values)

    +

    Drops statistics for a subset of partitions or the entire table. The partitions are specified as an +array whose elements are arrays of partition values (similar to the partition_values argument in +create_empty_partition). If partition_values argument is omitted, stats are dropped for the +entire table.

    +
  • +
+
    +
  • system.register_partition(schema_name, table_name, partition_columns, partition_values, location)

    +

    Registers existing location as a new partition in the metastore for the specified table.

    +

    When the location argument is omitted, the partition location is +constructed using partition_columns and partition_values.

    +

    Due to security reasons, the procedure is enabled only when hive.allow-register-partition-procedure +is set to true.

    +
  • +
+
    +
  • system.unregister_partition(schema_name, table_name, partition_columns, partition_values)

    +

    Unregisters given, existing partition in the metastore for the specified table. +The partition data is not deleted.

    +
  • +
+
    +
  • system.flush_metadata_cache()

    +

    Flush all Hive metadata caches.

    +
  • +
  • system.flush_metadata_cache(schema_name => ..., table_name => ...)

    +

    Flush Hive metadata caches entries connected with selected table. +Procedure requires named parameters to be passed

    +
  • +
  • system.flush_metadata_cache(schema_name => ..., table_name => ..., partition_columns => ARRAY[...], partition_values => ARRAY[...])

    +

    Flush Hive metadata cache entries connected with selected partition. +Procedure requires named parameters to be passed.

    +
  • +
+
+
+

Data management#

+

Some data management statements may be affected by +the Hive catalog’s authorization check policy. In the default legacy policy, +some statements are disabled by default. See Hive connector security configuration for more +information.

+

The Data management functionality includes support for INSERT, +UPDATE, DELETE, and MERGE statements, with the exact support +depending on the storage system, file format, and metastore.

+

When connecting to a Hive metastore version 3.x, the Hive connector supports +reading from and writing to insert-only and ACID tables, with full support for +partitioning and bucketing.

+

DELETE applied to non-transactional tables is only supported if the +table is partitioned and the WHERE clause matches entire partitions. +Transactional Hive tables with ORC format support “row-by-row” deletion, in +which the WHERE clause may match arbitrary sets of rows.

+

UPDATE is only supported for transactional Hive tables with format +ORC. UPDATE of partition or bucket columns is not supported.

+

MERGE is only supported for ACID tables.

+

ACID tables created with Hive Streaming Ingest +are not supported.

+
+
+

Schema and table management#

+

The Hive connector supports querying and manipulating Hive tables and schemas +(databases). While some uncommon operations must be performed using +Hive directly, most operations can be performed using Trino.

+
+

Schema evolution#

+

Hive allows the partitions in a table to have a different schema than the +table. This occurs when the column types of a table are changed after +partitions already exist (that use the original column types). The Hive +connector supports this by allowing the same conversions as Hive:

+
    +
  • VARCHAR to and from TINYINT, SMALLINT, INTEGER and BIGINT

  • +
  • REAL to DOUBLE

  • +
  • Widening conversions for integers, such as TINYINT to SMALLINT

  • +
+

Any conversion failure results in null, which is the same behavior +as Hive. For example, converting the string 'foo' to a number, +or converting the string '1234' to a TINYINT (which has a +maximum value of 127).

+
+
+

Avro schema evolution#

+

Trino supports querying and manipulating Hive tables with the Avro storage +format, which has the schema set based on an Avro schema file/literal. Trino is +also capable of creating the tables in Trino by infering the schema from a +valid Avro schema file located locally, or remotely in HDFS/Web server.

+

To specify that the Avro schema should be used for interpreting table data, use +the avro_schema_url table property.

+

The schema can be placed in the local file system or remotely in the following +locations:

+
    +
  • HDFS (e.g. avro_schema_url = 'hdfs://user/avro/schema/avro_data.avsc')

  • +
  • S3 (e.g. avro_schema_url = 's3n:///schema_bucket/schema/avro_data.avsc')

  • +
  • A web server (e.g. avro_schema_url = 'http://example.org/schema/avro_data.avsc')

  • +
+

The URL, where the schema is located, must be accessible from the Hive metastore +and Trino coordinator/worker nodes.

+

Alternatively, you can use the table property avro_schema_literal to define +the Avro schema.

+

The table created in Trino using the avro_schema_url or +avro_schema_literal property behaves the same way as a Hive table with +avro.schema.url or avro.schema.literal set.

+

Example:

+
CREATE TABLE example.avro.avro_data (
+   id BIGINT
+ )
+WITH (
+   format = 'AVRO',
+   avro_schema_url = '/usr/local/avro_data.avsc'
+)
+
+
+

The columns listed in the DDL (id in the above example) is ignored if avro_schema_url is specified. +The table schema matches the schema in the Avro schema file. Before any read operation, the Avro schema is +accessed so the query result reflects any changes in schema. Thus Trino takes advantage of Avro’s backward compatibility abilities.

+

If the schema of the table changes in the Avro schema file, the new schema can still be used to read old data. +Newly added/renamed fields must have a default value in the Avro schema file.

+

The schema evolution behavior is as follows:

+
    +
  • Column added in new schema: +Data created with an older schema produces a default value when table is using the new schema.

  • +
  • Column removed in new schema: +Data created with an older schema no longer outputs the data from the column that was removed.

  • +
  • Column is renamed in the new schema: +This is equivalent to removing the column and adding a new one, and data created with an older schema +produces a default value when table is using the new schema.

  • +
  • Changing type of column in the new schema: +If the type coercion is supported by Avro or the Hive connector, then the conversion happens. +An error is thrown for incompatible types.

  • +
+
+
Limitations#
+

The following operations are not supported when avro_schema_url is set:

+
    +
  • CREATE TABLE AS is not supported.

  • +
  • Bucketing(bucketed_by) columns are not supported in CREATE TABLE.

  • +
  • ALTER TABLE commands modifying columns are not supported.

  • +
+
+
+
+

ALTER TABLE EXECUTE#

+

The connector supports the following commands for use with ALTER TABLE EXECUTE.

+
+
optimize#
+

The optimize command is used for rewriting the content of the specified +table so that it is merged into fewer but larger files. If the table is +partitioned, the data compaction acts separately on each partition selected for +optimization. This operation improves read performance.

+

All files with a size below the optional file_size_threshold parameter +(default value for the threshold is 100MB) are merged:

+
ALTER TABLE test_table EXECUTE optimize
+
+
+

The following statement merges files in a table that are +under 10 megabytes in size:

+
ALTER TABLE test_table EXECUTE optimize(file_size_threshold => '10MB')
+
+
+

You can use a WHERE clause with the columns used to partition the table +to filter which partitions are optimized:

+
ALTER TABLE test_partitioned_table EXECUTE optimize
+WHERE partition_key = 1
+
+
+

The optimize command is disabled by default, and can be enabled for a +catalog with the <catalog-name>.non_transactional_optimize_enabled +session property:

+
SET SESSION <catalog_name>.non_transactional_optimize_enabled=true
+
+
+
+

Warning

+

Because Hive tables are non-transactional, take note of the following possible +outcomes:

+
    +
  • If queries are run against tables that are currently being optimized, +duplicate rows may be read.

  • +
  • In rare cases where exceptions occur during the optimize operation, +a manual cleanup of the table directory is needed. In this situation, refer +to the Trino logs and query failure messages to see which files must be +deleted.

  • +
+
+
+
+
+

Table properties#

+

Table properties supply or set metadata for the underlying tables. This +is key for CREATE TABLE AS statements. Table properties are passed +to the connector using a WITH clause:

+
CREATE TABLE tablename
+WITH (format='CSV',
+      csv_escape = '"')
+
+
+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Hive connector table properties#

Property name

Description

Default

auto_purge

Indicates to the configured metastore to perform a purge when a table or +partition is deleted instead of a soft deletion using the trash.

avro_schema_url

The URI pointing to Avro schema evolution for the table.

bucket_count

The number of buckets to group data into. Only valid if used with +bucketed_by.

0

bucketed_by

The bucketing column for the storage table. Only valid if used with +bucket_count.

[]

bucketing_version

Specifies which Hive bucketing version to use. Valid values are 1 +or 2.

csv_escape

The CSV escape character. Requires CSV format.

csv_quote

The CSV quote character. Requires CSV format.

csv_separator

The CSV separator character. Requires CSV format. You can use other +separators such as | or use Unicode to configure invisible separators +such tabs with U&'\0009'.

,

external_location

The URI for an external Hive table on S3, Azure Blob Storage, etc. See the +Basic usage examples for more information.

format

The table file format. Valid values include ORC, PARQUET, +AVRO, RCBINARY, RCTEXT, SEQUENCEFILE, JSON, +TEXTFILE, CSV, and REGEX. The catalog property +hive.storage-format sets the default value and can change it to a +different default.

null_format

The serialization format for NULL value. Requires TextFile, RCText, +or SequenceFile format.

orc_bloom_filter_columns

Comma separated list of columns to use for ORC bloom filter. It improves +the performance of queries using range predicates when reading ORC files. +Requires ORC format.

[]

orc_bloom_filter_fpp

The ORC bloom filters false positive probability. Requires ORC format.

0.05

partitioned_by

The partitioning column for the storage table. The columns listed in the +partitioned_by clause must be the last columns as defined in the DDL.

[]

skip_footer_line_count

The number of footer lines to ignore when parsing the file for data. +Requires TextFile or CSV format tables.

skip_header_line_count

The number of header lines to ignore when parsing the file for data. +Requires TextFile or CSV format tables.

sorted_by

The column to sort by to determine bucketing for row. Only valid if +bucketed_by and bucket_count are specified as well.

[]

textfile_field_separator

Allows the use of custom field separators, such as ‘|’, for TextFile +formatted tables.

textfile_field_separator_escape

Allows the use of a custom escape character for TextFile formatted tables.

transactional

Set this property to true to create an ORC ACID transactional table. +Requires ORC format. This property may be shown as true for insert-only +tables created using older versions of Hive.

partition_projection_enabled

Enables partition projection for selected table. +Mapped from AWS Athena table property +projection.enabled.

partition_projection_ignore

Ignore any partition projection properties stored in the metastore for +the selected table. This is a Trino-only property which allows you to +work around compatibility issues on a specific table, and if enabled, +Trino ignores all other configuration options related to partition +projection.

partition_projection_location_template

Projected partition location template, such as +s3a://test/name=${name}/. Mapped from the AWS Athena table property +storage.location.template

${table_location}/${partition_name}

extra_properties

Additional properties added to a Hive table. The properties are not used by Trino, +and are available in the $properties metadata table. +The properties are not included in the output of SHOW CREATE TABLE statements.

+
+
+

Metadata tables#

+

The raw Hive table properties are available as a hidden table, containing a +separate column per table property, with a single row containing the property +values.

+
+
$properties table#
+

The properties table name is composed with the table name and $properties appended. +It exposes the parameters of the table in the metastore.

+

You can inspect the property names and values with a simple query:

+
SELECT * FROM example.web."page_views$properties";
+
+
+
       stats_generated_via_stats_task        | auto.purge |       presto_query_id       | presto_version | transactional
+---------------------------------------------+------------+-----------------------------+----------------+---------------
+ workaround for potential lack of HIVE-12730 | false      | 20230705_152456_00001_nfugi | 423            | false
+
+
+
+
+
$partitions table#
+

The $partitions table provides a list of all partition values +of a partitioned table.

+

The following example query returns all partition values from the +page_views table in the web schema of the example catalog:

+
SELECT * FROM example.web."page_views$partitions";
+
+
+
     day    | country
+------------+---------
+ 2023-07-01 | POL
+ 2023-07-02 | POL
+ 2023-07-03 | POL
+ 2023-03-01 | USA
+ 2023-03-02 | USA
+
+
+
+
+
+

Column properties#

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Hive connector column properties#

Property name

Description

Default

partition_projection_type

Defines the type of partition projection to use on this column. +May be used only on partition columns. Available types: +ENUM, INTEGER, DATE, INJECTED. +Mapped from the AWS Athena table property +projection.${columnName}.type.

partition_projection_values

Used with partition_projection_type set to ENUM. Contains a static +list of values used to generate partitions. +Mapped from the AWS Athena table property +projection.${columnName}.values.

partition_projection_range

Used with partition_projection_type set to INTEGER or DATE to +define a range. It is a two-element array, describing the minimum and +maximum range values used to generate partitions. Generation starts from +the minimum, then increments by the defined +partition_projection_interval to the maximum. For example, the format +is ['1', '4'] for a partition_projection_type of INTEGER and +['2001-01-01', '2001-01-07'] or ['NOW-3DAYS', 'NOW'] for a +partition_projection_type of DATE. Mapped from the AWS Athena +table property +projection.${columnName}.range.

partition_projection_interval

Used with partition_projection_type set to INTEGER or DATE. It +represents the interval used to generate partitions within +the given range partition_projection_range. Mapped from the AWS Athena +table property +projection.${columnName}.interval.

partition_projection_digits

Used with partition_projection_type set to INTEGER. +The number of digits to be used with integer column projection. +Mapped from the AWS Athena table property +projection.${columnName}.digits.

partition_projection_format

Used with partition_projection_type set to DATE. +The date column projection format, defined as a string such as yyyy MM +or MM-dd-yy HH:mm:ss for use with the +Java DateTimeFormatter class. +Mapped from the AWS Athena table property +projection.${columnName}.format.

partition_projection_interval_unit

Used with partition_projection_type=DATA. +The date column projection range interval unit +given in partition_projection_interval. +Mapped from the AWS Athena table property +projection.${columnName}.interval.unit.

+
+
+

Metadata columns#

+

In addition to the defined columns, the Hive connector automatically exposes +metadata in a number of hidden columns in each table:

+
    +
  • $bucket: Bucket number for this row

  • +
  • $path: Full file system path name of the file for this row

  • +
  • $file_modified_time: Date and time of the last modification of the file for this row

  • +
  • $file_size: Size of the file for this row

  • +
  • $partition: Partition name for this row

  • +
+

You can use these columns in your SQL statements like any other column. They +can be selected directly, or used in conditional statements. For example, you +can inspect the file size, location and partition for each record:

+
SELECT *, "$path", "$file_size", "$partition"
+FROM example.web.page_views;
+
+
+

Retrieve all records that belong to files stored in the partition +ds=2016-08-09/country=US:

+
SELECT *, "$path", "$file_size"
+FROM example.web.page_views
+WHERE "$partition" = 'ds=2016-08-09/country=US'
+
+
+
+
+
+

View management#

+

Trino allows reading from Hive materialized views, and can be configured to +support reading Hive views.

+
+

Materialized views#

+

The Hive connector supports reading from Hive materialized views. +In Trino, these views are presented as regular, read-only tables.

+
+
+

Hive views#

+

Hive views are defined in HiveQL and stored in the Hive Metastore Service. They +are analyzed to allow read access to the data.

+

The Hive connector includes support for reading Hive views with three different +modes.

+
    +
  • Disabled

  • +
  • Legacy

  • +
  • Experimental

  • +
+

If using Hive views from Trino is required, you must compare results in Hive and +Trino for each view definition to ensure identical results. Use the experimental +mode whenever possible. Avoid using the legacy mode. Leave Hive views support +disabled, if you are not accessing any Hive views from Trino.

+

You can configure the behavior in your catalog properties file.

+

By default, Hive views are executed with the RUN AS DEFINER security mode. +Set the hive.hive-views.run-as-invoker catalog configuration property to +true to use RUN AS INVOKER semantics.

+

Disabled

+

The default behavior is to ignore Hive views. This means that your business +logic and data encoded in the views is not available in Trino.

+

Legacy

+

A very simple implementation to execute Hive views, and therefore allow read +access to the data in Trino, can be enabled with +hive.hive-views.enabled=true and +hive.hive-views.legacy-translation=true.

+

For temporary usage of the legacy behavior for a specific catalog, you can set +the hive_views_legacy_translation catalog session property to true.

+

This legacy behavior interprets any HiveQL query that defines a view as if it +is written in SQL. It does not do any translation, but instead relies on the +fact that HiveQL is very similar to SQL.

+

This works for very simple Hive views, but can lead to problems for more complex +queries. For example, if a HiveQL function has an identical signature but +different behaviors to the SQL version, the returned results may differ. In more +extreme cases the queries might fail, or not even be able to be parsed and +executed.

+

Experimental

+

The new behavior is better engineered and has the potential to become a lot +more powerful than the legacy implementation. It can analyze, process, and +rewrite Hive views and contained expressions and statements.

+

It supports the following Hive view functionality:

+
    +
  • UNION [DISTINCT] and UNION ALL against Hive views

  • +
  • Nested GROUP BY clauses

  • +
  • current_user()

  • +
  • LATERAL VIEW OUTER EXPLODE

  • +
  • LATERAL VIEW [OUTER] EXPLODE on array of struct

  • +
  • LATERAL VIEW json_tuple

  • +
+

You can enable the experimental behavior with +hive.hive-views.enabled=true. Remove the +hive.hive-views.legacy-translation property or set it to false to make +sure legacy is not enabled.

+

Keep in mind that numerous features are not yet implemented when experimenting +with this feature. The following is an incomplete list of missing +functionality:

+
    +
  • HiveQL current_date, current_timestamp, and others

  • +
  • Hive function calls including translate(), window functions, and others

  • +
  • Common table expressions and simple case expressions

  • +
  • Honor timestamp precision setting

  • +
  • Support all Hive data types and correct mapping to Trino types

  • +
  • Ability to process custom UDFs

  • +
+
+
+
+
+

Fault-tolerant execution support#

+

The connector supports Fault-tolerant execution of query +processing. Read and write operations are both supported with any retry policy +on non-transactional tables.

+

Read operations are supported with any retry policy on transactional tables. +Write operations and CREATE TABLE ... AS operations are not supported with +any retry policy on transactional tables.

+
+
+

Performance#

+

The connector includes a number of performance improvements, detailed in the +following sections.

+
+

Table statistics#

+

The Hive connector supports collecting and managing table statistics to improve query processing performance.

+

When writing data, the Hive connector always collects basic statistics +(numFiles, numRows, rawDataSize, totalSize) +and by default will also collect column level statistics:

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Available table statistics#

Column type

Collectible statistics

TINYINT

Number of nulls, number of distinct values, min/max values

SMALLINT

Number of nulls, number of distinct values, min/max values

INTEGER

Number of nulls, number of distinct values, min/max values

BIGINT

Number of nulls, number of distinct values, min/max values

DOUBLE

Number of nulls, number of distinct values, min/max values

REAL

Number of nulls, number of distinct values, min/max values

DECIMAL

Number of nulls, number of distinct values, min/max values

DATE

Number of nulls, number of distinct values, min/max values

TIMESTAMP

Number of nulls, number of distinct values, min/max values

VARCHAR

Number of nulls, number of distinct values

CHAR

Number of nulls, number of distinct values

VARBINARY

Number of nulls

BOOLEAN

Number of nulls, number of true/false values

+
+

Updating table and partition statistics#

+

If your queries are complex and include joining large data sets, +running ANALYZE on tables/partitions may improve query performance +by collecting statistical information about the data.

+

When analyzing a partitioned table, the partitions to analyze can be specified +via the optional partitions property, which is an array containing +the values of the partition keys in the order they are declared in the table schema:

+
ANALYZE table_name WITH (
+    partitions = ARRAY[
+        ARRAY['p1_value1', 'p1_value2'],
+        ARRAY['p2_value1', 'p2_value2']])
+
+
+

This query will collect statistics for two partitions with keys +p1_value1, p1_value2 and p2_value1, p2_value2.

+

On wide tables, collecting statistics for all columns can be expensive and can have a +detrimental effect on query planning. It is also typically unnecessary - statistics are +only useful on specific columns, like join keys, predicates, grouping keys. One can +specify a subset of columns to be analyzed via the optional columns property:

+
ANALYZE table_name WITH (
+    partitions = ARRAY[ARRAY['p2_value1', 'p2_value2']],
+    columns = ARRAY['col_1', 'col_2'])
+
+
+

This query collects statistics for columns col_1 and col_2 for the partition +with keys p2_value1, p2_value2.

+

Note that if statistics were previously collected for all columns, they must be dropped +before re-analyzing just a subset:

+
CALL system.drop_stats('schema_name', 'table_name')
+
+
+

You can also drop statistics for selected partitions only:

+
CALL system.drop_stats(
+    schema_name => 'schema',
+    table_name => 'table',
+    partition_values => ARRAY[ARRAY['p2_value1', 'p2_value2']])
+
+
+
+
+
+

Dynamic filtering#

+

The Hive connector supports the dynamic filtering optimization. +Dynamic partition pruning is supported for partitioned tables stored in any file format +for broadcast as well as partitioned joins. +Dynamic bucket pruning is supported for bucketed tables stored in any file format for +broadcast joins only.

+

For tables stored in ORC or Parquet file format, dynamic filters are also pushed into +local table scan on worker nodes for broadcast joins. Dynamic filter predicates +pushed into the ORC and Parquet readers are used to perform stripe or row-group pruning +and save on disk I/O. Sorting the data within ORC or Parquet files by the columns used in +join criteria significantly improves the effectiveness of stripe or row-group pruning. +This is because grouping similar data within the same stripe or row-group +greatly improves the selectivity of the min/max indexes maintained at stripe or +row-group level.

+
+

Delaying execution for dynamic filters#

+

It can often be beneficial to wait for the collection of dynamic filters before starting +a table scan. This extra wait time can potentially result in significant overall savings +in query and CPU time, if dynamic filtering is able to reduce the amount of scanned data.

+

For the Hive connector, a table scan can be delayed for a configured amount of +time until the collection of dynamic filters by using the configuration property +hive.dynamic-filtering.wait-timeout in the catalog file or the catalog +session property <hive-catalog>.dynamic_filtering_wait_timeout.

+
+
+
+

Table redirection#

+

Trino offers the possibility to transparently redirect operations on an existing +table to the appropriate catalog based on the format of the table and catalog configuration.

+

In the context of connectors which depend on a metastore service +(for example, Hive connector, Iceberg connector and Delta Lake connector), +the metastore (Hive metastore service, AWS Glue Data Catalog) +can be used to accustom tables with different table formats. +Therefore, a metastore database can hold a variety of tables with different table formats.

+

As a concrete example, let’s use the following +simple scenario which makes use of table redirection:

+
USE example.example_schema;
+
+EXPLAIN SELECT * FROM example_table;
+
+
+
                               Query Plan
+-------------------------------------------------------------------------
+Fragment 0 [SOURCE]
+     ...
+     Output[columnNames = [...]]
+     │   ...
+     └─ TableScan[table = another_catalog:example_schema:example_table]
+            ...
+
+
+

The output of the EXPLAIN statement points out the actual +catalog which is handling the SELECT query over the table example_table.

+

The table redirection functionality works also when using +fully qualified names for the tables:

+
EXPLAIN SELECT * FROM example.example_schema.example_table;
+
+
+
                               Query Plan
+-------------------------------------------------------------------------
+Fragment 0 [SOURCE]
+     ...
+     Output[columnNames = [...]]
+     │   ...
+     └─ TableScan[table = another_catalog:example_schema:example_table]
+            ...
+
+
+

Trino offers table redirection support for the following operations:

+ +

Trino does not offer view redirection support.

+

The connector supports redirection from Hive tables to Iceberg +and Delta Lake tables with the following catalog configuration properties:

+ +
+
+

Performance tuning configuration properties#

+

The following table describes performance tuning properties for the Hive +connector.

+
+

Warning

+

Performance tuning configuration properties are considered expert-level +features. Altering these properties from their default values is likely to +cause instability and performance degradation.

+
+ +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

Default value

hive.max-outstanding-splits

The target number of buffered splits for each table scan in a query, +before the scheduler tries to pause.

1000

hive.max-outstanding-splits-size

The maximum size allowed for buffered splits for each table scan +in a query, before the query fails.

256 MB

hive.max-splits-per-second

The maximum number of splits generated per second per table scan. This +can be used to reduce the load on the storage system. By default, there +is no limit, which results in Trino maximizing the parallelization of +data access.

hive.max-initial-splits

For each table scan, the coordinator first assigns file sections of up +to max-initial-split-size. After max-initial-splits have been +assigned, max-split-size is used for the remaining splits.

200

hive.max-initial-split-size

The size of a single file section assigned to a worker until +max-initial-splits have been assigned. Smaller splits results in +more parallelism, which gives a boost to smaller queries.

32 MB

hive.max-split-size

The largest size of a single file section assigned to a worker. Smaller +splits result in more parallelism and thus can decrease latency, but +also have more overhead and increase load on the system.

64 MB

+
+
+ +
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/hudi.html b/430/connector/hudi.html new file mode 100644 index 000000000..2f5908866 --- /dev/null +++ b/430/connector/hudi.html @@ -0,0 +1,925 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Hudi connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Hudi connector#

+

The Hudi connector enables querying Hudi tables.

+
+

Requirements#

+

To use the Hudi connector, you need:

+
    +
  • Hudi version 0.12.3 or higher.

  • +
  • Network access from the Trino coordinator and workers to the Hudi storage.

  • +
  • Access to a Hive metastore service (HMS).

  • +
  • Network access from the Trino coordinator to the HMS.

  • +
  • Data files stored in the Parquet file format. These can be configured using +file format configuration properties per +catalog.

  • +
+
+
+

General configuration#

+

To configure the Hive connector, create a catalog properties file +etc/catalog/example.properties that references the hudi +connector and defines the HMS to use with the hive.metastore.uri +configuration property:

+
connector.name=hudi
+hive.metastore.uri=thrift://example.net:9083
+
+
+

There are HMS configuration properties +available for use with the Hudi connector. The connector recognizes Hudi tables +synced to the metastore by the Hudi sync tool.

+

Additionally, following configuration properties can be set depending on the use-case:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Hudi configuration properties#

Property name

Description

Default

hudi.columns-to-hide

List of column names that are hidden from the query output. +It can be used to hide Hudi meta fields. By default, no fields are hidden.

hudi.parquet.use-column-names

Access Parquet columns using names from the file. If disabled, then columns +are accessed using the index. Only applicable to Parquet file format.

true

hudi.split-generator-parallelism

Number of threads to generate splits from partitions.

4

hudi.split-loader-parallelism

Number of threads to run background split loader. +A single background split loader is needed per query.

4

hudi.size-based-split-weights-enabled

Unlike uniform splitting, size-based splitting ensures that each batch of splits +has enough data to process. By default, it is enabled to improve performance.

true

hudi.standard-split-weight-size

The split size corresponding to the standard weight (1.0) +when size-based split weights are enabled.

128MB

hudi.minimum-assigned-split-weight

Minimum weight that a split can be assigned +when size-based split weights are enabled.

0.05

hudi.max-splits-per-second

Rate at which splits are queued for processing. +The queue is throttled if this rate limit is breached.

Integer.MAX_VALUE

hudi.max-outstanding-splits

Maximum outstanding splits in a batch enqueued for processing.

1000

hudi.per-transaction-metastore-cache-maximum-size

Maximum number of metastore data objects per transaction in +the Hive metastore cache.

2000

+
+
+

SQL support#

+

The connector provides read access to data in the Hudi table that has been synced to +Hive metastore. The globally available +and read operation statements are supported.

+
+

Basic usage examples#

+

In the following example queries, stock_ticks_cow is the Hudi copy-on-write +table referred to in the Hudi quickstart guide.

+
USE example.example_schema;
+
+SELECT symbol, max(ts)
+FROM stock_ticks_cow
+GROUP BY symbol
+HAVING symbol = 'GOOG';
+
+
+
  symbol   |        _col1         |
+-----------+----------------------+
+ GOOG      | 2018-08-31 10:59:00  |
+(1 rows)
+
+
+
SELECT dt, symbol
+FROM stock_ticks_cow
+WHERE symbol = 'GOOG';
+
+
+
    dt      | symbol |
+------------+--------+
+ 2018-08-31 |  GOOG  |
+(1 rows)
+
+
+
SELECT dt, count(*)
+FROM stock_ticks_cow
+GROUP BY dt;
+
+
+
    dt      | _col1 |
+------------+--------+
+ 2018-08-31 |  99  |
+(1 rows)
+
+
+
+
+

Schema and table management#

+

Hudi supports two types of tables +depending on how the data is indexed and laid out on the file system. The following +table displays a support matrix of tables types and query types for the connector:

+ + ++++ + + + + + + + + + + + + + +
Hudi configuration properties#

Table type

Supported query type

Copy on write

Snapshot queries

Merge on read

Read-optimized queries

+
+

Metadata tables#

+

The connector exposes a metadata table for each Hudi table. +The metadata table contains information about the internal structure +of the Hudi table. You can query each metadata table by appending the +metadata table name to the table name:

+
SELECT * FROM "test_table$timeline"
+
+
+
+
$timeline table#
+

The $timeline table provides a detailed view of meta-data instants +in the Hudi table. Instants are specific points in time.

+

You can retrieve the information about the timeline of the Hudi table +test_table by using the following query:

+
SELECT * FROM "test_table$timeline"
+
+
+
 timestamp          | action  | state
+--------------------+---------+-----------
+8667764846443717831 | commit  | COMPLETED
+7860805980949777961 | commit  | COMPLETED
+
+
+

The output of the query has the following columns:

+ + +++++ + + + + + + + + + + + + + + + + + + + + +
Timeline columns#

Name

Type

Description

timestamp

VARCHAR

Instant time is typically a timestamp when the actions performed.

action

VARCHAR

Type of action performed on the table.

state

VARCHAR

Current state of the instant.

+
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/iceberg.html b/430/connector/iceberg.html new file mode 100644 index 000000000..e26a4d6d4 --- /dev/null +++ b/430/connector/iceberg.html @@ -0,0 +1,2478 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Iceberg connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Iceberg connector#

+

Apache Iceberg is an open table format for huge analytic datasets. The Iceberg +connector allows querying data stored in files written in Iceberg format, as +defined in the Iceberg Table Spec. The +connector supports Apache Iceberg table spec versions 1 and 2.

+

The table state is maintained in metadata files. All changes to table +state create a new metadata file and replace the old metadata with an atomic +swap. The table metadata file tracks the table schema, partitioning +configuration, custom properties, and snapshots of the table contents.

+

Iceberg data files are stored in either Parquet, ORC, or Avro format, as +determined by the format property in the table definition.

+

Iceberg is designed to improve on the known scalability limitations of Hive, +which stores table metadata in a metastore that is backed by a relational +database such as MySQL. It tracks partition locations in the metastore, but not +individual data files. Trino queries using the Hive connector must +first call the metastore to get partition locations, then call the underlying +file system to list all data files inside each partition, and then read metadata +from each data file.

+

Since Iceberg stores the paths to data files in the metadata files, it only +consults the underlying file system for files that must be read.

+
+

Requirements#

+

To use Iceberg, you need:

+ +
+
+

General configuration#

+

To configure the Iceberg connector, create a catalog properties file +etc/catalog/example.properties that references the iceberg +connector and defines a metastore type. The Hive metastore catalog is the +default implementation. To use a Hive metastore, +iceberg.catalog.type must be set to hive_metastore and +hive.metastore.uri must be configured:

+
connector.name=iceberg
+iceberg.catalog.type=hive_metastore
+hive.metastore.uri=thrift://example.net:9083
+
+
+

Other metadata catalog types as listed in the requirements section of this topic +are available. Each metastore type has specific configuration properties along +with general metastore configuration properties.

+

The following configuration properties are independent of which catalog +implementation is used:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Iceberg general configuration properties#

Property name

Description

Default

iceberg.catalog.type

Define the metastore type to use. Possible values are:

+
    +
  • hive_metastore

  • +
  • glue

  • +
  • jdbc

  • +
  • rest

  • +
  • nessie

  • +
+

iceberg.file-format

Define the data storage file format for Iceberg tables. +Possible values are:

+
    +
  • PARQUET

  • +
  • ORC

  • +
  • AVRO

  • +
+

PARQUET

iceberg.compression-codec

The compression codec used when writing files. +Possible values are:

+
    +
  • NONE

  • +
  • SNAPPY

  • +
  • LZ4

  • +
  • ZSTD

  • +
  • GZIP

  • +
+

ZSTD

iceberg.use-file-size-from-metadata

Read file sizes from metadata instead of file system. This property must +only be used as a workaround for this issue. The problem was fixed +in Iceberg version 0.11.0.

true

iceberg.max-partitions-per-writer

Maximum number of partitions handled per writer.

100

iceberg.target-max-file-size

Target maximum size of written files; the actual size may be larger.

1GB

iceberg.unique-table-location

Use randomized, unique table locations.

true

iceberg.dynamic-filtering.wait-timeout

Maximum duration to wait for completion of dynamic filters during split +generation.

0s

iceberg.delete-schema-locations-fallback

Whether schema locations are deleted when Trino can’t determine whether +they contain external files.

false

iceberg.minimum-assigned-split-weight

A decimal value in the range (0, 1] used as a minimum for weights assigned +to each split. A low value may improve performance on tables with small +files. A higher value may improve performance for queries with highly +skewed aggregations or joins.

0.05

iceberg.table-statistics-enabled

Enables Table statistics. The equivalent catalog session +property is statistics_enabled for session +specific use. Set to false to disable statistics. Disabling statistics +means that Cost-based optimizations cannot make better +decisions about the query plan.

true

iceberg.projection-pushdown-enabled

Enable projection pushdown

true

iceberg.hive-catalog-name

Catalog to redirect to when a Hive table is referenced.

iceberg.materialized-views.storage-schema

Schema for creating materialized views storage tables. When this property +is not configured, storage tables are created in the same schema as the +materialized view definition. When the storage_schema materialized +view property is specified, it takes precedence over this catalog +property.

Empty

iceberg.register-table-procedure.enabled

Enable to allow user to call register_table procedure.

false

iceberg.query-partition-filter-required

Set to true to force a query to use a partition filter. +You can use the query_partition_filter_required catalog session property for temporary, catalog specific use.

false

+
+
+

Type mapping#

+

The connector reads and writes data into the supported data file formats Avro, +ORC, and Parquet, following the Iceberg specification.

+

Because Trino and Iceberg each support types that the other does not, this +connector modifies some types when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction.

+

The Iceberg specification includes supported data types and the mapping to the +formating in the Avro, ORC, or Parquet files:

+ +
+

Iceberg to Trino type mapping#

+

The connector maps Iceberg types to the corresponding Trino types according to +the following table:

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Iceberg to Trino type mapping#

Iceberg type

Trino type

BOOLEAN

BOOLEAN

INT

INTEGER

LONG

BIGINT

FLOAT

REAL

DOUBLE

DOUBLE

DECIMAL(p,s)

DECIMAL(p,s)

DATE

DATE

TIME

TIME(6)

TIMESTAMP

TIMESTAMP(6)

TIMESTAMPTZ

TIMESTAMP(6) WITH TIME ZONE

STRING

VARCHAR

UUID

UUID

BINARY

VARBINARY

FIXED (L)

VARBINARY

STRUCT(...)

ROW(...)

LIST(e)

ARRAY(e)

MAP(k,v)

MAP(k,v)

+

No other types are supported.

+
+
+

Trino to Iceberg type mapping#

+

The connector maps Trino types to the corresponding Iceberg types according to +the following table:

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Trino to Iceberg type mapping#

Trino type

Iceberg type

BOOLEAN

BOOLEAN

INTEGER

INT

BIGINT

LONG

REAL

FLOAT

DOUBLE

DOUBLE

DECIMAL(p,s)

DECIMAL(p,s)

DATE

DATE

TIME(6)

TIME

TIMESTAMP(6)

TIMESTAMP

TIMESTAMP(6) WITH TIME ZONE

TIMESTAMPTZ

VARCHAR

STRING

UUID

UUID

VARBINARY

BINARY

ROW(...)

STRUCT(...)

ARRAY(e)

LIST(e)

MAP(k,v)

MAP(k,v)

+

No other types are supported.

+
+
+
+

Security#

+

The Iceberg connector allows you to choose one of several means of providing +authorization at the catalog level.

+
+

Authorization checks#

+

You can enable authorization checks for the connector by setting the +iceberg.security property in the catalog properties file. This property must +be one of the following values:

+ + ++++ + + + + + + + + + + + + + + + + + + + +
Iceberg security values#

Property value

Description

ALLOW_ALL

No authorization checks are enforced.

SYSTEM

The connector relies on system-level access control.

READ_ONLY

Operations that read data or metadata, such as SELECT are +permitted. No operations that write data or metadata, such as +CREATE TABLE, INSERT, or DELETE are +allowed.

FILE

Authorization checks are enforced using a catalog-level access control +configuration file whose path is specified in the security.config-file +catalog configuration property. See +Catalog-level access control files for information on the +authorization configuration file.

+
+
+
+

SQL support#

+

This connector provides read access and write access to data and metadata in +Iceberg. In addition to the globally available +and read operation statements, the connector +supports the following features:

+ +
+

Basic usage examples#

+

The connector supports creating schemas. You can create a schema with or without +a specified location.

+

You can create a schema with the CREATE SCHEMA statement and the +location schema property. The tables in this schema, which have no explicit +location set in CREATE TABLE statement, are located in a +subdirectory under the directory corresponding to the schema location.

+

Create a schema on S3:

+
CREATE SCHEMA example.example_s3_schema
+WITH (location = 's3://my-bucket/a/path/');
+
+
+

Create a schema on an S3-compatible object storage such as MinIO:

+
CREATE SCHEMA example.example_s3a_schema
+WITH (location = 's3a://my-bucket/a/path/');
+
+
+

Create a schema on HDFS:

+
CREATE SCHEMA example.example_hdfs_schema
+WITH (location='hdfs://hadoop-master:9000/user/hive/warehouse/a/path/');
+
+
+

Optionally, on HDFS, the location can be omitted:

+
CREATE SCHEMA example.example_hdfs_schema;
+
+
+

The Iceberg connector supports creating tables using the CREATE TABLE syntax. Optionally, specify the table properties supported by this connector:

+
CREATE TABLE example_table (
+    c1 INTEGER,
+    c2 DATE,
+    c3 DOUBLE
+)
+WITH (
+    format = 'PARQUET',
+    partitioning = ARRAY['c1', 'c2'],
+    sorted_by = ARRAY['c3'],
+    location = 's3://my-bucket/a/path/'
+);
+
+
+

When the location table property is omitted, the content of the table is +stored in a subdirectory under the directory corresponding to the schema +location.

+

The Iceberg connector supports creating tables using the CREATE TABLE AS with SELECT syntax:

+
CREATE TABLE tiny_nation
+WITH (
+    format = 'PARQUET'
+)
+AS
+    SELECT *
+    FROM nation
+    WHERE nationkey < 10;
+
+
+

Another flavor of creating tables with CREATE TABLE AS is with VALUES syntax:

+
CREATE TABLE yearly_clicks (
+    year,
+    clicks
+)
+WITH (
+    partitioning = ARRAY['year']
+)
+AS VALUES
+    (2021, 10000),
+    (2022, 20000);
+
+
+
+
+

Procedures#

+

Use the CALL statement to perform data manipulation or +administrative tasks. Procedures are available in the system schema of each +catalog. The following code snippet displays how to call the +example_procedure in the examplecatalog catalog:

+
CALL examplecatalog.system.example_procedure()
+
+
+
+

Register table#

+

The connector can register existing Iceberg tables with the catalog.

+

The procedure system.register_table allows the caller to register an +existing Iceberg table in the metastore, using its existing metadata and data +files:

+
CALL example.system.register_table(schema_name => 'testdb', table_name => 'customer_orders', table_location => 'hdfs://hadoop-master:9000/user/hive/warehouse/customer_orders-581fad8517934af6be1857a903559d44')
+
+
+

In addition, you can provide a file name to register a table with specific +metadata. This may be used to register the table with some specific table state, +or may be necessary if the connector cannot automatically figure out the +metadata version to use:

+
CALL example.system.register_table(schema_name => 'testdb', table_name => 'customer_orders', table_location => 'hdfs://hadoop-master:9000/user/hive/warehouse/customer_orders-581fad8517934af6be1857a903559d44', metadata_file_name => '00003-409702ba-4735-4645-8f14-09537cc0b2c8.metadata.json')
+
+
+

To prevent unauthorized users from accessing data, this procedure is disabled by +default. The procedure is enabled only when +iceberg.register-table-procedure.enabled is set to true.

+
+
+

Unregister table#

+

The connector can unregister existing Iceberg tables from the catalog.

+

The procedure system.unregister_table allows the caller to unregister an +existing Iceberg table from the metastores without deleting the data:

+
CALL example.system.unregister_table(schema_name => 'testdb', table_name => 'customer_orders')
+
+
+
+
+

Migrate table#

+

The connector can read from or write to Hive tables that have been migrated to +Iceberg.

+

Use the procedure system.migrate to move a table from the Hive format to the +Iceberg format, loaded with the source’s data files. Table schema, partitioning, +properties, and location are copied from the source table. A bucketed Hive table +will be migrated as a non-bucketed Iceberg table. The data files in the Hive table +must use the Parquet, ORC, or Avro file format.

+

The procedure must be called for a specific catalog example with the +relevant schema and table names supplied with the required parameters +schema_name and table_name:

+
CALL example.system.migrate(
+    schema_name => 'testdb',
+    table_name => 'customer_orders')
+
+
+

Migrate fails if any table partition uses an unsupported file format.

+

In addition, you can provide a recursive_directory argument to migrate a +Hive table that contains subdirectories:

+
CALL example.system.migrate(
+    schema_name => 'testdb',
+    table_name => 'customer_orders',
+    recursive_directory => 'true')
+
+
+

The default value is fail, which causes the migrate procedure to throw an +exception if subdirectories are found. Set the value to true to migrate +nested directories, or false to ignore them.

+
+
+
+

Data management#

+

The Data management functionality includes support for INSERT, +UPDATE, DELETE, and MERGE statements.

+
+

Deletion by partition#

+

For partitioned tables, the Iceberg connector supports the deletion of entire +partitions if the WHERE clause specifies filters only on the +identity-transformed partitioning columns, that can match entire partitions. +Given the table definition from Partitioned Tables +section, the following SQL statement deletes all partitions for which +country is US:

+
DELETE FROM example.testdb.customer_orders
+WHERE country = 'US'
+
+
+

A partition delete is performed if the WHERE clause meets these conditions.

+
+
+

Row level deletion#

+

Tables using v2 of the Iceberg specification support deletion of individual rows +by writing position delete files.

+
+
+
+

Schema and table management#

+

The Schema and table management functionality includes support for:

+ +
+

Schema evolution#

+

Iceberg supports schema evolution, with safe column add, drop, reorder, and +rename operations, including in nested structures. Table partitioning can also +be changed and the connector can still query data created before the +partitioning change.

+
+
+

ALTER TABLE EXECUTE#

+

The connector supports the following commands for use with ALTER TABLE EXECUTE.

+
+
optimize#
+

The optimize command is used for rewriting the content of the specified +table so that it is merged into fewer but larger files. If the table is +partitioned, the data compaction acts separately on each partition selected for +optimization. This operation improves read performance.

+

All files with a size below the optional file_size_threshold parameter +(default value for the threshold is 100MB) are merged:

+
ALTER TABLE test_table EXECUTE optimize
+
+
+

The following statement merges files in a table that are +under 10 megabytes in size:

+
ALTER TABLE test_table EXECUTE optimize(file_size_threshold => '10MB')
+
+
+

You can use a WHERE clause with the columns used to partition the table +to filter which partitions are optimized:

+
ALTER TABLE test_partitioned_table EXECUTE optimize
+WHERE partition_key = 1
+
+
+
+
+
expire_snapshots#
+

The expire_snapshots command removes all snapshots and all related metadata +and data files. Regularly expiring snapshots is recommended to delete data files +that are no longer needed, and to keep the size of table metadata small. The +procedure affects all snapshots that are older than the time period configured +with the retention_threshold parameter.

+

expire_snapshots can be run as follows:

+
ALTER TABLE test_table EXECUTE expire_snapshots(retention_threshold => '7d')
+
+
+

The value for retention_threshold must be higher than or equal to +iceberg.expire_snapshots.min-retention in the catalog, otherwise the +procedure fails with a similar message: Retention specified (1.00d) is shorter than the minimum retention configured in the system (7.00d). The default value +for this property is 7d.

+
+
+
remove_orphan_files#
+

The remove_orphan_files command removes all files from a table’s data +directory that are not linked from metadata files and that are older than the +value of retention_threshold parameter. Deleting orphan files from time to +time is recommended to keep size of a table’s data directory under control.

+

remove_orphan_files can be run as follows:

+
ALTER TABLE test_table EXECUTE remove_orphan_files(retention_threshold => '7d')
+
+
+

The value for retention_threshold must be higher than or equal to +iceberg.remove_orphan_files.min-retention in the catalog otherwise the +procedure fails with a similar message: Retention specified (1.00d) is shorter than the minimum retention configured in the system (7.00d). The default value +for this property is 7d.

+
+
+
drop_extended_stats#
+

The drop_extended_stats command removes all extended statistics information +from the table.

+

drop_extended_stats can be run as follows:

+
ALTER TABLE test_table EXECUTE drop_extended_stats
+
+
+
+
+
+

ALTER TABLE SET PROPERTIES#

+

The connector supports modifying the properties on existing tables using +ALTER TABLE SET PROPERTIES.

+

The following table properties can be updated after a table is created:

+
    +
  • format

  • +
  • format_version

  • +
  • partitioning

  • +
  • sorted_by

  • +
+

For example, to update a table from v1 of the Iceberg specification to v2:

+
ALTER TABLE table_name SET PROPERTIES format_version = 2;
+
+
+

Or to set the column my_new_partition_column as a partition column on a +table:

+
ALTER TABLE table_name SET PROPERTIES partitioning = ARRAY[<existing partition columns>, 'my_new_partition_column'];
+
+
+

The current values of a table’s properties can be shown using SHOW CREATE TABLE.

+
+
Table properties#
+

Table properties supply or set metadata for the underlying tables. This is key +for CREATE TABLE AS statements. Table properties are passed to the +connector using a WITH clause.

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + + + + +
Iceberg table properties#

Property name

Description

format

Optionally specifies the format of table data files; either PARQUET, +ORC`, or ``AVRO. Defaults to the value of the iceberg.file-format +catalog configuration property, which defaults to PARQUET.

partitioning

Optionally specifies table partitioning. If a table is partitioned by +columns c1 and c2, the partitioning property is partitioning = +ARRAY['c1', 'c2'].

location

Optionally specifies the file system location URI for the table.

format_version

Optionally specifies the format version of the Iceberg specification to +use for new tables; either 1 or 2. Defaults to 2. Version +2 is required for row level deletes.

orc_bloom_filter_columns

Comma-separated list of columns to use for ORC bloom filter. It improves +the performance of queries using Equality and IN predicates when reading +ORC files. Requires ORC format. Defaults to [].

orc_bloom_filter_fpp

The ORC bloom filters false positive probability. Requires ORC format. +Defaults to 0.05.

+

The table definition below specifies to use Parquet files, partitioning by columns +c1 and c2, and a file system location of +/var/example_tables/test_table:

+
CREATE TABLE test_table (
+    c1 INTEGER,
+    c2 DATE,
+    c3 DOUBLE)
+WITH (
+    format = 'PARQUET',
+    partitioning = ARRAY['c1', 'c2'],
+    location = '/var/example_tables/test_table')
+
+
+

The table definition below specifies to use ORC files, bloom filter index by columns +c1 and c2, fpp is 0.05, and a file system location of +/var/example_tables/test_table:

+
CREATE TABLE test_table (
+    c1 INTEGER,
+    c2 DATE,
+    c3 DOUBLE)
+WITH (
+    format = 'ORC',
+    location = '/var/example_tables/test_table',
+    orc_bloom_filter_columns = ARRAY['c1', 'c2'],
+    orc_bloom_filter_fpp = 0.05)
+
+
+
+
+
+

Metadata tables#

+

The connector exposes several metadata tables for each Iceberg table. These +metadata tables contain information about the internal structure of the Iceberg +table. You can query each metadata table by appending the metadata table name to +the table name:

+
SELECT * FROM "test_table$properties"
+
+
+
+
$properties table#
+

The $properties table provides access to general information about Iceberg +table configuration and any additional metadata key/value pairs that the table +is tagged with.

+

You can retrieve the properties of the current snapshot of the Iceberg table +test_table by using the following query:

+
SELECT * FROM "test_table$properties"
+
+
+
 key                   | value    |
+-----------------------+----------+
+write.format.default   | PARQUET  |
+
+
+
+
+
$history table#
+

The $history table provides a log of the metadata changes performed on the +Iceberg table.

+

You can retrieve the changelog of the Iceberg table test_table by using the +following query:

+
SELECT * FROM "test_table$history"
+
+
+
 made_current_at                  | snapshot_id          | parent_id            | is_current_ancestor
+----------------------------------+----------------------+----------------------+--------------------
+2022-01-10 08:11:20 Europe/Vienna | 8667764846443717831  |  <null>              |  true
+2022-01-10 08:11:34 Europe/Vienna | 7860805980949777961  | 8667764846443717831  |  true
+
+
+

The output of the query has the following columns:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + +
History columns#

Name

Type

Description

made_current_at

TIMESTAMP(3) WITH TIME ZONE

The time when the snapshot became active.

snapshot_id

BIGINT

The identifier of the snapshot.

parent_id

BIGINT

The identifier of the parent snapshot.

is_current_ancestor

BOOLEAN

Whether or not this snapshot is an ancestor of the current snapshot.

+
+
+
$snapshots table#
+

The $snapshots table provides a detailed view of snapshots of the Iceberg +table. A snapshot consists of one or more file manifests, and the complete table +contents are represented by the union of all the data files in those manifests.

+

You can retrieve the information about the snapshots of the Iceberg table +test_table by using the following query:

+
SELECT * FROM "test_table$snapshots"
+
+
+
 committed_at                      | snapshot_id          | parent_id            | operation          |  manifest_list                                                                                                                           |   summary
+----------------------------------+----------------------+----------------------+--------------------+------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+2022-01-10 08:11:20 Europe/Vienna | 8667764846443717831  |  <null>              |  append            |   hdfs://hadoop-master:9000/user/hive/warehouse/test_table/metadata/snap-8667764846443717831-1-100cf97e-6d56-446e-8961-afdaded63bc4.avro | {changed-partition-count=0, total-equality-deletes=0, total-position-deletes=0, total-delete-files=0, total-files-size=0, total-records=0, total-data-files=0}
+2022-01-10 08:11:34 Europe/Vienna | 7860805980949777961  | 8667764846443717831  |  append            |   hdfs://hadoop-master:9000/user/hive/warehouse/test_table/metadata/snap-7860805980949777961-1-faa19903-1455-4bb8-855a-61a1bbafbaa7.avro | {changed-partition-count=1, added-data-files=1, total-equality-deletes=0, added-records=1, total-position-deletes=0, added-files-size=442, total-delete-files=0, total-files-size=442, total-records=1, total-data-files=1}
+
+
+

The output of the query has the following columns:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Snapshots columns#

Name

Type

Description

committed_at

TIMESTAMP(3) WITH TIME ZONE

The time when the snapshot became active.

snapshot_id

BIGINT

The identifier for the snapshot.

parent_id

BIGINT

The identifier for the parent snapshot.

operation

VARCHAR

The type of operation performed on the Iceberg table. The supported +operation types in Iceberg are:

+
    +
  • append when new data is appended.

  • +
  • replace when files are removed and replaced without changing the +data in the table.

  • +
  • overwrite when new data is added to overwrite existing data.

  • +
  • delete when data is deleted from the table and no new data is added.

  • +
+

manifest_list

VARCHAR

The list of Avro manifest files containing the detailed information about +the snapshot changes.

summary

map(VARCHAR, VARCHAR)

A summary of the changes made from the previous snapshot to the current +snapshot.

+
+
+
$manifests table#
+

The $manifests table provides a detailed overview of the manifests +corresponding to the snapshots performed in the log of the Iceberg table.

+

You can retrieve the information about the manifests of the Iceberg table +test_table by using the following query:

+
SELECT * FROM "test_table$manifests"
+
+
+
 path                                                                                                           | length          | partition_spec_id    | added_snapshot_id     | added_data_files_count  | added_rows_count | existing_data_files_count   | existing_rows_count | deleted_data_files_count    | deleted_rows_count | partitions
+----------------------------------------------------------------------------------------------------------------+-----------------+----------------------+-----------------------+-------------------------+------------------+-----------------------------+---------------------+-----------------------------+--------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------
+ hdfs://hadoop-master:9000/user/hive/warehouse/test_table/metadata/faa19903-1455-4bb8-855a-61a1bbafbaa7-m0.avro |  6277           |   0                  | 7860805980949777961   | 1                       | 100              | 0                           | 0                   | 0                           | 0                  | {{contains_null=false, contains_nan= false, lower_bound=1, upper_bound=1},{contains_null=false, contains_nan= false, lower_bound=2021-01-12, upper_bound=2021-01-12}}
+
+
+

The output of the query has the following columns:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Manifests columns#

Name

Type

Description

path

VARCHAR

The manifest file location.

length

BIGINT

The manifest file length.

partition_spec_id

INTEGER

The identifier for the partition specification used to write the manifest +file.

added_snapshot_id

BIGINT

The identifier of the snapshot during which this manifest entry has been +added.

added_data_files_count

INTEGER

The number of data files with status ADDED in the manifest file.

added_rows_count

BIGINT

The total number of rows in all data files with status ADDED in the +manifest file.

existing_data_files_count

INTEGER

The number of data files with status EXISTING in the manifest file.

existing_rows_count

BIGINT

The total number of rows in all data files with status EXISTING in the +manifest file.

deleted_data_files_count

INTEGER

The number of data files with status DELETED in the manifest file.

deleted_rows_count

BIGINT

The total number of rows in all data files with status DELETED in the +manifest file.

partitions

ARRAY(row(contains_null BOOLEAN, contains_nan BOOLEAN, lower_bound VARCHAR, upper_bound VARCHAR))

Partition range metadata.

+
+
+
$partitions table#
+

The $partitions table provides a detailed overview of the partitions of the +Iceberg table.

+

You can retrieve the information about the partitions of the Iceberg table +test_table by using the following query:

+
SELECT * FROM "test_table$partitions"
+
+
+
 partition             | record_count  | file_count    | total_size    |  data
+-----------------------+---------------+---------------+---------------+------------------------------------------------------
+{c1=1, c2=2021-01-12}  |  2            | 2             |  884          | {c3={min=1.0, max=2.0, null_count=0, nan_count=NULL}}
+{c1=1, c2=2021-01-13}  |  1            | 1             |  442          | {c3={min=1.0, max=1.0, null_count=0, nan_count=NULL}}
+
+
+

The output of the query has the following columns:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Partitions columns#

Name

Type

Description

partition

ROW(...)

A row that contains the mapping of the partition column names to the +partition column values.

record_count

BIGINT

The number of records in the partition.

file_count

BIGINT

The number of files mapped in the partition.

total_size

BIGINT

The size of all the files in the partition.

data

ROW(... ROW (min ..., max ... , null_count BIGINT, nan_count BIGINT))

Partition range metadata.

+
+
+
$files table#
+

The $files table provides a detailed overview of the data files in current +snapshot of the Iceberg table.

+

To retrieve the information about the data files of the Iceberg table +test_table, use the following query:

+
SELECT * FROM "test_table$files"
+
+
+
 content  | file_path                                                                                                                     | record_count    | file_format   | file_size_in_bytes   |  column_sizes        |  value_counts     |  null_value_counts | nan_value_counts  | lower_bounds                |  upper_bounds               |  key_metadata  | split_offsets  |  equality_ids
+----------+-------------------------------------------------------------------------------------------------------------------------------+-----------------+---------------+----------------------+----------------------+-------------------+--------------------+-------------------+-----------------------------+-----------------------------+----------------+----------------+---------------
+ 0        | hdfs://hadoop-master:9000/user/hive/warehouse/test_table/data/c1=3/c2=2021-01-14/af9872b2-40f3-428f-9c87-186d2750d84e.parquet |  1              |  PARQUET      |  442                 | {1=40, 2=40, 3=44}   |  {1=1, 2=1, 3=1}  |  {1=0, 2=0, 3=0}   | <null>            |  {1=3, 2=2021-01-14, 3=1.3} |  {1=3, 2=2021-01-14, 3=1.3} |  <null>        | <null>         |   <null>
+
+
+

The output of the query has the following columns:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Files columns#

Name

Type

Description

content

INTEGER

Type of content stored in the file. The supported content types in Iceberg +are:

+
    +
  • DATA(0)

  • +
  • POSITION_DELETES(1)

  • +
  • EQUALITY_DELETES(2)

  • +
+

file_path

VARCHAR

The data file location.

file_format

VARCHAR

The format of the data file.

record_count

BIGINT

The number of entries contained in the data file.

file_size_in_bytes

BIGINT

The data file size

column_sizes

map(INTEGER, BIGINT)

Mapping between the Iceberg column ID and its corresponding size in the +file.

value_counts

map(INTEGER, BIGINT)

Mapping between the Iceberg column ID and its corresponding count of +entries in the file.

null_value_counts

map(INTEGER, BIGINT)

Mapping between the Iceberg column ID and its corresponding count of +NULL values in the file.

nan_value_counts

map(INTEGER, BIGINT)

Mapping between the Iceberg column ID and its corresponding count of non- +numerical values in the file.

lower_bounds

map(INTEGER, BIGINT)

Mapping between the Iceberg column ID and its corresponding lower bound in +the file.

upper_bounds

map(INTEGER, BIGINT)

Mapping between the Iceberg column ID and its corresponding upper bound in +the file.

key_metadata

VARBINARY

Metadata about the encryption key used to encrypt this file, if applicable.

split_offsets

array(BIGINT)

List of recommended split locations.

equality_ids

array(INTEGER)

The set of field IDs used for equality comparison in equality delete files.

+
+
+
$refs table#
+

The $refs table provides information about Iceberg references including +branches and tags.

+

You can retrieve the references of the Iceberg table test_table by using the +following query:

+
SELECT * FROM "test_table$refs"
+
+
+
name            | type   | snapshot_id | max_reference_age_in_ms | min_snapshots_to_keep | max_snapshot_age_in_ms |
+----------------+--------+-------------+-------------------------+-----------------------+------------------------+
+example_tag     | TAG    | 10000000000 | 10000                   | null                  | null                   |
+example_branch  | BRANCH | 20000000000 | 20000                   | 2                     | 30000                  |
+
+
+

The output of the query has the following columns:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Refs columns#

Name

Type

Description

name

VARCHAR

Name of the reference.

type

VARCHAR

Type of the reference, either BRANCH or TAG.

snapshot_id

BIGINT

The snapshot ID of the reference.

max_reference_age_in_ms

BIGINT

The maximum age of the reference before it could be expired.

min_snapshots_to_keep

INTEGER

For branch only, the minimum number of snapshots to keep in a branch.

max_snapshot_age_in_ms

BIGINT

For branch only, the max snapshot age allowed in a branch. Older snapshots +in the branch will be expired.

+
+
+
+

Metadata columns#

+

In addition to the defined columns, the Iceberg connector automatically exposes +path metadata as a hidden column in each table:

+
    +
  • $path: Full file system path name of the file for this row

  • +
  • $file_modified_time: Timestamp of the last modification of the file for +this row

  • +
+

You can use these columns in your SQL statements like any other column. This can +be selected directly, or used in conditional statements. For example, you can +inspect the file path for each record:

+
SELECT *, "$path", "$file_modified_time"
+FROM example.web.page_views;
+
+
+

Retrieve all records that belong to a specific file using "$path" filter:

+
SELECT *
+FROM example.web.page_views
+WHERE "$path" = '/usr/iceberg/table/web.page_views/data/file_01.parquet'
+
+
+

Retrieve all records that belong to a specific file using +"$file_modified_time" filter:

+
SELECT *
+FROM example.web.page_views
+WHERE "$file_modified_time" = CAST('2022-07-01 01:02:03.456 UTC' AS TIMESTAMP WIOTH TIMEZONE)
+
+
+
+
+

DROP TABLE#

+

The Iceberg connector supports dropping a table by using the +DROP TABLE syntax. When the command succeeds, both the data of the +Iceberg table and also the information related to the table in the metastore +service are removed. Dropping tables that have their data/metadata stored in a +different location than the table’s corresponding base directory on the object +store is not supported.

+
+
+

COMMENT#

+

The Iceberg connector supports setting comments on the following objects:

+
    +
  • tables

  • +
  • views

  • +
  • table columns

  • +
  • materialized view columns

  • +
+

The COMMENT option is supported on both the table and the table columns for +the CREATE TABLE operation.

+

The COMMENT option is supported for adding table columns through the +ALTER TABLE operations.

+

The connector supports the command COMMENT for setting +comments on existing entities.

+
+
+

Partitioned tables#

+

Iceberg supports partitioning by specifying transforms over the table columns. A +partition is created for each unique tuple value produced by the transforms. +Identity transforms are simply the column name. Other transforms are:

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + + + + +
Iceberg column transforms#

Transform

Description

year(ts)

A partition is created for each year. The partition value is the integer +difference in years between ts and January 1 1970.

month(ts)

A partition is created for each month of each year. The partition value +is the integer difference in months between ts and January 1 1970.

day(ts)

A partition is created for each day of each year. The partition value is +the integer difference in days between ts and January 1 1970.

hour(ts)

A partition is created hour of each day. The partition value is a +timestamp with the minutes and seconds set to zero.

bucket(x, nbuckets)

The data is hashed into the specified number of buckets. The partition +value is an integer hash of x, with a value between 0 and nbuckets - +1 inclusive.

truncate(s, nchars)

The partition value is the first nchars characters of s.

+

In this example, the table is partitioned by the month of order_date, a hash +of account_number (with 10 buckets), and country:

+
CREATE TABLE example.testdb.customer_orders (
+    order_id BIGINT,
+    order_date DATE,
+    account_number BIGINT,
+    customer VARCHAR,
+    country VARCHAR)
+WITH (partitioning = ARRAY['month(order_date)', 'bucket(account_number, 10)', 'country'])
+
+
+
+
+

Sorted tables#

+

The connector supports sorted files as a performance improvement. Data is sorted +during writes within each file based on the specified array of one or more +columns.

+

Sorting is particularly beneficial when the sorted columns show a high +cardinality and are used as a filter for selective reads.

+

The sort order is configured with the sorted_by table property. Specify an +array of one or more columns to use for sorting when creating the table. The +following example configures the order_date column of the orders table +in the customers schema in the example catalog:

+
CREATE TABLE example.customers.orders (
+    order_id BIGINT,
+    order_date DATE,
+    account_number BIGINT,
+    customer VARCHAR,
+    country VARCHAR)
+WITH (sorted_by = ARRAY['order_date'])
+
+
+

Sorting can be combined with partitioning on the same column. For example:

+
CREATE TABLE example.customers.orders (
+    order_id BIGINT,
+    order_date DATE,
+    account_number BIGINT,
+    customer VARCHAR,
+    country VARCHAR)
+WITH (
+    partitioning = ARRAY['month(order_date)'],
+    sorted_by = ARRAY['order_date']
+)
+
+
+

You can disable sorted writing with the session property +sorted_writing_enabled set to false.

+
+
+

Using snapshots#

+

Iceberg supports a snapshot model of data, where table snapshots are +identified by a snapshot ID.

+

The connector provides a system table exposing snapshot information for every +Iceberg table. Snapshots are identified by BIGINT snapshot IDs. For example, +you can find the snapshot IDs for the customer_orders table by running the +following query:

+
SELECT snapshot_id
+FROM example.testdb."customer_orders$snapshots"
+ORDER BY committed_at DESC
+
+
+
+
Time travel queries#
+

The connector offers the ability to query historical data. This allows you to +query the table as it was when a previous snapshot of the table was taken, even +if the data has since been modified or deleted.

+

The historical data of the table can be retrieved by specifying the snapshot +identifier corresponding to the version of the table to be retrieved:

+
SELECT *
+FROM example.testdb.customer_orders FOR VERSION AS OF 8954597067493422955
+
+
+

A different approach of retrieving historical data is to specify a point in time +in the past, such as a day or week ago. The latest snapshot of the table taken +before or at the specified timestamp in the query is internally used for +providing the previous state of the table:

+
SELECT *
+FROM example.testdb.customer_orders FOR TIMESTAMP AS OF TIMESTAMP '2022-03-23 09:59:29.803 Europe/Vienna'
+
+
+

You can use a date to specify a point a time in the past for using a snapshot of a table in a query. +Assuming that the session time zone is Europe/Vienna the following queries are equivalent:

+
SELECT *
+FROM example.testdb.customer_orders FOR TIMESTAMP AS OF DATE '2022-03-23'
+
+
+
SELECT *
+FROM example.testdb.customer_orders FOR TIMESTAMP AS OF TIMESTAMP '2022-03-23 00:00:00'
+
+
+
SELECT *
+FROM example.testdb.customer_orders FOR TIMESTAMP AS OF TIMESTAMP '2022-03-23 00:00:00.000 Europe/Vienna'
+
+
+

Iceberg supports named references of snapshots via branches and tags. +Time travel can be performed to branches and tags in the table.

+
SELECT *
+FROM example.testdb.customer_orders FOR VERSION AS OF 'historical-tag'
+
+SELECT *
+FROM example.testdb.customer_orders FOR VERSION AS OF 'test-branch'
+
+
+
+
+
Rolling back to a previous snapshot#
+

Use the $snapshots metadata table to determine the latest snapshot ID of the +table like in the following query:

+
SELECT snapshot_id
+FROM example.testdb."customer_orders$snapshots"
+ORDER BY committed_at DESC LIMIT 1
+
+
+

The procedure system.rollback_to_snapshot allows the caller to roll back the +state of the table to a previous snapshot id:

+
CALL example.system.rollback_to_snapshot('testdb', 'customer_orders', 8954597067493422955)
+
+
+
+
+
+

NOT NULL column constraint#

+

The Iceberg connector supports setting NOT NULL constraints on the table +columns.

+

The NOT NULL constraint can be set on the columns, while creating tables by +using the CREATE TABLE syntax:

+
CREATE TABLE example_table (
+    year INTEGER NOT NULL,
+    name VARCHAR NOT NULL,
+    age INTEGER,
+    address VARCHAR
+);
+
+
+

When trying to insert/update data in the table, the query fails if trying to set +NULL value on a column having the NOT NULL constraint.

+
+
+
+

View management#

+

Trino allows reading from Iceberg materialized views.

+
+

Materialized views#

+

The Iceberg connector supports Materialized view management. In the +underlying system, each materialized view consists of a view definition and an +Iceberg storage table. The storage table name is stored as a materialized view +property. The data is stored in that storage table.

+

You can use the Table properties to control the created storage +table and therefore the layout and performance. For example, you can use the +following clause with CREATE MATERIALIZED VIEW to use the ORC format +for the data files and partition the storage per day using the column +_date:

+
WITH ( format = 'ORC', partitioning = ARRAY['event_date'] )
+
+
+

By default, the storage table is created in the same schema as the materialized +view definition. The iceberg.materialized-views.storage-schema catalog +configuration property or storage_schema materialized view property can be +used to specify the schema where the storage table is created.

+

Creating a materialized view does not automatically populate it with data. You +must run REFRESH MATERIALIZED VIEW to populate data in the +materialized view.

+

Updating the data in the materialized view with REFRESH MATERIALIZED VIEW +deletes the data from the storage table, and inserts the data that is the result +of executing the materialized view query into the existing table. Data is +replaced atomically, so users can continue to query the materialized view while +it is being refreshed. Refreshing a materialized view also stores the +snapshot-ids of all Iceberg tables that are part of the materialized view’s +query in the materialized view metadata. When the materialized view is queried, +the snapshot-ids are used to check if the data in the storage table is up to +date. If the data is outdated, the materialized view behaves like a normal view, +and the data is queried directly from the base tables. Detecting outdated data +is possible only when the materialized view uses Iceberg tables only, or when it +uses a mix of Iceberg and non-Iceberg tables but some Iceberg tables are outdated. +When the materialized view is based on non-Iceberg tables, querying it can +return outdated data, since the connector has no information whether the +underlying non-Iceberg tables have changed.

+

Dropping a materialized view with DROP MATERIALIZED VIEW removes +the definition and the storage table.

+
+
+
+
+

Fault-tolerant execution support#

+

The connector supports Fault-tolerant execution of query +processing. Read and write operations are both supported with any retry policy.

+
+
+

Performance#

+

The connector includes a number of performance improvements, detailed in the +following sections.

+
+

Table statistics#

+

The Iceberg connector can collect column statistics using ANALYZE +statement. This can be disabled using iceberg.extended-statistics.enabled +catalog configuration property, or the corresponding +extended_statistics_enabled session property.

+
+

Updating table statistics#

+

If your queries are complex and include joining large data sets, running +ANALYZE on tables may improve query performance by collecting +statistical information about the data:

+
ANALYZE table_name
+
+
+

This query collects statistics for all columns.

+

On wide tables, collecting statistics for all columns can be expensive. It is +also typically unnecessary - statistics are only useful on specific columns, +like join keys, predicates, or grouping keys. You can specify a subset of +columns to analyzed with the optional columns property:

+
ANALYZE table_name WITH (columns = ARRAY['col_1', 'col_2'])
+
+
+

This query collects statistics for columns col_1 and col_2.

+

Note that if statistics were previously collected for all columns, they must be +dropped using the drop_extended_stats command +before re-analyzing.

+
+
+
+

Table redirection#

+

Trino offers the possibility to transparently redirect operations on an existing +table to the appropriate catalog based on the format of the table and catalog configuration.

+

In the context of connectors which depend on a metastore service +(for example, Hive connector, Iceberg connector and Delta Lake connector), +the metastore (Hive metastore service, AWS Glue Data Catalog) +can be used to accustom tables with different table formats. +Therefore, a metastore database can hold a variety of tables with different table formats.

+

As a concrete example, let’s use the following +simple scenario which makes use of table redirection:

+
USE example.example_schema;
+
+EXPLAIN SELECT * FROM example_table;
+
+
+
                               Query Plan
+-------------------------------------------------------------------------
+Fragment 0 [SOURCE]
+     ...
+     Output[columnNames = [...]]
+     │   ...
+     └─ TableScan[table = another_catalog:example_schema:example_table]
+            ...
+
+
+

The output of the EXPLAIN statement points out the actual +catalog which is handling the SELECT query over the table example_table.

+

The table redirection functionality works also when using +fully qualified names for the tables:

+
EXPLAIN SELECT * FROM example.example_schema.example_table;
+
+
+
                               Query Plan
+-------------------------------------------------------------------------
+Fragment 0 [SOURCE]
+     ...
+     Output[columnNames = [...]]
+     │   ...
+     └─ TableScan[table = another_catalog:example_schema:example_table]
+            ...
+
+
+

Trino offers table redirection support for the following operations:

+ +

Trino does not offer view redirection support.

+

The connector supports redirection from Iceberg tables to Hive tables with the +iceberg.hive-catalog-name catalog configuration property.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/ignite.html b/430/connector/ignite.html new file mode 100644 index 000000000..993a934df --- /dev/null +++ b/430/connector/ignite.html @@ -0,0 +1,1189 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Ignite connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Ignite connector#

+

The Ignite connector allows querying an Apache Ignite +database from Trino.

+
+

Requirements#

+

To connect to a Ignite server, you need:

+
    +
  • Ignite version 2.9.0 or latter

  • +
  • Network access from the Trino coordinator and workers to the Ignite +server. Port 10800 is the default port.

  • +
  • Specify --add-opens=java.base/java.nio=ALL-UNNAMED in the jvm.config when starting the Trino server.

  • +
+
+
+

Configuration#

+

The Ignite connector expose public schema by default.

+

The connector can query a Ignite instance. Create a catalog properties file +that specifies the Ignite connector by setting the connector.name to +ignite.

+

For example, to access an instance as example, create the file +etc/catalog/example.properties. Replace the connection properties as +appropriate for your setup:

+
connector.name=ignite
+connection-url=jdbc:ignite:thin://host1:10800/
+connection-user=exampleuser
+connection-password=examplepassword
+
+
+

The connection-url defines the connection information and parameters to pass +to the Ignite JDBC driver. The parameters for the URL are available in the +Ignite JDBC driver documentation. +Some parameters can have adverse effects on the connector behavior or not work +with the connector.

+

The connection-user and connection-password are typically required and +determine the user credentials for the connection, often a service user. You can +use secrets to avoid actual values in the catalog +properties files.

+
+

Multiple Ignite servers#

+

If you have multiple Ignite servers you need to configure one +catalog for each server. To add another catalog:

+
    +
  • Add another properties file to etc/catalog

  • +
  • Save it with a different name that ends in .properties

  • +
+

For example, if you name the property file sales.properties, Trino uses the +configured connector to create a catalog named sales.

+
+
+

General configuration properties#

+

The following table describes general catalog configuration properties for the +connector:

+ +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

Default value

case-insensitive-name-matching

Support case insensitive schema and table names.

false

case-insensitive-name-matching.cache-ttl

This value should be a duration.

1m

case-insensitive-name-matching.config-file

Path to a name mapping configuration file in JSON format that allows +Trino to disambiguate between schemas and tables with similar names in +different cases.

null

case-insensitive-name-matching.config-file.refresh-period

Frequency with which Trino checks the name matching configuration file +for changes. This value should be a duration.

(refresh disabled)

metadata.cache-ttl

The duration for which metadata, including +table and column statistics, is cached.

0s (caching disabled)

metadata.cache-missing

Cache the fact that metadata, including table and column statistics, is +not available

false

metadata.cache-maximum-size

Maximum number of objects stored in the metadata cache

10000

write.batch-size

Maximum number of statements in a batched execution. +Do not change this setting from the default. Non-default values may +negatively impact performance.

1000

dynamic-filtering.enabled

Push down dynamic filters into JDBC queries

true

dynamic-filtering.wait-timeout

Maximum duration for which Trino will wait for dynamic +filters to be collected from the build side of joins before starting a +JDBC query. Using a large timeout can potentially result in more detailed +dynamic filters. However, it can also increase latency for some queries.

20s

+
+
+

Appending query metadata#

+

The optional parameter query.comment-format allows you to configure a SQL +comment that is sent to the datasource with each query. The format of this +comment can contain any characters and the following metadata:

+
    +
  • $QUERY_ID: The identifier of the query.

  • +
  • $USER: The name of the user who submits the query to Trino.

  • +
  • $SOURCE: The identifier of the client tool used to submit the query, for +example trino-cli.

  • +
  • $TRACE_TOKEN: The trace token configured with the client tool.

  • +
+

The comment can provide more context about the query. This additional +information is available in the logs of the datasource. To include environment +variables from the Trino cluster with the comment , use the +${ENV:VARIABLE-NAME} syntax.

+

The following example sets a simple comment that identifies each query sent by +Trino:

+
query.comment-format=Query sent by Trino.
+
+
+

With this configuration, a query such as SELECT * FROM example_table; is +sent to the datasource with the comment appended:

+
SELECT * FROM example_table; /*Query sent by Trino.*/
+
+
+

The following example improves on the preceding example by using metadata:

+
query.comment-format=Query $QUERY_ID sent by user $USER from Trino.
+
+
+

If Jane sent the query with the query identifier +20230622_180528_00000_bkizg, the following comment string is sent to the +datasource:

+
SELECT * FROM example_table; /*Query 20230622_180528_00000_bkizg sent by user Jane from Trino.*/
+
+
+
+

Note

+

Certain JDBC driver settings and logging configurations might cause the +comment to be removed.

+
+
+
+

Domain compaction threshold#

+

Pushing down a large list of predicates to the data source can compromise +performance. Trino compacts large predicates into a simpler range predicate +by default to ensure a balance between performance and predicate pushdown. +If necessary, the threshold for this compaction can be increased to improve +performance when the data source is capable of taking advantage of large +predicates. Increasing this threshold may improve pushdown of large +dynamic filters. +The domain-compaction-threshold catalog configuration property or the +domain_compaction_threshold catalog session property can be used to adjust the default value of +1000 for this threshold.

+
+
+

Procedures#

+
    +
  • system.flush_metadata_cache()

    +

    Flush JDBC metadata caches. For example, the following system call +flushes the metadata caches for all schemas in the example catalog

    +
    USE example.example_schema;
    +CALL system.flush_metadata_cache();
    +
    +
    +
  • +
+
+
+

Case insensitive matching#

+

When case-insensitive-name-matching is set to true, Trino +is able to query non-lowercase schemas and tables by maintaining a mapping of +the lowercase name to the actual name in the remote system. However, if two +schemas and/or tables have names that differ only in case (such as “customers” +and “Customers”) then Trino fails to query them due to ambiguity.

+

In these cases, use the case-insensitive-name-matching.config-file catalog +configuration property to specify a configuration file that maps these remote +schemas/tables to their respective Trino schemas/tables:

+
{
+  "schemas": [
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "mapping": "case_insensitive_1"
+    },
+    {
+      "remoteSchema": "cASEsENSITIVEnAME",
+      "mapping": "case_insensitive_2"
+    }],
+  "tables": [
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "remoteTable": "tablex",
+      "mapping": "table_1"
+    },
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "remoteTable": "TABLEX",
+      "mapping": "table_2"
+    }]
+}
+
+
+

Queries against one of the tables or schemes defined in the mapping +attributes are run against the corresponding remote entity. For example, a query +against tables in the case_insensitive_1 schema is forwarded to the +CaseSensitiveName schema and a query against case_insensitive_2 is forwarded +to the cASEsENSITIVEnAME schema.

+

At the table mapping level, a query on case_insensitive_1.table_1 as +configured above is forwarded to CaseSensitiveName.tablex, and a query on +case_insensitive_1.table_2 is forwarded to CaseSensitiveName.TABLEX.

+

By default, when a change is made to the mapping configuration file, Trino must +be restarted to load the changes. Optionally, you can set the +case-insensitive-name-mapping.refresh-period to have Trino refresh the +properties without requiring a restart:

+
case-insensitive-name-mapping.refresh-period=30s
+
+
+
+
+

Non-transactional INSERT#

+

The connector supports adding rows using INSERT statements. +By default, data insertion is performed by writing data to a temporary table. +You can skip this step to improve performance and write directly to the target +table. Set the insert.non-transactional-insert.enabled catalog property +or the corresponding non_transactional_insert catalog session property to +true.

+

Note that with this property enabled, data can be corrupted in rare cases where +exceptions occur during the insert operation. With transactions disabled, no +rollback can be performed.

+
+
+
+

Table properties#

+

Table property usage example:

+
CREATE TABLE public.person (
+  id BIGINT NOT NULL,
+  birthday DATE NOT NULL,
+  name VARCHAR(26),
+  age BIGINT,
+  logdate DATE
+)
+WITH (
+  primary_key = ARRAY['id', 'birthday']
+);
+
+
+

The following are supported Ignite table properties from https://ignite.apache.org/docs/latest/sql-reference/ddl

+ +++++ + + + + + + + + + + + + +

Property name

Required

Description

primary_key

No

The primary key of the table, can chose multi columns as the table primary key. Table at least contains one column not in primary key.

+
+

primary_key#

+

This is a list of columns to be used as the table’s primary key. If not specified, a VARCHAR primary key column named DUMMY_ID is generated, +the value is derived from the value generated by the UUID function in Ignite.

+
+
+
+

Type mapping#

+

The following are supported Ignite SQL data types from https://ignite.apache.org/docs/latest/sql-reference/data-types

+ +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Ignite SQL data type name

Map to Trino type

Possible values

BOOLEAN

BOOLEAN

TRUE and FALSE

BIGINT

BIGINT

-9223372036854775808, 9223372036854775807, etc.

DECIMAL

DECIMAL

Data type with fixed precision and scale

DOUBLE

DOUBLE

3.14, -10.24, etc.

INT

INT

-2147483648, 2147483647, etc.

REAL

REAL`

3.14, -10.24, etc.

SMALLINT

SMALLINT

-32768, 32767, etc.

TINYINT

TINYINT

-128, 127, etc.

CHAR

CHAR

hello, Trino, etc.

VARCHAR

VARCHAR

hello, Trino, etc.

DATE

DATE

1972-01-01, 2021-07-15, etc.

BINARY

VARBINARY

Represents a byte array.

+
+
+

SQL support#

+

The connector provides read access and write access to data and metadata in +Ignite. In addition to the globally available and read operation +statements, the connector supports the following features:

+ +
+

UPDATE#

+

Only UPDATE statements with constant assignments and predicates are +supported. For example, the following statement is supported because the values +assigned are constants:

+
UPDATE table SET col1 = 1 WHERE col3 = 1
+
+
+

Arithmetic expressions, function calls, and other non-constant UPDATE +statements are not supported. For example, the following statement is not +supported because arithmetic expressions cannot be used with the SET +command:

+
UPDATE table SET col1 = col2 + 2 WHERE col3 = 1
+
+
+

The =, !=, >, <, >=, <=, IN, NOT IN operators are supported in +predicates. The following statement is not supported because the AND operator +cannot be used in predicates:

+
UPDATE table SET col1 = 1 WHERE col3 = 1 AND col2 = 3
+
+
+

All column values of a table row cannot be updated simultaneously. For a three +column table, the following statement is not supported:

+
UPDATE table SET col1 = 1, col2 = 2, col3 = 3 WHERE col3 = 1
+
+
+
+
+

ALTER TABLE RENAME TO#

+

The connector does not support renaming tables across multiple schemas. For +example, the following statement is supported:

+
ALTER TABLE example.schema_one.table_one RENAME TO example.schema_one.table_two
+
+
+

The following statement attempts to rename a table across schemas, and therefore +is not supported:

+
ALTER TABLE example.schema_one.table_one RENAME TO example.schema_two.table_two
+
+
+
+
+

Pushdown#

+

The connector supports pushdown for a number of operations:

+ +

Aggregate pushdown for the following functions:

+ +
+

Predicate pushdown support#

+

The connector does not support pushdown of any predicates on columns with +textual types like CHAR or VARCHAR. +This ensures correctness of results since the data source may compare strings +case-insensitively.

+

In the following example, the predicate is not pushed down for either query +since name is a column of type VARCHAR:

+
SELECT * FROM nation WHERE name > 'CANADA';
+SELECT * FROM nation WHERE name = 'CANADA';
+
+
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/jmx.html b/430/connector/jmx.html new file mode 100644 index 000000000..4927bcf22 --- /dev/null +++ b/430/connector/jmx.html @@ -0,0 +1,786 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + JMX connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

JMX connector#

+

The JMX connector provides the ability to query Java Management Extensions (JMX) +information from all +nodes in a Trino cluster. This is very useful for monitoring or debugging. +JMX provides information about the Java +Virtual Machine and all of the software running inside it. Trino itself +is heavily instrumented via JMX.

+

This connector can be configured so that chosen JMX information is +periodically dumped and stored in memory for later access.

+
+

Configuration#

+

To configure the JMX connector, create a catalog properties file +etc/catalog/example.properties with the following contents:

+
connector.name=jmx
+
+
+

To enable periodical dumps, define the following properties:

+
connector.name=jmx
+jmx.dump-tables=java.lang:type=Runtime,trino.execution.scheduler:name=NodeScheduler
+jmx.dump-period=10s
+jmx.max-entries=86400
+
+
+

dump-tables is a comma separated list of Managed Beans (MBean). It specifies +which MBeans are sampled and stored in memory every dump-period. You can +configure the maximum number of history entries with max-entries and it +defaults to 86400. The time between dumps can be configured using +dump-period and it defaults to 10s.

+

Commas in MBean names must be escaped using double backslashes (\\) in the +following manner:

+
connector.name=jmx
+jmx.dump-tables=trino.memory:name=general\\,type=memorypool,trino.memory:name=reserved\\,type=memorypool
+
+
+

Double backslashes are required because a single backslash (\) is used to +split the value across multiple lines in the following manner:

+
connector.name=jmx
+jmx.dump-tables=trino.memory:name=general\\,type=memorypool,\
+    trino.memory:name=reserved\\,type=memorypool
+
+
+
+
+

Querying JMX#

+

The JMX connector provides two schemas.

+

The first one is current that contains every MBean from every node in the Trino +cluster. You can see all of the available MBeans by running SHOW TABLES:

+
SHOW TABLES FROM example.current;
+
+
+

MBean names map to non-standard table names, and must be quoted with +double quotes when referencing them in a query. For example, the +following query shows the JVM version of every node:

+
SELECT node, vmname, vmversion
+FROM example.current."java.lang:type=runtime";
+
+
+
                 node                 |              vmname               | vmversion
+--------------------------------------+-----------------------------------+-----------
+ ddc4df17-0b8e-4843-bb14-1b8af1a7451a | Java HotSpot(TM) 64-Bit Server VM | 24.60-b09
+(1 row)
+
+
+

The following query shows the open and maximum file descriptor counts +for each node:

+
SELECT openfiledescriptorcount, maxfiledescriptorcount
+FROM example.current."java.lang:type=operatingsystem";
+
+
+
 openfiledescriptorcount | maxfiledescriptorcount
+-------------------------+------------------------
+                     329 |                  10240
+(1 row)
+
+
+

The wildcard character * may be used with table names in the current schema. +This allows matching several MBean objects within a single query. The following query +returns information from the different Trino memory pools on each node:

+
SELECT freebytes, node, object_name
+FROM example.current."trino.memory:*type=memorypool*";
+
+
+
 freebytes  |  node   |                       object_name
+------------+---------+----------------------------------------------------------
+  214748364 | example | trino.memory:type=MemoryPool,name=reserved
+ 1073741825 | example | trino.memory:type=MemoryPool,name=general
+  858993459 | example | trino.memory:type=MemoryPool,name=system
+(3 rows)
+
+
+

The history schema contains the list of tables configured in the connector properties file. +The tables have the same columns as those in the current schema, but with an additional +timestamp column that stores the time at which the snapshot was taken:

+
SELECT "timestamp", "uptime" FROM example.history."java.lang:type=runtime";
+
+
+
        timestamp        | uptime
+-------------------------+--------
+ 2016-01-28 10:18:50.000 |  11420
+ 2016-01-28 10:19:00.000 |  21422
+ 2016-01-28 10:19:10.000 |  31412
+(3 rows)
+
+
+
+
+

SQL support#

+

The connector provides globally available and +read operation statements to access JMX information +on your Trino nodes.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/kafka-tutorial.html b/430/connector/kafka-tutorial.html new file mode 100644 index 000000000..7e8510ffe --- /dev/null +++ b/430/connector/kafka-tutorial.html @@ -0,0 +1,1280 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Kafka connector tutorial — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Kafka connector tutorial#

+
+

Introduction#

+

The Kafka connector for Trino allows access to live topic data from +Apache Kafka using Trino. This tutorial shows how to set up topics, and +how to create the topic description files that back Trino tables.

+
+
+

Installation#

+

This tutorial assumes familiarity with Trino and a working local Trino +installation (see Deploying Trino). It focuses on +setting up Apache Kafka and integrating it with Trino.

+
+

Step 1: Install Apache Kafka#

+

Download and extract Apache Kafka.

+
+

Note

+

This tutorial was tested with Apache Kafka 0.8.1. +It should work with any 0.8.x version of Apache Kafka.

+
+

Start ZooKeeper and the Kafka server:

+
$ bin/zookeeper-server-start.sh config/zookeeper.properties
+[2013-04-22 15:01:37,495] INFO Reading configuration from: config/zookeeper.properties (org.apache.zookeeper.server.quorum.QuorumPeerConfig)
+...
+
+
+
$ bin/kafka-server-start.sh config/server.properties
+[2013-04-22 15:01:47,028] INFO Verifying properties (kafka.utils.VerifiableProperties)
+[2013-04-22 15:01:47,051] INFO Property socket.send.buffer.bytes is overridden to 1048576 (kafka.utils.VerifiableProperties)
+...
+
+
+

This starts Zookeeper on port 2181 and Kafka on port 9092.

+
+
+

Step 2: Load data#

+

Download the tpch-kafka loader from Maven Central:

+
$ curl -o kafka-tpch https://repo1.maven.org/maven2/de/softwareforge/kafka_tpch_0811/1.0/kafka_tpch_0811-1.0.sh
+$ chmod 755 kafka-tpch
+
+
+

Now run the kafka-tpch program to preload a number of topics with tpch data:

+
$ ./kafka-tpch load --brokers localhost:9092 --prefix tpch. --tpch-type tiny
+2014-07-28T17:17:07.594-0700     INFO    main    io.airlift.log.Logging    Logging to stderr
+2014-07-28T17:17:07.623-0700     INFO    main    de.softwareforge.kafka.LoadCommand    Processing tables: [customer, orders, lineitem, part, partsupp, supplier, nation, region]
+2014-07-28T17:17:07.981-0700     INFO    pool-1-thread-1    de.softwareforge.kafka.LoadCommand    Loading table 'customer' into topic 'tpch.customer'...
+2014-07-28T17:17:07.981-0700     INFO    pool-1-thread-2    de.softwareforge.kafka.LoadCommand    Loading table 'orders' into topic 'tpch.orders'...
+2014-07-28T17:17:07.981-0700     INFO    pool-1-thread-3    de.softwareforge.kafka.LoadCommand    Loading table 'lineitem' into topic 'tpch.lineitem'...
+2014-07-28T17:17:07.982-0700     INFO    pool-1-thread-4    de.softwareforge.kafka.LoadCommand    Loading table 'part' into topic 'tpch.part'...
+2014-07-28T17:17:07.982-0700     INFO    pool-1-thread-5    de.softwareforge.kafka.LoadCommand    Loading table 'partsupp' into topic 'tpch.partsupp'...
+2014-07-28T17:17:07.982-0700     INFO    pool-1-thread-6    de.softwareforge.kafka.LoadCommand    Loading table 'supplier' into topic 'tpch.supplier'...
+2014-07-28T17:17:07.982-0700     INFO    pool-1-thread-7    de.softwareforge.kafka.LoadCommand    Loading table 'nation' into topic 'tpch.nation'...
+2014-07-28T17:17:07.982-0700     INFO    pool-1-thread-8    de.softwareforge.kafka.LoadCommand    Loading table 'region' into topic 'tpch.region'...
+2014-07-28T17:17:10.612-0700    ERROR    pool-1-thread-8    kafka.producer.async.DefaultEventHandler    Failed to collate messages by topic, partition due to: Failed to fetch topic metadata for topic: tpch.region
+2014-07-28T17:17:10.781-0700     INFO    pool-1-thread-8    de.softwareforge.kafka.LoadCommand    Generated 5 rows for table 'region'.
+2014-07-28T17:17:10.797-0700    ERROR    pool-1-thread-3    kafka.producer.async.DefaultEventHandler    Failed to collate messages by topic, partition due to: Failed to fetch topic metadata for topic: tpch.lineitem
+2014-07-28T17:17:10.932-0700    ERROR    pool-1-thread-1    kafka.producer.async.DefaultEventHandler    Failed to collate messages by topic, partition due to: Failed to fetch topic metadata for topic: tpch.customer
+2014-07-28T17:17:11.068-0700    ERROR    pool-1-thread-2    kafka.producer.async.DefaultEventHandler    Failed to collate messages by topic, partition due to: Failed to fetch topic metadata for topic: tpch.orders
+2014-07-28T17:17:11.200-0700    ERROR    pool-1-thread-6    kafka.producer.async.DefaultEventHandler    Failed to collate messages by topic, partition due to: Failed to fetch topic metadata for topic: tpch.supplier
+2014-07-28T17:17:11.319-0700     INFO    pool-1-thread-6    de.softwareforge.kafka.LoadCommand    Generated 100 rows for table 'supplier'.
+2014-07-28T17:17:11.333-0700    ERROR    pool-1-thread-4    kafka.producer.async.DefaultEventHandler    Failed to collate messages by topic, partition due to: Failed to fetch topic metadata for topic: tpch.part
+2014-07-28T17:17:11.466-0700    ERROR    pool-1-thread-5    kafka.producer.async.DefaultEventHandler    Failed to collate messages by topic, partition due to: Failed to fetch topic metadata for topic: tpch.partsupp
+2014-07-28T17:17:11.597-0700    ERROR    pool-1-thread-7    kafka.producer.async.DefaultEventHandler    Failed to collate messages by topic, partition due to: Failed to fetch topic metadata for topic: tpch.nation
+2014-07-28T17:17:11.706-0700     INFO    pool-1-thread-7    de.softwareforge.kafka.LoadCommand    Generated 25 rows for table 'nation'.
+2014-07-28T17:17:12.180-0700     INFO    pool-1-thread-1    de.softwareforge.kafka.LoadCommand    Generated 1500 rows for table 'customer'.
+2014-07-28T17:17:12.251-0700     INFO    pool-1-thread-4    de.softwareforge.kafka.LoadCommand    Generated 2000 rows for table 'part'.
+2014-07-28T17:17:12.905-0700     INFO    pool-1-thread-2    de.softwareforge.kafka.LoadCommand    Generated 15000 rows for table 'orders'.
+2014-07-28T17:17:12.919-0700     INFO    pool-1-thread-5    de.softwareforge.kafka.LoadCommand    Generated 8000 rows for table 'partsupp'.
+2014-07-28T17:17:13.877-0700     INFO    pool-1-thread-3    de.softwareforge.kafka.LoadCommand    Generated 60175 rows for table 'lineitem'.
+
+
+

Kafka now has a number of topics that are preloaded with data to query.

+
+
+

Step 3: Make the Kafka topics known to Trino#

+

In your Trino installation, add a catalog properties file +etc/catalog/kafka.properties for the Kafka connector. +This file lists the Kafka nodes and topics:

+
connector.name=kafka
+kafka.nodes=localhost:9092
+kafka.table-names=tpch.customer,tpch.orders,tpch.lineitem,tpch.part,tpch.partsupp,tpch.supplier,tpch.nation,tpch.region
+kafka.hide-internal-columns=false
+
+
+

Now start Trino:

+
$ bin/launcher start
+
+
+

Because the Kafka tables all have the tpch. prefix in the configuration, +the tables are in the tpch schema. The connector is mounted into the +kafka catalog, because the properties file is named kafka.properties.

+

Start the Trino CLI:

+
$ ./trino --catalog kafka --schema tpch
+
+
+

List the tables to verify that things are working:

+
trino:tpch> SHOW TABLES;
+  Table
+----------
+ customer
+ lineitem
+ nation
+ orders
+ part
+ partsupp
+ region
+ supplier
+(8 rows)
+
+
+
+
+

Step 4: Basic data querying#

+

Kafka data is unstructured, and it has no metadata to describe the format of +the messages. Without further configuration, the Kafka connector can access +the data, and map it in raw form. However there are no actual columns besides the +built-in ones:

+
trino:tpch> DESCRIBE customer;
+      Column       |  Type      | Extra |                   Comment
+-------------------+------------+-------+---------------------------------------------
+ _partition_id     | bigint     |       | Partition Id
+ _partition_offset | bigint     |       | Offset for the message within the partition
+ _key              | varchar    |       | Key text
+ _key_corrupt      | boolean    |       | Key data is corrupt
+ _key_length       | bigint     |       | Total number of key bytes
+ _message          | varchar    |       | Message text
+ _message_corrupt  | boolean    |       | Message data is corrupt
+ _message_length   | bigint     |       | Total number of message bytes
+ _timestamp        | timestamp  |       | Message timestamp
+(11 rows)
+
+trino:tpch> SELECT count(*) FROM customer;
+ _col0
+-------
+  1500
+
+trino:tpch> SELECT _message FROM customer LIMIT 5;
+                                                                                                                                                 _message
+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+ {"rowNumber":1,"customerKey":1,"name":"Customer#000000001","address":"IVhzIApeRb ot,c,E","nationKey":15,"phone":"25-989-741-2988","accountBalance":711.56,"marketSegment":"BUILDING","comment":"to the even, regular platelets. regular, ironic epitaphs nag e"}
+ {"rowNumber":3,"customerKey":3,"name":"Customer#000000003","address":"MG9kdTD2WBHm","nationKey":1,"phone":"11-719-748-3364","accountBalance":7498.12,"marketSegment":"AUTOMOBILE","comment":" deposits eat slyly ironic, even instructions. express foxes detect slyly. blithel
+ {"rowNumber":5,"customerKey":5,"name":"Customer#000000005","address":"KvpyuHCplrB84WgAiGV6sYpZq7Tj","nationKey":3,"phone":"13-750-942-6364","accountBalance":794.47,"marketSegment":"HOUSEHOLD","comment":"n accounts will have to unwind. foxes cajole accor"}
+ {"rowNumber":7,"customerKey":7,"name":"Customer#000000007","address":"TcGe5gaZNgVePxU5kRrvXBfkasDTea","nationKey":18,"phone":"28-190-982-9759","accountBalance":9561.95,"marketSegment":"AUTOMOBILE","comment":"ainst the ironic, express theodolites. express, even pinto bean
+ {"rowNumber":9,"customerKey":9,"name":"Customer#000000009","address":"xKiAFTjUsCuxfeleNqefumTrjS","nationKey":8,"phone":"18-338-906-3675","accountBalance":8324.07,"marketSegment":"FURNITURE","comment":"r theodolites according to the requests wake thinly excuses: pending
+(5 rows)
+
+trino:tpch> SELECT sum(cast(json_extract_scalar(_message, '$.accountBalance') AS DOUBLE)) FROM customer LIMIT 10;
+   _col0
+------------
+ 6681865.59
+(1 row)
+
+
+

The data from Kafka can be queried using Trino, but it is not yet in +actual table shape. The raw data is available through the _message and +_key columns, but it is not decoded into columns. As the sample data is +in JSON format, the JSON functions and operators built into Trino can be used +to slice the data.

+
+
+

Step 5: Add a topic description file#

+

The Kafka connector supports topic description files to turn raw data into +table format. These files are located in the etc/kafka folder in the +Trino installation and must end with .json. It is recommended that +the file name matches the table name, but this is not necessary.

+

Add the following file as etc/kafka/tpch.customer.json and restart Trino:

+
{
+    "tableName": "customer",
+    "schemaName": "tpch",
+    "topicName": "tpch.customer",
+    "key": {
+        "dataFormat": "raw",
+        "fields": [
+            {
+                "name": "kafka_key",
+                "dataFormat": "LONG",
+                "type": "BIGINT",
+                "hidden": "false"
+            }
+        ]
+    }
+}
+
+
+

The customer table now has an additional column: kafka_key.

+
trino:tpch> DESCRIBE customer;
+      Column       |  Type      | Extra |                   Comment
+-------------------+------------+-------+---------------------------------------------
+ kafka_key         | bigint     |       |
+ _partition_id     | bigint     |       | Partition Id
+ _partition_offset | bigint     |       | Offset for the message within the partition
+ _key              | varchar    |       | Key text
+ _key_corrupt      | boolean    |       | Key data is corrupt
+ _key_length       | bigint     |       | Total number of key bytes
+ _message          | varchar    |       | Message text
+ _message_corrupt  | boolean    |       | Message data is corrupt
+ _message_length   | bigint     |       | Total number of message bytes
+ _timestamp        | timestamp  |       | Message timestamp
+(12 rows)
+
+trino:tpch> SELECT kafka_key FROM customer ORDER BY kafka_key LIMIT 10;
+ kafka_key
+-----------
+         0
+         1
+         2
+         3
+         4
+         5
+         6
+         7
+         8
+         9
+(10 rows)
+
+
+

The topic definition file maps the internal Kafka key, which is a raw long +in eight bytes, onto a Trino BIGINT column.

+
+
+

Step 6: Map all the values from the topic message onto columns#

+

Update the etc/kafka/tpch.customer.json file to add fields for the +message, and restart Trino. As the fields in the message are JSON, it uses +the JSON data format. This is an example, where different data formats +are used for the key and the message.

+
{
+    "tableName": "customer",
+    "schemaName": "tpch",
+    "topicName": "tpch.customer",
+    "key": {
+        "dataFormat": "raw",
+        "fields": [
+            {
+                "name": "kafka_key",
+                "dataFormat": "LONG",
+                "type": "BIGINT",
+                "hidden": "false"
+            }
+        ]
+    },
+    "message": {
+        "dataFormat": "json",
+        "fields": [
+            {
+                "name": "row_number",
+                "mapping": "rowNumber",
+                "type": "BIGINT"
+            },
+            {
+                "name": "customer_key",
+                "mapping": "customerKey",
+                "type": "BIGINT"
+            },
+            {
+                "name": "name",
+                "mapping": "name",
+                "type": "VARCHAR"
+            },
+            {
+                "name": "address",
+                "mapping": "address",
+                "type": "VARCHAR"
+            },
+            {
+                "name": "nation_key",
+                "mapping": "nationKey",
+                "type": "BIGINT"
+            },
+            {
+                "name": "phone",
+                "mapping": "phone",
+                "type": "VARCHAR"
+            },
+            {
+                "name": "account_balance",
+                "mapping": "accountBalance",
+                "type": "DOUBLE"
+            },
+            {
+                "name": "market_segment",
+                "mapping": "marketSegment",
+                "type": "VARCHAR"
+            },
+            {
+                "name": "comment",
+                "mapping": "comment",
+                "type": "VARCHAR"
+            }
+        ]
+    }
+}
+
+
+

Now for all the fields in the JSON of the message, columns are defined and +the sum query from earlier can operate on the account_balance column directly:

+
trino:tpch> DESCRIBE customer;
+      Column       |  Type      | Extra |                   Comment
+-------------------+------------+-------+---------------------------------------------
+ kafka_key         | bigint     |       |
+ row_number        | bigint     |       |
+ customer_key      | bigint     |       |
+ name              | varchar    |       |
+ address           | varchar    |       |
+ nation_key        | bigint     |       |
+ phone             | varchar    |       |
+ account_balance   | double     |       |
+ market_segment    | varchar    |       |
+ comment           | varchar    |       |
+ _partition_id     | bigint     |       | Partition Id
+ _partition_offset | bigint     |       | Offset for the message within the partition
+ _key              | varchar    |       | Key text
+ _key_corrupt      | boolean    |       | Key data is corrupt
+ _key_length       | bigint     |       | Total number of key bytes
+ _message          | varchar    |       | Message text
+ _message_corrupt  | boolean    |       | Message data is corrupt
+ _message_length   | bigint     |       | Total number of message bytes
+ _timestamp        | timestamp  |       | Message timestamp
+(21 rows)
+
+trino:tpch> SELECT * FROM customer LIMIT 5;
+ kafka_key | row_number | customer_key |        name        |                address                | nation_key |      phone      | account_balance | market_segment |                                                      comment
+-----------+------------+--------------+--------------------+---------------------------------------+------------+-----------------+-----------------+----------------+---------------------------------------------------------------------------------------------------------
+         1 |          2 |            2 | Customer#000000002 | XSTf4,NCwDVaWNe6tEgvwfmRchLXak        |         13 | 23-768-687-3665 |          121.65 | AUTOMOBILE     | l accounts. blithely ironic theodolites integrate boldly: caref
+         3 |          4 |            4 | Customer#000000004 | XxVSJsLAGtn                           |          4 | 14-128-190-5944 |         2866.83 | MACHINERY      |  requests. final, regular ideas sleep final accou
+         5 |          6 |            6 | Customer#000000006 | sKZz0CsnMD7mp4Xd0YrBvx,LREYKUWAh yVn  |         20 | 30-114-968-4951 |         7638.57 | AUTOMOBILE     | tions. even deposits boost according to the slyly bold packages. final accounts cajole requests. furious
+         7 |          8 |            8 | Customer#000000008 | I0B10bB0AymmC, 0PrRYBCP1yGJ8xcBPmWhl5 |         17 | 27-147-574-9335 |         6819.74 | BUILDING       | among the slyly regular theodolites kindle blithely courts. carefully even theodolites haggle slyly alon
+         9 |         10 |           10 | Customer#000000010 | 6LrEaV6KR6PLVcgl2ArL Q3rqzLzcT1 v2    |          5 | 15-741-346-9870 |         2753.54 | HOUSEHOLD      | es regular deposits haggle. fur
+(5 rows)
+
+trino:tpch> SELECT sum(account_balance) FROM customer LIMIT 10;
+   _col0
+------------
+ 6681865.59
+(1 row)
+
+
+

Now all the fields from the customer topic messages are available as +Trino table columns.

+
+
+

Step 7: Use live data#

+

Trino can query live data in Kafka as it arrives. To simulate a live feed +of data, this tutorial sets up a feed of live tweets into Kafka.

+
+

Setup a live Twitter feed#

+
    +
  • Download the twistr tool

  • +
+
$ curl -o twistr https://repo1.maven.org/maven2/de/softwareforge/twistr_kafka_0811/1.2/twistr_kafka_0811-1.2.sh
+$ chmod 755 twistr
+
+
+
    +
  • Create a developer account at https://dev.twitter.com/ and set up an +access and consumer token.

  • +
  • Create a twistr.properties file and put the access and consumer key +and secrets into it:

  • +
+
twistr.access-token-key=...
+twistr.access-token-secret=...
+twistr.consumer-key=...
+twistr.consumer-secret=...
+twistr.kafka.brokers=localhost:9092
+
+
+
+
+

Create a tweets table on Trino#

+

Add the tweets table to the etc/catalog/kafka.properties file:

+
connector.name=kafka
+kafka.nodes=localhost:9092
+kafka.table-names=tpch.customer,tpch.orders,tpch.lineitem,tpch.part,tpch.partsupp,tpch.supplier,tpch.nation,tpch.region,tweets
+kafka.hide-internal-columns=false
+
+
+

Add a topic definition file for the Twitter feed as etc/kafka/tweets.json:

+
{
+    "tableName": "tweets",
+    "topicName": "twitter_feed",
+    "dataFormat": "json",
+    "key": {
+        "dataFormat": "raw",
+        "fields": [
+            {
+                "name": "kafka_key",
+                "dataFormat": "LONG",
+                "type": "BIGINT",
+                "hidden": "false"
+            }
+        ]
+    },
+    "message": {
+        "dataFormat":"json",
+        "fields": [
+            {
+                "name": "text",
+                "mapping": "text",
+                "type": "VARCHAR"
+            },
+            {
+                "name": "user_name",
+                "mapping": "user/screen_name",
+                "type": "VARCHAR"
+            },
+            {
+                "name": "lang",
+                "mapping": "lang",
+                "type": "VARCHAR"
+            },
+            {
+                "name": "created_at",
+                "mapping": "created_at",
+                "type": "TIMESTAMP",
+                "dataFormat": "rfc2822"
+            },
+            {
+                "name": "favorite_count",
+                "mapping": "favorite_count",
+                "type": "BIGINT"
+            },
+            {
+                "name": "retweet_count",
+                "mapping": "retweet_count",
+                "type": "BIGINT"
+            },
+            {
+                "name": "favorited",
+                "mapping": "favorited",
+                    "type": "BOOLEAN"
+            },
+            {
+                "name": "id",
+                "mapping": "id_str",
+                "type": "VARCHAR"
+            },
+            {
+                "name": "in_reply_to_screen_name",
+                "mapping": "in_reply_to_screen_name",
+                "type": "VARCHAR"
+            },
+            {
+                "name": "place_name",
+                "mapping": "place/full_name",
+                "type": "VARCHAR"
+            }
+        ]
+    }
+}
+
+
+

As this table does not have an explicit schema name, it is placed +into the default schema.

+
+
+

Feed live data#

+

Start the twistr tool:

+
$ java -Dness.config.location=file:$(pwd) -Dness.config=twistr -jar ./twistr
+
+
+

twistr connects to the Twitter API and feeds the “sample tweet” feed +into a Kafka topic called twitter_feed.

+

Now run queries against live data:

+
$ ./trino --catalog kafka --schema default
+
+trino:default> SELECT count(*) FROM tweets;
+ _col0
+-------
+  4467
+(1 row)
+
+trino:default> SELECT count(*) FROM tweets;
+ _col0
+-------
+  4517
+(1 row)
+
+trino:default> SELECT count(*) FROM tweets;
+ _col0
+-------
+  4572
+(1 row)
+
+trino:default> SELECT kafka_key, user_name, lang, created_at FROM tweets LIMIT 10;
+     kafka_key      |    user_name    | lang |       created_at
+--------------------+-----------------+------+-------------------------
+ 494227746231685121 | burncaniff      | en   | 2014-07-29 14:07:31.000
+ 494227746214535169 | gu8tn           | ja   | 2014-07-29 14:07:31.000
+ 494227746219126785 | pequitamedicen  | es   | 2014-07-29 14:07:31.000
+ 494227746201931777 | josnyS          | ht   | 2014-07-29 14:07:31.000
+ 494227746219110401 | Cafe510         | en   | 2014-07-29 14:07:31.000
+ 494227746210332673 | Da_JuanAnd_Only | en   | 2014-07-29 14:07:31.000
+ 494227746193956865 | Smile_Kidrauhl6 | pt   | 2014-07-29 14:07:31.000
+ 494227750426017793 | CashforeverCD   | en   | 2014-07-29 14:07:32.000
+ 494227750396653569 | FilmArsivimiz   | tr   | 2014-07-29 14:07:32.000
+ 494227750388256769 | jmolas          | es   | 2014-07-29 14:07:32.000
+(10 rows)
+
+
+

There is now a live feed into Kafka, which can be queried using Trino.

+
+
+
+

Epilogue: Time stamps#

+

The tweets feed, that was set up in the last step, contains a timestamp in +RFC 2822 format as created_at attribute in each tweet.

+
trino:default> SELECT DISTINCT json_extract_scalar(_message, '$.created_at')) AS raw_date
+             -> FROM tweets LIMIT 5;
+            raw_date
+--------------------------------
+ Tue Jul 29 21:07:31 +0000 2014
+ Tue Jul 29 21:07:32 +0000 2014
+ Tue Jul 29 21:07:33 +0000 2014
+ Tue Jul 29 21:07:34 +0000 2014
+ Tue Jul 29 21:07:35 +0000 2014
+(5 rows)
+
+
+

The topic definition file for the tweets table contains a mapping onto a +timestamp using the rfc2822 converter:

+
...
+{
+    "name": "created_at",
+    "mapping": "created_at",
+    "type": "TIMESTAMP",
+    "dataFormat": "rfc2822"
+},
+...
+
+
+

This allows the raw data to be mapped onto a Trino TIMESTAMP column:

+
trino:default> SELECT created_at, raw_date FROM (
+             ->   SELECT created_at, json_extract_scalar(_message, '$.created_at') AS raw_date
+             ->   FROM tweets)
+             -> GROUP BY 1, 2 LIMIT 5;
+       created_at        |            raw_date
+-------------------------+--------------------------------
+ 2014-07-29 14:07:20.000 | Tue Jul 29 21:07:20 +0000 2014
+ 2014-07-29 14:07:21.000 | Tue Jul 29 21:07:21 +0000 2014
+ 2014-07-29 14:07:22.000 | Tue Jul 29 21:07:22 +0000 2014
+ 2014-07-29 14:07:23.000 | Tue Jul 29 21:07:23 +0000 2014
+ 2014-07-29 14:07:24.000 | Tue Jul 29 21:07:24 +0000 2014
+(5 rows)
+
+
+

The Kafka connector contains converters for ISO 8601, RFC 2822 text +formats and for number-based timestamps using seconds or miilliseconds +since the epoch. There is also a generic, text-based formatter, which uses +Joda-Time format strings to parse text columns.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/kafka.html b/430/connector/kafka.html new file mode 100644 index 000000000..bc777716b --- /dev/null +++ b/430/connector/kafka.html @@ -0,0 +1,2573 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Kafka connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Kafka connector#

+
+
+

This connector allows the use of Apache Kafka +topics as tables in Trino. Each message is presented as a row in Trino.

+

Topics can be live. Rows appear as data arrives, and disappear as +segments get dropped. This can result in strange behavior if accessing the +same table multiple times in a single query (e.g., performing a self join).

+

The connector reads and writes message data from Kafka topics in parallel across +workers to achieve a significant performance gain. The size of data sets for this +parallelization is configurable and can therefore be adapted to your specific +needs.

+

See the Kafka connector tutorial.

+
+

Requirements#

+

To connect to Kafka, you need:

+
    +
  • Kafka broker version 0.10.0 or higher.

  • +
  • Network access from the Trino coordinator and workers to the Kafka nodes. +Port 9092 is the default port.

  • +
+

When using Protobuf decoder with the Confluent table description supplier, the following additional steps +must be taken:

+
    +
  • Copy the kafka-protobuf-provider and kafka-protobuf-types JAR files +from Confluent for +Confluent version 7.3.1 to the Kafka connector plugin directory (<install directory>/plugin/kafka) on all nodes in the cluster. +The plugin directory depends on the Installation method.

  • +
  • By copying those JARs and using them, you agree to the terms of the Confluent +Community License Agreement +under which Confluent makes them available.

  • +
+

These steps are not required if you are not using Protobuf and Confluent table +description supplier.

+
+
+

Configuration#

+

To configure the Kafka connector, create a catalog properties file +etc/catalog/example.properties with the following content, replacing the +properties as appropriate.

+

In some cases, such as when using specialized authentication methods, it is necessary to specify +additional Kafka client properties in order to access your Kafka cluster. To do so, +add the kafka.config.resources property to reference your Kafka config files. Note that configs +can be overwritten if defined explicitly in kafka.properties:

+
connector.name=kafka
+kafka.table-names=table1,table2
+kafka.nodes=host1:port,host2:port
+kafka.config.resources=/etc/kafka-configuration.properties
+
+
+
+

Multiple Kafka clusters#

+

You can have as many catalogs as you need, so if you have additional +Kafka clusters, simply add another properties file to etc/catalog +with a different name (making sure it ends in .properties). For +example, if you name the property file sales.properties, Trino +creates a catalog named sales using the configured connector.

+
+
+

Log levels#

+

Kafka consumer logging can be verbose and pollute Trino logs. To lower the +log level, simply add the following to etc/log.properties:

+
org.apache.kafka=WARN
+
+
+
+
+
+

Configuration properties#

+

The following configuration properties are available:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

kafka.default-schema

Default schema name for tables.

kafka.nodes

List of nodes in the Kafka cluster.

kafka.buffer-size

Kafka read buffer size.

kafka.hide-internal-columns

Controls whether internal columns are part of the table schema or not.

kafka.internal-column-prefix

Prefix for internal columns, defaults to _

kafka.messages-per-split

Number of messages that are processed by each Trino split; defaults to 100000.

kafka.protobuf-any-support-enabled

Enable support for encoding Protobuf any types to JSON by setting the property to true, defaults to false.

kafka.timestamp-upper-bound-force-push-down-enabled

Controls if upper bound timestamp pushdown is enabled for topics using CreateTime mode.

kafka.security-protocol

Security protocol for connection to Kafka cluster; defaults to PLAINTEXT.

kafka.ssl.keystore.location

Location of the keystore file.

kafka.ssl.keystore.password

Password for the keystore file.

kafka.ssl.keystore.type

File format of the keystore file; defaults to JKS.

kafka.ssl.truststore.location

Location of the truststore file.

kafka.ssl.truststore.password

Password for the truststore file.

kafka.ssl.truststore.type

File format of the truststore file; defaults to JKS.

kafka.ssl.key.password

Password for the private key in the keystore file.

kafka.ssl.endpoint-identification-algorithm

Endpoint identification algorithm used by clients to validate server host name; defaults to https.

kafka.config.resources

A comma-separated list of Kafka client configuration files. These files must exist on the machines running Trino. Only specify this if absolutely necessary to access Kafka. Example: /etc/kafka-configuration.properties

+

In addition, you must configure table schema and schema registry usage with the relevant properties.

+
+

kafka.default-schema#

+

Defines the schema which contains all tables that were defined without +a qualifying schema name.

+

This property is optional; the default is default.

+
+
+

kafka.nodes#

+

A comma separated list of hostname:port pairs for the Kafka data nodes.

+

This property is required; there is no default and at least one node must be defined.

+
+

Note

+

Trino must still be able to connect to all nodes of the cluster +even if only a subset is specified here, as segment files may be +located only on a specific node.

+
+
+
+

kafka.buffer-size#

+

Size of the internal data buffer for reading data from Kafka. The data +buffer must be able to hold at least one message and ideally can hold many +messages. There is one data buffer allocated per worker and data node.

+

This property is optional; the default is 64kb.

+
+
+

kafka.timestamp-upper-bound-force-push-down-enabled#

+

The upper bound predicate on _timestamp column +is pushed down only for topics using LogAppendTime mode.

+

For topics using CreateTime mode, upper bound pushdown must be explicitly +enabled via kafka.timestamp-upper-bound-force-push-down-enabled config property +or timestamp_upper_bound_force_push_down_enabled session property.

+

This property is optional; the default is false.

+
+
+

kafka.hide-internal-columns#

+

In addition to the data columns defined in a table description file, the +connector maintains a number of additional columns for each table. If +these columns are hidden, they can still be used in queries but do not +show up in DESCRIBE <table-name> or SELECT *.

+

This property is optional; the default is true.

+
+
+

kafka.security-protocol#

+

Protocol used to communicate with brokers. +Valid values are: PLAINTEXT, SSL.

+

This property is optional; default is PLAINTEXT.

+
+
+

kafka.ssl.keystore.location#

+

Location of the keystore file used for connection to Kafka cluster.

+

This property is optional.

+
+
+

kafka.ssl.keystore.password#

+

Password for the keystore file used for connection to Kafka cluster.

+

This property is optional, but required when kafka.ssl.keystore.location is given.

+
+
+

kafka.ssl.keystore.type#

+

File format of the keystore file. +Valid values are: JKS, PKCS12.

+

This property is optional; default is JKS.

+
+
+

kafka.ssl.truststore.location#

+

Location of the truststore file used for connection to Kafka cluster.

+

This property is optional.

+
+
+

kafka.ssl.truststore.password#

+

Password for the truststore file used for connection to Kafka cluster.

+

This property is optional, but required when kafka.ssl.truststore.location is given.

+
+
+

kafka.ssl.truststore.type#

+

File format of the truststore file. +Valid values are: JKS, PKCS12.

+

This property is optional; default is JKS.

+
+
+

kafka.ssl.key.password#

+

Password for the private key in the keystore file used for connection to Kafka cluster.

+

This property is optional. This is required for clients only if two-way authentication is configured, i.e. ssl.client.auth=required.

+
+
+

kafka.ssl.endpoint-identification-algorithm#

+

The endpoint identification algorithm used by clients to validate server host name for connection to Kafka cluster. +Kafka uses https as default. Use disabled to disable server host name validation.

+

This property is optional; default is https.

+
+
+
+

Internal columns#

+

The internal column prefix is configurable by kafka.internal-column-prefix +configuration property and defaults to _. A different prefix affects the +internal column names in the following sections. For example, a value of +internal_ changes the partition ID column name from _partition_id +to internal_partition_id.

+

For each defined table, the connector maintains the following columns:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Column name

Type

Description

_partition_id

BIGINT

ID of the Kafka partition which contains this row.

_partition_offset

BIGINT

Offset within the Kafka partition for this row.

_segment_start

BIGINT

Lowest offset in the segment (inclusive) which contains this row. This offset is partition specific.

_segment_end

BIGINT

Highest offset in the segment (exclusive) which contains this row. The offset is partition specific. This is the same value as _segment_start of the next segment (if it exists).

_segment_count

BIGINT

Running count for the current row within the segment. For an uncompacted topic, _segment_start + _segment_count is equal to _partition_offset.

_message_corrupt

BOOLEAN

True if the decoder could not decode the message for this row. When true, data columns mapped from the message should be treated as invalid.

_message

VARCHAR

Message bytes as a UTF-8 encoded string. This is only useful for a text topic.

_message_length

BIGINT

Number of bytes in the message.

_headers

map(VARCHAR, array(VARBINARY))

Headers of the message where values with the same key are grouped as array.

_key_corrupt

BOOLEAN

True if the key decoder could not decode the key for this row. When true, data columns mapped from the key should be treated as invalid.

_key

VARCHAR

Key bytes as a UTF-8 encoded string. This is only useful for textual keys.

_key_length

BIGINT

Number of bytes in the key.

_timestamp

TIMESTAMP

Message timestamp.

+

For tables without a table definition file, the _key_corrupt and +_message_corrupt columns will always be false.

+
+
+

Table schema and schema registry usage#

+

The table schema for the messages can be supplied to the connector with a +configuration file or a schema registry. It also provides a mechanism for the +connector to discover tables.

+

You must configure the supplier with the kafka.table-description-supplier +property, setting it to FILE or CONFLUENT. Each table description +supplier has a separate set of configuration properties.

+

Refer to the following subsections for more detail. The FILE table +description supplier is the default, and the value is case insensitive.

+
+

File table description supplier#

+

In order to use the file-based table description supplier, the +kafka.table-description-supplier must be set to FILE, which is the +default.

+

In addition, you must set kafka.table-names and +kafka.table-description-dir as described in the following sections:

+
+

kafka.table-names#

+

Comma-separated list of all tables provided by this catalog. A table name can be +unqualified (simple name), and is placed into the default schema (see +below), or it can be qualified with a schema name +(<schema-name>.<table-name>).

+

For each table defined here, a table description file (see below) may exist. If +no table description file exists, the table name is used as the topic name on +Kafka, and no data columns are mapped into the table. The table still contains +all internal columns (see below).

+

This property is required; there is no default and at least one table must be +defined.

+
+
+

kafka.table-description-dir#

+

References a folder within Trino deployment that holds one or more JSON files +(must end with .json) which contain table description files.

+

This property is optional; the default is etc/kafka.

+
+
+

Table definition files#

+

Kafka maintains topics only as byte messages and leaves it to producers +and consumers to define how a message should be interpreted. For Trino, +this data must be mapped into columns to allow queries against the data.

+
+

Note

+

For textual topics that contain JSON data, it is entirely possible to not +use any table definition files, but instead use the Trino +JSON functions and operators to parse the _message column which contains +the bytes mapped into a UTF-8 string. This is cumbersome and makes it +difficult to write SQL queries. This only works when reading data.

+
+

A table definition file consists of a JSON definition for a table. The +name of the file can be arbitrary but must end in .json. Place the +file in the directory configured with the kafka.table-description-dir +property. The table definition file must be accessible from all Trino nodes.

+
{
+    "tableName": ...,
+    "schemaName": ...,
+    "topicName": ...,
+    "key": {
+        "dataFormat": ...,
+        "fields": [
+            ...
+        ]
+    },
+    "message": {
+        "dataFormat": ...,
+        "fields": [
+            ...
+       ]
+    }
+}
+
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Field

Required

Type

Description

tableName

required

string

Trino table name defined by this file.

schemaName

optional

string

Schema containing the table. If omitted, the default schema name is used.

topicName

required

string

Kafka topic that is mapped.

key

optional

JSON object

Field definitions for data columns mapped to the message key.

message

optional

JSON object

Field definitions for data columns mapped to the message itself.

+
+
+

Key and message in Kafka#

+

Starting with Kafka 0.8, each message in a topic can have an optional key. +A table definition file contains sections for both key and message to map +the data onto table columns.

+

Each of the key and message fields in the table definition is a +JSON object that must contain two fields:

+ + + + + + + + + + + + + + + + + + + + +

Field

Required

Type

Description

dataFormat

required

string

Selects the decoder for this group of fields.

fields

required

JSON array

A list of field definitions. Each field definition creates a new column in the Trino table.

+

Each field definition is a JSON object:

+
{
+    "name": ...,
+    "type": ...,
+    "dataFormat": ...,
+    "mapping": ...,
+    "formatHint": ...,
+    "hidden": ...,
+    "comment": ...
+}
+
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Field

Required

Type

Description

name

required

string

Name of the column in the Trino table.

type

required

string

Trino type of the column.

dataFormat

optional

string

Selects the column decoder for this field. Defaults to the default decoder for this row data format and column type.

dataSchema

optional

string

The path or URL where the Avro schema resides. Used only for Avro decoder.

mapping

optional

string

Mapping information for the column. This is decoder specific, see below.

formatHint

optional

string

Sets a column-specific format hint to the column decoder.

hidden

optional

boolean

Hides the column from DESCRIBE <table name> and SELECT *. Defaults to false.

comment

optional

string

Adds a column comment, which is shown with DESCRIBE <table name>.

+

There is no limit on field descriptions for either key or message.

+
+
+
+

Confluent table description supplier#

+

The Confluent table description supplier uses the Confluent Schema Registry to discover +table definitions. It is only tested to work with the Confluent Schema +Registry.

+

The benefits of using the Confluent table description supplier over the file +table description supplier are:

+
    +
  • New tables can be defined without a cluster restart.

  • +
  • Schema updates are detected automatically.

  • +
  • There is no need to define tables manually.

  • +
  • Some Protobuf specific types like oneof and any are supported and mapped to JSON.

  • +
+

When using Protobuf decoder with the Confluent table description supplier, some +additional steps are necessary. For details, refer to Requirements.

+

Set kafka.table-description-supplier to CONFLUENT to use the +schema registry. You must also configure the additional properties in the following table:

+
+

Note

+

Inserts are not supported, and the only data format supported is AVRO.

+
+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + +
Confluent table description supplier properties#

Property name

Description

Default value

kafka.confluent-schema-registry-url

Comma-separated list of URL addresses for the Confluent schema registry. +For example, http://schema-registry-1.example.org:8081,http://schema-registry-2.example.org:8081

kafka.confluent-schema-registry-client-cache-size

The maximum number of subjects that can be stored in the local cache. The +cache stores the schemas locally by subjectId, and is provided by the +Confluent CachingSchemaRegistry client.

1000

kafka.empty-field-strategy

Avro allows empty struct fields, but this is not allowed in Trino. +There are three strategies for handling empty struct fields:

+
+
    +
  • IGNORE - Ignore structs with no fields. This propagates to parents. +For example, an array of structs with no fields is ignored.

  • +
  • FAIL - Fail the query if a struct with no fields is defined.

  • +
  • MARK - Add a marker field named $empty_field_marker, which of type boolean with a null value. +This may be desired if the struct represents a marker field.

  • +
+
+

This can also be modified via the empty_field_strategy session property.

+

IGNORE

kafka.confluent-subjects-cache-refresh-interval

The interval used for refreshing the list of subjects and the definition +of the schema for the subject in the subject’s cache.

1s

+
+

Confluent subject to table name mapping#

+

The subject naming strategy +determines how a subject is resolved from the table name.

+

The default strategy is the TopicNameStrategy, where the key subject is +defined as <topic-name>-key and the value subject is defined as +<topic-name>-value. If other strategies are used there is no way to +determine the subject name beforehand, so it must be specified manually in the +table name.

+

To manually specify the key and value subjects, append to the topic name, +for example: <topic name>&key-subject=<key subject>&value-subject=<value subject>. Both the key-subject and value-subject parameters are +optional. If neither is specified, then the default TopicNameStrategy is +used to resolve the subject name via the topic name. Note that a case +insensitive match must be done, as identifiers cannot contain upper case +characters.

+
+
+

Protobuf-specific type handling in Confluent table description supplier#

+

When using the Confluent table description supplier, the following Protobuf +specific types are supported in addition to the normally supported types:

+
+
oneof#
+

Protobuf schemas containing oneof fields are mapped to a JSON field in +Trino.

+

For example, given the following Protobuf schema:

+
syntax = "proto3";
+
+message schema {
+    oneof test_oneof_column {
+        string string_column = 1;
+        uint32 integer_column = 2;
+        uint64 long_column = 3;
+        double double_column = 4;
+        float float_column = 5;
+        bool boolean_column = 6;
+    }
+}
+
+
+

The corresponding Trino row is a JSON field test_oneof_column +containing a JSON object with a single key. The value of the key matches +the name of the oneof type that is present.

+

In the above example, if the Protobuf message has the +test_oneof_column containing string_column set to a value Trino +then the corresponding Trino row includes a column named +test_oneof_column with the value JSON '{"string_column": "Trino"}'.

+
+
+
+
+
+

Kafka inserts#

+

The Kafka connector supports the use of INSERT statements to write +data to a Kafka topic. Table column data is mapped to Kafka messages as defined +in the table definition file. There are +five supported data formats for key and message encoding:

+ +

These data formats each have an encoder that maps column values into bytes to be +sent to a Kafka topic.

+

Trino supports at-least-once delivery for Kafka producers. This means that +messages are guaranteed to be sent to Kafka topics at least once. If a producer +acknowledgement times out, or if the producer receives an error, it might retry +sending the message. This could result in a duplicate message being sent to the +Kafka topic.

+

The Kafka connector does not allow the user to define which partition will be +used as the target for a message. If a message includes a key, the producer will +use a hash algorithm to choose the target partition for the message. The same +key will always be assigned the same partition.

+
+
+

Type mapping#

+

Because Trino and Kafka each support types that the other does not, this +connector maps some types when reading +(decoding) or writing (encoding) data. Type mapping depends on the format (Raw, Avro, +JSON, CSV).

+
+

Row encoding#

+

Encoding is required to allow writing data; it defines how table columns in +Trino map to Kafka keys and message data.

+

The Kafka connector contains the following encoders:

+
    +
  • raw encoder - Table columns are mapped to a Kafka +message as raw bytes.

  • +
  • CSV encoder - Kafka message is formatted as a +comma-separated value.

  • +
  • JSON encoder - Table columns are mapped to JSON +fields.

  • +
  • Avro encoder - Table columns are mapped to Avro +fields based on an Avro schema.

  • +
  • Protobuf encoder - Table columns are mapped to +Protobuf fields based on a Protobuf schema.

  • +
+
+

Note

+

A table definition file must be defined +for the encoder to work.

+
+
+

Raw encoder#

+

The raw encoder formats the table columns as raw bytes using the mapping +information specified in the +table definition file.

+

The following field attributes are supported:

+
    +
  • dataFormat - Specifies the width of the column data type.

  • +
  • type - Trino data type.

  • +
  • mapping - start and optional end position of bytes to convert +(specified as start or start:end).

  • +
+

The dataFormat attribute selects the number of bytes converted. +If absent, BYTE is assumed. All values are signed.

+

Supported values:

+
    +
  • BYTE - one byte

  • +
  • SHORT - two bytes (big-endian)

  • +
  • INT - four bytes (big-endian)

  • +
  • LONG - eight bytes (big-endian)

  • +
  • FLOAT - four bytes (IEEE 754 format, big-endian)

  • +
  • DOUBLE - eight bytes (IEEE 754 format, big-endian)

  • +
+

The type attribute defines the Trino data type.

+

Different values of dataFormat are supported, depending on the Trino data +type:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Trino data type

dataFormat values

BIGINT

BYTE, SHORT, INT, LONG

INTEGER

BYTE, SHORT, INT

SMALLINT

BYTE, SHORT

TINYINT

BYTE

REAL

FLOAT

DOUBLE

FLOAT, DOUBLE

BOOLEAN

BYTE, SHORT, INT, LONG

VARCHAR / VARCHAR(x)

BYTE

+

No other types are supported.

+

The mapping attribute specifies the range of bytes in a key or +message used for encoding.

+
+

Note

+

Both a start and end position must be defined for VARCHAR types. +Otherwise, there is no way to know how many bytes the message contains. The +raw format mapping information is static and cannot be dynamically changed +to fit the variable width of some Trino data types.

+
+

If only a start position is given:

+
    +
  • For fixed width types, the appropriate number of bytes are used for the +specified dataFormat (see above).

  • +
+

If both a start and end position are given, then:

+
    +
  • For fixed width types, the size must be equal to number of bytes used by +specified dataFormat.

  • +
  • All bytes between start (inclusive) and end (exclusive) are used.

  • +
+
+

Note

+

All mappings must include a start position for encoding to work.

+
+

The encoding for numeric data types (BIGINT, INTEGER, SMALLINT, +TINYINT, REAL, DOUBLE) is straightforward. All numeric types use +big-endian. Floating point types use IEEE 754 format.

+

Example raw field definition in a table definition file +for a Kafka message:

+
{
+  "tableName": "example_table_name",
+  "schemaName": "example_schema_name",
+  "topicName": "example_topic_name",
+  "key": { "..." },
+  "message": {
+    "dataFormat": "raw",
+    "fields": [
+      {
+        "name": "field1",
+        "type": "BIGINT",
+        "dataFormat": "LONG",
+        "mapping": "0"
+      },
+      {
+        "name": "field2",
+        "type": "INTEGER",
+        "dataFormat": "INT",
+        "mapping": "8"
+      },
+      {
+        "name": "field3",
+        "type": "SMALLINT",
+        "dataFormat": "LONG",
+        "mapping": "12"
+      },
+      {
+        "name": "field4",
+        "type": "VARCHAR(6)",
+        "dataFormat": "BYTE",
+        "mapping": "20:26"
+      }
+    ]
+  }
+}
+
+
+

Columns should be defined in the same order they are mapped. There can be no +gaps or overlaps between column mappings. The width of the column as defined by +the column mapping must be equivalent to the width of the dataFormat for all +types except for variable width types.

+

Example insert query for the above table definition:

+
INSERT INTO example_raw_table (field1, field2, field3, field4)
+  VALUES (123456789, 123456, 1234, 'abcdef');
+
+
+
+

Note

+

The raw encoder requires the field size to be known ahead of time, including +for variable width data types like VARCHAR. It also disallows inserting +values that do not match the width defined in the table definition +file. This is done to ensure correctness, as otherwise longer values are +truncated, and shorter values are read back incorrectly due to an undefined +padding character.

+
+
+
+

CSV encoder#

+

The CSV encoder formats the values for each row as a line of +comma-separated-values (CSV) using UTF-8 encoding. The CSV line is formatted +with a comma , as the column delimiter.

+

The type and mapping attributes must be defined for each field:

+
    +
  • type - Trino data type

  • +
  • mapping - The integer index of the column in the CSV line (the first +column is 0, the second is 1, and so on)

  • +
+

dataFormat and formatHint are not supported and must be omitted.

+

The following Trino data types are supported by the CSV encoder:

+
    +
  • BIGINT

  • +
  • INTEGER

  • +
  • SMALLINT

  • +
  • TINYINT

  • +
  • DOUBLE

  • +
  • REAL

  • +
  • BOOLEAN

  • +
  • VARCHAR / VARCHAR(x)

  • +
+

No other types are supported.

+

Column values are converted to strings before they are formatted as a CSV line.

+

The following is an example CSV field definition in a table definition file for a Kafka message:

+
{
+  "tableName": "example_table_name",
+  "schemaName": "example_schema_name",
+  "topicName": "example_topic_name",
+  "key": { "..." },
+  "message": {
+    "dataFormat": "csv",
+    "fields": [
+      {
+        "name": "field1",
+        "type": "BIGINT",
+        "mapping": "0"
+      },
+      {
+        "name": "field2",
+        "type": "VARCHAR",
+        "mapping": "1"
+      },
+      {
+        "name": "field3",
+        "type": "BOOLEAN",
+        "mapping": "2"
+      }
+    ]
+  }
+}
+
+
+

Example insert query for the above table definition:

+
INSERT INTO example_csv_table (field1, field2, field3)
+  VALUES (123456789, 'example text', TRUE);
+
+
+
+
+

JSON encoder#

+

The JSON encoder maps table columns to JSON fields defined in the +table definition file according to +RFC 4627.

+

For fields, the following attributes are supported:

+
    +
  • type - Trino data type of column.

  • +
  • mapping - A slash-separated list of field names to select a field from the +JSON object.

  • +
  • dataFormat - Name of formatter. Required for temporal types.

  • +
  • formatHint - Pattern to format temporal data. Only use with +custom-date-time formatter.

  • +
+

The following Trino data types are supported by the JSON encoder:

+
    +
  • BIGINT

  • +
  • INTEGER

  • +
  • SMALLINT

  • +
  • TINYINT

  • +
  • DOUBLE

  • +
  • REAL

  • +
  • BOOLEAN

  • +
  • VARCHAR

  • +
  • DATE

  • +
  • TIME

  • +
  • TIME WITH TIME ZONE

  • +
  • TIMESTAMP

  • +
  • TIMESTAMP WITH TIME ZONE

  • +
+

No other types are supported.

+

The following dataFormats are available for temporal data:

+
    +
  • iso8601

  • +
  • rfc2822

  • +
  • custom-date-time - Formats temporal data according to +Joda Time +pattern given by formatHint field.

  • +
  • milliseconds-since-epoch

  • +
  • seconds-since-epoch

  • +
+

All temporal data in Kafka supports milliseconds precision.

+

The following table defines which temporal data types are supported by +dataFormats:

+ + + + + + + + + + + + + + + + + + + + + + + +

Trino data type

Decoding rules

DATE

custom-date-time, iso8601

TIME

custom-date-time, iso8601, milliseconds-since-epoch, seconds-since-epoch

TIME WITH TIME ZONE

custom-date-time, iso8601

TIMESTAMP

custom-date-time, iso8601, rfc2822, milliseconds-since-epoch, seconds-since-epoch

TIMESTAMP WITH TIME ZONE

custom-date-time, iso8601, rfc2822, milliseconds-since-epoch, seconds-since-epoch

+

The following is an example JSON field definition in a table definition file for a Kafka message:

+
{
+  "tableName": "example_table_name",
+  "schemaName": "example_schema_name",
+  "topicName": "example_topic_name",
+  "key": { "..." },
+  "message": {
+    "dataFormat": "json",
+    "fields": [
+      {
+        "name": "field1",
+        "type": "BIGINT",
+        "mapping": "field1"
+      },
+      {
+        "name": "field2",
+        "type": "VARCHAR",
+        "mapping": "field2"
+      },
+      {
+        "name": "field3",
+        "type": "TIMESTAMP",
+        "dataFormat": "custom-date-time",
+        "formatHint": "yyyy-dd-MM HH:mm:ss.SSS",
+        "mapping": "field3"
+      }
+    ]
+  }
+}
+
+
+

The following shows an example insert query for the preceding table definition:

+
INSERT INTO example_json_table (field1, field2, field3)
+  VALUES (123456789, 'example text', TIMESTAMP '2020-07-15 01:02:03.456');
+
+
+
+
+

Avro encoder#

+

The Avro encoder serializes rows to Avro records as defined by the +Avro schema. +Trino does not support schemaless Avro encoding.

+
+

Note

+

The Avro schema is encoded with the table column values in each Kafka message.

+
+

The dataSchema must be defined in the table definition file to use the Avro +encoder. It points to the location of the Avro schema file for the key or message.

+

Avro schema files can be retrieved via HTTP or HTTPS from remote server with the +syntax:

+

"dataSchema": "http://example.org/schema/avro_data.avsc"

+

Local files need to be available on all Trino nodes and use an absolute path in +the syntax, for example:

+

"dataSchema": "/usr/local/schema/avro_data.avsc"

+

The following field attributes are supported:

+
    +
  • name - Name of the column in the Trino table.

  • +
  • type - Trino data type of column.

  • +
  • mapping - A slash-separated list of field names to select a field from the +Avro schema. If the field specified in mapping does not exist +in the original Avro schema, then a write operation fails.

  • +
+

The following table lists supported Trino data types, which can be used in type +for the equivalent Avro field type.

+ + + + + + + + + + + + + + + + + + + + + + + +

Trino data type

Avro data type

BIGINT

INT, LONG

REAL

FLOAT

DOUBLE

FLOAT, DOUBLE

BOOLEAN

BOOLEAN

VARCHAR / VARCHAR(x)

STRING

+

No other types are supported.

+

The following example shows an Avro field definition in a kafka.table-description-dirition file for a Kafka message:

+
{
+  "tableName": "example_table_name",
+  "schemaName": "example_schema_name",
+  "topicName": "example_topic_name",
+  "key": { "..." },
+  "message":
+  {
+    "dataFormat": "avro",
+    "dataSchema": "/avro_message_schema.avsc",
+    "fields":
+    [
+      {
+        "name": "field1",
+        "type": "BIGINT",
+        "mapping": "field1"
+      },
+      {
+        "name": "field2",
+        "type": "VARCHAR",
+        "mapping": "field2"
+      },
+      {
+        "name": "field3",
+        "type": "BOOLEAN",
+        "mapping": "field3"
+      }
+    ]
+  }
+}
+
+
+

In the following example, an Avro schema definition for the preceding table +definition is shown:

+
{
+  "type" : "record",
+  "name" : "example_avro_message",
+  "namespace" : "io.trino.plugin.kafka",
+  "fields" :
+  [
+    {
+      "name":"field1",
+      "type":["null", "long"],
+      "default": null
+    },
+    {
+      "name": "field2",
+      "type":["null", "string"],
+      "default": null
+    },
+    {
+      "name":"field3",
+      "type":["null", "boolean"],
+      "default": null
+    }
+  ],
+  "doc:" : "A basic avro schema"
+}
+
+
+

The following is an example insert query for the preceding table definition:

+
+
+
INSERT INTO example_avro_table (field1, field2, field3)

VALUES (123456789, ‘example text’, FALSE);

+
+
+
+
+
+

Protobuf encoder#

+

The Protobuf encoder serializes rows to Protobuf DynamicMessages as defined by +the Protobuf schema.

+
+

Note

+

The Protobuf schema is encoded with the table column values in each Kafka message.

+
+

The dataSchema must be defined in the table definition file to use the +Protobuf encoder. It points to the location of the proto file for the key +or message.

+

Protobuf schema files can be retrieved via HTTP or HTTPS from a remote server +with the syntax:

+

"dataSchema": "http://example.org/schema/schema.proto"

+

Local files need to be available on all Trino nodes and use an absolute path in +the syntax, for example:

+

"dataSchema": "/usr/local/schema/schema.proto"

+

The following field attributes are supported:

+
    +
  • name - Name of the column in the Trino table.

  • +
  • type - Trino type of column.

  • +
  • mapping - slash-separated list of field names to select a field from the +Protobuf schema. If the field specified in mapping does not exist in the +original Protobuf schema, then a write operation fails.

  • +
+

The following table lists supported Trino data types, which can be used in type +for the equivalent Protobuf field type.

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Trino data type

Protobuf data type

BOOLEAN

bool

INTEGER

int32, uint32, sint32, fixed32, sfixed32

BIGINT

int64, uint64, sint64, fixed64, sfixed64

DOUBLE

double

REAL

float

VARCHAR / VARCHAR(x)

string

VARBINARY

bytes

ROW

Message

ARRAY

Protobuf type with repeated field

MAP

Map

TIMESTAMP

Timestamp, predefined in timestamp.proto

+

The following example shows a Protobuf field definition in a table definition +file for a Kafka message:

+
{
+  "tableName": "example_table_name",
+  "schemaName": "example_schema_name",
+  "topicName": "example_topic_name",
+  "key": { "..." },
+  "message":
+  {
+    "dataFormat": "protobuf",
+    "dataSchema": "/message_schema.proto",
+    "fields":
+    [
+      {
+        "name": "field1",
+        "type": "BIGINT",
+        "mapping": "field1"
+      },
+      {
+        "name": "field2",
+        "type": "VARCHAR",
+        "mapping": "field2"
+      },
+      {
+        "name": "field3",
+        "type": "BOOLEAN",
+        "mapping": "field3"
+      }
+    ]
+  }
+}
+
+
+

In the following example, a Protobuf schema definition for the preceding table +definition is shown:

+
syntax = "proto3";
+
+message schema {
+  uint64 field1 = 1 ;
+  string field2 = 2;
+  bool field3 = 3;
+}
+
+
+

The following is an example insert query for the preceding table definition:

+
INSERT INTO example_protobuf_table (field1, field2, field3)
+  VALUES (123456789, 'example text', FALSE);
+
+
+
+
+
+

Row decoding#

+

For key and message, a decoder is used to map message and key data onto table columns.

+

The Kafka connector contains the following decoders:

+
    +
  • raw - Kafka message is not interpreted; ranges of raw message bytes are mapped to table columns.

  • +
  • csv - Kafka message is interpreted as comma separated message, and fields are mapped to table columns.

  • +
  • json - Kafka message is parsed as JSON, and JSON fields are mapped to table columns.

  • +
  • avro - Kafka message is parsed based on an Avro schema, and Avro fields are mapped to table columns.

  • +
  • protobuf - Kafka message is parsed based on a Protobuf schema, and Protobuf fields are mapped to table columns.

  • +
+
+

Note

+

If no table definition file exists for a table, the dummy decoder is used, +which does not expose any columns.

+
+
+

Raw decoder#

+

The raw decoder supports reading of raw byte-based values from Kafka message +or key, and converting it into Trino columns.

+

For fields, the following attributes are supported:

+
    +
  • dataFormat - Selects the width of the data type converted.

  • +
  • type - Trino data type. See table later min this document for list of +supported data types.

  • +
  • mapping - <start>[:<end>] - Start and end position of bytes to convert (optional).

  • +
+

The dataFormat attribute selects the number of bytes converted. +If absent, BYTE is assumed. All values are signed.

+

Supported values are:

+
    +
  • BYTE - one byte

  • +
  • SHORT - two bytes (big-endian)

  • +
  • INT - four bytes (big-endian)

  • +
  • LONG - eight bytes (big-endian)

  • +
  • FLOAT - four bytes (IEEE 754 format)

  • +
  • DOUBLE - eight bytes (IEEE 754 format)

  • +
+

The type attribute defines the Trino data type on which the value is mapped.

+

Depending on the Trino type assigned to a column, different values of dataFormat can be used:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Trino data type

Allowed dataFormat values

BIGINT

BYTE, SHORT, INT, LONG

INTEGER

BYTE, SHORT, INT

SMALLINT

BYTE, SHORT

TINYINT

BYTE

DOUBLE

DOUBLE, FLOAT

BOOLEAN

BYTE, SHORT, INT, LONG

VARCHAR / VARCHAR(x)

BYTE

+

No other types are supported.

+

The mapping attribute specifies the range of the bytes in a key or +message used for decoding. It can be one or two numbers separated by a colon (<start>[:<end>]).

+

If only a start position is given:

+
    +
  • For fixed width types, the column will use the appropriate number of bytes for the specified dataFormat (see above).

  • +
  • When VARCHAR value is decoded, all bytes from start position till the end of the message will be used.

  • +
+

If start and end position are given:

+
    +
  • For fixed width types, the size must be equal to number of bytes used by specified dataFormat.

  • +
  • For VARCHAR all bytes between start (inclusive) and end (exclusive) are used.

  • +
+

If no mapping attribute is specified, it is equivalent to setting start position to 0 and leaving end position undefined.

+

The decoding scheme of numeric data types (BIGINT, INTEGER, SMALLINT, TINYINT, DOUBLE) is straightforward. +A sequence of bytes is read from input message and decoded according to either:

+
    +
  • big-endian encoding (for integer types)

  • +
  • IEEE 754 format for (for DOUBLE).

  • +
+

Length of decoded byte sequence is implied by the dataFormat.

+

For VARCHAR data type a sequence of bytes is interpreted according to UTF-8 +encoding.

+
+
+

CSV decoder#

+

The CSV decoder converts the bytes representing a message or key into a +string using UTF-8 encoding and then interprets the result as a CSV +(comma-separated value) line.

+

For fields, the type and mapping attributes must be defined:

+
    +
  • type - Trino data type. See the following table for a list of supported data types.

  • +
  • mapping - The index of the field in the CSV record.

  • +
+

The dataFormat and formatHint attributes are not supported and must be omitted.

+

Table below lists supported Trino types, which can be used in type and decoding scheme:

+ ++++ + + + + + + + + + + + + + + + + + + + +

Trino data type

Decoding rules

BIGINT, INTEGER, SMALLINT, TINYINT

Decoded using Java Long.parseLong()

DOUBLE

Decoded using Java Double.parseDouble()

BOOLEAN

“true” character sequence maps to true; Other character sequences map to false

VARCHAR, VARCHAR(x)

Used as is

+

No other types are supported.

+
+
+

JSON decoder#

+

The JSON decoder converts the bytes representing a message or key into a +JSON according to RFC 4627. Note that the message or key MUST convert +into a JSON object, not an array or simple type.

+

For fields, the following attributes are supported:

+
    +
  • type - Trino data type of column.

  • +
  • dataFormat - Field decoder to be used for column.

  • +
  • mapping - slash-separated list of field names to select a field from the JSON object.

  • +
  • formatHint - Only for custom-date-time.

  • +
+

The JSON decoder supports multiple field decoders, with _default being +used for standard table columns and a number of decoders for date- and +time-based types.

+

The following table lists Trino data types, which can be used as in type, and matching field decoders, +which can be specified via dataFormat attribute.

+ ++++ + + + + + + + + + + + + + + + + + + + + + + + + + +

Trino data type

Allowed dataFormat values

BIGINT, INTEGER, SMALLINT, TINYINT, DOUBLE, BOOLEAN, VARCHAR, VARCHAR(x)

Default field decoder (omitted dataFormat attribute)

DATE

custom-date-time, iso8601

TIME

custom-date-time, iso8601, milliseconds-since-epoch, seconds-since-epoch

TIME WITH TIME ZONE

custom-date-time, iso8601

TIMESTAMP

custom-date-time, iso8601, rfc2822, milliseconds-since-epoch, seconds-since-epoch

TIMESTAMP WITH TIME ZONE

custom-date-time, iso8601, rfc2822, milliseconds-since-epoch seconds-since-epoch

+

No other types are supported.

+
+
Default field decoder#
+

This is the standard field decoder, supporting all the Trino physical data +types. A field value is transformed under JSON conversion rules into +boolean, long, double or string values. For non-date/time based columns, +this decoder should be used.

+
+
+
Date and time decoders#
+

To convert values from JSON objects into Trino DATE, TIME, TIME WITH TIME ZONE, +TIMESTAMP or TIMESTAMP WITH TIME ZONE columns, special decoders must be selected using the +dataFormat attribute of a field definition.

+
    +
  • iso8601 - Text based, parses a text field as an ISO 8601 timestamp.

  • +
  • rfc2822 - Text based, parses a text field as an RFC 2822 timestamp.

  • +
  • +
    custom-date-time - Text based, parses a text field according to Joda format pattern

    specified via formatHint attribute. Format pattern should conform +to https://www.joda.org/joda-time/apidocs/org/joda/time/format/DateTimeFormat.html.

    +
    +
    +
  • +
  • milliseconds-since-epoch - Number-based; interprets a text or number as number of milliseconds since the epoch.

  • +
  • seconds-since-epoch - Number-based; interprets a text or number as number of milliseconds since the epoch.

  • +
+

For TIMESTAMP WITH TIME ZONE and TIME WITH TIME ZONE data types, if timezone information is present in decoded value, it will +be used as Trino value. Otherwise result time zone will be set to UTC.

+
+
+
+

Avro decoder#

+

The Avro decoder converts the bytes representing a message or key in +Avro format based on a schema. The message must have the Avro schema embedded. +Trino does not support schemaless Avro decoding.

+

For key/message, using avro decoder, the dataSchema must be defined. +This should point to the location of a valid Avro schema file of the message which needs to be decoded. This location can be a remote web server +(e.g.: dataSchema: 'http://example.org/schema/avro_data.avsc') or local file system(e.g.: dataSchema: '/usr/local/schema/avro_data.avsc'). +The decoder fails if this location is not accessible from the Trino coordinator node.

+

For fields, the following attributes are supported:

+
    +
  • name - Name of the column in the Trino table.

  • +
  • type - Trino data type of column.

  • +
  • mapping - A slash-separated list of field names to select a field from the Avro schema. If field specified in mapping does not exist in the original Avro schema, then a read operation returns NULL.

  • +
+

The following table lists the supported Trino types which can be used in type for the equivalent Avro field types:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Trino data type

Allowed Avro data type

BIGINT

INT, LONG

DOUBLE

DOUBLE, FLOAT

BOOLEAN

BOOLEAN

VARCHAR / VARCHAR(x)

STRING

VARBINARY

FIXED, BYTES

ARRAY

ARRAY

MAP

MAP

+

No other types are supported.

+
+
Avro schema evolution#
+

The Avro decoder supports schema evolution feature with backward compatibility. With backward compatibility, +a newer schema can be used to read Avro data created with an older schema. Any change in the Avro schema must also be +reflected in Trino’s topic definition file. Newly added/renamed fields must have a default value in the Avro schema file.

+

The schema evolution behavior is as follows:

+
    +
  • Column added in new schema: +Data created with an older schema produces a default value when the table is using the new schema.

  • +
  • Column removed in new schema: +Data created with an older schema no longer outputs the data from the column that was removed.

  • +
  • Column is renamed in the new schema: +This is equivalent to removing the column and adding a new one, and data created with an older schema +produces a default value when table is using the new schema.

  • +
  • Changing type of column in the new schema: +If the type coercion is supported by Avro, then the conversion happens. An +error is thrown for incompatible types.

  • +
+
+
+
+

Protobuf decoder#

+

The Protobuf decoder converts the bytes representing a message or key in +Protobuf formatted message based on a schema.

+

For key/message, using the protobuf decoder, the dataSchema must be +defined. It points to the location of a valid proto file of the message +which needs to be decoded. This location can be a remote web server, +dataSchema: 'http://example.org/schema/schema.proto', or local file, +dataSchema: '/usr/local/schema/schema.proto'. The decoder fails if the +location is not accessible from the coordinator.

+

For fields, the following attributes are supported:

+
    +
  • name - Name of the column in the Trino table.

  • +
  • type - Trino data type of column.

  • +
  • mapping - slash-separated list of field names to select a field from the +Protobuf schema. If field specified in mapping does not exist in the +original proto file then a read operation returns NULL.

  • +
+

The following table lists the supported Trino types which can be used in +type for the equivalent Protobuf field types:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Trino data type

Allowed Protobuf data type

BOOLEAN

bool

INTEGER

int32, uint32, sint32, fixed32, sfixed32

BIGINT

int64, uint64, sint64, fixed64, sfixed64

DOUBLE

double

REAL

float

VARCHAR / VARCHAR(x)

string

VARBINARY

bytes

ROW

Message

ARRAY

Protobuf type with repeated field

MAP

Map

TIMESTAMP

Timestamp, predefined in timestamp.proto

JSON

oneof (Confluent table supplier only), Any

+
+
any#
+

Message types with an Any +field contain an arbitrary serialized message as bytes and a type URL to resolve +that message’s type with a scheme of file://, http://, or https://. +The connector reads the contents of the URL to create the type descriptor +for the Any message and convert the message to JSON. This behavior is enabled +by setting kafka.protobuf-any-support-enabled to true.

+

The descriptors for each distinct URL are cached for performance reasons and +any modifications made to the type returned by the URL requires a restart of +Trino.

+

For example, given the following Protobuf schema which defines MyMessage +with three columns:

+
syntax = "proto3";
+
+message MyMessage {
+  string stringColumn = 1;
+  uint32 integerColumn = 2;
+  uint64 longColumn = 3;
+}
+
+
+

And a separate schema which uses an Any type which is a packed message +of the above type and a valid URL:

+
syntax = "proto3";
+
+import "google/protobuf/any.proto";
+
+message schema {
+    google.protobuf.Any any_message = 1;
+}
+
+
+

The corresponding Trino column is named any_message of type JSON +containing a JSON-serialized representation of the Protobuf message:

+
{
+  "@type":"file:///path/to/schemas/MyMessage",
+  "longColumn":"493857959588286460",
+  "numberColumn":"ONE",
+  "stringColumn":"Trino"
+}
+
+
+
+
+
Protobuf schema evolution#
+

The Protobuf decoder supports the schema evolution feature with backward +compatibility. With backward compatibility, a newer schema can be used to read +Protobuf data created with an older schema. Any change in the Protobuf schema +must also be reflected in the topic definition file.

+

The schema evolution behavior is as follows:

+
    +
  • Column added in new schema: +Data created with an older schema produces a default value when the table is using the new schema.

  • +
  • Column removed in new schema: +Data created with an older schema no longer outputs the data from the column that was removed.

  • +
  • Column is renamed in the new schema: +This is equivalent to removing the column and adding a new one, and data created with an older schema +produces a default value when table is using the new schema.

  • +
  • Changing type of column in the new schema: +If the type coercion is supported by Protobuf, then the conversion happens. An error is thrown for incompatible types.

  • +
+
+
+
Protobuf limitations#
+
    +
  • Protobuf Timestamp has a nanosecond precision but Trino supports +decoding/encoding at microsecond precision.

  • +
+
+
+
+
+
+

SQL support#

+

The connector provides read and write access to data and metadata in Trino +tables populated by Kafka topics. See Row decoding for more +information.

+

In addition to the globally available +and read operation statements, the connector +supports the following features:

+ +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/kinesis.html b/430/connector/kinesis.html new file mode 100644 index 000000000..fcf99329b --- /dev/null +++ b/430/connector/kinesis.html @@ -0,0 +1,1455 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Kinesis connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Kinesis connector#

+

Kinesis is Amazon’s fully managed cloud-based service for real-time processing of large, distributed data streams.

+

This connector allows the use of Kinesis streams as tables in Trino, such that each data-blob/message +in a Kinesis stream is presented as a row in Trino. A flexible table mapping approach lets us +treat fields of the messages as columns in the table.

+

Under the hood, a Kinesis +shard iterator +is used to retrieve the records, along with a series of +GetRecords calls. +The shard iterator starts by default 24 hours before the current time, and works its way forward. +To be able to query a stream, table mappings are needed. These table definitions can be +stored on Amazon S3 (preferred), or stored in a local directory on each Trino node.

+

This connector is a read-only connector. It can only fetch data from Kinesis streams, +but cannot create streams or push data into existing streams.

+

To configure the Kinesis connector, create a catalog properties file +etc/catalog/example.properties with the following contents, replacing the +properties as appropriate:

+
connector.name=kinesis
+kinesis.access-key=XXXXXX
+kinesis.secret-key=XXXXXX
+
+
+
+

Configuration properties#

+

The following configuration properties are available:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

kinesis.access-key

Access key to AWS account or blank to use default provider chain

kinesis.secret-key

Secret key to AWS account or blank to use default provider chain

kinesis.aws-region

AWS region to be used to read kinesis stream from

kinesis.default-schema

Default schema name for tables

kinesis.table-description-location

Directory containing table description files

kinesis.table-description-refresh-interval

How often to get the table description from S3

kinesis.hide-internal-columns

Controls whether internal columns are part of the table schema or not

kinesis.batch-size

Maximum number of records to return in one batch

kinesis.fetch-attempts

Read attempts made when no records returned and not caught up

kinesis.max-batches

Maximum batches to read from Kinesis in one single query

kinesis.sleep-time

Time for thread to sleep waiting to make next attempt to fetch batch

kinesis.iterator-from-timestamp

Begin iterating from a given timestamp instead of the trim horizon

kinesis.iterator-offset-seconds

Number of seconds before current time to start iterating

+
+

kinesis.access-key#

+

Defines the access key ID for AWS root account or IAM roles, which is used to sign programmatic requests to AWS Kinesis.

+

This property is optional; if not defined, the connector tries to follow Default-Credential-Provider-Chain provided by AWS in the following order:

+
    +
  • Environment Variable: Load credentials from environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY.

  • +
  • Java System Variable: Load from java system as aws.accessKeyId and aws.secretKey.

  • +
  • Profile Credentials File: Load from file typically located at ~/.aws/credentials.

  • +
  • Instance profile credentials: These credentials can be used on EC2 instances, and are delivered through the Amazon EC2 metadata service.

  • +
+
+
+

kinesis.secret-key#

+

Defines the secret key for AWS root account or IAM roles, which together with Access Key ID, is used to sign programmatic requests to AWS Kinesis.

+

This property is optional; if not defined, connector will try to follow Default-Credential-Provider-Chain same as above.

+
+
+

kinesis.aws-region#

+

Defines AWS Kinesis regional endpoint. Selecting appropriate region may reduce latency in fetching data.

+

This field is optional; The default region is us-east-1 referring to end point ‘kinesis.us-east-1.amazonaws.com’.

+

See Kinesis Data Streams regions +for a current list of available regions.

+
+
+

kinesis.default-schema#

+

Defines the schema which contains all tables that were defined without a qualifying schema name.

+

This property is optional; the default is default.

+
+
+

kinesis.table-description-location#

+

References an S3 URL or a folder within Trino deployment that holds one or more JSON files ending with .json, which contain table description files. +The S3 bucket and folder will be checked every 10 minutes for updates and changed files.

+

This property is optional; the default is etc/kinesis.

+
+
+

kinesis.table-description-refresh-interval#

+

This property controls how often the table description is refreshed from S3.

+

This property is optional; the default is 10m.

+
+
+

kinesis.batch-size#

+

Defines the maximum number of records to return in one request to Kinesis Streams. Maximum limit is 10000 records.

+

This field is optional; the default value is 10000.

+
+
+

kinesis.max-batches#

+

The maximum number of batches to read in a single query. The default value is 1000.

+
+
+

kinesis.fetch-attempts#

+

Defines the number of attempts made to read a batch from Kinesis Streams, when no records are returned and the millis behind latest +parameter shows we are not yet caught up. When records are returned no additional attempts are necessary. +GetRecords has been observed to return no records even though the shard is not empty. +That is why multiple attempts need to be made.

+

This field is optional; the default value is 2.

+
+
+

kinesis.sleep-time#

+

Defines the duration for which a thread needs to sleep between kinesis.fetch-attempts made to fetch data.

+

This field is optional; the default value is 1000ms.

+
+
+

kinesis.iterator-from-timestamp#

+

Use an initial shard iterator type of AT_TIMESTAMP starting kinesis.iterator-offset-seconds before the current time. +When this is false, an iterator type of TRIM_HORIZON is used, meaning it starts from the oldest record in the stream.

+

The default is true.

+
+
+

kinesis.iterator-offset-seconds#

+

When kinesis.iterator-from-timestamp is true, the shard iterator starts at kinesis.iterator-offset-seconds before the current time.

+

The default is 86400 seconds (24 hours).

+
+
+

kinesis.hide-internal-columns#

+

In addition to the data columns defined in a table description file, the connector maintains a number of additional columns for each table. +If these columns are hidden, they can still be used in queries, but they do not show up in DESCRIBE <table-name> or SELECT *.

+

This property is optional; the default is true.

+
+
+
+

Internal columns#

+

For each defined table, the connector maintains the following columns:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Column name

Type

Description

_shard_id

VARCHAR

ID of the Kinesis stream shard which contains this row.

_shard_sequence_id

VARCHAR

Sequence id within the Kinesis shard for this row.

_segment_start

BIGINT

Lowest offset in the segment (inclusive) which contains this row. This offset is partition specific.

_segment_end

BIGINT

Highest offset in the segment (exclusive) which contains this row. The offset is partition specific. This is the same value as _segment_start of the next segment (if it exists).

_segment_count

BIGINT

Running count for the current row within the segment. For an uncompacted topic, _segment_start + _segment_count is equal to _partition_offset.

_message_valid

BOOLEAN

True if the decoder could decode the message successfully for this row. When false, data columns mapped from the message should be treated as invalid.

_message

VARCHAR

Message bytes as an UTF-8 encoded string. This is only useful for a text topic.

_message_length

BIGINT

Number of bytes in the message.

_message_timestamp

TIMESTAMP

Approximate arrival time of the message (milliseconds granularity).

_key

VARCHAR

Key bytes as an UTF-8 encoded string. This is only useful for textual keys.

_partition_key

VARCHAR

Partition Key bytes as a UTF-8 encoded string.

+

For tables without a table definition file, the _message_valid column is always true.

+
+
+

Table definition#

+

A table definition file consists of a JSON definition for a table, which corresponds to one stream in Kinesis. +The name of the file can be arbitrary but must end in .json. The structure of the table definition is as follows:

+
{
+      "tableName": ...,
+      "schemaName": ...,
+      "streamName": ...,
+      "message": {
+          "dataFormat": ...,
+          "fields": [
+              ...
+         ]
+      }
+  }
+
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Field

Required

Type

Description

tableName

required

string

Trino table name defined by this file.

schemaName

optional

string

Schema which contains the table. If omitted, the default schema name is used.

streamName

required

string

Name of the Kinesis Stream that is mapped

message

optional

JSON object

Field definitions for data columns mapped to the message itself.

+

Every message in a Kinesis stream can be decoded using the definition provided in the message object. +The JSON object message in the table definition contains two fields:

+ + + + + + + + + + + + + + + + + + + + +

Field

Required

Type

Description

dataFormat

required

string

Selects the decoder for this group of fields.

fields

required

JSON array

A list of field definitions. Each field definition creates a new column in the Trino table.

+

Each field definition is a JSON object. At a minimum, a name, type, and mapping must be provided. +The overall structure looks like this:

+
{
+    "name": ...,
+    "type": ...,
+    "dataFormat": ...,
+    "mapping": ...,
+    "formatHint": ...,
+    "hidden": ...,
+    "comment": ...
+}
+
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Field

Required

Type

Description

name

required

string

Name of the column in the Trino table.

type

required

string

Trino type of the column.

dataFormat

optional

string

Selects the column decoder for this field. Defaults to the default decoder for this row data format and column type.

mapping

optional

string

Mapping information for the column. This is decoder specific – see below.

formatHint

optional

string

Sets a column specific format hint to the column decoder.

hidden

optional

boolean

Hides the column from DESCRIBE <table name> and SELECT *. Defaults to false.

comment

optional

string

Adds a column comment which is shown with DESCRIBE <table name>.

+

The name field is exposed to Trino as the column name, while the mapping field is the portion of the message that gets +mapped to that column. For JSON object messages, this refers to the field name of an object, and can be a path that drills +into the object structure of the message. Additionally, you can map a field of the JSON object to a string column type, +and if it is a more complex type (JSON array or JSON object) then the JSON itself becomes the field value.

+

There is no limit on field descriptions for either key or message.

+
+
+

Type mapping#

+

Because Trino and Kinesis each support types that the other does not, this +connector maps some types when reading data. Type +mapping depends on the RAW, CSV, JSON, and AVRO file formats.

+
+

Row decoding#

+

A decoder is used to map data to table columns.

+

The connector contains the following decoders:

+
    +
  • raw: Message is not interpreted; ranges of raw message bytes are mapped +to table columns.

  • +
  • csv: Message is interpreted as comma separated message, and fields are +mapped to table columns.

  • +
  • json: Message is parsed as JSON, and JSON fields are mapped to table +columns.

  • +
  • avro: Message is parsed based on an Avro schema, and Avro fields are +mapped to table columns.

  • +
+
+

Note

+

If no table definition file exists for a table, the dummy decoder is +used, which does not expose any columns.

+
+
+

Raw decoder#

+

The raw decoder supports reading of raw byte-based values from message or key, +and converting it into Trino columns.

+

For fields, the following attributes are supported:

+
    +
  • dataFormat - Selects the width of the data type converted.

  • +
  • type - Trino data type. See the following table for a list of supported +data types.

  • +
  • mapping - <start>[:<end>] - Start and end position of bytes to convert +(optional).

  • +
+

The dataFormat attribute selects the number of bytes converted. If absent, +BYTE is assumed. All values are signed.

+

Supported values are:

+
    +
  • BYTE - one byte

  • +
  • SHORT - two bytes (big-endian)

  • +
  • INT - four bytes (big-endian)

  • +
  • LONG - eight bytes (big-endian)

  • +
  • FLOAT - four bytes (IEEE 754 format)

  • +
  • DOUBLE - eight bytes (IEEE 754 format)

  • +
+

The type attribute defines the Trino data type on which the value is mapped.

+

Depending on the Trino type assigned to a column, different values of dataFormat +can be used:

+ ++++ + + + + + + + + + + + + + + + + + + + + + + + + + +

Trino data type

Allowed dataFormat values

BIGINT

BYTE, SHORT, INT, LONG

INTEGER

BYTE, SHORT, INT

SMALLINT

BYTE, SHORT

DOUBLE

DOUBLE, FLOAT

BOOLEAN

BYTE, SHORT, INT, LONG

VARCHAR / VARCHAR(x)

BYTE

+

No other types are supported.

+

The mapping attribute specifies the range of the bytes in a key or message +used for decoding. It can be one or two numbers separated by a colon +(<start>[:<end>]).

+

If only a start position is given:

+
    +
  • For fixed width types, the column uses the appropriate number of bytes for +the specified dataFormat (see above).

  • +
  • When the VARCHAR value is decoded, all bytes from the start position to +the end of the message is used.

  • +
+

If start and end position are given:

+
    +
  • For fixed width types, the size must be equal to the number of bytes used by +specified dataFormat.

  • +
  • For the VARCHAR data type all bytes between start (inclusive) and end +(exclusive) are used.

  • +
+

If no mapping attribute is specified, it is equivalent to setting the start +position to 0 and leaving the end position undefined.

+

The decoding scheme of numeric data types (BIGINT, INTEGER, +SMALLINT, TINYINT, DOUBLE) is straightforward. A sequence of bytes +is read from input message and decoded according to either:

+
    +
  • big-endian encoding (for integer types)

  • +
  • IEEE 754 format for (for DOUBLE).

  • +
+

The length of a decoded byte sequence is implied by the dataFormat.

+

For the VARCHAR data type, a sequence of bytes is interpreted according to +UTF-8 encoding.

+
+
+

CSV decoder#

+

The CSV decoder converts the bytes representing a message or key into a string +using UTF-8 encoding, and interprets the result as a link of comma-separated +values.

+

For fields, the type and mapping attributes must be defined:

+
    +
  • type - Trino data type. See the following table for a list of supported +data types.

  • +
  • mapping - The index of the field in the CSV record.

  • +
+

The dataFormat and formatHint attributes are not supported and must be +omitted.

+ ++++ + + + + + + + + + + + + + + + + + + + +

Trino data type

Decoding rules

BIGINT, INTEGER, SMALLINT, TINYINT

Decoded using Java Long.parseLong()

DOUBLE

Decoded using Java Double.parseDouble()

BOOLEAN

“true” character sequence maps to true. Other character sequences map +to false

VARCHAR / VARCHAR(x)

Used as is

+

No other types are supported.

+
+
+

JSON decoder#

+

The JSON decoder converts the bytes representing a message or key into +Javascript Object Notaion (JSON) according to RFC 4627. The message or key +must convert into a JSON object, not an array or simple type.

+

For fields, the following attributes are supported:

+
    +
  • type - Trino data type of column.

  • +
  • dataFormat - Field decoder to be used for column.

  • +
  • mapping - Slash-separated list of field names to select a field from the +JSON object.

  • +
  • formatHint - Only for custom-date-time.

  • +
+

The JSON decoder supports multiple field decoders with _default being used +for standard table columns and a number of decoders for date and time-based +types.

+

The following table lists Trino data types, which can be used in type and +matching field decoders, and specified via dataFormat attribute:

+ ++++ + + + + + + + + + + + + + + + + + + + + + + + + + +

Trino data type

Allowed dataFormat values

BIGINT, INTEGER, SMALLINT, TINYINT, DOUBLE, +BOOLEAN, VARCHAR, VARCHAR(x)

Default field decoder (omitted dataFormat attribute)

DATE

custom-date-time, iso8601

TIME

custom-date-time, iso8601, milliseconds-since-epoch, +seconds-since-epoch

TIME WITH TIME ZONE

custom-date-time, iso8601

TIMESTAMP

custom-date-time, iso8601, rfc2822, +milliseconds-since-epoch, seconds-since-epoch

TIMESTAMP WITH TIME ZONE

custom-date-time, iso8601, rfc2822, +milliseconds-since-epoch, seconds-since-epoch

+

No other types are supported.

+
+
Default field decoder#
+

This is the standard field decoder. It supports all the Trino physical data +types. A field value is transformed under JSON conversion rules into boolean, +long, double, or string values. This decoder should be used for columns that are +not date or time based.

+
+
+
Date and time decoders#
+

To convert values from JSON objects to Trino DATE, TIME, TIME WITH TIME ZONE, TIMESTAMP or TIMESTAMP WITH TIME ZONE columns, select +special decoders using the dataFormat attribute of a field definition.

+
    +
  • iso8601 - Text based, parses a text field as an ISO 8601 timestamp.

  • +
  • rfc2822 - Text based, parses a text field as an RFC 2822 timestamp.

  • +
  • custom-date-time - Text based, parses a text field according to Joda +format pattern specified via formatHint attribute. The format pattern +should conform to +https://www.joda.org/joda-time/apidocs/org/joda/time/format/DateTimeFormat.html.

  • +
  • milliseconds-since-epoch - Number-based, interprets a text or number as +number of milliseconds since the epoch.

  • +
  • seconds-since-epoch - Number-based, interprets a text or number as number +of milliseconds since the epoch.

  • +
+

For TIMESTAMP WITH TIME ZONE and TIME WITH TIME ZONE data types, if +timezone information is present in decoded value, it is used as a Trino value. +Otherwise, the result time zone is set to UTC.

+
+
+
+

Avro decoder#

+

The Avro decoder converts the bytes representing a message or key in Avro format +based on a schema. The message must have the Avro schema embedded. Trino does +not support schemaless Avro decoding.

+

The dataSchema must be defined for any key or message using Avro +decoder. Avro decoder should point to the location of a valid Avro +schema file of the message which must be decoded. This location can be a remote +web server (e.g.: dataSchema: 'http://example.org/schema/avro_data.avsc') or +local file system(e.g.: dataSchema: '/usr/local/schema/avro_data.avsc'). The +decoder fails if this location is not accessible from the Trino cluster.

+

The following attributes are supported:

+
    +
  • name - Name of the column in the Trino table.

  • +
  • type - Trino data type of column.

  • +
  • mapping - A slash-separated list of field names to select a field from the +Avro schema. If the field specified in mapping does not exist in the +original Avro schema, a read operation returns NULL.

  • +
+

The following table lists the supported Trino types that can be used in type +for the equivalent Avro field types:

+ ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Trino data type

Allowed Avro data type

BIGINT

INT, LONG

DOUBLE

DOUBLE, FLOAT

BOOLEAN

BOOLEAN

VARCHAR / VARCHAR(x)

STRING

VARBINARY

FIXED, BYTES

ARRAY

ARRAY

MAP

MAP

+

No other types are supported.

+
+
Avro schema evolution#
+

The Avro decoder supports schema evolution with backward compatibility. With +backward compatibility, a newer schema can be used to read Avro data created +with an older schema. Any change in the Avro schema must also be reflected in +Trino’s topic definition file. Newly added or renamed fields must have a +default value in the Avro schema file.

+

The schema evolution behavior is as follows:

+
    +
  • Column added in new schema: Data created with an older schema produces a +default value when the table is using the new schema.

  • +
  • Column removed in new schema: Data created with an older schema no longer +outputs the data from the column that was removed.

  • +
  • Column is renamed in the new schema: This is equivalent to removing the column +and adding a new one, and data created with an older schema produces a +default value when the table is using the new schema.

  • +
  • Changing type of column in the new schema: If the type coercion is supported +by Avro, then the conversion happens. An error is thrown for incompatible +types.

  • +
+
+
+
+
+
+

SQL support#

+

The connector provides globally available and +read operation statements to access data and +metadata from Kinesis streams.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/kudu.html b/430/connector/kudu.html new file mode 100644 index 000000000..71e5e3a47 --- /dev/null +++ b/430/connector/kudu.html @@ -0,0 +1,1431 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Kudu connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Kudu connector#

+

The Kudu connector allows querying, inserting and deleting data in Apache Kudu.

+
+

Requirements#

+

To connect to Kudu, you need:

+
    +
  • Kudu version 1.13.0 or higher.

  • +
  • Network access from the Trino coordinator and workers to Kudu. Port 7051 is +the default port.

  • +
+
+
+

Configuration#

+

To configure the Kudu connector, create a catalog properties file +etc/catalog/kudu.properties with the following contents, +replacing the properties as appropriate:

+
connector.name=kudu
+
+## Defaults to NONE
+kudu.authentication.type = NONE
+
+## List of Kudu master addresses, at least one is needed (comma separated)
+## Supported formats: example.com, example.com:7051, 192.0.2.1, 192.0.2.1:7051,
+##                    [2001:db8::1], [2001:db8::1]:7051, 2001:db8::1
+kudu.client.master-addresses=localhost
+
+## Kudu does not support schemas, but the connector can emulate them optionally.
+## By default, this feature is disabled, and all tables belong to the default schema.
+## For more details see connector documentation.
+#kudu.schema-emulation.enabled=false
+
+## Prefix to use for schema emulation (only relevant if `kudu.schema-emulation.enabled=true`)
+## The standard prefix is `presto::`. Empty prefix is also supported.
+## For more details see connector documentation.
+#kudu.schema-emulation.prefix=
+
+###########################################
+### Advanced Kudu Java client configuration
+###########################################
+
+## Default timeout used for administrative operations (e.g. createTable, deleteTable, etc.)
+#kudu.client.default-admin-operation-timeout = 30s
+
+## Default timeout used for user operations
+#kudu.client.default-operation-timeout = 30s
+
+## Disable Kudu client's collection of statistics.
+#kudu.client.disable-statistics = false
+
+## Assign Kudu splits to replica host if worker and kudu share the same cluster
+#kudu.allow-local-scheduling = false
+
+
+
+
+

Kerberos support#

+

In order to connect to a kudu cluster that uses kerberos +authentication, you need to configure the following kudu properties:

+
kudu.authentication.type = KERBEROS
+
+## The kerberos client principal name
+kudu.authentication.client.principal = clientprincipalname
+
+## The path to the kerberos keytab file
+## The configured client principal must exist in this keytab file
+kudu.authentication.client.keytab = /path/to/keytab/file.keytab
+
+## The path to the krb5.conf kerberos config file
+kudu.authentication.config = /path/to/kerberos/krb5.conf
+
+## Optional and defaults to "kudu"
+## If kudu is running with a custom SPN this needs to be configured
+kudu.authentication.server.principal.primary = kudu
+
+
+
+
+

Querying data#

+

Apache Kudu does not support schemas, i.e. namespaces for tables. +The connector can optionally emulate schemas by table naming conventions.

+
+

Default behaviour (without schema emulation)#

+

The emulation of schemas is disabled by default. +In this case all Kudu tables are part of the default schema.

+

For example, a Kudu table named orders can be queried in Trino +with SELECT * FROM example.default.orders or simple with SELECT * FROM orders +if catalog and schema are set to kudu and default respectively.

+

Table names can contain any characters in Kudu. In this case, use double quotes. +E.g. To query a Kudu table named special.table! use SELECT * FROM example.default."special.table!".

+
+

Example#

+
    +
  • Create a users table in the default schema:

    +
    CREATE TABLE example.default.users (
    +  user_id int WITH (primary_key = true),
    +  first_name VARCHAR,
    +  last_name VARCHAR
    +) WITH (
    +  partition_by_hash_columns = ARRAY['user_id'],
    +  partition_by_hash_buckets = 2
    +);
    +
    +
    +

    On creating a Kudu table you must/can specify additional information about +the primary key, encoding, and compression of columns and hash or range +partitioning. For details see the Creating a table section.

    +
  • +
  • Describe the table:

    +
    DESCRIBE example.default.users;
    +
    +
    +
       Column   |  Type   |                      Extra                      | Comment
    +------------+---------+-------------------------------------------------+---------
    + user_id    | integer | primary_key, encoding=auto, compression=default |
    + first_name | varchar | nullable, encoding=auto, compression=default    |
    + last_name  | varchar | nullable, encoding=auto, compression=default    |
    +(3 rows)
    +
    +
    +
  • +
  • Insert some data:

    +
    INSERT INTO example.default.users VALUES (1, 'Donald', 'Duck'), (2, 'Mickey', 'Mouse');
    +
    +
    +
  • +
  • Select the inserted data:

    +
    SELECT * FROM example.default.users;
    +
    +
    +
  • +
+
+
+
+

Behavior with schema emulation#

+

If schema emulation has been enabled in the connector properties, i.e. +etc/catalog/example.properties, tables are mapped to schemas depending on +some conventions.

+
    +
  • With kudu.schema-emulation.enabled=true and kudu.schema-emulation.prefix=, +the mapping works like:

    + + + + + + + + + + + + + + + + + +

    Kudu table name

    Trino qualified name

    orders

    kudu.default.orders

    part1.part2

    kudu.part1.part2

    x.y.z

    kudu.x."y.z"

    +

    As schemas are not directly supported by Kudu, a special table named +$schemas is created for managing the schemas.

    +
  • +
  • With kudu.schema-emulation.enabled=true and kudu.schema-emulation.prefix=presto::, +the mapping works like:

    + + + + + + + + + + + + + + + + + + + + + + + +

    Kudu table name

    Trino qualified name

    orders

    kudu.default.orders

    part1.part2

    kudu.default."part1.part2"

    x.y.z

    kudu.default."x.y.z"

    presto::part1.part2

    kudu.part1.part2

    presto:x.y.z

    kudu.x."y.z"

    +

    As schemas are not directly supported by Kudu, a special table named +presto::$schemas is created for managing the schemas.

    +
  • +
+
+
+
+

Type mapping#

+

Because Trino and Kudu each support types that the other does not, this +connector modifies some types when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction.

+
+

Kudu type to Trino type mapping#

+

The connector maps Kudu types to the corresponding Trino types following +this table:

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Kudu type to Trino type mapping#

Kudu type

Trino type

BOOL

BOOLEAN

INT8

TINYINT

INT16

SMALLINT

INT32

INTEGER

INT64

BIGINT

FLOAT

REAL

DOUBLE

DOUBLE

DECIMAL(p,s)

DECIMAL(p,s)

STRING

VARCHAR

BINARY

VARBINARY

UNIXTIME_MICROS

TIMESTAMP(3)

+

No other types are supported.

+
+
+

Trino type to Kudu type mapping#

+

The connector maps Trino types to the corresponding Kudu types following +this table:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Trino type to Kudu type mapping#

Trino type

Kudu type

Notes

BOOLEAN

BOOL

TINYINT

INT8

SMALLINT

INT16

INTEGER

INT32

BIGINT

INT64

REAL

FLOAT

DOUBLE

DOUBLE

DECIMAL(p,s)

DECIMAL(p,s)

Only supported for Kudu server >= 1.7.0

VARCHAR

STRING

The optional maximum length is lost

VARBINARY

BINARY

DATE

STRING

TIMESTAMP(3)

UNIXTIME_MICROS

µs resolution in Kudu column is reduced to ms resolution

+

No other types are supported.

+
+
+
+

SQL support#

+

The connector provides read and write access to data and metadata in +Kudu. In addition to the globally available and read operation +statements, the connector supports the following features:

+ +
+

Inserting into tables#

+

INSERT INTO ... values and INSERT INTO ... select behave like +UPSERT.

+
+
+

SQL DELETE#

+

If a WHERE clause is specified, the DELETE operation only works if the +predicate in the clause can be fully pushed down to the data source.

+
+
+

Creating schemas#

+

CREATE SCHEMA is only allowed if schema emulation is enabled. See the +Behavior with schema emulation section.

+
+
+

Dropping schemas#

+

DROP SCHEMA is only allowed if schema emulation is enabled. See the +Behavior with schema emulation section.

+
+
+

Creating a table#

+

On creating a Kudu table, you need to provide the columns and their types, of +course, but Kudu needs information about partitioning and optionally +for column encoding and compression.

+

Simple Example:

+
CREATE TABLE user_events (
+  user_id INTEGER WITH (primary_key = true),
+  event_name VARCHAR WITH (primary_key = true),
+  message VARCHAR,
+  details VARCHAR WITH (nullable = true, encoding = 'plain')
+) WITH (
+  partition_by_hash_columns = ARRAY['user_id'],
+  partition_by_hash_buckets = 5,
+  number_of_replicas = 3
+);
+
+
+

The primary key consists of user_id and event_name. The table is partitioned into +five partitions by hash values of the column user_id, and the number_of_replicas is +explicitly set to 3.

+

The primary key columns must always be the first columns of the column list. +All columns used in partitions must be part of the primary key.

+

The table property number_of_replicas is optional. It defines the +number of tablet replicas, and must be an odd number. If it is not specified, +the default replication factor from the Kudu master configuration is used.

+

Kudu supports two different kinds of partitioning: hash and range partitioning. +Hash partitioning distributes rows by hash value into one of many buckets. +Range partitions distributes rows using a totally-ordered range partition key. +The concrete range partitions must be created explicitly. +Kudu also supports multi-level partitioning. A table must have at least one +partitioning, either hash or range. It can have at most one range partitioning, +but multiple hash partitioning ‘levels’.

+

For more details see Partitioning design.

+
+
+

Column properties#

+

Besides column name and type, you can specify some more properties of a column.

+ + + + + + + + + + + + + + + + + + + + + + + + + +

Column property name

Type

Description

primary_key

BOOLEAN

If true, the column belongs to primary key columns. The Kudu primary key enforces a uniqueness constraint. Inserting a second row with the same primary key results in updating the existing row (‘UPSERT’). See also Primary Key Design in the Kudu documentation.

nullable

BOOLEAN

If true, the value can be null. Primary key columns must not be nullable.

encoding

VARCHAR

The column encoding can help to save storage space and to improve query performance. Kudu uses an auto encoding depending on the column type if not specified. Valid values are: 'auto', 'plain', 'bitshuffle', 'runlength', 'prefix', 'dictionary', 'group_varint'. See also Column encoding in the Kudu documentation.

compression

VARCHAR

The encoded column values can be compressed. Kudu uses a default compression if not specified. Valid values are: 'default', 'no', 'lz4', 'snappy', 'zlib'. See also Column compression in the Kudu documentation.

+

Example:

+
CREATE TABLE example_table (
+  name VARCHAR WITH (primary_key = true, encoding = 'dictionary', compression = 'snappy'),
+  index BIGINT WITH (nullable = true, encoding = 'runlength', compression = 'lz4'),
+  comment VARCHAR WITH (nullable = true, encoding = 'plain', compression = 'default'),
+   ...
+) WITH (...);
+
+
+
+
+

Changing tables#

+

Adding a column to an existing table uses the SQL statement ALTER TABLE ... ADD COLUMN .... +You can specify the same column properties as on creating a table.

+

Example:

+
ALTER TABLE example_table ADD COLUMN extraInfo VARCHAR WITH (nullable = true, encoding = 'plain')
+
+
+

See also Column properties.

+

ALTER TABLE ... RENAME COLUMN is only allowed if not part of a primary key.

+

ALTER TABLE ... DROP COLUMN is only allowed if not part of a primary key.

+
+
+
+

Procedures#

+ +
+

Partitioning design#

+

A table must have at least one partitioning (either hash or range). +It can have at most one range partitioning, but multiple hash partitioning ‘levels’. +For more details see Apache Kudu documentation: Partitioning.

+

If you create a Kudu table in Trino, the partitioning design is given by +several table properties.

+
+

Hash partitioning#

+

You can provide the first hash partition group with two table properties:

+

The partition_by_hash_columns defines the column(s) belonging to the +partition group and partition_by_hash_buckets the number of partitions to +split the hash values range into. All partition columns must be part of the +primary key.

+

Example:

+
CREATE TABLE example_table (
+  col1 VARCHAR WITH (primary_key=true),
+  col2 VARCHAR WITH (primary_key=true),
+  ...
+) WITH (
+  partition_by_hash_columns = ARRAY['col1', 'col2'],
+  partition_by_hash_buckets = 4
+)
+
+
+

This defines a hash partitioning with the columns col1 and col2 +distributed over 4 partitions.

+

To define two separate hash partition groups, also use the second pair +of table properties named partition_by_second_hash_columns and +partition_by_second_hash_buckets.

+

Example:

+
CREATE TABLE example_table (
+  col1 VARCHAR WITH (primary_key=true),
+  col2 VARCHAR WITH (primary_key=true),
+  ...
+) WITH (
+  partition_by_hash_columns = ARRAY['col1'],
+  partition_by_hash_buckets = 2,
+  partition_by_second_hash_columns = ARRAY['col2'],
+  partition_by_second_hash_buckets = 3
+)
+
+
+

This defines a two-level hash partitioning, with the first hash partition group +over the column col1 distributed over 2 buckets, and the second +hash partition group over the column col2 distributed over 3 buckets. +As a result you have table with 2 x 3 = 6 partitions.

+
+
+

Range partitioning#

+

You can provide at most one range partitioning in Apache Kudu. The columns +are defined with the table property partition_by_range_columns. +The ranges themselves are given either in the +table property range_partitions on creating the table. +Or alternatively, the procedures kudu.system.add_range_partition and +kudu.system.drop_range_partition can be used to manage range +partitions for existing tables. For both ways see below for more +details.

+

Example:

+
CREATE TABLE events (
+  rack VARCHAR WITH (primary_key=true),
+  machine VARCHAR WITH (primary_key=true),
+  event_time TIMESTAMP WITH (primary_key=true),
+  ...
+) WITH (
+  partition_by_hash_columns = ARRAY['rack'],
+  partition_by_hash_buckets = 2,
+  partition_by_second_hash_columns = ARRAY['machine'],
+  partition_by_second_hash_buckets = 3,
+  partition_by_range_columns = ARRAY['event_time'],
+  range_partitions = '[{"lower": null, "upper": "2018-01-01T00:00:00"},
+                       {"lower": "2018-01-01T00:00:00", "upper": null}]'
+)
+
+
+

This defines a tree-level partitioning with two hash partition groups and +one range partitioning on the event_time column. +Two range partitions are created with a split at “2018-01-01T00:00:00”.

+
+
+
+

Table property range_partitions#

+

With the range_partitions table property you specify the concrete +range partitions to be created. The range partition definition itself +must be given in the table property partition_design separately.

+

Example:

+
CREATE TABLE events (
+  serialno VARCHAR WITH (primary_key = true),
+  event_time TIMESTAMP WITH (primary_key = true),
+  message VARCHAR
+) WITH (
+  partition_by_hash_columns = ARRAY['serialno'],
+  partition_by_hash_buckets = 4,
+  partition_by_range_columns = ARRAY['event_time'],
+  range_partitions = '[{"lower": null, "upper": "2017-01-01T00:00:00"},
+                       {"lower": "2017-01-01T00:00:00", "upper": "2017-07-01T00:00:00"},
+                       {"lower": "2017-07-01T00:00:00", "upper": "2018-01-01T00:00:00"}]'
+);
+
+
+

This creates a table with a hash partition on column serialno with 4 +buckets and range partitioning on column event_time. Additionally, +three range partitions are created:

+
    +
  1. for all event_times before the year 2017, lower bound = null means it is unbound

  2. +
  3. for the first half of the year 2017

  4. +
  5. for the second half the year 2017

  6. +
+

This means any attempt to add rows with event_time of year 2018 or greater fails, as no partition is defined. +The next section shows how to define a new range partition for an existing table.

+
+

Managing range partitions#

+

For existing tables, there are procedures to add and drop a range +partition.

+
    +
  • adding a range partition

    +
    CALL example.system.add_range_partition(<schema>, <table>, <range_partition_as_json_string>)
    +
    +
    +
  • +
  • dropping a range partition

    +
    CALL example.system.drop_range_partition(<schema>, <table>, <range_partition_as_json_string>)
    +
    +
    +
      +
    • <schema>: schema of the table

    • +
    • <table>: table names

    • +
    • <range_partition_as_json_string>: lower and upper bound of the +range partition as JSON string in the form +'{"lower": <value>, "upper": <value>}', or if the range partition +has multiple columns: +'{"lower": [<value_col1>,...], "upper": [<value_col1>,...]}'. The +concrete literal for lower and upper bound values are depending on +the column types.

      +

      Examples:

      + + + + + + + + + + + + + + + + + + + + + + + + + + +

      Trino data Type

      JSON string example

      BIGINT

      ‘{“lower”: 0, “upper”: 1000000}’

      SMALLINT

      ‘{“lower”: 10, “upper”: null}’

      VARCHAR

      ‘{“lower”: “A”, “upper”: “M”}’

      TIMESTAMP

      ‘{“lower”: “2018-02-01T00:00:00.000”, “upper”: “2018-02-01T12:00:00.000”}’

      BOOLEAN

      ‘{“lower”: false, “upper”: true}’

      VARBINARY

      values encoded as base64 strings

      +

      To specified an unbounded bound, use the value null.

      +
    • +
    +
  • +
+

Example:

+
CALL example.system.add_range_partition('example_schema', 'events', '{"lower": "2018-01-01", "upper": "2018-06-01"}')
+
+
+

This adds a range partition for a table events in the schema +example_schema with the lower bound 2018-01-01, more exactly +2018-01-01T00:00:00.000, and the upper bound 2018-07-01.

+

Use the SQL statement SHOW CREATE TABLE to query the existing +range partitions (they are shown in the table property +range_partitions).

+
+
+
+
+

Limitations#

+
    +
  • Only lower case table and column names in Kudu are supported.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/localfile.html b/430/connector/localfile.html new file mode 100644 index 000000000..92b2d53cd --- /dev/null +++ b/430/connector/localfile.html @@ -0,0 +1,725 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Local file connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Local file connector#

+

The local file connector allows querying the HTTP request log files stored on +the local file system of each worker.

+
+

Configuration#

+

To configure the local file connector, create a catalog properties file under +etc/catalog named, for example, example.properties with the following +contents:

+
connector.name=localfile
+
+
+
+
+

Configuration properties#

+ + + + + + + + + + + + + + +

Property name

Description

trino-logs.http-request-log.location

Directory or file where HTTP request logs are written

trino-logs.http-request-log.pattern

If the log location is a directory, this glob is used to match file names in the directory

+
+
+

Local file connector schemas and tables#

+

The local file connector provides a single schema named logs. +You can see all the available tables by running SHOW TABLES:

+
SHOW TABLES FROM example.logs;
+
+
+
+

http_request_log#

+

This table contains the HTTP request logs from each node on the cluster.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/mariadb.html b/430/connector/mariadb.html new file mode 100644 index 000000000..c029dd35e --- /dev/null +++ b/430/connector/mariadb.html @@ -0,0 +1,1463 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + MariaDB connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

MariaDB connector#

+

The MariaDB connector allows querying and creating tables in an external MariaDB +database.

+
+

Requirements#

+

To connect to MariaDB, you need:

+
    +
  • MariaDB version 10.2 or higher.

  • +
  • Network access from the Trino coordinator and workers to MariaDB. Port +3306 is the default port.

  • +
+
+
+

Configuration#

+

To configure the MariaDB connector, create a catalog properties file in +etc/catalog named, for example, example.properties, to mount the MariaDB +connector as the example catalog. Create the file with the following +contents, replacing the connection properties as appropriate for your setup:

+
connector.name=mariadb
+connection-url=jdbc:mariadb://example.net:3306
+connection-user=root
+connection-password=secret
+
+
+

The connection-user and connection-password are typically required and +determine the user credentials for the connection, often a service user. You can +use secrets to avoid actual values in the catalog +properties files.

+
+

Data source authentication#

+

The connector can provide credentials for the data source connection +in multiple ways:

+
    +
  • inline, in the connector configuration file

  • +
  • in a separate properties file

  • +
  • in a key store file

  • +
  • as extra credentials set when connecting to Trino

  • +
+

You can use secrets to avoid storing sensitive +values in the catalog properties files.

+

The following table describes configuration properties +for connection credentials:

+ ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

credential-provider.type

Type of the credential provider. Must be one of INLINE, FILE, or +KEYSTORE; defaults to INLINE.

connection-user

Connection user name.

connection-password

Connection password.

user-credential-name

Name of the extra credentials property, whose value to use as the user +name. See extraCredentials in Parameter +reference.

password-credential-name

Name of the extra credentials property, whose value to use as the +password.

connection-credential-file

Location of the properties file where credentials are present. It must +contain the connection-user and connection-password properties.

keystore-file-path

The location of the Java Keystore file, from which to read credentials.

keystore-type

File format of the keystore file, for example JKS or PEM.

keystore-password

Password for the key store.

keystore-user-credential-name

Name of the key store entity to use as the user name.

keystore-user-credential-password

Password for the user name key store entity.

keystore-password-credential-name

Name of the key store entity to use as the password.

keystore-password-credential-password

Password for the password key store entity.

+
+
+

General configuration properties#

+

The following table describes general catalog configuration properties for the +connector:

+ +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

Default value

case-insensitive-name-matching

Support case insensitive schema and table names.

false

case-insensitive-name-matching.cache-ttl

This value should be a duration.

1m

case-insensitive-name-matching.config-file

Path to a name mapping configuration file in JSON format that allows +Trino to disambiguate between schemas and tables with similar names in +different cases.

null

case-insensitive-name-matching.config-file.refresh-period

Frequency with which Trino checks the name matching configuration file +for changes. This value should be a duration.

(refresh disabled)

metadata.cache-ttl

The duration for which metadata, including +table and column statistics, is cached.

0s (caching disabled)

metadata.cache-missing

Cache the fact that metadata, including table and column statistics, is +not available

false

metadata.cache-maximum-size

Maximum number of objects stored in the metadata cache

10000

write.batch-size

Maximum number of statements in a batched execution. +Do not change this setting from the default. Non-default values may +negatively impact performance.

1000

dynamic-filtering.enabled

Push down dynamic filters into JDBC queries

true

dynamic-filtering.wait-timeout

Maximum duration for which Trino will wait for dynamic +filters to be collected from the build side of joins before starting a +JDBC query. Using a large timeout can potentially result in more detailed +dynamic filters. However, it can also increase latency for some queries.

20s

+
+
+

Domain compaction threshold#

+

Pushing down a large list of predicates to the data source can compromise +performance. Trino compacts large predicates into a simpler range predicate +by default to ensure a balance between performance and predicate pushdown. +If necessary, the threshold for this compaction can be increased to improve +performance when the data source is capable of taking advantage of large +predicates. Increasing this threshold may improve pushdown of large +dynamic filters. +The domain-compaction-threshold catalog configuration property or the +domain_compaction_threshold catalog session property can be used to adjust the default value of +32 for this threshold.

+
+
+

Case insensitive matching#

+

When case-insensitive-name-matching is set to true, Trino +is able to query non-lowercase schemas and tables by maintaining a mapping of +the lowercase name to the actual name in the remote system. However, if two +schemas and/or tables have names that differ only in case (such as “customers” +and “Customers”) then Trino fails to query them due to ambiguity.

+

In these cases, use the case-insensitive-name-matching.config-file catalog +configuration property to specify a configuration file that maps these remote +schemas/tables to their respective Trino schemas/tables:

+
{
+  "schemas": [
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "mapping": "case_insensitive_1"
+    },
+    {
+      "remoteSchema": "cASEsENSITIVEnAME",
+      "mapping": "case_insensitive_2"
+    }],
+  "tables": [
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "remoteTable": "tablex",
+      "mapping": "table_1"
+    },
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "remoteTable": "TABLEX",
+      "mapping": "table_2"
+    }]
+}
+
+
+

Queries against one of the tables or schemes defined in the mapping +attributes are run against the corresponding remote entity. For example, a query +against tables in the case_insensitive_1 schema is forwarded to the +CaseSensitiveName schema and a query against case_insensitive_2 is forwarded +to the cASEsENSITIVEnAME schema.

+

At the table mapping level, a query on case_insensitive_1.table_1 as +configured above is forwarded to CaseSensitiveName.tablex, and a query on +case_insensitive_1.table_2 is forwarded to CaseSensitiveName.TABLEX.

+

By default, when a change is made to the mapping configuration file, Trino must +be restarted to load the changes. Optionally, you can set the +case-insensitive-name-mapping.refresh-period to have Trino refresh the +properties without requiring a restart:

+
case-insensitive-name-mapping.refresh-period=30s
+
+
+
+
+

Non-transactional INSERT#

+

The connector supports adding rows using INSERT statements. +By default, data insertion is performed by writing data to a temporary table. +You can skip this step to improve performance and write directly to the target +table. Set the insert.non-transactional-insert.enabled catalog property +or the corresponding non_transactional_insert catalog session property to +true.

+

Note that with this property enabled, data can be corrupted in rare cases where +exceptions occur during the insert operation. With transactions disabled, no +rollback can be performed.

+
+
+
+

Querying MariaDB#

+

The MariaDB connector provides a schema for every MariaDB database. +You can see the available MariaDB databases by running SHOW SCHEMAS:

+
SHOW SCHEMAS FROM example;
+
+
+

If you have a MariaDB database named web, you can view the tables +in this database by running SHOW TABLES:

+
SHOW TABLES FROM example.web;
+
+
+

You can see a list of the columns in the clicks table in the web +database using either of the following:

+
DESCRIBE example.web.clicks;
+SHOW COLUMNS FROM example.web.clicks;
+
+
+

Finally, you can access the clicks table in the web database:

+
SELECT * FROM example.web.clicks;
+
+
+

If you used a different name for your catalog properties file, use +that catalog name instead of example in the above examples.

+
+
+

Type mapping#

+

Because Trino and MariaDB each support types that the other does not, this +connector modifies some types when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction.

+
+

MariaDB type to Trino type mapping#

+

The connector maps MariaDB types to the corresponding Trino types according +to the following table:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
MariaDB type to Trino type mapping#

MariaDB type

Trino type

Notes

BOOLEAN

TINYINT

BOOL and BOOLEAN are aliases of TINYINT(1)

TINYINT

TINYINT

TINYINT UNSIGNED

SMALLINT

SMALLINT

SMALLINT

SMALLINT UNSIGNED

INTEGER

INT

INTEGER

INT UNSIGNED

BIGINT

BIGINT

BIGINT

BIGINT UNSIGNED

DECIMAL(20, 0)

FLOAT

REAL

DOUBLE

DOUBLE

DECIMAL(p,s)

DECIMAL(p,s)

CHAR(n)

CHAR(n)

TINYTEXT

VARCHAR(255)

TEXT

VARCHAR(65535)

MEDIUMTEXT

VARCHAR(16777215)

LONGTEXT

VARCHAR

VARCHAR(n)

VARCHAR(n)

TINYBLOB

VARBINARY

BLOB

VARBINARY

MEDIUMBLOB

VARBINARY

LONGBLOB

VARBINARY

VARBINARY(n)

VARBINARY

DATE

DATE

TIME(n)

TIME(n)

TIMESTAMP(n)

TIMESTAMP(n)

MariaDB stores the current timestamp by default. Enable +explicit_defaults_for_timestamp +to avoid implicit default values and use NULL as the default value.

+

No other types are supported.

+
+
+

Trino type mapping to MariaDB type mapping#

+

The connector maps Trino types to the corresponding MariaDB types according +to the following table:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Trino type mapping to MariaDB type mapping#

Trino type

MariaDB type

Notes

BOOLEAN

BOOLEAN

TINYINT

TINYINT

SMALLINT

SMALLINT

INTEGER

INT

BIGINT

BIGINT

REAL

FLOAT

DOUBLE

DOUBLE

DECIMAL(p,s)

DECIMAL(p,s)

CHAR(n)

CHAR(n)

VARCHAR(255)

TINYTEXT

Maps on VARCHAR of length 255 or less.

VARCHAR(65535)

TEXT

Maps on VARCHAR of length between 256 and 65535, inclusive.

VARCHAR(16777215)

MEDIUMTEXT

Maps on VARCHAR of length between 65536 and 16777215, inclusive.

VARCHAR

LONGTEXT

VARCHAR of length greater than 16777215 and unbounded VARCHAR map +to LONGTEXT.

VARBINARY

MEDIUMBLOB

DATE

DATE

TIME(n)

TIME(n)

TIMESTAMP(n)

TIMESTAMP(n)

MariaDB stores the current timestamp by default. Enable +explicit_defaults_for_timestamp +to avoid implicit default values and use NULL as the default value.

+

No other types are supported.

+

Complete list of MariaDB data types.

+
+
+

Type mapping configuration properties#

+

The following properties can be used to configure how data types from the +connected data source are mapped to Trino data types and how the metadata is +cached in Trino.

+ +++++ + + + + + + + + + + + + + + + + +

Property name

Description

Default value

unsupported-type-handling

Configure how unsupported column data types are handled:

+
    +
  • IGNORE, column is not accessible.

  • +
  • CONVERT_TO_VARCHAR, column is converted to unbounded VARCHAR.

  • +
+

The respective catalog session property is unsupported_type_handling.

+

IGNORE

jdbc-types-mapped-to-varchar

Allow forced mapping of comma separated lists of data types to convert to +unbounded VARCHAR

+
+
+
+

SQL support#

+

The connector provides read access and write access to data and metadata in +a MariaDB database. In addition to the globally available and read operation +statements, the connector supports the following features:

+ +
+

UPDATE#

+

Only UPDATE statements with constant assignments and predicates are +supported. For example, the following statement is supported because the values +assigned are constants:

+
UPDATE table SET col1 = 1 WHERE col3 = 1
+
+
+

Arithmetic expressions, function calls, and other non-constant UPDATE +statements are not supported. For example, the following statement is not +supported because arithmetic expressions cannot be used with the SET +command:

+
UPDATE table SET col1 = col2 + 2 WHERE col3 = 1
+
+
+

The =, !=, >, <, >=, <=, IN, NOT IN operators are supported in +predicates. The following statement is not supported because the AND operator +cannot be used in predicates:

+
UPDATE table SET col1 = 1 WHERE col3 = 1 AND col2 = 3
+
+
+

All column values of a table row cannot be updated simultaneously. For a three +column table, the following statement is not supported:

+
UPDATE table SET col1 = 1, col2 = 2, col3 = 3 WHERE col3 = 1
+
+
+
+
+

SQL DELETE#

+

If a WHERE clause is specified, the DELETE operation only works if the +predicate in the clause can be fully pushed down to the data source.

+
+
+
+

Table functions#

+

The connector provides specific table functions to +access MariaDB.

+
+

query(varchar) -> table#

+

The query function allows you to query the underlying database directly. It +requires syntax native to MariaDB, because the full query is pushed down and +processed in MariaDB. This can be useful for accessing native features which are +not available in Trino or for improving query performance in situations where +running a query natively may be faster.

+

The native query passed to the underlying data source is required to return a +table as a result set. Only the data source performs validation or security +checks for these queries using its own configuration. Trino does not perform +these tasks. Only use passthrough queries to read data.

+

As an example, query the example catalog and select the age of employees by +using TIMESTAMPDIFF and CURDATE:

+
SELECT
+  age
+FROM
+  TABLE(
+    example.system.query(
+      query => 'SELECT
+        TIMESTAMPDIFF(
+          YEAR,
+          date_of_birth,
+          CURDATE()
+        ) AS age
+      FROM
+        tiny.employees'
+    )
+  );
+
+
+
+

Note

+

The query engine does not preserve the order of the results of this +function. If the passed query contains an ORDER BY clause, the +function result may not be ordered as expected.

+
+
+
+
+

Performance#

+

The connector includes a number of performance improvements, detailed in the +following sections.

+
+

Table statistics#

+

The MariaDB connector can use table and column +statistics for cost based +optimizations to improve query processing +performance based on the actual data in the data source.

+

The statistics are collected by MariaDB and retrieved by the connector.

+

To collect statistics for a table, execute the following statement in +MariaDB.

+
ANALYZE TABLE table_name;
+
+
+

Refer to MariaDB documentation for +additional information.

+
+
+

Pushdown#

+

The connector supports pushdown for a number of operations:

+ +

Aggregate pushdown for the following functions:

+ +
+

Note

+

The connector performs pushdown where performance may be improved, but in +order to preserve correctness an operation may not be pushed down. When +pushdown of an operation may result in better performance but risks +correctness, the connector prioritizes correctness.

+
+
+

Predicate pushdown support#

+

The connector does not support pushdown of any predicates on columns with +textual types like CHAR or VARCHAR. +This ensures correctness of results since the data source may compare strings +case-insensitively.

+

In the following example, the predicate is not pushed down for either query +since name is a column of type VARCHAR:

+
SELECT * FROM nation WHERE name > 'CANADA';
+SELECT * FROM nation WHERE name = 'CANADA';
+
+
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/memory.html b/430/connector/memory.html new file mode 100644 index 000000000..02eb5d8d9 --- /dev/null +++ b/430/connector/memory.html @@ -0,0 +1,793 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Memory connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+ +
+ +
+
+ +
+

Memory connector#

+

The Memory connector stores all data and metadata in RAM on workers +and both are discarded when Trino restarts.

+
+

Configuration#

+

To configure the Memory connector, create a catalog properties file +etc/catalog/example.properties with the following contents:

+
connector.name=memory
+memory.max-data-per-node=128MB
+
+
+

memory.max-data-per-node defines memory limit for pages stored in this +connector per each node (default value is 128MB).

+
+
+

Examples#

+

Create a table using the Memory connector:

+
CREATE TABLE example.default.nation AS
+SELECT * from tpch.tiny.nation;
+
+
+

Insert data into a table in the Memory connector:

+
INSERT INTO example.default.nation
+SELECT * FROM tpch.tiny.nation;
+
+
+

Select from the Memory connector:

+
SELECT * FROM example.default.nation;
+
+
+

Drop table:

+
DROP TABLE example.default.nation;
+
+
+
+
+

Type mapping#

+

Trino supports all data types used within the Memory schemas so no mapping is +required.

+
+
+

SQL support#

+

The connector provides read and write access to temporary data and metadata +stored in memory. In addition to the globally available and read operation +statements, the connector supports the following features:

+ +
+

DROP TABLE#

+

Upon execution of a DROP TABLE operation, memory is not released +immediately. It is instead released after the next write operation to the +catalog.

+
+
+
+

Dynamic filtering#

+

The Memory connector supports the dynamic filtering optimization. +Dynamic filters are pushed into local table scan on worker nodes for broadcast joins.

+
+

Delayed execution for dynamic filters#

+

For the Memory connector, a table scan is delayed until the collection of dynamic filters. +This can be disabled by using the configuration property memory.enable-lazy-dynamic-filtering +in the catalog file.

+
+
+
+

Limitations#

+
    +
  • When one worker fails/restarts, all data that was stored in its +memory is lost. To prevent silent data loss the +connector throws an error on any read access to such +corrupted table.

  • +
  • When a query fails for any reason during writing to memory table, +the table enters an undefined state. The table should be dropped +and recreated manually. Reading attempts from the table may fail, +or may return partial data.

  • +
  • When the coordinator fails/restarts, all metadata about tables is +lost. The tables remain on the workers, but become inaccessible.

  • +
  • This connector does not work properly with multiple +coordinators, since each coordinator has different +metadata.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/memsql.html b/430/connector/memsql.html new file mode 100644 index 000000000..b360f30a7 --- /dev/null +++ b/430/connector/memsql.html @@ -0,0 +1,15 @@ + + + + + + + diff --git a/430/connector/metastores.html b/430/connector/metastores.html new file mode 100644 index 000000000..e604bec4f --- /dev/null +++ b/430/connector/metastores.html @@ -0,0 +1,1306 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Metastores — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Metastores#

+

Object storage access is mediated through a metastore. Metastores provide +information on directory structure, file format, and metadata about the stored +data. Object storage connectors support the use of one or more metastores. A +supported metastore is required to use any object storage connector.

+

Additional configuration is required in order to access tables with Athena +partition projection metadata or implement first class support for Avro tables. +These requirements are discussed later in this topic.

+
+

General metastore configuration properties#

+

The following table describes general metastore configuration properties, most +of which are used with either metastore.

+

At a minimum, each Delta Lake, Hive or Hudi object storage catalog file must set +the hive.metastore configuration property to define the type of metastore to +use. Iceberg catalogs instead use the iceberg.catalog.type configuration +property to define the type of metastore to use.

+

Additional configuration properties specific to the Thrift and Glue Metastores +are also available. They are discussed later in this topic.

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
General metastore configuration properties#

Property Name

Description

Default

hive.metastore

The type of Hive metastore to use. Trino currently supports the default +Hive Thrift metastore (thrift), and the AWS Glue Catalog (glue) +as metadata sources. You must use this for all object storage catalogs +except Iceberg.

thrift

iceberg.catalog.type

The Iceberg table format manages most metadata in metadata files in the +object storage itself. A small amount of metadata, however, still +requires the use of a metastore. In the Iceberg ecosystem, these smaller +metastores are called Iceberg metadata catalogs, or just catalogs. The +examples in each subsection depict the contents of a Trino catalog file +that uses the the Iceberg connector to configures different Iceberg +metadata catalogs.

+

You must set this property in all Iceberg catalog property files. +Valid values are HIVE_METASTORE, GLUE, JDBC, REST, and +NESSIE.

+

hive.metastore-cache.cache-partitions

Enable caching for partition metadata. You can disable caching to avoid +inconsistent behavior that results from it.

true

hive.metastore-cache-ttl

Duration of how long cached metastore data is considered valid.

0s

hive.metastore-stats-cache-ttl

Duration of how long cached metastore statistics are considered valid. +If hive.metastore-cache-ttl is larger then it takes precedence +over hive.metastore-stats-cache-ttl.

5m

hive.metastore-cache-maximum-size

Maximum number of metastore data objects in the Hive metastore cache.

10000

hive.metastore-refresh-interval

Asynchronously refresh cached metastore data after access if it is older +than this but is not yet expired, allowing subsequent accesses to see +fresh data.

hive.metastore-refresh-max-threads

Maximum threads used to refresh cached metastore data.

10

hive.metastore-timeout

Timeout for Hive metastore requests.

10s

hive.hide-delta-lake-tables

Controls whether to hide Delta Lake tables in table listings. Currently +applies only when using the AWS Glue metastore.

false

+
+
+

Thrift metastore configuration properties#

+

In order to use a Hive Thrift metastore, you must configure the metastore with +hive.metastore=thrift and provide further details with the following +properties:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Thrift metastore configuration properties#

Property name

Description

Default

hive.metastore.uri

The URIs of the Hive metastore to connect to using the Thrift protocol. +If a comma-separated list of URIs is provided, the first URI is used by +default, and the rest of the URIs are fallback metastores. This property +is required. Example: thrift://192.0.2.3:9083 or +thrift://192.0.2.3:9083,thrift://192.0.2.4:9083

hive.metastore.username

The username Trino uses to access the Hive metastore.

hive.metastore.authentication.type

Hive metastore authentication type. Possible values are NONE or +KERBEROS.

NONE

hive.metastore.thrift.impersonation.enabled

Enable Hive metastore end user impersonation.

hive.metastore.thrift.use-spark-table-statistics-fallback

Enable usage of table statistics generated by Apache Spark when Hive +table statistics are not available.

true

hive.metastore.thrift.delegation-token.cache-ttl

Time to live delegation token cache for metastore.

1h

hive.metastore.thrift.delegation-token.cache-maximum-size

Delegation token cache maximum size.

1000

hive.metastore.thrift.client.ssl.enabled

Use SSL when connecting to metastore.

false

hive.metastore.thrift.client.ssl.key

Path to private key and client certification (key store).

hive.metastore.thrift.client.ssl.key-password

Password for the private key.

hive.metastore.thrift.client.ssl.trust-certificate

Path to the server certificate chain (trust store). Required when SSL is +enabled.

hive.metastore.thrift.client.ssl.trust-certificate-password

Password for the trust store.

hive.metastore.thrift.batch-fetch.enabled

Enable fetching tables and views from all schemas in a single request.

true

hive.metastore.service.principal

The Kerberos principal of the Hive metastore service.

hive.metastore.client.principal

The Kerberos principal that Trino uses when connecting to the Hive +metastore service.

hive.metastore.client.keytab

Hive metastore client keytab location.

hive.metastore.thrift.delete-files-on-drop

Actively delete the files for managed tables when performing drop table +or partition operations, for cases when the metastore does not delete the +files.

false

hive.metastore.thrift.assume-canonical-partition-keys

Allow the metastore to assume that the values of partition columns can be +converted to string values. This can lead to performance improvements in +queries which apply filters on the partition columns. Partition keys with +a TIMESTAMP type do not get canonicalized.

false

hive.metastore.thrift.client.socks-proxy

SOCKS proxy to use for the Thrift Hive metastore.

hive.metastore.thrift.client.max-retries

Maximum number of retry attempts for metastore requests.

9

hive.metastore.thrift.client.backoff-scale-factor

Scale factor for metastore request retry delay.

2.0

hive.metastore.thrift.client.max-retry-time

Total allowed time limit for a metastore request to be retried.

30s

hive.metastore.thrift.client.min-backoff-delay

Minimum delay between metastore request retries.

1s

hive.metastore.thrift.client.max-backoff-delay

Maximum delay between metastore request retries.

1s

hive.metastore.thrift.txn-lock-max-wait

Maximum time to wait to acquire hive transaction lock.

10m

+
+
+

AWS Glue catalog configuration properties#

+

In order to use an AWS Glue catalog, you must configure your catalog file as +follows:

+

hive.metastore=glue and provide further details with the following +properties:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
AWS Glue catalog configuration properties#

Property Name

Description

Default

hive.metastore.glue.region

AWS region of the Glue Catalog. This is required when not running in +EC2, or when the catalog is in a different region. Example: +us-east-1

hive.metastore.glue.endpoint-url

Glue API endpoint URL (optional). Example: +https://glue.us-east-1.amazonaws.com

hive.metastore.glue.sts.region

AWS region of the STS service to authenticate with. This is required +when running in a GovCloud region. Example: us-gov-east-1

hive.metastore.glue.proxy-api-id

The ID of the Glue Proxy API, when accessing Glue via an VPC endpoint in +API Gateway.

hive.metastore.glue.sts.endpoint

STS endpoint URL to use when authenticating to Glue (optional). Example: +https://sts.us-gov-east-1.amazonaws.com

hive.metastore.glue.pin-client-to-current-region

Pin Glue requests to the same region as the EC2 instance where Trino is +running.

false

hive.metastore.glue.max-connections

Max number of concurrent connections to Glue.

30

hive.metastore.glue.max-error-retries

Maximum number of error retries for the Glue client.

10

hive.metastore.glue.default-warehouse-dir

Default warehouse directory for schemas created without an explicit +location property.

hive.metastore.glue.aws-credentials-provider

Fully qualified name of the Java class to use for obtaining AWS +credentials. Can be used to supply a custom credentials provider.

hive.metastore.glue.aws-access-key

AWS access key to use to connect to the Glue Catalog. If specified along +with hive.metastore.glue.aws-secret-key, this parameter takes +precedence over hive.metastore.glue.iam-role.

hive.metastore.glue.aws-secret-key

AWS secret key to use to connect to the Glue Catalog. If specified along +with hive.metastore.glue.aws-access-key, this parameter takes +precedence over hive.metastore.glue.iam-role.

hive.metastore.glue.catalogid

The ID of the Glue Catalog in which the metadata database resides.

hive.metastore.glue.iam-role

ARN of an IAM role to assume when connecting to the Glue Catalog.

hive.metastore.glue.external-id

External ID for the IAM role trust policy when connecting to the Glue +Catalog.

hive.metastore.glue.partitions-segments

Number of segments for partitioned Glue tables.

5

hive.metastore.glue.get-partition-threads

Number of threads for parallel partition fetches from Glue.

20

hive.metastore.glue.read-statistics-threads

Number of threads for parallel statistic fetches from Glue.

5

hive.metastore.glue.write-statistics-threads

Number of threads for parallel statistic writes to Glue.

5

+
+

Iceberg-specific Glue catalog configuration properties#

+

When using the Glue catalog, the Iceberg connector supports the same +general Glue configuration properties as previously +described with the following additional property:

+ + +++++ + + + + + + + + + + + + +
Iceberg Glue catalog configuration property#

Property name

Description

Default

iceberg.glue.skip-archive

Skip archiving an old table version when creating a new version in a +commit. See AWS Glue Skip Archive.

false

+
+
+
+

Iceberg-specific metastores#

+

The Iceberg table format manages most metadata in metadata files in the object +storage itself. A small amount of metadata, however, still requires the use of a +metastore. In the Iceberg ecosystem, these smaller metastores are called Iceberg +metadata catalogs, or just catalogs.

+

You can use a general metastore such as an HMS or AWS Glue, or you can use the +Iceberg-specific REST, Nessie or JDBC metadata catalogs, as discussed in this +section.

+
+

REST catalog#

+

In order to use the Iceberg REST catalog, configure the catalog type +with iceberg.catalog.type=rest, and provide further details with the +following properties:

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + + + + +
Iceberg REST catalog configuration properties#

Property name

Description

iceberg.rest-catalog.uri

REST server API endpoint URI (required). +Example: http://iceberg-with-rest:8181

iceberg.rest-catalog.warehouse

Warehouse identifier/location for the catalog (optional). +Example: s3://my_bucket/warehouse_location

iceberg.rest-catalog.security

The type of security to use (default: NONE). OAUTH2 requires +either a token or credential. Example: OAUTH2

iceberg.rest-catalog.session

Session information included when communicating with the REST Catalog. +Options are NONE or USER (default: NONE).

iceberg.rest-catalog.oauth2.token

The bearer token used for interactions with the server. A +token or credential is required for OAUTH2 security. +Example: AbCdEf123456

iceberg.rest-catalog.oauth2.credential

The credential to exchange for a token in the OAuth2 client credentials +flow with the server. A token or credential is required for +OAUTH2 security. Example: AbCdEf123456

+

The following example shows a minimal catalog configuration using an Iceberg +REST metadata catalog:

+
connector.name=iceberg
+iceberg.catalog.type=rest
+iceberg.rest-catalog.uri=http://iceberg-with-rest:8181
+
+
+

The REST catalog does not support views or +materialized views.

+
+
+

JDBC catalog#

+

The Iceberg REST catalog is supported for the Iceberg connector. At a minimum, +iceberg.jdbc-catalog.driver-class, iceberg.jdbc-catalog.connection-url +and iceberg.jdbc-catalog.catalog-name must be configured. When using any +database besides PostgreSQL, a JDBC driver jar file must be placed in the plugin +directory.

+
+

Warning

+

The JDBC catalog may have compatibility issues if Iceberg introduces breaking +changes in the future. Consider the REST catalog as an alternative solution.

+
+

At a minimum, iceberg.jdbc-catalog.driver-class, +iceberg.jdbc-catalog.connection-url, and +iceberg.jdbc-catalog.catalog-name must be configured. When using any +database besides PostgreSQL, a JDBC driver jar file must be placed in the plugin +directory. The following example shows a minimal catalog configuration using an +Iceberg REST metadata catalog:

+
connector.name=iceberg
+iceberg.catalog.type=jdbc
+iceberg.jdbc-catalog.catalog-name=test
+iceberg.jdbc-catalog.driver-class=org.postgresql.Driver
+iceberg.jdbc-catalog.connection-url=jdbc:postgresql://example.net:5432/database
+iceberg.jdbc-catalog.connection-user=admin
+iceberg.jdbc-catalog.connection-password=test
+iceberg.jdbc-catalog.default-warehouse-dir=s3://bucket
+
+
+

The JDBC catalog does not support views or +materialized views.

+
+
+

Nessie catalog#

+

In order to use a Nessie catalog, configure the catalog type with +iceberg.catalog.type=nessie and provide further details with the following +properties:

+ + ++++ + + + + + + + + + + + + + + + + +
Nessie catalog configuration properties#

Property name

Description

iceberg.nessie-catalog.uri

Nessie API endpoint URI (required). +Example: https://localhost:19120/api/v1

iceberg.nessie-catalog.ref

The branch/tag to use for Nessie, defaults to main.

iceberg.nessie-catalog.default-warehouse-dir

Default warehouse directory for schemas created without an explicit +location property. Example: /tmp

+
connector.name=iceberg
+iceberg.catalog.type=nessie
+iceberg.nessie-catalog.uri=https://localhost:19120/api/v1
+iceberg.nessie-catalog.default-warehouse-dir=/tmp
+
+
+
+
+
+

Access tables with Athena partition projection metadata#

+

Partition projection +is a feature of AWS Athena often used to speed up query processing with highly +partitioned tables when using the Hive connector.

+

Trino supports partition projection table properties stored in the Hive +metastore or Glue catalog, and it reimplements this functionality. Currently, +there is a limitation in comparison to AWS Athena for date projection, as it +only supports intervals of DAYS, HOURS, MINUTES, and SECONDS.

+

If there are any compatibility issues blocking access to a requested table when +partition projection is enabled, set the +partition_projection_ignore table property to true for a table to bypass +any errors.

+

Refer to Table properties and Column properties for +configuration of partition projection.

+
+
+

Configure metastore for Avro#

+

For catalogs using the Hive connector, you must add the following property +definition to the Hive metastore configuration file hive-site.xml and +restart the metastore service to enable first-class support for Avro tables when +using Hive 3.x:

+
<property>
+     <!-- https://community.hortonworks.com/content/supportkb/247055/errorjavalangunsupportedoperationexception-storage.html -->
+     <name>metastore.storage.schema.reader.impl</name>
+     <value>org.apache.hadoop.hive.metastore.SerDeStorageSchemaReader</value>
+ </property>
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/mongodb.html b/430/connector/mongodb.html new file mode 100644 index 000000000..96686da6c --- /dev/null +++ b/430/connector/mongodb.html @@ -0,0 +1,1377 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + MongoDB connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

MongoDB connector#

+

The mongodb connector allows the use of MongoDB collections as tables in Trino.

+
+

Requirements#

+

To connect to MongoDB, you need:

+
    +
  • MongoDB 4.2 or higher.

  • +
  • Network access from the Trino coordinator and workers to MongoDB. +Port 27017 is the default port.

  • +
  • Write access to the schema information collection +in MongoDB.

  • +
+
+
+

Configuration#

+

To configure the MongoDB connector, create a catalog properties file +etc/catalog/example.properties with the following contents, +replacing the properties as appropriate:

+
connector.name=mongodb
+mongodb.connection-url=mongodb://user:pass@sample.host:27017/
+
+
+
+

Multiple MongoDB clusters#

+

You can have as many catalogs as you need, so if you have additional +MongoDB clusters, simply add another properties file to etc/catalog +with a different name, making sure it ends in .properties). For +example, if you name the property file sales.properties, Trino +will create a catalog named sales using the configured connector.

+
+
+
+

Configuration properties#

+

The following configuration properties are available:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

mongodb.connection-url

The connection url that the driver uses to connect to a MongoDB deployment

mongodb.schema-collection

A collection which contains schema information

mongodb.case-insensitive-name-matching

Match database and collection names case insensitively

mongodb.min-connections-per-host

The minimum size of the connection pool per host

mongodb.connections-per-host

The maximum size of the connection pool per host

mongodb.max-wait-time

The maximum wait time

mongodb.max-connection-idle-time

The maximum idle time of a pooled connection

mongodb.connection-timeout

The socket connect timeout

mongodb.socket-timeout

The socket timeout

mongodb.tls.enabled

Use TLS/SSL for connections to mongod/mongos

mongodb.tls.keystore-path

Path to the or JKS key store

mongodb.tls.truststore-path

Path to the or JKS trust store

mongodb.tls.keystore-password

Password for the key store

mongodb.tls.truststore-password

Password for the trust store

mongodb.read-preference

The read preference

mongodb.write-concern

The write concern

mongodb.required-replica-set

The required replica set name

mongodb.cursor-batch-size

The number of elements to return in a batch

+
+

mongodb.connection-url#

+

A connection string containing the protocol, credential, and host info for use +inconnection to your MongoDB deployment.

+

For example, the connection string may use the format +mongodb://<user>:<pass>@<host>:<port>/?<options> or +mongodb+srv://<user>:<pass>@<host>/?<options>, depending on the protocol +used. The user/pass credentials must be for a user with write access to the +schema information collection.

+

See the MongoDB Connection URI for more information.

+

This property is required; there is no default. A connection URL must be +provided to connect to a MongoDB deployment.

+
+
+

mongodb.schema-collection#

+

As MongoDB is a document database, there is no fixed schema information in the system. So a special collection in each MongoDB database should define the schema of all tables. Please refer the Table definition section for the details.

+

At startup, the connector tries to guess the data type of fields based on the type mapping.

+

The initial guess can be incorrect for your specific collection. In that case, you need to modify it manually. Please refer the Table definition section for the details.

+

Creating new tables using CREATE TABLE and CREATE TABLE AS SELECT automatically create an entry for you.

+

This property is optional; the default is _schema.

+
+
+

mongodb.case-insensitive-name-matching#

+

Match database and collection names case insensitively.

+

This property is optional; the default is false.

+
+
+

mongodb.min-connections-per-host#

+

The minimum number of connections per host for this MongoClient instance. Those connections are kept in a pool when idle, and the pool ensures over time that it contains at least this minimum number.

+

This property is optional; the default is 0.

+
+
+

mongodb.connections-per-host#

+

The maximum number of connections allowed per host for this MongoClient instance. Those connections are kept in a pool when idle. Once the pool is exhausted, any operation requiring a connection blocks waiting for an available connection.

+

This property is optional; the default is 100.

+
+
+

mongodb.max-wait-time#

+

The maximum wait time in milliseconds, that a thread may wait for a connection to become available. +A value of 0 means that it does not wait. A negative value means to wait indefinitely for a connection to become available.

+

This property is optional; the default is 120000.

+
+
+

mongodb.max-connection-idle-time#

+

The maximum idle time of a pooled connection in milliseconds. A value of 0 indicates no limit to the idle time. +A pooled connection that has exceeded its idle time will be closed and replaced when necessary by a new connection.

+

This property is optional; the default is 0.

+
+
+

mongodb.connection-timeout#

+

The connection timeout in milliseconds. A value of 0 means no timeout. It is used solely when establishing a new connection.

+

This property is optional; the default is 10000.

+
+
+

mongodb.socket-timeout#

+

The socket timeout in milliseconds. It is used for I/O socket read and write operations.

+

This property is optional; the default is 0 and means no timeout.

+
+
+

mongodb.tls.enabled#

+

This flag enables TLS connections to MongoDB servers.

+

This property is optional; the default is false.

+
+
+

mongodb.tls.keystore-path#

+

The path to the PEM or +JKS key store.

+

This property is optional.

+
+
+

mongodb.tls.truststore-path#

+

The path to PEM or +JKS trust store.

+

This property is optional.

+
+
+

mongodb.tls.keystore-password#

+

The key password for the key store specified by mongodb.tls.keystore-path.

+

This property is optional.

+
+
+

mongodb.tls.truststore-password#

+

The key password for the trust store specified by mongodb.tls.truststore-path.

+

This property is optional.

+
+
+

mongodb.read-preference#

+

The read preference to use for queries, map-reduce, aggregation, and count. +The available values are PRIMARY, PRIMARY_PREFERRED, SECONDARY, SECONDARY_PREFERRED and NEAREST.

+

This property is optional; the default is PRIMARY.

+
+
+

mongodb.write-concern#

+

The write concern to use. The available values are +ACKNOWLEDGED, JOURNALED, MAJORITY and UNACKNOWLEDGED.

+

This property is optional; the default is ACKNOWLEDGED.

+
+
+

mongodb.required-replica-set#

+

The required replica set name. With this option set, the MongoClient instance performs the following actions:

+
#. Connect in replica set mode, and discover all members of the set based on the given servers
+#. Make sure that the set name reported by all members matches the required set name.
+#. Refuse to service any requests, if authenticated user is not part of a replica set with the required name.
+
+
+

This property is optional; no default value.

+
+
+

mongodb.cursor-batch-size#

+

Limits the number of elements returned in one batch. A cursor typically fetches a batch of result objects and stores them locally. +If batchSize is 0, Driver’s default are used. +If batchSize is positive, it represents the size of each batch of objects retrieved. It can be adjusted to optimize performance and limit data transfer. +If batchSize is negative, it limits the number of objects returned, that fit within the max batch size limit (usually 4MB), and the cursor is closed. For example if batchSize is -10, then the server returns a maximum of 10 documents, and as many as can fit in 4MB, then closes the cursor.

+
+

Note

+

Do not use a batch size of 1.

+
+

This property is optional; the default is 0.

+
+
+
+

Table definition#

+

MongoDB maintains table definitions on the special collection where mongodb.schema-collection configuration value specifies.

+
+

Note

+

The plugin cannot detect that a collection has been deleted. You must +delete the entry by executing db.getCollection("_schema").remove( { table: deleted_table_name }) in the MongoDB Shell. You can also drop a collection in +Trino by running DROP TABLE table_name.

+
+

A schema collection consists of a MongoDB document for a table.

+
{
+    "table": ...,
+    "fields": [
+          { "name" : ...,
+            "type" : "varchar|bigint|boolean|double|date|array(bigint)|...",
+            "hidden" : false },
+            ...
+        ]
+    }
+}
+
+
+

The connector quotes the fields for a row type when auto-generating the schema; +however, the auto-generated schema must be corrected manually in the collection +to match the information in the tables.

+

Manually altered fields must be explicitly quoted, for example, row("UpperCase" varchar).

+ + + + + + + + + + + + + + + + + + + + +

Field

Required

Type

Description

table

required

string

Trino table name

fields

required

array

A list of field definitions. Each field definition creates a new column in the Trino table.

+

Each field definition:

+
{
+    "name": ...,
+    "type": ...,
+    "hidden": ...
+}
+
+
+ + + + + + + + + + + + + + + + + + + + + + + + + +

Field

Required

Type

Description

name

required

string

Name of the column in the Trino table.

type

required

string

Trino type of the column.

hidden

optional

boolean

Hides the column from DESCRIBE <table name> and SELECT *. Defaults to false.

+

There is no limit on field descriptions for either key or message.

+
+
+

ObjectId#

+

MongoDB collection has the special field _id. The connector tries to follow the same rules for this special field, so there will be hidden field _id.

+
CREATE TABLE IF NOT EXISTS orders (
+    orderkey BIGINT,
+    orderstatus VARCHAR,
+    totalprice DOUBLE,
+    orderdate DATE
+);
+
+INSERT INTO orders VALUES(1, 'bad', 50.0, current_date);
+INSERT INTO orders VALUES(2, 'good', 100.0, current_date);
+SELECT _id, * FROM orders;
+
+
+
                 _id                 | orderkey | orderstatus | totalprice | orderdate
+-------------------------------------+----------+-------------+------------+------------
+ 55 b1 51 63 38 64 d6 43 8c 61 a9 ce |        1 | bad         |       50.0 | 2015-07-23
+ 55 b1 51 67 38 64 d6 43 8c 61 a9 cf |        2 | good        |      100.0 | 2015-07-23
+(2 rows)
+
+
+
SELECT _id, * FROM orders WHERE _id = ObjectId('55b151633864d6438c61a9ce');
+
+
+
                 _id                 | orderkey | orderstatus | totalprice | orderdate
+-------------------------------------+----------+-------------+------------+------------
+ 55 b1 51 63 38 64 d6 43 8c 61 a9 ce |        1 | bad         |       50.0 | 2015-07-23
+(1 row)
+
+
+

You can render the _id field to readable values with a cast to VARCHAR:

+
SELECT CAST(_id AS VARCHAR), * FROM orders WHERE _id = ObjectId('55b151633864d6438c61a9ce');
+
+
+
           _id             | orderkey | orderstatus | totalprice | orderdate
+---------------------------+----------+-------------+------------+------------
+ 55b151633864d6438c61a9ce  |        1 | bad         |       50.0 | 2015-07-23
+(1 row)
+
+
+
+

ObjectId timestamp functions#

+

The first four bytes of each ObjectId represent +an embedded timestamp of its creation time. Trino provides a couple of functions to take advantage of this MongoDB feature.

+
+
+objectid_timestamp(ObjectId) timestamp#
+

Extracts the TIMESTAMP WITH TIME ZONE from a given ObjectId:

+
SELECT objectid_timestamp(ObjectId('507f191e810c19729de860ea'));
+-- 2012-10-17 20:46:22.000 UTC
+
+
+
+
+
+timestamp_objectid(timestamp) ObjectId#
+

Creates an ObjectId from a TIMESTAMP WITH TIME ZONE:

+
SELECT timestamp_objectid(TIMESTAMP '2021-08-07 17:51:36 +00:00');
+-- 61 0e c8 28 00 00 00 00 00 00 00 00
+
+
+
+

In MongoDB, you can filter all the documents created after 2021-08-07 17:51:36 +with a query like this:

+
db.collection.find({"_id": {"$gt": ObjectId("610ec8280000000000000000")}})
+
+
+

In Trino, the same can be achieved with this query:

+
SELECT *
+FROM collection
+WHERE _id > timestamp_objectid(TIMESTAMP '2021-08-07 17:51:36 +00:00');
+
+
+
+
+
+

Type mapping#

+

Because Trino and MongoDB each support types that the other does not, this +connector modifies some types when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction.

+
+

MongoDB to Trino type mapping#

+

The connector maps MongoDB types to the corresponding Trino types following +this table:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
MongoDB to Trino type mapping#

MongoDB type

Trino type

Notes

Boolean

BOOLEAN

Int32

BIGINT

Int64

BIGINT

Double

DOUBLE

Decimal128

DECIMAL(p, s)

Date

TIMESTAMP(3)

String

VARCHAR

Binary

VARBINARY

ObjectId

ObjectId

Object

ROW

Array

ARRAY

Map to ROW if the element type is not unique.

DBRef

ROW

+

No other types are supported.

+
+
+

Trino to MongoDB type mapping#

+

The connector maps Trino types to the corresponding MongoDB types following +this table:

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Trino to MongoDB type mapping#

Trino type

MongoDB type

BOOLEAN

Boolean

BIGINT

Int64

DOUBLE

Double

DECIMAL(p, s)

Decimal128

TIMESTAMP(3)

Date

VARCHAR

String

VARBINARY

Binary

ObjectId

ObjectId

ROW

Object

ARRAY

Array

+

No other types are supported.

+
+
+
+

SQL support#

+

The connector provides read and write access to data and metadata in +MongoDB. In addition to the globally available and read operation +statements, the connector supports the following features:

+ +
+

ALTER TABLE#

+

The connector supports ALTER TABLE RENAME TO, ALTER TABLE ADD COLUMN +and ALTER TABLE DROP COLUMN operations. +Other uses of ALTER TABLE are not supported.

+
+
+
+

Fault-tolerant execution support#

+

The connector supports Fault-tolerant execution of query +processing. Read and write operations are both supported with any retry policy.

+
+
+

Table functions#

+

The connector provides specific table functions to +access MongoDB.

+
+

query(database, collection, filter) -> table#

+

The query function allows you to query the underlying MongoDB directly. It +requires syntax native to MongoDB, because the full query is pushed down and +processed by MongoDB. This can be useful for accessing native features which are +not available in Trino or for improving query performance in situations where +running a query natively may be faster.

+

For example, get all rows where regionkey field is 0:

+
SELECT
+  *
+FROM
+  TABLE(
+    example.system.query(
+      database => 'tpch',
+      collection => 'region',
+      filter => '{ regionkey: 0 }'
+    )
+  );
+
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/mysql.html b/430/connector/mysql.html new file mode 100644 index 000000000..b5a2504c8 --- /dev/null +++ b/430/connector/mysql.html @@ -0,0 +1,1667 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + MySQL connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

MySQL connector#

+

The MySQL connector allows querying and creating tables in an external +MySQL instance. This can be used to join data between different +systems like MySQL and Hive, or between two different MySQL instances.

+
+

Requirements#

+

To connect to MySQL, you need:

+
    +
  • MySQL 5.7, 8.0 or higher.

  • +
  • Network access from the Trino coordinator and workers to MySQL. +Port 3306 is the default port.

  • +
+
+
+

Configuration#

+

To configure the MySQL connector, create a catalog properties file in +etc/catalog named, for example, example.properties, to mount the MySQL +connector as the mysql catalog. Create the file with the following contents, +replacing the connection properties as appropriate for your setup:

+
connector.name=mysql
+connection-url=jdbc:mysql://example.net:3306
+connection-user=root
+connection-password=secret
+
+
+

The connection-url defines the connection information and parameters to pass +to the MySQL JDBC driver. The supported parameters for the URL are +available in the MySQL Developer Guide.

+

For example, the following connection-url allows you to require encrypted +connections to the MySQL server:

+
connection-url=jdbc:mysql://example.net:3306?sslMode=REQUIRED
+
+
+

The connection-user and connection-password are typically required and +determine the user credentials for the connection, often a service user. You can +use secrets to avoid actual values in the catalog +properties files.

+
+

Connection security#

+

If you have TLS configured with a globally-trusted certificate installed on your +data source, you can enable TLS between your cluster and the data +source by appending a parameter to the JDBC connection string set in the +connection-url catalog configuration property.

+

For example, with version 8.0 of MySQL Connector/J, use the sslMode +parameter to secure the connection with TLS. By default the parameter is set to +PREFERRED which secures the connection if enabled by the server. You can +also set this parameter to REQUIRED which causes the connection to fail if +TLS is not established.

+

You can set the sslMode parameter in the catalog configuration file by +appending it to the connection-url configuration property:

+
connection-url=jdbc:mysql://example.net:3306/?sslMode=REQUIRED
+
+
+

For more information on TLS configuration options, see the MySQL JDBC security +documentation.

+
+
+

Data source authentication#

+

The connector can provide credentials for the data source connection +in multiple ways:

+
    +
  • inline, in the connector configuration file

  • +
  • in a separate properties file

  • +
  • in a key store file

  • +
  • as extra credentials set when connecting to Trino

  • +
+

You can use secrets to avoid storing sensitive +values in the catalog properties files.

+

The following table describes configuration properties +for connection credentials:

+ ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

credential-provider.type

Type of the credential provider. Must be one of INLINE, FILE, or +KEYSTORE; defaults to INLINE.

connection-user

Connection user name.

connection-password

Connection password.

user-credential-name

Name of the extra credentials property, whose value to use as the user +name. See extraCredentials in Parameter +reference.

password-credential-name

Name of the extra credentials property, whose value to use as the +password.

connection-credential-file

Location of the properties file where credentials are present. It must +contain the connection-user and connection-password properties.

keystore-file-path

The location of the Java Keystore file, from which to read credentials.

keystore-type

File format of the keystore file, for example JKS or PEM.

keystore-password

Password for the key store.

keystore-user-credential-name

Name of the key store entity to use as the user name.

keystore-user-credential-password

Password for the user name key store entity.

keystore-password-credential-name

Name of the key store entity to use as the password.

keystore-password-credential-password

Password for the password key store entity.

+
+
+

Multiple MySQL servers#

+

You can have as many catalogs as you need, so if you have additional +MySQL servers, simply add another properties file to etc/catalog +with a different name, making sure it ends in .properties. For +example, if you name the property file sales.properties, Trino +creates a catalog named sales using the configured connector.

+
+
+

General configuration properties#

+

The following table describes general catalog configuration properties for the +connector:

+ +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

Default value

case-insensitive-name-matching

Support case insensitive schema and table names.

false

case-insensitive-name-matching.cache-ttl

This value should be a duration.

1m

case-insensitive-name-matching.config-file

Path to a name mapping configuration file in JSON format that allows +Trino to disambiguate between schemas and tables with similar names in +different cases.

null

case-insensitive-name-matching.config-file.refresh-period

Frequency with which Trino checks the name matching configuration file +for changes. This value should be a duration.

(refresh disabled)

metadata.cache-ttl

The duration for which metadata, including +table and column statistics, is cached.

0s (caching disabled)

metadata.cache-missing

Cache the fact that metadata, including table and column statistics, is +not available

false

metadata.cache-maximum-size

Maximum number of objects stored in the metadata cache

10000

write.batch-size

Maximum number of statements in a batched execution. +Do not change this setting from the default. Non-default values may +negatively impact performance.

1000

dynamic-filtering.enabled

Push down dynamic filters into JDBC queries

true

dynamic-filtering.wait-timeout

Maximum duration for which Trino will wait for dynamic +filters to be collected from the build side of joins before starting a +JDBC query. Using a large timeout can potentially result in more detailed +dynamic filters. However, it can also increase latency for some queries.

20s

+
+
+

Appending query metadata#

+

The optional parameter query.comment-format allows you to configure a SQL +comment that is sent to the datasource with each query. The format of this +comment can contain any characters and the following metadata:

+
    +
  • $QUERY_ID: The identifier of the query.

  • +
  • $USER: The name of the user who submits the query to Trino.

  • +
  • $SOURCE: The identifier of the client tool used to submit the query, for +example trino-cli.

  • +
  • $TRACE_TOKEN: The trace token configured with the client tool.

  • +
+

The comment can provide more context about the query. This additional +information is available in the logs of the datasource. To include environment +variables from the Trino cluster with the comment , use the +${ENV:VARIABLE-NAME} syntax.

+

The following example sets a simple comment that identifies each query sent by +Trino:

+
query.comment-format=Query sent by Trino.
+
+
+

With this configuration, a query such as SELECT * FROM example_table; is +sent to the datasource with the comment appended:

+
SELECT * FROM example_table; /*Query sent by Trino.*/
+
+
+

The following example improves on the preceding example by using metadata:

+
query.comment-format=Query $QUERY_ID sent by user $USER from Trino.
+
+
+

If Jane sent the query with the query identifier +20230622_180528_00000_bkizg, the following comment string is sent to the +datasource:

+
SELECT * FROM example_table; /*Query 20230622_180528_00000_bkizg sent by user Jane from Trino.*/
+
+
+
+

Note

+

Certain JDBC driver settings and logging configurations might cause the +comment to be removed.

+
+
+
+

Domain compaction threshold#

+

Pushing down a large list of predicates to the data source can compromise +performance. Trino compacts large predicates into a simpler range predicate +by default to ensure a balance between performance and predicate pushdown. +If necessary, the threshold for this compaction can be increased to improve +performance when the data source is capable of taking advantage of large +predicates. Increasing this threshold may improve pushdown of large +dynamic filters. +The domain-compaction-threshold catalog configuration property or the +domain_compaction_threshold catalog session property can be used to adjust the default value of +32 for this threshold.

+
+
+

Procedures#

+
    +
  • system.flush_metadata_cache()

    +

    Flush JDBC metadata caches. For example, the following system call +flushes the metadata caches for all schemas in the example catalog

    +
    USE example.example_schema;
    +CALL system.flush_metadata_cache();
    +
    +
    +
  • +
+
+
+

Case insensitive matching#

+

When case-insensitive-name-matching is set to true, Trino +is able to query non-lowercase schemas and tables by maintaining a mapping of +the lowercase name to the actual name in the remote system. However, if two +schemas and/or tables have names that differ only in case (such as “customers” +and “Customers”) then Trino fails to query them due to ambiguity.

+

In these cases, use the case-insensitive-name-matching.config-file catalog +configuration property to specify a configuration file that maps these remote +schemas/tables to their respective Trino schemas/tables:

+
{
+  "schemas": [
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "mapping": "case_insensitive_1"
+    },
+    {
+      "remoteSchema": "cASEsENSITIVEnAME",
+      "mapping": "case_insensitive_2"
+    }],
+  "tables": [
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "remoteTable": "tablex",
+      "mapping": "table_1"
+    },
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "remoteTable": "TABLEX",
+      "mapping": "table_2"
+    }]
+}
+
+
+

Queries against one of the tables or schemes defined in the mapping +attributes are run against the corresponding remote entity. For example, a query +against tables in the case_insensitive_1 schema is forwarded to the +CaseSensitiveName schema and a query against case_insensitive_2 is forwarded +to the cASEsENSITIVEnAME schema.

+

At the table mapping level, a query on case_insensitive_1.table_1 as +configured above is forwarded to CaseSensitiveName.tablex, and a query on +case_insensitive_1.table_2 is forwarded to CaseSensitiveName.TABLEX.

+

By default, when a change is made to the mapping configuration file, Trino must +be restarted to load the changes. Optionally, you can set the +case-insensitive-name-mapping.refresh-period to have Trino refresh the +properties without requiring a restart:

+
case-insensitive-name-mapping.refresh-period=30s
+
+
+
+
+

Non-transactional INSERT#

+

The connector supports adding rows using INSERT statements. +By default, data insertion is performed by writing data to a temporary table. +You can skip this step to improve performance and write directly to the target +table. Set the insert.non-transactional-insert.enabled catalog property +or the corresponding non_transactional_insert catalog session property to +true.

+

Note that with this property enabled, data can be corrupted in rare cases where +exceptions occur during the insert operation. With transactions disabled, no +rollback can be performed.

+
+
+
+

Type mapping#

+

Because Trino and MySQL each support types that the other does not, this +connector modifies some types when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction.

+
+

MySQL to Trino type mapping#

+

The connector maps MySQL types to the corresponding Trino types following +this table:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
MySQL to Trino type mapping#

MySQL database type

Trino type

Notes

BIT

BOOLEAN

BOOLEAN

TINYINT

TINYINT

TINYINT

TINYINT UNSIGNED

SMALLINT

SMALLINT

SMALLINT

SMALLINT UNSIGNED

INTEGER

INTEGER

INTEGER

INTEGER UNSIGNED

BIGINT

BIGINT

BIGINT

BIGINT UNSIGNED

DECIMAL(20, 0)

DOUBLE PRECISION

DOUBLE

FLOAT

REAL

REAL

REAL

DECIMAL(p, s)

DECIMAL(p, s)

See MySQL DECIMAL type handling

CHAR(n)

CHAR(n)

VARCHAR(n)

VARCHAR(n)

TINYTEXT

VARCHAR(255)

TEXT

VARCHAR(65535)

MEDIUMTEXT

VARCHAR(16777215)

LONGTEXT

VARCHAR

ENUM(n)

VARCHAR(n)

BINARY, VARBINARY, TINYBLOB, BLOB, MEDIUMBLOB, LONGBLOB

VARBINARY

JSON

JSON

DATE

DATE

TIME(n)

TIME(n)

DATETIME(n)

TIMESTAMP(n)

TIMESTAMP(n)

TIMESTAMP(n) WITH TIME ZONE

+

No other types are supported.

+
+
+

Trino to MySQL type mapping#

+

The connector maps Trino types to the corresponding MySQL types following +this table:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Trino to MySQL type mapping#

Trino type

MySQL type

Notes

BOOLEAN

TINYINT

TINYINT

TINYINT

SMALLINT

SMALLINT

INTEGER

INTEGER

BIGINT

BIGINT

REAL

REAL

DOUBLE

DOUBLE PRECISION

DECIMAL(p, s)

DECIMAL(p, s)

MySQL DECIMAL type handling

CHAR(n)

CHAR(n)

VARCHAR(n)

VARCHAR(n)

JSON

JSON

DATE

DATE

TIME(n)

TIME(n)

TIMESTAMP(n)

DATETIME(n)

TIMESTAMP(n) WITH TIME ZONE

TIMESTAMP(n)

+

No other types are supported.

+
+
+

Timestamp type handling#

+

MySQL TIMESTAMP types are mapped to Trino TIMESTAMP WITH TIME ZONE. +To preserve time instants, Trino sets the session time zone +of the MySQL connection to match the JVM time zone. +As a result, error messages similar to the following example occur when +a timezone from the JVM does not exist on the MySQL server:

+
com.mysql.cj.exceptions.CJException: Unknown or incorrect time zone: 'UTC'
+
+
+

To avoid the errors, you must use a time zone that is known on both systems, +or install the missing time zone on the MySQL server.

+
+
+

Decimal type handling#

+

DECIMAL types with unspecified precision or scale are mapped to a Trino +DECIMAL with a default precision of 38 and default scale of 0. The scale can +be changed by setting the decimal-mapping configuration property or the +decimal_mapping session property to allow_overflow. The scale of the +resulting type is controlled via the decimal-default-scale configuration +property or the decimal-rounding-mode session property. The precision is +always 38.

+

By default, values that require rounding or truncation to fit will cause a +failure at runtime. This behavior is controlled via the +decimal-rounding-mode configuration property or the +decimal_rounding_mode session property, which can be set to UNNECESSARY +(the default), UP, DOWN, CEILING, FLOOR, HALF_UP, +HALF_DOWN, or HALF_EVEN (see RoundingMode).

+
+
+

Type mapping configuration properties#

+

The following properties can be used to configure how data types from the +connected data source are mapped to Trino data types and how the metadata is +cached in Trino.

+ +++++ + + + + + + + + + + + + + + + + +

Property name

Description

Default value

unsupported-type-handling

Configure how unsupported column data types are handled:

+
    +
  • IGNORE, column is not accessible.

  • +
  • CONVERT_TO_VARCHAR, column is converted to unbounded VARCHAR.

  • +
+

The respective catalog session property is unsupported_type_handling.

+

IGNORE

jdbc-types-mapped-to-varchar

Allow forced mapping of comma separated lists of data types to convert to +unbounded VARCHAR

+
+
+
+

Querying MySQL#

+

The MySQL connector provides a schema for every MySQL database. +You can see the available MySQL databases by running SHOW SCHEMAS:

+
SHOW SCHEMAS FROM example;
+
+
+

If you have a MySQL database named web, you can view the tables +in this database by running SHOW TABLES:

+
SHOW TABLES FROM example.web;
+
+
+

You can see a list of the columns in the clicks table in the web database +using either of the following:

+
DESCRIBE example.web.clicks;
+SHOW COLUMNS FROM example.web.clicks;
+
+
+

Finally, you can access the clicks table in the web database:

+
SELECT * FROM example.web.clicks;
+
+
+

If you used a different name for your catalog properties file, use +that catalog name instead of example in the above examples.

+
+
+

SQL support#

+

The connector provides read access and write access to data and metadata in the +MySQL database. In addition to the globally available and +read operation statements, the connector supports +the following statements:

+ +
+

UPDATE#

+

Only UPDATE statements with constant assignments and predicates are +supported. For example, the following statement is supported because the values +assigned are constants:

+
UPDATE table SET col1 = 1 WHERE col3 = 1
+
+
+

Arithmetic expressions, function calls, and other non-constant UPDATE +statements are not supported. For example, the following statement is not +supported because arithmetic expressions cannot be used with the SET +command:

+
UPDATE table SET col1 = col2 + 2 WHERE col3 = 1
+
+
+

The =, !=, >, <, >=, <=, IN, NOT IN operators are supported in +predicates. The following statement is not supported because the AND operator +cannot be used in predicates:

+
UPDATE table SET col1 = 1 WHERE col3 = 1 AND col2 = 3
+
+
+

All column values of a table row cannot be updated simultaneously. For a three +column table, the following statement is not supported:

+
UPDATE table SET col1 = 1, col2 = 2, col3 = 3 WHERE col3 = 1
+
+
+
+
+

SQL DELETE#

+

If a WHERE clause is specified, the DELETE operation only works if the +predicate in the clause can be fully pushed down to the data source.

+
+
+
+

Fault-tolerant execution support#

+

The connector supports Fault-tolerant execution of query +processing. Read and write operations are both supported with any retry policy.

+
+
+

Table functions#

+

The connector provides specific table functions to +access MySQL.

+
+

query(varchar) -> table#

+

The query function allows you to query the underlying database directly. It +requires syntax native to MySQL, because the full query is pushed down and +processed in MySQL. This can be useful for accessing native features which are +not available in Trino or for improving query performance in situations where +running a query natively may be faster.

+

The native query passed to the underlying data source is required to return a +table as a result set. Only the data source performs validation or security +checks for these queries using its own configuration. Trino does not perform +these tasks. Only use passthrough queries to read data.

+

For example, query the example catalog and group and concatenate all +employee IDs by manager ID:

+
SELECT
+  *
+FROM
+  TABLE(
+    example.system.query(
+      query => 'SELECT
+        manager_id, GROUP_CONCAT(employee_id)
+      FROM
+        company.employees
+      GROUP BY
+        manager_id'
+    )
+  );
+
+
+
+

Note

+

The query engine does not preserve the order of the results of this +function. If the passed query contains an ORDER BY clause, the +function result may not be ordered as expected.

+
+
+
+
+

Performance#

+

The connector includes a number of performance improvements, detailed in the +following sections.

+
+

Table statistics#

+

The MySQL connector can use table and column statistics for cost based optimizations, to improve query processing performance +based on the actual data in the data source.

+

The statistics are collected by MySQL and retrieved by the connector.

+

The table-level statistics are based on MySQL’s INFORMATION_SCHEMA.TABLES +table. The column-level statistics are based on MySQL’s index statistics +INFORMATION_SCHEMA.STATISTICS table. The connector can return column-level +statistics only when the column is the first column in some index.

+

MySQL database can automatically update its table and index statistics. In some +cases, you may want to force statistics update, for example after creating new +index, or after changing data in the table. You can do that by executing the +following statement in MySQL Database.

+
ANALYZE TABLE table_name;
+
+
+
+

Note

+

MySQL and Trino may use statistics information in different ways. For this +reason, the accuracy of table and column statistics returned by the MySQL +connector might be lower than than that of others connectors.

+
+

Improving statistics accuracy

+

You can improve statistics accuracy with histogram statistics (available since +MySQL 8.0). To create histogram statistics execute the following statement in +MySQL Database.

+
ANALYZE TABLE table_name UPDATE HISTOGRAM ON column_name1, column_name2, ...;
+
+
+

Refer to MySQL documentation for information about options, limitations +and additional considerations.

+
+
+

Pushdown#

+

The connector supports pushdown for a number of operations:

+ +

Aggregate pushdown for the following functions:

+ +
+

Note

+

The connector performs pushdown where performance may be improved, but in +order to preserve correctness an operation may not be pushed down. When +pushdown of an operation may result in better performance but risks +correctness, the connector prioritizes correctness.

+
+
+

Cost-based join pushdown#

+

The connector supports cost-based Join pushdown to make intelligent +decisions about whether to push down a join operation to the data source.

+

When cost-based join pushdown is enabled, the connector only pushes down join +operations if the available Table statistics suggest that doing so +improves performance. Note that if no table statistics are available, join +operation pushdown does not occur to avoid a potential decrease in query +performance.

+

The following table describes catalog configuration properties for +join pushdown:

+ +++++ + + + + + + + + + + + + + + + + +

Property name

Description

Default value

join-pushdown.enabled

Enable join pushdown. Equivalent catalog +session property is +join_pushdown_enabled.

true

join-pushdown.strategy

Strategy used to evaluate whether join operations are pushed down. Set to +AUTOMATIC to enable cost-based join pushdown, or EAGER to +push down joins whenever possible. Note that EAGER can push down joins +even when table statistics are unavailable, which may result in degraded +query performance. Because of this, EAGER is only recommended for +testing and troubleshooting purposes.

AUTOMATIC

+
+
+

Predicate pushdown support#

+

The connector does not support pushdown of any predicates on columns with +textual types like CHAR or VARCHAR. +This ensures correctness of results since the data source may compare strings +case-insensitively.

+

In the following example, the predicate is not pushed down for either query +since name is a column of type VARCHAR:

+
SELECT * FROM nation WHERE name > 'CANADA';
+SELECT * FROM nation WHERE name = 'CANADA';
+
+
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/object-storage-file-formats.html b/430/connector/object-storage-file-formats.html new file mode 100644 index 000000000..8aefb6e7b --- /dev/null +++ b/430/connector/object-storage-file-formats.html @@ -0,0 +1,861 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Object storage file formats — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Object storage file formats#

+

Object storage connectors support one or more file formats specified by the +underlying data source.

+

In the case of serializable formats, only specific +SerDes are allowed:

+
    +
  • RCText - RCFile ColumnarSerDe

  • +
  • RCBinary - RCFile LazyBinaryColumnarSerDe

  • +
  • JSON - org.apache.hive.hcatalog.data.JsonSerDe

  • +
  • CSV - org.apache.hadoop.hive.serde2.OpenCSVSerde

  • +
+
+

ORC format configuration properties#

+

The following properties are used to configure the read and write operations +with ORC files performed by supported object storage connectors:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + +
ORC format configuration properties#

Property Name

Description

Default

hive.orc.time-zone

Sets the default time zone for legacy ORC files that did not declare a +time zone.

JVM default

hive.orc.use-column-names

Access ORC columns by name. By default, columns in ORC files are +accessed by their ordinal position in the Hive table definition. The +equivalent catalog session property is orc_use_column_names.

false

hive.orc.bloom-filters.enabled

Enable bloom filters for predicate pushdown.

false

hive.orc.read-legacy-short-zone-id

Allow reads on ORC files with short zone ID in the stripe footer.

false

+
+
+

Parquet format configuration properties#

+

The following properties are used to configure the read and write operations +with Parquet files performed by supported object storage connectors:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Parquet format configuration properties#

Property Name

Description

Default

hive.parquet.time-zone

Adjusts timestamp values to a specific time zone. For Hive 3.1+, set +this to UTC.

JVM default

hive.parquet.use-column-names

Access Parquet columns by name by default. Set this property to +false to access columns by their ordinal position in the Hive table +definition. The equivalent catalog session property is +parquet_use_column_names.

true

parquet.writer.validation-percentage

Percentage of parquet files to validate after write by re-reading the whole file. +The equivalent catalog session property is parquet_optimized_writer_validation_percentage. +Validation can be turned off by setting this property to 0.

5

parquet.writer.page-size

Maximum page size for the Parquet writer.

1 MB

parquet.writer.block-size

Maximum row group size for the Parquet writer.

128 MB

parquet.writer.batch-size

Maximum number of rows processed by the parquet writer in a batch.

10000

parquet.use-bloom-filter

Whether bloom filters are used for predicate pushdown when reading +Parquet files. Set this property to false to disable the usage of +bloom filters by default. The equivalent catalog session property is +parquet_use_bloom_filter.

true

parquet.use-column-index

Skip reading Parquet pages by using Parquet column indices. The +equivalent catalog session property is parquet_use_column_index. +Only supported by the Delta Lake and Hive connectors.

true

parquet.max-read-block-row-count

Sets the maximum number of rows read in a batch. The equivalent catalog +session property is named parquet_max_read_block_row_count and +supported by the Delta Lake, Hive, and Iceberg connectors.

8192

parquet.small-file-threshold

Data size below which a Parquet file is +read entirely. The equivalent catalog session property is named +parquet_small_file_threshold.

3MB

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/oracle.html b/430/connector/oracle.html new file mode 100644 index 000000000..01f035346 --- /dev/null +++ b/430/connector/oracle.html @@ -0,0 +1,1753 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Oracle connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Oracle connector#

+

The Oracle connector allows querying and creating tables in an external Oracle +database. Connectors let Trino join data provided by different databases, +like Oracle and Hive, or different Oracle database instances.

+
+

Requirements#

+

To connect to Oracle, you need:

+
    +
  • Oracle 12 or higher.

  • +
  • Network access from the Trino coordinator and workers to Oracle. +Port 1521 is the default port.

  • +
+
+
+

Configuration#

+

To configure the Oracle connector as the example catalog, create a file +named example.properties in etc/catalog. Include the following +connection properties in the file:

+
connector.name=oracle
+# The correct syntax of the connection-url varies by Oracle version and
+# configuration. The following example URL connects to an Oracle SID named
+# "orcl".
+connection-url=jdbc:oracle:thin:@example.net:1521:orcl
+connection-user=root
+connection-password=secret
+
+
+

The connection-url defines the connection information and parameters to pass +to the JDBC driver. The Oracle connector uses the Oracle JDBC Thin driver, +and the syntax of the URL may be different depending on your Oracle +configuration. For example, the connection URL is different if you are +connecting to an Oracle SID or an Oracle service name. See the Oracle +Database JDBC driver documentation +for more information.

+

The connection-user and connection-password are typically required and +determine the user credentials for the connection, often a service user. You can +use secrets to avoid actual values in the catalog +properties files.

+
+

Note

+

Oracle does not expose metadata comment via REMARKS column by default +in JDBC driver. You can enable it using oracle.remarks-reporting.enabled +config option. See Additional Oracle Performance Extensions +for more details.

+
+

By default, the Oracle connector uses connection pooling for performance +improvement. The below configuration shows the typical default values. To update +them, change the properties in the catalog configuration file:

+
oracle.connection-pool.max-size=30
+oracle.connection-pool.min-size=1
+oracle.connection-pool.inactive-timeout=20m
+
+
+

To disable connection pooling, update properties to include the following:

+
oracle.connection-pool.enabled=false
+
+
+
+

Data source authentication#

+

The connector can provide credentials for the data source connection +in multiple ways:

+
    +
  • inline, in the connector configuration file

  • +
  • in a separate properties file

  • +
  • in a key store file

  • +
  • as extra credentials set when connecting to Trino

  • +
+

You can use secrets to avoid storing sensitive +values in the catalog properties files.

+

The following table describes configuration properties +for connection credentials:

+ ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

credential-provider.type

Type of the credential provider. Must be one of INLINE, FILE, or +KEYSTORE; defaults to INLINE.

connection-user

Connection user name.

connection-password

Connection password.

user-credential-name

Name of the extra credentials property, whose value to use as the user +name. See extraCredentials in Parameter +reference.

password-credential-name

Name of the extra credentials property, whose value to use as the +password.

connection-credential-file

Location of the properties file where credentials are present. It must +contain the connection-user and connection-password properties.

keystore-file-path

The location of the Java Keystore file, from which to read credentials.

keystore-type

File format of the keystore file, for example JKS or PEM.

keystore-password

Password for the key store.

keystore-user-credential-name

Name of the key store entity to use as the user name.

keystore-user-credential-password

Password for the user name key store entity.

keystore-password-credential-name

Name of the key store entity to use as the password.

keystore-password-credential-password

Password for the password key store entity.

+
+
+

Multiple Oracle servers#

+

If you want to connect to multiple Oracle servers, configure another instance of +the Oracle connector as a separate catalog.

+

To add another Oracle catalog, create a new properties file. For example, if +you name the property file sales.properties, Trino creates a catalog named +sales.

+
+
+

General configuration properties#

+

The following table describes general catalog configuration properties for the +connector:

+ +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

Default value

case-insensitive-name-matching

Support case insensitive schema and table names.

false

case-insensitive-name-matching.cache-ttl

This value should be a duration.

1m

case-insensitive-name-matching.config-file

Path to a name mapping configuration file in JSON format that allows +Trino to disambiguate between schemas and tables with similar names in +different cases.

null

case-insensitive-name-matching.config-file.refresh-period

Frequency with which Trino checks the name matching configuration file +for changes. This value should be a duration.

(refresh disabled)

metadata.cache-ttl

The duration for which metadata, including +table and column statistics, is cached.

0s (caching disabled)

metadata.cache-missing

Cache the fact that metadata, including table and column statistics, is +not available

false

metadata.cache-maximum-size

Maximum number of objects stored in the metadata cache

10000

write.batch-size

Maximum number of statements in a batched execution. +Do not change this setting from the default. Non-default values may +negatively impact performance.

1000

dynamic-filtering.enabled

Push down dynamic filters into JDBC queries

true

dynamic-filtering.wait-timeout

Maximum duration for which Trino will wait for dynamic +filters to be collected from the build side of joins before starting a +JDBC query. Using a large timeout can potentially result in more detailed +dynamic filters. However, it can also increase latency for some queries.

20s

+
+
+

Appending query metadata#

+

The optional parameter query.comment-format allows you to configure a SQL +comment that is sent to the datasource with each query. The format of this +comment can contain any characters and the following metadata:

+
    +
  • $QUERY_ID: The identifier of the query.

  • +
  • $USER: The name of the user who submits the query to Trino.

  • +
  • $SOURCE: The identifier of the client tool used to submit the query, for +example trino-cli.

  • +
  • $TRACE_TOKEN: The trace token configured with the client tool.

  • +
+

The comment can provide more context about the query. This additional +information is available in the logs of the datasource. To include environment +variables from the Trino cluster with the comment , use the +${ENV:VARIABLE-NAME} syntax.

+

The following example sets a simple comment that identifies each query sent by +Trino:

+
query.comment-format=Query sent by Trino.
+
+
+

With this configuration, a query such as SELECT * FROM example_table; is +sent to the datasource with the comment appended:

+
SELECT * FROM example_table; /*Query sent by Trino.*/
+
+
+

The following example improves on the preceding example by using metadata:

+
query.comment-format=Query $QUERY_ID sent by user $USER from Trino.
+
+
+

If Jane sent the query with the query identifier +20230622_180528_00000_bkizg, the following comment string is sent to the +datasource:

+
SELECT * FROM example_table; /*Query 20230622_180528_00000_bkizg sent by user Jane from Trino.*/
+
+
+
+

Note

+

Certain JDBC driver settings and logging configurations might cause the +comment to be removed.

+
+
+
+

Domain compaction threshold#

+

Pushing down a large list of predicates to the data source can compromise +performance. Trino compacts large predicates into a simpler range predicate +by default to ensure a balance between performance and predicate pushdown. +If necessary, the threshold for this compaction can be increased to improve +performance when the data source is capable of taking advantage of large +predicates. Increasing this threshold may improve pushdown of large +dynamic filters. +The domain-compaction-threshold catalog configuration property or the +domain_compaction_threshold catalog session property can be used to adjust the default value of +32 for this threshold.

+
+
+

Procedures#

+
    +
  • system.flush_metadata_cache()

    +

    Flush JDBC metadata caches. For example, the following system call +flushes the metadata caches for all schemas in the example catalog

    +
    USE example.example_schema;
    +CALL system.flush_metadata_cache();
    +
    +
    +
  • +
+
+
+

Case insensitive matching#

+

When case-insensitive-name-matching is set to true, Trino +is able to query non-lowercase schemas and tables by maintaining a mapping of +the lowercase name to the actual name in the remote system. However, if two +schemas and/or tables have names that differ only in case (such as “customers” +and “Customers”) then Trino fails to query them due to ambiguity.

+

In these cases, use the case-insensitive-name-matching.config-file catalog +configuration property to specify a configuration file that maps these remote +schemas/tables to their respective Trino schemas/tables:

+
{
+  "schemas": [
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "mapping": "case_insensitive_1"
+    },
+    {
+      "remoteSchema": "cASEsENSITIVEnAME",
+      "mapping": "case_insensitive_2"
+    }],
+  "tables": [
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "remoteTable": "tablex",
+      "mapping": "table_1"
+    },
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "remoteTable": "TABLEX",
+      "mapping": "table_2"
+    }]
+}
+
+
+

Queries against one of the tables or schemes defined in the mapping +attributes are run against the corresponding remote entity. For example, a query +against tables in the case_insensitive_1 schema is forwarded to the +CaseSensitiveName schema and a query against case_insensitive_2 is forwarded +to the cASEsENSITIVEnAME schema.

+

At the table mapping level, a query on case_insensitive_1.table_1 as +configured above is forwarded to CaseSensitiveName.tablex, and a query on +case_insensitive_1.table_2 is forwarded to CaseSensitiveName.TABLEX.

+

By default, when a change is made to the mapping configuration file, Trino must +be restarted to load the changes. Optionally, you can set the +case-insensitive-name-mapping.refresh-period to have Trino refresh the +properties without requiring a restart:

+
case-insensitive-name-mapping.refresh-period=30s
+
+
+
+
+

Non-transactional INSERT#

+

The connector supports adding rows using INSERT statements. +By default, data insertion is performed by writing data to a temporary table. +You can skip this step to improve performance and write directly to the target +table. Set the insert.non-transactional-insert.enabled catalog property +or the corresponding non_transactional_insert catalog session property to +true.

+

Note that with this property enabled, data can be corrupted in rare cases where +exceptions occur during the insert operation. With transactions disabled, no +rollback can be performed.

+
+
+
+

Querying Oracle#

+

The Oracle connector provides a schema for every Oracle database.

+

Run SHOW SCHEMAS to see the available Oracle databases:

+
SHOW SCHEMAS FROM example;
+
+
+

If you used a different name for your catalog properties file, use that catalog +name instead of example.

+
+

Note

+

The Oracle user must have access to the table in order to access it from Trino. +The user configuration, in the connection properties file, determines your +privileges in these schemas.

+
+
+

Examples#

+

If you have an Oracle database named web, run SHOW TABLES to see the +tables it contains:

+
SHOW TABLES FROM example.web;
+
+
+

To see a list of the columns in the clicks table in the web +database, run either of the following:

+
DESCRIBE example.web.clicks;
+SHOW COLUMNS FROM example.web.clicks;
+
+
+

To access the clicks table in the web database, run the following:

+
SELECT * FROM example.web.clicks;
+
+
+
+
+
+

Type mapping#

+

Because Trino and Oracle each support types that the other does not, this +connector modifies some types when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction.

+
+

Oracle to Trino type mapping#

+

Trino supports selecting Oracle database types. This table shows the Oracle to +Trino data type mapping:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Oracle to Trino type mapping#

Oracle database type

Trino type

Notes

NUMBER(p, s)

DECIMAL(p, s)

See Mapping numeric types

NUMBER(p)

DECIMAL(p, 0)

See Mapping numeric types

FLOAT[(p)]

DOUBLE

BINARY_FLOAT

REAL

BINARY_DOUBLE

DOUBLE

VARCHAR2(n CHAR)

VARCHAR(n)

VARCHAR2(n BYTE)

VARCHAR(n)

NVARCHAR2(n)

VARCHAR(n)

CHAR(n)

CHAR(n)

NCHAR(n)

CHAR(n)

CLOB

VARCHAR

NCLOB

VARCHAR

RAW(n)

VARBINARY

BLOB

VARBINARY

DATE

TIMESTAMP(0)

See Mapping datetime types

TIMESTAMP(p)

TIMESTAMP(p)

See Mapping datetime types

TIMESTAMP(p) WITH TIME ZONE

TIMESTAMP WITH TIME ZONE

See Mapping datetime types

+

No other types are supported.

+
+
+

Trino to Oracle type mapping#

+

Trino supports creating tables with the following types in an Oracle database. +The table shows the mappings from Trino to Oracle data types:

+
+

Note

+

For types not listed in the table below, Trino can’t perform the CREATE TABLE <table> AS SELECT operations. When data is inserted into existing +tables, Oracle to Trino type mapping is used.

+
+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Trino to Oracle Type Mapping#

Trino type

Oracle database type

Notes

TINYINT

NUMBER(3)

SMALLINT

NUMBER(5)

INTEGER

NUMBER(10)

BIGINT

NUMBER(19)

DECIMAL(p, s)

NUMBER(p, s)

REAL

BINARY_FLOAT

DOUBLE

BINARY_DOUBLE

VARCHAR

NCLOB

VARCHAR(n)

VARCHAR2(n CHAR) or NCLOB

See Mapping character types

CHAR(n)

CHAR(n CHAR) or NCLOB

See Mapping character types

VARBINARY

BLOB

DATE

DATE

See Mapping datetime types

TIMESTAMP

TIMESTAMP(3)

See Mapping datetime types

TIMESTAMP WITH TIME ZONE

TIMESTAMP(3) WITH TIME ZONE

See Mapping datetime types

+

No other types are supported.

+
+
+

Mapping numeric types#

+

An Oracle NUMBER(p, s) maps to Trino’s DECIMAL(p, s) except in these +conditions:

+
    +
  • No precision is specified for the column (example: NUMBER or +NUMBER(*)), unless oracle.number.default-scale is set.

  • +
  • Scale (s ) is greater than precision.

  • +
  • Precision (p ) is greater than 38.

  • +
  • Scale is negative and the difference between p and s is greater than +38, unless oracle.number.rounding-mode is set to a different value than +UNNECESSARY.

  • +
+

If s is negative, NUMBER(p, s) maps to DECIMAL(p + s, 0).

+

For Oracle NUMBER (without precision and scale), you can change +oracle.number.default-scale=s and map the column to DECIMAL(38, s).

+
+
+

Mapping datetime types#

+

Writing a timestamp with fractional second precision (p) greater than 9 +rounds the fractional seconds to nine digits.

+

Oracle DATE type stores hours, minutes, and seconds, so it is mapped +to Trino TIMESTAMP(0).

+
+

Warning

+

Due to date and time differences in the libraries used by Trino and the +Oracle JDBC driver, attempting to insert or select a datetime value earlier +than 1582-10-15 results in an incorrect date inserted.

+
+
+
+

Mapping character types#

+

Trino’s VARCHAR(n) maps to VARCHAR2(n CHAR) if n is no greater +than 4000. A larger or unbounded VARCHAR maps to NCLOB.

+

Trino’s CHAR(n) maps to CHAR(n CHAR) if n is no greater than 2000. +A larger CHAR maps to NCLOB.

+

Using CREATE TABLE AS to create an NCLOB column from a CHAR value +removes the trailing spaces from the initial values for the column. Inserting +CHAR values into existing NCLOB columns keeps the trailing spaces. For +example:

+
CREATE TABLE vals AS SELECT CAST('A' as CHAR(2001)) col;
+INSERT INTO vals (col) VALUES (CAST('BB' as CHAR(2001)));
+SELECT LENGTH(col) FROM vals;
+
+
+
 _col0
+-------
+  2001
+     1
+(2 rows)
+
+
+

Attempting to write a CHAR that doesn’t fit in the column’s actual size +fails. This is also true for the equivalent VARCHAR types.

+
+
+

Type mapping configuration properties#

+

The following properties can be used to configure how data types from the +connected data source are mapped to Trino data types and how the metadata is +cached in Trino.

+ +++++ + + + + + + + + + + + + + + + + +

Property name

Description

Default value

unsupported-type-handling

Configure how unsupported column data types are handled:

+
    +
  • IGNORE, column is not accessible.

  • +
  • CONVERT_TO_VARCHAR, column is converted to unbounded VARCHAR.

  • +
+

The respective catalog session property is unsupported_type_handling.

+

IGNORE

jdbc-types-mapped-to-varchar

Allow forced mapping of comma separated lists of data types to convert to +unbounded VARCHAR

+
+
+

Number to decimal configuration properties#

+ ++++++ + + + + + + + + + + + + + + + + + + + +

Configuration property name

Session property name

Description

Default

oracle.number.default-scale

number_default_scale

Default Trino DECIMAL scale for Oracle NUMBER (without precision +and scale) date type. When not set then such column is treated as not +supported.

not set

oracle.number.rounding-mode

number_rounding_mode

Rounding mode for the Oracle NUMBER data type. This is useful when +Oracle NUMBER data type specifies higher scale than is supported in +Trino. Possible values are:

+
    +
  • UNNECESSARY - Rounding mode to assert that the +requested operation has an exact result, +hence no rounding is necessary.

  • +
  • CEILING - Rounding mode to round towards +positive infinity.

  • +
  • FLOOR - Rounding mode to round towards negative +infinity.

  • +
  • HALF_DOWN - Rounding mode to round towards +nearest neighbor unless both neighbors are +equidistant, in which case rounding down is used.

  • +
  • HALF_EVEN - Rounding mode to round towards the +nearest neighbor unless both neighbors are equidistant, +in which case rounding towards the even neighbor is +performed.

  • +
  • HALF_UP - Rounding mode to round towards +nearest neighbor unless both neighbors are +equidistant, in which case rounding up is used

  • +
  • UP - Rounding mode to round towards zero.

  • +
  • DOWN - Rounding mode to round towards zero.

  • +
+

UNNECESSARY

+
+
+
+

SQL support#

+

The connector provides read access and write access to data and metadata in +Oracle. In addition to the globally available +and read operation statements, the connector +supports the following statements:

+ +
+

UPDATE#

+

Only UPDATE statements with constant assignments and predicates are +supported. For example, the following statement is supported because the values +assigned are constants:

+
UPDATE table SET col1 = 1 WHERE col3 = 1
+
+
+

Arithmetic expressions, function calls, and other non-constant UPDATE +statements are not supported. For example, the following statement is not +supported because arithmetic expressions cannot be used with the SET +command:

+
UPDATE table SET col1 = col2 + 2 WHERE col3 = 1
+
+
+

The =, !=, >, <, >=, <=, IN, NOT IN operators are supported in +predicates. The following statement is not supported because the AND operator +cannot be used in predicates:

+
UPDATE table SET col1 = 1 WHERE col3 = 1 AND col2 = 3
+
+
+

All column values of a table row cannot be updated simultaneously. For a three +column table, the following statement is not supported:

+
UPDATE table SET col1 = 1, col2 = 2, col3 = 3 WHERE col3 = 1
+
+
+
+
+

SQL DELETE#

+

If a WHERE clause is specified, the DELETE operation only works if the +predicate in the clause can be fully pushed down to the data source.

+
+
+

ALTER TABLE RENAME TO#

+

The connector does not support renaming tables across multiple schemas. For +example, the following statement is supported:

+
ALTER TABLE example.schema_one.table_one RENAME TO example.schema_one.table_two
+
+
+

The following statement attempts to rename a table across schemas, and therefore +is not supported:

+
ALTER TABLE example.schema_one.table_one RENAME TO example.schema_two.table_two
+
+
+
+
+
+

Fault-tolerant execution support#

+

The connector supports Fault-tolerant execution of query +processing. Read and write operations are both supported with any retry policy.

+
+
+

Table functions#

+

The connector provides specific table functions to +access Oracle.

+
+

query(varchar) -> table#

+

The query function allows you to query the underlying database directly. It +requires syntax native to Oracle, because the full query is pushed down and +processed in Oracle. This can be useful for accessing native features which are +not available in Trino or for improving query performance in situations where +running a query natively may be faster.

+

The native query passed to the underlying data source is required to return a +table as a result set. Only the data source performs validation or security +checks for these queries using its own configuration. Trino does not perform +these tasks. Only use passthrough queries to read data.

+

As a simple example, query the example catalog and select an entire table:

+
SELECT
+  *
+FROM
+  TABLE(
+    example.system.query(
+      query => 'SELECT
+        *
+      FROM
+        tpch.nation'
+    )
+  );
+
+
+

As a practical example, you can use the +MODEL clause from Oracle SQL:

+
SELECT
+  SUBSTR(country, 1, 20) country,
+  SUBSTR(product, 1, 15) product,
+  year,
+  sales
+FROM
+  TABLE(
+    example.system.query(
+      query => 'SELECT
+        *
+      FROM
+        sales_view
+      MODEL
+        RETURN UPDATED ROWS
+        MAIN
+          simple_model
+        PARTITION BY
+          country
+        MEASURES
+          sales
+        RULES
+          (sales['Bounce', 2001] = 1000,
+          sales['Bounce', 2002] = sales['Bounce', 2001] + sales['Bounce', 2000],
+          sales['Y Box', 2002] = sales['Y Box', 2001])
+      ORDER BY
+        country'
+    )
+  );
+
+
+
+

Note

+

The query engine does not preserve the order of the results of this +function. If the passed query contains an ORDER BY clause, the +function result may not be ordered as expected.

+
+
+
+
+

Performance#

+

The connector includes a number of performance improvements, detailed in the +following sections.

+
+

Synonyms#

+

Based on performance reasons, Trino disables support for Oracle SYNONYM. To +include SYNONYM, add the following configuration property:

+
oracle.synonyms.enabled=true
+
+
+
+
+

Pushdown#

+

The connector supports pushdown for a number of operations:

+ +

In addition, the connector supports Aggregation pushdown for the +following functions:

+ +

Pushdown is only supported for DOUBLE type columns with the +following functions:

+ +

Pushdown is only supported for REAL or DOUBLE type column +with the following functions:

+ +
+

Note

+

The connector performs pushdown where performance may be improved, but in +order to preserve correctness an operation may not be pushed down. When +pushdown of an operation may result in better performance but risks +correctness, the connector prioritizes correctness.

+
+
+

Join pushdown#

+

The join-pushdown.enabled catalog configuration property or +join_pushdown_enabled catalog session property control whether the connector pushes +down join operations. The property defaults to false, and enabling join +pushdowns may negatively impact performance for some queries.

+
+
+

Predicate pushdown support#

+

The connector does not support pushdown of any predicates on columns that use +the CLOB, NCLOB, BLOB, or RAW(n) Oracle database types, or Trino +data types that map to these Oracle database types.

+

In the following example, the predicate is not pushed down for either query +since name is a column of type VARCHAR, which maps to NCLOB in +Oracle:

+
SHOW CREATE TABLE nation;
+
+--             Create Table
+----------------------------------------
+-- CREATE TABLE oracle.trino_test.nation (
+--    name VARCHAR
+-- )
+-- (1 row)
+
+SELECT * FROM nation WHERE name > 'CANADA';
+SELECT * FROM nation WHERE name = 'CANADA';
+
+
+

In the following example, the predicate is pushed down for both queries +since name is a column of type VARCHAR(25), which maps to +VARCHAR2(25) in Oracle:

+
SHOW CREATE TABLE nation;
+
+--             Create Table
+----------------------------------------
+-- CREATE TABLE oracle.trino_test.nation (
+--    name VARCHAR(25)
+-- )
+-- (1 row)
+
+SELECT * FROM nation WHERE name > 'CANADA';
+SELECT * FROM nation WHERE name = 'CANADA';
+
+
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/phoenix.html b/430/connector/phoenix.html new file mode 100644 index 000000000..9d9220c0e --- /dev/null +++ b/430/connector/phoenix.html @@ -0,0 +1,1356 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Phoenix connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Phoenix connector#

+

The Phoenix connector allows querying data stored in +Apache HBase using +Apache Phoenix.

+
+

Requirements#

+

To query HBase data through Phoenix, you need:

+
    +
  • Network access from the Trino coordinator and workers to the ZooKeeper +servers. The default port is 2181.

  • +
  • A compatible version of Phoenix: all 5.x versions starting from 5.1.0 are supported.

  • +
+
+
+

Configuration#

+

To configure the Phoenix connector, create a catalog properties file +etc/catalog/example.properties with the following contents, +replacing host1,host2,host3 with a comma-separated list of the ZooKeeper +nodes used for discovery of the HBase cluster:

+
connector.name=phoenix5
+phoenix.connection-url=jdbc:phoenix:host1,host2,host3:2181:/hbase
+phoenix.config.resources=/path/to/hbase-site.xml
+
+
+

The optional paths to Hadoop resource files, such as hbase-site.xml are used +to load custom Phoenix client connection properties.

+

The following Phoenix-specific configuration properties are available:

+ + + + + + + + + + + + + + + + + + + + + +

Property name

Required

Description

phoenix.connection-url

Yes

jdbc:phoenix[:zk_quorum][:zk_port][:zk_hbase_path]. The zk_quorum is a comma separated list of ZooKeeper servers. The zk_port is the ZooKeeper port. The zk_hbase_path is the HBase root znode path, that is configurable using hbase-site.xml. By default the location is /hbase

phoenix.config.resources

No

Comma-separated list of configuration files (e.g. hbase-site.xml) to use for connection properties. These files must exist on the machines running Trino.

phoenix.max-scans-per-split

No

Maximum number of HBase scans that will be performed in a single split. Default is 20. Lower values will lead to more splits in Trino. Can also be set via session propery max_scans_per_split. For details see: https://phoenix.apache.org/update_statistics.html. (This setting has no effect when guideposts are disabled in Phoenix.)

+
+

General configuration properties#

+

The following table describes general catalog configuration properties for the +connector:

+ +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

Default value

case-insensitive-name-matching

Support case insensitive schema and table names.

false

case-insensitive-name-matching.cache-ttl

This value should be a duration.

1m

case-insensitive-name-matching.config-file

Path to a name mapping configuration file in JSON format that allows +Trino to disambiguate between schemas and tables with similar names in +different cases.

null

case-insensitive-name-matching.config-file.refresh-period

Frequency with which Trino checks the name matching configuration file +for changes. This value should be a duration.

(refresh disabled)

metadata.cache-ttl

The duration for which metadata, including +table and column statistics, is cached.

0s (caching disabled)

metadata.cache-missing

Cache the fact that metadata, including table and column statistics, is +not available

false

metadata.cache-maximum-size

Maximum number of objects stored in the metadata cache

10000

write.batch-size

Maximum number of statements in a batched execution. +Do not change this setting from the default. Non-default values may +negatively impact performance.

1000

dynamic-filtering.enabled

Push down dynamic filters into JDBC queries

true

dynamic-filtering.wait-timeout

Maximum duration for which Trino will wait for dynamic +filters to be collected from the build side of joins before starting a +JDBC query. Using a large timeout can potentially result in more detailed +dynamic filters. However, it can also increase latency for some queries.

20s

+
+
+

Appending query metadata#

+

The optional parameter query.comment-format allows you to configure a SQL +comment that is sent to the datasource with each query. The format of this +comment can contain any characters and the following metadata:

+
    +
  • $QUERY_ID: The identifier of the query.

  • +
  • $USER: The name of the user who submits the query to Trino.

  • +
  • $SOURCE: The identifier of the client tool used to submit the query, for +example trino-cli.

  • +
  • $TRACE_TOKEN: The trace token configured with the client tool.

  • +
+

The comment can provide more context about the query. This additional +information is available in the logs of the datasource. To include environment +variables from the Trino cluster with the comment , use the +${ENV:VARIABLE-NAME} syntax.

+

The following example sets a simple comment that identifies each query sent by +Trino:

+
query.comment-format=Query sent by Trino.
+
+
+

With this configuration, a query such as SELECT * FROM example_table; is +sent to the datasource with the comment appended:

+
SELECT * FROM example_table; /*Query sent by Trino.*/
+
+
+

The following example improves on the preceding example by using metadata:

+
query.comment-format=Query $QUERY_ID sent by user $USER from Trino.
+
+
+

If Jane sent the query with the query identifier +20230622_180528_00000_bkizg, the following comment string is sent to the +datasource:

+
SELECT * FROM example_table; /*Query 20230622_180528_00000_bkizg sent by user Jane from Trino.*/
+
+
+
+

Note

+

Certain JDBC driver settings and logging configurations might cause the +comment to be removed.

+
+
+
+

Domain compaction threshold#

+

Pushing down a large list of predicates to the data source can compromise +performance. Trino compacts large predicates into a simpler range predicate +by default to ensure a balance between performance and predicate pushdown. +If necessary, the threshold for this compaction can be increased to improve +performance when the data source is capable of taking advantage of large +predicates. Increasing this threshold may improve pushdown of large +dynamic filters. +The domain-compaction-threshold catalog configuration property or the +domain_compaction_threshold catalog session property can be used to adjust the default value of +5000 for this threshold.

+
+
+

Procedures#

+
    +
  • system.flush_metadata_cache()

    +

    Flush JDBC metadata caches. For example, the following system call +flushes the metadata caches for all schemas in the example catalog

    +
    USE example.example_schema;
    +CALL system.flush_metadata_cache();
    +
    +
    +
  • +
+
+
+

Case insensitive matching#

+

When case-insensitive-name-matching is set to true, Trino +is able to query non-lowercase schemas and tables by maintaining a mapping of +the lowercase name to the actual name in the remote system. However, if two +schemas and/or tables have names that differ only in case (such as “customers” +and “Customers”) then Trino fails to query them due to ambiguity.

+

In these cases, use the case-insensitive-name-matching.config-file catalog +configuration property to specify a configuration file that maps these remote +schemas/tables to their respective Trino schemas/tables:

+
{
+  "schemas": [
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "mapping": "case_insensitive_1"
+    },
+    {
+      "remoteSchema": "cASEsENSITIVEnAME",
+      "mapping": "case_insensitive_2"
+    }],
+  "tables": [
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "remoteTable": "tablex",
+      "mapping": "table_1"
+    },
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "remoteTable": "TABLEX",
+      "mapping": "table_2"
+    }]
+}
+
+
+

Queries against one of the tables or schemes defined in the mapping +attributes are run against the corresponding remote entity. For example, a query +against tables in the case_insensitive_1 schema is forwarded to the +CaseSensitiveName schema and a query against case_insensitive_2 is forwarded +to the cASEsENSITIVEnAME schema.

+

At the table mapping level, a query on case_insensitive_1.table_1 as +configured above is forwarded to CaseSensitiveName.tablex, and a query on +case_insensitive_1.table_2 is forwarded to CaseSensitiveName.TABLEX.

+

By default, when a change is made to the mapping configuration file, Trino must +be restarted to load the changes. Optionally, you can set the +case-insensitive-name-mapping.refresh-period to have Trino refresh the +properties without requiring a restart:

+
case-insensitive-name-mapping.refresh-period=30s
+
+
+
+
+

Non-transactional INSERT#

+

The connector supports adding rows using INSERT statements. +By default, data insertion is performed by writing data to a temporary table. +You can skip this step to improve performance and write directly to the target +table. Set the insert.non-transactional-insert.enabled catalog property +or the corresponding non_transactional_insert catalog session property to +true.

+

Note that with this property enabled, data can be corrupted in rare cases where +exceptions occur during the insert operation. With transactions disabled, no +rollback can be performed.

+
+
+
+

Querying Phoenix tables#

+

The default empty schema in Phoenix maps to a schema named default in Trino. +You can see the available Phoenix schemas by running SHOW SCHEMAS:

+
SHOW SCHEMAS FROM example;
+
+
+

If you have a Phoenix schema named web, you can view the tables +in this schema by running SHOW TABLES:

+
SHOW TABLES FROM example.web;
+
+
+

You can see a list of the columns in the clicks table in the web schema +using either of the following:

+
DESCRIBE example.web.clicks;
+SHOW COLUMNS FROM example.web.clicks;
+
+
+

Finally, you can access the clicks table in the web schema:

+
SELECT * FROM example.web.clicks;
+
+
+

If you used a different name for your catalog properties file, use +that catalog name instead of example in the above examples.

+
+
+

Type mapping#

+

Because Trino and Phoenix each support types that the other does not, this +connector modifies some types when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction.

+
+

Phoenix type to Trino type mapping#

+

The connector maps Phoenix types to the corresponding Trino types following this +table:

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Phoenix type to Trino type mapping#

Phoenix database type

Trino type

BOOLEAN

BOOLEAN

TINYINT

TINYINT

UNSIGNED_TINYINT

TINYINT

SMALLINT

SMALLINT

UNSIGNED_SMALLINT

SMALLINT

INTEGER

INTEGER

UNSIGNED_INT

INTEGER

BIGINT

BIGINT

UNSIGNED_LONG

BIGINT

FLOAT

REAL

UNSIGNED_FLOAT

REAL

DOUBLE

DOUBLE

UNSIGNED_DOUBLE

DOUBLE

DECIMAL(p,s)

DECIMAL(p,s)

CHAR(n)

CHAR(n)

VARCHAR(n)

VARCHAR(n)

BINARY

VARBINARY

VARBINARY

VARBINARY

DATE

DATE

UNSIGNED_DATE

DATE

ARRAY

ARRAY

+

No other types are supported.

+
+
+

Trino type to Phoenix type mapping#

+

The Phoenix fixed length BINARY data type is mapped to the Trino variable +length VARBINARY data type. There is no way to create a Phoenix table in +Trino that uses the BINARY data type, as Trino does not have an equivalent +type.

+

The connector maps Trino types to the corresponding Phoenix types following this +table:

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Trino type to Phoenix type mapping#

Trino database type

Phoenix type

BOOLEAN

BOOLEAN

TINYINT

TINYINT

SMALLINT

SMALLINT

INTEGER

INTEGER

BIGINT

BIGINT

REAL

FLOAT

DOUBLE

DOUBLE

DECIMAL(p,s)

DECIMAL(p,s)

CHAR(n)

CHAR(n)

VARCHAR(n)

VARCHAR(n)

VARBINARY

VARBINARY

TIME

TIME

DATE

DATE

ARRAY

ARRAY

+

No other types are supported.

+
+
+

Decimal type handling#

+

DECIMAL types with unspecified precision or scale are mapped to a Trino +DECIMAL with a default precision of 38 and default scale of 0. The scale can +be changed by setting the decimal-mapping configuration property or the +decimal_mapping session property to allow_overflow. The scale of the +resulting type is controlled via the decimal-default-scale configuration +property or the decimal-rounding-mode session property. The precision is +always 38.

+

By default, values that require rounding or truncation to fit will cause a +failure at runtime. This behavior is controlled via the +decimal-rounding-mode configuration property or the +decimal_rounding_mode session property, which can be set to UNNECESSARY +(the default), UP, DOWN, CEILING, FLOOR, HALF_UP, +HALF_DOWN, or HALF_EVEN (see RoundingMode).

+
+
+

Type mapping configuration properties#

+

The following properties can be used to configure how data types from the +connected data source are mapped to Trino data types and how the metadata is +cached in Trino.

+ +++++ + + + + + + + + + + + + + + + + +

Property name

Description

Default value

unsupported-type-handling

Configure how unsupported column data types are handled:

+
    +
  • IGNORE, column is not accessible.

  • +
  • CONVERT_TO_VARCHAR, column is converted to unbounded VARCHAR.

  • +
+

The respective catalog session property is unsupported_type_handling.

+

IGNORE

jdbc-types-mapped-to-varchar

Allow forced mapping of comma separated lists of data types to convert to +unbounded VARCHAR

+
+
+
+

Table properties - Phoenix#

+

Table property usage example:

+
CREATE TABLE example_schema.scientists (
+  recordkey VARCHAR,
+  birthday DATE,
+  name VARCHAR,
+  age BIGINT
+)
+WITH (
+  rowkeys = 'recordkey,birthday',
+  salt_buckets = 10
+);
+
+
+

The following are supported Phoenix table properties from https://phoenix.apache.org/language/index.html#options

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Default value

Description

rowkeys

ROWKEY

Comma-separated list of primary key columns. See further description below

split_on

(none)

List of keys to presplit the table on. See Split Point.

salt_buckets

(none)

Number of salt buckets for this table.

disable_wal

false

Whether to disable WAL writes in HBase for this table.

immutable_rows

false

Declares whether this table has rows which are write-once, append-only.

default_column_family

0

Default column family name to use for this table.

+
+

rowkeys#

+

This is a comma-separated list of columns to be used as the table’s primary key. If not specified, a BIGINT primary key column named ROWKEY is generated +, as well as a sequence with the same name as the table suffixed with _seq (i.e. <schema>.<table>_seq) +, which is used to automatically populate the ROWKEY for each row during insertion.

+
+
+
+

Table properties - HBase#

+

The following are the supported HBase table properties that are passed through by Phoenix during table creation. +Use them in the same way as above: in the WITH clause of the CREATE TABLE statement.

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Default value

Description

versions

1

The maximum number of versions of each cell to keep.

min_versions

0

The minimum number of cell versions to keep.

compression

NONE

Compression algorithm to use. Valid values are NONE (default), SNAPPY, LZO, LZ4, or GZ.

data_block_encoding

FAST_DIFF

Block encoding algorithm to use. Valid values are: NONE, PREFIX, DIFF, FAST_DIFF (default), or ROW_INDEX_V1.

ttl

FOREVER

Time To Live for each cell.

bloomfilter

NONE

Bloomfilter to use. Valid values are NONE (default), ROW, or ROWCOL.

+
+
+

SQL support#

+

The connector provides read and write access to data and metadata in +Phoenix. In addition to the globally available and read operation +statements, the connector supports the following features:

+ +
+

SQL DELETE#

+

If a WHERE clause is specified, the DELETE operation only works if the +predicate in the clause can be fully pushed down to the data source.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/pinot.html b/430/connector/pinot.html new file mode 100644 index 000000000..2d3386713 --- /dev/null +++ b/430/connector/pinot.html @@ -0,0 +1,1050 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Pinot connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Pinot connector#

+

The Pinot connector allows Trino to query data stored in +Apache Pinot™.

+
+

Requirements#

+

To connect to Pinot, you need:

+
    +
  • Pinot 0.11.0 or higher.

  • +
  • Network access from the Trino coordinator and workers to the Pinot controller +nodes. Port 8098 is the default port.

  • +
+
+
+

Configuration#

+

To configure the Pinot connector, create a catalog properties file +e.g. etc/catalog/example.properties with at least the following contents:

+
connector.name=pinot
+pinot.controller-urls=host1:8098,host2:8098
+
+
+

Replace host1:8098,host2:8098 with a comma-separated list of Pinot controller nodes. +This can be the ip or the FDQN, the url scheme (http://) is optional.

+
+
+

Configuration properties#

+
+

General configuration properties#

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Required

Description

pinot.controller-urls

Yes

A comma separated list of controller hosts. If Pinot is deployed via Kubernetes this needs to point to the controller service endpoint. The Pinot broker and server must be accessible via DNS as Pinot returns hostnames and not IP addresses.

pinot.connection-timeout

No

Pinot connection timeout, default is 15s.

pinot.metadata-expiry

No

Pinot metadata expiration time, default is 2m.

pinot.controller.authentication.type

No

Pinot authentication method for controller requests. Allowed values are NONE and PASSWORD - defaults to NONE which is no authentication.

pinot.controller.authentication.user

No

Controller username for basic authentication method.

pinot.controller.authentication.password

No

Controller password for basic authentication method.

pinot.broker.authentication.type

No

Pinot authentication method for broker requests. Allowed values are NONE and PASSWORD - defaults to NONE which is no authentication.

pinot.broker.authentication.user

No

Broker username for basic authentication method.

pinot.broker.authentication.password

No

Broker password for basic authentication method.

pinot.max-rows-per-split-for-segment-queries

No

Fail query if Pinot server split returns more rows than configured, default to 50,000 for non-gRPC connection, 2,147,483,647 for gRPC connection.

pinot.estimated-size-in-bytes-for-non-numeric-column

No

Estimated byte size for non-numeric column for page pre-allocation in non-gRPC connection, default is 20.

pinot.prefer-broker-queries

No

Pinot query plan prefers to query Pinot broker, default is true.

pinot.forbid-segment-queries

No

Forbid parallel querying and force all querying to happen via the broker, default is false.

pinot.segments-per-split

No

The number of segments processed in a split. Setting this higher reduces the number of requests made to Pinot. This is useful for smaller Pinot clusters, default is 1.

pinot.fetch-retry-count

No

Retry count for retriable Pinot data fetch calls, default is 2.

pinot.non-aggregate-limit-for-broker-queries

No

Max limit for non aggregate queries to the Pinot broker, default is 25,000.

pinot.max-rows-for-broker-queries

No

Max rows for a broker query can return, default is 50,000.

pinot.aggregation-pushdown.enabled

No

Push down aggregation queries, default is true.

pinot.count-distinct-pushdown.enabled

No

Push down count distinct queries to Pinot, default is true.

pinot.target-segment-page-size

No

Max allowed page size for segment query, default is 1MB.

pinot.proxy.enabled

No

Use Pinot Proxy for controller and broker requests, default is false.

+

If pinot.controller.authentication.type is set to PASSWORD then both pinot.controller.authentication.user and +pinot.controller.authentication.password are required.

+

If pinot.broker.authentication.type is set to PASSWORD then both pinot.broker.authentication.user and +pinot.broker.authentication.password are required.

+

If pinot.controller-urls uses https scheme then TLS is enabled for all connections including brokers.

+
+
+

gRPC configuration properties#

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Required

Description

pinot.grpc.enabled

No

Use gRPC endpoint for Pinot server queries, default is true.

pinot.grpc.port

No

Pinot gRPC port, default to 8090.

pinot.grpc.max-inbound-message-size

No

Max inbound message bytes when init gRPC client, default is 128MB.

pinot.grpc.use-plain-text

No

Use plain text for gRPC communication, default to true.

pinot.grpc.tls.keystore-type

No

TLS keystore type for gRPC connection, default is JKS.

pinot.grpc.tls.keystore-path

No

TLS keystore file location for gRPC connection, default is empty.

pinot.grpc.tls.keystore-password

No

TLS keystore password, default is empty.

pinot.grpc.tls.truststore-type

No

TLS truststore type for gRPC connection, default is JKS.

pinot.grpc.tls.truststore-path

No

TLS truststore file location for gRPC connection, default is empty.

pinot.grpc.tls.truststore-password

No

TLS truststore password, default is empty.

pinot.grpc.tls.ssl-provider

No

SSL provider, default is JDK.

pinot.grpc.proxy-uri

No

Pinot Rest Proxy gRPC endpoint URI, default is null.

+

For more Apache Pinot TLS configurations, please also refer to Configuring TLS/SSL.

+

You can use secrets to avoid actual values in the catalog properties files.

+
+
+
+

Querying Pinot tables#

+

The Pinot connector automatically exposes all tables in the default schema of the catalog. +You can list all tables in the pinot catalog with the following query:

+
SHOW TABLES FROM example.default;
+
+
+

You can list columns in the flight_status table:

+
DESCRIBE example.default.flight_status;
+SHOW COLUMNS FROM example.default.flight_status;
+
+
+

Queries written with SQL are fully supported and can include filters and limits:

+
SELECT foo
+FROM pinot_table
+WHERE bar = 3 AND baz IN ('ONE', 'TWO', 'THREE')
+LIMIT 25000;
+
+
+
+
+

Dynamic tables#

+

To leverage Pinot’s fast aggregation, a Pinot query written in PQL can be used as the table name. +Filters and limits in the outer query are pushed down to Pinot. +Let’s look at an example query:

+
SELECT *
+FROM example.default."SELECT MAX(col1), COUNT(col2) FROM pinot_table GROUP BY col3, col4"
+WHERE col3 IN ('FOO', 'BAR') AND col4 > 50
+LIMIT 30000
+
+
+

Filtering and limit processing is pushed down to Pinot.

+

The queries are routed to the broker and are more suitable to aggregate queries.

+

For SELECT queries without aggregates it is more performant to issue a regular SQL query. +Processing is routed directly to the servers that store the data.

+

The above query is translated to the following Pinot PQL query:

+
SELECT MAX(col1), COUNT(col2)
+FROM pinot_table
+WHERE col3 IN('FOO', 'BAR') and col4 > 50
+TOP 30000
+
+
+
+
+

Type mapping#

+

Because Trino and Pinot each support types that the other does not, this +connector maps some types when reading data.

+
+

Pinot type to Trino type mapping#

+

The connector maps Pinot types to the corresponding Trino types +according to the following table:

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Pinot type to Trino type mapping#

Pinot type

Trino type

INT

INTEGER

LONG

BIGINT

FLOAT

REAL

DOUBLE

DOUBLE

STRING

VARCHAR

BYTES

VARBINARY

JSON

JSON

TIMESTAMP

TIMESTAMP

INT_ARRAY

VARCHAR

LONG_ARRAY

VARCHAR

FLOAT_ARRAY

VARCHAR

DOUBLE_ARRAY

VARCHAR

STRING_ARRAY

VARCHAR

+

Pinot does not allow null values in any data type.

+

No other types are supported.

+
+
+
+

SQL support#

+

The connector provides globally available and +read operation statements to access data and +metadata in Pinot.

+
+
+

Pushdown#

+

The connector supports pushdown for a number of operations:

+ +

Aggregate pushdown for the following functions:

+ +

Aggregate function pushdown is enabled by default, but can be disabled with the +catalog property pinot.aggregation-pushdown.enabled or the catalog session +property aggregation_pushdown_enabled.

+

A count(distint) pushdown may cause Pinot to run a full table scan with +significant performance impact. If you encounter this problem, you can disable +it with the catalog property pinot.count-distinct-pushdown.enabled or the +catalog session property count_distinct_pushdown_enabled.

+
+

Note

+

The connector performs pushdown where performance may be improved, but in +order to preserve correctness an operation may not be pushed down. When +pushdown of an operation may result in better performance but risks +correctness, the connector prioritizes correctness.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/postgresql.html b/430/connector/postgresql.html new file mode 100644 index 000000000..fe6724c26 --- /dev/null +++ b/430/connector/postgresql.html @@ -0,0 +1,1722 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + PostgreSQL connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

PostgreSQL connector#

+

The PostgreSQL connector allows querying and creating tables in an +external PostgreSQL database. This can be used to join data between +different systems like PostgreSQL and Hive, or between different +PostgreSQL instances.

+
+

Requirements#

+

To connect to PostgreSQL, you need:

+
    +
  • PostgreSQL 11.x or higher.

  • +
  • Network access from the Trino coordinator and workers to PostgreSQL. +Port 5432 is the default port.

  • +
+
+
+

Configuration#

+

The connector can query a database on a PostgreSQL server. Create a catalog +properties file that specifies the PostgreSQL connector by setting the +connector.name to postgresql.

+

For example, to access a database as the example catalog, create the file +etc/catalog/example.properties. Replace the connection properties as +appropriate for your setup:

+
connector.name=postgresql
+connection-url=jdbc:postgresql://example.net:5432/database
+connection-user=root
+connection-password=secret
+
+
+

The connection-url defines the connection information and parameters to pass +to the PostgreSQL JDBC driver. The parameters for the URL are available in the +PostgreSQL JDBC driver documentation. +Some parameters can have adverse effects on the connector behavior or not work +with the connector.

+

The connection-user and connection-password are typically required and +determine the user credentials for the connection, often a service user. You can +use secrets to avoid actual values in the catalog +properties files.

+
+

Access to system tables#

+

The PostgreSQL connector supports reading PostgreSQ catalog +tables, such as +pg_namespace. The functionality is turned off by default, and can be enabled +using the postgresql.include-system-tables configuration property.

+

You can see more details in the pg_catalog schema in the example catalog, +for example about the pg_namespace system table:

+
SHOW TABLES FROM example.pg_catalog;
+SELECT * FROM example.pg_catalog.pg_namespace;
+
+
+
+
+

Connection security#

+

If you have TLS configured with a globally-trusted certificate installed on your +data source, you can enable TLS between your cluster and the data +source by appending a parameter to the JDBC connection string set in the +connection-url catalog configuration property.

+

For example, with version 42 of the PostgreSQL JDBC driver, enable TLS by +appending the ssl=true parameter to the connection-url configuration +property:

+
connection-url=jdbc:postgresql://example.net:5432/database?ssl=true
+
+
+

For more information on TLS configuration options, see the PostgreSQL JDBC +driver documentation.

+
+
+

Data source authentication#

+

The connector can provide credentials for the data source connection +in multiple ways:

+
    +
  • inline, in the connector configuration file

  • +
  • in a separate properties file

  • +
  • in a key store file

  • +
  • as extra credentials set when connecting to Trino

  • +
+

You can use secrets to avoid storing sensitive +values in the catalog properties files.

+

The following table describes configuration properties +for connection credentials:

+ ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

credential-provider.type

Type of the credential provider. Must be one of INLINE, FILE, or +KEYSTORE; defaults to INLINE.

connection-user

Connection user name.

connection-password

Connection password.

user-credential-name

Name of the extra credentials property, whose value to use as the user +name. See extraCredentials in Parameter +reference.

password-credential-name

Name of the extra credentials property, whose value to use as the +password.

connection-credential-file

Location of the properties file where credentials are present. It must +contain the connection-user and connection-password properties.

keystore-file-path

The location of the Java Keystore file, from which to read credentials.

keystore-type

File format of the keystore file, for example JKS or PEM.

keystore-password

Password for the key store.

keystore-user-credential-name

Name of the key store entity to use as the user name.

keystore-user-credential-password

Password for the user name key store entity.

keystore-password-credential-name

Name of the key store entity to use as the password.

keystore-password-credential-password

Password for the password key store entity.

+
+
+

Multiple PostgreSQL databases or servers#

+

The PostgreSQL connector can only access a single database within +a PostgreSQL server. Thus, if you have multiple PostgreSQL databases, +or want to connect to multiple PostgreSQL servers, you must configure +multiple instances of the PostgreSQL connector.

+

To add another catalog, simply add another properties file to etc/catalog +with a different name, making sure it ends in .properties. For example, +if you name the property file sales.properties, Trino creates a +catalog named sales using the configured connector.

+
+
+

General configuration properties#

+

The following table describes general catalog configuration properties for the +connector:

+ +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

Default value

case-insensitive-name-matching

Support case insensitive schema and table names.

false

case-insensitive-name-matching.cache-ttl

This value should be a duration.

1m

case-insensitive-name-matching.config-file

Path to a name mapping configuration file in JSON format that allows +Trino to disambiguate between schemas and tables with similar names in +different cases.

null

case-insensitive-name-matching.config-file.refresh-period

Frequency with which Trino checks the name matching configuration file +for changes. This value should be a duration.

(refresh disabled)

metadata.cache-ttl

The duration for which metadata, including +table and column statistics, is cached.

0s (caching disabled)

metadata.cache-missing

Cache the fact that metadata, including table and column statistics, is +not available

false

metadata.cache-maximum-size

Maximum number of objects stored in the metadata cache

10000

write.batch-size

Maximum number of statements in a batched execution. +Do not change this setting from the default. Non-default values may +negatively impact performance.

1000

dynamic-filtering.enabled

Push down dynamic filters into JDBC queries

true

dynamic-filtering.wait-timeout

Maximum duration for which Trino will wait for dynamic +filters to be collected from the build side of joins before starting a +JDBC query. Using a large timeout can potentially result in more detailed +dynamic filters. However, it can also increase latency for some queries.

20s

+
+
+

Appending query metadata#

+

The optional parameter query.comment-format allows you to configure a SQL +comment that is sent to the datasource with each query. The format of this +comment can contain any characters and the following metadata:

+
    +
  • $QUERY_ID: The identifier of the query.

  • +
  • $USER: The name of the user who submits the query to Trino.

  • +
  • $SOURCE: The identifier of the client tool used to submit the query, for +example trino-cli.

  • +
  • $TRACE_TOKEN: The trace token configured with the client tool.

  • +
+

The comment can provide more context about the query. This additional +information is available in the logs of the datasource. To include environment +variables from the Trino cluster with the comment , use the +${ENV:VARIABLE-NAME} syntax.

+

The following example sets a simple comment that identifies each query sent by +Trino:

+
query.comment-format=Query sent by Trino.
+
+
+

With this configuration, a query such as SELECT * FROM example_table; is +sent to the datasource with the comment appended:

+
SELECT * FROM example_table; /*Query sent by Trino.*/
+
+
+

The following example improves on the preceding example by using metadata:

+
query.comment-format=Query $QUERY_ID sent by user $USER from Trino.
+
+
+

If Jane sent the query with the query identifier +20230622_180528_00000_bkizg, the following comment string is sent to the +datasource:

+
SELECT * FROM example_table; /*Query 20230622_180528_00000_bkizg sent by user Jane from Trino.*/
+
+
+
+

Note

+

Certain JDBC driver settings and logging configurations might cause the +comment to be removed.

+
+
+
+

Domain compaction threshold#

+

Pushing down a large list of predicates to the data source can compromise +performance. Trino compacts large predicates into a simpler range predicate +by default to ensure a balance between performance and predicate pushdown. +If necessary, the threshold for this compaction can be increased to improve +performance when the data source is capable of taking advantage of large +predicates. Increasing this threshold may improve pushdown of large +dynamic filters. +The domain-compaction-threshold catalog configuration property or the +domain_compaction_threshold catalog session property can be used to adjust the default value of +32 for this threshold.

+
+
+

Procedures#

+
    +
  • system.flush_metadata_cache()

    +

    Flush JDBC metadata caches. For example, the following system call +flushes the metadata caches for all schemas in the example catalog

    +
    USE example.example_schema;
    +CALL system.flush_metadata_cache();
    +
    +
    +
  • +
+
+
+

Case insensitive matching#

+

When case-insensitive-name-matching is set to true, Trino +is able to query non-lowercase schemas and tables by maintaining a mapping of +the lowercase name to the actual name in the remote system. However, if two +schemas and/or tables have names that differ only in case (such as “customers” +and “Customers”) then Trino fails to query them due to ambiguity.

+

In these cases, use the case-insensitive-name-matching.config-file catalog +configuration property to specify a configuration file that maps these remote +schemas/tables to their respective Trino schemas/tables:

+
{
+  "schemas": [
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "mapping": "case_insensitive_1"
+    },
+    {
+      "remoteSchema": "cASEsENSITIVEnAME",
+      "mapping": "case_insensitive_2"
+    }],
+  "tables": [
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "remoteTable": "tablex",
+      "mapping": "table_1"
+    },
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "remoteTable": "TABLEX",
+      "mapping": "table_2"
+    }]
+}
+
+
+

Queries against one of the tables or schemes defined in the mapping +attributes are run against the corresponding remote entity. For example, a query +against tables in the case_insensitive_1 schema is forwarded to the +CaseSensitiveName schema and a query against case_insensitive_2 is forwarded +to the cASEsENSITIVEnAME schema.

+

At the table mapping level, a query on case_insensitive_1.table_1 as +configured above is forwarded to CaseSensitiveName.tablex, and a query on +case_insensitive_1.table_2 is forwarded to CaseSensitiveName.TABLEX.

+

By default, when a change is made to the mapping configuration file, Trino must +be restarted to load the changes. Optionally, you can set the +case-insensitive-name-mapping.refresh-period to have Trino refresh the +properties without requiring a restart:

+
case-insensitive-name-mapping.refresh-period=30s
+
+
+
+
+

Non-transactional INSERT#

+

The connector supports adding rows using INSERT statements. +By default, data insertion is performed by writing data to a temporary table. +You can skip this step to improve performance and write directly to the target +table. Set the insert.non-transactional-insert.enabled catalog property +or the corresponding non_transactional_insert catalog session property to +true.

+

Note that with this property enabled, data can be corrupted in rare cases where +exceptions occur during the insert operation. With transactions disabled, no +rollback can be performed.

+
+
+
+

Type mapping#

+

Because Trino and PostgreSQL each support types that the other does not, this +connector modifies some types when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction.

+
+

PostgreSQL type to Trino type mapping#

+

The connector maps PostgreSQL types to the corresponding Trino types following +this table:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
PostgreSQL type to Trino type mapping#

PostgreSQL type

Trino type

Notes

BIT

BOOLEAN

BOOLEAN

BOOLEAN

SMALLINT

SMALLINT

INTEGER

INTEGER

BIGINT

BIGINT

REAL

REAL

DOUBLE

DOUBLE

NUMERIC(p, s)

DECIMAL(p, s)

DECIMAL(p, s) is an alias of NUMERIC(p, s). See +Decimal type handling for more information.

CHAR(n)

CHAR(n)

VARCHAR(n)

VARCHAR(n)

ENUM

VARCHAR

BYTEA

VARBINARY

DATE

DATE

TIME(n)

TIME(n)

TIMESTAMP(n)

TIMESTAMP(n)

TIMESTAMPTZ(n)

TIMESTAMP(n) WITH TIME ZONE

MONEY

VARCHAR

UUID

UUID

JSON

JSON

JSONB

JSON

HSTORE

MAP(VARCHAR, VARCHAR)

ARRAY

Disabled, ARRAY, or JSON

See Array type handling for more information.

+

No other types are supported.

+
+
+

Trino type to PostgreSQL type mapping#

+

The connector maps Trino types to the corresponding PostgreSQL types following +this table:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Trino type to PostgreSQL type mapping#

Trino type

PostgreSQL type

Notes

BOOLEAN

BOOLEAN

SMALLINT

SMALLINT

TINYINT

SMALLINT

INTEGER

INTEGER

BIGINT

BIGINT

DOUBLE

DOUBLE

DECIMAL(p, s)

NUMERIC(p, s)

DECIMAL(p, s) is an alias of NUMERIC(p, s). See +Decimal type handling for more information.

CHAR(n)

CHAR(n)

VARCHAR(n)

VARCHAR(n)

VARBINARY

BYTEA

DATE

DATE

TIME(n)

TIME(n)

TIMESTAMP(n)

TIMESTAMP(n)

TIMESTAMP(n) WITH TIME ZONE

TIMESTAMPTZ(n)

UUID

UUID

JSON

JSONB

ARRAY

ARRAY

See Array type handling for more information.

+

No other types are supported.

+
+
+

Decimal type handling#

+

DECIMAL types with unspecified precision or scale are mapped to a Trino +DECIMAL with a default precision of 38 and default scale of 0. The scale can +be changed by setting the decimal-mapping configuration property or the +decimal_mapping session property to allow_overflow. The scale of the +resulting type is controlled via the decimal-default-scale configuration +property or the decimal-rounding-mode session property. The precision is +always 38.

+

By default, values that require rounding or truncation to fit will cause a +failure at runtime. This behavior is controlled via the +decimal-rounding-mode configuration property or the +decimal_rounding_mode session property, which can be set to UNNECESSARY +(the default), UP, DOWN, CEILING, FLOOR, HALF_UP, +HALF_DOWN, or HALF_EVEN (see RoundingMode).

+
+
+

Array type handling#

+

The PostgreSQL array implementation does not support fixed dimensions whereas Trino +support only arrays with fixed dimensions. +You can configure how the PostgreSQL connector handles arrays with the postgresql.array-mapping configuration property in your catalog file +or the array_mapping session property. +The following values are accepted for this property:

+
    +
  • DISABLED (default): array columns are skipped.

  • +
  • AS_ARRAY: array columns are interpreted as Trino ARRAY type, for array columns with fixed dimensions.

  • +
  • AS_JSON: array columns are interpreted as Trino JSON type, with no constraint on dimensions.

  • +
+
+
+

Type mapping configuration properties#

+

The following properties can be used to configure how data types from the +connected data source are mapped to Trino data types and how the metadata is +cached in Trino.

+ +++++ + + + + + + + + + + + + + + + + +

Property name

Description

Default value

unsupported-type-handling

Configure how unsupported column data types are handled:

+
    +
  • IGNORE, column is not accessible.

  • +
  • CONVERT_TO_VARCHAR, column is converted to unbounded VARCHAR.

  • +
+

The respective catalog session property is unsupported_type_handling.

+

IGNORE

jdbc-types-mapped-to-varchar

Allow forced mapping of comma separated lists of data types to convert to +unbounded VARCHAR

+
+
+
+

Querying PostgreSQL#

+

The PostgreSQL connector provides a schema for every PostgreSQL schema. +You can see the available PostgreSQL schemas by running SHOW SCHEMAS:

+
SHOW SCHEMAS FROM example;
+
+
+

If you have a PostgreSQL schema named web, you can view the tables +in this schema by running SHOW TABLES:

+
SHOW TABLES FROM example.web;
+
+
+

You can see a list of the columns in the clicks table in the web database +using either of the following:

+
DESCRIBE example.web.clicks;
+SHOW COLUMNS FROM example.web.clicks;
+
+
+

Finally, you can access the clicks table in the web schema:

+
SELECT * FROM example.web.clicks;
+
+
+

If you used a different name for your catalog properties file, use +that catalog name instead of example in the above examples.

+
+
+

SQL support#

+

The connector provides read access and write access to data and metadata in +PostgreSQL. In addition to the globally available and read operation +statements, the connector supports the following features:

+ +
+

UPDATE#

+

Only UPDATE statements with constant assignments and predicates are +supported. For example, the following statement is supported because the values +assigned are constants:

+
UPDATE table SET col1 = 1 WHERE col3 = 1
+
+
+

Arithmetic expressions, function calls, and other non-constant UPDATE +statements are not supported. For example, the following statement is not +supported because arithmetic expressions cannot be used with the SET +command:

+
UPDATE table SET col1 = col2 + 2 WHERE col3 = 1
+
+
+

The =, !=, >, <, >=, <=, IN, NOT IN operators are supported in +predicates. The following statement is not supported because the AND operator +cannot be used in predicates:

+
UPDATE table SET col1 = 1 WHERE col3 = 1 AND col2 = 3
+
+
+

All column values of a table row cannot be updated simultaneously. For a three +column table, the following statement is not supported:

+
UPDATE table SET col1 = 1, col2 = 2, col3 = 3 WHERE col3 = 1
+
+
+
+
+

SQL DELETE#

+

If a WHERE clause is specified, the DELETE operation only works if the +predicate in the clause can be fully pushed down to the data source.

+
+
+

ALTER TABLE RENAME TO#

+

The connector does not support renaming tables across multiple schemas. For +example, the following statement is supported:

+
ALTER TABLE example.schema_one.table_one RENAME TO example.schema_one.table_two
+
+
+

The following statement attempts to rename a table across schemas, and therefore +is not supported:

+
ALTER TABLE example.schema_one.table_one RENAME TO example.schema_two.table_two
+
+
+
+
+

ALTER SCHEMA#

+

The connector supports renaming a schema with the ALTER SCHEMA RENAME +statement. ALTER SCHEMA SET AUTHORIZATION is not supported.

+
+
+
+

Fault-tolerant execution support#

+

The connector supports Fault-tolerant execution of query +processing. Read and write operations are both supported with any retry policy.

+
+
+

Table functions#

+

The connector provides specific table functions to +access PostgreSQL.

+
+

query(varchar) -> table#

+

The query function allows you to query the underlying database directly. It +requires syntax native to PostgreSQL, because the full query is pushed down and +processed in PostgreSQL. This can be useful for accessing native features which +are not available in Trino or for improving query performance in situations +where running a query natively may be faster.

+

The native query passed to the underlying data source is required to return a +table as a result set. Only the data source performs validation or security +checks for these queries using its own configuration. Trino does not perform +these tasks. Only use passthrough queries to read data.

+

As a simple example, query the example catalog and select an entire table:

+
SELECT
+  *
+FROM
+  TABLE(
+    example.system.query(
+      query => 'SELECT
+        *
+      FROM
+        tpch.nation'
+    )
+  );
+
+
+

As a practical example, you can leverage +frame exclusion from PostgresQL +when using window functions:

+
SELECT
+  *
+FROM
+  TABLE(
+    example.system.query(
+      query => 'SELECT
+        *,
+        array_agg(week) OVER (
+          ORDER BY
+            week
+          ROWS
+            BETWEEN 2 PRECEDING
+            AND 2 FOLLOWING
+            EXCLUDE GROUP
+        ) AS week,
+        array_agg(week) OVER (
+          ORDER BY
+            day
+          ROWS
+            BETWEEN 2 PRECEDING
+            AND 2 FOLLOWING
+            EXCLUDE GROUP
+        ) AS all
+      FROM
+        test.time_data'
+    )
+  );
+
+
+
+

Note

+

The query engine does not preserve the order of the results of this +function. If the passed query contains an ORDER BY clause, the +function result may not be ordered as expected.

+
+
+
+
+

Performance#

+

The connector includes a number of performance improvements, detailed in the +following sections.

+
+

Table statistics#

+

The PostgreSQL connector can use table and column statistics for cost based optimizations, to improve query processing performance +based on the actual data in the data source.

+

The statistics are collected by PostgreSQL and retrieved by the connector.

+

To collect statistics for a table, execute the following statement in +PostgreSQL.

+
ANALYZE table_schema.table_name;
+
+
+

Refer to PostgreSQL documentation for additional ANALYZE options.

+
+
+

Pushdown#

+

The connector supports pushdown for a number of operations:

+ +

Aggregate pushdown for the following functions:

+ +
+

Note

+

The connector performs pushdown where performance may be improved, but in +order to preserve correctness an operation may not be pushed down. When +pushdown of an operation may result in better performance but risks +correctness, the connector prioritizes correctness.

+
+
+

Cost-based join pushdown#

+

The connector supports cost-based Join pushdown to make intelligent +decisions about whether to push down a join operation to the data source.

+

When cost-based join pushdown is enabled, the connector only pushes down join +operations if the available Table statistics suggest that doing so +improves performance. Note that if no table statistics are available, join +operation pushdown does not occur to avoid a potential decrease in query +performance.

+

The following table describes catalog configuration properties for +join pushdown:

+ +++++ + + + + + + + + + + + + + + + + +

Property name

Description

Default value

join-pushdown.enabled

Enable join pushdown. Equivalent catalog +session property is +join_pushdown_enabled.

true

join-pushdown.strategy

Strategy used to evaluate whether join operations are pushed down. Set to +AUTOMATIC to enable cost-based join pushdown, or EAGER to +push down joins whenever possible. Note that EAGER can push down joins +even when table statistics are unavailable, which may result in degraded +query performance. Because of this, EAGER is only recommended for +testing and troubleshooting purposes.

AUTOMATIC

+
+
+
+

Predicate pushdown support#

+

Predicates are pushed down for most types, including UUID and temporal +types, such as DATE.

+

The connector does not support pushdown of range predicates, such as >, +<, or BETWEEN, on columns with character string types like CHAR or VARCHAR. Equality predicates, such as +IN or =, and inequality predicates, such as != on columns with +textual types are pushed down. This ensures correctness of results since the +remote data source may sort strings differently than Trino.

+

In the following example, the predicate of the first query is not pushed down +since name is a column of type VARCHAR and > is a range predicate. +The other queries are pushed down.

+
-- Not pushed down
+SELECT * FROM nation WHERE name > 'CANADA';
+-- Pushed down
+SELECT * FROM nation WHERE name != 'CANADA';
+SELECT * FROM nation WHERE name = 'CANADA';
+
+
+

There is experimental support to enable pushdown of range predicates on columns +with character string types which can be enabled by setting the +postgresql.experimental.enable-string-pushdown-with-collate catalog +configuration property or the corresponding +enable_string_pushdown_with_collate session property to true. +Enabling this configuration will make the predicate of all the queries in the +above example get pushed down.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/prometheus.html b/430/connector/prometheus.html new file mode 100644 index 000000000..867e317e4 --- /dev/null +++ b/430/connector/prometheus.html @@ -0,0 +1,836 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Prometheus connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Prometheus connector#

+

The Prometheus connector allows reading +Prometheus +metrics as tables in Trino.

+

The mechanism for querying Prometheus is to use the Prometheus HTTP API. Specifically, all queries are resolved to Prometheus Instant queries +with a form like: http://localhost:9090/api/v1/query?query=up[21d]&time=1568229904.000. +In this case the up metric is taken from the Trino query table name, 21d is the duration of the query. The Prometheus time value +corresponds to the TIMESTAMP field. Trino queries are translated from their use of the TIMESTAMP field to a duration and time value +as needed. Trino splits are generated by dividing the query range into attempted equal chunks.

+
+

Requirements#

+

To query Prometheus, you need:

+
    +
  • Network access from the Trino coordinator and workers to the Prometheus +server. The default port is 9090.

  • +
  • Prometheus version 2.15.1 or later.

  • +
+
+
+

Configuration#

+

Create etc/catalog/example.properties to mount the Prometheus connector as +the example catalog, replacing the properties as appropriate:

+
connector.name=prometheus
+prometheus.uri=http://localhost:9090
+prometheus.query.chunk.size.duration=1d
+prometheus.max.query.range.duration=21d
+prometheus.cache.ttl=30s
+prometheus.bearer.token.file=/path/to/bearer/token/file
+prometheus.read-timeout=10s
+
+
+
+
+

Configuration properties#

+

The following configuration properties are available:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

prometheus.uri

Where to find Prometheus coordinator host

prometheus.query.chunk.size.duration

The duration of each query to Prometheus

prometheus.max.query.range.duration

Width of overall query to Prometheus, will be divided into query-chunk-size-duration queries

prometheus.cache.ttl

How long values from this config file are cached

prometheus.auth.user

Username for basic authentication

prometheus.auth.password

Password for basic authentication

prometheus.bearer.token.file

File holding bearer token if needed for access to Prometheus

prometheus.read-timeout

How much time a query to Prometheus has before timing out

prometheus.case-insensitive-name-matching

Match Prometheus metric names case insensitively. Defaults to false

+
+
+

Not exhausting your Trino available heap#

+

The prometheus.query.chunk.size.duration and prometheus.max.query.range.duration are values to protect Trino from +too much data coming back from Prometheus. The prometheus.max.query.range.duration is the item of +particular interest.

+

On a Prometheus instance that has been running for awhile and depending +on data retention settings, 21d might be far too much. Perhaps 1h might be a more reasonable setting. +In the case of 1h it might be then useful to set prometheus.query.chunk.size.duration to 10m, dividing the +query window into 6 queries each of which can be handled in a Trino split.

+

Primarily query issuers can limit the amount of data returned by Prometheus by taking +advantage of WHERE clause limits on TIMESTAMP, setting an upper bound and lower bound that define +a relatively small window. For example:

+
SELECT * FROM example.default.up WHERE TIMESTAMP > (NOW() - INTERVAL '10' second);
+
+
+

If the query does not include a WHERE clause limit, these config +settings are meant to protect against an unlimited query.

+
+
+

Bearer token authentication#

+

Prometheus can be setup to require a Authorization header with every query. The value in +prometheus.bearer.token.file allows for a bearer token to be read from the configured file. This file +is optional and not required unless your Prometheus setup requires it.

+
+
+

Type mapping#

+

Because Trino and Prometheus each support types that the other does not, this +connector modifies some types when reading data.

+

The connector returns fixed columns that have a defined mapping to Trino types +according to the following table:

+ + ++++ + + + + + + + + + + + + + + + + +
Prometheus column to Trino type mapping#

Prometheus column

Trino type

labels

MAP(VARCHAR,VARCHAR)

TIMESTAMP

TIMESTAMP(3) WITH TIMEZONE

value

DOUBLE

+

No other types are supported.

+

The following example query result shows how the Prometheus up metric is +represented in Trino:

+
SELECT * FROM example.default.up;
+
+
+
                        labels                         |           timestamp            | value
+--------------------------------------------------------+--------------------------------+-------
+{instance=localhost:9090, job=prometheus, __name__=up} | 2022-09-01 06:18:54.481 +09:00 |   1.0
+{instance=localhost:9090, job=prometheus, __name__=up} | 2022-09-01 06:19:09.446 +09:00 |   1.0
+(2 rows)
+
+
+
+
+

SQL support#

+

The connector provides globally available and +read operation statements to access data and +metadata in Prometheus.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/redis.html b/430/connector/redis.html new file mode 100644 index 000000000..0f1cb98cc --- /dev/null +++ b/430/connector/redis.html @@ -0,0 +1,1464 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Redis connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Redis connector#

+

The Redis connector allows querying of live data stored in Redis. This can be +used to join data between different systems like Redis and Hive.

+

Each Redis key/value pair is presented as a single row in Trino. Rows can be +broken down into cells by using table definition files.

+

Currently, only Redis key of string and zset types are supported, only Redis value of +string and hash types are supported.

+
+

Requirements#

+

Requirements for using the connector in a catalog to connect to a Redis data +source are:

+
    +
  • Redis 2.8.0 or higher (Redis Cluster is not supported)

  • +
  • Network access, by default on port 6379, from the Trino coordinator and +workers to Redis.

  • +
+
+
+

Configuration#

+

To configure the Redis connector, create a catalog properties file +etc/catalog/example.properties with the following content, replacing the +properties as appropriate:

+
connector.name=redis
+redis.table-names=schema1.table1,schema1.table2
+redis.nodes=host:port
+
+
+
+

Multiple Redis servers#

+

You can have as many catalogs as you need. If you have additional +Redis servers, simply add another properties file to etc/catalog +with a different name, making sure it ends in .properties.

+
+
+
+

Configuration properties#

+

The following configuration properties are available:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

redis.table-names

List of all tables provided by the catalog

redis.default-schema

Default schema name for tables

redis.nodes

Location of the Redis server

redis.scan-count

Redis parameter for scanning of the keys

redis.max-keys-per-fetch

Get values associated with the specified number of keys in the redis command such as MGET(key…)

redis.key-prefix-schema-table

Redis keys have schema-name:table-name prefix

redis.key-delimiter

Delimiter separating schema_name and table_name if redis.key-prefix-schema-table is used

redis.table-description-dir

Directory containing table description files

redis.table-description-cache-ttl

The cache time for table description files

redis.hide-internal-columns

Controls whether internal columns are part of the table schema or not

redis.database-index

Redis database index

redis.user

Redis server username

redis.password

Redis server password

+
+

redis.table-names#

+

Comma-separated list of all tables provided by this catalog. A table name +can be unqualified (simple name) and is placed into the default schema +(see below), or qualified with a schema name (<schema-name>.<table-name>).

+

For each table defined, a table description file (see below) may +exist. If no table description file exists, the +table only contains internal columns (see below).

+

This property is optional; the connector relies on the table description files +specified in the redis.table-description-dir property.

+
+
+

redis.default-schema#

+

Defines the schema which will contain all tables that were defined without +a qualifying schema name.

+

This property is optional; the default is default.

+
+
+

redis.nodes#

+

The hostname:port pair for the Redis server.

+

This property is required; there is no default.

+

Redis Cluster is not supported.

+
+
+

redis.scan-count#

+

The internal COUNT parameter for the Redis SCAN command when connector is using +SCAN to find keys for the data. This parameter can be used to tune performance +of the Redis connector.

+

This property is optional; the default is 100.

+
+
+

redis.max-keys-per-fetch#

+

The internal number of keys for the Redis MGET command and Pipeline HGETALL command +when connector is using these commands to find values of keys. This parameter can be +used to tune performance of the Redis connector.

+

This property is optional; the default is 100.

+
+
+

redis.key-prefix-schema-table#

+

If true, only keys prefixed with the schema-name:table-name are scanned +for a table, and all other keys are filtered out. If false, all keys are +scanned.

+

This property is optional; the default is false.

+
+
+

redis.key-delimiter#

+

The character used for separating schema-name and table-name when +redis.key-prefix-schema-table is true

+

This property is optional; the default is :.

+
+
+

redis.table-description-dir#

+

References a folder within Trino deployment that holds one or more JSON +files, which must end with .json and contain table description files.

+

Note that the table description files will only be used by the Trino coordinator +node.

+

This property is optional; the default is etc/redis.

+
+
+

redis.table-description-cache-ttl#

+

The Redis connector dynamically loads the table description files after waiting +for the time specified by this property. Therefore, there is no need to update +the redis.table-names property and restart the Trino service when adding, +updating, or deleting a file end with .json to redis.table-description-dir +folder.

+

This property is optional; the default is 5m.

+
+
+

redis.hide-internal-columns#

+

In addition to the data columns defined in a table description file, the +connector maintains a number of additional columns for each table. If +these columns are hidden, they can still be used in queries, but they do not +show up in DESCRIBE <table-name> or SELECT *.

+

This property is optional; the default is true.

+
+
+

redis.database-index#

+

The Redis database to query.

+

This property is optional; the default is 0.

+
+
+

redis.user#

+

The username for Redis server.

+

This property is optional; the default is null.

+
+
+

redis.password#

+

The password for password-protected Redis server.

+

This property is optional; the default is null.

+
+
+
+

Internal columns#

+

For each defined table, the connector maintains the following columns:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Column name

Type

Description

_key

VARCHAR

Redis key.

_value

VARCHAR

Redis value corresponding to the key.

_key_length

BIGINT

Number of bytes in the key.

_value_length

BIGINT

Number of bytes in the value.

_key_corrupt

BOOLEAN

True if the decoder could not decode the key for this row. When true, data columns mapped from the key should be treated as invalid.

_value_corrupt

BOOLEAN

True if the decoder could not decode the message for this row. When true, data columns mapped from the value should be treated as invalid.

+

For tables without a table definition file, the _key_corrupt and +_value_corrupt columns are false.

+
+
+

Table definition files#

+

With the Redis connector it is possible to further reduce Redis key/value pairs into +granular cells, provided the key/value string follows a particular format. This process +defines new columns that can be further queried from Trino.

+

A table definition file consists of a JSON definition for a table. The +name of the file can be arbitrary, but must end in .json.

+
{
+    "tableName": ...,
+    "schemaName": ...,
+    "key": {
+        "dataFormat": ...,
+        "fields": [
+            ...
+        ]
+    },
+    "value": {
+        "dataFormat": ...,
+        "fields": [
+            ...
+       ]
+    }
+}
+
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Field

Required

Type

Description

tableName

required

string

Trino table name defined by this file.

schemaName

optional

string

Schema which will contain the table. If omitted, the default schema name is used.

key

optional

JSON object

Field definitions for data columns mapped to the value key.

value

optional

JSON object

Field definitions for data columns mapped to the value itself.

+

Please refer to the Kafka connector page for the description of the dataFormat as well as various available decoders.

+

In addition to the above Kafka types, the Redis connector supports hash type for the value field which represent data stored in the Redis hash.

+
{
+    "tableName": ...,
+    "schemaName": ...,
+    "value": {
+        "dataFormat": "hash",
+        "fields": [
+            ...
+       ]
+    }
+}
+
+
+
+
+

Type mapping#

+

Because Trino and Redis each support types that the other does not, this +connector maps some types when reading data. Type +mapping depends on the RAW, CSV, JSON, and AVRO file formats.

+
+

Row decoding#

+

A decoder is used to map data to table columns.

+

The connector contains the following decoders:

+
    +
  • raw: Message is not interpreted; ranges of raw message bytes are mapped +to table columns.

  • +
  • csv: Message is interpreted as comma separated message, and fields are +mapped to table columns.

  • +
  • json: Message is parsed as JSON, and JSON fields are mapped to table +columns.

  • +
  • avro: Message is parsed based on an Avro schema, and Avro fields are +mapped to table columns.

  • +
+
+

Note

+

If no table definition file exists for a table, the dummy decoder is +used, which does not expose any columns.

+
+
+

Raw decoder#

+

The raw decoder supports reading of raw byte-based values from message or key, +and converting it into Trino columns.

+

For fields, the following attributes are supported:

+
    +
  • dataFormat - Selects the width of the data type converted.

  • +
  • type - Trino data type. See the following table for a list of supported +data types.

  • +
  • mapping - <start>[:<end>] - Start and end position of bytes to convert +(optional).

  • +
+

The dataFormat attribute selects the number of bytes converted. If absent, +BYTE is assumed. All values are signed.

+

Supported values are:

+
    +
  • BYTE - one byte

  • +
  • SHORT - two bytes (big-endian)

  • +
  • INT - four bytes (big-endian)

  • +
  • LONG - eight bytes (big-endian)

  • +
  • FLOAT - four bytes (IEEE 754 format)

  • +
  • DOUBLE - eight bytes (IEEE 754 format)

  • +
+

The type attribute defines the Trino data type on which the value is mapped.

+

Depending on the Trino type assigned to a column, different values of dataFormat +can be used:

+ ++++ + + + + + + + + + + + + + + + + + + + + + + + + + +

Trino data type

Allowed dataFormat values

BIGINT

BYTE, SHORT, INT, LONG

INTEGER

BYTE, SHORT, INT

SMALLINT

BYTE, SHORT

DOUBLE

DOUBLE, FLOAT

BOOLEAN

BYTE, SHORT, INT, LONG

VARCHAR / VARCHAR(x)

BYTE

+

No other types are supported.

+

The mapping attribute specifies the range of the bytes in a key or message +used for decoding. It can be one or two numbers separated by a colon +(<start>[:<end>]).

+

If only a start position is given:

+
    +
  • For fixed width types, the column uses the appropriate number of bytes for +the specified dataFormat (see above).

  • +
  • When the VARCHAR value is decoded, all bytes from the start position to +the end of the message is used.

  • +
+

If start and end position are given:

+
    +
  • For fixed width types, the size must be equal to the number of bytes used by +specified dataFormat.

  • +
  • For the VARCHAR data type all bytes between start (inclusive) and end +(exclusive) are used.

  • +
+

If no mapping attribute is specified, it is equivalent to setting the start +position to 0 and leaving the end position undefined.

+

The decoding scheme of numeric data types (BIGINT, INTEGER, +SMALLINT, TINYINT, DOUBLE) is straightforward. A sequence of bytes +is read from input message and decoded according to either:

+
    +
  • big-endian encoding (for integer types)

  • +
  • IEEE 754 format for (for DOUBLE).

  • +
+

The length of a decoded byte sequence is implied by the dataFormat.

+

For the VARCHAR data type, a sequence of bytes is interpreted according to +UTF-8 encoding.

+
+
+

CSV decoder#

+

The CSV decoder converts the bytes representing a message or key into a string +using UTF-8 encoding, and interprets the result as a link of comma-separated +values.

+

For fields, the type and mapping attributes must be defined:

+
    +
  • type - Trino data type. See the following table for a list of supported +data types.

  • +
  • mapping - The index of the field in the CSV record.

  • +
+

The dataFormat and formatHint attributes are not supported and must be +omitted.

+ ++++ + + + + + + + + + + + + + + + + + + + +

Trino data type

Decoding rules

BIGINT, INTEGER, SMALLINT, TINYINT

Decoded using Java Long.parseLong()

DOUBLE

Decoded using Java Double.parseDouble()

BOOLEAN

“true” character sequence maps to true. Other character sequences map +to false

VARCHAR / VARCHAR(x)

Used as is

+

No other types are supported.

+
+
+

JSON decoder#

+

The JSON decoder converts the bytes representing a message or key into +Javascript Object Notaion (JSON) according to RFC 4627. The message or key +must convert into a JSON object, not an array or simple type.

+

For fields, the following attributes are supported:

+
    +
  • type - Trino data type of column.

  • +
  • dataFormat - Field decoder to be used for column.

  • +
  • mapping - Slash-separated list of field names to select a field from the +JSON object.

  • +
  • formatHint - Only for custom-date-time.

  • +
+

The JSON decoder supports multiple field decoders with _default being used +for standard table columns and a number of decoders for date and time-based +types.

+

The following table lists Trino data types, which can be used in type and +matching field decoders, and specified via dataFormat attribute:

+ ++++ + + + + + + + + + + + + + + + + + + + + + + + + + +

Trino data type

Allowed dataFormat values

BIGINT, INTEGER, SMALLINT, TINYINT, DOUBLE, +BOOLEAN, VARCHAR, VARCHAR(x)

Default field decoder (omitted dataFormat attribute)

DATE

custom-date-time, iso8601

TIME

custom-date-time, iso8601, milliseconds-since-epoch, +seconds-since-epoch

TIME WITH TIME ZONE

custom-date-time, iso8601

TIMESTAMP

custom-date-time, iso8601, rfc2822, +milliseconds-since-epoch, seconds-since-epoch

TIMESTAMP WITH TIME ZONE

custom-date-time, iso8601, rfc2822, +milliseconds-since-epoch, seconds-since-epoch

+

No other types are supported.

+
+
Default field decoder#
+

This is the standard field decoder. It supports all the Trino physical data +types. A field value is transformed under JSON conversion rules into boolean, +long, double, or string values. This decoder should be used for columns that are +not date or time based.

+
+
+
Date and time decoders#
+

To convert values from JSON objects to Trino DATE, TIME, TIME WITH TIME ZONE, TIMESTAMP or TIMESTAMP WITH TIME ZONE columns, select +special decoders using the dataFormat attribute of a field definition.

+
    +
  • iso8601 - Text based, parses a text field as an ISO 8601 timestamp.

  • +
  • rfc2822 - Text based, parses a text field as an RFC 2822 timestamp.

  • +
  • custom-date-time - Text based, parses a text field according to Joda +format pattern specified via formatHint attribute. The format pattern +should conform to +https://www.joda.org/joda-time/apidocs/org/joda/time/format/DateTimeFormat.html.

  • +
  • milliseconds-since-epoch - Number-based, interprets a text or number as +number of milliseconds since the epoch.

  • +
  • seconds-since-epoch - Number-based, interprets a text or number as number +of milliseconds since the epoch.

  • +
+

For TIMESTAMP WITH TIME ZONE and TIME WITH TIME ZONE data types, if +timezone information is present in decoded value, it is used as a Trino value. +Otherwise, the result time zone is set to UTC.

+
+
+
+

Avro decoder#

+

The Avro decoder converts the bytes representing a message or key in Avro format +based on a schema. The message must have the Avro schema embedded. Trino does +not support schemaless Avro decoding.

+

The dataSchema must be defined for any key or message using Avro +decoder. Avro decoder should point to the location of a valid Avro +schema file of the message which must be decoded. This location can be a remote +web server (e.g.: dataSchema: 'http://example.org/schema/avro_data.avsc') or +local file system(e.g.: dataSchema: '/usr/local/schema/avro_data.avsc'). The +decoder fails if this location is not accessible from the Trino cluster.

+

The following attributes are supported:

+
    +
  • name - Name of the column in the Trino table.

  • +
  • type - Trino data type of column.

  • +
  • mapping - A slash-separated list of field names to select a field from the +Avro schema. If the field specified in mapping does not exist in the +original Avro schema, a read operation returns NULL.

  • +
+

The following table lists the supported Trino types that can be used in type +for the equivalent Avro field types:

+ ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Trino data type

Allowed Avro data type

BIGINT

INT, LONG

DOUBLE

DOUBLE, FLOAT

BOOLEAN

BOOLEAN

VARCHAR / VARCHAR(x)

STRING

VARBINARY

FIXED, BYTES

ARRAY

ARRAY

MAP

MAP

+

No other types are supported.

+
+
Avro schema evolution#
+

The Avro decoder supports schema evolution with backward compatibility. With +backward compatibility, a newer schema can be used to read Avro data created +with an older schema. Any change in the Avro schema must also be reflected in +Trino’s topic definition file. Newly added or renamed fields must have a +default value in the Avro schema file.

+

The schema evolution behavior is as follows:

+
    +
  • Column added in new schema: Data created with an older schema produces a +default value when the table is using the new schema.

  • +
  • Column removed in new schema: Data created with an older schema no longer +outputs the data from the column that was removed.

  • +
  • Column is renamed in the new schema: This is equivalent to removing the column +and adding a new one, and data created with an older schema produces a +default value when the table is using the new schema.

  • +
  • Changing type of column in the new schema: If the type coercion is supported +by Avro, then the conversion happens. An error is thrown for incompatible +types.

  • +
+
+
+
+
+
+

SQL support#

+

The connector provides globally available and +read operation statements to access data and +metadata in Redis.

+
+
+

Performance#

+

The connector includes a number of performance improvements, detailed in the +following sections.

+
+

Pushdown#

+
+

Note

+

The connector performs pushdown where performance may be improved, but in +order to preserve correctness an operation may not be pushed down. When +pushdown of an operation may result in better performance but risks +correctness, the connector prioritizes correctness.

+
+
+

Predicate pushdown support#

+

The connector supports pushdown of keys of string type only, the zset +type is not supported. Key pushdown is not supported when multiple key fields +are defined in the table definition file.

+

The connector supports pushdown of equality predicates, such as IN or =. +Inequality predicates, such as !=, and range predicates, such as >, +<, or BETWEEN are not pushed down.

+

In the following example, the predicate of the first query is not pushed down +since > is a range predicate. The other queries are pushed down:

+
-- Not pushed down
+SELECT * FROM nation WHERE redis_key > 'CANADA';
+-- Pushed down
+SELECT * FROM nation WHERE redis_key = 'CANADA';
+SELECT * FROM nation WHERE redis_key IN ('CANADA', 'POLAND');
+
+
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/redshift.html b/430/connector/redshift.html new file mode 100644 index 000000000..beac77262 --- /dev/null +++ b/430/connector/redshift.html @@ -0,0 +1,1268 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Redshift connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Redshift connector#

+

The Redshift connector allows querying and creating tables in an +external Amazon Redshift cluster. This can be used to join data between +different systems like Redshift and Hive, or between two different +Redshift clusters.

+
+

Requirements#

+

To connect to Redshift, you need:

+
    +
  • Network access from the Trino coordinator and workers to Redshift. +Port 5439 is the default port.

  • +
+
+
+

Configuration#

+

To configure the Redshift connector, create a catalog properties file in +etc/catalog named, for example, example.properties, to mount the +Redshift connector as the example catalog. Create the file with the +following contents, replacing the connection properties as appropriate for your +setup:

+
connector.name=redshift
+connection-url=jdbc:redshift://example.net:5439/database
+connection-user=root
+connection-password=secret
+
+
+

The connection-user and connection-password are typically required and +determine the user credentials for the connection, often a service user. You can +use secrets to avoid actual values in the catalog +properties files.

+
+

Connection security#

+

If you have TLS configured with a globally-trusted certificate installed on your +data source, you can enable TLS between your cluster and the data +source by appending a parameter to the JDBC connection string set in the +connection-url catalog configuration property.

+

For example, on version 2.1 of the Redshift JDBC driver, TLS/SSL is enabled by +default with the SSL parameter. You can disable or further configure TLS +by appending parameters to the connection-url configuration property:

+
connection-url=jdbc:redshift://example.net:5439/database;SSL=TRUE;
+
+
+

For more information on TLS configuration options, see the Redshift JDBC driver +documentation.

+
+
+

Data source authentication#

+

The connector can provide credentials for the data source connection +in multiple ways:

+
    +
  • inline, in the connector configuration file

  • +
  • in a separate properties file

  • +
  • in a key store file

  • +
  • as extra credentials set when connecting to Trino

  • +
+

You can use secrets to avoid storing sensitive +values in the catalog properties files.

+

The following table describes configuration properties +for connection credentials:

+ ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

credential-provider.type

Type of the credential provider. Must be one of INLINE, FILE, or +KEYSTORE; defaults to INLINE.

connection-user

Connection user name.

connection-password

Connection password.

user-credential-name

Name of the extra credentials property, whose value to use as the user +name. See extraCredentials in Parameter +reference.

password-credential-name

Name of the extra credentials property, whose value to use as the +password.

connection-credential-file

Location of the properties file where credentials are present. It must +contain the connection-user and connection-password properties.

keystore-file-path

The location of the Java Keystore file, from which to read credentials.

keystore-type

File format of the keystore file, for example JKS or PEM.

keystore-password

Password for the key store.

keystore-user-credential-name

Name of the key store entity to use as the user name.

keystore-user-credential-password

Password for the user name key store entity.

keystore-password-credential-name

Name of the key store entity to use as the password.

keystore-password-credential-password

Password for the password key store entity.

+
+
+

Multiple Redshift databases or clusters#

+

The Redshift connector can only access a single database within +a Redshift cluster. Thus, if you have multiple Redshift databases, +or want to connect to multiple Redshift clusters, you must configure +multiple instances of the Redshift connector.

+

To add another catalog, simply add another properties file to etc/catalog +with a different name, making sure it ends in .properties. For example, +if you name the property file sales.properties, Trino creates a +catalog named sales using the configured connector.

+
+
+

General configuration properties#

+

The following table describes general catalog configuration properties for the +connector:

+ +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

Default value

case-insensitive-name-matching

Support case insensitive schema and table names.

false

case-insensitive-name-matching.cache-ttl

This value should be a duration.

1m

case-insensitive-name-matching.config-file

Path to a name mapping configuration file in JSON format that allows +Trino to disambiguate between schemas and tables with similar names in +different cases.

null

case-insensitive-name-matching.config-file.refresh-period

Frequency with which Trino checks the name matching configuration file +for changes. This value should be a duration.

(refresh disabled)

metadata.cache-ttl

The duration for which metadata, including +table and column statistics, is cached.

0s (caching disabled)

metadata.cache-missing

Cache the fact that metadata, including table and column statistics, is +not available

false

metadata.cache-maximum-size

Maximum number of objects stored in the metadata cache

10000

write.batch-size

Maximum number of statements in a batched execution. +Do not change this setting from the default. Non-default values may +negatively impact performance.

1000

dynamic-filtering.enabled

Push down dynamic filters into JDBC queries

true

dynamic-filtering.wait-timeout

Maximum duration for which Trino will wait for dynamic +filters to be collected from the build side of joins before starting a +JDBC query. Using a large timeout can potentially result in more detailed +dynamic filters. However, it can also increase latency for some queries.

20s

+
+
+

Appending query metadata#

+

The optional parameter query.comment-format allows you to configure a SQL +comment that is sent to the datasource with each query. The format of this +comment can contain any characters and the following metadata:

+
    +
  • $QUERY_ID: The identifier of the query.

  • +
  • $USER: The name of the user who submits the query to Trino.

  • +
  • $SOURCE: The identifier of the client tool used to submit the query, for +example trino-cli.

  • +
  • $TRACE_TOKEN: The trace token configured with the client tool.

  • +
+

The comment can provide more context about the query. This additional +information is available in the logs of the datasource. To include environment +variables from the Trino cluster with the comment , use the +${ENV:VARIABLE-NAME} syntax.

+

The following example sets a simple comment that identifies each query sent by +Trino:

+
query.comment-format=Query sent by Trino.
+
+
+

With this configuration, a query such as SELECT * FROM example_table; is +sent to the datasource with the comment appended:

+
SELECT * FROM example_table; /*Query sent by Trino.*/
+
+
+

The following example improves on the preceding example by using metadata:

+
query.comment-format=Query $QUERY_ID sent by user $USER from Trino.
+
+
+

If Jane sent the query with the query identifier +20230622_180528_00000_bkizg, the following comment string is sent to the +datasource:

+
SELECT * FROM example_table; /*Query 20230622_180528_00000_bkizg sent by user Jane from Trino.*/
+
+
+
+

Note

+

Certain JDBC driver settings and logging configurations might cause the +comment to be removed.

+
+
+
+

Domain compaction threshold#

+

Pushing down a large list of predicates to the data source can compromise +performance. Trino compacts large predicates into a simpler range predicate +by default to ensure a balance between performance and predicate pushdown. +If necessary, the threshold for this compaction can be increased to improve +performance when the data source is capable of taking advantage of large +predicates. Increasing this threshold may improve pushdown of large +dynamic filters. +The domain-compaction-threshold catalog configuration property or the +domain_compaction_threshold catalog session property can be used to adjust the default value of +32 for this threshold.

+
+
+

Procedures#

+
    +
  • system.flush_metadata_cache()

    +

    Flush JDBC metadata caches. For example, the following system call +flushes the metadata caches for all schemas in the example catalog

    +
    USE example.example_schema;
    +CALL system.flush_metadata_cache();
    +
    +
    +
  • +
+
+
+

Case insensitive matching#

+

When case-insensitive-name-matching is set to true, Trino +is able to query non-lowercase schemas and tables by maintaining a mapping of +the lowercase name to the actual name in the remote system. However, if two +schemas and/or tables have names that differ only in case (such as “customers” +and “Customers”) then Trino fails to query them due to ambiguity.

+

In these cases, use the case-insensitive-name-matching.config-file catalog +configuration property to specify a configuration file that maps these remote +schemas/tables to their respective Trino schemas/tables:

+
{
+  "schemas": [
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "mapping": "case_insensitive_1"
+    },
+    {
+      "remoteSchema": "cASEsENSITIVEnAME",
+      "mapping": "case_insensitive_2"
+    }],
+  "tables": [
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "remoteTable": "tablex",
+      "mapping": "table_1"
+    },
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "remoteTable": "TABLEX",
+      "mapping": "table_2"
+    }]
+}
+
+
+

Queries against one of the tables or schemes defined in the mapping +attributes are run against the corresponding remote entity. For example, a query +against tables in the case_insensitive_1 schema is forwarded to the +CaseSensitiveName schema and a query against case_insensitive_2 is forwarded +to the cASEsENSITIVEnAME schema.

+

At the table mapping level, a query on case_insensitive_1.table_1 as +configured above is forwarded to CaseSensitiveName.tablex, and a query on +case_insensitive_1.table_2 is forwarded to CaseSensitiveName.TABLEX.

+

By default, when a change is made to the mapping configuration file, Trino must +be restarted to load the changes. Optionally, you can set the +case-insensitive-name-mapping.refresh-period to have Trino refresh the +properties without requiring a restart:

+
case-insensitive-name-mapping.refresh-period=30s
+
+
+
+
+

Non-transactional INSERT#

+

The connector supports adding rows using INSERT statements. +By default, data insertion is performed by writing data to a temporary table. +You can skip this step to improve performance and write directly to the target +table. Set the insert.non-transactional-insert.enabled catalog property +or the corresponding non_transactional_insert catalog session property to +true.

+

Note that with this property enabled, data can be corrupted in rare cases where +exceptions occur during the insert operation. With transactions disabled, no +rollback can be performed.

+
+
+
+

Querying Redshift#

+

The Redshift connector provides a schema for every Redshift schema. +You can see the available Redshift schemas by running SHOW SCHEMAS:

+
SHOW SCHEMAS FROM example;
+
+
+

If you have a Redshift schema named web, you can view the tables +in this schema by running SHOW TABLES:

+
SHOW TABLES FROM example.web;
+
+
+

You can see a list of the columns in the clicks table in the web database +using either of the following:

+
DESCRIBE example.web.clicks;
+SHOW COLUMNS FROM example.web.clicks;
+
+
+

Finally, you can access the clicks table in the web schema:

+
SELECT * FROM example.web.clicks;
+
+
+

If you used a different name for your catalog properties file, use that catalog +name instead of example in the above examples.

+
+
+

Type mapping#

+
+

Type mapping configuration properties#

+

The following properties can be used to configure how data types from the +connected data source are mapped to Trino data types and how the metadata is +cached in Trino.

+ +++++ + + + + + + + + + + + + + + + + +

Property name

Description

Default value

unsupported-type-handling

Configure how unsupported column data types are handled:

+
    +
  • IGNORE, column is not accessible.

  • +
  • CONVERT_TO_VARCHAR, column is converted to unbounded VARCHAR.

  • +
+

The respective catalog session property is unsupported_type_handling.

+

IGNORE

jdbc-types-mapped-to-varchar

Allow forced mapping of comma separated lists of data types to convert to +unbounded VARCHAR

+
+
+
+

SQL support#

+

The connector provides read access and write access to data and metadata in +Redshift. In addition to the globally available and read operation +statements, the connector supports the following features:

+ +
+

UPDATE#

+

Only UPDATE statements with constant assignments and predicates are +supported. For example, the following statement is supported because the values +assigned are constants:

+
UPDATE table SET col1 = 1 WHERE col3 = 1
+
+
+

Arithmetic expressions, function calls, and other non-constant UPDATE +statements are not supported. For example, the following statement is not +supported because arithmetic expressions cannot be used with the SET +command:

+
UPDATE table SET col1 = col2 + 2 WHERE col3 = 1
+
+
+

The =, !=, >, <, >=, <=, IN, NOT IN operators are supported in +predicates. The following statement is not supported because the AND operator +cannot be used in predicates:

+
UPDATE table SET col1 = 1 WHERE col3 = 1 AND col2 = 3
+
+
+

All column values of a table row cannot be updated simultaneously. For a three +column table, the following statement is not supported:

+
UPDATE table SET col1 = 1, col2 = 2, col3 = 3 WHERE col3 = 1
+
+
+
+
+

SQL DELETE#

+

If a WHERE clause is specified, the DELETE operation only works if the +predicate in the clause can be fully pushed down to the data source.

+
+
+

ALTER TABLE RENAME TO#

+

The connector does not support renaming tables across multiple schemas. For +example, the following statement is supported:

+
ALTER TABLE example.schema_one.table_one RENAME TO example.schema_one.table_two
+
+
+

The following statement attempts to rename a table across schemas, and therefore +is not supported:

+
ALTER TABLE example.schema_one.table_one RENAME TO example.schema_two.table_two
+
+
+
+
+

ALTER SCHEMA#

+

The connector supports renaming a schema with the ALTER SCHEMA RENAME +statement. ALTER SCHEMA SET AUTHORIZATION is not supported.

+
+
+
+

Fault-tolerant execution support#

+

The connector supports Fault-tolerant execution of query +processing. Read and write operations are both supported with any retry policy.

+
+
+

Table functions#

+

The connector provides specific table functions to +access Redshift.

+
+

query(varchar) -> table#

+

The query function allows you to query the underlying database directly. It +requires syntax native to Redshift, because the full query is pushed down and +processed in Redshift. This can be useful for accessing native features which +are not implemented in Trino or for improving query performance in situations +where running a query natively may be faster.

+

The native query passed to the underlying data source is required to return a +table as a result set. Only the data source performs validation or security +checks for these queries using its own configuration. Trino does not perform +these tasks. Only use passthrough queries to read data.

+

For example, query the example catalog and select the top 10 nations by +population:

+
SELECT
+  *
+FROM
+  TABLE(
+    example.system.query(
+      query => 'SELECT
+        TOP 10 *
+      FROM
+        tpch.nation
+      ORDER BY
+        population DESC'
+    )
+  );
+
+
+
+

Note

+

The query engine does not preserve the order of the results of this +function. If the passed query contains an ORDER BY clause, the +function result may not be ordered as expected.

+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/singlestore.html b/430/connector/singlestore.html new file mode 100644 index 000000000..71235d5ea --- /dev/null +++ b/430/connector/singlestore.html @@ -0,0 +1,1451 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SingleStore connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

SingleStore connector#

+

The SingleStore (formerly known as MemSQL) connector allows querying and +creating tables in an external SingleStore database.

+
+

Requirements#

+

To connect to SingleStore, you need:

+
    +
  • SingleStore version 7.1.4 or higher.

  • +
  • Network access from the Trino coordinator and workers to SingleStore. Port +3306 is the default port.

  • +
+
+
+

Configuration#

+

To configure the SingleStore connector, create a catalog properties file in +etc/catalog named, for example, example.properties, to mount the +SingleStore connector as the example catalog. Create the file with the +following contents, replacing the connection properties as appropriate for your +setup:

+
connector.name=singlestore
+connection-url=jdbc:singlestore://example.net:3306
+connection-user=root
+connection-password=secret
+
+
+

The connection-url defines the connection information and parameters to pass +to the SingleStore JDBC driver. The supported parameters for the URL are +available in the SingleStore JDBC driver documentation.

+

The connection-user and connection-password are typically required and +determine the user credentials for the connection, often a service user. You can +use secrets to avoid actual values in the catalog +properties files.

+
+

Connection security#

+

If you have TLS configured with a globally-trusted certificate installed on your +data source, you can enable TLS between your cluster and the data +source by appending a parameter to the JDBC connection string set in the +connection-url catalog configuration property.

+

Enable TLS between your cluster and SingleStore by appending the useSsl=true +parameter to the connection-url configuration property:

+
connection-url=jdbc:singlestore://example.net:3306/?useSsl=true
+
+
+

For more information on TLS configuration options, see the JDBC driver +documentation.

+
+
+

Multiple SingleStore servers#

+

You can have as many catalogs as you need, so if you have additional +SingleStore servers, simply add another properties file to etc/catalog +with a different name (making sure it ends in .properties). For +example, if you name the property file sales.properties, Trino +will create a catalog named sales using the configured connector.

+
+
+

General configuration properties#

+

The following table describes general catalog configuration properties for the +connector:

+ +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

Default value

case-insensitive-name-matching

Support case insensitive schema and table names.

false

case-insensitive-name-matching.cache-ttl

This value should be a duration.

1m

case-insensitive-name-matching.config-file

Path to a name mapping configuration file in JSON format that allows +Trino to disambiguate between schemas and tables with similar names in +different cases.

null

case-insensitive-name-matching.config-file.refresh-period

Frequency with which Trino checks the name matching configuration file +for changes. This value should be a duration.

(refresh disabled)

metadata.cache-ttl

The duration for which metadata, including +table and column statistics, is cached.

0s (caching disabled)

metadata.cache-missing

Cache the fact that metadata, including table and column statistics, is +not available

false

metadata.cache-maximum-size

Maximum number of objects stored in the metadata cache

10000

write.batch-size

Maximum number of statements in a batched execution. +Do not change this setting from the default. Non-default values may +negatively impact performance.

1000

dynamic-filtering.enabled

Push down dynamic filters into JDBC queries

true

dynamic-filtering.wait-timeout

Maximum duration for which Trino will wait for dynamic +filters to be collected from the build side of joins before starting a +JDBC query. Using a large timeout can potentially result in more detailed +dynamic filters. However, it can also increase latency for some queries.

20s

+
+
+

Appending query metadata#

+

The optional parameter query.comment-format allows you to configure a SQL +comment that is sent to the datasource with each query. The format of this +comment can contain any characters and the following metadata:

+
    +
  • $QUERY_ID: The identifier of the query.

  • +
  • $USER: The name of the user who submits the query to Trino.

  • +
  • $SOURCE: The identifier of the client tool used to submit the query, for +example trino-cli.

  • +
  • $TRACE_TOKEN: The trace token configured with the client tool.

  • +
+

The comment can provide more context about the query. This additional +information is available in the logs of the datasource. To include environment +variables from the Trino cluster with the comment , use the +${ENV:VARIABLE-NAME} syntax.

+

The following example sets a simple comment that identifies each query sent by +Trino:

+
query.comment-format=Query sent by Trino.
+
+
+

With this configuration, a query such as SELECT * FROM example_table; is +sent to the datasource with the comment appended:

+
SELECT * FROM example_table; /*Query sent by Trino.*/
+
+
+

The following example improves on the preceding example by using metadata:

+
query.comment-format=Query $QUERY_ID sent by user $USER from Trino.
+
+
+

If Jane sent the query with the query identifier +20230622_180528_00000_bkizg, the following comment string is sent to the +datasource:

+
SELECT * FROM example_table; /*Query 20230622_180528_00000_bkizg sent by user Jane from Trino.*/
+
+
+
+

Note

+

Certain JDBC driver settings and logging configurations might cause the +comment to be removed.

+
+
+
+

Domain compaction threshold#

+

Pushing down a large list of predicates to the data source can compromise +performance. Trino compacts large predicates into a simpler range predicate +by default to ensure a balance between performance and predicate pushdown. +If necessary, the threshold for this compaction can be increased to improve +performance when the data source is capable of taking advantage of large +predicates. Increasing this threshold may improve pushdown of large +dynamic filters. +The domain-compaction-threshold catalog configuration property or the +domain_compaction_threshold catalog session property can be used to adjust the default value of +32 for this threshold.

+
+
+

Procedures#

+
    +
  • system.flush_metadata_cache()

    +

    Flush JDBC metadata caches. For example, the following system call +flushes the metadata caches for all schemas in the example catalog

    +
    USE example.example_schema;
    +CALL system.flush_metadata_cache();
    +
    +
    +
  • +
+
+
+

Case insensitive matching#

+

When case-insensitive-name-matching is set to true, Trino +is able to query non-lowercase schemas and tables by maintaining a mapping of +the lowercase name to the actual name in the remote system. However, if two +schemas and/or tables have names that differ only in case (such as “customers” +and “Customers”) then Trino fails to query them due to ambiguity.

+

In these cases, use the case-insensitive-name-matching.config-file catalog +configuration property to specify a configuration file that maps these remote +schemas/tables to their respective Trino schemas/tables:

+
{
+  "schemas": [
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "mapping": "case_insensitive_1"
+    },
+    {
+      "remoteSchema": "cASEsENSITIVEnAME",
+      "mapping": "case_insensitive_2"
+    }],
+  "tables": [
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "remoteTable": "tablex",
+      "mapping": "table_1"
+    },
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "remoteTable": "TABLEX",
+      "mapping": "table_2"
+    }]
+}
+
+
+

Queries against one of the tables or schemes defined in the mapping +attributes are run against the corresponding remote entity. For example, a query +against tables in the case_insensitive_1 schema is forwarded to the +CaseSensitiveName schema and a query against case_insensitive_2 is forwarded +to the cASEsENSITIVEnAME schema.

+

At the table mapping level, a query on case_insensitive_1.table_1 as +configured above is forwarded to CaseSensitiveName.tablex, and a query on +case_insensitive_1.table_2 is forwarded to CaseSensitiveName.TABLEX.

+

By default, when a change is made to the mapping configuration file, Trino must +be restarted to load the changes. Optionally, you can set the +case-insensitive-name-mapping.refresh-period to have Trino refresh the +properties without requiring a restart:

+
case-insensitive-name-mapping.refresh-period=30s
+
+
+
+
+

Non-transactional INSERT#

+

The connector supports adding rows using INSERT statements. +By default, data insertion is performed by writing data to a temporary table. +You can skip this step to improve performance and write directly to the target +table. Set the insert.non-transactional-insert.enabled catalog property +or the corresponding non_transactional_insert catalog session property to +true.

+

Note that with this property enabled, data can be corrupted in rare cases where +exceptions occur during the insert operation. With transactions disabled, no +rollback can be performed.

+
+
+
+

Querying SingleStore#

+

The SingleStore connector provides a schema for every SingleStore database. +You can see the available SingleStore databases by running SHOW SCHEMAS:

+
SHOW SCHEMAS FROM example;
+
+
+

If you have a SingleStore database named web, you can view the tables +in this database by running SHOW TABLES:

+
SHOW TABLES FROM example.web;
+
+
+

You can see a list of the columns in the clicks table in the web +database using either of the following:

+
DESCRIBE example.web.clicks;
+SHOW COLUMNS FROM example.web.clicks;
+
+
+

Finally, you can access the clicks table in the web database:

+
SELECT * FROM example.web.clicks;
+
+
+

If you used a different name for your catalog properties file, use +that catalog name instead of example in the above examples.

+
+
+

Type mapping#

+

Because Trino and Singlestore each support types that the other does not, this +connector modifies some types when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction.

+
+

Singlestore to Trino type mapping#

+

The connector maps Singlestore types to the corresponding Trino types following +this table:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Singlestore to Trino type mapping#

Singlestore type

Trino type

Notes

BIT

BOOLEAN

BOOLEAN

BOOLEAN

TINYINT

TINYINT

TINYINT UNSIGNED

SMALLINT

SMALLINT

SMALLINT

SMALLINT UNSIGNED

INTEGER

INTEGER

INTEGER

INTEGER UNSIGNED

BIGINT

BIGINT

BIGINT

BIGINT UNSIGNED

DECIMAL(20, 0)

DOUBLE

DOUBLE

REAL

DOUBLE

DECIMAL(p, s)

DECIMAL(p, s)

See Singlestore DECIMAL type handling

CHAR(n)

CHAR(n)

TINYTEXT

VARCHAR(255)

TEXT

VARCHAR(65535)

MEDIUMTEXT

VARCHAR(16777215)

LONGTEXT

VARCHAR

VARCHAR(n)

VARCHAR(n)

LONGBLOB

VARBINARY

DATE

DATE

TIME

TIME(0)

TIME(6)

TIME(6)

DATETIME

TIMESTAMP(0)

DATETIME(6)

TIMESTAMP(6)

JSON

JSON

+

No other types are supported.

+
+
+

Trino to Singlestore type mapping#

+

The connector maps Trino types to the corresponding Singlestore types following +this table:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Trino to Singlestore type mapping#

Trino type

Singlestore type

Notes

BOOLEAN

BOOLEAN

TINYINT

TINYINT

SMALLINT

SMALLINT

INTEGER

INTEGER

BIGINT

BIGINT

DOUBLE

DOUBLE

REAL

FLOAT

DECIMAL(p, s)

DECIMAL(p, s)

See Singlestore DECIMAL type handling

CHAR(n)

CHAR(n)

VARCHAR(65535)

TEXT

VARCHAR(16777215)

MEDIUMTEXT

VARCHAR

LONGTEXT

VARCHAR(n)

VARCHAR(n)

VARBINARY

LONGBLOB

DATE

DATE

TIME(0)

TIME

TIME(6)

TIME(6)

TIMESTAMP(0)

DATETIME

TIMESTAMP(6)

DATETIME(6)

JSON

JSON

+

No other types are supported.

+
+
+

Decimal type handling#

+

DECIMAL types with unspecified precision or scale are mapped to a Trino +DECIMAL with a default precision of 38 and default scale of 0. The scale can +be changed by setting the decimal-mapping configuration property or the +decimal_mapping session property to allow_overflow. The scale of the +resulting type is controlled via the decimal-default-scale configuration +property or the decimal-rounding-mode session property. The precision is +always 38.

+

By default, values that require rounding or truncation to fit will cause a +failure at runtime. This behavior is controlled via the +decimal-rounding-mode configuration property or the +decimal_rounding_mode session property, which can be set to UNNECESSARY +(the default), UP, DOWN, CEILING, FLOOR, HALF_UP, +HALF_DOWN, or HALF_EVEN (see RoundingMode).

+
+
+

Type mapping configuration properties#

+

The following properties can be used to configure how data types from the +connected data source are mapped to Trino data types and how the metadata is +cached in Trino.

+ +++++ + + + + + + + + + + + + + + + + +

Property name

Description

Default value

unsupported-type-handling

Configure how unsupported column data types are handled:

+
    +
  • IGNORE, column is not accessible.

  • +
  • CONVERT_TO_VARCHAR, column is converted to unbounded VARCHAR.

  • +
+

The respective catalog session property is unsupported_type_handling.

+

IGNORE

jdbc-types-mapped-to-varchar

Allow forced mapping of comma separated lists of data types to convert to +unbounded VARCHAR

+
+
+
+

SQL support#

+

The connector provides read access and write access to data and metadata in +a SingleStore database. In addition to the globally available and read operation +statements, the connector supports the following features:

+ +
+

UPDATE#

+

Only UPDATE statements with constant assignments and predicates are +supported. For example, the following statement is supported because the values +assigned are constants:

+
UPDATE table SET col1 = 1 WHERE col3 = 1
+
+
+

Arithmetic expressions, function calls, and other non-constant UPDATE +statements are not supported. For example, the following statement is not +supported because arithmetic expressions cannot be used with the SET +command:

+
UPDATE table SET col1 = col2 + 2 WHERE col3 = 1
+
+
+

The =, !=, >, <, >=, <=, IN, NOT IN operators are supported in +predicates. The following statement is not supported because the AND operator +cannot be used in predicates:

+
UPDATE table SET col1 = 1 WHERE col3 = 1 AND col2 = 3
+
+
+

All column values of a table row cannot be updated simultaneously. For a three +column table, the following statement is not supported:

+
UPDATE table SET col1 = 1, col2 = 2, col3 = 3 WHERE col3 = 1
+
+
+
+
+

SQL DELETE#

+

If a WHERE clause is specified, the DELETE operation only works if the +predicate in the clause can be fully pushed down to the data source.

+
+
+

ALTER TABLE RENAME TO#

+

The connector does not support renaming tables across multiple schemas. For +example, the following statement is supported:

+
ALTER TABLE example.schema_one.table_one RENAME TO example.schema_one.table_two
+
+
+

The following statement attempts to rename a table across schemas, and therefore +is not supported:

+
ALTER TABLE example.schema_one.table_one RENAME TO example.schema_two.table_two
+
+
+
+
+
+

Performance#

+

The connector includes a number of performance improvements, detailed in the +following sections.

+
+

Pushdown#

+

The connector supports pushdown for a number of operations:

+ +
+

Note

+

The connector performs pushdown where performance may be improved, but in +order to preserve correctness an operation may not be pushed down. When +pushdown of an operation may result in better performance but risks +correctness, the connector prioritizes correctness.

+
+
+

Join pushdown#

+

The join-pushdown.enabled catalog configuration property or +join_pushdown_enabled catalog session property control whether the connector pushes +down join operations. The property defaults to false, and enabling join +pushdowns may negatively impact performance for some queries.

+
+
+

Predicate pushdown support#

+

The connector does not support pushdown of any predicates on columns with +textual types like CHAR or VARCHAR. +This ensures correctness of results since the data source may compare strings +case-insensitively.

+

In the following example, the predicate is not pushed down for either query +since name is a column of type VARCHAR:

+
SELECT * FROM nation WHERE name > 'CANADA';
+SELECT * FROM nation WHERE name = 'CANADA';
+
+
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/sqlserver.html b/430/connector/sqlserver.html new file mode 100644 index 000000000..ab01b7435 --- /dev/null +++ b/430/connector/sqlserver.html @@ -0,0 +1,1765 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SQL Server connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

SQL Server connector#

+

The SQL Server connector allows querying and creating tables in an external +Microsoft SQL Server database. This +can be used to join data between different systems like SQL Server and Hive, or +between two different SQL Server instances.

+
+

Requirements#

+

To connect to SQL Server, you need:

+
    +
  • SQL Server 2012 or higher, or Azure SQL Database.

  • +
  • Network access from the Trino coordinator and workers to SQL Server. +Port 1433 is the default port.

  • +
+
+
+

Configuration#

+

The connector can query a single database on a given SQL Server instance. Create +a catalog properties file that specifies the SQL server connector by setting the +connector.name to sqlserver.

+

For example, to access a database as example, create the file +etc/catalog/example.properties. Replace the connection properties as +appropriate for your setup:

+
connector.name=sqlserver
+connection-url=jdbc:sqlserver://<host>:<port>;databaseName=<databaseName>;encrypt=false
+connection-user=root
+connection-password=secret
+
+
+

The connection-url defines the connection information and parameters to pass +to the SQL Server JDBC driver. The supported parameters for the URL are +available in the SQL Server JDBC driver documentation.

+

The connection-user and connection-password are typically required and +determine the user credentials for the connection, often a service user. You can +use secrets to avoid actual values in the catalog +properties files.

+
+

Connection security#

+

The JDBC driver, and therefore the connector, automatically use Transport Layer +Security (TLS) encryption and certificate validation. This requires a suitable +TLS certificate configured on your SQL Server database host.

+

If you do not have the necessary configuration established, you can disable +encryption in the connection string with the encrypt property:

+
connection-url=jdbc:sqlserver://<host>:<port>;databaseName=<databaseName>;encrypt=false
+
+
+

Further parameters like trustServerCertificate, hostNameInCertificate, +trustStore, and trustStorePassword are details in the TLS section of +SQL Server JDBC driver documentation.

+
+
+

Data source authentication#

+

The connector can provide credentials for the data source connection +in multiple ways:

+
    +
  • inline, in the connector configuration file

  • +
  • in a separate properties file

  • +
  • in a key store file

  • +
  • as extra credentials set when connecting to Trino

  • +
+

You can use secrets to avoid storing sensitive +values in the catalog properties files.

+

The following table describes configuration properties +for connection credentials:

+ ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

credential-provider.type

Type of the credential provider. Must be one of INLINE, FILE, or +KEYSTORE; defaults to INLINE.

connection-user

Connection user name.

connection-password

Connection password.

user-credential-name

Name of the extra credentials property, whose value to use as the user +name. See extraCredentials in Parameter +reference.

password-credential-name

Name of the extra credentials property, whose value to use as the +password.

connection-credential-file

Location of the properties file where credentials are present. It must +contain the connection-user and connection-password properties.

keystore-file-path

The location of the Java Keystore file, from which to read credentials.

keystore-type

File format of the keystore file, for example JKS or PEM.

keystore-password

Password for the key store.

keystore-user-credential-name

Name of the key store entity to use as the user name.

keystore-user-credential-password

Password for the user name key store entity.

keystore-password-credential-name

Name of the key store entity to use as the password.

keystore-password-credential-password

Password for the password key store entity.

+
+
+

Multiple SQL Server databases or servers#

+

The SQL Server connector can only access a single SQL Server database +within a single catalog. Thus, if you have multiple SQL Server databases, +or want to connect to multiple SQL Server instances, you must configure +multiple instances of the SQL Server connector.

+

To add another catalog, simply add another properties file to etc/catalog +with a different name, making sure it ends in .properties. For example, +if you name the property file sales.properties, Trino creates a +catalog named sales using the configured connector.

+
+
+

General configuration properties#

+

The following table describes general catalog configuration properties for the +connector:

+ +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

Default value

case-insensitive-name-matching

Support case insensitive schema and table names.

false

case-insensitive-name-matching.cache-ttl

This value should be a duration.

1m

case-insensitive-name-matching.config-file

Path to a name mapping configuration file in JSON format that allows +Trino to disambiguate between schemas and tables with similar names in +different cases.

null

case-insensitive-name-matching.config-file.refresh-period

Frequency with which Trino checks the name matching configuration file +for changes. This value should be a duration.

(refresh disabled)

metadata.cache-ttl

The duration for which metadata, including +table and column statistics, is cached.

0s (caching disabled)

metadata.cache-missing

Cache the fact that metadata, including table and column statistics, is +not available

false

metadata.cache-maximum-size

Maximum number of objects stored in the metadata cache

10000

write.batch-size

Maximum number of statements in a batched execution. +Do not change this setting from the default. Non-default values may +negatively impact performance.

1000

dynamic-filtering.enabled

Push down dynamic filters into JDBC queries

true

dynamic-filtering.wait-timeout

Maximum duration for which Trino will wait for dynamic +filters to be collected from the build side of joins before starting a +JDBC query. Using a large timeout can potentially result in more detailed +dynamic filters. However, it can also increase latency for some queries.

20s

+
+
+

Appending query metadata#

+

The optional parameter query.comment-format allows you to configure a SQL +comment that is sent to the datasource with each query. The format of this +comment can contain any characters and the following metadata:

+
    +
  • $QUERY_ID: The identifier of the query.

  • +
  • $USER: The name of the user who submits the query to Trino.

  • +
  • $SOURCE: The identifier of the client tool used to submit the query, for +example trino-cli.

  • +
  • $TRACE_TOKEN: The trace token configured with the client tool.

  • +
+

The comment can provide more context about the query. This additional +information is available in the logs of the datasource. To include environment +variables from the Trino cluster with the comment , use the +${ENV:VARIABLE-NAME} syntax.

+

The following example sets a simple comment that identifies each query sent by +Trino:

+
query.comment-format=Query sent by Trino.
+
+
+

With this configuration, a query such as SELECT * FROM example_table; is +sent to the datasource with the comment appended:

+
SELECT * FROM example_table; /*Query sent by Trino.*/
+
+
+

The following example improves on the preceding example by using metadata:

+
query.comment-format=Query $QUERY_ID sent by user $USER from Trino.
+
+
+

If Jane sent the query with the query identifier +20230622_180528_00000_bkizg, the following comment string is sent to the +datasource:

+
SELECT * FROM example_table; /*Query 20230622_180528_00000_bkizg sent by user Jane from Trino.*/
+
+
+
+

Note

+

Certain JDBC driver settings and logging configurations might cause the +comment to be removed.

+
+
+
+

Domain compaction threshold#

+

Pushing down a large list of predicates to the data source can compromise +performance. Trino compacts large predicates into a simpler range predicate +by default to ensure a balance between performance and predicate pushdown. +If necessary, the threshold for this compaction can be increased to improve +performance when the data source is capable of taking advantage of large +predicates. Increasing this threshold may improve pushdown of large +dynamic filters. +The domain-compaction-threshold catalog configuration property or the +domain_compaction_threshold catalog session property can be used to adjust the default value of +32 for this threshold.

+
+
+

Specific configuration properties#

+

The SQL Server connector supports additional catalog properties to configure the +behavior of the connector and the issues queries to the database.

+ ++++ + + + + + + + + + + +

Property name

Description

sqlserver.snapshot-isolation.disabled

Control the automatic use of snapshot isolation for transactions issued by +Trino in SQL Server. Defaults to false, which means that snapshot +isolation is enabled.

+
+
+

Procedures#

+
    +
  • system.flush_metadata_cache()

    +

    Flush JDBC metadata caches. For example, the following system call +flushes the metadata caches for all schemas in the example catalog

    +
    USE example.example_schema;
    +CALL system.flush_metadata_cache();
    +
    +
    +
  • +
+
+
+

Case insensitive matching#

+

When case-insensitive-name-matching is set to true, Trino +is able to query non-lowercase schemas and tables by maintaining a mapping of +the lowercase name to the actual name in the remote system. However, if two +schemas and/or tables have names that differ only in case (such as “customers” +and “Customers”) then Trino fails to query them due to ambiguity.

+

In these cases, use the case-insensitive-name-matching.config-file catalog +configuration property to specify a configuration file that maps these remote +schemas/tables to their respective Trino schemas/tables:

+
{
+  "schemas": [
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "mapping": "case_insensitive_1"
+    },
+    {
+      "remoteSchema": "cASEsENSITIVEnAME",
+      "mapping": "case_insensitive_2"
+    }],
+  "tables": [
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "remoteTable": "tablex",
+      "mapping": "table_1"
+    },
+    {
+      "remoteSchema": "CaseSensitiveName",
+      "remoteTable": "TABLEX",
+      "mapping": "table_2"
+    }]
+}
+
+
+

Queries against one of the tables or schemes defined in the mapping +attributes are run against the corresponding remote entity. For example, a query +against tables in the case_insensitive_1 schema is forwarded to the +CaseSensitiveName schema and a query against case_insensitive_2 is forwarded +to the cASEsENSITIVEnAME schema.

+

At the table mapping level, a query on case_insensitive_1.table_1 as +configured above is forwarded to CaseSensitiveName.tablex, and a query on +case_insensitive_1.table_2 is forwarded to CaseSensitiveName.TABLEX.

+

By default, when a change is made to the mapping configuration file, Trino must +be restarted to load the changes. Optionally, you can set the +case-insensitive-name-mapping.refresh-period to have Trino refresh the +properties without requiring a restart:

+
case-insensitive-name-mapping.refresh-period=30s
+
+
+
+
+

Non-transactional INSERT#

+

The connector supports adding rows using INSERT statements. +By default, data insertion is performed by writing data to a temporary table. +You can skip this step to improve performance and write directly to the target +table. Set the insert.non-transactional-insert.enabled catalog property +or the corresponding non_transactional_insert catalog session property to +true.

+

Note that with this property enabled, data can be corrupted in rare cases where +exceptions occur during the insert operation. With transactions disabled, no +rollback can be performed.

+
+
+
+

Querying SQL Server#

+

The SQL Server connector provides access to all schemas visible to the specified +user in the configured database. For the following examples, assume the SQL +Server catalog is example.

+

You can see the available schemas by running SHOW SCHEMAS:

+
SHOW SCHEMAS FROM example;
+
+
+

If you have a schema named web, you can view the tables +in this schema by running SHOW TABLES:

+
SHOW TABLES FROM example.web;
+
+
+

You can see a list of the columns in the clicks table in the web database +using either of the following:

+
DESCRIBE example.web.clicks;
+SHOW COLUMNS FROM example.web.clicks;
+
+
+

Finally, you can query the clicks table in the web schema:

+
SELECT * FROM example.web.clicks;
+
+
+

If you used a different name for your catalog properties file, use +that catalog name instead of example in the above examples.

+
+
+

Type mapping#

+

Because Trino and SQL Server each support types that the other does not, this +connector modifies some types when reading or +writing data. Data types may not map the same way in both directions between +Trino and the data source. Refer to the following sections for type mapping in +each direction.

+
+

SQL Server type to Trino type mapping#

+

The connector maps SQL Server types to the corresponding Trino types following this table:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
SQL Server type to Trino type mapping#

SQL Server database type

Trino type

Notes

BIT

BOOLEAN

TINYINT

SMALLINT

SQL Server TINYINT is actually unsigned TINYINT

SMALLINT

SMALLINT

INTEGER

INTEGER

BIGINT

BIGINT

DOUBLE PRECISION

DOUBLE

FLOAT[(n)]

REAL or DOUBLE

See Numeric type mapping

REAL

REAL

DECIMAL[(p[, s])], NUMERIC[(p[, s])]

DECIMAL(p, s)

CHAR[(n)]

CHAR(n)

1 <= n <= 8000

NCHAR[(n)]

CHAR(n)

1 <= n <= 4000

VARCHAR[(n | max)], NVARCHAR[(n | max)]

VARCHAR(n)

1 <= n <= 8000, max = 2147483647

TEXT

VARCHAR(2147483647)

NTEXT

VARCHAR(1073741823)

VARBINARY[(n | max)]

VARBINARY

1 <= n <= 8000, max = 2147483647

DATE

DATE

TIME[(n)]

TIME(n)

0 <= n <= 7

DATETIME2[(n)]

TIMESTAMP(n)

0 <= n <= 7

SMALLDATETIME

TIMESTAMP(0)

DATETIMEOFFSET[(n)]

TIMESTAMP(n) WITH TIME ZONE

0 <= n <= 7

+
+
+

Trino type to SQL Server type mapping#

+

The connector maps Trino types to the corresponding SQL Server types following this table:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Trino type to SQL Server type mapping#

Trino type

SQL Server type

Notes

BOOLEAN

BIT

TINYINT

TINYINT

Trino only supports writing values belonging to [0, 127]

SMALLINT

SMALLINT

INTEGER

INTEGER

BIGINT

BIGINT

REAL

REAL

DOUBLE

DOUBLE PRECISION

DECIMAL(p, s)

DECIMAL(p, s)

CHAR(n)

NCHAR(n) or NVARCHAR(max)

See Character type mapping

VARCHAR(n)

NVARCHAR(n) or NVARCHAR(max)

See Character type mapping

VARBINARY

VARBINARY(max)

DATE

DATE

TIME(n)

TIME(n)

0 <= n <= 7

TIMESTAMP(n)

DATETIME2(n)

0 <= n <= 7

+

Complete list of SQL Server data types.

+
+
+

Numeric type mapping#

+

For SQL Server FLOAT[(n)]:

+
    +
  • If n is not specified maps to Trino Double

  • +
  • If 1 <= n <= 24 maps to Trino REAL

  • +
  • If 24 < n <= 53 maps to Trino DOUBLE

  • +
+
+
+

Character type mapping#

+

For Trino CHAR(n):

+
    +
  • If 1 <= n <= 4000 maps SQL Server NCHAR(n)

  • +
  • If n > 4000 maps SQL Server NVARCHAR(max)

  • +
+

For Trino VARCHAR(n):

+
    +
  • If 1 <= n <= 4000 maps SQL Server NVARCHAR(n)

  • +
  • If n > 4000 maps SQL Server NVARCHAR(max)

  • +
+
+
+

Type mapping configuration properties#

+

The following properties can be used to configure how data types from the +connected data source are mapped to Trino data types and how the metadata is +cached in Trino.

+ +++++ + + + + + + + + + + + + + + + + +

Property name

Description

Default value

unsupported-type-handling

Configure how unsupported column data types are handled:

+
    +
  • IGNORE, column is not accessible.

  • +
  • CONVERT_TO_VARCHAR, column is converted to unbounded VARCHAR.

  • +
+

The respective catalog session property is unsupported_type_handling.

+

IGNORE

jdbc-types-mapped-to-varchar

Allow forced mapping of comma separated lists of data types to convert to +unbounded VARCHAR

+
+
+
+

SQL support#

+

The connector provides read access and write access to data and metadata in SQL +Server. In addition to the globally available +and read operation statements, the connector +supports the following features:

+ +
+

UPDATE#

+

Only UPDATE statements with constant assignments and predicates are +supported. For example, the following statement is supported because the values +assigned are constants:

+
UPDATE table SET col1 = 1 WHERE col3 = 1
+
+
+

Arithmetic expressions, function calls, and other non-constant UPDATE +statements are not supported. For example, the following statement is not +supported because arithmetic expressions cannot be used with the SET +command:

+
UPDATE table SET col1 = col2 + 2 WHERE col3 = 1
+
+
+

The =, !=, >, <, >=, <=, IN, NOT IN operators are supported in +predicates. The following statement is not supported because the AND operator +cannot be used in predicates:

+
UPDATE table SET col1 = 1 WHERE col3 = 1 AND col2 = 3
+
+
+

All column values of a table row cannot be updated simultaneously. For a three +column table, the following statement is not supported:

+
UPDATE table SET col1 = 1, col2 = 2, col3 = 3 WHERE col3 = 1
+
+
+
+
+

SQL DELETE#

+

If a WHERE clause is specified, the DELETE operation only works if the +predicate in the clause can be fully pushed down to the data source.

+
+
+

ALTER TABLE RENAME TO#

+

The connector does not support renaming tables across multiple schemas. For +example, the following statement is supported:

+
ALTER TABLE example.schema_one.table_one RENAME TO example.schema_one.table_two
+
+
+

The following statement attempts to rename a table across schemas, and therefore +is not supported:

+
ALTER TABLE example.schema_one.table_one RENAME TO example.schema_two.table_two
+
+
+
+
+
+

Fault-tolerant execution support#

+

The connector supports Fault-tolerant execution of query +processing. Read and write operations are both supported with any retry policy.

+
+
+

Table functions#

+

The connector provides specific table functions to +access SQL Server.

+
+

query(varchar) -> table#

+

The query function allows you to query the underlying database directly. It +requires syntax native to SQL Server, because the full query is pushed down and +processed in SQL Server. This can be useful for accessing native features which +are not implemented in Trino or for improving query performance in situations +where running a query natively may be faster.

+

The native query passed to the underlying data source is required to return a +table as a result set. Only the data source performs validation or security +checks for these queries using its own configuration. Trino does not perform +these tasks. Only use passthrough queries to read data.

+

For example, query the example catalog and select the top 10 percent of +nations by population:

+
SELECT
+  *
+FROM
+  TABLE(
+    example.system.query(
+      query => 'SELECT
+        TOP(10) PERCENT *
+      FROM
+        tpch.nation
+      ORDER BY
+        population DESC'
+    )
+  );
+
+
+
+
+

procedure(varchar) -> table#

+

The procedure function allows you to run stored procedures on the underlying +database directly. It requires syntax native to SQL Server, because the full query +is pushed down and processed in SQL Server. In order to use this table function set +sqlserver.experimental.stored-procedure-table-function-enabled to true.

+
+

Note

+

The procedure function does not support running StoredProcedures that return multiple statements, +use a non-select statement, use output parameters, or use conditional statements.

+
+
+

Warning

+

This feature is experimental only. The function has security implication and syntax might change and +be backward incompatible.

+
+

The follow example runs the stored procedure employee_sp in the example catalog and the +example_schema schema in the underlying SQL Server database:

+
SELECT
+  *
+FROM
+  TABLE(
+    example.system.procedure(
+      query => 'EXECUTE example_schema.employee_sp'
+    )
+  );
+
+
+

If the stored procedure employee_sp requires any input +append the parameter value to the procedure statement:

+
SELECT
+  *
+FROM
+  TABLE(
+    example.system.procedure(
+      query => 'EXECUTE example_schema.employee_sp 0'
+    )
+  );
+
+
+
+

Note

+

The query engine does not preserve the order of the results of this +function. If the passed query contains an ORDER BY clause, the +function result may not be ordered as expected.

+
+
+
+
+

Performance#

+

The connector includes a number of performance improvements, detailed in the +following sections.

+
+

Table statistics#

+

The SQL Server connector can use table and column statistics for cost based optimizations, to improve query processing performance +based on the actual data in the data source.

+

The statistics are collected by SQL Server and retrieved by the connector.

+

The connector can use information stored in single-column statistics. SQL Server +Database can automatically create column statistics for certain columns. If +column statistics are not created automatically for a certain column, you can +create them by executing the following statement in SQL Server Database.

+
CREATE STATISTICS example_statistics_name ON table_schema.table_name (column_name);
+
+
+

SQL Server Database routinely updates the statistics. In some cases, you may +want to force statistics update (e.g. after defining new column statistics or +after changing data in the table). You can do that by executing the following +statement in SQL Server Database.

+
UPDATE STATISTICS table_schema.table_name;
+
+
+

Refer to SQL Server documentation for information about options, limitations and +additional considerations.

+
+
+

Pushdown#

+

The connector supports pushdown for a number of operations:

+ +

Aggregate pushdown for the following functions:

+ +
+

Note

+

The connector performs pushdown where performance may be improved, but in +order to preserve correctness an operation may not be pushed down. When +pushdown of an operation may result in better performance but risks +correctness, the connector prioritizes correctness.

+
+
+

Cost-based join pushdown#

+

The connector supports cost-based Join pushdown to make intelligent +decisions about whether to push down a join operation to the data source.

+

When cost-based join pushdown is enabled, the connector only pushes down join +operations if the available Table statistics suggest that doing so +improves performance. Note that if no table statistics are available, join +operation pushdown does not occur to avoid a potential decrease in query +performance.

+

The following table describes catalog configuration properties for +join pushdown:

+ +++++ + + + + + + + + + + + + + + + + +

Property name

Description

Default value

join-pushdown.enabled

Enable join pushdown. Equivalent catalog +session property is +join_pushdown_enabled.

true

join-pushdown.strategy

Strategy used to evaluate whether join operations are pushed down. Set to +AUTOMATIC to enable cost-based join pushdown, or EAGER to +push down joins whenever possible. Note that EAGER can push down joins +even when table statistics are unavailable, which may result in degraded +query performance. Because of this, EAGER is only recommended for +testing and troubleshooting purposes.

AUTOMATIC

+
+
+

Predicate pushdown support#

+

The connector supports pushdown of predicates on VARCHAR and NVARCHAR +columns if the underlying columns in SQL Server use a case-sensitive collation.

+

The following operators are pushed down:

+
    +
  • =

  • +
  • <>

  • +
  • IN

  • +
  • NOT IN

  • +
+

To ensure correct results, operators are not pushed down for columns using a +case-insensitive collation.

+
+
+
+

Bulk insert#

+

You can optionally use the bulk copy API +to drastically speed up write operations.

+

Enable bulk copying and a lock on the destination table to meet minimal +logging requirements.

+

The following table shows the relevant catalog configuration properties and +their default values:

+ + +++++ + + + + + + + + + + + + + + + + +
Bulk load properties#

Property name

Description

Default

sqlserver.bulk-copy-for-write.enabled

Use the SQL Server bulk copy API for writes. The corresponding catalog +session property is bulk_copy_for_write.

false

sqlserver.bulk-copy-for-write.lock-destination-table

Obtain a bulk update lock on the destination table for write operations. +The corresponding catalog session property is +bulk_copy_for_write_lock_destination_table. Setting is only used when +bulk-copy-for-write.enabled=true.

false

+

Limitations:

+
    +
  • Column names with leading and trailing spaces are not supported.

  • +
+
+
+
+

Data compression#

+

You can specify the data compression policy for SQL Server tables +with the data_compression table property. Valid policies are NONE, ROW or PAGE.

+

Example:

+
CREATE TABLE example_schema.scientists (
+  recordkey VARCHAR,
+  name VARCHAR,
+  age BIGINT,
+  birthday DATE
+)
+WITH (
+  data_compression = 'ROW'
+);
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/system.html b/430/connector/system.html new file mode 100644 index 000000000..32f3b7508 --- /dev/null +++ b/430/connector/system.html @@ -0,0 +1,897 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + System connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

System connector#

+

The System connector provides information and metrics about the currently +running Trino cluster. It makes this available via normal SQL queries.

+
+

Configuration#

+

The System connector doesn’t need to be configured: it is automatically +available via a catalog named system.

+
+
+

Using the System connector#

+

List the available system schemas:

+
SHOW SCHEMAS FROM system;
+
+
+

List the tables in one of the schemas:

+
SHOW TABLES FROM system.runtime;
+
+
+

Query one of the tables:

+
SELECT * FROM system.runtime.nodes;
+
+
+

Kill a running query:

+
CALL system.runtime.kill_query(query_id => '20151207_215727_00146_tx3nr', message => 'Using too many resources');
+
+
+
+
+

System connector tables#

+
+

metadata.catalogs#

+

The catalogs table contains the list of available catalogs.

+
+
+

metadata.schema_properties#

+

The schema properties table contains the list of available properties +that can be set when creating a new schema.

+
+
+

metadata.table_properties#

+

The table properties table contains the list of available properties +that can be set when creating a new table.

+
+
+

metadata.materialized_views#

+

The materialized views table contains the following information about all +materialized views:

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Metadata for materialized views#

Column

Description

catalog_name

Name of the catalog containing the materialized view.

schema_name

Name of the schema in catalog_name containing the materialized view.

name

Name of the materialized view.

storage_catalog

Name of the catalog used for the storage table backing the materialized +view.

storage_schema

Name of the schema in storage_catalog used for the storage table +backing the materialized view.

storage_table

Name of the storage table backing the materialized view.

freshness

Freshness of data in the storage table. Queries on the +materialized view access the storage table if not STALE, otherwise +the definition is used to access the underlying data in the source +tables.

owner

Username of the creator and owner of the materialized view.

comment

User supplied text about the materialized view.

definition

SQL query that defines the data provided by the materialized view.

+
+
+

metadata.materialized_view_properties#

+

The materialized view properties table contains the list of available properties +that can be set when creating a new materialized view.

+
+
+

metadata.table_comments#

+

The table comments table contains the list of table comment.

+
+
+

runtime.nodes#

+

The nodes table contains the list of visible nodes in the Trino +cluster along with their status.

+
+
+

runtime.optimizer_rule_stats#

+

The optimizer_rule_stats table contains the statistics for optimizer +rule invocations during the query planning phase. The statistics are +aggregated over all queries since the server start-up. The table contains +information about invocation frequency, failure rates and performance for +optimizer rules. For example, you can look at the multiplication of columns +invocations and average_time to get an idea about which rules +generally impact query planning times the most.

+
+
+

runtime.queries#

+

The queries table contains information about currently and recently +running queries on the Trino cluster. From this table you can find out +the original query SQL text, the identity of the user who ran the query, +and performance information about the query, including how long the query +was queued and analyzed.

+
+
+

runtime.tasks#

+

The tasks table contains information about the tasks involved in a +Trino query, including where they were executed, and how many rows +and bytes each task processed.

+
+
+

runtime.transactions#

+

The transactions table contains the list of currently open transactions +and related metadata. This includes information such as the create time, +idle time, initialization parameters, and accessed catalogs.

+
+
+
+

System connector procedures#

+
+
+runtime.kill_query(query_id, message)#
+

Kill the query identified by query_id. The query failure message +includes the specified message. message is optional.

+
+
+
+

Type mapping#

+

Trino supports all data types used within the System schemas so no mapping +is required.

+
+
+

SQL support#

+

The connector provides globally available and +read operation statements to access Trino system +data and metadata.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/thrift.html b/430/connector/thrift.html new file mode 100644 index 000000000..e68ad35d6 --- /dev/null +++ b/430/connector/thrift.html @@ -0,0 +1,1207 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Thrift connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Thrift connector#

+

The Thrift connector makes it possible to integrate with external storage systems +without a custom Trino connector implementation by using +Apache Thrift on these servers. It is therefore +generic and can provide access to any backend, as long as it exposes the expected +API by using Thrift.

+

In order to use the Thrift connector with an external system, you need to implement +the TrinoThriftService interface, found below. Next, you configure the Thrift connector +to point to a set of machines, called Thrift servers, that implement the interface. +As part of the interface implementation, the Thrift servers provide metadata, +splits and data. The connector randomly chooses a server to talk to from the available +instances for metadata calls, or for data calls unless the splits include a list of addresses. +All requests are assumed to be idempotent and can be retried freely among any server.

+
+

Requirements#

+

To connect to your custom servers with the Thrift protocol, you need:

+ +
+
+

Configuration#

+

To configure the Thrift connector, create a catalog properties file +etc/catalog/example.properties with the following content, replacing the +properties as appropriate:

+
connector.name=trino_thrift
+trino.thrift.client.addresses=host:port,host:port
+
+
+
+

Multiple Thrift systems#

+

You can have as many catalogs as you need, so if you have additional +Thrift systems to connect to, simply add another properties file to etc/catalog +with a different name, making sure it ends in .properties.

+
+
+
+

Configuration properties#

+

The following configuration properties are available:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property name

Description

trino.thrift.client.addresses

Location of Thrift servers

trino-thrift.max-response-size

Maximum size of data returned from Thrift server

trino-thrift.metadata-refresh-threads

Number of refresh threads for metadata cache

trino.thrift.client.max-retries

Maximum number of retries for failed Thrift requests

trino.thrift.client.max-backoff-delay

Maximum interval between retry attempts

trino.thrift.client.min-backoff-delay

Minimum interval between retry attempts

trino.thrift.client.max-retry-time

Maximum duration across all attempts of a Thrift request

trino.thrift.client.backoff-scale-factor

Scale factor for exponential back off

trino.thrift.client.connect-timeout

Connect timeout

trino.thrift.client.request-timeout

Request timeout

trino.thrift.client.socks-proxy

SOCKS proxy address

trino.thrift.client.max-frame-size

Maximum size of a raw Thrift response

trino.thrift.client.transport

Thrift transport type (UNFRAMED, FRAMED, HEADER)

trino.thrift.client.protocol

Thrift protocol type (BINARY, COMPACT, FB_COMPACT)

+
+

trino.thrift.client.addresses#

+

Comma-separated list of thrift servers in the form of host:port. For example:

+
trino.thrift.client.addresses=192.0.2.3:7777,192.0.2.4:7779
+
+
+

This property is required; there is no default.

+
+
+

trino-thrift.max-response-size#

+

Maximum size of a data response that the connector accepts. This value is sent +by the connector to the Thrift server when requesting data, allowing it to size +the response appropriately.

+

This property is optional; the default is 16MB.

+
+
+

trino-thrift.metadata-refresh-threads#

+

Number of refresh threads for metadata cache.

+

This property is optional; the default is 1.

+
+
+
+

TrinoThriftService implementation#

+

The following IDL describes the TrinoThriftService that must be implemented:

+
enum TrinoThriftBound {
+  BELOW = 1;
+  EXACTLY = 2;
+  ABOVE = 3;
+}
+
+exception TrinoThriftServiceException {
+  1: string message;
+  2: bool retryable;
+}
+
+struct TrinoThriftNullableSchemaName {
+  1: optional string schemaName;
+}
+
+struct TrinoThriftSchemaTableName {
+  1: string schemaName;
+  2: string tableName;
+}
+
+struct TrinoThriftTableMetadata {
+  1: TrinoThriftSchemaTableName schemaTableName;
+  2: list<TrinoThriftColumnMetadata> columns;
+  3: optional string comment;
+
+  /**
+   * Returns a list of key sets which can be used for index lookups.
+   * The list is expected to have only unique key sets.
+   * {@code set<set<string>>} is not used here because some languages (like php) don't support it.
+   */
+  4: optional list<set<string>> indexableKeys;
+}
+
+struct TrinoThriftColumnMetadata {
+  1: string name;
+  2: string type;
+  3: optional string comment;
+  4: bool hidden;
+}
+
+struct TrinoThriftNullableColumnSet {
+  1: optional set<string> columns;
+}
+
+struct TrinoThriftTupleDomain {
+  /**
+   * Return a map of column names to constraints.
+   */
+  1: optional map<string, TrinoThriftDomain> domains;
+}
+
+/**
+ * Set that either includes all values, or excludes all values.
+ */
+struct TrinoThriftAllOrNoneValueSet {
+  1: bool all;
+}
+
+/**
+ * A set containing values that are uniquely identifiable.
+ * Assumes an infinite number of possible values. The values may be collectively included
+ * or collectively excluded.
+ * This structure is used with comparable, but not orderable types like "json", "map".
+ */
+struct TrinoThriftEquatableValueSet {
+  1: bool inclusive;
+  2: list<TrinoThriftBlock> values;
+}
+
+/**
+ * Elements of {@code nulls} array determine if a value for a corresponding row is null.
+ * Elements of {@code ints} array are values for each row. If row is null then value is ignored.
+ */
+struct TrinoThriftInteger {
+  1: optional list<bool> nulls;
+  2: optional list<i32> ints;
+}
+
+/**
+ * Elements of {@code nulls} array determine if a value for a corresponding row is null.
+ * Elements of {@code longs} array are values for each row. If row is null then value is ignored.
+ */
+struct TrinoThriftBigint {
+  1: optional list<bool> nulls;
+  2: optional list<i64> longs;
+}
+
+/**
+ * Elements of {@code nulls} array determine if a value for a corresponding row is null.
+ * Elements of {@code doubles} array are values for each row. If row is null then value is ignored.
+ */
+struct TrinoThriftDouble {
+  1: optional list<bool> nulls;
+  2: optional list<double> doubles;
+}
+
+/**
+ * Elements of {@code nulls} array determine if a value for a corresponding row is null.
+ * Each elements of {@code sizes} array contains the length in bytes for the corresponding element.
+ * If row is null then the corresponding element in {@code sizes} is ignored.
+ * {@code bytes} array contains UTF-8 encoded byte values.
+ * Values for all rows are written to {@code bytes} array one after another.
+ * The total number of bytes must be equal to the sum of all sizes.
+ */
+struct TrinoThriftVarchar {
+  1: optional list<bool> nulls;
+  2: optional list<i32> sizes;
+  3: optional binary bytes;
+}
+
+/**
+ * Elements of {@code nulls} array determine if a value for a corresponding row is null.
+ * Elements of {@code booleans} array are values for each row. If row is null then value is ignored.
+ */
+struct TrinoThriftBoolean {
+  1: optional list<bool> nulls;
+  2: optional list<bool> booleans;
+}
+
+/**
+ * Elements of {@code nulls} array determine if a value for a corresponding row is null.
+ * Elements of {@code dates} array are date values for each row represented as the number
+ * of days passed since 1970-01-01.
+ * If row is null then value is ignored.
+ */
+struct TrinoThriftDate {
+  1: optional list<bool> nulls;
+  2: optional list<i32> dates;
+}
+
+/**
+ * Elements of {@code nulls} array determine if a value for a corresponding row is null.
+ * Elements of {@code timestamps} array are values for each row represented as the number
+ * of milliseconds passed since 1970-01-01T00:00:00 UTC.
+ * If row is null then value is ignored.
+ */
+struct TrinoThriftTimestamp {
+  1: optional list<bool> nulls;
+  2: optional list<i64> timestamps;
+}
+
+/**
+ * Elements of {@code nulls} array determine if a value for a corresponding row is null.
+ * Each elements of {@code sizes} array contains the length in bytes for the corresponding element.
+ * If row is null then the corresponding element in {@code sizes} is ignored.
+ * {@code bytes} array contains UTF-8 encoded byte values for string representation of json.
+ * Values for all rows are written to {@code bytes} array one after another.
+ * The total number of bytes must be equal to the sum of all sizes.
+ */
+struct TrinoThriftJson {
+  1: optional list<bool> nulls;
+  2: optional list<i32> sizes;
+  3: optional binary bytes;
+}
+
+/**
+ * Elements of {@code nulls} array determine if a value for a corresponding row is null.
+ * Each elements of {@code sizes} array contains the length in bytes for the corresponding element.
+ * If row is null then the corresponding element in {@code sizes} is ignored.
+ * {@code bytes} array contains encoded byte values for HyperLogLog representation as defined in
+ * Airlift specification: href="https://github.com/airlift/airlift/blob/master/stats/docs/hll.md
+ * Values for all rows are written to {@code bytes} array one after another.
+ * The total number of bytes must be equal to the sum of all sizes.
+ */
+struct TrinoThriftHyperLogLog {
+  1: optional list<bool> nulls;
+  2: optional list<i32> sizes;
+  3: optional binary bytes;
+}
+
+/**
+ * Elements of {@code nulls} array determine if a value for a corresponding row is null.
+ * Each elements of {@code sizes} array contains the number of elements in the corresponding values array.
+ * If row is null then the corresponding element in {@code sizes} is ignored.
+ * {@code values} is a bigint block containing array elements one after another for all rows.
+ * The total number of elements in bigint block must be equal to the sum of all sizes.
+ */
+struct TrinoThriftBigintArray {
+  1: optional list<bool> nulls;
+  2: optional list<i32> sizes;
+  3: optional TrinoThriftBigint values;
+}
+
+/**
+ * A set containing zero or more Ranges of the same type over a continuous space of possible values.
+ * Ranges are coalesced into the most compact representation of non-overlapping Ranges.
+ * This structure is used with comparable and orderable types like bigint, integer, double, varchar, etc.
+ */
+struct TrinoThriftRangeValueSet {
+  1: list<TrinoThriftRange> ranges;
+}
+
+struct TrinoThriftId {
+  1: binary id;
+}
+
+struct TrinoThriftSplitBatch {
+  1: list<TrinoThriftSplit> splits;
+  2: optional TrinoThriftId nextToken;
+}
+
+struct TrinoThriftSplit {
+  /**
+   * Encodes all the information needed to identify a batch of rows to return to Trino.
+   * For a basic scan, includes schema name, table name, and output constraint.
+   * For an index scan, includes schema name, table name, set of keys to lookup and output constraint.
+   */
+  1: TrinoThriftId splitId;
+
+  /**
+   * Identifies the set of hosts on which the rows are available. If empty, then the rows
+   * are expected to be available on any host. The hosts in this list may be independent
+   * from the hosts used to serve metadata requests.
+   */
+  2: list<TrinoThriftHostAddress> hosts;
+}
+
+struct TrinoThriftHostAddress {
+  1: string host;
+  2: i32 port;
+}
+
+struct TrinoThriftPageResult {
+  /**
+   * Returns data in a columnar format.
+   * Columns in this list must be in the order they were requested by the engine.
+   */
+  1: list<TrinoThriftBlock> columnBlocks;
+
+  2: i32 rowCount;
+  3: optional TrinoThriftId nextToken;
+}
+
+struct TrinoThriftNullableTableMetadata {
+  1: optional TrinoThriftTableMetadata tableMetadata;
+}
+
+struct TrinoThriftValueSet {
+  1: optional TrinoThriftAllOrNoneValueSet allOrNoneValueSet;
+  2: optional TrinoThriftEquatableValueSet equatableValueSet;
+  3: optional TrinoThriftRangeValueSet rangeValueSet;
+}
+
+struct TrinoThriftBlock {
+  1: optional TrinoThriftInteger integerData;
+  2: optional TrinoThriftBigint bigintData;
+  3: optional TrinoThriftDouble doubleData;
+  4: optional TrinoThriftVarchar varcharData;
+  5: optional TrinoThriftBoolean booleanData;
+  6: optional TrinoThriftDate dateData;
+  7: optional TrinoThriftTimestamp timestampData;
+  8: optional TrinoThriftJson jsonData;
+  9: optional TrinoThriftHyperLogLog hyperLogLogData;
+  10: optional TrinoThriftBigintArray bigintArrayData;
+}
+
+/**
+ * LOWER UNBOUNDED is specified with an empty value and an ABOVE bound
+ * UPPER UNBOUNDED is specified with an empty value and a BELOW bound
+ */
+struct TrinoThriftMarker {
+  1: optional TrinoThriftBlock value;
+  2: TrinoThriftBound bound;
+}
+
+struct TrinoThriftNullableToken {
+  1: optional TrinoThriftId token;
+}
+
+struct TrinoThriftDomain {
+  1: TrinoThriftValueSet valueSet;
+  2: bool nullAllowed;
+}
+
+struct TrinoThriftRange {
+  1: TrinoThriftMarker low;
+  2: TrinoThriftMarker high;
+}
+
+/**
+ * Trino Thrift service definition.
+ * This thrift service needs to be implemented in order to be used with Thrift Connector.
+ */
+service TrinoThriftService {
+  /**
+   * Returns available schema names.
+   */
+  list<string> trinoListSchemaNames()
+    throws (1: TrinoThriftServiceException ex1);
+
+  /**
+   * Returns tables for the given schema name.
+   *
+   * @param schemaNameOrNull a structure containing schema name or {@literal null}
+   * @return a list of table names with corresponding schemas. If schema name is null then returns
+   * a list of tables for all schemas. Returns an empty list if a schema does not exist
+   */
+  list<TrinoThriftSchemaTableName> trinoListTables(
+      1: TrinoThriftNullableSchemaName schemaNameOrNull)
+    throws (1: TrinoThriftServiceException ex1);
+
+  /**
+   * Returns metadata for a given table.
+   *
+   * @param schemaTableName schema and table name
+   * @return metadata for a given table, or a {@literal null} value inside if it does not exist
+   */
+  TrinoThriftNullableTableMetadata trinoGetTableMetadata(
+      1: TrinoThriftSchemaTableName schemaTableName)
+    throws (1: TrinoThriftServiceException ex1);
+
+  /**
+   * Returns a batch of splits.
+   *
+   * @param schemaTableName schema and table name
+   * @param desiredColumns a superset of columns to return; empty set means "no columns", {@literal null} set means "all columns"
+   * @param outputConstraint constraint on the returned data
+   * @param maxSplitCount maximum number of splits to return
+   * @param nextToken token from a previous split batch or {@literal null} if it is the first call
+   * @return a batch of splits
+   */
+  TrinoThriftSplitBatch trinoGetSplits(
+      1: TrinoThriftSchemaTableName schemaTableName,
+      2: TrinoThriftNullableColumnSet desiredColumns,
+      3: TrinoThriftTupleDomain outputConstraint,
+      4: i32 maxSplitCount,
+      5: TrinoThriftNullableToken nextToken)
+    throws (1: TrinoThriftServiceException ex1);
+
+  /**
+   * Returns a batch of index splits for the given batch of keys.
+   * This method is called if index join strategy is chosen for a query.
+   *
+   * @param schemaTableName schema and table name
+   * @param indexColumnNames specifies columns and their order for keys
+   * @param outputColumnNames a list of column names to return
+   * @param keys keys for which records need to be returned; includes only unique and non-null values
+   * @param outputConstraint constraint on the returned data
+   * @param maxSplitCount maximum number of splits to return
+   * @param nextToken token from a previous split batch or {@literal null} if it is the first call
+   * @return a batch of splits
+   */
+  TrinoThriftSplitBatch trinoGetIndexSplits(
+      1: TrinoThriftSchemaTableName schemaTableName,
+      2: list<string> indexColumnNames,
+      3: list<string> outputColumnNames,
+      4: TrinoThriftPageResult keys,
+      5: TrinoThriftTupleDomain outputConstraint,
+      6: i32 maxSplitCount,
+      7: TrinoThriftNullableToken nextToken)
+    throws (1: TrinoThriftServiceException ex1);
+
+  /**
+   * Returns a batch of rows for the given split.
+   *
+   * @param splitId split id as returned in split batch
+   * @param columns a list of column names to return
+   * @param maxBytes maximum size of returned data in bytes
+   * @param nextToken token from a previous batch or {@literal null} if it is the first call
+   * @return a batch of table data
+   */
+  TrinoThriftPageResult trinoGetRows(
+      1: TrinoThriftId splitId,
+      2: list<string> columns,
+      3: i64 maxBytes,
+      4: TrinoThriftNullableToken nextToken)
+    throws (1: TrinoThriftServiceException ex1);
+}
+
+
+
+
+

Type mapping#

+

The Thrift service defines data type support and mappings to Trino data types.

+
+
+

SQL support#

+

The connector provides globally available and +read operation statements to access data and +metadata in your Thrift service.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/tpcds.html b/430/connector/tpcds.html new file mode 100644 index 000000000..1107ec590 --- /dev/null +++ b/430/connector/tpcds.html @@ -0,0 +1,740 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + TPCDS connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

TPCDS connector#

+

The TPCDS connector provides a set of schemas to support the +TPC Benchmark™ DS (TPC-DS). TPC-DS is a database +benchmark used to measure the performance of complex decision support databases.

+

This connector can be used to test the capabilities and query +syntax of Trino without configuring access to an external data +source. When you query a TPCDS schema, the connector generates the +data on the fly using a deterministic algorithm.

+
+

Configuration#

+

To configure the TPCDS connector, create a catalog properties file +etc/catalog/example.properties with the following contents:

+
connector.name=tpcds
+
+
+
+
+

TPCDS schemas#

+

The TPCDS connector supplies several schemas:

+
SHOW SCHEMAS FROM example;
+
+
+
       Schema
+--------------------
+ information_schema
+ sf1
+ sf10
+ sf100
+ sf1000
+ sf10000
+ sf100000
+ sf300
+ sf3000
+ sf30000
+ tiny
+(11 rows)
+
+
+

Ignore the standard schema information_schema, which exists in every +catalog, and is not directly provided by the TPCDS connector.

+

Every TPCDS schema provides the same set of tables. Some tables are +identical in all schemas. The scale factor of the tables in a particular +schema is determined from the schema name. For example, the schema +sf1 corresponds to scale factor 1 and the schema sf300 +corresponds to scale factor 300. Every unit in the scale factor +corresponds to a gigabyte of data. For example, for scale factor 300, +a total of 300 gigabytes are generated. The tiny schema is an +alias for scale factor 0.01, which is a very small data set useful for +testing.

+
+
+

Type mapping#

+

Trino supports all data types used within the TPCDS schemas so no mapping is +required.

+
+
+

SQL support#

+

The connector provides globally available and +read operation statements to access data and +metadata in the TPC-DS dataset.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/connector/tpch.html b/430/connector/tpch.html new file mode 100644 index 000000000..385a76515 --- /dev/null +++ b/430/connector/tpch.html @@ -0,0 +1,746 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + TPCH connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

TPCH connector#

+

The TPCH connector provides a set of schemas to support the +TPC Benchmark™ H (TPC-H). TPC-H is a database +benchmark used to measure the performance of highly-complex decision support databases.

+

This connector can be used to test the capabilities and query +syntax of Trino without configuring access to an external data +source. When you query a TPCH schema, the connector generates the +data on the fly using a deterministic algorithm.

+
+

Configuration#

+

To configure the TPCH connector, create a catalog properties file +etc/catalog/example.properties with the following contents:

+
connector.name=tpch
+
+
+

In the TPC-H specification, each column is assigned a prefix based on its +corresponding table name, such as l_ for the lineitem table. By default, the +TPCH connector simplifies column names by excluding these prefixes with the +default of tpch.column-naming to SIMPLIFIED. To use the long, standard +column names, use the configuration in the catalog properties file:

+
tpch.column-naming=STANDARD
+
+
+
+
+

TPCH schemas#

+

The TPCH connector supplies several schemas:

+
SHOW SCHEMAS FROM example;
+
+
+
       Schema
+--------------------
+ information_schema
+ sf1
+ sf100
+ sf1000
+ sf10000
+ sf100000
+ sf300
+ sf3000
+ sf30000
+ tiny
+(11 rows)
+
+
+

Ignore the standard schema information_schema, which exists in every +catalog, and is not directly provided by the TPCH connector.

+

Every TPCH schema provides the same set of tables. Some tables are +identical in all schemas. Other tables vary based on the scale factor, +which is determined based on the schema name. For example, the schema +sf1 corresponds to scale factor 1 and the schema sf300 +corresponds to scale factor 300. The TPCH connector provides an +infinite number of schemas for any scale factor, not just the few common +ones listed by SHOW SCHEMAS. The tiny schema is an alias for scale +factor 0.01, which is a very small data set useful for testing.

+
+
+

Type mapping#

+

Trino supports all data types used within the TPCH schemas so no mapping +is required.

+
+
+

SQL support#

+

The connector provides globally available and +read operation statements to access data and +metadata in the TPC-H dataset.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/develop.html b/430/develop.html new file mode 100644 index 000000000..0c10abbb4 --- /dev/null +++ b/430/develop.html @@ -0,0 +1,543 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Developer guide — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ + +
+
+
+ + + + + \ No newline at end of file diff --git a/430/develop/certificate-authenticator.html b/430/develop/certificate-authenticator.html new file mode 100644 index 000000000..1d989b459 --- /dev/null +++ b/430/develop/certificate-authenticator.html @@ -0,0 +1,565 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Certificate authenticator — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Certificate authenticator#

+

Trino supports TLS-based authentication with X509 certificates via a custom +certificate authenticator that extracts the principal from a client certificate.

+
+

Implementation#

+

CertificateAuthenticatorFactory is responsible for creating a +CertificateAuthenticator instance. It also defines the name of this +authenticator which is used by the administrator in a Trino configuration.

+

CertificateAuthenticator contains a single method, authenticate(), +which authenticates the client certificate and returns a Principal, which is then +authorized by the System access control.

+

The implementation of CertificateAuthenticatorFactory must be wrapped +as a plugin and installed on the Trino cluster.

+
+
+

Configuration#

+

After a plugin that implements CertificateAuthenticatorFactory has been +installed on the coordinator, it is configured using an +etc/certificate-authenticator.properties file. All of the +properties other than certificate-authenticator.name are specific to the +CertificateAuthenticatorFactory implementation.

+

The certificate-authenticator.name property is used by Trino to find a +registered CertificateAuthenticatorFactory based on the name returned by +CertificateAuthenticatorFactory.getName(). The remaining properties are +passed as a map to CertificateAuthenticatorFactory.create().

+

Example configuration file:

+
certificate-authenticator.name=custom
+custom-property1=custom-value1
+custom-property2=custom-value2
+
+
+

Additionally, the coordinator must be configured to use certificate authentication +and have HTTPS enabled (or HTTPS forwarding enabled).

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/develop/client-protocol.html b/430/develop/client-protocol.html new file mode 100644 index 000000000..4674e7922 --- /dev/null +++ b/430/develop/client-protocol.html @@ -0,0 +1,880 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Trino client REST API — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Trino client REST API#

+

The REST API allows clients to submit SQL queries to Trino and receive the +results. Clients include the CLI, the JDBC driver, and others provided by +the community. The preferred method to interact with Trino is using these +existing clients. This document provides details about the API for reference. +It can also be used to implement your own client, if necessary.

+
+

HTTP methods#

+
    +
  • A POST to /v1/statement runs the query string in the POST body, +and returns a JSON document containing the query results. If there are more +results, the JSON document contains a nextUri URL attribute.

  • +
  • A GET to the nextUri attribute returns the next batch of query results.

  • +
  • A DELETE to nextUri terminates a running query.

  • +
+
+
+

Overview of query processing#

+

A Trino client request is initiated by an HTTP POST to the endpoint +/v1/statement, with a POST body consisting of the SQL query string. +The caller may set various Client request headers. The headers are +only required on the initial POST request, and not when following the +nextUri links.

+

If the client request returns an HTTP 502, 503 or 504, that means there was +intermittent problem processing request and the client should try again +in 50-100 milliseconds. Trino does not generate those codes by itself +but those can be generated by gateways/load balancers in front of Trino. +Any HTTP status other than 502, 503, 504 or 200 means that query processing +has failed.

+

The /v1/statement POST request returns a JSON document of type +QueryResults, as well as a collection of response headers. The +QueryResults document contains an error field of type +QueryError if the query has failed, and if that object is not present, +the query succeeded. Important members of QueryResults are documented +in the following sections.

+

If the data field of the JSON document is set, it contains a list of the +rows of data. The columns field is set to a list of the +names and types of the columns returned by the query. Most of the response +headers are treated like browser cookies by the client, and echoed back +as request headers in subsequent client requests, as documented below.

+

If the JSON document returned by the POST to /v1/statement does not +contain a nextUri link, the query has completed, either successfully or +unsuccessfully, and no additional requests need to be made. If the +nextUri link is present in the document, there are more query results +to be fetched. The client should loop executing a GET request +to the nextUri returned in the QueryResults response object until +nextUri is absent from the response.

+

The status field of the JSON document is for human consumption only, and +provides a hint about the query state. It can not be used to tell if the +query is finished.

+
+
+

Important QueryResults attributes#

+

The most important attributes of the QueryResults JSON document returned by +the REST API endpoints are listed in this table. For more details, refer to the +class io.trino.client.QueryResults in module trino-client in the +client directory of the Trino source code.

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + + + + +
QueryResults attributes#

Attribute

Description

id

The ID of the query.

nextUri

If present, the URL to use for subsequent GET or +DELETE requests. If not present, the query is complete or +ended in error.

columns

A list of the names and types of the columns returned by the query.

data

The data attribute contains a list of the rows returned by the +query request. Each row is itself a list that holds values of the +columns in the row, in the order specified by the columns +attribute.

updateType

A human-readable string representing the operation. For a +CREATE TABLE request, the updateType is +“CREATE TABLE”; for SET SESSION it is “SET SESSION”; etc.

error

If query failed, the error attribute contains a QueryError object. +That object contains a message, an errorCode and other information +about the error. See the io.trino.client.QueryError class in module +trino-client in the client directory for more details.

+
+
+

QueryResults diagnostic attributes#

+

These QueryResults data members may be useful in tracking down problems:

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + +
QueryResults diagnostic attributes#

Attribute

Type

Description

queryError

QueryError

Non-null only if the query resulted in an error.

failureInfo

FailureInfo

failureInfo has detail on the reason for the failure, including +a stack trace, and FailureInfo.errorLocation, providing the +query line number and column number where the failure was detected.

warnings

List<TrinoWarning>

A usually-empty list of warnings.

statementStats

StatementStats

A class containing statistics about the query execution. Of +particular interest is StatementStats.rootStage, of type +StageStats, providing statistics on the execution of each of +the stages of query processing.

+
+
+

Client request headers#

+

This table lists all supported client request headers. Many of the +headers can be updated in the client as response headers, and supplied +in subsequent requests, just like browser cookies.

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Client request headers#

Header name

Description

X-Trino-User

Specifies the session user. If not supplied, the session user is +automatically determined via User mapping.

X-Trino-Original-User

Specifies the session’s original user.

X-Trino-Source

For reporting purposes, this supplies the name of the software +that submitted the query.

X-Trino-Catalog

The catalog context for query processing. Set by response +header X-Trino-Set-Catalog.

X-Trino-Schema

The schema context for query processing. Set by response +header X-Trino-Set-Schema.

X-Trino-Time-Zone

The timezone for query processing. Defaults to the timezone +of the Trino cluster, and not the timezone of the client.

X-Trino-Language

The language to use when processing the query and formatting +results, formatted as a Java Locale string, e.g., en-US +for US English. The language of the +session can be set on a per-query basis using the +X-Trino-Language HTTP header.

X-Trino-Trace-Token

Supplies a trace token to the Trino engine to help identify +log lines that originate with this query request.

X-Trino-Session

Supplies a comma-separated list of name=value pairs as session +properties. When the Trino client run a +SET SESSION name=value query, the name=value pair +is returned in the X-Set-Trino-Session response header, +and added to the client’s list of session properties. +If the response header X-Trino-Clear-Session is returned, +its value is the name of a session property that is +removed from the client’s accumulated list.

X-Trino-Role

Sets the “role” for query processing. A “role” represents +a collection of permissions. Set by response header +X-Trino-Set-Role. See CREATE ROLE to +understand roles.

X-Trino-Prepared-Statement

A comma-separated list of the name=value pairs, where the +names are names of previously prepared SQL statements, and +the values are keys that identify the executable form of the +named prepared statements.

X-Trino-Transaction-Id

The transaction ID to use for query processing. Set +by response header X-Trino-Started-Transaction-Id and +cleared by X-Trino-Clear-Transaction-Id.

X-Trino-Client-Info

Contains arbitrary information about the client program +submitting the query.

X-Trino-Client-Tags

A comma-separated list of “tag” strings, used to identify +Trino resource groups.

X-Trino-Resource-Estimate

A comma-separated list of resource=value type +assigments. The possible choices of resource are +EXECUTION_TIME, CPU_TIME, PEAK_MEMORY and +PEAK_TASK_MEMORY. EXECUTION_TIME and CPU_TIME +have values specified as airlift Duration strings +The format is a double precision number followed by +a TimeUnit string, e.g., of s for seconds, +m for minutes, h for hours, etc. “PEAK_MEMORY” and +“PEAK_TASK_MEMORY” are specified as as airlift DataSize strings, +whose format is an integer followed by B for bytes; kB for +kilobytes; mB for megabytes, gB for gigabytes, etc.

X-Trino-Extra-Credential

Provides extra credentials to the connector. The header is +a name=value string that is saved in the session Identity +object. The name and value are only meaningful to the connector.

+
+
+

Client response headers#

+

This table lists the supported client response headers. After receiving a +response, a client must update the request headers used in +subsequent requests to be consistent with the response headers received.

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Client response headers#

Header name

Description

X-Trino-Set-Catalog

Instructs the client to set the catalog in the +X-Trino-Catalog request header in subsequent client requests.

X-Trino-Set-Schema

Instructs the client to set the schema in the +X-Trino-Schema request header in subsequent client requests.

X-Trino-Set-Authorization-User

Instructs the client to set the session authorization user in the +X-Trino-Authorization-User request header in subsequent client requests.

X-Trino-Reset-Authorization-User

Instructs the client to remove X-Trino-Authorization-User request header +in subsequent client requests to reset the authorization user back to the +original user.

X-Trino-Set-Session

The value of the X-Trino-Set-Session response header is a +string of the form property = value. It +instructs the client include session property property with value +value in the X-Trino-Session header of subsequent +client requests.

X-Trino-Clear-Session

Instructs the client to remove the session property with the +whose name is the value of the X-Trino-Clear-Session header +from the list of session properties +in the X-Trino-Session header in subsequent client requests.

X-Trino-Set-Role

Instructs the client to set X-Trino-Role request header to the +catalog role supplied by the X-Trino-Set-Role header +in subsequent client requests.

X-Trino-Added-Prepare

Instructs the client to add the name=value pair to the set of +prepared statements in the X-Trino-Prepared-Statement +request header in subsequent client requests.

X-Trino-Deallocated-Prepare

Instructs the client to remove the prepared statement whose name +is the value of the X-Trino-Deallocated-Prepare header from +the client’s list of prepared statements sent in the +X-Trino-Prepared-Statement request header in subsequent client +requests.

X-Trino-Started-Transaction-Id

Provides the transaction ID that the client should pass back in the +X-Trino-Transaction-Id request header in subsequent requests.

X-Trino-Clear-Transaction-Id

Instructs the client to clear the X-Trino-Transaction-Id request +header in subsequent requests.

+
+
+

ProtocolHeaders#

+

Class io.trino.client.ProtocolHeaders in module trino-client in the +client directory of Trino source enumerates all the HTTP request and +response headers allowed by the Trino client REST API.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/develop/connectors.html b/430/develop/connectors.html new file mode 100644 index 000000000..3668c6d74 --- /dev/null +++ b/430/develop/connectors.html @@ -0,0 +1,1386 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Connectors — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Connectors#

+

Connectors are the source of all data for queries in Trino. Even if your data +source doesn’t have underlying tables backing it, as long as you adapt your data +source to the API expected by Trino, you can write queries against this data.

+
+

ConnectorFactory#

+

Instances of your connector are created by a ConnectorFactory instance which +is created when Trino calls getConnectorFactory() on the plugin. The +connector factory is a simple interface responsible for providing the connector +name and creating an instance of a Connector object. A basic connector +implementation that only supports reading, but not writing data, should return +instances of the following services:

+ +
+

Configuration#

+

The create() method of the connector factory receives a config map, +containing all properties from the catalog properties file. It can be used +to configure the connector, but because all the values are strings, they +might require additional processing if they represent other data types. +It also doesn’t validate if all the provided properties are known. This +can lead to the connector behaving differently than expected when a +connector ignores a property due to the user making a mistake in +typing the name of the property.

+

To make the configuration more robust, define a Configuration class. This +class describes all the available properties, their types, and additional +validation rules.

+
import io.airlift.configuration.Config;
+import io.airlift.configuration.ConfigDescription;
+import io.airlift.configuration.ConfigSecuritySensitive;
+import io.airlift.units.Duration;
+import io.airlift.units.MaxDuration;
+import io.airlift.units.MinDuration;
+
+import javax.validation.constraints.NotNull;
+
+public class ExampleConfig
+{
+    private String secret;
+    private Duration timeout = Duration.succinctDuration(10, TimeUnit.SECONDS);
+
+    public String getSecret()
+    {
+        return secret;
+    }
+
+    @Config("secret")
+    @ConfigDescription("Secret required to access the data source")
+    @ConfigSecuritySensitive
+    public ExampleConfig setSecret(String secret)
+    {
+        this.secret = secret;
+        return this;
+    }
+
+    @NotNull
+    @MaxDuration("10m")
+    @MinDuration("1ms")
+    public Duration getTimeout()
+    {
+        return timeout;
+    }
+
+    @Config("timeout")
+    public ExampleConfig setTimeout(Duration timeout)
+    {
+        this.timeout = timeout;
+        return this;
+    }
+}
+
+
+

The preceding example defines two configuration properties and makes +the connector more robust by:

+
    +
  • defining all supported properties, which allows detecting spelling mistakes +in the configuration on server startup

  • +
  • defining a default timeout value, to prevent connections getting stuck +indefinitely

  • +
  • preventing invalid timeout values, like 0 ms, that would make +all requests fail

  • +
  • parsing timeout values in different units, detecting invalid values

  • +
  • preventing logging the secret value in plain text

  • +
+

The configuration class needs to be bound in a Guice module:

+
import com.google.inject.Binder;
+import com.google.inject.Module;
+
+import static io.airlift.configuration.ConfigBinder.configBinder;
+
+public class ExampleModule
+        implements Module
+{
+    public ExampleModule()
+    {
+    }
+
+    @Override
+    public void configure(Binder binder)
+    {
+        configBinder(binder).bindConfig(ExampleConfig.class);
+    }
+}
+
+
+

And then the module needs to be initialized in the connector factory, when +creating a new instance of the connector:

+
@Override
+public Connector create(String connectorName, Map<String, String> config, ConnectorContext context)
+{
+    requireNonNull(config, "config is null");
+    Bootstrap app = new Bootstrap(new ExampleModule());
+    Injector injector = app
+            .doNotInitializeLogging()
+            .setRequiredConfigurationProperties(config)
+            .initialize();
+
+    return injector.getInstance(ExampleConnector.class);
+}
+
+
+
+

Note

+

Environment variables in the catalog properties file +(ex. secret=${ENV:SECRET}) are resolved only when using +the io.airlift.bootstrap.Bootstrap class to initialize the module. +See Secrets for more information.

+
+

If you end up needing to define multiple catalogs using the same connector +just to change one property, consider adding support for schema and/or +table properties. That would allow a more fine-grained configuration. +If a connector doesn’t support managing the schema, query predicates for +selected columns could be used as a way of passing the required configuration +at run time.

+

For example, when building a connector to read commits from a Git repository, +the repository URL could be a configuration property. But this would result +in a catalog being able to return data only from a single repository. +Alternatively, it can be a column, where every select query would require +a predicate for it:

+
SELECT *
+FROM git.default.commits
+WHERE url = 'https://github.com/trinodb/trino.git'
+
+
+
+
+
+

ConnectorMetadata#

+

The connector metadata interface allows Trino to get a lists of schemas, +tables, columns, and other metadata about a particular data source.

+

A basic read-only connector should implement the following methods:

+
    +
  • listSchemaNames

  • +
  • listTables

  • +
  • streamTableColumns

  • +
  • getTableHandle

  • +
  • getTableMetadata

  • +
  • getColumnHandles

  • +
  • getColumnMetadata

  • +
+

If you are interested in seeing strategies for implementing more methods, +look at the Example HTTP connector and the Cassandra connector. If your underlying +data source supports schemas, tables, and columns, this interface should be +straightforward to implement. If you are attempting to adapt something that +isn’t a relational database, as the Example HTTP connector does, you may +need to get creative about how you map your data source to Trino’s schema, +table, and column concepts.

+

The connector metadata interface allows to also implement other connector +features, like:

+
    +
  • Schema management, which is creating, altering and dropping schemas, tables, +table columns, views, and materialized views.

  • +
  • Support for table and column comments, and properties.

  • +
  • Schema, table and view authorization.

  • +
  • Executing Table functions.

  • +
  • Providing table statistics used by the Cost Based Optimizer (CBO) +and collecting statistics during writes and when analyzing selected tables.

  • +
  • Data modification, which is:

    +
      +
    • inserting, updating, and deleting rows in tables,

    • +
    • refreshing materialized views,

    • +
    • truncating whole tables,

    • +
    • and creating tables from query results.

    • +
    +
  • +
  • Role and grant management.

  • +
  • Pushing down:

    + +
  • +
+

Note that data modification also requires implementing +a ConnectorPageSinkProvider.

+

When Trino receives a SELECT query, it parses it into an Intermediate +Representation (IR). Then, during optimization, it checks if connectors +can handle operations related to SQL clauses by calling one of the following +methods of the ConnectorMetadata service:

+
    +
  • applyLimit

  • +
  • applyTopN

  • +
  • applyFilter

  • +
  • applyProjection

  • +
  • applySample

  • +
  • applyAggregation

  • +
  • applyJoin

  • +
  • applyTableFunction

  • +
  • applyTableScanRedirect

  • +
+

Connectors can indicate that they don’t support a particular pushdown or that +the action had no effect by returning Optional.empty(). Connectors should +expect these methods to be called multiple times during the optimization of +a given query.

+
+

Warning

+

It’s critical for connectors to return Optional.empty() if calling +this method has no effect for that invocation, even if the connector generally +supports a particular pushdown. Doing otherwise can cause the optimizer +to loop indefinitely.

+
+

Otherwise, these methods return a result object containing a new table handle. +The new table handle represents the virtual table derived from applying the +operation (filter, project, limit, etc.) to the table produced by the table +scan node. Once the query actually runs, ConnectorRecordSetProvider or +ConnectorPageSourceProvider can use whatever optimizations were pushed down to +ConnectorTableHandle.

+

The returned table handle is later passed to other services that the connector +implements, like the ConnectorRecordSetProvider or +ConnectorPageSourceProvider.

+
+

Limit and top-N pushdown#

+

When executing a SELECT query with LIMIT or ORDER BY clauses, +the query plan may contain a Sort or Limit operations.

+

When the plan contains a Sort and Limit operations, the engine +tries to push down the limit into the connector by calling the applyTopN +method of the connector metadata service. If there’s no Sort operation, but +only a Limit, the applyLimit method is called, and the connector can +return results in an arbitrary order.

+

If the connector could benefit from the information passed to these methods but +can’t guarantee that it’s be able to produce fewer rows than the provided +limit, it should return a non-empty result containing a new handle for the +derived table and the limitGuaranteed (in LimitApplicationResult) or +topNGuaranteed (in TopNApplicationResult) flag set to false.

+

If the connector can guarantee to produce fewer rows than the provided +limit, it should return a non-empty result with the “limit guaranteed” or +“topN guaranteed” flag set to true.

+
+

Note

+

The applyTopN is the only method that receives sort items from the +Sort operation.

+
+

In a query, the ORDER BY section can include any column with any order. +But the data source for the connector might only support limited combinations. +Plugin authors have to decide if the connector should ignore the pushdown, +return all the data and let the engine sort it, or throw an exception +to inform the user that particular order isn’t supported, if fetching all +the data would be too expensive or time consuming. When throwing +an exception, use the TrinoException class with the INVALID_ORDER_BY +error code and an actionable message, to let users know how to write a valid +query.

+
+
+

Predicate pushdown#

+

When executing a query with a WHERE clause, the query plan can +contain a ScanFilterProject plan node/node with a predicate constraint.

+

A predicate constraint is a description of the constraint imposed on the +results of the stage/fragment as expressed in the WHERE clause. For example, +WHERE x > 5 AND y = 3 translates into a constraint where the +summary field means the x column’s domain must be greater than +5 and the y column domain equals 3.

+

When the query plan contains a ScanFilterProject operation, Trino +tries to optimize the query by pushing down the predicate constraint +into the connector by calling the applyFilter method of the +connector metadata service. This method receives a table handle with +all optimizations applied thus far, and returns either +Optional.empty() or a response with a new table handle derived from +the old one.

+

The query optimizer may call applyFilter for a single query multiple times, +as it searches for an optimal query plan. Connectors must +return Optional.empty() from applyFilter if they cannot apply the +constraint for this invocation, even if they support ScanFilterProject +pushdown in general. Connectors must also return Optional.empty() if the +constraint has already been applied.

+

A constraint contains the following elements:

+
    +
  • A TupleDomain defining the mapping between columns and their domains. +A Domain is either a list of possible values, or a list of ranges, and +also contains information about nullability.

  • +
  • Expression for pushing down function calls.

  • +
  • Map of assignments from variables in the expression to columns.

  • +
  • (optional) Predicate which tests a map of columns and their values; +it cannot be held on to after the applyFilter call returns.

  • +
  • (optional) Set of columns the predicate depends on; must be present +if predicate is present.

  • +
+

If both a predicate and a summary are available, the predicate is guaranteed to +be more strict in filtering of values, and can provide a significant boost to +query performance if used.

+

However it is not possible to store a predicate in the table handle and use +it later, as the predicate cannot be held on to after the applyFilter +call returns. It is used for filtering of entire partitions, and is not pushed +down. The summary can be pushed down instead by storing it in the table handle.

+

This overlap between the predicate and summary is due to historical reasons, +as simple comparison pushdown was implemented first via summary, and more +complex filters such as LIKE which required more expressive predicates +were added later.

+

If a constraint can only be partially pushed down, for example when a connector +for a database that does not support range matching is used in a query with +WHERE x = 2 AND y > 5, the y column constraint must be +returned in the ConstraintApplicationResult from applyFilter. +In this case the y > 5 condition is applied in Trino, +and not pushed down.

+

The following is a simple example which only looks at TupleDomain:

+
@Override
+public Optional<ConstraintApplicationResult<ConnectorTableHandle>> applyFilter(
+        ConnectorSession session,
+        ConnectorTableHandle tableHandle,
+        Constraint constraint)
+{
+    ExampleTableHandle handle = (ExampleTableHandle) tableHandle;
+
+    TupleDomain<ColumnHandle> oldDomain = handle.getConstraint();
+    TupleDomain<ColumnHandle> newDomain = oldDomain.intersect(constraint.getSummary());
+    if (oldDomain.equals(newDomain)) {
+        // Nothing has changed, return empty Option
+        return Optional.empty();
+    }
+
+    handle = new ExampleTableHandle(newDomain);
+    return Optional.of(new ConstraintApplicationResult<>(handle, TupleDomain.all(), false));
+}
+
+
+

The TupleDomain from the constraint is intersected with the TupleDomain +already applied to the TableHandle to form newDomain. +If filtering has not changed, an Optional.empty() result is returned to +notify the planner that this optimization path has reached its end.

+

In this example, the connector pushes down the TupleDomain +with all Trino data types supported with same semantics in the +data source. As a result, no filters are needed in Trino, +and the ConstraintApplicationResult sets remainingFilter to +TupleDomain.all().

+

This pushdown implementation is quite similar to many Trino connectors, +including MongoMetadata, BigQueryMetadata, KafkaMetadata.

+

The following, more complex example shows data types from Trino that are +not available directly in the underlying data source, and must be mapped:

+
@Override
+public Optional<ConstraintApplicationResult<ConnectorTableHandle>> applyFilter(
+        ConnectorSession session,
+        ConnectorTableHandle table,
+        Constraint constraint)
+{
+    JdbcTableHandle handle = (JdbcTableHandle) table;
+
+    TupleDomain<ColumnHandle> oldDomain = handle.getConstraint();
+    TupleDomain<ColumnHandle> newDomain = oldDomain.intersect(constraint.getSummary());
+    TupleDomain<ColumnHandle> remainingFilter;
+    if (newDomain.isNone()) {
+        newConstraintExpressions = ImmutableList.of();
+        remainingFilter = TupleDomain.all();
+        remainingExpression = Optional.of(Constant.TRUE);
+    }
+    else {
+        // We need to decide which columns to push down.
+        // Since this is a base class for many JDBC-based connectors, each
+        // having different Trino type mappings and comparison semantics
+        // it needs to be flexible.
+
+        Map<ColumnHandle, Domain> domains = newDomain.getDomains().orElseThrow();
+        List<JdbcColumnHandle> columnHandles = domains.keySet().stream()
+                .map(JdbcColumnHandle.class::cast)
+                .collect(toImmutableList());
+
+        // Get information about how to push down every column based on its
+        // JDBC data type
+        List<ColumnMapping> columnMappings = jdbcClient.toColumnMappings(
+                session,
+                columnHandles.stream()
+                        .map(JdbcColumnHandle::getJdbcTypeHandle)
+                        .collect(toImmutableList()));
+
+        // Calculate the domains which can be safely pushed down (supported)
+        // and those which need to be filtered in Trino (unsupported)
+        Map<ColumnHandle, Domain> supported = new HashMap<>();
+        Map<ColumnHandle, Domain> unsupported = new HashMap<>();
+        for (int i = 0; i < columnHandles.size(); i++) {
+            JdbcColumnHandle column = columnHandles.get(i);
+            DomainPushdownResult pushdownResult =
+                columnMappings.get(i).getPredicatePushdownController().apply(
+                    session,
+                    domains.get(column));
+            supported.put(column, pushdownResult.getPushedDown());
+            unsupported.put(column, pushdownResult.getRemainingFilter());
+        }
+
+        newDomain = TupleDomain.withColumnDomains(supported);
+        remainingFilter = TupleDomain.withColumnDomains(unsupported);
+    }
+
+    // Return empty Optional if nothing changed in filtering
+    if (oldDomain.equals(newDomain)) {
+        return Optional.empty();
+    }
+
+    handle = new JdbcTableHandle(
+            handle.getRelationHandle(),
+            newDomain,
+            ...);
+
+    return Optional.of(
+            new ConstraintApplicationResult<>(
+                handle,
+                remainingFilter));
+}
+
+
+

This example illustrates implementing a base class for many JDBC connectors +while handling the specific requirements of multiple JDBC-compliant data sources. +It ensures that if a constraint gets pushed down, it works exactly the same in +the underlying data source, and produces the same results as it would in Trino. +For example, in databases where string comparisons are case-insensitive, +pushdown does not work, as string comparison operations in Trino are +case-sensitive.

+

The PredicatePushdownController interface determines if a column domain can +be pushed down in JDBC-compliant data sources. In the preceding example, it is +called from a JdbcClient implementation specific to that database. +In non-JDBC-compliant data sources, type-based push downs are implemented +directly, without going through the PredicatePushdownController interface.

+

The following example adds expression pushdown enabled by a session flag:

+
@Override
+public Optional<ConstraintApplicationResult<ConnectorTableHandle>> applyFilter(
+        ConnectorSession session,
+        ConnectorTableHandle table,
+        Constraint constraint)
+{
+    JdbcTableHandle handle = (JdbcTableHandle) table;
+
+    TupleDomain<ColumnHandle> oldDomain = handle.getConstraint();
+    TupleDomain<ColumnHandle> newDomain = oldDomain.intersect(constraint.getSummary());
+    List<String> newConstraintExpressions;
+    TupleDomain<ColumnHandle> remainingFilter;
+    Optional<ConnectorExpression> remainingExpression;
+    if (newDomain.isNone()) {
+        newConstraintExpressions = ImmutableList.of();
+        remainingFilter = TupleDomain.all();
+        remainingExpression = Optional.of(Constant.TRUE);
+    }
+    else {
+        // We need to decide which columns to push down.
+        // Since this is a base class for many JDBC-based connectors, each
+        // having different Trino type mappings and comparison semantics
+        // it needs to be flexible.
+
+        Map<ColumnHandle, Domain> domains = newDomain.getDomains().orElseThrow();
+        List<JdbcColumnHandle> columnHandles = domains.keySet().stream()
+                .map(JdbcColumnHandle.class::cast)
+                .collect(toImmutableList());
+
+        // Get information about how to push down every column based on its
+        // JDBC data type
+        List<ColumnMapping> columnMappings = jdbcClient.toColumnMappings(
+                session,
+                columnHandles.stream()
+                        .map(JdbcColumnHandle::getJdbcTypeHandle)
+                        .collect(toImmutableList()));
+
+        // Calculate the domains which can be safely pushed down (supported)
+        // and those which need to be filtered in Trino (unsupported)
+        Map<ColumnHandle, Domain> supported = new HashMap<>();
+        Map<ColumnHandle, Domain> unsupported = new HashMap<>();
+        for (int i = 0; i < columnHandles.size(); i++) {
+            JdbcColumnHandle column = columnHandles.get(i);
+            DomainPushdownResult pushdownResult =
+                columnMappings.get(i).getPredicatePushdownController().apply(
+                    session,
+                    domains.get(column));
+            supported.put(column, pushdownResult.getPushedDown());
+            unsupported.put(column, pushdownResult.getRemainingFilter());
+        }
+
+        newDomain = TupleDomain.withColumnDomains(supported);
+        remainingFilter = TupleDomain.withColumnDomains(unsupported);
+
+        // Do we want to handle expression pushdown?
+        if (isComplexExpressionPushdown(session)) {
+            List<String> newExpressions = new ArrayList<>();
+            List<ConnectorExpression> remainingExpressions = new ArrayList<>();
+            // Each expression can be broken down into a list of conjuncts
+            // joined with AND. We handle each conjunct separately.
+            for (ConnectorExpression expression : extractConjuncts(constraint.getExpression())) {
+                // Try to convert the conjunct into something which is
+                // understood by the underlying JDBC data source
+                Optional<String> converted = jdbcClient.convertPredicate(
+                    session,
+                    expression,
+                    constraint.getAssignments());
+                if (converted.isPresent()) {
+                    newExpressions.add(converted.get());
+                }
+                else {
+                    remainingExpressions.add(expression);
+                }
+            }
+            // Calculate which parts of the expression can be pushed down
+            // and which need to be calculated in Trino engine
+            newConstraintExpressions = ImmutableSet.<String>builder()
+                    .addAll(handle.getConstraintExpressions())
+                    .addAll(newExpressions)
+                    .build().asList();
+            remainingExpression = Optional.of(and(remainingExpressions));
+        }
+        else {
+            newConstraintExpressions = ImmutableList.of();
+            remainingExpression = Optional.empty();
+        }
+    }
+
+    // Return empty Optional if nothing changed in filtering
+    if (oldDomain.equals(newDomain) &&
+            handle.getConstraintExpressions().equals(newConstraintExpressions)) {
+        return Optional.empty();
+    }
+
+    handle = new JdbcTableHandle(
+            handle.getRelationHandle(),
+            newDomain,
+            newConstraintExpressions,
+            ...);
+
+    return Optional.of(
+            remainingExpression.isPresent()
+                    ? new ConstraintApplicationResult<>(
+                        handle,
+                        remainingFilter,
+                        remainingExpression.get())
+                    : new ConstraintApplicationResult<>(
+                        handle,
+                        remainingFilter));
+}
+
+
+

ConnectorExpression is split similarly to TupleDomain. +Each expression can be broken down into independent conjuncts. Conjuncts are +smaller expressions which, if joined together using an AND operator, are +equivalent to the original expression. Every conjunct can be handled +individually. Each one is converted using connector-specific rules, as defined +by the JdbcClient implementation, to be more flexible. Unconverted +conjuncts are returned as remainingExpression and are evaluated by +the Trino engine.

+
+
+
+

ConnectorSplitManager#

+

The split manager partitions the data for a table into the individual chunks +that Trino distributes to workers for processing. For example, the Hive +connector lists the files for each Hive partition and creates one or more +splits per file. For data sources that don’t have partitioned data, a good +strategy here is to simply return a single split for the entire table. This is +the strategy employed by the Example HTTP connector.

+
+
+

ConnectorRecordSetProvider#

+

Given a split, a table handle, and a list of columns, the record set provider +is responsible for delivering data to the Trino execution engine.

+

The table and column handles represent a virtual table. They’re created by the +connector’s metadata service, called by Trino during query planning and +optimization. Such a virtual table doesn’t have to map directly to a single +collection in the connector’s data source. If the connector supports pushdowns, +there can be multiple virtual tables derived from others, presenting a different +view of the underlying data.

+

The provider creates a RecordSet, which in turn creates a RecordCursor +that’s used by Trino to read the column values for each row.

+

The provided record set must only include requested columns in the order +matching the list of column handles passed to the +ConnectorRecordSetProvider.getRecordSet() method. The record set must return +all the rows contained in the “virtual table” represented by the TableHandle +associated with the TableScan operation.

+

For simple connectors, where performance isn’t critical, the record set +provider can return an instance of InMemoryRecordSet. The in-memory record +set can be built using lists of values for every row, which can be simpler than +implementing a RecordCursor.

+

A RecordCursor implementation needs to keep track of the current record. +It return values for columns by a numerical position, in the data type matching +the column definition in the table. When the engine is done reading the current +record it calls advanceNextPosition on the cursor.

+
+

Type mapping#

+

The built-in SQL data types use different Java types as carrier types.

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
SQL type to carrier type mapping#

SQL type

Java type

BOOLEAN

boolean

TINYINT

long

SMALLINT

long

INTEGER

long

BIGINT

long

REAL

double

DOUBLE

double

DECIMAL

long for precision up to 19, inclusive; +Int128 for precision greater than 19

VARCHAR

Slice

CHAR

Slice

VARBINARY

Slice

JSON

Slice

DATE

long

TIME(P)

long

TIME WITH TIME ZONE

long for precision up to 9; +LongTimeWithTimeZone for precision greater than 9

TIMESTAMP(P)

long for precision up to 6; +LongTimestamp for precision greater than 6

TIMESTAMP(P) WITH TIME ZONE

long for precision up to 3; +LongTimestampWithTimeZone for precision greater than 3

INTERVAL YEAR TO MONTH

long

INTERVAL DAY TO SECOND

long

ARRAY

Block

MAP

Block

ROW

Block

IPADDRESS

Slice

UUID

Slice

HyperLogLog

Slice

P4HyperLogLog

Slice

SetDigest

Slice

QDigest

Slice

TDigest

TDigest

+

The RecordCursor.getType(int field) method returns the SQL type for a field +and the field value is returned by one of the following methods, matching +the carrier type:

+
    +
  • getBoolean(int field)

  • +
  • getLong(int field)

  • +
  • getDouble(int field)

  • +
  • getSlice(int field)

  • +
  • getObject(int field)

  • +
+

Values for the timestamp(p) with time zone and time(p) with time zone +types of regular precision can be converted into long using static methods +from the io.trino.spi.type.DateTimeEncoding class, like pack() or +packDateTimeWithZone().

+

UTF-8 encoded strings can be converted to Slices using +the Slices.utf8Slice() static method.

+
+

Note

+

The Slice class is provided by the io.airlift:slice package.

+
+

Int128 objects can be created using the Int128.valueOf() method.

+

The following example creates a block for an array(varchar) column:

+
private Block encodeArray(List<String> names)
+{
+    BlockBuilder builder = VARCHAR.createBlockBuilder(null, names.size());
+    for (String name : names) {
+        if (name == null) {
+            builder.appendNull();
+        }
+        else {
+            VARCHAR.writeString(builder, name);
+        }
+    }
+    return builder.build();
+}
+
+
+

The following example creates a block for a map(varchar, varchar) column:

+
private Block encodeMap(Map<String, ?> map)
+{
+    MapType mapType = typeManager.getType(TypeSignature.mapType(
+                            VARCHAR.getTypeSignature(),
+                            VARCHAR.getTypeSignature()));
+    BlockBuilder values = mapType.createBlockBuilder(null, map != null ? map.size() : 0);
+    if (map == null) {
+        values.appendNull();
+        return values.build().getObject(0, Block.class);
+    }
+    BlockBuilder builder = values.beginBlockEntry();
+    for (Map.Entry<String, ?> entry : map.entrySet()) {
+        VARCHAR.writeString(builder, entry.getKey());
+        Object value = entry.getValue();
+        if (value == null) {
+            builder.appendNull();
+        }
+        else {
+            VARCHAR.writeString(builder, value.toString());
+        }
+    }
+    values.closeEntry();
+    return values.build().getObject(0, Block.class);
+}
+
+
+
+
+
+

ConnectorPageSourceProvider#

+

Given a split, a table handle, and a list of columns, the page source provider +is responsible for delivering data to the Trino execution engine. It creates +a ConnectorPageSource, which in turn creates Page objects that are used +by Trino to read the column values.

+

If not implemented, a default RecordPageSourceProvider is used. +Given a record set provider, it returns an instance of RecordPageSource +that builds Page objects from records in a record set.

+

A connector should implement a page source provider instead of a record set +provider when it’s possible to create pages directly. The conversion of +individual records from a record set provider into pages adds overheads during +query execution.

+
+
+

ConnectorPageSinkProvider#

+

Given an insert table handle, the page sink provider is responsible for +consuming data from the Trino execution engine. +It creates a ConnectorPageSink, which in turn accepts Page objects +that contains the column values.

+

Example that shows how to iterate over the page to access single values:

+
@Override
+public CompletableFuture<?> appendPage(Page page)
+{
+    for (int channel = 0; channel < page.getChannelCount(); channel++) {
+        Block block = page.getBlock(channel);
+        for (int position = 0; position < page.getPositionCount(); position++) {
+            if (block.isNull(position)) {
+                // or handle this differently
+                continue;
+            }
+
+            // channel should match the column number in the table
+            // use it to determine the expected column type
+            String value = VARCHAR.getSlice(block, position).toStringUtf8();
+            // TODO do something with the value
+        }
+    }
+    return NOT_BLOCKED;
+}
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/develop/event-listener.html b/430/develop/event-listener.html new file mode 100644 index 000000000..d1515c18e --- /dev/null +++ b/430/develop/event-listener.html @@ -0,0 +1,585 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Event listener — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Event listener#

+

Trino supports custom event listeners that are invoked for the following +events:

+
    +
  • Query creation

  • +
  • Query completion (success or failure)

  • +
  • Split completion (success or failure)

  • +
+

Event details include session, query execution, resource utilization, timeline, +and more.

+

This functionality enables development of custom logging, debugging and +performance analysis plugins.

+
+

Implementation#

+

EventListenerFactory is responsible for creating an +EventListener instance. It also defines an EventListener +name which is used by the administrator in a Trino configuration. +Implementations of EventListener implement methods for the event types +they are interested in handling.

+

The implementation of EventListener and EventListenerFactory +must be wrapped as a plugin and installed on the Trino cluster.

+
+
+

Configuration#

+

After a plugin that implements EventListener and +EventListenerFactory has been installed on the coordinator, it is +configured using an etc/event-listener.properties file. All of the +properties other than event-listener.name are specific to the +EventListener implementation.

+

The event-listener.name property is used by Trino to find a registered +EventListenerFactory based on the name returned by +EventListenerFactory.getName(). The remaining properties are passed +as a map to EventListenerFactory.create().

+

Example configuration file:

+
event-listener.name=custom-event-listener
+custom-property1=custom-value1
+custom-property2=custom-value2
+
+
+
+
+

Multiple event listeners#

+

Trino supports multiple instances of the same or different event listeners. +Install and configure multiple instances by setting +event-listener.config-files in Config properties to a comma-separated +list of the event listener configuration files:

+
event-listener.config-files=etc/event-listener.properties,etc/event-listener-second.properties
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/develop/example-http.html b/430/develop/example-http.html new file mode 100644 index 000000000..2103b9059 --- /dev/null +++ b/430/develop/example-http.html @@ -0,0 +1,648 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Example HTTP connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Example HTTP connector#

+

The Example HTTP connector has a simple goal: it reads comma-separated +data over HTTP. For example, if you have a large amount of data in a +CSV format, you can point the example HTTP connector at this data and +write a query to process it.

+
+

Code#

+

The Example HTTP connector can be found in the trino-example-http +directory within the Trino source tree.

+
+
+

Plugin implementation#

+

The plugin implementation in the Example HTTP connector looks very +similar to other plugin implementations. Most of the implementation is +devoted to handling optional configuration and the only function of +interest is the following:

+
@Override
+public Iterable<ConnectorFactory> getConnectorFactories()
+{
+    return ImmutableList.of(new ExampleConnectorFactory());
+}
+
+
+

Note that the ImmutableList class is a utility class from Guava.

+

As with all connectors, this plugin overrides the getConnectorFactories() method +and returns an ExampleConnectorFactory.

+
+
+

ConnectorFactory implementation#

+

In Trino, the primary object that handles the connection between +Trino and a particular type of data source is the Connector object, +which are created using ConnectorFactory.

+

This implementation is available in the class ExampleConnectorFactory. +The first thing the connector factory implementation does is specify the +name of this connector. This is the same string used to reference this +connector in Trino configuration.

+
@Override
+public String getName()
+{
+    return "example_http";
+}
+
+
+

The real work in a connector factory happens in the create() +method. In the ExampleConnectorFactory class, the create() method +configures the connector and then asks Guice to create the object. +This is the meat of the create() method without parameter validation +and exception handling:

+
// A plugin is not required to use Guice; it is just very convenient
+Bootstrap app = new Bootstrap(
+        new JsonModule(),
+        new ExampleModule(catalogName));
+
+Injector injector = app
+        .doNotInitializeLogging()
+        .setRequiredConfigurationProperties(requiredConfig)
+        .initialize();
+
+return injector.getInstance(ExampleConnector.class);
+
+
+
+

Connector: ExampleConnector#

+

This class allows Trino to obtain references to the various services +provided by the connector.

+
+
+

Metadata: ExampleMetadata#

+

This class is responsible for reporting table names, table metadata, +column names, column metadata and other information about the schemas +that are provided by this connector. ConnectorMetadata is also called +by Trino to ensure that a particular connector can understand and +handle a given table name.

+

The ExampleMetadata implementation delegates many of these calls to +ExampleClient, a class that implements much of the core functionality +of the connector.

+
+
+

Split manager: ExampleSplitManager#

+

The split manager partitions the data for a table into the individual +chunks that Trino will distribute to workers for processing. +In the case of the Example HTTP connector, each table contains one or +more URIs pointing at the actual data. One split is created per URI.

+
+
+

Record set provider: ExampleRecordSetProvider#

+

The record set provider creates a record set which in turn creates a +record cursor that returns the actual data to Trino. +ExampleRecordCursor reads data from a URI via HTTP. Each line +corresponds to a single row. Lines are split on comma into individual +field values which are then returned to Trino.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/develop/example-jdbc.html b/430/develop/example-jdbc.html new file mode 100644 index 000000000..95c2e15cc --- /dev/null +++ b/430/develop/example-jdbc.html @@ -0,0 +1,609 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Example JDBC connector — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Example JDBC connector#

+

The Example JDBC connector shows how to extend the base JdbcPlugin +to read data from a source using a JDBC driver, without having +to implement different Trino SPI services, like ConnectorMetadata +or ConnectorRecordSetProvider.

+
+

Note

+

This connector is just an example. It supports a very limited set of data +types and does not support any advanced functions, like predicacte or other +kind of pushdowns.

+
+
+

Code#

+

The Example JDBC connector can be found in the trino-example-jdbc +directory within the Trino source tree.

+
+
+

Plugin implementation#

+

The plugin implementation in the Example JDBC connector extends +the JdbcPlugin class and uses the ExampleClientModule.

+

The module:

+
    +
  • binds the ExampleClient class so it can be used by the base JDBC +connector;

  • +
  • provides a connection factory that will create new connections using a JDBC +driver based on the JDBC URL specified in configuration properties.

  • +
+
+
+

JdbcClient implementation#

+

The base JDBC plugin maps the Trino SPI calls to the JDBC API. Operations like +reading table and columns names are well defined in JDBC so the base JDBC plugin +can implement it in a way that works for most JDBC drivers.

+

One behavior that is not implemented by default is mapping of the data types +when reading and writing data. The Example JDBC connector implements +the JdbcClient interface in the ExampleClient class that extends +the BaseJdbcClient and implements two methods.

+
+

toColumnMapping#

+

toColumnMapping is used when reading data from the connector. +Given a ConnectorSession, Connection and a JdbcTypeHandle, +it returns a ColumnMapping, if there is a matching data type.

+

The column mapping includes:

+
    +
  • a Trino type,

  • +
  • a write function, used to set query parameter values when preparing a +JDBC statement to execute in the data source,

  • +
  • and a read function, used to read a value from the JDBC statement result set, +and return it using an internal Trino representation (for example, a Slice).

  • +
+
+
+

toWriteMapping#

+

toWriteMapping is used when writing data to the connector. Given a +ConnectorSession and a Trino type, it returns a WriteMapping.

+

The mapping includes:

+
    +
  • a data type name

  • +
  • a write function

  • +
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/develop/functions.html b/430/develop/functions.html new file mode 100644 index 000000000..b38d81d55 --- /dev/null +++ b/430/develop/functions.html @@ -0,0 +1,847 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Functions — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Functions#

+
+

Plugin implementation#

+

The function framework is used to implement SQL functions. Trino includes a +number of built-in functions. In order to implement new functions, you can +write a plugin that returns one or more functions from getFunctions():

+
public class ExampleFunctionsPlugin
+        implements Plugin
+{
+    @Override
+    public Set<Class<?>> getFunctions()
+    {
+        return ImmutableSet.<Class<?>>builder()
+                .add(ExampleNullFunction.class)
+                .add(IsNullFunction.class)
+                .add(IsEqualOrNullFunction.class)
+                .add(ExampleStringFunction.class)
+                .add(ExampleAverageFunction.class)
+                .build();
+    }
+}
+
+
+

Note that the ImmutableSet class is a utility class from Guava. +The getFunctions() method contains all of the classes for the functions +that we will implement below in this tutorial.

+

For a full example in the codebase, see either the trino-ml module for +machine learning functions or the trino-teradata-functions module for +Teradata-compatible functions, both in the plugin directory of the Trino +source.

+
+
+

Scalar function implementation#

+

The function framework uses annotations to indicate relevant information +about functions, including name, description, return type and parameter +types. Below is a sample function which implements is_null:

+
public class ExampleNullFunction
+{
+    @ScalarFunction("is_null", deterministic = true)
+    @Description("Returns TRUE if the argument is NULL")
+    @SqlType(StandardTypes.BOOLEAN)
+    public static boolean isNull(
+            @SqlNullable @SqlType(StandardTypes.VARCHAR) Slice string)
+    {
+        return (string == null);
+    }
+}
+
+
+

The function is_null takes a single VARCHAR argument and returns a +BOOLEAN indicating if the argument was NULL. Note that the argument to +the function is of type Slice. VARCHAR uses Slice, which is essentially +a wrapper around byte[], rather than String for its native container type.

+

The deterministic argument indicates that a function has no side effects and, +for subsequent calls with the same argument(s), the function returns the exact +same value(s).

+

In Trino, deterministic functions don’t rely on any changing state +and don’t modify any state. The deterministic flag is optional and defaults +to true.

+

For example, the function shuffle() is non-deterministic, since it uses random +values. On the other hand, now() is deterministic, because subsequent calls in a +single query return the same timestamp.

+

Any function with non-deterministic behavior is required to set deterministic = false +to avoid unexpected results.

+
    +
  • @SqlType:

    +

    The @SqlType annotation is used to declare the return type and the argument +types. Note that the return type and arguments of the Java code must match +the native container types of the corresponding annotations.

    +
  • +
  • @SqlNullable:

    +

    The @SqlNullable annotation indicates that the argument may be NULL. Without +this annotation the framework assumes that all functions return NULL if +any of their arguments are NULL. When working with a Type that has a +primitive native container type, such as BigintType, use the object wrapper for the +native container type when using @SqlNullable. The method must be annotated with +@SqlNullable if it can return NULL when the arguments are non-null.

    +
  • +
+
+
+

Parametric scalar functions#

+

Scalar functions that have type parameters have some additional complexity. +To make our previous example work with any type we need the following:

+
@ScalarFunction(name = "is_null")
+@Description("Returns TRUE if the argument is NULL")
+public final class IsNullFunction
+{
+    @TypeParameter("T")
+    @SqlType(StandardTypes.BOOLEAN)
+    public static boolean isNullSlice(@SqlNullable @SqlType("T") Slice value)
+    {
+        return (value == null);
+    }
+
+    @TypeParameter("T")
+    @SqlType(StandardTypes.BOOLEAN)
+    public static boolean isNullLong(@SqlNullable @SqlType("T") Long value)
+    {
+        return (value == null);
+    }
+
+    @TypeParameter("T")
+    @SqlType(StandardTypes.BOOLEAN)
+    public static boolean isNullDouble(@SqlNullable @SqlType("T") Double value)
+    {
+        return (value == null);
+    }
+
+    // ...and so on for each native container type
+}
+
+
+
    +
  • @TypeParameter:

    +

    The @TypeParameter annotation is used to declare a type parameter which can +be used in the argument types @SqlType annotation, or return type of the function. +It can also be used to annotate a parameter of type Type. At runtime, the engine +will bind the concrete type to this parameter. @OperatorDependency may be used +to declare that an additional function for operating on the given type parameter is needed. +For example, the following function will only bind to types which have an equals function +defined:

    +
  • +
+
@ScalarFunction(name = "is_equal_or_null")
+@Description("Returns TRUE if arguments are equal or both NULL")
+public final class IsEqualOrNullFunction
+{
+    @TypeParameter("T")
+    @SqlType(StandardTypes.BOOLEAN)
+    public static boolean isEqualOrNullSlice(
+            @OperatorDependency(
+                    operator = OperatorType.EQUAL,
+                    returnType = StandardTypes.BOOLEAN,
+                    argumentTypes = {"T", "T"}) MethodHandle equals,
+            @SqlNullable @SqlType("T") Slice value1,
+            @SqlNullable @SqlType("T") Slice value2)
+    {
+        if (value1 == null && value2 == null) {
+            return true;
+        }
+        if (value1 == null || value2 == null) {
+            return false;
+        }
+        return (boolean) equals.invokeExact(value1, value2);
+    }
+
+    // ...and so on for each native container type
+}
+
+
+
+
+

Another scalar function example#

+

The lowercaser function takes a single VARCHAR argument and returns a +VARCHAR, which is the argument converted to lower case:

+
public class ExampleStringFunction
+{
+    @ScalarFunction("lowercaser")
+    @Description("Converts the string to alternating case")
+    @SqlType(StandardTypes.VARCHAR)
+    public static Slice lowercaser(@SqlType(StandardTypes.VARCHAR) Slice slice)
+    {
+        String argument = slice.toStringUtf8();
+        return Slices.utf8Slice(argument.toLowerCase());
+    }
+}
+
+
+

Note that for most common string functions, including converting a string to +lower case, the Slice library also provides implementations that work directly +on the underlying byte[], which have much better performance. This function +has no @SqlNullable annotations, meaning that if the argument is NULL, +the result will automatically be NULL (the function will not be called).

+
+
+

Aggregation function implementation#

+

Aggregation functions use a similar framework to scalar functions, but are +a bit more complex.

+
    +
  • AccumulatorState:

    +

    All aggregation functions accumulate input rows into a state object; this +object must implement AccumulatorState. For simple aggregations, just +extend AccumulatorState into a new interface with the getters and setters +you want, and the framework will generate all the implementations and +serializers for you. If you need a more complex state object, you will need +to implement AccumulatorStateFactory and AccumulatorStateSerializer +and provide these via the AccumulatorStateMetadata annotation.

    +
  • +
+

The following code implements the aggregation function avg_double which computes the +average of a DOUBLE column:

+
@AggregationFunction("avg_double")
+public class AverageAggregation
+{
+    @InputFunction
+    public static void input(
+            LongAndDoubleState state,
+            @SqlType(StandardTypes.DOUBLE) double value)
+    {
+        state.setLong(state.getLong() + 1);
+        state.setDouble(state.getDouble() + value);
+    }
+
+    @CombineFunction
+    public static void combine(
+            LongAndDoubleState state,
+            LongAndDoubleState otherState)
+    {
+        state.setLong(state.getLong() + otherState.getLong());
+        state.setDouble(state.getDouble() + otherState.getDouble());
+    }
+
+    @OutputFunction(StandardTypes.DOUBLE)
+    public static void output(LongAndDoubleState state, BlockBuilder out)
+    {
+        long count = state.getLong();
+        if (count == 0) {
+            out.appendNull();
+        }
+        else {
+            double value = state.getDouble();
+            DOUBLE.writeDouble(out, value / count);
+        }
+    }
+}
+
+
+

The average has two parts: the sum of the DOUBLE in each row of the column +and the LONG count of the number of rows seen. LongAndDoubleState is an interface +which extends AccumulatorState:

+
public interface LongAndDoubleState
+        extends AccumulatorState
+{
+    long getLong();
+
+    void setLong(long value);
+
+    double getDouble();
+
+    void setDouble(double value);
+}
+
+
+

As stated above, for simple AccumulatorState objects, it is sufficient to +just to define the interface with the getters and setters, and the framework +will generate the implementation for you.

+

An in-depth look at the various annotations relevant to writing an aggregation +function follows:

+
    +
  • @InputFunction:

    +

    The @InputFunction annotation declares the function which accepts input +rows and stores them in the AccumulatorState. Similar to scalar functions +you must annotate the arguments with @SqlType. Note that, unlike in the above +scalar example where Slice is used to hold VARCHAR, the primitive +double type is used for the argument to input. In this example, the input +function simply keeps track of the running count of rows (via setLong()) +and the running sum (via setDouble()).

    +
  • +
  • @CombineFunction:

    +

    The @CombineFunction annotation declares the function used to combine two +state objects. This function is used to merge all the partial aggregation states. +It takes two state objects, and merges the results into the first one (in the +above example, just by adding them together).

    +
  • +
  • @OutputFunction:

    +

    The @OutputFunction is the last function called when computing an +aggregation. It takes the final state object (the result of merging all +partial states) and writes the result to a BlockBuilder.

    +
  • +
  • Where does serialization happen, and what is GroupedAccumulatorState?

    +

    The @InputFunction is usually run on a different worker from the +@CombineFunction, so the state objects are serialized and transported +between these workers by the aggregation framework. GroupedAccumulatorState +is used when performing a GROUP BY aggregation, and an implementation +will be automatically generated for you, if you don’t specify a +AccumulatorStateFactory

    +
  • +
+
+
+

Deprecated function#

+

The @Deprecated annotation has to be used on any function that should no longer be +used. The annotation causes Trino to generate a warning whenever SQL statements +use a deprecated function. When a function is deprecated, the @Description +needs to be replaced with a note about the deprecation and the replacement function:

+
public class ExampleDeprecatedFunction
+{
+    @Deprecated
+    @ScalarFunction("bad_function")
+    @Description("(DEPRECATED) Use good_function() instead")
+    @SqlType(StandardTypes.BOOLEAN)
+    public static boolean bad_function()
+    {
+        return false;
+    }
+}
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/develop/group-provider.html b/430/develop/group-provider.html new file mode 100644 index 000000000..e4d4fe0d9 --- /dev/null +++ b/430/develop/group-provider.html @@ -0,0 +1,564 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Group provider — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Group provider#

+

Trino can map user names onto groups for easier access control management. +This mapping is performed by a GroupProvider implementation.

+
+

Implementation#

+

GroupProviderFactory is responsible for creating a GroupProvider instance. +It also defines the name of the group provider as used in the configuration file.

+

GroupProvider contains a one method, getGroups(String user) +which returns a Set<String> of group names. +This set of group names becomes part of the Identity and ConnectorIdentity +objects representing the user, and can then be used by System access control.

+

The implementation of GroupProvider and its corresponding GroupProviderFactory +must be wrapped as a Trino plugin and installed on the cluster.

+
+
+

Configuration#

+

After a plugin that implements GroupProviderFactory has been installed on the coordinator, +it is configured using an etc/group-provider.properties file. +All of the properties other than group-provider.name are specific to +the GroupProviderFactory implementation.

+

The group-provider.name property is used by Trino to find a registered +GroupProviderFactory based on the name returned by GroupProviderFactory.getName(). +The remaining properties are passed as a map to +GroupProviderFactory.create(Map<String, String>).

+

Example configuration file:

+
group-provider.name=custom-group-provider
+custom-property1=custom-value1
+custom-property2=custom-value2
+
+
+

With that file in place, Trino will attempt user group name resolution, +and will be able to use the group names while evaluating access control rules.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/develop/header-authenticator.html b/430/develop/header-authenticator.html new file mode 100644 index 000000000..6db070ad3 --- /dev/null +++ b/430/develop/header-authenticator.html @@ -0,0 +1,566 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Header authenticator — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Header authenticator#

+

Trino supports header authentication over TLS via a custom header authenticator +that extracts the principal from a predefined header(s), performs any validation it needs and creates +an authenticated principal.

+
+

Implementation#

+

HeaderAuthenticatorFactory is responsible for creating a +HeaderAuthenticator instance. It also defines the name of this +authenticator which is used by the administrator in a Trino configuration.

+

HeaderAuthenticator contains a single method, createAuthenticatedPrincipal(), +which validates the request headers wrapped by the Headers interface; has the method getHeader(String name) +and returns a Principal, which is then authorized by the System access control.

+

The implementation of HeaderAuthenticatorFactory must be wrapped +as a plugin and installed on the Trino cluster.

+
+
+

Configuration#

+

After a plugin that implements HeaderAuthenticatorFactory has been +installed on the coordinator, it is configured using an +etc/header-authenticator.properties file. All of the +properties other than header-authenticator.name are specific to the +HeaderAuthenticatorFactory implementation.

+

The header-authenticator.name property is used by Trino to find a +registered HeaderAuthenticatorFactory based on the name returned by +HeaderAuthenticatorFactory.getName(). The remaining properties are +passed as a map to HeaderAuthenticatorFactory.create().

+

Example configuration file:

+
header-authenticator.name=custom
+custom-property1=custom-value1
+custom-property2=custom-value2
+
+
+

Additionally, the coordinator must be configured to use header authentication +and have HTTPS enabled (or HTTPS forwarding enabled).

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/develop/insert.html b/430/develop/insert.html new file mode 100644 index 000000000..77bbf28eb --- /dev/null +++ b/430/develop/insert.html @@ -0,0 +1,545 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Supporting INSERT and CREATE TABLE AS — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Supporting INSERT and CREATE TABLE AS#

+

To support INSERT, a connector must implement:

+
    +
  • beginInsert() and finishInsert() from the ConnectorMetadata +interface;

  • +
  • a ConnectorPageSinkProvider that receives a table handle and returns +a ConnectorPageSink.

  • +
+

When executing an INSERT statement, the engine calls the beginInsert() +method in the connector, which receives a table handle and a list of columns. +It should return a ConnectorInsertTableHandle, that can carry any +connector specific information, and it’s passed to the page sink provider. +The PageSinkProvider creates a page sink, that accepts Page objects.

+

When all the pages for a specific split have been processed, Trino calls +ConnectorPageSink.finish(), which returns a Collection<Slice> +of fragments representing connector-specific information about the processed +rows.

+

When all pages for all splits have been processed, Trino calls +ConnectorMetadata.finishInsert(), passing a collection containing all +the fragments from all the splits. The connector does what is required +to finalize the operation, for example, committing the transaction.

+

To support CREATE TABLE AS, the ConnectorPageSinkProvider must also +return a page sink when receiving a ConnectorOutputTableHandle. This handle +is returned from ConnectorMetadata.beginCreateTable().

+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/develop/password-authenticator.html b/430/develop/password-authenticator.html new file mode 100644 index 000000000..1647503ef --- /dev/null +++ b/430/develop/password-authenticator.html @@ -0,0 +1,565 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Password authenticator — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Password authenticator#

+

Trino supports authentication with a username and password via a custom +password authenticator that validates the credentials and creates a principal.

+
+

Implementation#

+

PasswordAuthenticatorFactory is responsible for creating a +PasswordAuthenticator instance. It also defines the name of this +authenticator which is used by the administrator in a Trino configuration.

+

PasswordAuthenticator contains a single method, createAuthenticatedPrincipal(), +that validates the credential and returns a Principal, which is then +authorized by the System access control.

+

The implementation of PasswordAuthenticatorFactory must be wrapped +as a plugin and installed on the Trino cluster.

+
+
+

Configuration#

+

After a plugin that implements PasswordAuthenticatorFactory has been +installed on the coordinator, it is configured using an +etc/password-authenticator.properties file. All of the +properties other than password-authenticator.name are specific to the +PasswordAuthenticatorFactory implementation.

+

The password-authenticator.name property is used by Trino to find a +registered PasswordAuthenticatorFactory based on the name returned by +PasswordAuthenticatorFactory.getName(). The remaining properties are +passed as a map to PasswordAuthenticatorFactory.create().

+

Example configuration file:

+
password-authenticator.name=custom-access-control
+custom-property1=custom-value1
+custom-property2=custom-value2
+
+
+

Additionally, the coordinator must be configured to use password authentication +and have HTTPS enabled (or HTTPS forwarding enabled).

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/develop/spi-overview.html b/430/develop/spi-overview.html new file mode 100644 index 000000000..514f0bb05 --- /dev/null +++ b/430/develop/spi-overview.html @@ -0,0 +1,643 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SPI overview — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+ +
+
+ +
+
+ +
+

SPI overview#

+

When you implement a new Trino plugin, you implement interfaces and +override methods defined by the Service Provider Interface (SPI).

+

Plugins can provide additional:

+ +

In particular, connectors are the source of all data for queries in +Trino: they back each catalog available to Trino.

+
+

Code#

+

The SPI source can be found in the core/trino-spi directory in the Trino +source tree.

+
+
+

Plugin metadata#

+

Each plugin identifies an entry point: an implementation of the +Plugin interface. This class name is provided to Trino via +the standard Java ServiceLoader interface: the classpath contains +a resource file named io.trino.spi.Plugin in the +META-INF/services directory. The content of this file is a +single line listing the name of the plugin class:

+
com.example.plugin.ExamplePlugin
+
+
+

For a built-in plugin that is included in the Trino source code, +this resource file is created whenever the pom.xml file of a plugin +contains the following line:

+
<packaging>trino-plugin</packaging>
+
+
+
+
+

Plugin#

+

The Plugin interface is a good starting place for developers looking +to understand the Trino SPI. It contains access methods to retrieve +various classes that a Plugin can provide. For example, the getConnectorFactories() +method is a top-level function that Trino calls to retrieve a ConnectorFactory when Trino +is ready to create an instance of a connector to back a catalog. There are similar +methods for Type, ParametricType, Function, SystemAccessControl, and +EventListenerFactory objects.

+
+
+

Building plugins via Maven#

+

Plugins depend on the SPI from Trino:

+
<dependency>
+    <groupId>io.trino</groupId>
+    <artifactId>trino-spi</artifactId>
+    <scope>provided</scope>
+</dependency>
+
+
+

The plugin uses the Maven provided scope because Trino provides +the classes from the SPI at runtime and thus the plugin should not +include them in the plugin assembly.

+

There are a few other dependencies that are provided by Trino, +including Slice and Jackson annotations. In particular, Jackson is +used for serializing connector handles and thus plugins must use the +annotations version provided by Trino.

+

All other dependencies are based on what the plugin needs for its +own implementation. Plugins are loaded in a separate class loader +to provide isolation and to allow plugins to use a different version +of a library that Trino uses internally.

+

For an example pom.xml file, see the example HTTP connector in the +plugin/trino-example-http directory in the Trino source tree.

+
+
+

Deploying a custom plugin#

+

Because Trino plugins use the trino-plugin packaging type, building +a plugin will create a ZIP file in the target directory. This file +contains the plugin JAR and all its dependencies JAR files.

+

In order to add a custom plugin to a Trino installation, extract the plugin +ZIP file and move the extracted directory into the Trino plugin directory. +For example, for a plugin called my-functions, with a version of 1.0, +you would extract my-functions-1.0.zip and then move my-functions-1.0 +to my-functions in the Trino plugin directory.

+
+

Note

+

Every Trino plugin should be in a separate directory. Do not put JAR files +directly into the plugin directory. Plugins should only contain JAR files, +so any subdirectories will not be traversed and will be ignored.

+
+

By default, the plugin directory is the plugin directory relative to the +directory in which Trino is installed, but it is configurable using the +configuration variable plugin.dir. In order for Trino to pick up +the new plugin, you must restart Trino.

+

Plugins must be installed on all nodes in the Trino cluster (coordinator and workers).

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/develop/supporting-merge.html b/430/develop/supporting-merge.html new file mode 100644 index 000000000..c366a2a1d --- /dev/null +++ b/430/develop/supporting-merge.html @@ -0,0 +1,969 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Supporting MERGE — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Supporting MERGE#

+

The Trino engine provides APIs to support row-level SQL MERGE. +To implement MERGE, a connector must provide the following:

+
    +
  • An implementation of ConnectorMergeSink, which is typically +layered on top of a ConnectorPageSink.

  • +
  • Methods in ConnectorMetadata to get a “rowId” column handle, get the +row change paradigm, and to start and complete the MERGE operation.

  • +
+

The Trino engine machinery used to implement SQL MERGE is also used to +support SQL DELETE and UPDATE. This means that all a connector needs to +do is implement support for SQL MERGE, and the connector gets all the Data +Modification Language (DML) operations.

+
+

Standard SQL MERGE#

+

Different query engines support varying definitions of SQL MERGE. +Trino supports the strict SQL specification ISO/IEC 9075, published +in 2016. As a simple example, given tables target_table and +source_table defined as:

+
CREATE TABLE accounts (
+    customer VARCHAR,
+    purchases DECIMAL,
+    address VARCHAR);
+INSERT INTO accounts (customer, purchases, address) VALUES ...;
+CREATE TABLE monthly_accounts_update (
+    customer VARCHAR,
+    purchases DECIMAL,
+    address VARCHAR);
+INSERT INTO monthly_accounts_update (customer, purchases, address) VALUES ...;
+
+
+

Here is a possible MERGE operation, from monthly_accounts_update to +accounts:

+
MERGE INTO accounts t USING monthly_accounts_update s
+    ON (t.customer = s.customer)
+    WHEN MATCHED AND s.address = 'Berkeley' THEN
+        DELETE
+    WHEN MATCHED AND s.customer = 'Joe Shmoe' THEN
+        UPDATE SET purchases = purchases + 100.0
+    WHEN MATCHED THEN
+        UPDATE
+            SET purchases = s.purchases + t.purchases, address = s.address
+    WHEN NOT MATCHED THEN
+        INSERT (customer, purchases, address)
+            VALUES (s.customer, s.purchases, s.address);
+
+
+

SQL MERGE tries to match each WHEN clause in source order. When +a match is found, the corresponding DELETE, INSERT or UPDATE +is executed and subsequent WHEN clauses are ignored.

+

SQL MERGE supports two operations on the target table and source +when a row from the source table or query matches a row in the target table:

+
    +
  • UPDATE, in which the columns in the target row are updated.

  • +
  • DELETE, in which the target row is deleted.

  • +
+

In the NOT MATCHED case, SQL MERGE supports only INSERT +operations. The values inserted are arbitrary but usually come from +the unmatched row of the source table or query.

+
+
+

RowChangeParadigm#

+

Different connectors have different ways of representing row updates, +imposed by the underlying storage systems. The Trino engine classifies +these different paradigms as elements of the RowChangeParadigm +enumeration, returned by enumeration, returned by method +ConnectorMetadata.getRowChangeParadigm(...).

+

The RowChangeParadigm enumeration values are:

+
    +
  • CHANGE_ONLY_UPDATED_COLUMNS, intended for connectors that can update +individual columns of rows identified by a rowId. The corresponding +merge processor class is ChangeOnlyUpdatedColumnsMergeProcessor.

  • +
  • DELETE_ROW_AND_INSERT_ROW, intended for connectors that represent a +row change as a row deletion paired with a row insertion. The corresponding +merge processor class is DeleteAndInsertMergeProcessor.

  • +
+
+
+

Overview of MERGE processing#

+

A MERGE statement is processed by creating a RIGHT JOIN between the +target table and the source, on the MERGE criteria. The source may be +a table or an arbitrary query. For each row in the source table or query, +MERGE produces a ROW object containing:

+
    +
  • the data column values from the UPDATE or INSERT cases. For the +DELETE cases, only the partition columns, which determine +partitioning and bucketing, are non-null.

  • +
  • a boolean column containing true for source rows that matched some +target row, and false otherwise.

  • +
  • an integer that identifies whether the merge case operation is UPDATE, +DELETE or INSERT, or a source row for which no case matched. If a +source row doesn’t match any merge case, all data column values except +those that determine distribution are null, and the operation number +is -1.

  • +
+

A SearchedCaseExpression is constructed from RIGHT JOIN result +to represent the WHEN clauses of the MERGE. In the example preceding +the MERGE is executed as if the SearchedCaseExpression were written as:

+
SELECT
+ CASE
+   WHEN present AND s.address = 'Berkeley' THEN
+       -- Null values for delete; present=true; operation DELETE=2, case_number=0
+       row(null, null, null, true, 2, 0)
+   WHEN present AND s.customer = 'Joe Shmoe' THEN
+       -- Update column values; present=true; operation UPDATE=3, case_number=1
+       row(t.customer, t.purchases + 100.0, t.address, true, 3, 1)
+   WHEN present THEN
+       -- Update column values; present=true; operation UPDATE=3, case_number=2
+       row(t.customer, s.purchases + t.purchases, s.address, true, 3, 2)
+   WHEN (present IS NULL) THEN
+       -- Insert column values; present=false; operation INSERT=1, case_number=3
+       row(s.customer, s.purchases, s.address, false, 1, 3)
+   ELSE
+       -- Null values for no case matched; present=false; operation=-1,
+       --     case_number=-1
+       row(null, null, null, false, -1, -1)
+ END
+ FROM (SELECT *, true AS present FROM target_table) t
+   RIGHT JOIN source_table s ON s.customer = t.customer;
+
+
+

The Trino engine executes the RIGHT JOIN and CASE expression, +and ensures that no target table row matches more than one source expression +row, and ultimately creates a sequence of pages to be routed to the node that +runs the ConnectorMergeSink.storeMergedRows(...) method.

+

Like DELETE and UPDATE, MERGE target table rows are identified by +a connector-specific rowId column handle. For MERGE, the rowId +handle is returned by ConnectorMetadata.getMergeRowIdColumnHandle(...).

+
+
+

MERGE redistribution#

+

The Trino MERGE implementation allows UPDATE to change +the values of columns that determine partitioning and/or bucketing, and so +it must “redistribute” rows from the MERGE operation to the worker +nodes responsible for writing rows with the merged partitioning and/or +bucketing columns.

+

Since the MERGE process in general requires redistribution of +merged rows among Trino nodes, the order of rows in pages to be stored +are indeterminate. Connectors like Hive that depend on an ascending +rowId order for deleted rows must sort the deleted rows before storing +them.

+

To ensure that all inserted rows for a given partition end up on a +single node, the redistribution hash on the partition key/bucket columns +is applied to the page partition keys. As a result of the hash, all +rows for a specific partition/bucket hash together, whether they +were MATCHED rows or NOT MATCHED rows.

+

For connectors whose RowChangeParadigm is DELETE_ROW_AND_INSERT_ROW, +inserted rows are distributed using the layout supplied by +ConnectorMetadata.getInsertLayout(). For some connectors, the same +layout is used for updated rows. Other connectors require a special +layout for updated rows, supplied by ConnectorMetadata.getUpdateLayout().

+
+

Connector support for MERGE#

+

To start MERGE processing, the Trino engine calls:

+
    +
  • ConnectorMetadata.getMergeRowIdColumnHandle(...) to get the +rowId column handle.

  • +
  • ConnectorMetadata.getRowChangeParadigm(...) to get the paradigm +supported by the connector for changing existing table rows.

  • +
  • ConnectorMetadata.beginMerge(...) to get the a +ConnectorMergeTableHandle for the merge operation. That +ConnectorMergeTableHandle object contains whatever information the +connector needs to specify the MERGE operation.

  • +
  • ConnectorMetadata.getInsertLayout(...), from which it extracts the +the list of partition or table columns that impact write redistribution.

  • +
  • ConnectorMetadata.getUpdateLayout(...). If that layout is non-empty, +it is used to distribute updated rows resulting from the MERGE +operation.

  • +
+

On nodes that are targets of the hash, the Trino engine calls +ConnectorPageSinkProvider.createMergeSink(...) to create a +ConnectorMergeSink.

+

To write out each page of merged rows, the Trino engine calls +ConnectorMergeSink.storeMergedRows(Page). The storeMergedRows(Page) +method iterates over the rows in the page, performing updates and deletes +in the MATCHED cases, and inserts in the NOT MATCHED cases.

+

When using RowChangeParadigm.DELETE_ROW_AND_INSERT_ROW, the engine +translates UPDATE operations into a pair of DELETE and INSERT +operations before storeMergedRows(Page) is called.

+

To complete the MERGE operation, the Trino engine calls +ConnectorMetadata.finishMerge(...), passing the table handle +and a collection of JSON objects encoded as Slice instances. These +objects contain connector-specific information specifying what was changed +by the MERGE operation. Typically this JSON object contains the files +written and table and partition statistics generated by the MERGE +operation. The connector takes appropriate actions, if any.

+
+
+
+

RowChangeProcessor implementation for MERGE#

+

In the MERGE implementation, each RowChangeParadigm +corresponds to an internal Trino engine class that implements interface +RowChangeProcessor. RowChangeProcessor has one interesting method: +Page transformPage(Page). The format of the output page depends +on the RowChangeParadigm.

+

The connector has no access to the RowChangeProcessor instance – it +is used inside the Trino engine to transform the merge page rows into rows +to be stored, based on the connector’s choice of RowChangeParadigm.

+

The page supplied to transformPage() consists of:

+
    +
  • The write redistribution columns if any

  • +
  • For partitioned or bucketed tables, a long hash value column.

  • +
  • The rowId column for the row from the target table if matched, or +null if not matched

  • +
  • The merge case RowBlock

  • +
  • The integer case number block

  • +
  • The byte is_distinct block, with value 0 if not distinct.

  • +
+

The merge case RowBlock has the following layout:

+
    +
  • Blocks for each column in the table, including partition columns, in +table column order.

  • +
  • A block containing the boolean “present” value which is true if the +source row matched a target row, and false otherwise.

  • +
  • A block containing the MERGE case operation number, encoded as +INSERT = 1, DELETE = 2, UPDATE = 3 and if no MERGE +case matched, -1.

  • +
  • A block containing the number, starting with 0, for the +WHEN clause that matched for the row, or -1 if no clause +matched.

  • +
+

The page returned from transformPage consists of:

+
    +
  • All table columns, in table column order.

  • +
  • The merge case operation block.

  • +
  • The rowId block.

  • +
  • A byte block containing 1 if the row is an insert derived from an +update operation, and 0 otherwise. This block is used to correctly +calculate the count of rows changed for connectors that represent +updates and deletes plus inserts.

  • +
+

transformPage +must ensure that there are no rows whose operation number is -1 in +the page it returns.

+
+
+

Detecting duplicate matching target rows#

+

The SQL MERGE specification requires that in each MERGE case, +a single target table row must match at most one source row, after +applying the MERGE case condition expression. The first step +toward finding these error is done by labeling each row in the target +table with a unique id, using an AssignUniqueId node above the +target table scan. The projected results from the RIGHT JOIN +have these unique ids for matched target table rows as well as +the WHEN clause number. A MarkDistinct node adds an +is_distinct column which is true if no other row has the same +unique id and WHEN clause number, and false otherwise. If +any row has is_distinct equal to false, a +MERGE_TARGET_ROW_MULTIPLE_MATCHES exception is raised and +the MERGE operation fails.

+
+
+

ConnectorMergeTableHandle API#

+

Interface ConnectorMergeTableHandle defines one method, +getTableHandle() to retrieve the ConnectorTableHandle +originally passed to ConnectorMetadata.beginMerge().

+
+
+

ConnectorPageSinkProvider API#

+

To support SQL MERGE, ConnectorPageSinkProvider must implement +the method that creates the ConnectorMergeSink:

+
    +
  • createMergeSink:

    +
    ConnectorMergeSink createMergeSink(
    +    ConnectorTransactionHandle transactionHandle,
    +    ConnectorSession session,
    +    ConnectorMergeTableHandle mergeHandle)
    +
    +
    +
  • +
+
+
+

ConnectorMergeSink API#

+

To support MERGE, the connector must define an +implementation of ConnectorMergeSink, usually layered over the +connector’s ConnectorPageSink.

+

The ConnectorMergeSink is created by a call to +ConnectorPageSinkProvider.createMergeSink().

+

The only interesting methods are:

+
    +
  • storeMergedRows:

    +
    void storeMergedRows(Page page)
    +
    +
    +

    The Trino engine calls the storeMergedRows(Page) method of the +ConnectorMergeSink instance returned by +ConnectorPageSinkProvider.createMergeSink(), passing the page +generated by the RowChangeProcessor.transformPage() method. +That page consists of all table columns, in table column order, +followed by the TINYINT operation column, followed by the rowId column.

    +

    The job of storeMergedRows() is iterate over the rows in the page, +and process them based on the value of the operation column, INSERT, +DELETE, UPDATE, or ignore the row. By choosing appropriate +paradigm, the connector can request that the UPDATE operation be +transformed into DELETE and INSERT operations.

    +
  • +
  • finish:

    +
    CompletableFuture<Collection<Slice>> finish()
    +
    +
    +

    The Trino engine calls finish() when all the data has been processed by +a specific ConnectorMergeSink instance. The connector returns a future +containing a collection of Slice, representing connector-specific +information about the rows processed. Usually this includes the row count, +and might include information like the files or partitions created or +changed.

    +
  • +
+
+
+

ConnectorMetadata MERGE API#

+

A connector implementing MERGE must implement these ConnectorMetadata +methods.

+
    +
  • getRowChangeParadigm():

    +
    RowChangeParadigm getRowChangeParadigm(
    +    ConnectorSession session,
    +    ConnectorTableHandle tableHandle)
    +
    +
    +

    This method is called as the engine starts processing a MERGE statement. +The connector must return a RowChangeParadigm enumeration instance. If +the connector doesn’t support MERGE, then it should throw a +NOT_SUPPORTED exception to indicate that SQL MERGE isn’t supported by +the connector. Note that the default implementation already throws this +exception when the method isn’t implemented.

    +
  • +
  • getMergeRowIdColumnHandle():

    +
    ColumnHandle getMergeRowIdColumnHandle(
    +    ConnectorSession session,
    +    ConnectorTableHandle tableHandle)
    +
    +
    +

    This method is called in the early stages of query planning for MERGE +statements. The ColumnHandle returned provides the rowId used by the +connector to identify rows to be merged, as well as any other fields of +the row that the connector needs to complete the MERGE operation.

    +
  • +
  • getInsertLayout():

    +
    Optional<ConnectorTableLayout> getInsertLayout(
    +    ConnectorSession session,
    +    ConnectorTableHandle tableHandle)
    +
    +
    +

    This method is called during query planning to get the table layout to be +used for rows inserted by the MERGE operation. For some connectors, +this layout is used for rows deleted as well.

    +
  • +
  • getUpdateLayout():

    +
    Optional<ConnectorTableLayout> getUpdateLayout(
    +    ConnectorSession session,
    +    ConnectorTableHandle tableHandle)
    +
    +
    +

    This method is called during query planning to get the table layout to be +used for rows deleted by the MERGE operation. If the optional return +value is present, the Trino engine uses the layout for updated rows. +Otherwise, it uses the result of ConnectorMetadata.getInsertLayout to +distribute updated rows.

    +
  • +
  • beginMerge():

    +
    ConnectorMergeTableHandle beginMerge(
    +     ConnectorSession session,
    +     ConnectorTableHandle tableHandle)
    +
    +
    +

    As the last step in creating the MERGE execution plan, the connector’s +beginMerge() method is called, passing the session, and the +tableHandle.

    +

    beginMerge() performs any orchestration needed in the connector to +start processing the MERGE. This orchestration varies from connector +to connector. In the case of Hive connector operating on transactional tables, +for example, beginMerge() checks that the table is transactional and +starts a Hive Metastore transaction.

    +

    beginMerge() returns a ConnectorMergeTableHandle with any added +information the connector needs when the handle is passed back to +finishMerge() and the split generation machinery. For most +connectors, the returned table handle contains at least a flag identifying +the table handle as a table handle for a MERGE operation.

    +
  • +
  • finishMerge():

    +
    void finishMerge(
    +    ConnectorSession session,
    +    ConnectorMergeTableHandle tableHandle,
    +    Collection<Slice> fragments)
    +
    +
    +

    During MERGE processing, the Trino engine accumulates the Slice +collections returned by ConnectorMergeSink.finish(). The engine calls +finishMerge(), passing the table handle and that collection of +Slice fragments. In response, the connector takes appropriate actions +to complete the MERGE operation. Those actions might include +committing an underlying transaction, if any, or freeing any other +resources.

    +
  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/develop/system-access-control.html b/430/develop/system-access-control.html new file mode 100644 index 000000000..aacb314b8 --- /dev/null +++ b/430/develop/system-access-control.html @@ -0,0 +1,574 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + System access control — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

System access control#

+

Trino separates the concept of the principal who authenticates to the +coordinator from the username that is responsible for running queries. When +running the Trino CLI, for example, the Trino username can be specified using +the --user option.

+

By default, the Trino coordinator allows any principal to run queries as any +Trino user. In a secure environment, this is probably not desirable behavior +and likely requires customization.

+
+

Implementation#

+

SystemAccessControlFactory is responsible for creating a +SystemAccessControl instance. It also defines a SystemAccessControl +name which is used by the administrator in a Trino configuration.

+

SystemAccessControl implementations have several responsibilities:

+
    +
  • Verifying whether or not a given principal is authorized to execute queries as a specific user.

  • +
  • Determining whether or not a given user can alter values for a given system property.

  • +
  • Performing access checks across all catalogs. These access checks happen before +any connector specific checks and thus can deny permissions that would otherwise +be allowed by ConnectorAccessControl.

  • +
+

The implementation of SystemAccessControl and SystemAccessControlFactory +must be wrapped as a plugin and installed on the Trino cluster.

+
+
+

Configuration#

+

After a plugin that implements SystemAccessControl and +SystemAccessControlFactory has been installed on the coordinator, it is +configured using the file(s) specified by the access-control.config-files +property (the default is a single etc/access-control.properties file). +All of the properties other than access-control.name are specific to +the SystemAccessControl implementation.

+

The access-control.name property is used by Trino to find a registered +SystemAccessControlFactory based on the name returned by +SystemAccessControlFactory.getName(). The remaining properties are passed +as a map to SystemAccessControlFactory.create().

+

Example configuration file:

+
access-control.name=custom-access-control
+custom-property1=custom-value1
+custom-property2=custom-value2
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/develop/table-functions.html b/430/develop/table-functions.html new file mode 100644 index 000000000..7b5e606ed --- /dev/null +++ b/430/develop/table-functions.html @@ -0,0 +1,828 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Table functions — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Table functions#

+

Table functions return tables. They allow users to dynamically invoke custom +logic from within the SQL query. They are invoked in the FROM clause of a +query, and the calling convention is similar to a scalar function call. For +description of table functions usage, see +table functions.

+

Trino supports adding custom table functions. They are declared by connectors +through implementing dedicated interfaces.

+
+

Table function declaration#

+

To declare a table function, you need to implement ConnectorTableFunction. +Subclassing AbstractConnectorTableFunction is a convenient way to do it. +The connector’s getTableFunctions() method must return a set of your +implementations.

+
+

The constructor#

+
public class MyFunction
+        extends AbstractConnectorTableFunction
+{
+    public MyFunction()
+    {
+        super(
+                "system",
+                "my_function",
+                List.of(
+                        ScalarArgumentSpecification.builder()
+                                .name("COLUMN_COUNT")
+                                .type(INTEGER)
+                                .defaultValue(2)
+                                .build(),
+                        ScalarArgumentSpecification.builder()
+                                .name("ROW_COUNT")
+                                .type(INTEGER)
+                                .build()),
+                GENERIC_TABLE);
+    }
+}
+
+
+

The constructor takes the following arguments:

+
    +
  • schema name

  • +
+

The schema name helps you organize functions, and it is used for function +resolution. When a table function is invoked, the right implementation is +identified by the catalog name, the schema name, and the function name.

+

The function can use the schema name, for example to use data from the +indicated schema, or ignore it.

+
    +
  • function name

  • +
  • list of expected arguments

  • +
+

Three different types of arguments are supported: scalar arguments, descriptor +arguments, and table arguments. See Argument types for details. You can +specify default values for scalar and descriptor arguments. The arguments with +specified default can be skipped during table function invocation.

+
    +
  • returned row type

  • +
+

It describes the row type produced by the table function.

+

If a table function takes table arguments, it can additionally pass the columns +of the input tables to output using the pass-through mechanism. The returned +row type is supposed to describe only the columns produced by the function, as +opposed to the pass-through columns.

+

In the example, the returned row type is GENERIC_TABLE, which means that +the row type is not known statically, and it is determined dynamically based on +the passed arguments.

+

When the returned row type is known statically, you can declare it using:

+
new DescribedTable(descriptor)
+
+
+

If a table function does not produce any columns, and it only outputs the +pass-through columns, use ONLY_PASS_THROUGH as the returned row type.

+
+

Note

+

A table function must return at least one column. It can either be a proper +column, i.e. produced by the function, or a pass-through column.

+
+
+
+

Argument types#

+

Table functions take three types of arguments: +scalar arguments, +descriptor arguments, and +table arguments.

+
+

Scalar arguments#

+

They can be of any supported data type. You can specify a default value.

+
ScalarArgumentSpecification.builder()
+        .name("COLUMN_COUNT")
+        .type(INTEGER)
+        .defaultValue(2)
+        .build()
+
+
+
ScalarArgumentSpecification.builder()
+        .name("ROW_COUNT")
+        .type(INTEGER)
+        .build()
+
+
+
+
+

Descriptor arguments#

+

Descriptors consist of fields with names and optional data types. They are a +convenient way to pass the required result row type to the function, or for +example inform the function which input columns it should use. You can specify +default values for descriptor arguments. Descriptor argument can be null.

+
DescriptorArgumentSpecification.builder()
+        .name("SCHEMA")
+        .defaultValue(null)
+        .build()
+
+
+
+
+

Table arguments#

+

A table function can take any number of input relations. It allows you to +process multiple data sources simultaneously.

+

When declaring a table argument, you must specify characteristics to determine +how the input table is processed. Also note that you cannot specify a default +value for a table argument.

+
TableArgumentSpecification.builder()
+        .name("INPUT")
+        .rowSemantics()
+        .pruneWhenEmpty()
+        .passThroughColumns()
+        .build()
+
+
+
+
Set or row semantics#
+

Set semantics is the default for table arguments. A table argument with set +semantics is processed on a partition-by-partition basis. During function +invocation, the user can specify partitioning and ordering for the argument. If +no partitioning is specified, the argument is processed as a single partition.

+

A table argument with row semantics is processed on a row-by-row basis. +Partitioning or ordering is not applicable.

+
+
+
Prune or keep when empty#
+

The prune when empty property indicates that if the given table argument is +empty, the function returns empty result. This property is used to optimize +queries involving table functions. The keep when empty property indicates +that the function should be executed even if the table argument is empty. The +user can override this property when invoking the function. Using the keep +when empty property can negatively affect performance when the table argument +is not empty.

+
+
+
Pass-through columns#
+

If a table argument has pass-through columns, all of its columns are passed +on output. For a table argument without this property, only the partitioning +columns are passed on output.

+
+
+
+
+

The analyze() method#

+

In order to provide all the necessary information to the Trino engine, the +class must implement the analyze() method. This method is called by the +engine during the analysis phase of query processing. The analyze() method +is also the place to perform custom checks on the arguments:

+
@Override
+public TableFunctionAnalysis analyze(ConnectorSession session, ConnectorTransactionHandle transaction, Map<String, Argument> arguments)
+{
+    long columnCount = (long) ((ScalarArgument) arguments.get("COLUMN_COUNT")).getValue();
+    long rowCount = (long) ((ScalarArgument) arguments.get("ROW_COUNT")).getValue();
+
+    // custom validation of arguments
+    if (columnCount < 1 || columnCount > 3) {
+         throw new TrinoException(INVALID_FUNCTION_ARGUMENT, "column_count must be in range [1, 3]");
+    }
+
+    if (rowCount < 1) {
+        throw new TrinoException(INVALID_FUNCTION_ARGUMENT, "row_count must be positive");
+    }
+
+    // determine the returned row type
+    List<Descriptor.Field> fields = List.of("col_a", "col_b", "col_c").subList(0, (int) columnCount).stream()
+            .map(name -> new Descriptor.Field(name, Optional.of(BIGINT)))
+            .collect(toList());
+
+    Descriptor returnedType = new Descriptor(fields);
+
+    return TableFunctionAnalysis.builder()
+            .returnedType(returnedType)
+            .handle(new MyHandle(columnCount, rowCount))
+            .build();
+}
+
+
+

The analyze() method returns a TableFunctionAnalysis object, which +comprises all the information required by the engine to analyze, plan, and +execute the table function invocation:

+
    +
  • The returned row type, specified as an optional Descriptor. It should be +passed if and only if the table function is declared with the +GENERIC_TABLE returned type.

  • +
  • Required columns from the table arguments, specified as a map of table +argument names to lists of column indexes.

  • +
  • Any information gathered during analysis that is useful during planning or +execution, in the form of a ConnectorTableFunctionHandle. +ConnectorTableFunctionHandle is a marker interface intended to carry +information throughout subsequent phases of query processing in a manner that +is opaque to the engine.

  • +
+
+
+
+

Table function execution#

+

There are two paths of execution available for table functions.

+
    +
  1. Pushdown to the connector

  2. +
+

The connector that provides the table function implements the +applyTableFunction() method. This method is called during the optimization +phase of query processing. It returns a ConnectorTableHandle and a list of +ColumnHandle s representing the table function result. The table function +invocation is then replaced with a TableScanNode.

+

This execution path is convenient for table functions whose results are easy to +represent as a ConnectorTableHandle, for example query pass-through. It +only supports scalar and descriptor arguments.

+
    +
  1. Execution by operator

  2. +
+

Trino has a dedicated operator for table functions. It can handle table +functions with any number of table arguments as well as scalar and descriptor +arguments. To use this execution path, you provide an implementation of a +processor.

+

If your table function has one or more table arguments, you must implement +TableFunctionDataProcessor. It processes pages of input data.

+

If your table function is a source operator (it does not have table arguments), +you must implement TableFunctionSplitProcessor. It processes splits. The +connector that provides the function must provide a ConnectorSplitSource +for the function. With splits, the task can be divided so that each split +represents a subtask.

+
+
+

Access control#

+

The access control for table functions can be provided both on system and +connector level. It is based on the fully qualified table function name, +which consists of the catalog name, the schema name, and the function name, +in the syntax of catalog.schema.function.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/develop/types.html b/430/develop/types.html new file mode 100644 index 000000000..69d890bfa --- /dev/null +++ b/430/develop/types.html @@ -0,0 +1,569 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Types — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Types#

+

The Type interface in Trino is used to implement a type in the SQL language. +Trino ships with a number of built-in types, like VarcharType and BigintType. +The ParametricType interface is used to provide type parameters for types, to +allow types like VARCHAR(10) or DECIMAL(22, 5). A Plugin can provide +new Type objects by returning them from getTypes() and new ParametricType +objects by returning them from getParametricTypes().

+

Below is a high level overview of the Type interface. For more details, see the +JavaDocs for Type.

+
+

Native container type#

+

All types define the getJavaType() method, frequently referred to as the +“native container type”. This is the Java type used to hold values during execution +and to store them in a Block. For example, this is the type used in +the Java code that implements functions that produce or consume this Type.

+
+
+

Native encoding#

+

The interpretation of a value in its native container type form is defined by its +Type. For some types, such as BigintType, it matches the Java +interpretation of the native container type (64bit 2’s complement). However, for other +types such as TimestampWithTimeZoneType, which also uses long for its +native container type, the value stored in the long is a 8byte binary value +combining the timezone and the milliseconds since the unix epoch. In particular, +this means that you cannot compare two native values and expect a meaningful +result, without knowing the native encoding.

+
+
+

Type signature#

+

The signature of a type defines its identity, and also encodes some general +information about the type, such as its type parameters (if it’s parametric), +and its literal parameters. The literal parameters are used in types like +VARCHAR(10).

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions.html b/430/functions.html new file mode 100644 index 000000000..92bac70ba --- /dev/null +++ b/430/functions.html @@ -0,0 +1,708 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Functions and operators — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Functions and operators#

+

This section describes the built-in SQL functions and operators supported by +Trino. They allow you to implement complex capabilities and behavior of the +queries executed by Trino operating on the underlying data sources.

+

Refer to the following sections for further details:

+ +
+

Functions by name#

+

If you are looking for a specific function or operator by name use +SHOW FUNCTIONS, or refer the to the following resources:

+ +
+
+

Functions per topic#

+ +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/aggregate.html b/430/functions/aggregate.html new file mode 100644 index 000000000..a5f784bde --- /dev/null +++ b/430/functions/aggregate.html @@ -0,0 +1,1255 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Aggregate functions — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Aggregate functions#

+

Aggregate functions operate on a set of values to compute a single result.

+

Except for count(), count_if(), max_by(), min_by() and +approx_distinct(), all of these aggregate functions ignore null values +and return null for no input rows or when all values are null. For example, +sum() returns null rather than zero and avg() does not include null +values in the count. The coalesce function can be used to convert null into +zero.

+
+

Ordering during aggregation#

+

Some aggregate functions such as array_agg() produce different results +depending on the order of input values. This ordering can be specified by writing +an ORDER BY clause within the aggregate function:

+
array_agg(x ORDER BY y DESC)
+array_agg(x ORDER BY x, y, z)
+
+
+
+
+

Filtering during aggregation#

+

The FILTER keyword can be used to remove rows from aggregation processing +with a condition expressed using a WHERE clause. This is evaluated for each +row before it is used in the aggregation and is supported for all aggregate +functions.

+
aggregate_function(...) FILTER (WHERE <condition>)
+
+
+

A common and very useful example is to use FILTER to remove nulls from +consideration when using array_agg:

+
SELECT array_agg(name) FILTER (WHERE name IS NOT NULL)
+FROM region;
+
+
+

As another example, imagine you want to add a condition on the count for Iris +flowers, modifying the following query:

+
SELECT species,
+       count(*) AS count
+FROM iris
+GROUP BY species;
+
+
+
species    | count
+-----------+-------
+setosa     |   50
+virginica  |   50
+versicolor |   50
+
+
+

If you just use a normal WHERE statement you lose information:

+
SELECT species,
+    count(*) AS count
+FROM iris
+WHERE petal_length_cm > 4
+GROUP BY species;
+
+
+
species    | count
+-----------+-------
+virginica  |   50
+versicolor |   34
+
+
+

Using a filter you retain all information:

+
SELECT species,
+       count(*) FILTER (where petal_length_cm > 4) AS count
+FROM iris
+GROUP BY species;
+
+
+
species    | count
+-----------+-------
+virginica  |   50
+setosa     |    0
+versicolor |   34
+
+
+
+
+

General aggregate functions#

+
+
+any_value(x) [same as input]#
+

Returns an arbitrary non-null value x, if one exists. x can be any +valid expression. This allows you to return values from columns that are not +directly part of the aggregation, inluding expressions using these columns, +in a query.

+

For example, the following query returns the customer name from the name +column, and returns the sum of all total prices as customer spend. The +aggregation however uses the rows grouped by the customer identifier +custkey a required, since only that column is guaranteed to be unique:

+
SELECT sum(o.totalprice) as spend,
+    any_value(c.name)
+FROM tpch.tiny.orders o
+JOIN tpch.tiny.customer c
+ON o.custkey  = c.custkey
+GROUP BY c.custkey;
+ORDER BY spend;
+
+
+
+
+
+arbitrary(x) [same as input]#
+

Returns an arbitrary non-null value of x, if one exists. Identical to +any_value().

+
+
+
+array_agg(x) array<[same as input]>#
+

Returns an array created from the input x elements.

+
+
+
+avg(x) double#
+

Returns the average (arithmetic mean) of all input values.

+
+
+
+avg(time interval type) time interval type
+

Returns the average interval length of all input values.

+
+
+
+bool_and(boolean) boolean#
+

Returns TRUE if every input value is TRUE, otherwise FALSE.

+
+
+
+bool_or(boolean) boolean#
+

Returns TRUE if any input value is TRUE, otherwise FALSE.

+
+
+
+checksum(x) varbinary#
+

Returns an order-insensitive checksum of the given values.

+
+
+
+count(*) bigint#
+

Returns the number of input rows.

+
+
+
+count(x) bigint
+

Returns the number of non-null input values.

+
+
+
+count_if(x) bigint#
+

Returns the number of TRUE input values. +This function is equivalent to count(CASE WHEN x THEN 1 END).

+
+
+
+every(boolean) boolean#
+

This is an alias for bool_and().

+
+
+
+geometric_mean(x) double#
+

Returns the geometric mean of all input values.

+
+
+
+listagg(x, separator) varchar#
+

Returns the concatenated input values, separated by the separator string.

+

Synopsis:

+
LISTAGG( expression [, separator] [ON OVERFLOW overflow_behaviour])
+    WITHIN GROUP (ORDER BY sort_item, [...])
+
+
+

If separator is not specified, the empty string will be used as separator.

+

In its simplest form the function looks like:

+
SELECT listagg(value, ',') WITHIN GROUP (ORDER BY value) csv_value
+FROM (VALUES 'a', 'c', 'b') t(value);
+
+
+

and results in:

+
csv_value
+-----------
+'a,b,c'
+
+
+

The overflow behaviour is by default to throw an error in case that the length of the output +of the function exceeds 1048576 bytes:

+
SELECT listagg(value, ',' ON OVERFLOW ERROR) WITHIN GROUP (ORDER BY value) csv_value
+FROM (VALUES 'a', 'b', 'c') t(value);
+
+
+

There exists also the possibility to truncate the output WITH COUNT or WITHOUT COUNT +of omitted non-null values in case that the length of the output of the +function exceeds 1048576 bytes:

+
SELECT LISTAGG(value, ',' ON OVERFLOW TRUNCATE '.....' WITH COUNT) WITHIN GROUP (ORDER BY value)
+FROM (VALUES 'a', 'b', 'c') t(value);
+
+
+

If not specified, the truncation filler string is by default '...'.

+

This aggregation function can be also used in a scenario involving grouping:

+
SELECT id, LISTAGG(value, ',') WITHIN GROUP (ORDER BY o) csv_value
+FROM (VALUES
+    (100, 1, 'a'),
+    (200, 3, 'c'),
+    (200, 2, 'b')
+) t(id, o, value)
+GROUP BY id
+ORDER BY id;
+
+
+

results in:

+
 id  | csv_value
+-----+-----------
+ 100 | a
+ 200 | b,c
+
+
+

The current implementation of LISTAGG function does not support window frames.

+
+
+
+max(x) [same as input]#
+

Returns the maximum value of all input values.

+
+
+
+max(x, n) array<[same as x]>
+

Returns n largest values of all input values of x.

+
+
+
+max_by(x, y) [same as x]#
+

Returns the value of x associated with the maximum value of y over all input values.

+
+
+
+max_by(x, y, n) array<[same as x]>
+

Returns n values of x associated with the n largest of all input values of y +in descending order of y.

+
+
+
+min(x) [same as input]#
+

Returns the minimum value of all input values.

+
+
+
+min(x, n) array<[same as x]>
+

Returns n smallest values of all input values of x.

+
+
+
+min_by(x, y) [same as x]#
+

Returns the value of x associated with the minimum value of y over all input values.

+
+
+
+min_by(x, y, n) array<[same as x]>
+

Returns n values of x associated with the n smallest of all input values of y +in ascending order of y.

+
+
+
+sum(x) [same as input]#
+

Returns the sum of all input values.

+
+
+
+

Bitwise aggregate functions#

+
+
+bitwise_and_agg(x) bigint#
+

Returns the bitwise AND of all input values in 2’s complement representation.

+
+
+
+bitwise_or_agg(x) bigint#
+

Returns the bitwise OR of all input values in 2’s complement representation.

+
+
+
+

Map aggregate functions#

+
+
+histogram(x) map<K,bigint>#
+

Returns a map containing the count of the number of times each input value occurs.

+
+
+
+map_agg(key, value) map<K,V>#
+

Returns a map created from the input key / value pairs.

+
+
+
+map_union(x(K, V)) map<K,V>#
+

Returns the union of all the input maps. If a key is found in multiple +input maps, that key’s value in the resulting map comes from an arbitrary input map.

+

For example, take the following histogram function that creates multiple maps from the Iris dataset:

+
SELECT histogram(floor(petal_length_cm)) petal_data
+FROM memory.default.iris
+GROUP BY species;
+
+        petal_data
+-- {4.0=6, 5.0=33, 6.0=11}
+-- {4.0=37, 5.0=2, 3.0=11}
+-- {1.0=50}
+
+
+

You can combine these maps using map_union:

+
SELECT map_union(petal_data) petal_data_union
+FROM (
+       SELECT histogram(floor(petal_length_cm)) petal_data
+       FROM memory.default.iris
+       GROUP BY species
+       );
+
+             petal_data_union
+--{4.0=6, 5.0=2, 6.0=11, 1.0=50, 3.0=11}
+
+
+
+
+
+multimap_agg(key, value) map<K,array(V)>#
+

Returns a multimap created from the input key / value pairs. +Each key can be associated with multiple values.

+
+
+
+

Approximate aggregate functions#

+
+
+approx_distinct(x) bigint#
+

Returns the approximate number of distinct input values. +This function provides an approximation of count(DISTINCT x). +Zero is returned if all input values are null.

+

This function should produce a standard error of 2.3%, which is the +standard deviation of the (approximately normal) error distribution over +all possible sets. It does not guarantee an upper bound on the error for +any specific input set.

+
+
+
+approx_distinct(x, e) bigint
+

Returns the approximate number of distinct input values. +This function provides an approximation of count(DISTINCT x). +Zero is returned if all input values are null.

+

This function should produce a standard error of no more than e, which +is the standard deviation of the (approximately normal) error distribution +over all possible sets. It does not guarantee an upper bound on the error +for any specific input set. The current implementation of this function +requires that e be in the range of [0.0040625, 0.26000].

+
+
+
+approx_most_frequent(buckets, value, capacity) map<[same as value], bigint>#
+

Computes the top frequent values up to buckets elements approximately. +Approximate estimation of the function enables us to pick up the frequent +values with less memory. Larger capacity improves the accuracy of +underlying algorithm with sacrificing the memory capacity. The returned +value is a map containing the top elements with corresponding estimated +frequency.

+

The error of the function depends on the permutation of the values and its +cardinality. We can set the capacity same as the cardinality of the +underlying data to achieve the least error.

+

buckets and capacity must be bigint. value can be numeric +or string type.

+

The function uses the stream summary data structure proposed in the paper +Efficient Computation of Frequent and Top-k Elements in Data Streams +by A. Metwalley, D. Agrawl and A. Abbadi.

+
+
+
+approx_percentile(x, percentage) [same as x]#
+

Returns the approximate percentile for all input values of x at the +given percentage. The value of percentage must be between zero and +one and must be constant for all input rows.

+
+
+
+approx_percentile(x, percentages) array<[same as x]>
+

Returns the approximate percentile for all input values of x at each of +the specified percentages. Each element of the percentages array must be +between zero and one, and the array must be constant for all input rows.

+
+
+
+approx_percentile(x, w, percentage) [same as x]
+

Returns the approximate weighed percentile for all input values of x +using the per-item weight w at the percentage percentage. Weights must be +greater or equal to 1. Integer-value weights can be thought of as a replication +count for the value x in the percentile set. The value of percentage must be +between zero and one and must be constant for all input rows.

+
+
+
+approx_percentile(x, w, percentages) array<[same as x]>
+

Returns the approximate weighed percentile for all input values of x +using the per-item weight w at each of the given percentages specified +in the array. Weights must be greater or equal to 1. Integer-value weights can +be thought of as a replication count for the value x in the percentile +set. Each element of the percentages array must be between zero and one, and the array +must be constant for all input rows.

+
+
+
+approx_set(x) HyperLogLog
+

See HyperLogLog functions.

+
+
+
+merge(x) HyperLogLog
+

See HyperLogLog functions.

+
+
+
+merge(qdigest(T)) -> qdigest(T)
+

See Quantile digest functions.

+
+
+
+merge(tdigest) tdigest
+

See T-Digest functions.

+
+
+
+numeric_histogram(buckets, value) map<double, double>
+

Computes an approximate histogram with up to buckets number of buckets +for all values. This function is equivalent to the variant of +numeric_histogram() that takes a weight, with a per-item weight of 1.

+
+
+
+numeric_histogram(buckets, value, weight) map<double, double>#
+

Computes an approximate histogram with up to buckets number of buckets +for all values with a per-item weight of weight. The algorithm +is based loosely on:

+
Yael Ben-Haim and Elad Tom-Tov, "A streaming parallel decision tree algorithm",
+J. Machine Learning Research 11 (2010), pp. 849--872.
+
+
+

buckets must be a bigint. value and weight must be numeric.

+
+
+
+qdigest_agg(x) -> qdigest([same as x])
+

See Quantile digest functions.

+
+
+
+qdigest_agg(x, w) -> qdigest([same as x])
+

See Quantile digest functions.

+
+
+
+qdigest_agg(x, w, accuracy) -> qdigest([same as x])
+

See Quantile digest functions.

+
+
+
+tdigest_agg(x) tdigest
+

See T-Digest functions.

+
+
+
+tdigest_agg(x, w) tdigest
+

See T-Digest functions.

+
+
+
+

Statistical aggregate functions#

+
+
+corr(y, x) double#
+

Returns correlation coefficient of input values.

+
+
+
+covar_pop(y, x) double#
+

Returns the population covariance of input values.

+
+
+
+covar_samp(y, x) double#
+

Returns the sample covariance of input values.

+
+
+
+kurtosis(x) double#
+

Returns the excess kurtosis of all input values. Unbiased estimate using +the following expression:

+
kurtosis(x) = n(n+1)/((n-1)(n-2)(n-3))sum[(x_i-mean)^4]/stddev(x)^4-3(n-1)^2/((n-2)(n-3))
+
+
+
+
+
+regr_intercept(y, x) double#
+

Returns linear regression intercept of input values. y is the dependent +value. x is the independent value.

+
+
+
+regr_slope(y, x) double#
+

Returns linear regression slope of input values. y is the dependent +value. x is the independent value.

+
+
+
+skewness(x) double#
+

Returns the Fisher’s moment coefficient of skewness of all input values.

+
+
+
+stddev(x) double#
+

This is an alias for stddev_samp().

+
+
+
+stddev_pop(x) double#
+

Returns the population standard deviation of all input values.

+
+
+
+stddev_samp(x) double#
+

Returns the sample standard deviation of all input values.

+
+
+
+variance(x) double#
+

This is an alias for var_samp().

+
+
+
+var_pop(x) double#
+

Returns the population variance of all input values.

+
+
+
+var_samp(x) double#
+

Returns the sample variance of all input values.

+
+
+
+

Lambda aggregate functions#

+
+
+reduce_agg(inputValue T, initialState S, inputFunction(S, T, S), combineFunction(S, S, S)) S#
+

Reduces all input values into a single value. inputFunction will be invoked +for each non-null input value. In addition to taking the input value, inputFunction +takes the current state, initially initialState, and returns the new state. +combineFunction will be invoked to combine two states into a new state. +The final state is returned:

+
SELECT id, reduce_agg(value, 0, (a, b) -> a + b, (a, b) -> a + b)
+FROM (
+    VALUES
+        (1, 3),
+        (1, 4),
+        (1, 5),
+        (2, 6),
+        (2, 7)
+) AS t(id, value)
+GROUP BY id;
+-- (1, 12)
+-- (2, 13)
+
+SELECT id, reduce_agg(value, 1, (a, b) -> a * b, (a, b) -> a * b)
+FROM (
+    VALUES
+        (1, 3),
+        (1, 4),
+        (1, 5),
+        (2, 6),
+        (2, 7)
+) AS t(id, value)
+GROUP BY id;
+-- (1, 60)
+-- (2, 42)
+
+
+

The state type must be a boolean, integer, floating-point, or date/time/interval.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/array.html b/430/functions/array.html new file mode 100644 index 000000000..bfb7b7382 --- /dev/null +++ b/430/functions/array.html @@ -0,0 +1,1084 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Array functions and operators — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+ +
+
+ +
+
+ +
+

Array functions and operators#

+
+

Subscript operator: []#

+

The [] operator is used to access an element of an array and is indexed starting from one:

+
SELECT my_array[1] AS first_element
+
+
+
+
+

Concatenation operator: ||#

+

The || operator is used to concatenate an array with an array or an element of the same type:

+
SELECT ARRAY[1] || ARRAY[2];
+-- [1, 2]
+
+SELECT ARRAY[1] || 2;
+-- [1, 2]
+
+SELECT 2 || ARRAY[1];
+-- [2, 1]
+
+
+
+
+

Array functions#

+
+
+all_match(array(T), function(T, boolean)) boolean#
+

Returns whether all elements of an array match the given predicate. Returns true if all the elements +match the predicate (a special case is when the array is empty); false if one or more elements don’t +match; NULL if the predicate function returns NULL for one or more elements and true for all +other elements.

+
+
+
+any_match(array(T), function(T, boolean)) boolean#
+

Returns whether any elements of an array match the given predicate. Returns true if one or more +elements match the predicate; false if none of the elements matches (a special case is when the +array is empty); NULL if the predicate function returns NULL for one or more elements and false +for all other elements.

+
+
+
+array_distinct(x) array#
+

Remove duplicate values from the array x.

+
+
+
+array_intersect(x, y) array#
+

Returns an array of the elements in the intersection of x and y, without duplicates.

+
+
+
+array_union(x, y) array#
+

Returns an array of the elements in the union of x and y, without duplicates.

+
+
+
+array_except(x, y) array#
+

Returns an array of elements in x but not in y, without duplicates.

+
+
+
+array_histogram(x) map<K, bigint>#
+

Returns a map where the keys are the unique elements in the input array +x and the values are the number of times that each element appears in +x. Null values are ignored.

+
SELECT array_histogram(ARRAY[42, 7, 42, NULL]);
+-- {42=2, 7=1}
+
+
+

Returns an empty map if the input array has no non-null elements.

+
SELECT array_histogram(ARRAY[NULL, NULL]);
+-- {}
+
+
+
+
+
+array_join(x, delimiter, null_replacement) varchar#
+

Concatenates the elements of the given array using the delimiter and an optional string to replace nulls.

+
+
+
+array_max(x) x#
+

Returns the maximum value of input array.

+
+
+
+array_min(x) x#
+

Returns the minimum value of input array.

+
+
+
+array_position(x, element) bigint#
+

Returns the position of the first occurrence of the element in array x (or 0 if not found).

+
+
+
+array_remove(x, element) array#
+

Remove all elements that equal element from array x.

+
+
+
+array_sort(x) array#
+

Sorts and returns the array x. The elements of x must be orderable. +Null elements will be placed at the end of the returned array.

+
+
+
+array_sort(array(T), function(T, T, int)) -> array(T)
+

Sorts and returns the array based on the given comparator function. +The comparator will take two nullable arguments representing two nullable +elements of the array. It returns -1, 0, or 1 as the first nullable +element is less than, equal to, or greater than the second nullable element. +If the comparator function returns other values (including NULL), the +query will fail and raise an error.

+
SELECT array_sort(ARRAY[3, 2, 5, 1, 2],
+                  (x, y) -> IF(x < y, 1, IF(x = y, 0, -1)));
+-- [5, 3, 2, 2, 1]
+
+SELECT array_sort(ARRAY['bc', 'ab', 'dc'],
+                  (x, y) -> IF(x < y, 1, IF(x = y, 0, -1)));
+-- ['dc', 'bc', 'ab']
+
+
+SELECT array_sort(ARRAY[3, 2, null, 5, null, 1, 2],
+                  -- sort null first with descending order
+                  (x, y) -> CASE WHEN x IS NULL THEN -1
+                                 WHEN y IS NULL THEN 1
+                                 WHEN x < y THEN 1
+                                 WHEN x = y THEN 0
+                                 ELSE -1 END);
+-- [null, null, 5, 3, 2, 2, 1]
+
+SELECT array_sort(ARRAY[3, 2, null, 5, null, 1, 2],
+                  -- sort null last with descending order
+                  (x, y) -> CASE WHEN x IS NULL THEN 1
+                                 WHEN y IS NULL THEN -1
+                                 WHEN x < y THEN 1
+                                 WHEN x = y THEN 0
+                                 ELSE -1 END);
+-- [5, 3, 2, 2, 1, null, null]
+
+SELECT array_sort(ARRAY['a', 'abcd', 'abc'],
+                  -- sort by string length
+                  (x, y) -> IF(length(x) < length(y), -1,
+                               IF(length(x) = length(y), 0, 1)));
+-- ['a', 'abc', 'abcd']
+
+SELECT array_sort(ARRAY[ARRAY[2, 3, 1], ARRAY[4, 2, 1, 4], ARRAY[1, 2]],
+                  -- sort by array length
+                  (x, y) -> IF(cardinality(x) < cardinality(y), -1,
+                               IF(cardinality(x) = cardinality(y), 0, 1)));
+-- [[1, 2], [2, 3, 1], [4, 2, 1, 4]]
+
+
+
+
+
+arrays_overlap(x, y) boolean#
+

Tests if arrays x and y have any non-null elements in common. +Returns null if there are no non-null elements in common but either array contains null.

+
+
+
+cardinality(x) bigint#
+

Returns the cardinality (size) of the array x.

+
+
+
+concat(array1, array2, ..., arrayN) array
+

Concatenates the arrays array1, array2, ..., arrayN. +This function provides the same functionality as the SQL-standard concatenation operator (||).

+
+
+
+combinations(array(T), n) -> array(array(T))#
+

Returns n-element sub-groups of input array. If the input array has no duplicates, +combinations returns n-element subsets.

+
SELECT combinations(ARRAY['foo', 'bar', 'baz'], 2);
+-- [['foo', 'bar'], ['foo', 'baz'], ['bar', 'baz']]
+
+SELECT combinations(ARRAY[1, 2, 3], 2);
+-- [[1, 2], [1, 3], [2, 3]]
+
+SELECT combinations(ARRAY[1, 2, 2], 2);
+-- [[1, 2], [1, 2], [2, 2]]
+
+
+

Order of sub-groups is deterministic but unspecified. Order of elements within +a sub-group deterministic but unspecified. n must be not be greater than 5, +and the total size of sub-groups generated must be smaller than 100,000.

+
+
+
+contains(x, element) boolean#
+

Returns true if the array x contains the element.

+
+
+
+contains_sequence(x, seq) boolean#
+

Return true if array x contains all of array seq as a subsequence (all values in the same consecutive order).

+
+
+
+element_at(array(E), index) E#
+

Returns element of array at given index. +If index > 0, this function provides the same functionality as the SQL-standard subscript operator ([]), +except that the function returns NULL when accessing an index larger than array length, whereas +the subscript operator would fail in such a case. +If index < 0, element_at accesses elements from the last to the first.

+
+
+
+filter(array(T), function(T, boolean)) -> array(T)#
+

Constructs an array from those elements of array for which function returns true:

+
SELECT filter(ARRAY[], x -> true);
+-- []
+
+SELECT filter(ARRAY[5, -6, NULL, 7], x -> x > 0);
+-- [5, 7]
+
+SELECT filter(ARRAY[5, NULL, 7, NULL], x -> x IS NOT NULL);
+-- [5, 7]
+
+
+
+
+
+flatten(x) array#
+

Flattens an array(array(T)) to an array(T) by concatenating the contained arrays.

+
+
+
+ngrams(array(T), n) -> array(array(T))#
+

Returns n-grams (sub-sequences of adjacent n elements) for the array. +The order of the n-grams in the result is unspecified.

+
SELECT ngrams(ARRAY['foo', 'bar', 'baz', 'foo'], 2);
+-- [['foo', 'bar'], ['bar', 'baz'], ['baz', 'foo']]
+
+SELECT ngrams(ARRAY['foo', 'bar', 'baz', 'foo'], 3);
+-- [['foo', 'bar', 'baz'], ['bar', 'baz', 'foo']]
+
+SELECT ngrams(ARRAY['foo', 'bar', 'baz', 'foo'], 4);
+-- [['foo', 'bar', 'baz', 'foo']]
+
+SELECT ngrams(ARRAY['foo', 'bar', 'baz', 'foo'], 5);
+-- [['foo', 'bar', 'baz', 'foo']]
+
+SELECT ngrams(ARRAY[1, 2, 3, 4], 2);
+-- [[1, 2], [2, 3], [3, 4]]
+
+
+
+
+
+none_match(array(T), function(T, boolean)) boolean#
+

Returns whether no elements of an array match the given predicate. Returns true if none of the elements +matches the predicate (a special case is when the array is empty); false if one or more elements match; +NULL if the predicate function returns NULL for one or more elements and false for all other elements.

+
+
+
+reduce(array(T), initialState S, inputFunction(S, T, S), outputFunction(S, R)) R#
+

Returns a single value reduced from array. inputFunction will +be invoked for each element in array in order. In addition to taking +the element, inputFunction takes the current state, initially +initialState, and returns the new state. outputFunction will be +invoked to turn the final state into the result value. It may be the +identity function (i -> i).

+
SELECT reduce(ARRAY[], 0,
+              (s, x) -> s + x,
+              s -> s);
+-- 0
+
+SELECT reduce(ARRAY[5, 20, 50], 0,
+              (s, x) -> s + x,
+              s -> s);
+-- 75
+
+SELECT reduce(ARRAY[5, 20, NULL, 50], 0,
+              (s, x) -> s + x,
+              s -> s);
+-- NULL
+
+SELECT reduce(ARRAY[5, 20, NULL, 50], 0,
+              (s, x) -> s + coalesce(x, 0),
+              s -> s);
+-- 75
+
+SELECT reduce(ARRAY[5, 20, NULL, 50], 0,
+              (s, x) -> IF(x IS NULL, s, s + x),
+              s -> s);
+-- 75
+
+SELECT reduce(ARRAY[2147483647, 1], BIGINT '0',
+              (s, x) -> s + x,
+              s -> s);
+-- 2147483648
+
+-- calculates arithmetic average
+SELECT reduce(ARRAY[5, 6, 10, 20],
+              CAST(ROW(0.0, 0) AS ROW(sum DOUBLE, count INTEGER)),
+              (s, x) -> CAST(ROW(x + s.sum, s.count + 1) AS
+                             ROW(sum DOUBLE, count INTEGER)),
+              s -> IF(s.count = 0, NULL, s.sum / s.count));
+-- 10.25
+
+
+
+
+
+repeat(element, count) array#
+

Repeat element for count times.

+
+
+
+reverse(x) array
+

Returns an array which has the reversed order of array x.

+
+
+
+sequence(start, stop)#
+

Generate a sequence of integers from start to stop, incrementing +by 1 if start is less than or equal to stop, otherwise -1.

+
+
+
+sequence(start, stop, step)
+

Generate a sequence of integers from start to stop, incrementing by step.

+
+
+
+sequence(start, stop)
+

Generate a sequence of dates from start date to stop date, incrementing +by 1 day if start date is less than or equal to stop date, otherwise -1 day.

+
+
+
+sequence(start, stop, step)
+

Generate a sequence of dates from start to stop, incrementing by step. +The type of step can be either INTERVAL DAY TO SECOND or INTERVAL YEAR TO MONTH.

+
+
+
+sequence(start, stop, step)
+

Generate a sequence of timestamps from start to stop, incrementing by step. +The type of step can be either INTERVAL DAY TO SECOND or INTERVAL YEAR TO MONTH.

+
+
+
+shuffle(x) array#
+

Generate a random permutation of the given array x.

+
+
+
+slice(x, start, length) array#
+

Subsets array x starting from index start (or starting from the end +if start is negative) with a length of length.

+
+
+
+trim_array(x, n) array#
+

Remove n elements from the end of array:

+
SELECT trim_array(ARRAY[1, 2, 3, 4], 1);
+-- [1, 2, 3]
+
+SELECT trim_array(ARRAY[1, 2, 3, 4], 2);
+-- [1, 2]
+
+
+
+
+
+transform(array(T), function(T, U)) -> array(U)#
+

Returns an array that is the result of applying function to each element of array:

+
SELECT transform(ARRAY[], x -> x + 1);
+-- []
+
+SELECT transform(ARRAY[5, 6], x -> x + 1);
+-- [6, 7]
+
+SELECT transform(ARRAY[5, NULL, 6], x -> coalesce(x, 0) + 1);
+-- [6, 1, 7]
+
+SELECT transform(ARRAY['x', 'abc', 'z'], x -> x || '0');
+-- ['x0', 'abc0', 'z0']
+
+SELECT transform(ARRAY[ARRAY[1, NULL, 2], ARRAY[3, NULL]],
+                 a -> filter(a, x -> x IS NOT NULL));
+-- [[1, 2], [3]]
+
+
+
+
+
+zip(array1, array2[, ...]) -> array(row)#
+

Merges the given arrays, element-wise, into a single array of rows. The M-th element of +the N-th argument will be the N-th field of the M-th output element. +If the arguments have an uneven length, missing values are filled with NULL.

+
SELECT zip(ARRAY[1, 2], ARRAY['1b', null, '3b']);
+-- [ROW(1, '1b'), ROW(2, null), ROW(null, '3b')]
+
+
+
+
+
+zip_with(array(T), array(U), function(T, U, R)) -> array(R)#
+

Merges the two given arrays, element-wise, into a single array using function. +If one array is shorter, nulls are appended at the end to match the length of the +longer array, before applying function.

+
SELECT zip_with(ARRAY[1, 3, 5], ARRAY['a', 'b', 'c'],
+                (x, y) -> (y, x));
+-- [ROW('a', 1), ROW('b', 3), ROW('c', 5)]
+
+SELECT zip_with(ARRAY[1, 2], ARRAY[3, 4],
+                (x, y) -> x + y);
+-- [4, 6]
+
+SELECT zip_with(ARRAY['a', 'b', 'c'], ARRAY['d', 'e', 'f'],
+                (x, y) -> concat(x, y));
+-- ['ad', 'be', 'cf']
+
+SELECT zip_with(ARRAY['a'], ARRAY['d', null, 'f'],
+                (x, y) -> coalesce(x, y));
+-- ['a', null, 'f']
+
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/binary.html b/430/functions/binary.html new file mode 100644 index 000000000..d828a3656 --- /dev/null +++ b/430/functions/binary.html @@ -0,0 +1,904 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Binary functions and operators — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Binary functions and operators#

+
+

Binary operators#

+

The || operator performs concatenation.

+
+
+

Binary functions#

+
+
+concat(binary1, ..., binaryN) varbinary
+

Returns the concatenation of binary1, binary2, ..., binaryN. +This function provides the same functionality as the +SQL-standard concatenation operator (||).

+
+
+
+length(binary) bigint
+

Returns the length of binary in bytes.

+
+
+
+lpad(binary, size, padbinary) varbinary
+

Left pads binary to size bytes with padbinary. +If size is less than the length of binary, the result is +truncated to size characters. size must not be negative +and padbinary must be non-empty.

+
+
+
+rpad(binary, size, padbinary) varbinary
+

Right pads binary to size bytes with padbinary. +If size is less than the length of binary, the result is +truncated to size characters. size must not be negative +and padbinary must be non-empty.

+
+
+
+substr(binary, start) varbinary
+

Returns the rest of binary from the starting position start, +measured in bytes. Positions start with 1. A negative starting position +is interpreted as being relative to the end of the string.

+
+
+
+substr(binary, start, length) varbinary
+

Returns a substring from binary of length length from the starting +position start, measured in bytes. Positions start with 1. A +negative starting position is interpreted as being relative to the end of +the string.

+
+
+
+reverse(binary) varbinary
+

Returns binary with the bytes in reverse order.

+
+
+
+

Base64 encoding functions#

+

The Base64 functions implement the encoding specified in RFC 4648.

+
+
+from_base64(string) varbinary#
+

Decodes binary data from the base64 encoded string.

+
+
+
+to_base64(binary) varchar#
+

Encodes binary into a base64 string representation.

+
+
+
+from_base64url(string) varbinary#
+

Decodes binary data from the base64 encoded string using the URL safe alphabet.

+
+
+
+to_base64url(binary) varchar#
+

Encodes binary into a base64 string representation using the URL safe alphabet.

+
+
+
+from_base32(string) varbinary#
+

Decodes binary data from the base32 encoded string.

+
+
+
+to_base32(binary) varchar#
+

Encodes binary into a base32 string representation.

+
+
+
+

Hex encoding functions#

+
+
+from_hex(string) varbinary#
+

Decodes binary data from the hex encoded string.

+
+
+
+to_hex(binary) varchar#
+

Encodes binary into a hex string representation.

+
+
+
+

Integer encoding functions#

+
+
+from_big_endian_32(binary) integer#
+

Decodes the 32-bit two’s complement big-endian binary. +The input must be exactly 4 bytes.

+
+
+
+to_big_endian_32(integer) varbinary#
+

Encodes integer into a 32-bit two’s complement big-endian format.

+
+
+
+from_big_endian_64(binary) bigint#
+

Decodes the 64-bit two’s complement big-endian binary. +The input must be exactly 8 bytes.

+
+
+
+to_big_endian_64(bigint) varbinary#
+

Encodes bigint into a 64-bit two’s complement big-endian format.

+
+
+
+

Floating-point encoding functions#

+
+
+from_ieee754_32(binary) real#
+

Decodes the 32-bit big-endian binary in IEEE 754 single-precision floating-point format. +The input must be exactly 4 bytes.

+
+
+
+to_ieee754_32(real) varbinary#
+

Encodes real into a 32-bit big-endian binary according to IEEE 754 single-precision floating-point format.

+
+
+
+from_ieee754_64(binary) double#
+

Decodes the 64-bit big-endian binary in IEEE 754 double-precision floating-point format. +The input must be exactly 8 bytes.

+
+
+
+to_ieee754_64(double) varbinary#
+

Encodes double into a 64-bit big-endian binary according to IEEE 754 double-precision floating-point format.

+
+
+
+

Hashing functions#

+
+
+crc32(binary) bigint#
+

Computes the CRC-32 of binary. For general purpose hashing, use +xxhash64(), as it is much faster and produces a better quality hash.

+
+
+
+md5(binary) varbinary#
+

Computes the MD5 hash of binary.

+
+
+
+sha1(binary) varbinary#
+

Computes the SHA1 hash of binary.

+
+
+
+sha256(binary) varbinary#
+

Computes the SHA256 hash of binary.

+
+
+
+sha512(binary) varbinary#
+

Computes the SHA512 hash of binary.

+
+
+
+spooky_hash_v2_32(binary) varbinary#
+

Computes the 32-bit SpookyHashV2 hash of binary.

+
+
+
+spooky_hash_v2_64(binary) varbinary#
+

Computes the 64-bit SpookyHashV2 hash of binary.

+
+
+
+xxhash64(binary) varbinary#
+

Computes the xxHash64 hash of binary.

+
+
+
+murmur3(binary) varbinary#
+

Computes the 128-bit MurmurHash3 +hash of binary.

+
+

SELECT murmur3(from_base64(‘aaaaaa’)); +– ba 58 55 63 55 69 b4 2f 49 20 37 2c a0 e3 96 ef

+
+
+
+
+

HMAC functions#

+
+
+hmac_md5(binary, key) varbinary#
+

Computes HMAC with MD5 of binary with the given key.

+
+
+
+hmac_sha1(binary, key) varbinary#
+

Computes HMAC with SHA1 of binary with the given key.

+
+
+
+hmac_sha256(binary, key) varbinary#
+

Computes HMAC with SHA256 of binary with the given key.

+
+
+
+hmac_sha512(binary, key) varbinary#
+

Computes HMAC with SHA512 of binary with the given key.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/bitwise.html b/430/functions/bitwise.html new file mode 100644 index 000000000..d35efdd1e --- /dev/null +++ b/430/functions/bitwise.html @@ -0,0 +1,758 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Bitwise functions — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Bitwise functions#

+
+
+bit_count(x, bits) bigint#
+

Count the number of bits set in x (treated as bits-bit signed +integer) in 2’s complement representation:

+
SELECT bit_count(9, 64); -- 2
+SELECT bit_count(9, 8); -- 2
+SELECT bit_count(-7, 64); -- 62
+SELECT bit_count(-7, 8); -- 6
+
+
+
+
+
+bitwise_and(x, y) bigint#
+

Returns the bitwise AND of x and y in 2’s complement representation.

+

Bitwise AND of 19 (binary: 10011) and 25 (binary: 11001) results in +17 (binary: 10001):

+
SELECT bitwise_and(19,25); -- 17
+
+
+
+
+
+bitwise_not(x) bigint#
+

Returns the bitwise NOT of x in 2’s complement representation +(NOT x = -x - 1):

+
SELECT bitwise_not(-12); --  11
+SELECT bitwise_not(19);  -- -20
+SELECT bitwise_not(25);  -- -26
+
+
+
+
+
+bitwise_or(x, y) bigint#
+

Returns the bitwise OR of x and y in 2’s complement representation.

+

Bitwise OR of 19 (binary: 10011) and 25 (binary: 11001) results in +27 (binary: 11011):

+
SELECT bitwise_or(19,25); -- 27
+
+
+
+
+
+bitwise_xor(x, y) bigint#
+

Returns the bitwise XOR of x and y in 2’s complement representation.

+

Bitwise XOR of 19 (binary: 10011) and 25 (binary: 11001) results in +10 (binary: 01010):

+
SELECT bitwise_xor(19,25); -- 10
+
+
+
+
+
+bitwise_left_shift(value, shift) [same as value]#
+

Returns the left shifted value of value.

+

Shifting 1 (binary: 001) by two bits results in 4 (binary: 00100):

+
SELECT bitwise_left_shift(1, 2); -- 4
+
+
+

Shifting 5 (binary: 0101) by two bits results in 20 (binary: 010100):

+
SELECT bitwise_left_shift(5, 2); -- 20
+
+
+

Shifting a value by 0 always results in the original value:

+
SELECT bitwise_left_shift(20, 0); -- 20
+SELECT bitwise_left_shift(42, 0); -- 42
+
+
+

Shifting 0 by a shift always results in 0:

+
SELECT bitwise_left_shift(0, 1); -- 0
+SELECT bitwise_left_shift(0, 2); -- 0
+
+
+
+
+
+bitwise_right_shift(value, shift) [same as value]#
+

Returns the logical right shifted value of value.

+

Shifting 8 (binary: 1000) by three bits results in 1 (binary: 001):

+
SELECT bitwise_right_shift(8, 3); -- 1
+
+
+

Shifting 9 (binary: 1001) by one bit results in 4 (binary: 100):

+
SELECT bitwise_right_shift(9, 1); -- 4
+
+
+

Shifting a value by 0 always results in the original value:

+
SELECT bitwise_right_shift(20, 0); -- 20
+SELECT bitwise_right_shift(42, 0); -- 42
+
+
+

Shifting a value by 64 or more bits results in 0:

+
SELECT bitwise_right_shift( 12, 64); -- 0
+SELECT bitwise_right_shift(-45, 64); -- 0
+
+
+

Shifting 0 by a shift always results in 0:

+
SELECT bitwise_right_shift(0, 1); -- 0
+SELECT bitwise_right_shift(0, 2); -- 0
+
+
+
+
+
+bitwise_right_shift_arithmetic(value, shift) [same as value]#
+

Returns the arithmetic right shifted value of value.

+

Returns the same values as bitwise_right_shift() when shifting by less than +64 bits. Shifting by 64 or more bits results in 0 for a positive and +-1 for a negative value:

+
SELECT bitwise_right_shift_arithmetic( 12, 64); --  0
+SELECT bitwise_right_shift_arithmetic(-45, 64); -- -1
+
+
+
+

See also bitwise_and_agg() and bitwise_or_agg().

+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/color.html b/430/functions/color.html new file mode 100644 index 000000000..fb1be21cf --- /dev/null +++ b/430/functions/color.html @@ -0,0 +1,714 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Color functions — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Color functions#

+
+
+bar(x, width) varchar#
+

Renders a single bar in an ANSI bar chart using a default +low_color of red and a high_color of green. For example, +if x of 25% and width of 40 are passed to this function. A +10-character red bar will be drawn followed by 30 spaces to create +a bar of 40 characters.

+
+
+
+bar(x, width, low_color, high_color) varchar
+

Renders a single line in an ANSI bar chart of the specified +width. The parameter x is a double value between 0 and 1. +Values of x that fall outside the range [0, 1] will be +truncated to either a 0 or a 1 value. The low_color and +high_color capture the color to use for either end of +the horizontal bar chart. For example, if x is 0.5, width +is 80, low_color is 0xFF0000, and high_color is 0x00FF00 +this function will return a 40 character bar that varies from red +(0xFF0000) and yellow (0xFFFF00) and the remainder of the 80 +character bar will be padded with spaces.

+
+../_images/functions_color_bar.png +
+
+
+
+color(string) color#
+

Returns a color capturing a decoded RGB value from a 4-character +string of the format “#000”. The input string should be varchar +containing a CSS-style short rgb string or one of black, +red, green, yellow, blue, magenta, cyan, +white.

+
+
+
+color(x, low, high, low_color, high_color) color
+

Returns a color interpolated between low_color and +high_color using the double parameters x, low, and +high to calculate a fraction which is then passed to the +color(fraction, low_color, high_color) function shown below. +If x falls outside the range defined by low and high +its value is truncated to fit within this range.

+
+
+
+color(x, low_color, high_color) color
+

Returns a color interpolated between low_color and +high_color according to the double argument x between 0 +and 1. The parameter x is a double value between 0 and 1. +Values of x that fall outside the range [0, 1] will be +truncated to either a 0 or a 1 value.

+
+
+
+render(x, color) varchar#
+

Renders value x using the specific color using ANSI +color codes. x can be either a double, bigint, or varchar.

+
+
+
+render(b) varchar
+

Accepts boolean value b and renders a green true or a red +false using ANSI color codes.

+
+
+
+rgb(red, green, blue) color#
+

Returns a color value capturing the RGB value of three +component color values supplied as int parameters ranging from 0 +to 255: red, green, blue.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/comparison.html b/430/functions/comparison.html new file mode 100644 index 000000000..2cfdca685 --- /dev/null +++ b/430/functions/comparison.html @@ -0,0 +1,976 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Comparison functions and operators — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Comparison functions and operators#

+
+

Comparison operators#

+ ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Operator

Description

<

Less than

>

Greater than

<=

Less than or equal to

>=

Greater than or equal to

=

Equal

<>

Not equal

!=

Not equal (non-standard but popular syntax)

+
+
+

Range operator: BETWEEN#

+

The BETWEEN operator tests if a value is within a specified range. +It uses the syntax value BETWEEN min AND max:

+
SELECT 3 BETWEEN 2 AND 6;
+
+
+

The statement shown above is equivalent to the following statement:

+
SELECT 3 >= 2 AND 3 <= 6;
+
+
+

To test if a value does not fall within the specified range +use NOT BETWEEN:

+
SELECT 3 NOT BETWEEN 2 AND 6;
+
+
+

The statement shown above is equivalent to the following statement:

+
SELECT 3 < 2 OR 3 > 6;
+
+
+

A NULL in a BETWEEN or NOT BETWEEN statement is evaluated +using the standard NULL evaluation rules applied to the equivalent +expression above:

+
SELECT NULL BETWEEN 2 AND 4; -- null
+
+SELECT 2 BETWEEN NULL AND 6; -- null
+
+SELECT 2 BETWEEN 3 AND NULL; -- false
+
+SELECT 8 BETWEEN NULL AND 6; -- false
+
+
+

The BETWEEN and NOT BETWEEN operators can also be used to +evaluate any orderable type. For example, a VARCHAR:

+
SELECT 'Paul' BETWEEN 'John' AND 'Ringo'; -- true
+
+
+

Note that the value, min, and max parameters to BETWEEN and NOT BETWEEN must be the same type. For example, Trino will produce an +error if you ask it if John is between 2.3 and 35.2.

+
+
+

IS NULL and IS NOT NULL#

+

The IS NULL and IS NOT NULL operators test whether a value +is null (undefined). Both operators work for all data types.

+

Using NULL with IS NULL evaluates to true:

+
select NULL IS NULL; -- true
+
+
+

But any other constant does not:

+
SELECT 3.0 IS NULL; -- false
+
+
+
+
+

IS DISTINCT FROM and IS NOT DISTINCT FROM#

+

In SQL a NULL value signifies an unknown value, so any comparison +involving a NULL will produce NULL. The IS DISTINCT FROM +and IS NOT DISTINCT FROM operators treat NULL as a known value +and both operators guarantee either a true or false outcome even in +the presence of NULL input:

+
SELECT NULL IS DISTINCT FROM NULL; -- false
+
+SELECT NULL IS NOT DISTINCT FROM NULL; -- true
+
+
+

In the example shown above, a NULL value is not considered +distinct from NULL. When you are comparing values which may +include NULL use these operators to guarantee either a TRUE or +FALSE result.

+

The following truth table demonstrate the handling of NULL in +IS DISTINCT FROM and IS NOT DISTINCT FROM:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

a

b

a = b

a <> b

a DISTINCT b

a NOT DISTINCT b

1

1

TRUE

FALSE

FALSE

TRUE

1

2

FALSE

TRUE

TRUE

FALSE

1

NULL

NULL

NULL

TRUE

FALSE

NULL

NULL

NULL

NULL

FALSE

TRUE

+
+
+

GREATEST and LEAST#

+

These functions are not in the SQL standard, but are a common extension. +Like most other functions in Trino, they return null if any argument is +null. Note that in some other databases, such as PostgreSQL, they only +return null if all arguments are null.

+

The following types are supported: +DOUBLE, +BIGINT, +VARCHAR, +TIMESTAMP, +TIMESTAMP WITH TIME ZONE, +DATE

+
+
+greatest(value1, value2, ..., valueN) [same as input]#
+

Returns the largest of the provided values.

+
+
+
+least(value1, value2, ..., valueN) [same as input]#
+

Returns the smallest of the provided values.

+
+
+
+

Quantified comparison predicates: ALL, ANY and SOME#

+

The ALL, ANY and SOME quantifiers can be used together with comparison operators in the +following way:

+
expression operator quantifier ( subquery )
+
+
+

For example:

+
SELECT 'hello' = ANY (VALUES 'hello', 'world'); -- true
+
+SELECT 21 < ALL (VALUES 19, 20, 21); -- false
+
+SELECT 42 >= SOME (SELECT 41 UNION ALL SELECT 42 UNION ALL SELECT 43); -- true
+
+
+

Here are the meanings of some quantifier and comparison operator combinations:

+ ++++ + + + + + + + + + + + + + + + + + + + + + + + + + +

Expression

Meaning

A = ALL (...)

Evaluates to true when A is equal to all values.

A <> ALL (...)

Evaluates to true when A doesn’t match any value.

A < ALL (...)

Evaluates to true when A is smaller than the smallest value.

A = ANY (...)

Evaluates to true when A is equal to any of the values. This form +is equivalent to A IN (...).

A <> ANY (...)

Evaluates to true when A doesn’t match one or more values.

A < ANY (...)

Evaluates to true when A is smaller than the biggest value.

+

ANY and SOME have the same meaning and can be used interchangeably.

+
+
+

Pattern comparison: LIKE#

+

The LIKE operator can be used to compare values with a pattern:

+
... column [NOT] LIKE 'pattern' ESCAPE 'character';
+
+
+

Matching characters is case sensitive, and the pattern supports two symbols for +matching:

+
    +
  • _ matches any single character

  • +
  • % matches zero or more characters

  • +
+

Typically it is often used as a condition in WHERE statements. An example is +a query to find all continents starting with E, which returns Europe:

+
SELECT * FROM (VALUES 'America', 'Asia', 'Africa', 'Europe', 'Australia', 'Antarctica') AS t (continent)
+WHERE continent LIKE 'E%';
+
+
+

You can negate the result by adding NOT, and get all other continents, all +not starting with E:

+
SELECT * FROM (VALUES 'America', 'Asia', 'Africa', 'Europe', 'Australia', 'Antarctica') AS t (continent)
+WHERE continent NOT LIKE 'E%';
+
+
+

If you only have one specific character to match, you can use the _ symbol +for each character. The following query uses two underscores and produces only +Asia as result:

+
SELECT * FROM (VALUES 'America', 'Asia', 'Africa', 'Europe', 'Australia', 'Antarctica') AS t (continent)
+WHERE continent LIKE 'A__A';
+
+
+

The wildcard characters _ and % must be escaped to allow you to match +them as literals. This can be achieved by specifying the ESCAPE character to +use:

+
SELECT 'South_America' LIKE 'South\_America' ESCAPE '\';
+
+
+

The above query returns true since the escaped underscore symbol matches. If +you need to match the used escape character as well, you can escape it.

+

If you want to match for the chosen escape character, you simply escape itself. +For example, you can use \\ to match for \.

+
+
+

Row comparison: IN#

+

The IN operator can be used in a WHERE clause to compare column values with +a list of values. The list of values can be supplied by a subquery or directly +as static values in an array:

+
... WHERE column [NOT] IN ('value1','value2');
+... WHERE column [NOT] IN ( subquery );
+
+
+

Use the optional NOT keyword to negate the condition.

+

The following example shows a simple usage with a static array:

+
SELECT * FROM region WHERE name IN ('AMERICA', 'EUROPE');
+
+
+

The values in the clause are used for multiple comparisons that are combined as +a logical OR. The preceding query is equivalent to the following query:

+
SELECT * FROM region WHERE name = 'AMERICA' OR name = 'EUROPE';
+
+
+

You can negate the comparisons by adding NOT, and get all other regions +except the values in list:

+
SELECT * FROM region WHERE name NOT IN ('AMERICA', 'EUROPE');
+
+
+

When using a subquery to determine the values to use in the comparison, the +subquery must return a single column and one or more rows.

+
SELECT name
+FROM nation
+WHERE regionkey IN (
+    SELECT starts_with(regionkey,"A") AS regionkey
+    FROM region
+);
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/conditional.html b/430/functions/conditional.html new file mode 100644 index 000000000..7f24c7004 --- /dev/null +++ b/430/functions/conditional.html @@ -0,0 +1,842 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Conditional expressions — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Conditional expressions#

+
+

CASE#

+

The standard SQL CASE expression has two forms. +The “simple” form searches each value expression from left to right +until it finds one that equals expression:

+
CASE expression
+    WHEN value THEN result
+    [ WHEN ... ]
+    [ ELSE result ]
+END
+
+
+

The result for the matching value is returned. +If no match is found, the result from the ELSE clause is +returned if it exists, otherwise null is returned. Example:

+
SELECT a,
+       CASE a
+           WHEN 1 THEN 'one'
+           WHEN 2 THEN 'two'
+           ELSE 'many'
+       END
+
+
+

The “searched” form evaluates each boolean condition from left +to right until one is true and returns the matching result:

+
CASE
+    WHEN condition THEN result
+    [ WHEN ... ]
+    [ ELSE result ]
+END
+
+
+

If no conditions are true, the result from the ELSE clause is +returned if it exists, otherwise null is returned. Example:

+
SELECT a, b,
+       CASE
+           WHEN a = 1 THEN 'aaa'
+           WHEN b = 2 THEN 'bbb'
+           ELSE 'ccc'
+       END
+
+
+
+
+

IF#

+

The IF expression has two forms, one supplying only a +true_value and the other supplying both a true_value and a +false_value:

+
+
+if(condition, true_value)#
+

Evaluates and returns true_value if condition is true, +otherwise null is returned and true_value is not evaluated.

+
+
+
+if(condition, true_value, false_value)
+

Evaluates and returns true_value if condition is true, +otherwise evaluates and returns false_value.

+
+

The following IF and CASE expressions are equivalent:

+
SELECT
+  orderkey,
+  totalprice,
+  IF(totalprice >= 150000, 'High Value', 'Low Value')
+FROM tpch.sf1.orders;
+
+
+
SELECT
+  orderkey,
+  totalprice,
+  CASE
+    WHEN totalprice >= 150000 THEN 'High Value'
+    ELSE 'Low Value'
+  END
+FROM tpch.sf1.orders;
+
+
+
+
+

COALESCE#

+
+
+coalesce(value1, value2[, ...])#
+

Returns the first non-null value in the argument list. +Like a CASE expression, arguments are only evaluated if necessary.

+
+
+
+

NULLIF#

+
+
+nullif(value1, value2)#
+

Returns null if value1 equals value2, otherwise returns value1.

+
+
+
+

TRY#

+
+
+try(expression)#
+

Evaluate an expression and handle certain types of errors by returning +NULL.

+
+

In cases where it is preferable that queries produce NULL or default values +instead of failing when corrupt or invalid data is encountered, the TRY +function may be useful. To specify default values, the TRY function can be +used in conjunction with the COALESCE function.

+

The following errors are handled by TRY:

+
    +
  • Division by zero

  • +
  • Invalid cast or function argument

  • +
  • Numeric value out of range

  • +
+
+

Examples#

+

Source table with some invalid data:

+
SELECT * FROM shipping;
+
+
+
 origin_state | origin_zip | packages | total_cost
+--------------+------------+----------+------------
+ California   |      94131 |       25 |        100
+ California   |      P332a |        5 |         72
+ California   |      94025 |        0 |        155
+ New Jersey   |      08544 |      225 |        490
+(4 rows)
+
+
+

Query failure without TRY:

+
SELECT CAST(origin_zip AS BIGINT) FROM shipping;
+
+
+
Query failed: Cannot cast 'P332a' to BIGINT
+
+
+

NULL values with TRY:

+
SELECT TRY(CAST(origin_zip AS BIGINT)) FROM shipping;
+
+
+
 origin_zip
+------------
+      94131
+ NULL
+      94025
+      08544
+(4 rows)
+
+
+

Query failure without TRY:

+
SELECT total_cost / packages AS per_package FROM shipping;
+
+
+
Query failed: Division by zero
+
+
+

Default values with TRY and COALESCE:

+
SELECT COALESCE(TRY(total_cost / packages), 0) AS per_package FROM shipping;
+
+
+
 per_package
+-------------
+          4
+         14
+          0
+         19
+(4 rows)
+
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/conversion.html b/430/functions/conversion.html new file mode 100644 index 000000000..5925b9a44 --- /dev/null +++ b/430/functions/conversion.html @@ -0,0 +1,799 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Conversion functions — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Conversion functions#

+

Trino will implicitly convert numeric and character values to the +correct type if such a conversion is possible. Trino will not convert +between character and numeric types. For example, a query that expects +a varchar will not automatically convert a bigint value to an +equivalent varchar.

+

When necessary, values can be explicitly cast to a particular type.

+
+

Conversion functions#

+
+
+cast(value AS type) type#
+

Explicitly cast a value as a type. This can be used to cast a +varchar to a numeric value type and vice versa.

+
+
+
+try_cast(value AS type) type#
+

Like cast(), but returns null if the cast fails.

+
+
+
+

Formatting#

+
+
+format(format, args...) varchar#
+

Returns a formatted string using the specified format string +and arguments:

+
SELECT format('%s%%', 123);
+-- '123%'
+
+SELECT format('%.5f', pi());
+-- '3.14159'
+
+SELECT format('%03d', 8);
+-- '008'
+
+SELECT format('%,.2f', 1234567.89);
+-- '1,234,567.89'
+
+SELECT format('%-7s,%7s', 'hello', 'world');
+-- 'hello  ,  world'
+
+SELECT format('%2$s %3$s %1$s', 'a', 'b', 'c');
+-- 'b c a'
+
+SELECT format('%1$tA, %1$tB %1$te, %1$tY', date '2006-07-04');
+-- 'Tuesday, July 4, 2006'
+
+
+
+
+
+format_number(number) varchar#
+

Returns a formatted string using a unit symbol:

+
SELECT format_number(123456); -- '123K'
+SELECT format_number(1000000); -- '1M'
+
+
+
+
+
+

Data size#

+

The parse_data_size function supports the following units:

+ +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Unit

Description

Value

B

Bytes

1

kB

Kilobytes

1024

MB

Megabytes

10242

GB

Gigabytes

10243

TB

Terabytes

10244

PB

Petabytes

10245

EB

Exabytes

10246

ZB

Zettabytes

10247

YB

Yottabytes

10248

+
+
+parse_data_size(string)#
+

Parses string of format value unit into a number, where +value is the fractional number of unit values:

+
SELECT parse_data_size('1B'); -- 1
+SELECT parse_data_size('1kB'); -- 1024
+SELECT parse_data_size('1MB'); -- 1048576
+SELECT parse_data_size('2.3MB'); -- 2411724
+
+
+
+
+
+

Miscellaneous#

+
+
+typeof(expr) varchar#
+

Returns the name of the type of the provided expression:

+
SELECT typeof(123); -- integer
+SELECT typeof('cat'); -- varchar(3)
+SELECT typeof(cos(2) + 1.5); -- double
+
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/datetime.html b/430/functions/datetime.html new file mode 100644 index 000000000..0661ff146 --- /dev/null +++ b/430/functions/datetime.html @@ -0,0 +1,1498 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Date and time functions and operators — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Date and time functions and operators#

+

These functions and operators operate on date and time data types.

+
+

Date and time operators#

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Operator

Example

Result

+

date '2012-08-08' + interval '2' day

2012-08-10

+

time '01:00' + interval '3' hour

04:00:00.000

+

timestamp '2012-08-08 01:00' + interval '29' hour

2012-08-09 06:00:00.000

+

timestamp '2012-10-31 01:00' + interval '1' month

2012-11-30 01:00:00.000

+

interval '2' day + interval '3' hour

2 03:00:00.000

+

interval '3' year + interval '5' month

3-5

-

date '2012-08-08' - interval '2' day

2012-08-06

-

time '01:00' - interval '3' hour

22:00:00.000

-

timestamp '2012-08-08 01:00' - interval '29' hour

2012-08-06 20:00:00.000

-

timestamp '2012-10-31 01:00' - interval '1' month

2012-09-30 01:00:00.000

-

interval '2' day - interval '3' hour

1 21:00:00.000

-

interval '3' year - interval '5' month

2-7

+
+
+

Time zone conversion#

+

The AT TIME ZONE operator sets the time zone of a timestamp:

+
SELECT timestamp '2012-10-31 01:00 UTC';
+-- 2012-10-31 01:00:00.000 UTC
+
+SELECT timestamp '2012-10-31 01:00 UTC' AT TIME ZONE 'America/Los_Angeles';
+-- 2012-10-30 18:00:00.000 America/Los_Angeles
+
+
+
+
+

Date and time functions#

+
+
+current_date#
+

Returns the current date as of the start of the query.

+
+
+
+current_time#
+

Returns the current time with time zone as of the start of the query.

+
+
+
+current_timestamp#
+

Returns the current timestamp with time zone as of the start of the query, +with 3 digits of subsecond precision,

+
+
+
+current_timestamp(p)
+

Returns the current timestamp with time zone as of the start of the query, with +p digits of subsecond precision:

+
SELECT current_timestamp(6);
+-- 2020-06-24 08:25:31.759993 America/Los_Angeles
+
+
+
+
+
+current_timezone() varchar#
+

Returns the current time zone in the format defined by IANA +(e.g., America/Los_Angeles) or as fixed offset from UTC (e.g., +08:35)

+
+
+
+date(x) date#
+

This is an alias for CAST(x AS date).

+
+
+
+last_day_of_month(x) date#
+

Returns the last day of the month.

+
+
+
+from_iso8601_timestamp(string) timestamp(3) with time zone#
+

Parses the ISO 8601 formatted date string, optionally with time and time +zone, into a timestamp(3) with time zone. The time defaults to +00:00:00.000, and the time zone defaults to the session time zone:

+
SELECT from_iso8601_timestamp('2020-05-11');
+-- 2020-05-11 00:00:00.000 America/Vancouver
+
+SELECT from_iso8601_timestamp('2020-05-11T11:15:05');
+-- 2020-05-11 11:15:05.000 America/Vancouver
+
+SELECT from_iso8601_timestamp('2020-05-11T11:15:05.055+01:00');
+-- 2020-05-11 11:15:05.055 +01:00
+
+
+
+
+
+from_iso8601_timestamp_nanos(string) timestamp(9) with time zone#
+

Parses the ISO 8601 formatted date and time string. The time zone +defaults to the session time zone:

+
SELECT from_iso8601_timestamp_nanos('2020-05-11T11:15:05');
+-- 2020-05-11 11:15:05.000000000 America/Vancouver
+
+SELECT from_iso8601_timestamp_nanos('2020-05-11T11:15:05.123456789+01:00');
+-- 2020-05-11 11:15:05.123456789 +01:00
+
+
+
+
+
+from_iso8601_date(string) date#
+

Parses the ISO 8601 formatted date string into a date. The date can +be a calendar date, a week date using ISO week numbering, or year and day +of year combined:

+
SELECT from_iso8601_date('2020-05-11');
+-- 2020-05-11
+
+SELECT from_iso8601_date('2020-W10');
+-- 2020-03-02
+
+SELECT from_iso8601_date('2020-123');
+-- 2020-05-02
+
+
+
+
+
+at_timezone(timestamp(p), zone) timestamp(p) with time zone#
+

Returns the timestamp specified in timestamp with the time zone +converted from the session time zone to the time zone specified in zone +with precision p. In the following example, the session time zone is set +to America/New_York, which is three hours ahead of +America/Los_Angeles:

+
SELECT current_timezone()
+-- America/New_York
+
+SELECT at_timezone(TIMESTAMP '2022-11-01 09:08:07.321', 'America/Los_Angeles')
+-- 2022-11-01 06:08:07.321 America/Los_Angeles
+
+
+
+
+
+with_timezone(timestamp(p), zone) timestamp(p) with time zone#
+

Returns the timestamp specified in timestamp with the time zone +specified in zone with precision p:

+
SELECT current_timezone()
+-- America/New_York
+
+SELECT with_timezone(TIMESTAMP '2022-11-01 09:08:07.321', 'America/Los_Angeles')
+-- 2022-11-01 09:08:07.321 America/Los_Angeles
+
+
+
+
+
+from_unixtime(unixtime) timestamp(3) with time zone#
+

Returns the UNIX timestamp unixtime as a timestamp with time zone. unixtime is the +number of seconds since 1970-01-01 00:00:00 UTC.

+
+
+
+from_unixtime(unixtime, zone) timestamp(3) with time zone
+

Returns the UNIX timestamp unixtime as a timestamp with time zone +using zone for the time zone. unixtime is the number of seconds +since 1970-01-01 00:00:00 UTC.

+
+
+
+from_unixtime(unixtime, hours, minutes) timestamp(3) with time zone
+

Returns the UNIX timestamp unixtime as a timestamp with time zone +using hours and minutes for the time zone offset. unixtime is +the number of seconds since 1970-01-01 00:00:00 in double data type.

+
+
+
+from_unixtime_nanos(unixtime) timestamp(9) with time zone#
+

Returns the UNIX timestamp unixtime as a timestamp with time zone. unixtime is the +number of nanoseconds since 1970-01-01 00:00:00.000000000 UTC:

+
SELECT from_unixtime_nanos(100);
+-- 1970-01-01 00:00:00.000000100 UTC
+
+SELECT from_unixtime_nanos(DECIMAL '1234');
+-- 1970-01-01 00:00:00.000001234 UTC
+
+SELECT from_unixtime_nanos(DECIMAL '1234.499');
+-- 1970-01-01 00:00:00.000001234 UTC
+
+SELECT from_unixtime_nanos(DECIMAL '-1234');
+-- 1969-12-31 23:59:59.999998766 UTC
+
+
+
+
+
+localtime#
+

Returns the current time as of the start of the query.

+
+
+
+localtimestamp#
+

Returns the current timestamp as of the start of the query, with 3 +digits of subsecond precision.

+
+
+
+localtimestamp(p)
+

Returns the current timestamp as of the start +of the query, with p digits of subsecond precision:

+
SELECT localtimestamp(6);
+-- 2020-06-10 15:55:23.383628
+
+
+
+
+
+now() timestamp(3) with time zone#
+

This is an alias for current_timestamp.

+
+
+
+to_iso8601(x) varchar#
+

Formats x as an ISO 8601 string. x can be date, timestamp, or +timestamp with time zone.

+
+
+
+to_milliseconds(interval) bigint#
+

Returns the day-to-second interval as milliseconds.

+
+
+
+to_unixtime(timestamp) double#
+

Returns timestamp as a UNIX timestamp.

+
+
+

Note

+

The following SQL-standard functions do not use parenthesis:

+
    +
  • current_date

  • +
  • current_time

  • +
  • current_timestamp

  • +
  • localtime

  • +
  • localtimestamp

  • +
+
+
+
+

Truncation function#

+

The date_trunc function supports the following units:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Unit

Example Truncated Value

second

2001-08-22 03:04:05.000

minute

2001-08-22 03:04:00.000

hour

2001-08-22 03:00:00.000

day

2001-08-22 00:00:00.000

week

2001-08-20 00:00:00.000

month

2001-08-01 00:00:00.000

quarter

2001-07-01 00:00:00.000

year

2001-01-01 00:00:00.000

+

The above examples use the timestamp 2001-08-22 03:04:05.321 as the input.

+
+
+date_trunc(unit, x) [same as input]#
+

Returns x truncated to unit:

+
SELECT date_trunc('day' , TIMESTAMP '2022-10-20 05:10:00');
+-- 2022-10-20 00:00:00.000
+
+SELECT date_trunc('month' , TIMESTAMP '2022-10-20 05:10:00');
+-- 2022-10-01 00:00:00.000
+
+SELECT date_trunc('year', TIMESTAMP '2022-10-20 05:10:00');
+-- 2022-01-01 00:00:00.000
+
+
+
+
+
+

Interval functions#

+

The functions in this section support the following interval units:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Unit

Description

millisecond

Milliseconds

second

Seconds

minute

Minutes

hour

Hours

day

Days

week

Weeks

month

Months

quarter

Quarters of a year

year

Years

+
+
+date_add(unit, value, timestamp) [same as input]#
+

Adds an interval value of type unit to timestamp. +Subtraction can be performed by using a negative value:

+
SELECT date_add('second', 86, TIMESTAMP '2020-03-01 00:00:00');
+-- 2020-03-01 00:01:26.000
+
+SELECT date_add('hour', 9, TIMESTAMP '2020-03-01 00:00:00');
+-- 2020-03-01 09:00:00.000
+
+SELECT date_add('day', -1, TIMESTAMP '2020-03-01 00:00:00 UTC');
+-- 2020-02-29 00:00:00.000 UTC
+
+
+
+
+
+date_diff(unit, timestamp1, timestamp2) bigint#
+

Returns timestamp2 - timestamp1 expressed in terms of unit:

+
SELECT date_diff('second', TIMESTAMP '2020-03-01 00:00:00', TIMESTAMP '2020-03-02 00:00:00');
+-- 86400
+
+SELECT date_diff('hour', TIMESTAMP '2020-03-01 00:00:00 UTC', TIMESTAMP '2020-03-02 00:00:00 UTC');
+-- 24
+
+SELECT date_diff('day', DATE '2020-03-01', DATE '2020-03-02');
+-- 1
+
+SELECT date_diff('second', TIMESTAMP '2020-06-01 12:30:45.000000000', TIMESTAMP '2020-06-02 12:30:45.123456789');
+-- 86400
+
+SELECT date_diff('millisecond', TIMESTAMP '2020-06-01 12:30:45.000000000', TIMESTAMP '2020-06-02 12:30:45.123456789');
+-- 86400123
+
+
+
+
+
+

Duration function#

+

The parse_duration function supports the following units:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Unit

Description

ns

Nanoseconds

us

Microseconds

ms

Milliseconds

s

Seconds

m

Minutes

h

Hours

d

Days

+
+
+parse_duration(string) interval#
+

Parses string of format value unit into an interval, where +value is fractional number of unit values:

+
SELECT parse_duration('42.8ms');
+-- 0 00:00:00.043
+
+SELECT parse_duration('3.81 d');
+-- 3 19:26:24.000
+
+SELECT parse_duration('5m');
+-- 0 00:05:00.000
+
+
+
+
+
+human_readable_seconds(double) varchar#
+

Formats the double value of seconds into a human readable string containing +weeks, days, hours, minutes, and seconds:

+
SELECT human_readable_seconds(96);
+-- 1 minute, 36 seconds
+
+SELECT human_readable_seconds(3762);
+-- 1 hour, 2 minutes, 42 seconds
+
+SELECT human_readable_seconds(56363463);
+-- 93 weeks, 1 day, 8 hours, 31 minutes, 3 seconds
+
+
+
+
+
+

MySQL date functions#

+

The functions in this section use a format string that is compatible with +the MySQL date_parse and str_to_date functions. The following table, +based on the MySQL manual, describes the format specifiers:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Specifier

Description

%a

Abbreviated weekday name (Sun .. Sat)

%b

Abbreviated month name (Jan .. Dec)

%c

Month, numeric (1 .. 12), this specifier does not support 0 as a month.

%D

Day of the month with English suffix (0th, 1st, 2nd, 3rd, …)

%d

Day of the month, numeric (01 .. 31), this specifier does not support 0 as a month or day.

%e

Day of the month, numeric (1 .. 31), this specifier does not support 0 as a day.

%f

Fraction of second (6 digits for printing: 000000 .. 999000; 1 - 9 digits for parsing: 0 .. 999999999), timestamp is truncated to milliseconds.

%H

Hour (00 .. 23)

%h

Hour (01 .. 12)

%I

Hour (01 .. 12)

%i

Minutes, numeric (00 .. 59)

%j

Day of year (001 .. 366)

%k

Hour (0 .. 23)

%l

Hour (1 .. 12)

%M

Month name (January .. December)

%m

Month, numeric (01 .. 12), this specifier does not support 0 as a month.

%p

AM or PM

%r

Time of day, 12-hour (equivalent to %h:%i:%s %p)

%S

Seconds (00 .. 59)

%s

Seconds (00 .. 59)

%T

Time of day, 24-hour (equivalent to %H:%i:%s)

%U

Week (00 .. 53), where Sunday is the first day of the week

%u

Week (00 .. 53), where Monday is the first day of the week

%V

Week (01 .. 53), where Sunday is the first day of the week; used with %X

%v

Week (01 .. 53), where Monday is the first day of the week; used with %x

%W

Weekday name (Sunday .. Saturday)

%w

Day of the week (0 .. 6), where Sunday is the first day of the week, this specifier is not supported,consider using day_of_week() (it uses 1-7 instead of 0-6).

%X

Year for the week where Sunday is the first day of the week, numeric, four digits; used with %V

%x

Year for the week, where Monday is the first day of the week, numeric, four digits; used with %v

%Y

Year, numeric, four digits

%y

Year, numeric (two digits), when parsing, two-digit year format assumes range 1970 .. 2069, so “70” will result in year 1970 but “69” will produce 2069.

%%

A literal % character

%x

x, for any x not listed above

+
+

Warning

+

The following specifiers are not currently supported: %D %U %u %V %w %X

+
+
+
+date_format(timestamp, format) varchar#
+

Formats timestamp as a string using format:

+
SELECT date_format(TIMESTAMP '2022-10-20 05:10:00', '%m-%d-%Y %H');
+-- 10-20-2022 05
+
+
+
+
+
+date_parse(string, format)#
+

Parses string into a timestamp using format:

+
SELECT date_parse('2022/10/20/05', '%Y/%m/%d/%H');
+-- 2022-10-20 05:00:00.000
+
+
+
+
+
+

Java date functions#

+

The functions in this section use a format string that is compatible with +JodaTime’s DateTimeFormat pattern format.

+
+
+format_datetime(timestamp, format) varchar#
+

Formats timestamp as a string using format.

+
+
+
+parse_datetime(string, format) timestamp with time zone#
+

Parses string into a timestamp with time zone using format.

+
+
+
+

Extraction function#

+

The extract function supports the following fields:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Field

Description

YEAR

year()

QUARTER

quarter()

MONTH

month()

WEEK

week()

DAY

day()

DAY_OF_MONTH

day()

DAY_OF_WEEK

day_of_week()

DOW

day_of_week()

DAY_OF_YEAR

day_of_year()

DOY

day_of_year()

YEAR_OF_WEEK

year_of_week()

YOW

year_of_week()

HOUR

hour()

MINUTE

minute()

SECOND

second()

TIMEZONE_HOUR

timezone_hour()

TIMEZONE_MINUTE

timezone_minute()

+

The types supported by the extract function vary depending on the +field to be extracted. Most fields support all date and time types.

+
+
+extract(field FROM x) bigint#
+

Returns field from x:

+
SELECT extract(YEAR FROM TIMESTAMP '2022-10-20 05:10:00');
+-- 2022
+
+
+
+

Note

+

This SQL-standard function uses special syntax for specifying the arguments.

+
+
+
+
+

Convenience extraction functions#

+
+
+day(x) bigint#
+

Returns the day of the month from x.

+
+
+
+day_of_month(x) bigint#
+

This is an alias for day().

+
+
+
+day_of_week(x) bigint#
+

Returns the ISO day of the week from x. +The value ranges from 1 (Monday) to 7 (Sunday).

+
+
+
+day_of_year(x) bigint#
+

Returns the day of the year from x. +The value ranges from 1 to 366.

+
+
+
+dow(x) bigint#
+

This is an alias for day_of_week().

+
+
+
+doy(x) bigint#
+

This is an alias for day_of_year().

+
+
+
+hour(x) bigint#
+

Returns the hour of the day from x. +The value ranges from 0 to 23.

+
+
+
+millisecond(x) bigint#
+

Returns the millisecond of the second from x.

+
+
+
+minute(x) bigint#
+

Returns the minute of the hour from x.

+
+
+
+month(x) bigint#
+

Returns the month of the year from x.

+
+
+
+quarter(x) bigint#
+

Returns the quarter of the year from x. +The value ranges from 1 to 4.

+
+
+
+second(x) bigint#
+

Returns the second of the minute from x.

+
+
+
+timezone_hour(timestamp) bigint#
+

Returns the hour of the time zone offset from timestamp.

+
+
+
+timezone_minute(timestamp) bigint#
+

Returns the minute of the time zone offset from timestamp.

+
+
+
+week(x) bigint#
+

Returns the ISO week of the year from x. +The value ranges from 1 to 53.

+
+
+
+week_of_year(x) bigint#
+

This is an alias for week().

+
+
+
+year(x) bigint#
+

Returns the year from x.

+
+
+
+year_of_week(x) bigint#
+

Returns the year of the ISO week from x.

+
+
+
+yow(x) bigint#
+

This is an alias for year_of_week().

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/decimal.html b/430/functions/decimal.html new file mode 100644 index 000000000..d56dd0566 --- /dev/null +++ b/430/functions/decimal.html @@ -0,0 +1,762 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Decimal functions and operators — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Decimal functions and operators#

+
+

Decimal literals#

+

Use the DECIMAL 'xxxxxxx.yyyyyyy' syntax to define a decimal literal.

+

The precision of a decimal type for a literal will be equal to the number of digits +in the literal (including trailing and leading zeros). The scale will be equal +to the number of digits in the fractional part (including trailing zeros).

+ ++++ + + + + + + + + + + + + + + + + +

Example literal

Data type

DECIMAL '0'

DECIMAL(1)

DECIMAL '12345'

DECIMAL(5)

DECIMAL '0000012345.1234500000'

DECIMAL(20, 10)

+
+
+

Binary arithmetic decimal operators#

+

Standard mathematical operators are supported. The table below explains +precision and scale calculation rules for result. +Assuming x is of type DECIMAL(xp, xs) and y is of type DECIMAL(yp, ys).

+ +++++ + + + + + + + + + + + + + + + + + + + + + + + + +

Operation

Result type precision

Result type scale

x + y and x - y

min(38,
+    1 +
+    max(xs, ys) +
+    max(xp - xs, yp - ys)
+)
+
+
+

max(xs, ys)

x * y

min(38, xp + yp)
+
+
+

xs + ys

x / y

min(38,
+    xp + ys-xs
+    + max(0, ys-xs)
+    )
+
+
+

max(xs, ys)

x % y

min(xp - xs, yp - ys) +
+max(xs, bs)
+
+
+

max(xs, ys)

+

If the mathematical result of the operation is not exactly representable with +the precision and scale of the result data type, +then an exception condition is raised: Value is out of range.

+

When operating on decimal types with different scale and precision, the values are +first coerced to a common super type. For types near the largest representable precision (38), +this can result in Value is out of range errors when one of the operands doesn’t fit +in the common super type. For example, the common super type of decimal(38, 0) and +decimal(38, 1) is decimal(38, 1), but certain values that fit in decimal(38, 0) +cannot be represented as a decimal(38, 1).

+
+
+

Comparison operators#

+

All standard Comparison functions and operators work for the decimal type.

+
+
+

Unary decimal operators#

+

The - operator performs negation. The type of result is same as type of argument.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/geospatial.html b/430/functions/geospatial.html new file mode 100644 index 000000000..480756b32 --- /dev/null +++ b/430/functions/geospatial.html @@ -0,0 +1,1240 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Geospatial functions — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Geospatial functions#

+

Trino Geospatial functions that begin with the ST_ prefix support the SQL/MM specification +and are compliant with the Open Geospatial Consortium’s (OGC) OpenGIS Specifications. +As such, many Trino Geospatial functions require, or more accurately, assume that +geometries that are operated on are both simple and valid. For example, it does not +make sense to calculate the area of a polygon that has a hole defined outside of the +polygon, or to construct a polygon from a non-simple boundary line.

+

Trino Geospatial functions support the Well-Known Text (WKT) and Well-Known Binary (WKB) form of spatial objects:

+
    +
  • POINT (0 0)

  • +
  • LINESTRING (0 0, 1 1, 1 2)

  • +
  • POLYGON ((0 0, 4 0, 4 4, 0 4, 0 0), (1 1, 2 1, 2 2, 1 2, 1 1))

  • +
  • MULTIPOINT (0 0, 1 2)

  • +
  • MULTILINESTRING ((0 0, 1 1, 1 2), (2 3, 3 2, 5 4))

  • +
  • MULTIPOLYGON (((0 0, 4 0, 4 4, 0 4, 0 0), (1 1, 2 1, 2 2, 1 2, 1 1)), ((-1 -1, -1 -2, -2 -2, -2 -1, -1 -1)))

  • +
  • GEOMETRYCOLLECTION (POINT(2 3), LINESTRING (2 3, 3 4))

  • +
+

Use ST_GeometryFromText() and ST_GeomFromBinary() functions to create geometry +objects from WKT or WKB.

+

The SphericalGeography type provides native support for spatial features represented on +geographic coordinates (sometimes called geodetic coordinates, or lat/lon, or lon/lat). +Geographic coordinates are spherical coordinates expressed in angular units (degrees).

+

The basis for the Geometry type is a plane. The shortest path between two points on the plane is a +straight line. That means calculations on geometries (areas, distances, lengths, intersections, etc) +can be calculated using cartesian mathematics and straight line vectors.

+

The basis for the SphericalGeography type is a sphere. The shortest path between two points on the +sphere is a great circle arc. That means that calculations on geographies (areas, distances, +lengths, intersections, etc) must be calculated on the sphere, using more complicated mathematics. +More accurate measurements that take the actual spheroidal shape of the world into account are not +supported.

+

Values returned by the measurement functions ST_Distance() and ST_Length() are in the unit of meters; +values returned by ST_Area() are in square meters.

+

Use to_spherical_geography() function to convert a geometry object to geography object.

+

For example, ST_Distance(ST_Point(-71.0882, 42.3607), ST_Point(-74.1197, 40.6976)) returns +3.4577 in the unit of the passed-in values on the euclidean plane, while +ST_Distance(to_spherical_geography(ST_Point(-71.0882, 42.3607)), to_spherical_geography(ST_Point(-74.1197, 40.6976))) +returns 312822.179 in meters.

+
+

Constructors#

+
+
+ST_AsBinary(Geometry) varbinary#
+

Returns the WKB representation of the geometry.

+
+
+
+ST_AsText(Geometry) varchar#
+

Returns the WKT representation of the geometry. For empty geometries, +ST_AsText(ST_LineFromText('LINESTRING EMPTY')) will produce 'MULTILINESTRING EMPTY' +and ST_AsText(ST_Polygon('POLYGON EMPTY')) will produce 'MULTIPOLYGON EMPTY'.

+
+
+
+ST_GeometryFromText(varchar) Geometry#
+

Returns a geometry type object from WKT representation.

+
+
+
+ST_GeomFromBinary(varbinary) Geometry#
+

Returns a geometry type object from WKB representation.

+
+
+
+geometry_from_hadoop_shape(varbinary) Geometry#
+

Returns a geometry type object from Spatial Framework for Hadoop representation.

+
+
+
+ST_LineFromText(varchar) LineString#
+

Returns a geometry type linestring object from WKT representation.

+
+
+
+ST_LineString(array(Point)) LineString#
+

Returns a LineString formed from an array of points. If there are fewer than +two non-empty points in the input array, an empty LineString will be returned. +Array elements must not be NULL or the same as the previous element. +The returned geometry may not be simple, e.g. may self-intersect or may contain +duplicate vertexes depending on the input.

+
+
+
+ST_MultiPoint(array(Point)) MultiPoint#
+

Returns a MultiPoint geometry object formed from the specified points. Returns NULL if input array is empty. +Array elements must not be NULL or empty. +The returned geometry may not be simple and may contain duplicate points if input array has duplicates.

+
+
+
+ST_Point(double, double) -> Point
+

Returns a geometry type point object with the given coordinate values.

+
+
+
+ST_Polygon(varchar) Polygon#
+

Returns a geometry type polygon object from WKT representation.

+
+
+
+to_spherical_geography(Geometry) SphericalGeography#
+

Converts a Geometry object to a SphericalGeography object on the sphere of the Earth’s radius. This +function is only applicable to POINT, MULTIPOINT, LINESTRING, MULTILINESTRING, +POLYGON, MULTIPOLYGON geometries defined in 2D space, or GEOMETRYCOLLECTION of such +geometries. For each point of the input geometry, it verifies that point.x is within +[-180.0, 180.0] and point.y is within [-90.0, 90.0], and uses them as (longitude, latitude) +degrees to construct the shape of the SphericalGeography result.

+
+
+
+to_geometry(SphericalGeography) Geometry#
+

Converts a SphericalGeography object to a Geometry object.

+
+
+
+

Relationship tests#

+
+
+ST_Contains(Geometry, Geometry) -> boolean
+

Returns true if and only if no points of the second geometry lie in the exterior +of the first geometry, and at least one point of the interior of the first geometry +lies in the interior of the second geometry.

+
+
+
+ST_Crosses(Geometry, Geometry) -> boolean
+

Returns true if the supplied geometries have some, but not all, interior points in common.

+
+
+
+ST_Disjoint(Geometry, Geometry) -> boolean
+

Returns true if the give geometries do not spatially intersect – +if they do not share any space together.

+
+
+
+ST_Equals(Geometry, Geometry) -> boolean
+

Returns true if the given geometries represent the same geometry.

+
+
+
+ST_Intersects(Geometry, Geometry) -> boolean
+

Returns true if the given geometries spatially intersect in two dimensions +(share any portion of space) and false if they do not (they are disjoint).

+
+
+
+ST_Overlaps(Geometry, Geometry) -> boolean
+

Returns true if the given geometries share space, are of the same dimension, +but are not completely contained by each other.

+
+
+
+ST_Relate(Geometry, Geometry) -> boolean
+

Returns true if first geometry is spatially related to second geometry.

+
+
+
+ST_Touches(Geometry, Geometry) -> boolean
+

Returns true if the given geometries have at least one point in common, +but their interiors do not intersect.

+
+
+
+ST_Within(Geometry, Geometry) -> boolean
+

Returns true if first geometry is completely inside second geometry.

+
+
+
+

Operations#

+
+
+geometry_nearest_points(Geometry, Geometry) -> row(Point, Point)
+

Returns the points on each geometry nearest the other. If either geometry +is empty, return NULL. Otherwise, return a row of two Points that have +the minimum distance of any two points on the geometries. The first Point +will be from the first Geometry argument, the second from the second Geometry +argument. If there are multiple pairs with the minimum distance, one pair +is chosen arbitrarily.

+
+
+
+geometry_union(array(Geometry)) Geometry#
+

Returns a geometry that represents the point set union of the input geometries. Performance +of this function, in conjunction with array_agg() to first aggregate the input geometries, +may be better than geometry_union_agg(), at the expense of higher memory utilization.

+
+
+
+ST_Boundary(Geometry) Geometry#
+

Returns the closure of the combinatorial boundary of this geometry.

+
+
+
+ST_Buffer(Geometry, distance) Geometry#
+

Returns the geometry that represents all points whose distance from the specified geometry +is less than or equal to the specified distance.

+
+
+
+ST_Difference(Geometry, Geometry) -> Geometry
+

Returns the geometry value that represents the point set difference of the given geometries.

+
+
+
+ST_Envelope(Geometry) Geometry#
+

Returns the bounding rectangular polygon of a geometry.

+
+
+
+ST_EnvelopeAsPts(Geometry)#
+

Returns an array of two points: the lower left and upper right corners of the bounding +rectangular polygon of a geometry. Returns NULL if input geometry is empty.

+
+
+
+ST_ExteriorRing(Geometry) Geometry#
+

Returns a line string representing the exterior ring of the input polygon.

+
+
+
+ST_Intersection(Geometry, Geometry) -> Geometry
+

Returns the geometry value that represents the point set intersection of two geometries.

+
+
+
+ST_SymDifference(Geometry, Geometry) -> Geometry
+

Returns the geometry value that represents the point set symmetric difference of two geometries.

+
+
+
+ST_Union(Geometry, Geometry) -> Geometry
+

Returns a geometry that represents the point set union of the input geometries.

+

See also: geometry_union(), geometry_union_agg()

+
+
+
+

Accessors#

+
+
+ST_Area(Geometry) double#
+

Returns the 2D Euclidean area of a geometry.

+

For Point and LineString types, returns 0.0. +For GeometryCollection types, returns the sum of the areas of the individual +geometries.

+
+
+
+ST_Area(SphericalGeography) double
+

Returns the area of a polygon or multi-polygon in square meters using a spherical model for Earth.

+
+
+
+ST_Centroid(Geometry) Geometry#
+

Returns the point value that is the mathematical centroid of a geometry.

+
+
+
+ST_ConvexHull(Geometry) Geometry#
+

Returns the minimum convex geometry that encloses all input geometries.

+
+
+
+ST_CoordDim(Geometry) bigint#
+

Returns the coordinate dimension of the geometry.

+
+
+
+ST_Dimension(Geometry) bigint#
+

Returns the inherent dimension of this geometry object, which must be +less than or equal to the coordinate dimension.

+
+
+
+ST_Distance(Geometry, Geometry) -> double
+

Returns the 2-dimensional cartesian minimum distance (based on spatial ref) +between two geometries in projected units.

+
+
+
+ST_Distance(SphericalGeography, SphericalGeography) -> double
+

Returns the great-circle distance in meters between two SphericalGeography points.

+
+
+
+ST_GeometryN(Geometry, index) Geometry#
+

Returns the geometry element at a given index (indices start at 1). +If the geometry is a collection of geometries (e.g., GEOMETRYCOLLECTION or MULTI*), +returns the geometry at a given index. +If the given index is less than 1 or greater than the total number of elements in the collection, +returns NULL. +Use ST_NumGeometries() to find out the total number of elements. +Singular geometries (e.g., POINT, LINESTRING, POLYGON), are treated as collections of one element. +Empty geometries are treated as empty collections.

+
+
+
+ST_InteriorRingN(Geometry, index) Geometry#
+

Returns the interior ring element at the specified index (indices start at 1). If +the given index is less than 1 or greater than the total number of interior rings +in the input geometry, returns NULL. The input geometry must be a polygon. +Use ST_NumInteriorRing() to find out the total number of elements.

+
+
+
+ST_GeometryType(Geometry) varchar#
+

Returns the type of the geometry.

+
+
+
+ST_IsClosed(Geometry) boolean#
+

Returns true if the linestring’s start and end points are coincident.

+
+
+
+ST_IsEmpty(Geometry) boolean#
+

Returns true if this Geometry is an empty geometrycollection, polygon, point etc.

+
+
+
+ST_IsSimple(Geometry) boolean#
+

Returns true if this Geometry has no anomalous geometric points, such as self intersection or self tangency.

+
+
+
+ST_IsRing(Geometry) boolean#
+

Returns true if and only if the line is closed and simple.

+
+
+
+ST_IsValid(Geometry) boolean#
+

Returns true if and only if the input geometry is well formed. +Use geometry_invalid_reason() to determine why the geometry is not well formed.

+
+
+
+ST_Length(Geometry) double#
+

Returns the length of a linestring or multi-linestring using Euclidean measurement on a +two dimensional plane (based on spatial ref) in projected units.

+
+
+
+ST_Length(SphericalGeography) double
+

Returns the length of a linestring or multi-linestring on a spherical model of the Earth. +This is equivalent to the sum of great-circle distances between adjacent points on the linestring.

+
+
+
+ST_PointN(LineString, index) Point#
+

Returns the vertex of a linestring at a given index (indices start at 1). +If the given index is less than 1 or greater than the total number of elements in the collection, +returns NULL. +Use ST_NumPoints() to find out the total number of elements.

+
+
+
+ST_Points(Geometry)#
+

Returns an array of points in a linestring.

+
+
+
+ST_XMax(Geometry) double#
+

Returns X maxima of a bounding box of a geometry.

+
+
+
+ST_YMax(Geometry) double#
+

Returns Y maxima of a bounding box of a geometry.

+
+
+
+ST_XMin(Geometry) double#
+

Returns X minima of a bounding box of a geometry.

+
+
+
+ST_YMin(Geometry) double#
+

Returns Y minima of a bounding box of a geometry.

+
+
+
+ST_StartPoint(Geometry) point#
+

Returns the first point of a LineString geometry as a Point. +This is a shortcut for ST_PointN(geometry, 1).

+
+
+
+simplify_geometry(Geometry, double) Geometry#
+

Returns a “simplified” version of the input geometry using the Douglas-Peucker algorithm. +Will avoid creating derived geometries (polygons in particular) that are invalid.

+
+
+
+ST_EndPoint(Geometry) point#
+

Returns the last point of a LineString geometry as a Point. +This is a shortcut for ST_PointN(geometry, ST_NumPoints(geometry)).

+
+
+
+ST_X(Point) double#
+

Returns the X coordinate of the point.

+
+
+
+ST_Y(Point) double#
+

Returns the Y coordinate of the point.

+
+
+
+ST_InteriorRings(Geometry)#
+

Returns an array of all interior rings found in the input geometry, or an empty +array if the polygon has no interior rings. Returns NULL if the input geometry +is empty. The input geometry must be a polygon.

+
+
+
+ST_NumGeometries(Geometry) bigint#
+

Returns the number of geometries in the collection. +If the geometry is a collection of geometries (e.g., GEOMETRYCOLLECTION or MULTI*), +returns the number of geometries, +for single geometries returns 1, +for empty geometries returns 0.

+
+
+
+ST_Geometries(Geometry)#
+

Returns an array of geometries in the specified collection. Returns a one-element array +if the input geometry is not a multi-geometry. Returns NULL if input geometry is empty.

+
+
+
+ST_NumPoints(Geometry) bigint#
+

Returns the number of points in a geometry. This is an extension to the SQL/MM +ST_NumPoints function which only applies to point and linestring.

+
+
+
+ST_NumInteriorRing(Geometry) bigint#
+

Returns the cardinality of the collection of interior rings of a polygon.

+
+
+
+line_interpolate_point(LineString, double) Geometry#
+

Returns a Point interpolated along a LineString at the fraction given. The fraction +must be between 0 and 1, inclusive.

+
+
+
+line_interpolate_points(LineString, double, repeated)#
+

Returns an array of Points interpolated along a LineString. The fraction must be +between 0 and 1, inclusive.

+
+
+
+line_locate_point(LineString, Point) double#
+

Returns a float between 0 and 1 representing the location of the closest point on +the LineString to the given Point, as a fraction of total 2d line length.

+

Returns NULL if a LineString or a Point is empty or NULL.

+
+
+
+geometry_invalid_reason(Geometry) varchar#
+

Returns the reason for why the input geometry is not valid. +Returns NULL if the input is valid.

+
+
+
+great_circle_distance(latitude1, longitude1, latitude2, longitude2) double#
+

Returns the great-circle distance between two points on Earth’s surface in kilometers.

+
+
+
+to_geojson_geometry(SphericalGeography) varchar#
+

Returns the GeoJSON encoded defined by the input spherical geography.

+
+
+
+from_geojson_geometry(varchar) SphericalGeography#
+

Returns the spherical geography type object from the GeoJSON representation stripping non geometry key/values. +Feature and FeatureCollection are not supported.

+
+
+
+

Aggregations#

+
+
+convex_hull_agg(Geometry) Geometry#
+

Returns the minimum convex geometry that encloses all input geometries.

+
+
+
+geometry_union_agg(Geometry) Geometry#
+

Returns a geometry that represents the point set union of all input geometries.

+
+
+
+

Bing tiles#

+

These functions convert between geometries and +Bing tiles.

+
+
+bing_tile(x, y, zoom_level) BingTile#
+

Creates a Bing tile object from XY coordinates and a zoom level. +Zoom levels from 1 to 23 are supported.

+
+
+
+bing_tile(quadKey) BingTile
+

Creates a Bing tile object from a quadkey.

+
+
+
+bing_tile_at(latitude, longitude, zoom_level) BingTile#
+

Returns a Bing tile at a given zoom level containing a point at a given latitude +and longitude. Latitude must be within [-85.05112878, 85.05112878] range. +Longitude must be within [-180, 180] range. Zoom levels from 1 to 23 are supported.

+
+
+
+bing_tiles_around(latitude, longitude, zoom_level)#
+

Returns a collection of Bing tiles that surround the point specified +by the latitude and longitude arguments at a given zoom level.

+
+
+
+bing_tiles_around(latitude, longitude, zoom_level, radius_in_km)
+

Returns a minimum set of Bing tiles at specified zoom level that cover a circle of specified +radius in km around a specified (latitude, longitude) point.

+
+
+
+bing_tile_coordinates(tile) row<x, y>#
+

Returns the XY coordinates of a given Bing tile.

+
+
+
+bing_tile_polygon(tile) Geometry#
+

Returns the polygon representation of a given Bing tile.

+
+
+
+bing_tile_quadkey(tile) varchar#
+

Returns the quadkey of a given Bing tile.

+
+
+
+bing_tile_zoom_level(tile) tinyint#
+

Returns the zoom level of a given Bing tile.

+
+
+
+geometry_to_bing_tiles(geometry, zoom_level)#
+

Returns the minimum set of Bing tiles that fully covers a given geometry at +a given zoom level. Zoom levels from 1 to 23 are supported.

+
+
+
+

Encoded polylines#

+

These functions convert between geometries and +encoded polylines.

+
+
+to_encoded_polyline(Geometry) varchar#
+

Encodes a linestring or multipoint to a polyline.

+
+
+
+from_encoded_polyline(varchar) Geometry#
+

Decodes a polyline to a linestring.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/hyperloglog.html b/430/functions/hyperloglog.html new file mode 100644 index 000000000..ff665f590 --- /dev/null +++ b/430/functions/hyperloglog.html @@ -0,0 +1,726 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + HyperLogLog functions — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

HyperLogLog functions#

+

Trino implements the approx_distinct() function using the +HyperLogLog data structure.

+
+

Data structures#

+

Trino implements HyperLogLog data sketches as a set of 32-bit buckets which +store a maximum hash. They can be stored sparsely (as a map from bucket ID +to bucket), or densely (as a contiguous memory block). The HyperLogLog data +structure starts as the sparse representation, switching to dense when it is +more efficient. The P4HyperLogLog structure is initialized densely and +remains dense for its lifetime.

+

HyperLogLog implicitly casts to P4HyperLogLog, +while one can explicitly cast HyperLogLog to P4HyperLogLog:

+
cast(hll AS P4HyperLogLog)
+
+
+
+
+

Serialization#

+

Data sketches can be serialized to and deserialized from varbinary. This +allows them to be stored for later use. Combined with the ability to merge +multiple sketches, this allows one to calculate approx_distinct() of the +elements of a partition of a query, then for the entirety of a query with very +little cost.

+

For example, calculating the HyperLogLog for daily unique users will allow +weekly or monthly unique users to be calculated incrementally by combining the +dailies. This is similar to computing weekly revenue by summing daily revenue. +Uses of approx_distinct() with GROUPING SETS can be converted to use +HyperLogLog. Examples:

+
CREATE TABLE visit_summaries (
+  visit_date date,
+  hll varbinary
+);
+
+INSERT INTO visit_summaries
+SELECT visit_date, cast(approx_set(user_id) AS varbinary)
+FROM user_visits
+GROUP BY visit_date;
+
+SELECT cardinality(merge(cast(hll AS HyperLogLog))) AS weekly_unique_users
+FROM visit_summaries
+WHERE visit_date >= current_date - interval '7' day;
+
+
+
+
+

Functions#

+
+
+approx_set(x) HyperLogLog#
+

Returns the HyperLogLog sketch of the input data set of x. This +data sketch underlies approx_distinct() and can be stored and +used later by calling cardinality().

+
+
+
+cardinality(hll) bigint
+

This will perform approx_distinct() on the data summarized by the +hll HyperLogLog data sketch.

+
+
+
+empty_approx_set() HyperLogLog#
+

Returns an empty HyperLogLog.

+
+
+
+merge(HyperLogLog) HyperLogLog#
+

Returns the HyperLogLog of the aggregate union of the individual hll +HyperLogLog structures.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/ipaddress.html b/430/functions/ipaddress.html new file mode 100644 index 000000000..4cb8490e7 --- /dev/null +++ b/430/functions/ipaddress.html @@ -0,0 +1,653 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + IP Address Functions — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

IP Address Functions#

+
+
+contains(network, address) boolean
+

Returns true if the address exists in the CIDR network:

+
SELECT contains('10.0.0.0/8', IPADDRESS '10.255.255.255'); -- true
+SELECT contains('10.0.0.0/8', IPADDRESS '11.255.255.255'); -- false
+
+SELECT contains('2001:0db8:0:0:0:ff00:0042:8329/128', IPADDRESS '2001:0db8:0:0:0:ff00:0042:8329'); -- true
+SELECT contains('2001:0db8:0:0:0:ff00:0042:8329/128', IPADDRESS '2001:0db8:0:0:0:ff00:0042:8328'); -- false
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/json.html b/430/functions/json.html new file mode 100644 index 000000000..a1fead946 --- /dev/null +++ b/430/functions/json.html @@ -0,0 +1,2547 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + JSON functions and operators — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

JSON functions and operators#

+

The SQL standard describes functions and operators to process JSON data. They +allow you to access JSON data according to its structure, generate JSON data, +and store it persistently in SQL tables.

+

Importantly, the SQL standard imposes that there is no dedicated data type to +represent JSON data in SQL. Instead, JSON data is represented as character or +binary strings. Although Trino supports JSON type, it is not used or +produced by the following functions.

+

Trino supports three functions for querying JSON data: +json_exists, +json_query, and json_value. Each of them +is based on the same mechanism of exploring and processing JSON input using +JSON path.

+

Trino also supports two functions for generating JSON data – +json_array, and json_object.

+
+

JSON path language#

+

The JSON path language is a special language, used exclusively by certain SQL +operators to specify the query to perform on the JSON input. Although JSON path +expressions are embedded in SQL queries, their syntax significantly differs +from SQL. The semantics of predicates, operators, etc. in JSON path expressions +generally follow the semantics of SQL. The JSON path language is case-sensitive +for keywords and identifiers.

+
+

JSON path syntax and semantics#

+

JSON path expressions are recursive structures. Although the name “path” +suggests a linear sequence of operations going step by step deeper into the JSON +structure, a JSON path expression is in fact a tree. It can access the input +JSON item multiple times, in multiple ways, and combine the results. Moreover, +the result of a JSON path expression is not a single item, but an ordered +sequence of items. Each of the sub-expressions takes one or more input +sequences, and returns a sequence as the result.

+
+

Note

+

In the lax mode, most path operations first unnest all JSON arrays in the +input sequence. Any divergence from this rule is mentioned in the following +listing. Path modes are explained in JSON path modes.

+
+

The JSON path language features are divided into: literals, variables, +arithmetic binary expressions, arithmetic unary expressions, and a group of +operators collectively known as accessors.

+
+

literals#

+
    +
  • numeric literals

    +

    They include exact and approximate numbers, and are interpreted as if they +were SQL values.

    +
  • +
+
-1, 1.2e3, NaN
+
+
+
    +
  • string literals

    +

    They are enclosed in double quotes.

    +
  • +
+
"Some text"
+
+
+
    +
  • boolean literals

  • +
+
true, false
+
+
+
    +
  • null literal

    +

    It has the semantics of the JSON null, not of SQL null. See Comparison rules.

    +
  • +
+
null
+
+
+
+
+

variables#

+
    +
  • context variable

    +

    It refers to the currently processed input of the JSON +function.

    +
  • +
+
$
+
+
+
    +
  • named variable

    +

    It refers to a named parameter by its name.

    +
  • +
+
$param
+
+
+
    +
  • current item variable

    +

    It is used inside the filter expression to refer to the currently processed +item from the input sequence.

    +
  • +
+
@
+
+
+
    +
  • last subscript variable

    +

    It refers to the last index of the innermost enclosing array. Array indexes +in JSON path expressions are zero-based.

    +
  • +
+
last
+
+
+
+
+

arithmetic binary expressions#

+

The JSON path language supports five arithmetic binary operators:

+
<path1> + <path2>
+<path1> - <path2>
+<path1> * <path2>
+<path1> / <path2>
+<path1> % <path2>
+
+
+

Both operands, <path1> and <path2>, are evaluated to sequences of +items. For arithmetic binary operators, each input sequence must contain a +single numeric item. The arithmetic operation is performed according to SQL +semantics, and it returns a sequence containing a single element with the +result.

+

The operators follow the same precedence rules as in SQL arithmetic operations, +and parentheses can be used for grouping.

+
+
+

arithmetic unary expressions#

+
+ <path>
+- <path>
+
+
+

The operand <path> is evaluated to a sequence of items. Every item must be +a numeric value. The unary plus or minus is applied to every item in the +sequence, following SQL semantics, and the results form the returned sequence.

+
+
+

member accessor#

+

The member accessor returns the value of the member with the specified key for +each JSON object in the input sequence.

+
<path>.key
+<path>."key"
+
+
+

The condition when a JSON object does not have such a member is called a +structural error. In the lax mode, it is suppressed, and the faulty object is +excluded from the result.

+

Let <path> return a sequence of three JSON objects:

+
{"customer" : 100, "region" : "AFRICA"},
+{"region" : "ASIA"},
+{"customer" : 300, "region" : "AFRICA", "comment" : null}
+
+
+

the expression <path>.customer succeeds in the first and the third object, +but the second object lacks the required member. In strict mode, path +evaluation fails. In lax mode, the second object is silently skipped, and the +resulting sequence is 100, 300.

+

All items in the input sequence must be JSON objects.

+
+

Note

+

Trino does not support JSON objects with duplicate keys.

+
+
+
+

wildcard member accessor#

+

Returns values from all key-value pairs for each JSON object in the input +sequence. All the partial results are concatenated into the returned sequence.

+
<path>.*
+
+
+

Let <path> return a sequence of three JSON objects:

+
{"customer" : 100, "region" : "AFRICA"},
+{"region" : "ASIA"},
+{"customer" : 300, "region" : "AFRICA", "comment" : null}
+
+
+

The results is:

+
100, "AFRICA", "ASIA", 300, "AFRICA", null
+
+
+

All items in the input sequence must be JSON objects.

+

The order of values returned from a single JSON object is arbitrary. The +sub-sequences from all JSON objects are concatenated in the same order in which +the JSON objects appear in the input sequence.

+
+
+

descendant member accessor#

+

Returns the values associated with the specified key in all JSON objects on all +levels of nesting in the input sequence.

+
<path>..key
+<path>.."key"
+
+
+

The order of returned values is that of preorder depth first search. First, the +enclosing object is visited, and then all child nodes are visited.

+

This method does not perform array unwrapping in the lax mode. The results +are the same in the lax and strict modes. The method traverses into JSON +arrays and JSON objects. Non-structural JSON items are skipped.

+

Let <path> be a sequence containing a JSON object:

+
{
+    "id" : 1,
+    "notes" : [{"type" : 1, "comment" : "foo"}, {"type" : 2, "comment" : null}],
+    "comment" : ["bar", "baz"]
+}
+
+
+
<path>..comment --> ["bar", "baz"], "foo", null
+
+
+
+
+

array accessor#

+

Returns the elements at the specified indexes for each JSON array in the input +sequence. Indexes are zero-based.

+
<path>[ <subscripts> ]
+
+
+

The <subscripts> list contains one or more subscripts. Each subscript +specifies a single index or a range (ends inclusive):

+
<path>[<path1>, <path2> to <path3>, <path4>,...]
+
+
+

In lax mode, any non-array items resulting from the evaluation of the input +sequence are wrapped into single-element arrays. Note that this is an exception +to the rule of automatic array wrapping.

+

Each array in the input sequence is processed in the following way:

+
    +
  • The variable last is set to the last index of the array.

  • +
  • All subscript indexes are computed in order of declaration. For a +singleton subscript <path1>, the result must be a singleton numeric item. +For a range subscript <path2> to <path3>, two numeric items are expected.

  • +
  • The specified array elements are added in order to the output sequence.

  • +
+

Let <path> return a sequence of three JSON arrays:

+
[0, 1, 2], ["a", "b", "c", "d"], [null, null]
+
+
+

The following expression returns a sequence containing the last element from +every array:

+
<path>[last] --> 2, "d", null
+
+
+

The following expression returns the third and fourth element from every array:

+
<path>[2 to 3] --> 2, "c", "d"
+
+
+

Note that the first array does not have the fourth element, and the last array +does not have the third or fourth element. Accessing non-existent elements is a +structural error. In strict mode, it causes the path expression to fail. In lax +mode, such errors are suppressed, and only the existing elements are returned.

+

Another example of a structural error is an improper range specification such +as 5 to 3.

+

Note that the subscripts may overlap, and they do not need to follow the +element order. The order in the returned sequence follows the subscripts:

+
<path>[1, 0, 0] --> 1, 0, 0, "b", "a", "a", null, null, null
+
+
+
+
+

wildcard array accessor#

+

Returns all elements of each JSON array in the input sequence.

+
<path>[*]
+
+
+

In lax mode, any non-array items resulting from the evaluation of the input +sequence are wrapped into single-element arrays. Note that this is an exception +to the rule of automatic array wrapping.

+

The output order follows the order of the original JSON arrays. Also, the order +of elements within the arrays is preserved.

+

Let <path> return a sequence of three JSON arrays:

+
[0, 1, 2], ["a", "b", "c", "d"], [null, null]
+<path>[*] --> 0, 1, 2, "a", "b", "c", "d", null, null
+
+
+
+
+

filter#

+

Retrieves the items from the input sequence which satisfy the predicate.

+
<path>?( <predicate> )
+
+
+

JSON path predicates are syntactically similar to boolean expressions in SQL. +However, the semantics are different in many aspects:

+
    +
  • They operate on sequences of items.

  • +
  • They have their own error handling (they never fail).

  • +
  • They behave different depending on the lax or strict mode.

  • +
+

The predicate evaluates to true, false, or unknown. Note that some +predicate expressions involve nested JSON path expression. When evaluating the +nested path, the variable @ refers to the currently examined item from the +input sequence.

+

The following predicate expressions are supported:

+
    +
  • Conjunction

  • +
+
<predicate1> && <predicate2>
+
+
+
    +
  • Disjunction

  • +
+
<predicate1> || <predicate2>
+
+
+
    +
  • Negation

  • +
+
! <predicate>
+
+
+
    +
  • exists predicate

  • +
+
exists( <path> )
+
+
+

Returns true if the nested path evaluates to a non-empty sequence, and +false when the nested path evaluates to an empty sequence. If the path +evaluation throws an error, returns unknown.

+
    +
  • starts with predicate

  • +
+
<path> starts with "Some text"
+<path> starts with $variable
+
+
+

The nested <path> must evaluate to a sequence of textual items, and the +other operand must evaluate to a single textual item. If evaluating of either +operand throws an error, the result is unknown. All items from the sequence +are checked for starting with the right operand. The result is true if a +match is found, otherwise false. However, if any of the comparisons throws +an error, the result in the strict mode is unknown. The result in the lax +mode depends on whether the match or the error was found first.

+
    +
  • is unknown predicate

  • +
+
( <predicate> ) is unknown
+
+
+

Returns true if the nested predicate evaluates to unknown, and +false otherwise.

+
    +
  • Comparisons

  • +
+
<path1> == <path2>
+<path1> <> <path2>
+<path1> != <path2>
+<path1> < <path2>
+<path1> > <path2>
+<path1> <= <path2>
+<path1> >= <path2>
+
+
+

Both operands of a comparison evaluate to sequences of items. If either +evaluation throws an error, the result is unknown. Items from the left and +right sequence are then compared pairwise. Similarly to the starts with +predicate, the result is true if any of the comparisons returns true, +otherwise false. However, if any of the comparisons throws an error, for +example because the compared types are not compatible, the result in the strict +mode is unknown. The result in the lax mode depends on whether the true +comparison or the error was found first.

+
+
Comparison rules#
+

Null values in the context of comparison behave different than SQL null:

+
    +
  • null == null –> true

  • +
  • null != null, null < null, … –> false

  • +
  • null compared to a scalar value –> false

  • +
  • null compared to a JSON array or a JSON object –> false

  • +
+

When comparing two scalar values, true or false is returned if the +comparison is successfully performed. The semantics of the comparison is the +same as in SQL. In case of an error, e.g. comparing text and number, +unknown is returned.

+

Comparing a scalar value with a JSON array or a JSON object, and comparing JSON +arrays/objects is an error, so unknown is returned.

+
+
+
Examples of filter#
+

Let <path> return a sequence of three JSON objects:

+
{"customer" : 100, "region" : "AFRICA"},
+{"region" : "ASIA"},
+{"customer" : 300, "region" : "AFRICA", "comment" : null}
+
+
+
<path>?(@.region != "ASIA") --> {"customer" : 100, "region" : "AFRICA"},
+                                {"customer" : 300, "region" : "AFRICA", "comment" : null}
+<path>?(!exists(@.customer)) --> {"region" : "ASIA"}
+
+
+

The following accessors are collectively referred to as item methods.

+
+
+
+

double()#

+

Converts numeric or text values into double values.

+
<path>.double()
+
+
+

Let <path> return a sequence -1, 23e4, "5.6":

+
<path>.double() --> -1e0, 23e4, 5.6e0
+
+
+
+
+

ceiling(), floor(), and abs()#

+

Gets the ceiling, the floor or the absolute value for every numeric item in the +sequence. The semantics of the operations is the same as in SQL.

+

Let <path> return a sequence -1.5, -1, 1.3:

+
<path>.ceiling() --> -1.0, -1, 2.0
+<path>.floor() --> -2.0, -1, 1.0
+<path>.abs() --> 1.5, 1, 1.3
+
+
+
+
+

keyvalue()#

+

Returns a collection of JSON objects including one object per every member of +the original object for every JSON object in the sequence.

+
<path>.keyvalue()
+
+
+

The returned objects have three members:

+
    +
  • “name”, which is the original key,

  • +
  • “value”, which is the original bound value,

  • +
  • “id”, which is the unique number, specific to an input object.

  • +
+

Let <path> be a sequence of three JSON objects:

+
{"customer" : 100, "region" : "AFRICA"},
+{"region" : "ASIA"},
+{"customer" : 300, "region" : "AFRICA", "comment" : null}
+
+
+
<path>.keyvalue() --> {"name" : "customer", "value" : 100, "id" : 0},
+                      {"name" : "region", "value" : "AFRICA", "id" : 0},
+                      {"name" : "region", "value" : "ASIA", "id" : 1},
+                      {"name" : "customer", "value" : 300, "id" : 2},
+                      {"name" : "region", "value" : "AFRICA", "id" : 2},
+                      {"name" : "comment", "value" : null, "id" : 2}
+
+
+

It is required that all items in the input sequence are JSON objects.

+

The order of the returned values follows the order of the original JSON +objects. However, within objects, the order of returned entries is arbitrary.

+
+
+

type()#

+

Returns a textual value containing the type name for every item in the +sequence.

+
<path>.type()
+
+
+

This method does not perform array unwrapping in the lax mode.

+

The returned values are:

+
    +
  • "null" for JSON null,

  • +
  • "number" for a numeric item,

  • +
  • "string" for a textual item,

  • +
  • "boolean" for a boolean item,

  • +
  • "date" for an item of type date,

  • +
  • "time without time zone" for an item of type time,

  • +
  • "time with time zone" for an item of type time with time zone,

  • +
  • "timestamp without time zone" for an item of type timestamp,

  • +
  • "timestamp with time zone" for an item of type timestamp with time zone,

  • +
  • "array" for JSON array,

  • +
  • "object" for JSON object,

  • +
+
+
+

size()#

+

Returns a numeric value containing the size for every JSON array in the +sequence.

+
<path>.size()
+
+
+

This method does not perform array unwrapping in the lax mode. Instead, all +non-array items are wrapped in singleton JSON arrays, so their size is 1.

+

It is required that all items in the input sequence are JSON arrays.

+

Let <path> return a sequence of three JSON arrays:

+
[0, 1, 2], ["a", "b", "c", "d"], [null, null]
+<path>.size() --> 3, 4, 2
+
+
+
+
+
+

Limitations#

+

The SQL standard describes the datetime() JSON path item method and the +like_regex() JSON path predicate. Trino does not support them.

+
+
+

JSON path modes#

+

The JSON path expression can be evaluated in two modes: strict and lax. In the +strict mode, it is required that the input JSON data strictly fits the schema +required by the path expression. In the lax mode, the input JSON data can +diverge from the expected schema.

+

The following table shows the differences between the two modes.

+ +++++ + + + + + + + + + + + + + + + + + + + + +

Condition

strict mode

lax mode

Performing an operation which requires a non-array on an array, e.g.:

+

$.key requires a JSON object

+

$.floor() requires a numeric value

+

ERROR

The array is automatically unnested, and the operation is performed on +each array element.

Performing an operation which requires an array on an non-array, e.g.:

+

$[0], $[*], $.size()

+

ERROR

The non-array item is automatically wrapped in a singleton array, and +the operation is performed on the array.

A structural error: accessing a non-existent element of an array or a +non-existent member of a JSON object, e.g.:

+

$[-1] (array index out of bounds)

+

$.key, where the input JSON object does not have a member key

+

ERROR

The error is suppressed, and the operation results in an empty sequence.

+
+

Examples of the lax mode behavior#

+

Let <path> return a sequence of three items, a JSON array, a JSON object, +and a scalar numeric value:

+
[1, "a", null], {"key1" : 1.0, "key2" : true}, -2e3
+
+
+

The following example shows the wildcard array accessor in the lax mode. The +JSON array returns all its elements, while the JSON object and the number are +wrapped in singleton arrays and then unnested, so effectively they appear +unchanged in the output sequence:

+
<path>[*] --> 1, "a", null, {"key1" : 1.0, "key2" : true}, -2e3
+
+
+

When calling the size() method, the JSON object and the number are also +wrapped in singleton arrays:

+
<path>.size() --> 3, 1, 1
+
+
+

In some cases, the lax mode cannot prevent failure. In the following example, +even though the JSON array is unwrapped prior to calling the floor() +method, the item "a" causes type mismatch.

+
<path>.floor() --> ERROR
+
+
+
+
+
+
+

json_exists#

+

The json_exists function determines whether a JSON value satisfies a JSON +path specification.

+
JSON_EXISTS(
+    json_input [ FORMAT JSON [ ENCODING { UTF8 | UTF16 | UTF32 } ] ],
+    json_path
+    [ PASSING json_argument [, ...] ]
+    [ { TRUE | FALSE | UNKNOWN | ERROR } ON ERROR ]
+    )
+
+
+

The json_path is evaluated using the json_input as the context variable +($), and the passed arguments as the named variables ($variable_name). +The returned value is true if the path returns a non-empty sequence, and +false if the path returns an empty sequence. If an error occurs, the +returned value depends on the ON ERROR clause. The default value returned +ON ERROR is FALSE. The ON ERROR clause is applied for the following +kinds of errors:

+
    +
  • Input conversion errors, such as malformed JSON

  • +
  • JSON path evaluation errors, e.g. division by zero

  • +
+

json_input is a character string or a binary string. It should contain +a single JSON item. For a binary string, you can specify encoding.

+

json_path is a string literal, containing the path mode specification, and +the path expression, following the syntax rules described in +JSON path syntax and semantics.

+
'strict ($.price + $.tax)?(@ > 99.9)'
+'lax $[0 to 1].floor()?(@ > 10)'
+
+
+

In the PASSING clause you can pass arbitrary expressions to be used by the +path expression.

+
PASSING orders.totalprice AS O_PRICE,
+        orders.tax % 10 AS O_TAX
+
+
+

The passed parameters can be referenced in the path expression by named +variables, prefixed with $.

+
'lax $?(@.price > $O_PRICE || @.tax > $O_TAX)'
+
+
+

Additionally to SQL values, you can pass JSON values, specifying the format and +optional encoding:

+
PASSING orders.json_desc FORMAT JSON AS o_desc,
+        orders.binary_record FORMAT JSON ENCODING UTF16 AS o_rec
+
+
+

Note that the JSON path language is case-sensitive, while the unquoted SQL +identifiers are upper-cased. Therefore, it is recommended to use quoted +identifiers in the PASSING clause:

+
'lax $.$KeyName' PASSING nation.name AS KeyName --> ERROR; no passed value found
+'lax $.$KeyName' PASSING nation.name AS "KeyName" --> correct
+
+
+
+

Examples#

+

Let customers be a table containing two columns: id:bigint, +description:varchar.

+ + + + + + + + + + + + + + + + + +

id

description

101

‘{“comment” : “nice”, “children” : [10, 13, 16]}’

102

‘{“comment” : “problematic”, “children” : [8, 11]}’

103

‘{“comment” : “knows best”, “children” : [2]}’

+

The following query checks which customers have children above the age of 10:

+
SELECT
+      id,
+      json_exists(
+                  description,
+                  'lax $.children[*]?(@ > 10)'
+                 ) AS children_above_ten
+FROM customers
+
+
+ + + + + + + + + + + + + + + + + +

id

children_above_ten

101

true

102

true

103

false

+

In the following query, the path mode is strict. We check the third child for +each customer. This should cause a structural error for the customers who do +not have three or more children. This error is handled according to the ON ERROR clause.

+
SELECT
+      id,
+      json_exists(
+                  description,
+                  'strict $.children[2]?(@ > 10)'
+                  UNKNOWN ON ERROR
+                 ) AS child_3_above_ten
+FROM customers
+
+
+ + + + + + + + + + + + + + + + + +

id

child_3_above_ten

101

true

102

NULL

103

NULL

+
+
+
+

json_query#

+

The json_query function extracts a JSON value from a JSON value.

+
JSON_QUERY(
+    json_input [ FORMAT JSON [ ENCODING { UTF8 | UTF16 | UTF32 } ] ],
+    json_path
+    [ PASSING json_argument [, ...] ]
+    [ RETURNING type [ FORMAT JSON [ ENCODING { UTF8 | UTF16 | UTF32 } ] ] ]
+    [ WITHOUT [ ARRAY ] WRAPPER |
+      WITH [ { CONDITIONAL | UNCONDITIONAL } ] [ ARRAY ] WRAPPER ]
+    [ { KEEP | OMIT } QUOTES [ ON SCALAR STRING ] ]
+    [ { ERROR | NULL | EMPTY ARRAY | EMPTY OBJECT } ON EMPTY ]
+    [ { ERROR | NULL | EMPTY ARRAY | EMPTY OBJECT } ON ERROR ]
+    )
+
+
+

The json_path is evaluated using the json_input as the context variable +($), and the passed arguments as the named variables ($variable_name).

+

The returned value is a JSON item returned by the path. By default, it is +represented as a character string (varchar). In the RETURNING clause, +you can specify other character string type or varbinary. With +varbinary, you can also specify the desired encoding.

+

json_input is a character string or a binary string. It should contain +a single JSON item. For a binary string, you can specify encoding.

+

json_path is a string literal, containing the path mode specification, and +the path expression, following the syntax rules described in +JSON path syntax and semantics.

+
'strict $.keyvalue()?(@.name == $cust_id)'
+'lax $[5 to last]'
+
+
+

In the PASSING clause you can pass arbitrary expressions to be used by the +path expression.

+
PASSING orders.custkey AS CUST_ID
+
+
+

The passed parameters can be referenced in the path expression by named +variables, prefixed with $.

+
'strict $.keyvalue()?(@.value == $CUST_ID)'
+
+
+

Additionally to SQL values, you can pass JSON values, specifying the format and +optional encoding:

+
PASSING orders.json_desc FORMAT JSON AS o_desc,
+        orders.binary_record FORMAT JSON ENCODING UTF16 AS o_rec
+
+
+

Note that the JSON path language is case-sensitive, while the unquoted SQL +identifiers are upper-cased. Therefore, it is recommended to use quoted +identifiers in the PASSING clause:

+
'lax $.$KeyName' PASSING nation.name AS KeyName --> ERROR; no passed value found
+'lax $.$KeyName' PASSING nation.name AS "KeyName" --> correct
+
+
+

The ARRAY WRAPPER clause lets you modify the output by wrapping the results +in a JSON array. WITHOUT ARRAY WRAPPER is the default option. WITH CONDITIONAL ARRAY WRAPPER wraps every result which is not a singleton JSON +array or JSON object. WITH UNCONDITIONAL ARRAY WRAPPER wraps every result.

+

The QUOTES clause lets you modify the result for a scalar string by +removing the double quotes being part of the JSON string representation.

+
+

Examples#

+

Let customers be a table containing two columns: id:bigint, +description:varchar.

+ + + + + + + + + + + + + + + + + +

id

description

101

‘{“comment” : “nice”, “children” : [10, 13, 16]}’

102

‘{“comment” : “problematic”, “children” : [8, 11]}’

103

‘{“comment” : “knows best”, “children” : [2]}’

+

The following query gets the children array for each customer:

+
SELECT
+      id,
+      json_query(
+                 description,
+                 'lax $.children'
+                ) AS children
+FROM customers
+
+
+ + + + + + + + + + + + + + + + + +

id

children

101

‘[10,13,16]’

102

‘[8,11]’

103

‘[2]’

+

The following query gets the collection of children for each customer. +Note that the json_query function can only output a single JSON item. If +you don’t use array wrapper, you get an error for every customer with multiple +children. The error is handled according to the ON ERROR clause.

+
SELECT
+      id,
+      json_query(
+                 description,
+                 'lax $.children[*]'
+                 WITHOUT ARRAY WRAPPER
+                 NULL ON ERROR
+                ) AS children
+FROM customers
+
+
+ + + + + + + + + + + + + + + + + +

id

children

101

NULL

102

NULL

103

‘2’

+

The following query gets the last child for each customer, wrapped in a JSON +array:

+
SELECT
+      id,
+      json_query(
+                 description,
+                 'lax $.children[last]'
+                 WITH ARRAY WRAPPER
+                ) AS last_child
+FROM customers
+
+
+ + + + + + + + + + + + + + + + + +

id

last_child

101

‘[16]’

102

‘[11]’

103

‘[2]’

+

The following query gets all children above the age of 12 for each customer, +wrapped in a JSON array. The second and the third customer don’t have children +of this age. Such case is handled according to the ON EMPTY clause. The +default value returned ON EMPTY is NULL. In the following example, +EMPTY ARRAY ON EMPTY is specified.

+
SELECT
+      id,
+      json_query(
+                 description,
+                 'strict $.children[*]?(@ > 12)'
+                 WITH ARRAY WRAPPER
+                 EMPTY ARRAY ON EMPTY
+                ) AS children
+FROM customers
+
+
+ + + + + + + + + + + + + + + + + +

id

children

101

‘[13,16]’

102

‘[]’

103

‘[]’

+

The following query shows the result of the QUOTES clause. Note that KEEP QUOTES is the default.

+
SELECT
+      id,
+      json_query(description, 'strict $.comment' KEEP QUOTES) AS quoted_comment,
+      json_query(description, 'strict $.comment' OMIT QUOTES) AS unquoted_comment
+FROM customers
+
+
+ + + + + + + + + + + + + + + + + + + + + +

id

quoted_comment

unquoted_comment

101

‘“nice”’

‘nice’

102

‘“problematic”’

‘problematic’

103

‘“knows best”’

‘knows best’

+

If an error occurs, the returned value depends on the ON ERROR clause. The +default value returned ON ERROR is NULL. One example of error is +multiple items returned by the path. Other errors caught and handled according +to the ON ERROR clause are:

+
    +
  • Input conversion errors, such as malformed JSON

  • +
  • JSON path evaluation errors, e.g. division by zero

  • +
  • Output conversion errors

  • +
+
+
+
+

json_value#

+

The json_value function extracts a scalar SQL value from a JSON value.

+
JSON_VALUE(
+    json_input [ FORMAT JSON [ ENCODING { UTF8 | UTF16 | UTF32 } ] ],
+    json_path
+    [ PASSING json_argument [, ...] ]
+    [ RETURNING type ]
+    [ { ERROR | NULL | DEFAULT expression } ON EMPTY ]
+    [ { ERROR | NULL | DEFAULT expression } ON ERROR ]
+    )
+
+
+

The json_path is evaluated using the json_input as the context variable +($), and the passed arguments as the named variables ($variable_name).

+

The returned value is the SQL scalar returned by the path. By default, it is +converted to string (varchar). In the RETURNING clause, you can specify +other desired type: a character string type, numeric, boolean or datetime type.

+

json_input is a character string or a binary string. It should contain +a single JSON item. For a binary string, you can specify encoding.

+

json_path is a string literal, containing the path mode specification, and +the path expression, following the syntax rules described in +JSON path syntax and semantics.

+
'strict $.price + $tax'
+'lax $[last].abs().floor()'
+
+
+

In the PASSING clause you can pass arbitrary expressions to be used by the +path expression.

+
PASSING orders.tax AS O_TAX
+
+
+

The passed parameters can be referenced in the path expression by named +variables, prefixed with $.

+
'strict $[last].price + $O_TAX'
+
+
+

Additionally to SQL values, you can pass JSON values, specifying the format and +optional encoding:

+
PASSING orders.json_desc FORMAT JSON AS o_desc,
+        orders.binary_record FORMAT JSON ENCODING UTF16 AS o_rec
+
+
+

Note that the JSON path language is case-sensitive, while the unquoted SQL +identifiers are upper-cased. Therefore, it is recommended to use quoted +identifiers in the PASSING clause:

+
'lax $.$KeyName' PASSING nation.name AS KeyName --> ERROR; no passed value found
+'lax $.$KeyName' PASSING nation.name AS "KeyName" --> correct
+
+
+

If the path returns an empty sequence, the ON EMPTY clause is applied. The +default value returned ON EMPTY is NULL. You can also specify the +default value:

+
DEFAULT -1 ON EMPTY
+
+
+

If an error occurs, the returned value depends on the ON ERROR clause. The +default value returned ON ERROR is NULL. One example of error is +multiple items returned by the path. Other errors caught and handled according +to the ON ERROR clause are:

+
    +
  • Input conversion errors, such as malformed JSON

  • +
  • JSON path evaluation errors, e.g. division by zero

  • +
  • Returned scalar not convertible to the desired type

  • +
+
+

Examples#

+

Let customers be a table containing two columns: id:bigint, +description:varchar.

+ + + + + + + + + + + + + + + + + +

id

description

101

‘{“comment” : “nice”, “children” : [10, 13, 16]}’

102

‘{“comment” : “problematic”, “children” : [8, 11]}’

103

‘{“comment” : “knows best”, “children” : [2]}’

+

The following query gets the comment for each customer as char(12):

+
SELECT id, json_value(
+                      description,
+                      'lax $.comment'
+                      RETURNING char(12)
+                     ) AS comment
+FROM customers
+
+
+ + + + + + + + + + + + + + + + + +

id

comment

101

‘nice ‘

102

‘problematic ‘

103

‘knows best ‘

+

The following query gets the first child’s age for each customer as +tinyint:

+
SELECT id, json_value(
+                      description,
+                      'lax $.children[0]'
+                      RETURNING tinyint
+                     ) AS child
+FROM customers
+
+
+ + + + + + + + + + + + + + + + + +

id

child

101

10

102

8

103

2

+

The following query gets the third child’s age for each customer. In the strict +mode, this should cause a structural error for the customers who do not have +the third child. This error is handled according to the ON ERROR clause.

+
SELECT id, json_value(
+                      description,
+                      'strict $.children[2]'
+                      DEFAULT 'err' ON ERROR
+                     ) AS child
+FROM customers
+
+
+ + + + + + + + + + + + + + + + + +

id

child

101

‘16’

102

‘err’

103

‘err’

+

After changing the mode to lax, the structural error is suppressed, and the +customers without a third child produce empty sequence. This case is handled +according to the ON EMPTY clause.

+
SELECT id, json_value(
+                      description,
+                      'lax $.children[2]'
+                      DEFAULT 'missing' ON EMPTY
+                     ) AS child
+FROM customers
+
+
+ + + + + + + + + + + + + + + + + +

id

child

101

‘16’

102

‘missing’

103

‘missing’

+
+
+
+

json_array#

+

The json_array function creates a JSON array containing given elements.

+
JSON_ARRAY(
+    [ array_element [, ...]
+      [ { NULL ON NULL | ABSENT ON NULL } ] ],
+    [ RETURNING type [ FORMAT JSON [ ENCODING { UTF8 | UTF16 | UTF32 } ] ] ]
+    )
+
+
+
+

Argument types#

+

The array elements can be arbitrary expressions. Each passed value is converted +into a JSON item according to its type, and optional FORMAT and +ENCODING specification.

+

You can pass SQL values of types boolean, numeric, and character string. They +are converted to corresponding JSON literals:

+
SELECT json_array(true, 12e-1, 'text')
+--> '[true,1.2,"text"]'
+
+
+

Additionally to SQL values, you can pass JSON values. They are character or +binary strings with a specified format and optional encoding:

+
SELECT json_array(
+                  '[  "text"  ] ' FORMAT JSON,
+                  X'5B0035005D00' FORMAT JSON ENCODING UTF16
+                 )
+--> '[["text"],[5]]'
+
+
+

You can also nest other JSON-returning functions. In that case, the FORMAT +option is implicit:

+
SELECT json_array(
+                  json_query('{"key" : [  "value"  ]}', 'lax $.key')
+                 )
+--> '[["value"]]'
+
+
+

Other passed values are cast to varchar, and they become JSON text literals:

+
SELECT json_array(
+                  DATE '2001-01-31',
+                  UUID '12151fd2-7586-11e9-8f9e-2a86e4085a59'
+                 )
+--> '["2001-01-31","12151fd2-7586-11e9-8f9e-2a86e4085a59"]'
+
+
+

You can omit the arguments altogether to get an empty array:

+
SELECT json_array() --> '[]'
+
+
+
+
+

Null handling#

+

If a value passed for an array element is null, it is treated according to +the specified null treatment option. If ABSENT ON NULL is specified, the +null element is omitted in the result. If NULL ON NULL is specified, JSON +null is added to the result. ABSENT ON NULL is the default +configuration:

+
SELECT json_array(true, null, 1)
+--> '[true,1]'
+
+SELECT json_array(true, null, 1 ABSENT ON NULL)
+--> '[true,1]'
+
+SELECT json_array(true, null, 1 NULL ON NULL)
+--> '[true,null,1]'
+
+
+
+
+

Returned type#

+

The SQL standard imposes that there is no dedicated data type to represent JSON +data in SQL. Instead, JSON data is represented as character or binary strings. +By default, the json_array function returns varchar containing the textual +representation of the JSON array. With the RETURNING clause, you can +specify other character string type:

+
SELECT json_array(true, 1 RETURNING VARCHAR(100))
+--> '[true,1]'
+
+
+

You can also specify to use varbinary and the required encoding as return type. +The default encoding is UTF8:

+
SELECT json_array(true, 1 RETURNING VARBINARY)
+--> X'5b 74 72 75 65 2c 31 5d'
+
+SELECT json_array(true, 1 RETURNING VARBINARY FORMAT JSON ENCODING UTF8)
+--> X'5b 74 72 75 65 2c 31 5d'
+
+SELECT json_array(true, 1 RETURNING VARBINARY FORMAT JSON ENCODING UTF16)
+--> X'5b 00 74 00 72 00 75 00 65 00 2c 00 31 00 5d 00'
+
+SELECT json_array(true, 1 RETURNING VARBINARY FORMAT JSON ENCODING UTF32)
+--> X'5b 00 00 00 74 00 00 00 72 00 00 00 75 00 00 00 65 00 00 00 2c 00 00 00 31 00 00 00 5d 00 00 00'
+
+
+
+
+
+

json_object#

+

The json_object function creates a JSON object containing given key-value pairs.

+
JSON_OBJECT(
+    [ key_value [, ...]
+      [ { NULL ON NULL | ABSENT ON NULL } ] ],
+      [ { WITH UNIQUE [ KEYS ] | WITHOUT UNIQUE [ KEYS ] } ]
+    [ RETURNING type [ FORMAT JSON [ ENCODING { UTF8 | UTF16 | UTF32 } ] ] ]
+    )
+
+
+
+

Argument passing conventions#

+

There are two conventions for passing keys and values:

+
SELECT json_object('key1' : 1, 'key2' : true)
+--> '{"key1":1,"key2":true}'
+
+SELECT json_object(KEY 'key1' VALUE 1, KEY 'key2' VALUE true)
+--> '{"key1":1,"key2":true}'
+
+
+

In the second convention, you can omit the KEY keyword:

+
SELECT json_object('key1' VALUE 1, 'key2' VALUE true)
+--> '{"key1":1,"key2":true}'
+
+
+
+
+

Argument types#

+

The keys can be arbitrary expressions. They must be of character string type. +Each key is converted into a JSON text item, and it becomes a key in the +created JSON object. Keys must not be null.

+

The values can be arbitrary expressions. Each passed value is converted +into a JSON item according to its type, and optional FORMAT and +ENCODING specification.

+

You can pass SQL values of types boolean, numeric, and character string. They +are converted to corresponding JSON literals:

+
SELECT json_object('x' : true, 'y' : 12e-1, 'z' : 'text')
+--> '{"x":true,"y":1.2,"z":"text"}'
+
+
+

Additionally to SQL values, you can pass JSON values. They are character or +binary strings with a specified format and optional encoding:

+
SELECT json_object(
+                   'x' : '[  "text"  ] ' FORMAT JSON,
+                   'y' : X'5B0035005D00' FORMAT JSON ENCODING UTF16
+                  )
+--> '{"x":["text"],"y":[5]}'
+
+
+

You can also nest other JSON-returning functions. In that case, the FORMAT +option is implicit:

+
SELECT json_object(
+                   'x' : json_query('{"key" : [  "value"  ]}', 'lax $.key')
+                  )
+--> '{"x":["value"]}'
+
+
+

Other passed values are cast to varchar, and they become JSON text literals:

+
SELECT json_object(
+                   'x' : DATE '2001-01-31',
+                   'y' : UUID '12151fd2-7586-11e9-8f9e-2a86e4085a59'
+                  )
+--> '{"x":"2001-01-31","y":"12151fd2-7586-11e9-8f9e-2a86e4085a59"}'
+
+
+

You can omit the arguments altogether to get an empty object:

+
SELECT json_object() --> '{}'
+
+
+
+
+

Null handling#

+

The values passed for JSON object keys must not be null. It is allowed to pass +null for JSON object values. A null value is treated according to the +specified null treatment option. If NULL ON NULL is specified, a JSON +object entry with null value is added to the result. If ABSENT ON NULL +is specified, the entry is omitted in the result. NULL ON NULL is the +default configuration.:

+
SELECT json_object('x' : null, 'y' : 1)
+--> '{"x":null,"y":1}'
+
+SELECT json_object('x' : null, 'y' : 1 NULL ON NULL)
+--> '{"x":null,"y":1}'
+
+SELECT json_object('x' : null, 'y' : 1 ABSENT ON NULL)
+--> '{"y":1}'
+
+
+
+
+

Key uniqueness#

+

If a duplicate key is encountered, it is handled according to the specified key +uniqueness constraint.

+

If WITH UNIQUE KEYS is specified, a duplicate key results in a query +failure:

+
SELECT json_object('x' : null, 'x' : 1 WITH UNIQUE KEYS)
+--> failure: "duplicate key passed to JSON_OBJECT function"
+
+
+

Note that this option is not supported if any of the arguments has a +FORMAT specification.

+

If WITHOUT UNIQUE KEYS is specified, duplicate keys are not supported due +to implementation limitation. WITHOUT UNIQUE KEYS is the default +configuration.

+
+
+

Returned type#

+

The SQL standard imposes that there is no dedicated data type to represent JSON +data in SQL. Instead, JSON data is represented as character or binary strings. +By default, the json_object function returns varchar containing the textual +representation of the JSON object. With the RETURNING clause, you can +specify other character string type:

+
SELECT json_object('x' : 1 RETURNING VARCHAR(100))
+--> '{"x":1}'
+
+
+

You can also specify to use varbinary and the required encoding as return type. +The default encoding is UTF8:

+
SELECT json_object('x' : 1 RETURNING VARBINARY)
+--> X'7b 22 78 22 3a 31 7d'
+
+SELECT json_object('x' : 1 RETURNING VARBINARY FORMAT JSON ENCODING UTF8)
+--> X'7b 22 78 22 3a 31 7d'
+
+SELECT json_object('x' : 1 RETURNING VARBINARY FORMAT JSON ENCODING UTF16)
+--> X'7b 00 22 00 78 00 22 00 3a 00 31 00 7d 00'
+
+SELECT json_object('x' : 1 RETURNING VARBINARY FORMAT JSON ENCODING UTF32)
+--> X'7b 00 00 00 22 00 00 00 78 00 00 00 22 00 00 00 3a 00 00 00 31 00 00 00 7d 00 00 00'
+
+
+
+

Warning

+

The following functions and operators are not compliant with the SQL +standard, and should be considered deprecated. According to the SQL +standard, there shall be no JSON data type. Instead, JSON values +should be represented as string values. The remaining functionality of the +following functions is covered by the functions described previously.

+
+
+
+
+

Cast to JSON#

+

The following types can be cast to JSON:

+
    +
  • BOOLEAN

  • +
  • TINYINT

  • +
  • SMALLINT

  • +
  • INTEGER

  • +
  • BIGINT

  • +
  • REAL

  • +
  • DOUBLE

  • +
  • VARCHAR

  • +
+

Additionally, ARRAY, MAP, and ROW types can be cast to JSON when +the following requirements are met:

+
    +
  • ARRAY types can be cast when the element type of the array is one +of the supported types.

  • +
  • MAP types can be cast when the key type of the map is VARCHAR and +the value type of the map is a supported type,

  • +
  • ROW types can be cast when every field type of the row is a supported +type.

  • +
+
+

Note

+

Cast operations with supported character string types treat the input as a string, not validated as JSON. +This means that a cast operation with a string-type input of invalid JSON +results in a succesful cast to invalid JSON.

+

Instead, consider using the json_parse() function to +create validated JSON from a string.

+
+

The following examples show the behavior of casting to JSON with these types:

+
SELECT CAST(NULL AS JSON);
+-- NULL
+
+SELECT CAST(1 AS JSON);
+-- JSON '1'
+
+SELECT CAST(9223372036854775807 AS JSON);
+-- JSON '9223372036854775807'
+
+SELECT CAST('abc' AS JSON);
+-- JSON '"abc"'
+
+SELECT CAST(true AS JSON);
+-- JSON 'true'
+
+SELECT CAST(1.234 AS JSON);
+-- JSON '1.234'
+
+SELECT CAST(ARRAY[1, 23, 456] AS JSON);
+-- JSON '[1,23,456]'
+
+SELECT CAST(ARRAY[1, NULL, 456] AS JSON);
+-- JSON '[1,null,456]'
+
+SELECT CAST(ARRAY[ARRAY[1, 23], ARRAY[456]] AS JSON);
+-- JSON '[[1,23],[456]]'
+
+SELECT CAST(MAP(ARRAY['k1', 'k2', 'k3'], ARRAY[1, 23, 456]) AS JSON);
+-- JSON '{"k1":1,"k2":23,"k3":456}'
+
+SELECT CAST(CAST(ROW(123, 'abc', true) AS
+            ROW(v1 BIGINT, v2 VARCHAR, v3 BOOLEAN)) AS JSON);
+-- JSON '{"v1":123,"v2":"abc","v3":true}'
+
+
+

Casting from NULL to JSON is not straightforward. Casting +from a standalone NULL will produce SQL NULL instead of +JSON 'null'. However, when casting from arrays or map containing +NULLs, the produced JSON will have nulls in it.

+
+
+

Cast from JSON#

+

Casting to BOOLEAN, TINYINT, SMALLINT, INTEGER, +BIGINT, REAL, DOUBLE or VARCHAR is supported. +Casting to ARRAY and MAP is supported when the element type of +the array is one of the supported types, or when the key type of the map +is VARCHAR and value type of the map is one of the supported types. +Behaviors of the casts are shown with the examples below:

+
SELECT CAST(JSON 'null' AS VARCHAR);
+-- NULL
+
+SELECT CAST(JSON '1' AS INTEGER);
+-- 1
+
+SELECT CAST(JSON '9223372036854775807' AS BIGINT);
+-- 9223372036854775807
+
+SELECT CAST(JSON '"abc"' AS VARCHAR);
+-- abc
+
+SELECT CAST(JSON 'true' AS BOOLEAN);
+-- true
+
+SELECT CAST(JSON '1.234' AS DOUBLE);
+-- 1.234
+
+SELECT CAST(JSON '[1,23,456]' AS ARRAY(INTEGER));
+-- [1, 23, 456]
+
+SELECT CAST(JSON '[1,null,456]' AS ARRAY(INTEGER));
+-- [1, NULL, 456]
+
+SELECT CAST(JSON '[[1,23],[456]]' AS ARRAY(ARRAY(INTEGER)));
+-- [[1, 23], [456]]
+
+SELECT CAST(JSON '{"k1":1,"k2":23,"k3":456}' AS MAP(VARCHAR, INTEGER));
+-- {k1=1, k2=23, k3=456}
+
+SELECT CAST(JSON '{"v1":123,"v2":"abc","v3":true}' AS
+            ROW(v1 BIGINT, v2 VARCHAR, v3 BOOLEAN));
+-- {v1=123, v2=abc, v3=true}
+
+SELECT CAST(JSON '[123,"abc",true]' AS
+            ROW(v1 BIGINT, v2 VARCHAR, v3 BOOLEAN));
+-- {v1=123, v2=abc, v3=true}
+
+
+

JSON arrays can have mixed element types and JSON maps can have mixed +value types. This makes it impossible to cast them to SQL arrays and maps in +some cases. To address this, Trino supports partial casting of arrays and maps:

+
SELECT CAST(JSON '[[1, 23], 456]' AS ARRAY(JSON));
+-- [JSON '[1,23]', JSON '456']
+
+SELECT CAST(JSON '{"k1": [1, 23], "k2": 456}' AS MAP(VARCHAR, JSON));
+-- {k1 = JSON '[1,23]', k2 = JSON '456'}
+
+SELECT CAST(JSON '[null]' AS ARRAY(JSON));
+-- [JSON 'null']
+
+
+

When casting from JSON to ROW, both JSON array and JSON object are supported.

+
+
+

Other JSON functions#

+

In addition to the functions explained in more details in the preceding +sections, the following functions are available:

+
+
+is_json_scalar(json) boolean#
+

Determine if json is a scalar (i.e. a JSON number, a JSON string, true, false or null):

+
SELECT is_json_scalar('1');         -- true
+SELECT is_json_scalar('[1, 2, 3]'); -- false
+
+
+
+
+
+json_array_contains(json, value) boolean#
+

Determine if value exists in json (a string containing a JSON array):

+
SELECT json_array_contains('[1, 2, 3]', 2); -- true
+
+
+
+
+
+json_array_get(json_array, index) json#
+
+

Warning

+

The semantics of this function are broken. If the extracted element +is a string, it will be converted into an invalid JSON value that +is not properly quoted (the value will not be surrounded by quotes +and any interior quotes will not be escaped).

+

We recommend against using this function. It cannot be fixed without +impacting existing usages and may be removed in a future release.

+
+

Returns the element at the specified index into the json_array. +The index is zero-based:

+
SELECT json_array_get('["a", [3, 9], "c"]', 0); -- JSON 'a' (invalid JSON)
+SELECT json_array_get('["a", [3, 9], "c"]', 1); -- JSON '[3,9]'
+
+
+

This function also supports negative indexes for fetching element indexed +from the end of an array:

+
SELECT json_array_get('["c", [3, 9], "a"]', -1); -- JSON 'a' (invalid JSON)
+SELECT json_array_get('["c", [3, 9], "a"]', -2); -- JSON '[3,9]'
+
+
+

If the element at the specified index doesn’t exist, the function returns null:

+
SELECT json_array_get('[]', 0);                -- NULL
+SELECT json_array_get('["a", "b", "c"]', 10);  -- NULL
+SELECT json_array_get('["c", "b", "a"]', -10); -- NULL
+
+
+
+
+
+json_array_length(json) bigint#
+

Returns the array length of json (a string containing a JSON array):

+
SELECT json_array_length('[1, 2, 3]'); -- 3
+
+
+
+
+
+json_extract(json, json_path) json#
+

Evaluates the JSONPath-like expression json_path on json +(a string containing JSON) and returns the result as a JSON string:

+
SELECT json_extract(json, '$.store.book');
+SELECT json_extract(json, '$.store[book]');
+SELECT json_extract(json, '$.store["book name"]');
+
+
+

The json_query function provides a more powerful and +feature-rich alternative to parse and extract JSON data.

+
+
+
+json_extract_scalar(json, json_path) varchar#
+

Like json_extract(), but returns the result value as a string (as opposed +to being encoded as JSON). The value referenced by json_path must be a +scalar (boolean, number or string).

+
SELECT json_extract_scalar('[1, 2, 3]', '$[2]');
+SELECT json_extract_scalar(json, '$.store.book[0].author');
+
+
+
+
+
+json_format(json) varchar#
+

Returns the JSON text serialized from the input JSON value. +This is inverse function to json_parse().

+
SELECT json_format(JSON '[1, 2, 3]'); -- '[1,2,3]'
+SELECT json_format(JSON '"a"');       -- '"a"'
+
+
+
+

Note

+

json_format() and CAST(json AS VARCHAR) have completely +different semantics.

+

json_format() serializes the input JSON value to JSON text conforming to +RFC 7159. The JSON value can be a JSON object, a JSON array, a JSON string, +a JSON number, true, false or null.

+
SELECT json_format(JSON '{"a": 1, "b": 2}'); -- '{"a":1,"b":2}'
+SELECT json_format(JSON '[1, 2, 3]');        -- '[1,2,3]'
+SELECT json_format(JSON '"abc"');            -- '"abc"'
+SELECT json_format(JSON '42');               -- '42'
+SELECT json_format(JSON 'true');             -- 'true'
+SELECT json_format(JSON 'null');             -- 'null'
+
+
+

CAST(json AS VARCHAR) casts the JSON value to the corresponding SQL VARCHAR value. +For JSON string, JSON number, true, false or null, the cast +behavior is same as the corresponding SQL type. JSON object and JSON array +cannot be cast to VARCHAR.

+
SELECT CAST(JSON '{"a": 1, "b": 2}' AS VARCHAR); -- ERROR!
+SELECT CAST(JSON '[1, 2, 3]' AS VARCHAR);        -- ERROR!
+SELECT CAST(JSON '"abc"' AS VARCHAR);            -- 'abc' (the double quote is gone)
+SELECT CAST(JSON '42' AS VARCHAR);               -- '42'
+SELECT CAST(JSON 'true' AS VARCHAR);             -- 'true'
+SELECT CAST(JSON 'null' AS VARCHAR);             -- NULL
+
+
+
+
+
+
+json_parse(string) json#
+

Returns the JSON value deserialized from the input JSON text. +This is inverse function to json_format():

+
SELECT json_parse('[1, 2, 3]');   -- JSON '[1,2,3]'
+SELECT json_parse('"abc"');       -- JSON '"abc"'
+
+
+
+

Note

+

json_parse() and CAST(string AS JSON) have completely +different semantics.

+

json_parse() expects a JSON text conforming to RFC 7159, and returns +the JSON value deserialized from the JSON text. +The JSON value can be a JSON object, a JSON array, a JSON string, a JSON number, +true, false or null.

+
SELECT json_parse('not_json');         -- ERROR!
+SELECT json_parse('["a": 1, "b": 2]'); -- JSON '["a": 1, "b": 2]'
+SELECT json_parse('[1, 2, 3]');        -- JSON '[1,2,3]'
+SELECT json_parse('"abc"');            -- JSON '"abc"'
+SELECT json_parse('42');               -- JSON '42'
+SELECT json_parse('true');             -- JSON 'true'
+SELECT json_parse('null');             -- JSON 'null'
+
+
+

CAST(string AS JSON) takes any VARCHAR value as input, and returns +a JSON string with its value set to input string.

+
SELECT CAST('not_json' AS JSON);         -- JSON '"not_json"'
+SELECT CAST('["a": 1, "b": 2]' AS JSON); -- JSON '"[\"a\": 1, \"b\": 2]"'
+SELECT CAST('[1, 2, 3]' AS JSON);        -- JSON '"[1, 2, 3]"'
+SELECT CAST('"abc"' AS JSON);            -- JSON '"\"abc\""'
+SELECT CAST('42' AS JSON);               -- JSON '"42"'
+SELECT CAST('true' AS JSON);             -- JSON '"true"'
+SELECT CAST('null' AS JSON);             -- JSON '"null"'
+
+
+
+
+
+
+json_size(json, json_path) bigint#
+

Like json_extract(), but returns the size of the value. +For objects or arrays, the size is the number of members, +and the size of a scalar value is zero.

+
SELECT json_size('{"x": {"a": 1, "b": 2}}', '$.x');   -- 2
+SELECT json_size('{"x": [1, 2, 3]}', '$.x');          -- 3
+SELECT json_size('{"x": {"a": 1, "b": 2}}', '$.x.a'); -- 0
+
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/lambda.html b/430/functions/lambda.html new file mode 100644 index 000000000..bacede6ac --- /dev/null +++ b/430/functions/lambda.html @@ -0,0 +1,762 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Lambda expressions — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Lambda expressions#

+

Lambda expressions are anonymous functions which are passed as +arguments to higher-order SQL functions.

+

Lambda expressions are written with ->:

+
x -> x + 1
+(x, y) -> x + y
+x -> regexp_like(x, 'a+')
+x -> x[1] / x[2]
+x -> IF(x > 0, x, -x)
+x -> COALESCE(x, 0)
+x -> CAST(x AS JSON)
+x -> x + TRY(1 / 0)
+
+
+
+

Limitations#

+

Most SQL expressions can be used in a lambda body, with a few exceptions:

+
    +
  • Subqueries are not supported: x -> 2 + (SELECT 3)

  • +
  • Aggregations are not supported: x -> max(y)

  • +
+
+
+

Examples#

+

Obtain the squared elements of an array column with transform():

+
SELECT numbers,
+       transform(numbers, n -> n * n) as squared_numbers
+FROM (
+    VALUES
+        (ARRAY[1, 2]),
+        (ARRAY[3, 4]),
+        (ARRAY[5, 6, 7])
+) AS t(numbers);
+
+
+
  numbers  | squared_numbers
+-----------+-----------------
+ [1, 2]    | [1, 4]
+ [3, 4]    | [9, 16]
+ [5, 6, 7] | [25, 36, 49]
+(3 rows)
+
+
+

The function transform() can be also employed to safely cast the elements +of an array to strings:

+
SELECT transform(prices, n -> TRY_CAST(n AS VARCHAR) || '$') as price_tags
+FROM (
+    VALUES
+        (ARRAY[100, 200]),
+        (ARRAY[30, 4])
+) AS t(prices);
+
+
+
  price_tags
+--------------
+ [100$, 200$]
+ [30$, 4$]
+(2 rows)
+
+
+

Besides the array column being manipulated, +other columns can be captured as well within the lambda expression. +The following statement provides a showcase of this feature +for calculating the value of the linear function f(x) = ax + b +with transform():

+
SELECT xvalues,
+       a,
+       b,
+       transform(xvalues, x -> a * x + b) as linear_function_values
+FROM (
+    VALUES
+        (ARRAY[1, 2], 10, 5),
+        (ARRAY[3, 4], 4, 2)
+) AS t(xvalues, a, b);
+
+
+
 xvalues | a  | b | linear_function_values
+---------+----+---+------------------------
+ [1, 2]  | 10 | 5 | [15, 25]
+ [3, 4]  |  4 | 2 | [14, 18]
+(2 rows)
+
+
+

Find the array elements containing at least one value greater than 100 +with any_match():

+
SELECT numbers
+FROM (
+    VALUES
+        (ARRAY[1,NULL,3]),
+        (ARRAY[10,20,30]),
+        (ARRAY[100,200,300])
+) AS t(numbers)
+WHERE any_match(numbers, n ->  COALESCE(n, 0) > 100);
+-- [100, 200, 300]
+
+
+

Capitalize the first word in a string via regexp_replace():

+
SELECT regexp_replace('once upon a time ...', '^(\w)(\w*)(\s+.*)$',x -> upper(x[1]) || x[2] || x[3]);
+-- Once upon a time ...
+
+
+

Lambda expressions can be also applied in aggregation functions. +Following statement is a sample the overly complex calculation of the sum of all elements of a column +by making use of reduce_agg():

+
SELECT reduce_agg(value, 0, (a, b) -> a + b, (a, b) -> a + b) sum_values
+FROM (
+    VALUES (1), (2), (3), (4), (5)
+) AS t(value);
+-- 15
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/list-by-topic.html b/430/functions/list-by-topic.html new file mode 100644 index 000000000..62ea7d0d2 --- /dev/null +++ b/430/functions/list-by-topic.html @@ -0,0 +1,1338 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + List of functions by topic — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+ +
+ +
+
+ +
+

List of functions by topic#

+
+

Aggregate#

+

For more details, see Aggregate functions

+ +
+
+

Array#

+

For more details, see Array functions and operators

+ +
+
+

Binary#

+

For more details, see Binary functions and operators

+ +
+
+

Bitwise#

+

For more details, see Bitwise functions

+ +
+
+

Color#

+

For more details, see Color functions

+ +
+
+

Comparison#

+

For more details, see Comparison functions and operators

+ +
+
+

Conditional#

+

For more details, see Conditional expressions

+ +
+
+

Conversion#

+

For more details, see Conversion functions

+ +
+
+

Date and time#

+

For more details, see Date and time functions and operators

+ +
+
+

Geospatial#

+

For more details, see Geospatial functions

+ +
+
+

HyperLogLog#

+

For more details, see HyperLogLog functions

+ +
+
+

JSON#

+

For more details, see JSON functions and operators

+ +
+
+

Lambda#

+

For more details, see Lambda expressions

+ +
+
+

Machine learning#

+

For more details, see Machine learning functions

+ +
+
+

Map#

+

For more details, see Map functions and operators

+ +
+
+

Math#

+

For more details, see Mathematical functions and operators

+ +
+
+

Quantile digest#

+

For more details, see Quantile digest functions

+ +
+
+

Regular expression#

+

For more details, see Regular expression functions

+ +
+
+

Row pattern recognition expressions#

+ +
+
+

Session#

+

For more details, see Session information

+ +
+
+

Set Digest#

+

For more details, see Set Digest functions

+ +
+
+

String#

+

For more details, see String functions and operators

+ +
+
+

System#

+

For more details, see System information

+ +
+
+

T-Digest#

+

For more details, see T-Digest functions

+ +
+
+

Teradata#

+

For more details, see Teradata functions

+ +
+
+

URL#

+

For more details, see URL functions

+ +
+
+

UUID#

+

For more details, see UUID functions

+ +
+
+

Window#

+

For more details, see Window functions

+ +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/list.html b/430/functions/list.html new file mode 100644 index 000000000..2c6bb4377 --- /dev/null +++ b/430/functions/list.html @@ -0,0 +1,1346 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + List of functions and operators — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

List of functions and operators#

+
+

##

+ +
+
+

A#

+ +
+
+

B#

+ +
+
+

C#

+ +
+
+

D#

+ +
+
+

E#

+ +
+
+

F#

+ +
+
+

G#

+ +
+
+

H#

+ +
+
+

I#

+ +
+
+

J#

+ +
+
+

K#

+ +
+
+

L#

+ +
+
+

M#

+ +
+
+

N#

+ +
+
+

O#

+ +
+
+

P#

+ +
+
+

Q#

+ +
+
+

R#

+ +
+
+

S#

+ +
+
+

T#

+ +
+
+

U#

+ +
+
+

V#

+ +
+
+

W#

+ +
+
+

X#

+ +
+
+

Y#

+ +
+
+

Z#

+ +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/logical.html b/430/functions/logical.html new file mode 100644 index 000000000..32ad57eb7 --- /dev/null +++ b/430/functions/logical.html @@ -0,0 +1,780 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Logical operators — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+ +
+
+ +
+
+ +
+

Logical operators#

+
+

Logical operators#

+ + + + + + + + + + + + + + + + + + + + + +

Operator

Description

Example

AND

True if both values are true

a AND b

OR

True if either value is true

a OR b

NOT

True if the value is false

NOT a

+
+
+

Effect of NULL on logical operators#

+

The result of an AND comparison may be NULL if one or both +sides of the expression are NULL. If at least one side of an +AND operator is FALSE the expression evaluates to FALSE:

+
SELECT CAST(null AS boolean) AND true; -- null
+
+SELECT CAST(null AS boolean) AND false; -- false
+
+SELECT CAST(null AS boolean) AND CAST(null AS boolean); -- null
+
+
+

The result of an OR comparison may be NULL if one or both +sides of the expression are NULL. If at least one side of an +OR operator is TRUE the expression evaluates to TRUE:

+
SELECT CAST(null AS boolean) OR CAST(null AS boolean); -- null
+
+SELECT CAST(null AS boolean) OR false; -- null
+
+SELECT CAST(null AS boolean) OR true; -- true
+
+
+

The following truth table demonstrates the handling of +NULL in AND and OR:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

a

b

a AND b

a OR b

TRUE

TRUE

TRUE

TRUE

TRUE

FALSE

FALSE

TRUE

TRUE

NULL

NULL

TRUE

FALSE

TRUE

FALSE

TRUE

FALSE

FALSE

FALSE

FALSE

FALSE

NULL

FALSE

NULL

NULL

TRUE

NULL

TRUE

NULL

FALSE

FALSE

NULL

NULL

NULL

NULL

NULL

+

The logical complement of NULL is NULL as shown in the following example:

+
SELECT NOT CAST(null AS boolean); -- null
+
+
+

The following truth table demonstrates the handling of NULL in NOT:

+ + + + + + + + + + + + + + + + + +

a

NOT a

TRUE

FALSE

FALSE

TRUE

NULL

NULL

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/map.html b/430/functions/map.html new file mode 100644 index 000000000..27e711bb7 --- /dev/null +++ b/430/functions/map.html @@ -0,0 +1,823 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Map functions and operators — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Map functions and operators#

+
+

Subscript operator: []#

+

The [] operator is used to retrieve the value corresponding to a given key from a map:

+
SELECT name_to_age_map['Bob'] AS bob_age;
+
+
+
+
+

Map functions#

+
+
+cardinality(x) bigint
+

Returns the cardinality (size) of the map x.

+
+
+
+element_at(map(K, V), key) V
+

Returns value for given key, or NULL if the key is not contained in the map.

+
+
+
+map() map<unknown, unknown>#
+

Returns an empty map.

+
SELECT map();
+-- {}
+
+
+
+
+
+map(array(K), array(V)) -> map(K, V)
+

Returns a map created using the given key/value arrays.

+
SELECT map(ARRAY[1,3], ARRAY[2,4]);
+-- {1 -> 2, 3 -> 4}
+
+
+

See also map_agg() and multimap_agg() for creating a map as an aggregation.

+
+
+
+map_from_entries(array(row(K, V))) -> map(K, V)#
+

Returns a map created from the given array of entries.

+
SELECT map_from_entries(ARRAY[(1, 'x'), (2, 'y')]);
+-- {1 -> 'x', 2 -> 'y'}
+
+
+
+
+
+multimap_from_entries(array(row(K, V))) -> map(K, array(V))#
+

Returns a multimap created from the given array of entries. Each key can be associated with multiple values.

+
SELECT multimap_from_entries(ARRAY[(1, 'x'), (2, 'y'), (1, 'z')]);
+-- {1 -> ['x', 'z'], 2 -> ['y']}
+
+
+
+
+
+map_entries(map(K, V)) -> array(row(K, V))#
+

Returns an array of all entries in the given map.

+
SELECT map_entries(MAP(ARRAY[1, 2], ARRAY['x', 'y']));
+-- [ROW(1, 'x'), ROW(2, 'y')]
+
+
+
+
+
+map_concat(map1(K, V), map2(K, V), ..., mapN(K, V)) -> map(K, V)#
+

Returns the union of all the given maps. If a key is found in multiple given maps, +that key’s value in the resulting map comes from the last one of those maps.

+
+
+
+map_filter(map(K, V), function(K, V, boolean)) -> map(K, V)#
+

Constructs a map from those entries of map for which function returns true:

+
SELECT map_filter(MAP(ARRAY[], ARRAY[]), (k, v) -> true);
+-- {}
+
+SELECT map_filter(MAP(ARRAY[10, 20, 30], ARRAY['a', NULL, 'c']),
+                  (k, v) -> v IS NOT NULL);
+-- {10 -> a, 30 -> c}
+
+SELECT map_filter(MAP(ARRAY['k1', 'k2', 'k3'], ARRAY[20, 3, 15]),
+                  (k, v) -> v > 10);
+-- {k1 -> 20, k3 -> 15}
+
+
+
+
+
+map_keys(x(K, V)) -> array(K)#
+

Returns all the keys in the map x.

+
+
+
+map_values(x(K, V)) -> array(V)#
+

Returns all the values in the map x.

+
+
+
+map_zip_with(map(K, V1), map(K, V2), function(K, V1, V2, V3)) -> map(K, V3)#
+

Merges the two given maps into a single map by applying function to the pair of values with the same key. +For keys only presented in one map, NULL will be passed as the value for the missing key.

+
SELECT map_zip_with(MAP(ARRAY[1, 2, 3], ARRAY['a', 'b', 'c']),
+                    MAP(ARRAY[1, 2, 3], ARRAY['d', 'e', 'f']),
+                    (k, v1, v2) -> concat(v1, v2));
+-- {1 -> ad, 2 -> be, 3 -> cf}
+
+SELECT map_zip_with(MAP(ARRAY['k1', 'k2'], ARRAY[1, 2]),
+                    MAP(ARRAY['k2', 'k3'], ARRAY[4, 9]),
+                    (k, v1, v2) -> (v1, v2));
+-- {k1 -> ROW(1, null), k2 -> ROW(2, 4), k3 -> ROW(null, 9)}
+
+SELECT map_zip_with(MAP(ARRAY['a', 'b', 'c'], ARRAY[1, 8, 27]),
+                    MAP(ARRAY['a', 'b', 'c'], ARRAY[1, 2, 3]),
+                    (k, v1, v2) -> k || CAST(v1 / v2 AS VARCHAR));
+-- {a -> a1, b -> b4, c -> c9}
+
+
+
+
+
+transform_keys(map(K1, V), function(K1, V, K2)) -> map(K2, V)#
+

Returns a map that applies function to each entry of map and transforms the keys:

+
SELECT transform_keys(MAP(ARRAY[], ARRAY[]), (k, v) -> k + 1);
+-- {}
+
+SELECT transform_keys(MAP(ARRAY [1, 2, 3], ARRAY ['a', 'b', 'c']),
+                      (k, v) -> k + 1);
+-- {2 -> a, 3 -> b, 4 -> c}
+
+SELECT transform_keys(MAP(ARRAY ['a', 'b', 'c'], ARRAY [1, 2, 3]),
+                      (k, v) -> v * v);
+-- {1 -> 1, 4 -> 2, 9 -> 3}
+
+SELECT transform_keys(MAP(ARRAY ['a', 'b'], ARRAY [1, 2]),
+                      (k, v) -> k || CAST(v as VARCHAR));
+-- {a1 -> 1, b2 -> 2}
+
+SELECT transform_keys(MAP(ARRAY [1, 2], ARRAY [1.0, 1.4]),
+                      (k, v) -> MAP(ARRAY[1, 2], ARRAY['one', 'two'])[k]);
+-- {one -> 1.0, two -> 1.4}
+
+
+
+
+
+transform_values(map(K, V1), function(K, V1, V2)) -> map(K, V2)#
+

Returns a map that applies function to each entry of map and transforms the values:

+
SELECT transform_values(MAP(ARRAY[], ARRAY[]), (k, v) -> v + 1);
+-- {}
+
+SELECT transform_values(MAP(ARRAY [1, 2, 3], ARRAY [10, 20, 30]),
+                        (k, v) -> v + k);
+-- {1 -> 11, 2 -> 22, 3 -> 33}
+
+SELECT transform_values(MAP(ARRAY [1, 2, 3], ARRAY ['a', 'b', 'c']),
+                        (k, v) -> k * k);
+-- {1 -> 1, 2 -> 4, 3 -> 9}
+
+SELECT transform_values(MAP(ARRAY ['a', 'b'], ARRAY [1, 2]),
+                        (k, v) -> k || CAST(v as VARCHAR));
+-- {a -> a1, b -> b2}
+
+SELECT transform_values(MAP(ARRAY [1, 2], ARRAY [1.0, 1.4]),
+                        (k, v) -> MAP(ARRAY[1, 2], ARRAY['one', 'two'])[k]
+                          || '_' || CAST(v AS VARCHAR));
+-- {1 -> one_1.0, 2 -> two_1.4}
+
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/math.html b/430/functions/math.html new file mode 100644 index 000000000..8ecf83bbf --- /dev/null +++ b/430/functions/math.html @@ -0,0 +1,1013 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Mathematical functions and operators — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Mathematical functions and operators#

+
+

Mathematical operators#

+ + + + + + + + + + + + + + + + + + + + + + + +

Operator

Description

+

Addition

-

Subtraction

*

Multiplication

/

Division (integer division performs truncation)

%

Modulus (remainder)

+
+
+

Mathematical functions#

+
+
+abs(x) [same as input]#
+

Returns the absolute value of x.

+
+
+
+cbrt(x) double#
+

Returns the cube root of x.

+
+
+
+ceil(x) [same as input]#
+

This is an alias for ceiling().

+
+
+
+ceiling(x) [same as input]#
+

Returns x rounded up to the nearest integer.

+
+
+
+degrees(x) double#
+

Converts angle x in radians to degrees.

+
+
+
+e() double#
+

Returns the constant Euler’s number.

+
+
+
+exp(x) double#
+

Returns Euler’s number raised to the power of x.

+
+
+
+floor(x) [same as input]#
+

Returns x rounded down to the nearest integer.

+
+
+
+ln(x) double#
+

Returns the natural logarithm of x.

+
+
+
+log(b, x) double#
+

Returns the base b logarithm of x.

+
+
+
+log2(x) double#
+

Returns the base 2 logarithm of x.

+
+
+
+log10(x) double#
+

Returns the base 10 logarithm of x.

+
+
+
+mod(n, m) [same as input]#
+

Returns the modulus (remainder) of n divided by m.

+
+
+
+pi() double#
+

Returns the constant Pi.

+
+
+
+pow(x, p) double#
+

This is an alias for power().

+
+
+
+power(x, p) double#
+

Returns x raised to the power of p.

+
+
+
+radians(x) double#
+

Converts angle x in degrees to radians.

+
+
+
+round(x) [same as input]#
+

Returns x rounded to the nearest integer.

+
+
+
+round(x, d) [same as input]
+

Returns x rounded to d decimal places.

+
+
+
+sign(x) [same as input]#
+

Returns the signum function of x, that is:

+
    +
  • 0 if the argument is 0,

  • +
  • 1 if the argument is greater than 0,

  • +
  • -1 if the argument is less than 0.

  • +
+

For double arguments, the function additionally returns:

+
    +
  • NaN if the argument is NaN,

  • +
  • 1 if the argument is +Infinity,

  • +
  • -1 if the argument is -Infinity.

  • +
+
+
+
+sqrt(x) double#
+

Returns the square root of x.

+
+
+
+truncate(x) double#
+

Returns x rounded to integer by dropping digits after decimal point.

+
+
+
+width_bucket(x, bound1, bound2, n) bigint#
+

Returns the bin number of x in an equi-width histogram with the +specified bound1 and bound2 bounds and n number of buckets.

+
+
+
+width_bucket(x, bins) bigint
+

Returns the bin number of x according to the bins specified by the +array bins. The bins parameter must be an array of doubles and is +assumed to be in sorted ascending order.

+
+
+
+

Random functions#

+
+
+rand() double#
+

This is an alias for random().

+
+
+
+random() double#
+

Returns a pseudo-random value in the range 0.0 <= x < 1.0.

+
+
+
+random(n) [same as input]
+

Returns a pseudo-random number between 0 and n (exclusive).

+
+
+
+random(m, n) [same as input]
+

Returns a pseudo-random number between m and n (exclusive).

+
+
+
+

Trigonometric functions#

+

All trigonometric function arguments are expressed in radians. +See unit conversion functions degrees() and radians().

+
+
+acos(x) double#
+

Returns the arc cosine of x.

+
+
+
+asin(x) double#
+

Returns the arc sine of x.

+
+
+
+atan(x) double#
+

Returns the arc tangent of x.

+
+
+
+atan2(y, x) double#
+

Returns the arc tangent of y / x.

+
+
+
+cos(x) double#
+

Returns the cosine of x.

+
+
+
+cosh(x) double#
+

Returns the hyperbolic cosine of x.

+
+
+
+sin(x) double#
+

Returns the sine of x.

+
+
+
+sinh(x) double#
+

Returns the hyperbolic sine of x.

+
+
+
+tan(x) double#
+

Returns the tangent of x.

+
+
+
+tanh(x) double#
+

Returns the hyperbolic tangent of x.

+
+
+
+

Floating point functions#

+
+
+infinity() double#
+

Returns the constant representing positive infinity.

+
+
+
+is_finite(x) boolean#
+

Determine if x is finite.

+
+
+
+is_infinite(x) boolean#
+

Determine if x is infinite.

+
+
+
+is_nan(x) boolean#
+

Determine if x is not-a-number.

+
+
+
+nan() double#
+

Returns the constant representing not-a-number.

+
+
+
+

Base conversion functions#

+
+
+from_base(string, radix) bigint#
+

Returns the value of string interpreted as a base-radix number.

+
+
+
+to_base(x, radix) varchar#
+

Returns the base-radix representation of x.

+
+
+
+

Statistical functions#

+
+
+cosine_similarity(x, y) double#
+

Returns the cosine similarity between the sparse vectors x and y:

+
SELECT cosine_similarity(MAP(ARRAY['a'], ARRAY[1.0]), MAP(ARRAY['a'], ARRAY[2.0])); -- 1.0
+
+
+
+
+
+wilson_interval_lower(successes, trials, z) double#
+

Returns the lower bound of the Wilson score interval of a Bernoulli trial process +at a confidence specified by the z-score z.

+
+
+
+wilson_interval_upper(successes, trials, z) double#
+

Returns the upper bound of the Wilson score interval of a Bernoulli trial process +at a confidence specified by the z-score z.

+
+
+
+

Cumulative distribution functions#

+
+
+beta_cdf(a, b, v) double#
+

Compute the Beta cdf with given a, b parameters: P(N < v; a, b). +The a, b parameters must be positive real numbers and value v must be a real value. +The value v must lie on the interval [0, 1].

+
+
+
+inverse_beta_cdf(a, b, p) double#
+

Compute the inverse of the Beta cdf with given a, b parameters for the cumulative +probability (p): P(N < n). The a, b parameters must be positive real values. +The probability p must lie on the interval [0, 1].

+
+
+
+inverse_normal_cdf(mean, sd, p) double#
+

Compute the inverse of the Normal cdf with given mean and standard +deviation (sd) for the cumulative probability (p): P(N < n). The mean must be +a real value and the standard deviation must be a real and positive value. +The probability p must lie on the interval (0, 1).

+
+
+
+normal_cdf(mean, sd, v) double#
+

Compute the Normal cdf with given mean and standard deviation (sd): P(N < v; mean, sd). +The mean and value v must be real values and the standard deviation must be a real +and positive value.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/ml.html b/430/functions/ml.html new file mode 100644 index 000000000..2a2c8c63e --- /dev/null +++ b/430/functions/ml.html @@ -0,0 +1,806 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Machine learning functions — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Machine learning functions#

+

The machine learning plugin provides machine learning functionality +as an aggregation function. It enables you to train Support Vector Machine (SVM) +based classifiers and regressors for the supervised learning problems.

+
+

Note

+

The machine learning functions are not optimized for distributed processing. +The capability to train large data sets is limited by this execution of the +final training on a single instance.

+
+
+

Feature vector#

+

To solve a problem with the machine learning technique, especially as a +supervised learning problem, it is necessary to represent the data set +with the sequence of pairs of labels and feature vector. A label is a +target value you want to predict from the unseen feature and a feature is a +A N-dimensional vector whose elements are numerical values. In Trino, a +feature vector is represented as a map-type value, whose key is an index +of each feature, so that it can express a sparse vector. +Since classifiers and regressors can recognize the map-type feature +vector, there is a function to construct the feature from the existing +numerical values, features():

+
SELECT features(1.0, 2.0, 3.0) AS features;
+
+
+
       features
+-----------------------
+ {0=1.0, 1=2.0, 2=3.0}
+
+
+

The output from features() can be directly passed to ML functions.

+
+
+

Classification#

+

Classification is a type of supervised learning problem to predict the distinct +label from the given feature vector. The interface looks similar to the +construction of the SVM model from the sequence of pairs of labels and features +implemented in Teradata Aster or BigQuery ML. +The function to train a classification model looks like as follows:

+
SELECT
+  learn_classifier(
+    species,
+    features(sepal_length, sepal_width, petal_length, petal_width)
+  ) AS model
+FROM
+  iris
+
+
+

It returns the trained model in a serialized format.

+
                      model
+-------------------------------------------------
+ 3c 43 6c 61 73 73 69 66 69 65 72 28 76 61 72 63
+ 68 61 72 29 3e
+
+
+

classify() returns the predicted label by using the trained model. +The trained model can not be saved natively, and needs to be passed in +the format of a nested query:

+
SELECT
+  classify(features(5.9, 3, 5.1, 1.8), model) AS predicted_label
+FROM (
+  SELECT
+    learn_classifier(species, features(sepal_length, sepal_width, petal_length, petal_width)) AS model
+  FROM
+    iris
+) t
+
+
+
 predicted_label
+-----------------
+ Iris-virginica
+
+
+

As a result you need to run the training process at the same time when predicting values. +Internally, the model is trained by libsvm. +You can use learn_libsvm_classifier() to control the internal parameters of the model.

+
+
+

Regression#

+

Regression is another type of supervised learning problem, predicting continuous +value, unlike the classification problem. The target must be numerical values that can +be described as double.

+

The following code shows the creation of the model predicting sepal_length +from the other 3 features:

+
SELECT
+  learn_regressor(sepal_length, features(sepal_width, petal_length, petal_width)) AS model
+FROM
+  iris
+
+
+

The way to use the model is similar to the classification case:

+
SELECT
+  regress(features(3, 5.1, 1.8), model) AS predicted_target
+FROM (
+  SELECT
+    learn_regressor(sepal_length, features(sepal_width, petal_length, petal_width)) AS model
+  FROM iris
+) t;
+
+
+
 predicted_target
+-------------------
+ 6.407376822560477
+
+
+

Internally, the model is trained by libsvm. +learn_libsvm_regressor() provides you a way to control the training process.

+
+
+

Machine learning functions#

+
+
+features(double, ...) -> map(bigint, double)#
+

Returns the map representing the feature vector.

+
+
+
+learn_classifier(label, features) Classifier#
+

Returns an SVM-based classifier model, trained with the given label and feature data sets.

+
+
+
+learn_libsvm_classifier(label, features, params) Classifier#
+

Returns an SVM-based classifier model, trained with the given label and feature data sets. +You can control the training process by libsvm parameters.

+
+
+
+classify(features, model) label#
+

Returns a label predicted by the given classifier SVM model.

+
+
+
+learn_regressor(target, features) Regressor#
+

Returns an SVM-based regressor model, trained with the given target and feature data sets.

+
+
+
+learn_libsvm_regressor(target, features, params) Regressor#
+

Returns an SVM-based regressor model, trained with the given target and feature data sets. +You can control the training process by libsvm parameters.

+
+
+
+regress(features, model) target#
+

Returns a predicted target value by the given regressor SVM model.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/qdigest.html b/430/functions/qdigest.html new file mode 100644 index 000000000..9a376b7a9 --- /dev/null +++ b/430/functions/qdigest.html @@ -0,0 +1,707 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Quantile digest functions — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Quantile digest functions#

+
+

Data structures#

+

A quantile digest is a data sketch which stores approximate percentile +information. The Trino type for this data structure is called qdigest, +and it takes a parameter which must be one of bigint, double or +real which represent the set of numbers that may be ingested by the +qdigest. They may be merged without losing precision, and for storage +and retrieval they may be cast to/from VARBINARY.

+
+
+

Functions#

+
+
+merge(qdigest) qdigest
+

Merges all input qdigests into a single qdigest.

+
+
+
+value_at_quantile(qdigest(T), quantile) T#
+

Returns the approximate percentile value from the quantile digest given +the number quantile between 0 and 1.

+
+
+
+quantile_at_value(qdigest(T), T) quantile#
+

Returns the approximate quantile number between 0 and 1 from the +quantile digest given an input value. Null is returned if the quantile digest +is empty or the input value is outside of the range of the quantile digest.

+
+
+
+values_at_quantiles(qdigest(T), quantiles) -> array(T)#
+

Returns the approximate percentile values as an array given the input +quantile digest and array of values between 0 and 1 which +represent the quantiles to return.

+
+
+
+qdigest_agg(x) -> qdigest([same as x])#
+

Returns the qdigest which is composed of all input values of x.

+
+
+
+qdigest_agg(x, w) -> qdigest([same as x])
+

Returns the qdigest which is composed of all input values of x using +the per-item weight w.

+
+
+
+qdigest_agg(x, w, accuracy) -> qdigest([same as x])
+

Returns the qdigest which is composed of all input values of x using +the per-item weight w and maximum error of accuracy. accuracy +must be a value greater than zero and less than one, and it must be constant +for all input rows.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/regexp.html b/430/functions/regexp.html new file mode 100644 index 000000000..ed6bfb138 --- /dev/null +++ b/430/functions/regexp.html @@ -0,0 +1,805 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Regular expression functions — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Regular expression functions#

+

All of the regular expression functions use the Java pattern syntax, +with a few notable exceptions:

+
    +
  • When using multi-line mode (enabled via the (?m) flag), +only \n is recognized as a line terminator. Additionally, +the (?d) flag is not supported and must not be used.

  • +
  • Case-insensitive matching (enabled via the (?i) flag) is always +performed in a Unicode-aware manner. However, context-sensitive and +local-sensitive matching is not supported. Additionally, the +(?u) flag is not supported and must not be used.

  • +
  • Surrogate pairs are not supported. For example, \uD800\uDC00 is +not treated as U+10000 and must be specified as \x{10000}.

  • +
  • Boundaries (\b) are incorrectly handled for a non-spacing mark +without a base character.

  • +
  • \Q and \E are not supported in character classes +(such as [A-Z123]) and are instead treated as literals.

  • +
  • Unicode character classes (\p{prop}) are supported with +the following differences:

    +
      +
    • All underscores in names must be removed. For example, use +OldItalic instead of Old_Italic.

    • +
    • Scripts must be specified directly, without the +Is, script= or sc= prefixes. +Example: \p{Hiragana}

    • +
    • Blocks must be specified with the In prefix. +The block= and blk= prefixes are not supported. +Example: \p{Mongolian}

    • +
    • Categories must be specified directly, without the Is, +general_category= or gc= prefixes. +Example: \p{L}

    • +
    • Binary properties must be specified directly, without the Is. +Example: \p{NoncharacterCodePoint}

    • +
    +
  • +
+
+
+regexp_count(string, pattern) bigint#
+

Returns the number of occurrence of pattern in string:

+
SELECT regexp_count('1a 2b 14m', '\s*[a-z]+\s*'); -- 3
+
+
+
+
+
+regexp_extract_all(string, pattern)#
+

Returns the substring(s) matched by the regular expression pattern +in string:

+
SELECT regexp_extract_all('1a 2b 14m', '\d+'); -- [1, 2, 14]
+
+
+
+
+
+regexp_extract_all(string, pattern, group)
+

Finds all occurrences of the regular expression pattern in string +and returns the capturing group number group:

+
SELECT regexp_extract_all('1a 2b 14m', '(\d+)([a-z]+)', 2); -- ['a', 'b', 'm']
+
+
+
+
+
+regexp_extract(string, pattern) varchar#
+

Returns the first substring matched by the regular expression pattern +in string:

+
SELECT regexp_extract('1a 2b 14m', '\d+'); -- 1
+
+
+
+
+
+regexp_extract(string, pattern, group) varchar
+

Finds the first occurrence of the regular expression pattern in +string and returns the capturing group number group:

+
SELECT regexp_extract('1a 2b 14m', '(\d+)([a-z]+)', 2); -- 'a'
+
+
+
+
+
+regexp_like(string, pattern) boolean#
+

Evaluates the regular expression pattern and determines if it is +contained within string.

+

The pattern only needs to be contained within +string, rather than needing to match all of string. In other words, +this performs a contains operation rather than a match operation. You can +match the entire string by anchoring the pattern using ^ and $:

+
SELECT regexp_like('1a 2b 14m', '\d+b'); -- true
+
+
+
+
+
+regexp_position(string, pattern) integer#
+

Returns the index of the first occurrence (counting from 1) of pattern in string. +Returns -1 if not found:

+
SELECT regexp_position('I have 23 apples, 5 pears and 13 oranges', '\b\d+\b'); -- 8
+
+
+
+
+
+regexp_position(string, pattern, start) integer
+

Returns the index of the first occurrence of pattern in string, +starting from start (include start). Returns -1 if not found:

+
SELECT regexp_position('I have 23 apples, 5 pears and 13 oranges', '\b\d+\b', 5); -- 8
+SELECT regexp_position('I have 23 apples, 5 pears and 13 oranges', '\b\d+\b', 12); -- 19
+
+
+
+
+
+regexp_position(string, pattern, start, occurrence) integer
+

Returns the index of the nth occurrence of pattern in string, +starting from start (include start). Returns -1 if not found:

+
SELECT regexp_position('I have 23 apples, 5 pears and 13 oranges', '\b\d+\b', 12, 1); -- 19
+SELECT regexp_position('I have 23 apples, 5 pears and 13 oranges', '\b\d+\b', 12, 2); -- 31
+SELECT regexp_position('I have 23 apples, 5 pears and 13 oranges', '\b\d+\b', 12, 3); -- -1
+
+
+
+
+
+regexp_replace(string, pattern) varchar#
+

Removes every instance of the substring matched by the regular expression +pattern from string:

+
SELECT regexp_replace('1a 2b 14m', '\d+[ab] '); -- '14m'
+
+
+
+
+
+regexp_replace(string, pattern, replacement) varchar
+

Replaces every instance of the substring matched by the regular expression +pattern in string with replacement. Capturing groups can be +referenced in replacement using $g for a numbered group or +${name} for a named group. A dollar sign ($) may be included in the +replacement by escaping it with a backslash (\$):

+
SELECT regexp_replace('1a 2b 14m', '(\d+)([ab]) ', '3c$2 '); -- '3ca 3cb 14m'
+
+
+
+
+
+regexp_replace(string, pattern, function) varchar
+

Replaces every instance of the substring matched by the regular expression +pattern in string using function. The lambda expression +function is invoked for each match with the capturing groups passed as an +array. Capturing group numbers start at one; there is no group for the entire match +(if you need this, surround the entire expression with parenthesis).

+
SELECT regexp_replace('new york', '(\w)(\w*)', x -> upper(x[1]) || lower(x[2])); --'New York'
+
+
+
+
+
+regexp_split(string, pattern)#
+

Splits string using the regular expression pattern and returns an +array. Trailing empty strings are preserved:

+
SELECT regexp_split('1a 2b 14m', '\s*[a-z]+\s*'); -- [1, 2, 14, ]
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/session.html b/430/functions/session.html new file mode 100644 index 000000000..495bc0ffb --- /dev/null +++ b/430/functions/session.html @@ -0,0 +1,666 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Session information — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Session information#

+

Functions providing information about the query execution environment.

+
+
+current_user#
+

Returns the current user running the query.

+
+
+
+current_groups()#
+

Returns the list of groups for the current user running the query.

+
+
+
+current_catalog#
+

Returns a character string that represents the current catalog name.

+
+
+
+current_schema#
+

Returns a character string that represents the current unqualified schema name.

+
+

Note

+

This is part of the SQL standard and does not use parenthesis.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/setdigest.html b/430/functions/setdigest.html new file mode 100644 index 000000000..4e34b3253 --- /dev/null +++ b/430/functions/setdigest.html @@ -0,0 +1,819 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Set Digest functions — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Set Digest functions#

+

Trino offers several functions that deal with the +MinHash technique.

+

MinHash is used to quickly estimate the +Jaccard similarity coefficient +between two sets.

+

It is commonly used in data mining to detect near-duplicate web pages at scale. +By using this information, the search engines efficiently avoid showing +within the search results two pages that are nearly identical.

+

The following example showcases how the Set Digest functions can be +used to naively estimate the similarity between texts. The input texts +are split by using the function ngrams() to +4-shingles which are +used as input for creating a set digest of each initial text. +The set digests are compared to each other to get an +approximation of the similarity of their corresponding +initial texts:

+
WITH text_input(id, text) AS (
+         VALUES
+             (1, 'The quick brown fox jumps over the lazy dog'),
+             (2, 'The quick and the lazy'),
+             (3, 'The quick brown fox jumps over the dog')
+     ),
+     text_ngrams(id, ngrams) AS (
+         SELECT id,
+                transform(
+                  ngrams(
+                    split(text, ' '),
+                    4
+                  ),
+                  token -> array_join(token, ' ')
+                )
+         FROM text_input
+     ),
+     minhash_digest(id, digest) AS (
+         SELECT id,
+                (SELECT make_set_digest(v) FROM unnest(ngrams) u(v))
+         FROM text_ngrams
+     ),
+     setdigest_side_by_side(id1, digest1, id2, digest2) AS (
+         SELECT m1.id as id1,
+                m1.digest as digest1,
+                m2.id as id2,
+                m2.digest as digest2
+         FROM (SELECT id, digest FROM minhash_digest) m1
+         JOIN (SELECT id, digest FROM minhash_digest) m2
+           ON m1.id != m2.id AND m1.id < m2.id
+     )
+SELECT id1,
+       id2,
+       intersection_cardinality(digest1, digest2) AS intersection_cardinality,
+       jaccard_index(digest1, digest2)            AS jaccard_index
+FROM setdigest_side_by_side
+ORDER BY id1, id2;
+
+
+
 id1 | id2 | intersection_cardinality | jaccard_index
+-----+-----+--------------------------+---------------
+   1 |   2 |                        0 |           0.0
+   1 |   3 |                        4 |           0.6
+   2 |   3 |                        0 |           0.0
+
+
+

The above result listing points out, as expected, that the texts +with the id 1 and 3 are quite similar.

+

One may argue that the text with the id 2 is somewhat similar to +the texts with the id 1 and 3. Due to the fact in the example above +4-shingles are taken into account for measuring the similarity of the texts, +there are no intersections found for the text pairs 1 and 2, respectively +3 and 2 and therefore there the similarity index for these text pairs +is 0.

+
+

Data structures#

+

Trino implements Set Digest data sketches by encapsulating the following components:

+ +

The HyperLogLog structure is used for the approximation of the distinct elements +in the original set.

+

The MinHash structure is used to store a low memory footprint signature of the original set. +The similarity of any two sets is estimated by comparing their signatures.

+

The Trino type for this data structure is called setdigest. +Trino offers the ability to merge multiple Set Digest data sketches.

+
+
+

Serialization#

+

Data sketches can be serialized to and deserialized from varbinary. This +allows them to be stored for later use.

+
+
+

Functions#

+
+
+make_set_digest(x) setdigest#
+

Composes all input values of x into a setdigest.

+

Create a setdigest corresponding to a bigint array:

+
SELECT make_set_digest(value)
+FROM (VALUES 1, 2, 3) T(value);
+
+
+

Create a setdigest corresponding to a varchar array:

+
SELECT make_set_digest(value)
+FROM (VALUES 'Trino', 'SQL', 'on', 'everything') T(value);
+
+
+
+
+
+merge_set_digest(setdigest) setdigest#
+

Returns the setdigest of the aggregate union of the individual setdigest +Set Digest structures.

+
+
+
+cardinality(setdigest) long
+

Returns the cardinality of the set digest from its internal +HyperLogLog component.

+

Examples:

+
SELECT cardinality(make_set_digest(value))
+FROM (VALUES 1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5) T(value);
+-- 5
+
+
+
+
+
+intersection_cardinality(x, y) long#
+

Returns the estimation for the cardinality of the intersection of the two set digests.

+

x and y must be of type setdigest

+

Examples:

+
SELECT intersection_cardinality(make_set_digest(v1), make_set_digest(v2))
+FROM (VALUES (1, 1), (NULL, 2), (2, 3), (3, 4)) T(v1, v2);
+-- 3
+
+
+
+
+
+jaccard_index(x, y) double#
+

Returns the estimation of Jaccard index for +the two set digests.

+

x and y must be of type setdigest.

+

Examples:

+
SELECT jaccard_index(make_set_digest(v1), make_set_digest(v2))
+FROM (VALUES (1, 1), (NULL,2), (2, 3), (NULL, 4)) T(v1, v2);
+-- 0.5
+
+
+
+
+
+hash_counts(x)#
+

Returns a map containing the Murmur3Hash128 +hashed values and the count of their occurences within +the internal MinHash structure belonging to x.

+

x must be of type setdigest.

+

Examples:

+
SELECT hash_counts(make_set_digest(value))
+FROM (VALUES 1, 1, 1, 2, 2) T(value);
+-- {19144387141682250=3, -2447670524089286488=2}
+
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/string.html b/430/functions/string.html new file mode 100644 index 000000000..47c1ac7cd --- /dev/null +++ b/430/functions/string.html @@ -0,0 +1,1021 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + String functions and operators — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

String functions and operators#

+
+

String operators#

+

The || operator performs concatenation.

+

The LIKE statement can be used for pattern matching and is documented in +Pattern comparison: LIKE.

+
+
+

String functions#

+
+

Note

+

These functions assume that the input strings contain valid UTF-8 encoded +Unicode code points. There are no explicit checks for valid UTF-8 and +the functions may return incorrect results on invalid UTF-8. +Invalid UTF-8 data can be corrected with from_utf8().

+

Additionally, the functions operate on Unicode code points and not user +visible characters (or grapheme clusters). Some languages combine +multiple code points into a single user-perceived character, the basic +unit of a writing system for a language, but the functions will treat each +code point as a separate unit.

+

The lower() and upper() functions do not perform +locale-sensitive, context-sensitive, or one-to-many mappings required for +some languages. Specifically, this will return incorrect results for +Lithuanian, Turkish and Azeri.

+
+
+
+chr(n) varchar#
+

Returns the Unicode code point n as a single character string.

+
+
+
+codepoint(string) integer#
+

Returns the Unicode code point of the only character of string.

+
+
+
+concat(string1, ..., stringN) varchar#
+

Returns the concatenation of string1, string2, ..., stringN. +This function provides the same functionality as the +SQL-standard concatenation operator (||).

+
+
+
+concat_ws(string0, string1, ..., stringN) varchar#
+

Returns the concatenation of string1, string2, ..., stringN +using string0 as a separator. If string0 is null, then the return +value is null. Any null values provided in the arguments after the +separator are skipped.

+
+
+
+concat_ws(string0, array(varchar)) varchar
+

Returns the concatenation of elements in the array using string0 as a +separator. If string0 is null, then the return value is null. Any +null values in the array are skipped.

+
+
+
+format(format, args...) varchar
+

See format().

+
+
+
+hamming_distance(string1, string2) bigint#
+

Returns the Hamming distance of string1 and string2, +i.e. the number of positions at which the corresponding characters are different. +Note that the two strings must have the same length.

+
+
+
+length(string) bigint#
+

Returns the length of string in characters.

+
+
+
+levenshtein_distance(string1, string2) bigint#
+

Returns the Levenshtein edit distance of string1 and string2, +i.e. the minimum number of single-character edits (insertions, +deletions or substitutions) needed to change string1 into string2.

+
+
+
+lower(string) varchar#
+

Converts string to lowercase.

+
+
+
+lpad(string, size, padstring) varchar#
+

Left pads string to size characters with padstring. +If size is less than the length of string, the result is +truncated to size characters. size must not be negative +and padstring must be non-empty.

+
+
+
+ltrim(string) varchar#
+

Removes leading whitespace from string.

+
+
+
+luhn_check(string) boolean#
+

Tests whether a string of digits is valid according to the +Luhn algorithm.

+

This checksum function, also known as modulo 10 or mod 10, is +widely applied on credit card numbers and government identification numbers +to distinguish valid numbers from mistyped, incorrect numbers.

+

Valid identification number:

+
select luhn_check('79927398713');
+-- true
+
+
+

Invalid identification number:

+
select luhn_check('79927398714');
+-- false
+
+
+
+
+
+position(substring IN string) bigint#
+

Returns the starting position of the first instance of substring in +string. Positions start with 1. If not found, 0 is returned.

+
+

Note

+

This SQL-standard function has special syntax and uses the +IN keyword for the arguments. See also strpos().

+
+
+
+
+replace(string, search) varchar#
+

Removes all instances of search from string.

+
+
+
+replace(string, search, replace) varchar
+

Replaces all instances of search with replace in string.

+
+
+
+reverse(string) varchar#
+

Returns string with the characters in reverse order.

+
+
+
+rpad(string, size, padstring) varchar#
+

Right pads string to size characters with padstring. +If size is less than the length of string, the result is +truncated to size characters. size must not be negative +and padstring must be non-empty.

+
+
+
+rtrim(string) varchar#
+

Removes trailing whitespace from string.

+
+
+
+soundex(char) string#
+
+
soundex returns a character string containing the phonetic representation of char.

It is typically used to evaluate the similarity of two expressions phonetically, that is +how the string sounds when spoken:

+
SELECT name
+FROM nation
+WHERE SOUNDEX(name)  = SOUNDEX('CHYNA');
+
+ name  |
+-------+----
+ CHINA |
+(1 row)
+
+
+
+
+
+
+
+split(string, delimiter)#
+

Splits string on delimiter and returns an array.

+
+
+
+split(string, delimiter, limit)
+

Splits string on delimiter and returns an array of size at most +limit. The last element in the array always contain everything +left in the string. limit must be a positive number.

+
+
+
+split_part(string, delimiter, index) varchar#
+

Splits string on delimiter and returns the field index. +Field indexes start with 1. If the index is larger than +the number of fields, then null is returned.

+
+
+
+split_to_map(string, entryDelimiter, keyValueDelimiter) map<varchar, varchar>#
+

Splits string by entryDelimiter and keyValueDelimiter and returns a map. +entryDelimiter splits string into key-value pairs. keyValueDelimiter splits +each pair into key and value.

+
+
+
+split_to_multimap(string, entryDelimiter, keyValueDelimiter)#
+

Splits string by entryDelimiter and keyValueDelimiter and returns a map +containing an array of values for each unique key. entryDelimiter splits string +into key-value pairs. keyValueDelimiter splits each pair into key and value. The +values for each key will be in the same order as they appeared in string.

+
+
+
+strpos(string, substring) bigint#
+

Returns the starting position of the first instance of substring in +string. Positions start with 1. If not found, 0 is returned.

+
+
+
+strpos(string, substring, instance) bigint
+

Returns the position of the N-th instance of substring in string. +When instance is a negative number the search will start from the end of string. +Positions start with 1. If not found, 0 is returned.

+
+
+
+starts_with(string, substring) boolean#
+

Tests whether substring is a prefix of string.

+
+
+
+substr(string, start) varchar#
+

This is an alias for substring().

+
+
+
+substring(string, start) varchar#
+

Returns the rest of string from the starting position start. +Positions start with 1. A negative starting position is interpreted +as being relative to the end of the string.

+
+
+
+substr(string, start, length) varchar
+

This is an alias for substring().

+
+
+
+substring(string, start, length) varchar
+

Returns a substring from string of length length from the starting +position start. Positions start with 1. A negative starting +position is interpreted as being relative to the end of the string.

+
+
+
+translate(source, from, to) varchar#
+

Returns the source string translated by replacing characters found in the +from string with the corresponding characters in the to string. If the from +string contains duplicates, only the first is used. If the source character +does not exist in the from string, the source character will be copied +without translation. If the index of the matching character in the from +string is beyond the length of the to string, the source character will +be omitted from the resulting string.

+

Here are some examples illustrating the translate function:

+
SELECT translate('abcd', '', ''); -- 'abcd'
+SELECT translate('abcd', 'a', 'z'); -- 'zbcd'
+SELECT translate('abcda', 'a', 'z'); -- 'zbcdz'
+SELECT translate('Palhoça', 'ç','c'); -- 'Palhoca'
+SELECT translate('abcd', 'b', U&'\+01F600'); -- a😀cd
+SELECT translate('abcd', 'a', ''); -- 'bcd'
+SELECT translate('abcd', 'a', 'zy'); -- 'zbcd'
+SELECT translate('abcd', 'ac', 'z'); -- 'zbd'
+SELECT translate('abcd', 'aac', 'zq'); -- 'zbd'
+
+
+
+
+
+trim(string) varchar
+

Removes leading and trailing whitespace from string.

+
+
+
+trim([ [ specification ] [ string ] FROM ] source ) varchar#
+

Removes any leading and/or trailing characters as specified up to and +including string from source:

+
SELECT trim('!' FROM '!foo!'); -- 'foo'
+SELECT trim(LEADING FROM '  abcd');  -- 'abcd'
+SELECT trim(BOTH '$' FROM '$var$'); -- 'var'
+SELECT trim(TRAILING 'ER' FROM upper('worker')); -- 'WORK'
+
+
+
+
+
+upper(string) varchar#
+

Converts string to uppercase.

+
+
+
+word_stem(word) varchar#
+

Returns the stem of word in the English language.

+
+
+
+word_stem(word, lang) varchar
+

Returns the stem of word in the lang language.

+
+
+
+

Unicode functions#

+
+
+normalize(string) varchar#
+

Transforms string with NFC normalization form.

+
+
+
+normalize(string, form) varchar
+

Transforms string with the specified normalization form. +form must be one of the following keywords:

+ + + + + + + + + + + + + + + + + + + + +

Form

Description

NFD

Canonical Decomposition

NFC

Canonical Decomposition, followed by Canonical Composition

NFKD

Compatibility Decomposition

NFKC

Compatibility Decomposition, followed by Canonical Composition

+
+

Note

+

This SQL-standard function has special syntax and requires +specifying form as a keyword, not as a string.

+
+
+
+
+to_utf8(string) varbinary#
+

Encodes string into a UTF-8 varbinary representation.

+
+
+
+from_utf8(binary) varchar#
+

Decodes a UTF-8 encoded string from binary. Invalid UTF-8 sequences +are replaced with the Unicode replacement character U+FFFD.

+
+
+
+from_utf8(binary, replace) varchar
+

Decodes a UTF-8 encoded string from binary. Invalid UTF-8 sequences +are replaced with replace. The replacement string replace must either +be a single character or empty (in which case invalid characters are +removed).

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/system.html b/430/functions/system.html new file mode 100644 index 000000000..99ee3a0c9 --- /dev/null +++ b/430/functions/system.html @@ -0,0 +1,650 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + System information — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

System information#

+

Functions providing information about the Trino cluster system environment. More +information is available by querying the various schemas and tables exposed by +the System connector.

+
+
+version() varchar#
+

Returns the Trino version used on the cluster. Equivalent to the value of +the node_version column in the system.runtime.nodes table.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/table.html b/430/functions/table.html new file mode 100644 index 000000000..7f02d0e98 --- /dev/null +++ b/430/functions/table.html @@ -0,0 +1,826 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Table functions — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Table functions#

+

A table function is a function returning a table. It can be invoked inside the +FROM clause of a query:

+
SELECT * FROM TABLE(my_function(1, 100))
+
+
+

The row type of the returned table can depend on the arguments passed with +invocation of the function. If different row types can be returned, the +function is a polymorphic table function.

+

Polymorphic table functions allow you to dynamically invoke custom logic from +within the SQL query. They can be used for working with external systems as +well as for enhancing Trino with capabilities going beyond the SQL standard.

+

For the list of built-in table functions available in Trino, see built in table functions.

+

Trino supports adding custom table functions. They are declared by connectors +through implementing dedicated interfaces. For guidance on adding new table +functions, see the developer guide.

+

Connectors offer support for different functions on a per-connector basis. For +more information about supported table functions, refer to the connector documentation.

+
+

Built-in table functions#

+
+
+exclude_columns(input => table, columns => descriptor) table#
+

Excludes from table all columns listed in descriptor:

+
SELECT *
+FROM TABLE(exclude_columns(
+                        input => TABLE(orders),
+                        columns => DESCRIPTOR(clerk, comment)))
+
+
+

The argument input is a table or a query. +The argument columns is a descriptor without types.

+
+
+
+sequence(start => bigint, stop => bigint, step => bigint) -> table(sequential_number bigint)
+

Returns a single column sequential_number containing a sequence of +bigint:

+
SELECT *
+FROM TABLE(sequence(
+                start => 1000000,
+                stop => -2000000,
+                step => -3))
+
+
+

start is the first element in te sequence. The default value is 0.

+

stop is the end of the range, inclusive. The last element in the +sequence is equal to stop, or it is the last value within range, +reachable by steps.

+

step is the difference between subsequent values. The default value is +1.

+
+
+

Note

+

The result of the sequence table function might not be ordered.

+
+
+
+

Table function invocation#

+

You invoke a table function in the FROM clause of a query. Table function +invocation syntax is similar to a scalar function call.

+
+

Function resolution#

+

Every table function is provided by a catalog, and it belongs to a schema in +the catalog. You can qualify the function name with a schema name, or with +catalog and schema names:

+
SELECT * FROM TABLE(schema_name.my_function(1, 100))
+SELECT * FROM TABLE(catalog_name.schema_name.my_function(1, 100))
+
+
+

Otherwise, the standard Trino name resolution is applied. The connection +between the function and the catalog must be identified, because the function +is executed by the corresponding connector. If the function is not registered +by the specified catalog, the query fails.

+

The table function name is resolved case-insensitive, analogically to scalar +function and table resolution in Trino.

+
+
+

Arguments#

+

There are three types of arguments.

+
    +
  1. Scalar arguments

  2. +
+

They must be constant expressions, and they can be of any SQL type, which is +compatible with the declared argument type:

+
factor => 42
+
+
+
    +
  1. Descriptor arguments

  2. +
+

Descriptors consist of fields with names and optional data types:

+
schema => DESCRIPTOR(id BIGINT, name VARCHAR)
+columns => DESCRIPTOR(date, status, comment)
+
+
+

To pass null for a descriptor, use:

+
schema => CAST(null AS DESCRIPTOR)
+
+
+
    +
  1. Table arguments

  2. +
+

You can pass a table name, or a query. Use the keyword TABLE:

+
input => TABLE(orders)
+data => TABLE(SELECT * FROM region, nation WHERE region.regionkey = nation.regionkey)
+
+
+

If the table argument is declared as set semantics, +you can specify partitioning and ordering. Each partition is processed +independently by the table function. If you do not specify partitioning, the +argument is processed as a single partition. You can also specify +PRUNE WHEN EMPTY or KEEP WHEN EMPTY. With PRUNE WHEN EMPTY you +declare that you are not interested in the function result if the argument is +empty. This information is used by the Trino engine to optimize the query. The +KEEP WHEN EMPTY option indicates that the function should be executed even +if the table argument is empty. Note that by specifying KEEP WHEN EMPTY or +PRUNE WHEN EMPTY, you override the property set for the argument by the +function author.

+

The following example shows how the table argument properties should be ordered:

+
input => TABLE(orders)
+                    PARTITION BY orderstatus
+                    KEEP WHEN EMPTY
+                    ORDER BY orderdate
+
+
+
+
+

Argument passing conventions#

+

There are two conventions of passing arguments to a table function:

+
    +
  • Arguments passed by name:

    +
    SELECT * FROM TABLE(my_function(row_count => 100, column_count => 1))
    +
    +
    +
  • +
+

In this convention, you can pass the arguments in arbitrary order. Arguments +declared with default values can be skipped. Argument names are resolved +case-sensitive, and with automatic uppercasing of unquoted names.

+
    +
  • Arguments passed positionally:

    +
    SELECT * FROM TABLE(my_function(1, 100))
    +
    +
    +
  • +
+

In this convention, you must follow the order in which the arguments are +declared. You can skip a suffix of the argument list, provided that all the +skipped arguments are declared with default values.

+

You cannot mix the argument conventions in one invocation.

+

You can also use parameters in arguments:

+
PREPARE stmt FROM
+SELECT * FROM TABLE(my_function(row_count => ? + 1, column_count => ?));
+
+EXECUTE stmt USING 100, 1;
+
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/tdigest.html b/430/functions/tdigest.html new file mode 100644 index 000000000..0fe4ae5c9 --- /dev/null +++ b/430/functions/tdigest.html @@ -0,0 +1,692 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + T-Digest functions — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

T-Digest functions#

+
+

Data structures#

+

A T-digest is a data sketch which stores approximate percentile +information. The Trino type for this data structure is called tdigest. +T-digests can be merged, and for storage and retrieval they can be cast +to and from VARBINARY.

+
+
+

Functions#

+
+
+merge(tdigest) tdigest
+

Aggregates all inputs into a single tdigest.

+
+
+
+value_at_quantile(tdigest, quantile) double
+

Returns the approximate percentile value from the T-digest, given +the number quantile between 0 and 1.

+
+
+
+values_at_quantiles(tdigest, quantiles)
+

Returns the approximate percentile values as an array, given the input +T-digest and an array of values between 0 and 1, which +represent the quantiles to return.

+
+
+
+tdigest_agg(x) tdigest#
+

Composes all input values of x into a tdigest. x can be +of any numeric type.

+
+
+
+tdigest_agg(x, w) tdigest
+

Composes all input values of x into a tdigest using +the per-item weight w. w must be greater or equal than 1. +x and w can be of any numeric type.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/teradata.html b/430/functions/teradata.html new file mode 100644 index 000000000..96fa03824 --- /dev/null +++ b/430/functions/teradata.html @@ -0,0 +1,726 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Teradata functions — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Teradata functions#

+

These functions provide compatibility with Teradata SQL.

+
+

String functions#

+
+
+char2hexint(string) varchar#
+

Returns the hexadecimal representation of the UTF-16BE encoding of the string.

+
+
+
+index(string, substring) bigint#
+

Alias for strpos() function.

+
+
+
+

Date functions#

+

The functions in this section use a format string that is compatible with +the Teradata datetime functions. The following table, based on the +Teradata reference manual, describes the supported format specifiers:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Specifier

Description

- / , . ; :

Punctuation characters are ignored

dd

Day of month (1-31)

hh

Hour of day (1-12)

hh24

Hour of the day (0-23)

mi

Minute (0-59)

mm

Month (01-12)

ss

Second (0-59)

yyyy

4-digit year

yy

2-digit year

+
+

Warning

+

Case insensitivity is not currently supported. All specifiers must be lowercase.

+
+
+
+to_char(timestamp, format) varchar#
+

Formats timestamp as a string using format.

+
+
+
+to_timestamp(string, format) timestamp#
+

Parses string into a TIMESTAMP using format.

+
+
+
+to_date(string, format) date#
+

Parses string into a DATE using format.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/url.html b/430/functions/url.html new file mode 100644 index 000000000..5ce93e6b0 --- /dev/null +++ b/430/functions/url.html @@ -0,0 +1,732 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + URL functions — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

URL functions#

+
+

Extraction functions#

+

The URL extraction functions extract components from HTTP URLs +(or any valid URIs conforming to RFC 2396). +The following syntax is supported:

+
[protocol:][//host[:port]][path][?query][#fragment]
+
+
+

The extracted components do not contain URI syntax separators +such as : or ?.

+
+
+url_extract_fragment(url) varchar#
+

Returns the fragment identifier from url.

+
+
+
+url_extract_host(url) varchar#
+

Returns the host from url.

+
+
+
+url_extract_parameter(url, name) varchar#
+

Returns the value of the first query string parameter named name +from url. Parameter extraction is handled in the typical manner +as specified by RFC 1866#section-8.2.1.

+
+
+
+url_extract_path(url) varchar#
+

Returns the path from url.

+
+
+
+url_extract_port(url) bigint#
+

Returns the port number from url.

+
+
+
+url_extract_protocol(url) varchar#
+

Returns the protocol from url:

+
SELECT url_extract_protocol('http://localhost:8080/req_path');
+-- http
+
+SELECT url_extract_protocol('https://127.0.0.1:8080/req_path');
+-- https
+
+SELECT url_extract_protocol('ftp://path/file');
+-- ftp
+
+
+
+
+
+url_extract_query(url) varchar#
+

Returns the query string from url.

+
+
+
+

Encoding functions#

+
+
+url_encode(value) varchar#
+

Escapes value by encoding it so that it can be safely included in +URL query parameter names and values:

+
    +
  • Alphanumeric characters are not encoded.

  • +
  • The characters ., -, * and _ are not encoded.

  • +
  • The ASCII space character is encoded as +.

  • +
  • All other characters are converted to UTF-8 and the bytes are encoded +as the string %XX where XX is the uppercase hexadecimal +value of the UTF-8 byte.

  • +
+
+
+
+url_decode(value) varchar#
+

Unescapes the URL encoded value. +This function is the inverse of url_encode().

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/uuid.html b/430/functions/uuid.html new file mode 100644 index 000000000..eb468906b --- /dev/null +++ b/430/functions/uuid.html @@ -0,0 +1,646 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + UUID functions — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

UUID functions#

+
+
+uuid() uuid#
+

Returns a pseudo randomly generated UUID (type 4).

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/functions/window.html b/430/functions/window.html new file mode 100644 index 000000000..4680445bf --- /dev/null +++ b/430/functions/window.html @@ -0,0 +1,788 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Window functions — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Window functions#

+

Window functions perform calculations across rows of the query result. +They run after the HAVING clause but before the ORDER BY clause. +Invoking a window function requires special syntax using the OVER +clause to specify the window. +For example, the following query ranks orders for each clerk by price:

+
SELECT orderkey, clerk, totalprice,
+       rank() OVER (PARTITION BY clerk
+                    ORDER BY totalprice DESC) AS rnk
+FROM orders
+ORDER BY clerk, rnk
+
+
+

The window can be specified in two ways (see WINDOW clause):

+
    +
  • By a reference to a named window specification defined in the WINDOW clause,

  • +
  • By an in-line window specification which allows to define window components +as well as refer to the window components pre-defined in the WINDOW clause.

  • +
+
+

Aggregate functions#

+

All Aggregate functions can be used as window functions by adding the OVER +clause. The aggregate function is computed for each row over the rows within +the current row’s window frame.

+

For example, the following query produces a rolling sum of order prices +by day for each clerk:

+
SELECT clerk, orderdate, orderkey, totalprice,
+       sum(totalprice) OVER (PARTITION BY clerk
+                             ORDER BY orderdate) AS rolling_sum
+FROM orders
+ORDER BY clerk, orderdate, orderkey
+
+
+
+
+

Ranking functions#

+
+
+cume_dist() bigint#
+

Returns the cumulative distribution of a value in a group of values. +The result is the number of rows preceding or peer with the row in the +window ordering of the window partition divided by the total number of +rows in the window partition. Thus, any tie values in the ordering will +evaluate to the same distribution value.

+
+
+
+dense_rank() bigint#
+

Returns the rank of a value in a group of values. This is similar to +rank(), except that tie values do not produce gaps in the sequence.

+
+
+
+ntile(n) bigint#
+

Divides the rows for each window partition into n buckets ranging +from 1 to at most n. Bucket values will differ by at most 1. +If the number of rows in the partition does not divide evenly into the +number of buckets, then the remainder values are distributed one per +bucket, starting with the first bucket.

+

For example, with 6 rows and 4 buckets, the bucket values would +be as follows: 1 1 2 2 3 4

+
+
+
+percent_rank() double#
+

Returns the percentage ranking of a value in group of values. The result +is (r - 1) / (n - 1) where r is the rank() of the row and +n is the total number of rows in the window partition.

+
+
+
+rank() bigint#
+

Returns the rank of a value in a group of values. The rank is one plus +the number of rows preceding the row that are not peer with the row. +Thus, tie values in the ordering will produce gaps in the sequence. +The ranking is performed for each window partition.

+
+
+
+row_number() bigint#
+

Returns a unique, sequential number for each row, starting with one, +according to the ordering of rows within the window partition.

+
+
+
+

Value functions#

+

By default, null values are respected. If IGNORE NULLS is specified, all rows where +x is null are excluded from the calculation. If IGNORE NULLS is specified and x +is null for all rows, the default_value is returned, or if it is not specified, +null is returned.

+
+
+first_value(x) [same as input]#
+

Returns the first value of the window.

+
+
+
+last_value(x) [same as input]#
+

Returns the last value of the window.

+
+
+
+nth_value(x, offset) [same as input]#
+

Returns the value at the specified offset from the beginning of the window. +Offsets start at 1. The offset can be any scalar +expression. If the offset is null or greater than the number of values in +the window, null is returned. It is an error for the offset to be zero or +negative.

+
+
+
+lead(x[, offset[, default_value]]) [same as input]#
+

Returns the value at offset rows after the current row in the window partition. +Offsets start at 0, which is the current row. The +offset can be any scalar expression. The default offset is 1. If the +offset is null, null is returned. If the offset refers to a row that is not +within the partition, the default_value is returned, or if it is not specified +null is returned. +The lead() function requires that the window ordering be specified. +Window frame must not be specified.

+
+
+
+lag(x[, offset[, default_value]]) [same as input]#
+

Returns the value at offset rows before the current row in the window partition. +Offsets start at 0, which is the current row. The +offset can be any scalar expression. The default offset is 1. If the +offset is null, null is returned. If the offset refers to a row that is not +within the partition, the default_value is returned, or if it is not specified +null is returned. +The lag() function requires that the window ordering be specified. +Window frame must not be specified.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/genindex.html b/430/genindex.html new file mode 100644 index 000000000..5827c1a7a --- /dev/null +++ b/430/genindex.html @@ -0,0 +1,4195 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Index — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ + +

Index

+ +
+ A + | B + | C + | D + | E + | F + | G + | H + | I + | J + | K + | L + | M + | N + | O + | P + | Q + | R + | S + | T + | U + | V + | W + | X + | Y + | Z + +
+

A

+ + + +
+ +

B

+ + +
+ +

C

+ + + +
+ +

D

+ + + +
+ +

E

+ + + +
+ +

F

+ + + +
+ +

G

+ + + +
+ +

H

+ + + +
+ +

I

+ + + +
+ +

J

+ + + +
+ +

K

+ + +
+ +

L

+ + + +
+ +

M

+ + + +
+ +

N

+ + + +
+ +

O

+ + +
+ +

P

+ + + +
+ +

Q

+ + + +
+ +

R

+ + + +
+ +

S

+ + + +
+ +

T

+ + + +
+ +

U

+ + + +
+ +

V

+ + + +
+ +

W

+ + + +
+ +

X

+ + +
+ +

Y

+ + + +
+ +

Z

+ + + +
+ + + +
+
+
+
+
+
+ + +
+ + + + \ No newline at end of file diff --git a/430/glossary.html b/430/glossary.html new file mode 100644 index 000000000..2d1da0d71 --- /dev/null +++ b/430/glossary.html @@ -0,0 +1,564 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Glossary — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Glossary#

+

The glossary contains a list of key Trino terms and definitions.

+
+
Catalog

Catalogs define and name a configuration for connecting to a data source, +allowing users to query the connected data. Each catalog’s configuration +specifies a connector to define which data source +the catalog connects to. For more information about catalogs, see +Catalog.

+
+
+
+
Certificate

A public key certificate issued by a CA, sometimes abbreviated as cert, that verifies the ownership of a +server’s private keys. Certificate format is specified in the X.509 standard.

+
+
+
+
Certificate Authority (CA)

A trusted organization that signs and issues certificates. Its signatures +can be used to verify the validity of certificates.

+
+
Cluster

A Trino cluster provides the resources to run queries against numerous data +sources. Clusters define the number of nodes, the configuration for the JVM +runtime, configured data sources, and others aspects. For more information, +see Cluster.

+
+
+
+
Connector

Translates data from a data source into Trino schemas, tables, columns, +rows, and data types. A connector is specific to a data +source, and is used in catalog configurations to +define what data source the catalog connects to. A connector is one of many +types of plugins

+
+
Container

A lightweight virtual package of software that contains libraries, binaries, +code, configuration files, and other dependencies needed to deploy an +application. A running container does not include an operating system, +instead using the operating system of the host machine. To learn more, read +read about containers +in the Kubernetes documentation.

+
+
+
+
Data source

A system from which data is retrieved - for example, PostgreSQL or Iceberg +on S3 data. In Trino, users query data sources with catalogs that connect to each source. See +Data sources for more information.

+
+
+
+
Data virtualization

Data virtualization is a +method of abstracting an interaction with multiple heterogeneous data sources, without needing to know the distributed nature +of the data, its format, or any other technical details involved in +presenting the data.

+
+
+
+
gzip

gzip is a compression format and +software that compresses and decompresses files. This format is used several +ways in Trino, including deployment and compressing files in object storage. The most common extension for gzip-compressed +files is .gz.

+
+
+
+
HDFS

Hadoop Distributed Filesystem (HDFS) is a scalable open source filesystem that was one of the earliest +distributed big data systems created to store large amounts of data for the +Hadoop ecosystem.

+
+
+
+
Java KeyStore (JKS)

The system of public key cryptography supported as one part of the Java +security APIs. The legacy JKS system recognizes keys and certificates stored in keystore files, typically with the .jks +extension, and by default relies on a system-level list of CAs in truststore files installed as part of the current Java +installation.

+
+
Key

A cryptographic key specified as a pair of public and private strings +generally used in the context of TLS to secure public +network traffic.

+
+
+
+
Load Balancer (LB)

Software or a hardware device that sits on a network edge and accepts +network connections on behalf of servers behind that wall, distributing +traffic across network and server infrastructure to balance the load on +networked services.

+
+
+
+
Object storage

Object storage is a file +storage mechanism. Examples of compatible object stores include the +following:

+ +
+
+
+
Open-source

Typically refers to open-source software. which is software that +has the source code made available for others to see, use, and contribute +to. Allowed usage varies depending on the license that the software is +licensed under. Trino is licensed under the Apache license, and is therefore maintained +by a community of contributors from all across the globe.

+
+
+
+
PEM file format

A format for storing and sending cryptographic keys and certificates. PEM +format can contain both a key and its certificate, plus the chain of +certificates from authorities back to the root CA, or back +to a CA vendor’s intermediate CA.

+
+
+
+
PKCS #12

A binary archive used to store keys and certificates or certificate chains +that validate a key. PKCS #12 files +have .p12 or .pfx extensions. This format is a less popular +alternative to PEM.

+
+
+
+
Plugin

A bundle of code implementing the Trino Service Provider Interface (SPI) that is used to add new connectors, data types, Functions and operators, +access control implementations, and +other features of Trino.

+
+
Presto and PrestoSQL

The old name for Trino. To learn more about the name change to Trino, read +the history.

+
+
Query federation

A type of data virtualization that provides a +common access point and data model across two or more heterogeneous data +sources. A popular data model used by many query federation engines is +translating different data sources to SQL tables.

+
+
+
+
Secure Sockets Layer (SSL)

Now superseded by TLS, but still recognized as the term +for what TLS does.

+
+
+
+
Structured Query Language (SQL)

The standard language used with relational databases. For more information, +see SQL.

+
+
Tarball

A common abbreviation for TAR file, which is a common software +distribution mechanism. This file format is a collection of multiple files +distributed as a single file, commonly compressed using gzip compression.

+
+
+
+
Transport Layer Security (TLS)

TLS is a security +protocol designed to provide secure communications over a network. It is the +successor to SSL, and used in many applications like +HTTPS, email, and Trino. These security topics use the term TLS to refer to +both TLS and SSL.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/index.html b/430/index.html new file mode 100644 index 000000000..dc7a74d80 --- /dev/null +++ b/430/index.html @@ -0,0 +1,650 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Trino documentation — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Trino documentation#

+
+ +
+ +
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/installation.html b/430/installation.html new file mode 100644 index 000000000..740ab8a93 --- /dev/null +++ b/430/installation.html @@ -0,0 +1,460 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Installation — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Installation#

+

A Trino server can be installed and deployed on a number of different +platforms. Typically you run a cluster of machines with one coordinator and many +workers. You can find instructions for deploying such a cluster, and related +information, in the following sections:

+ +

Once you have a completed the deployment, or if you have access to a running +cluster already, you can proceed to configure your client application.

+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/installation/containers.html b/430/installation/containers.html new file mode 100644 index 000000000..28cbece9c --- /dev/null +++ b/430/installation/containers.html @@ -0,0 +1,544 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Trino in a Docker container — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Trino in a Docker container#

+

The Trino project provides the trinodb/trino +Docker image that includes the Trino server and a default configuration. The +Docker image is published to Docker Hub and can be used with the Docker runtime, +among several others.

+
+

Running the container#

+

To run Trino in Docker, you must have the Docker engine installed on your +machine. You can download Docker from the Docker website, +or use the packaging system of your operating systems.

+

Use the docker command to create a container from the trinodb/trino +image. Assign it the trino name, to make it easier to reference it later. +Run it in the background, and map the default Trino port, which is 8080, +from inside the container to port 8080 on your workstation.

+
docker run --name trino -d -p 8080:8080 trinodb/trino
+
+
+

Without specifying the container image tag, it defaults to latest, +but a number of any released Trino version can be used, for example +trinodb/trino:430.

+

Run docker ps to see all the containers running in the background.

+
% docker ps
+CONTAINER ID   IMAGE               COMMAND                  CREATED        STATUS                  PORTS                    NAMES
+955c3b3d3d0a   trinodb/trino:390   "/usr/lib/trino/bin/…"   39 hours ago   Up 39 hours (healthy)   0.0.0.0:8080->8080/tcp   trino
+
+
+

When Trino is still starting, it shows (health: starting), +and (healthy) when it’s ready.

+
+

Note

+

There are multiple ways to use Trino within containers. You can either run +Trino in Docker containers locally, as explained in the following sections, +or use a container orchestration platform like Kubernetes. For the Kubernetes +instructions see Trino on Kubernetes with Helm.

+
+
+
+

Executing queries#

+

The image includes the Trino command-line interface (CLI) client, trino. +Execute it in the existing container to connect to the Trino server running +inside it. After starting the client, type and execute a query on a table +of the tpch catalog, which includes example data:

+
$ docker exec -it trino trino
+trino> select count(*) from tpch.sf1.nation;
+ _col0
+-------
+    25
+(1 row)
+
+Query 20181105_001601_00002_e6r6y, FINISHED, 1 node
+Splits: 21 total, 21 done (100.00%)
+0:06 [25 rows, 0B] [4 rows/s, 0B/s]
+
+
+

Once you are done with your exploration, enter the quit command.

+

Alternatively, you can use the Trino CLI installed directly on your workstation. +The default server URL in the CLI of http://localhost:8080 matches the port used +in the command to start the container. More information about using the CLI can +be found in Command line interface. You can also connect with any other client +application using the JDBC driver.

+
+
+

Configuring Trino#

+

The image already contains a default configuration to get started, and some +catalogs to allow you to explore Trino. You can also use the container with your +custom configuration files in a local etc directory structure as created in +the Deploying Trino. If you mount this directory as a volume +in the path /etc/trino when starting the container, your configuration +is used instead of the default in the image.

+
$ docker run --name trino -d -p 8080:8080 --volume $PWD/etc:/etc/trino trinodb/trino
+
+
+

To keep the default configuration and only configure catalogs, mount a folder +at /etc/trino/catalog, or individual catalog property files in it.

+

If you want to use additional plugins, mount them at /usr/lib/trino/plugin.

+
+
+

Cleaning up#

+

You can stop and start the container, using the docker stop trino and +docker start trino commands. To fully remove the stopped container, run +docker rm trino.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/installation/deployment.html b/430/installation/deployment.html new file mode 100644 index 000000000..44d869ec0 --- /dev/null +++ b/430/installation/deployment.html @@ -0,0 +1,836 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Deploying Trino — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Deploying Trino#

+
+

Requirements#

+
+

Linux operating system#

+
    +
  • 64-bit required

  • +
  • newer release preferred, especially when running on containers

  • +
  • adequate ulimits for the user that runs the Trino process. These limits may +depend on the specific Linux distribution you are using. The number of open +file descriptors needed for a particular Trino instance scales as roughly the +number of machines in the cluster, times some factor depending on the +workload. The nofile limit sets the maximum number of file descriptors +that a process can have, while the nproc limit restricts the number of +processes, and therefore threads on the JVM, a user can create. We recommend +setting limits to the following values at a minimum. Typically, this +configuration is located in /etc/security/limits.conf:

    +
    trino soft nofile 131072
    +trino hard nofile 131072
    +trino soft nproc 128000
    +trino hard nproc 128000
    +
    +
    +
  • +
+
+
+

Java runtime environment#

+

Trino requires a 64-bit version of Java 17, with a minimum required version of 17.0.3. +Earlier major versions such as Java 8 or Java 11 do not work. +Newer major versions such as Java 18 or 19, are not supported – they may work, but are not tested.

+

We recommend using the Eclipse Temurin OpenJDK distribution from +Adoptium as the JDK for Trino, as Trino is tested +against that distribution. Eclipse Temurin is also the JDK used by the Trino +Docker image.

+

If you are using Java 17 or 18, the JVM must be configured to use UTF-8 as the default charset by +adding -Dfile.encoding=UTF-8 to etc/jvm.config. Starting with Java 19, the Java default +charset is UTF-8, so this configuration is not needed.

+
+
+

Python#

+
    +
  • version 2.6.x, 2.7.x, or 3.x

  • +
  • required by the bin/launcher script only

  • +
+
+
+
+

Installing Trino#

+

Download the Trino server tarball, trino-server-430.tar.gz, and unpack it. The +tarball contains a single top-level directory, trino-server-430, +which we call the installation directory.

+

Trino needs a data directory for storing logs, etc. +We recommend creating a data directory outside of the installation directory, +which allows it to be easily preserved when upgrading Trino.

+
+
+

Configuring Trino#

+

Create an etc directory inside the installation directory. +This holds the following configuration:

+
    +
  • Node Properties: environmental configuration specific to each node

  • +
  • JVM Config: command line options for the Java Virtual Machine

  • +
  • Config Properties: configuration for the Trino server. See the +Properties reference for available configuration properties.

  • +
  • Catalog Properties: configuration for Connectors (data sources). +The available catalog configuration properties for a connector are described +in the respective connector documentation.

  • +
+
+

Node properties#

+

The node properties file, etc/node.properties, contains configuration +specific to each node. A node is a single installed instance of Trino +on a machine. This file is typically created by the deployment system when +Trino is first installed. The following is a minimal etc/node.properties:

+
node.environment=production
+node.id=ffffffff-ffff-ffff-ffff-ffffffffffff
+node.data-dir=/var/trino/data
+
+
+

The above properties are described below:

+
    +
  • node.environment: +The name of the environment. All Trino nodes in a cluster must have the same +environment name. The name must start with a lowercase alphanumeric character +and only contain lowercase alphanumeric or underscore (_) characters.

  • +
  • node.id: +The unique identifier for this installation of Trino. This must be +unique for every node. This identifier should remain consistent across +reboots or upgrades of Trino. If running multiple installations of +Trino on a single machine (i.e. multiple nodes on the same machine), +each installation must have a unique identifier. The identifier must start +with an alphanumeric character and only contain alphanumeric, -, or _ +characters.

  • +
  • node.data-dir: +The location (filesystem path) of the data directory. Trino stores +logs and other data here.

  • +
+
+
+

JVM config#

+

The JVM config file, etc/jvm.config, contains a list of command line +options used for launching the Java Virtual Machine. The format of the file +is a list of options, one per line. These options are not interpreted by +the shell, so options containing spaces or other special characters should +not be quoted.

+

The following provides a good starting point for creating etc/jvm.config:

+
-server
+-Xmx16G
+-XX:InitialRAMPercentage=80
+-XX:MaxRAMPercentage=80
+-XX:G1HeapRegionSize=32M
+-XX:+ExplicitGCInvokesConcurrent
+-XX:+ExitOnOutOfMemoryError
+-XX:+HeapDumpOnOutOfMemoryError
+-XX:-OmitStackTraceInFastThrow
+-XX:ReservedCodeCacheSize=512M
+-XX:PerMethodRecompilationCutoff=10000
+-XX:PerBytecodeRecompilationCutoff=10000
+-Djdk.attach.allowAttachSelf=true
+-Djdk.nio.maxCachedBufferSize=2000000
+-XX:+UnlockDiagnosticVMOptions
+-XX:+UseAESCTRIntrinsics
+-Dfile.encoding=UTF-8
+# Disable Preventive GC for performance reasons (JDK-8293861)
+-XX:-G1UsePreventiveGC  
+# Reduce starvation of threads by GClocker, recommend to set about the number of cpu cores (JDK-8192647)
+-XX:GCLockerRetryAllocationCount=32
+
+
+

You must adjust the value for the memory used by Trino, specified with -Xmx +to the available memory on your nodes. Typically, values representing 70 to 85 +percent of the total available memory is recommended. For example, if all +workers and the coordinator use nodes with 64GB of RAM, you can use -Xmx54G. +Trino uses most of the allocated memory for processing, with a small percentage +used by JVM-internal processes such as garbage collection.

+

The rest of the available node memory must be sufficient for the operating +system and other running services, as well as off-heap memory used for native +code initiated the JVM process.

+

On larger nodes, the percentage value can be lower. Allocation of all memory to +the JVM or using swap space is not supported, and disabling swap space on the +operating system level is recommended.

+

Large memory allocation beyond 32GB is recommended for production clusters.

+

Because an OutOfMemoryError typically leaves the JVM in an +inconsistent state, we write a heap dump, for debugging, and forcibly +terminate the process when this occurs.

+

The temporary directory used by the JVM must allow execution of code. +Specifically, the mount must not have the noexec flag set. The default +/tmp directory is mounted with this flag in some installations, which +prevents Trino from starting. You can workaround this by overriding the +temporary directory by adding -Djava.io.tmpdir=/path/to/other/tmpdir to the +list of JVM options.

+

We enable -XX:+UnlockDiagnosticVMOptions and -XX:+UseAESCTRIntrinsics to improve AES performance for S3, etc. on ARM64 (JDK-8271567)
+We disable Preventive GC (-XX:-G1UsePreventiveGC) for performance reasons (see JDK-8293861)
+We set GCLocker retry allocation count (-XX:GCLockerRetryAllocationCount=32) to avoid OOM too early (see JDK-8192647)

+
+
+

Config properties#

+

The config properties file, etc/config.properties, contains the +configuration for the Trino server. Every Trino server can function as both a +coordinator and a worker. A cluster is required to include one coordinator, and +dedicating a machine to only perform coordination work provides the best +performance on larger clusters. Scaling and parallelization is achieved by using +many workers.

+

The following is a minimal configuration for the coordinator:

+
coordinator=true
+node-scheduler.include-coordinator=false
+http-server.http.port=8080
+discovery.uri=http://example.net:8080
+
+
+

And this is a minimal configuration for the workers:

+
coordinator=false
+http-server.http.port=8080
+discovery.uri=http://example.net:8080
+
+
+

Alternatively, if you are setting up a single machine for testing, that +functions as both a coordinator and worker, use this configuration:

+
coordinator=true
+node-scheduler.include-coordinator=true
+http-server.http.port=8080
+discovery.uri=http://example.net:8080
+
+
+

These properties require some explanation:

+
    +
  • coordinator: +Allow this Trino instance to function as a coordinator, so to +accept queries from clients and manage query execution.

  • +
  • node-scheduler.include-coordinator: +Allow scheduling work on the coordinator. +For larger clusters, processing work on the coordinator +can impact query performance because the machine’s resources are not +available for the critical task of scheduling, managing and monitoring +query execution.

  • +
  • http-server.http.port: +Specifies the port for the HTTP server. Trino uses HTTP for all +communication, internal and external.

  • +
  • discovery.uri: +The Trino coordinator has a discovery service that is used by all the nodes +to find each other. Every Trino instance registers itself with the discovery +service on startup and continuously heartbeats to keep its registration +active. The discovery service shares the HTTP server with Trino and thus +uses the same port. Replace example.net:8080 to match the host and +port of the Trino coordinator. If you have disabled HTTP on the coordinator, +the URI scheme must be https, not http.

  • +
+

The above configuration properties are a minimal set to help you get started. +All additional configuration is optional and varies widely based on the specific +cluster and supported use cases. The Administration and Security sections +contain documentation for many aspects, including Resource groups +for configuring queuing policies and Fault-tolerant execution.

+

The Properties reference provides a comprehensive list of the supported +properties for topics such as General properties, +Resource management properties, +Query management properties, +Web UI properties, and others.

+
+
+

Log levels#

+

The optional log levels file, etc/log.properties, allows setting the +minimum log level for named logger hierarchies. Every logger has a name, +which is typically the fully qualified name of the class that uses the logger. +Loggers have a hierarchy based on the dots in the name, like Java packages. +For example, consider the following log levels file:

+
io.trino=INFO
+
+
+

This would set the minimum level to INFO for both +io.trino.server and io.trino.plugin.hive. +The default minimum level is INFO, +thus the above example does not actually change anything. +There are four levels: DEBUG, INFO, WARN and ERROR.

+
+
+

Catalog properties#

+

Trino accesses data via connectors, which are mounted in catalogs. +The connector provides all of the schemas and tables inside of the catalog. +For example, the Hive connector maps each Hive database to a schema. +If the Hive connector is mounted as the hive catalog, and Hive +contains a table clicks in database web, that table can be accessed +in Trino as hive.web.clicks.

+

Catalogs are registered by creating a catalog properties file +in the etc/catalog directory. +For example, create etc/catalog/jmx.properties with the following +contents to mount the jmx connector as the jmx catalog:

+
connector.name=jmx
+
+
+

See Connectors for more information about configuring connectors.

+
+
+
+

Running Trino#

+

The installation provides a bin/launcher script, which requires Python in +the PATH. The script can be used manually or as a daemon startup script. It +accepts the following commands:

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + + + + +
launcher commands#

Command

Action

run

Starts the server in the foreground and leaves it running. To shut down +the server, use Ctrl+C in this terminal or the stop command from +another terminal.

start

Starts the server as a daemon and returns its process ID.

stop

Shuts down a server started with either start or run. Sends the +SIGTERM signal.

restart

Stops then restarts a running server, or starts a stopped server, +assigning a new process ID.

kill

Shuts down a possibly hung server by sending the SIGKILL signal.

status

Prints a status line, either Stopped pid or Running as pid.

+

A number of additional options allow you to specify configuration file and +directory locations, as well as Java options. Run the launcher with --help +to see the supported commands and command line options.

+

The -v or --verbose option for each command prepends the server’s +current settings before the command’s usual output.

+

Trino can be started as a daemon by running the following:

+
bin/launcher start
+
+
+

Alternatively, it can be run in the foreground, with the logs and other +output written to stdout/stderr. Both streams should be captured +if using a supervision system like daemontools:

+
bin/launcher run
+
+
+

The launcher configures default values for the configuration +directory etc, configuration files, the data directory var, +and log files in the data directory. You can change these values +to adjust your Trino usage to any requirements, such as using a +directory outside the installation directory, specific mount points +or locations, and even using other file names. For example, the Trino +RPM adjusts the used directories to better follow the Linux Filesystem +Hierarchy Standard (FHS).

+

After starting Trino, you can find log files in the log directory inside +the data directory var:

+
    +
  • launcher.log: +This log is created by the launcher and is connected to the stdout +and stderr streams of the server. It contains a few log messages +that occur while the server logging is being initialized, and any +errors or diagnostics produced by the JVM.

  • +
  • server.log: +This is the main log file used by Trino. It typically contains +the relevant information if the server fails during initialization. +It is automatically rotated and compressed.

  • +
  • http-request.log: +This is the HTTP request log which contains every HTTP request +received by the server. It is automatically rotated and compressed.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/installation/kubernetes.html b/430/installation/kubernetes.html new file mode 100644 index 000000000..be5f34725 --- /dev/null +++ b/430/installation/kubernetes.html @@ -0,0 +1,827 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Trino on Kubernetes with Helm — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Trino on Kubernetes with Helm#

+

Kubernetes is a container orchestration platform that +allows you to deploy Trino and other applications in a repeatable manner across +different types of infrastructure. This can range from deploying on your laptop +using tools like kind, to running on a managed +Kubernetes service on cloud services like +Amazon Elastic Kubernetes Service, +Google Kubernetes Engine, +Azure Kubernetes Service, +and others.

+

The fastest way to run Trino on Kubernetes is to use the +Trino Helm chart. +Helm is a package manager for Kubernetes applications that +allows for simpler installation and versioning by templating Kubernetes +configuration files. This allows you to prototype on your local or on-premise +cluster and use the same deployment mechanism to deploy to the cloud to scale +up.

+
+

Requirements#

+ +
+
+

Running Trino using Helm#

+

Run the following commands from the system with helm and kubectl +installed and configured to connect to your running Kubernetes cluster:

+
    +
  1. Validate kubectl is pointing to the correct cluster by running the +command:

    +
    kubectl cluster-info
    +
    +
    +

    You should see output that shows the correct Kubernetes control plane +address.

    +
  2. +
  3. Add the Trino Helm chart repository to Helm if you haven’t done so already. +This tells Helm where to find the Trino charts. You can name the repository +whatever you want, trino is a good choice.

    +
    helm repo add trino https://trinodb.github.io/charts
    +
    +
    +
  4. +
  5. Install Trino on the Kubernetes cluster using the Helm chart. Start by +running the install command to use all default values and create +a cluster called example-trino-cluster.

    +
    helm install example-trino-cluster trino/trino
    +
    +
    +

    This generates the Kubernetes configuration files by inserting properties +into helm templates. The Helm chart contains +default values +that can be overridden by a YAML file to update default settings.

    +
      +
    1. (Optional) To override the default values, +create your own YAML configuration to +define the parameters of your deployment. To run the install command using +the example.yaml, add the f parameter in you install command. +Be sure to follow +best practices and naming conventions +for your configuration files.

      +
      helm install -f example.yaml example-trino-cluster trino/trino
      +
      +
      +
    2. +
    +

    You should see output as follows:

    +
    NAME: example-trino-cluster
    +LAST DEPLOYED: Tue Sep 13 14:12:09 2022
    +NAMESPACE: default
    +STATUS: deployed
    +REVISION: 1
    +TEST SUITE: None
    +NOTES:
    +Get the application URL by running these commands:
    +  export POD_NAME=$(kubectl get pods --namespace default -l "app=trino,release=example-trino-cluster,component=coordinator" -o jsonpath="{.items[0].metadata.name}")
    +  echo "Visit http://127.0.0.1:8080 to use your application"
    +  kubectl port-forward $POD_NAME 8080:8080
    +
    +
    +

    This output depends on your configuration and cluster name. For example, the +port 8080 is set by the .service.port in the example.yaml.

    +
  6. +
  7. Run the following command to check that all pods, deployments, and services +are running properly.

    +
    kubectl get all
    +
    +
    +

    You should expect to see output that shows running pods, deployments, and +replica sets. A good indicator that everything is running properly is to see +all pods are returning a ready status in the READY column.

    +
    NAME                                               READY   STATUS    RESTARTS   AGE
    +pod/example-trino-cluster-coordinator-bfb74c98d-rnrxd   1/1     Running   0          161m
    +pod/example-trino-cluster-worker-76f6bf54d6-hvl8n       1/1     Running   0          161m
    +pod/example-trino-cluster-worker-76f6bf54d6-tcqgb       1/1     Running   0          161m
    +
    +NAME                       TYPE        CLUSTER-IP    EXTERNAL-IP   PORT(S)    AGE
    +service/example-trino-cluster   ClusterIP   10.96.25.35   <none>        8080/TCP   161m
    +
    +NAME                                           READY   UP-TO-DATE   AVAILABLE   AGE
    +deployment.apps/example-trino-cluster-coordinator   1/1     1            1           161m
    +deployment.apps/example-trino-cluster-worker        2/2     2            2           161m
    +
    +NAME                                                     DESIRED   CURRENT   READY   AGE
    +replicaset.apps/example-trino-cluster-coordinator-bfb74c98d   1         1         1       161m
    +replicaset.apps/example-trino-cluster-worker-76f6bf54d6       2         2         2       161m
    +
    +
    +

    The output shows running pods. These include the actual Trino containers. To +better understand this output, check out the following resources:

    +
      +
    1. kubectl get command reference.

    2. +
    3. kubectl get command example.

    4. +
    5. Debugging Kubernetes reference.

    6. +
    +
  8. +
  9. If all pods, deployments, and replica sets are running and in the ready +state, Trino has been successfully deployed.

  10. +
+
+

Note

+

Unlike some Kubernetes applications, where it’s better to have many small +pods, Trino works best with fewer pods each having more resources +available. We strongly recommend to avoid having multiple Trino pods on a +single physical host to avoid contention for resources.

+
+
+
+

Executing queries#

+

The pods running the Trino containers are all running on a private network +internal to Kubernetes. In order to access them, specifically the coordinator, +you need to create a tunnel to the coordinator pod and your computer. You can do +this by running the commands generated upon installation.

+
    +
  1. Store the coordinator pod name in a shell variable called POD_NAME.

    +
    POD_NAME=$(kubectl get pods -l "app=trino,release=example-trino-cluster,component=coordinator" -o name)
    +
    +
    +
  2. +
  3. Create the tunnel from the coordinator pod to the client.

    +
    kubectl port-forward $POD_NAME 8080:8080
    +
    +
    +

    Now you can connect to the Trino coordinator at http://localhost:8080.

    +
  4. +
  5. To connect to Trino, you can use the +command-line interface, a +JDBC client, or any of the +other clients. For this example, +install the command-line interface, and connect to +Trino in a new console session.

    +
    trino --server http://localhost:8080
    +
    +
    +
  6. +
  7. Using the sample data in the tpch catalog, type and execute a query on +the nation table using the tiny schema:

    +
    trino> select count(*) from tpch.tiny.nation;
    + _col0
    +-------
    +  25
    +(1 row)
    +
    +Query 20181105_001601_00002_e6r6y, FINISHED, 1 node
    +Splits: 21 total, 21 done (100.00%)
    +0:06 [25 rows, 0B] [4 rows/s, 0B/s]
    +
    +
    +

    Try other SQL queries to explore the data set and test your cluster.

    +
  8. +
  9. Once you are done with your exploration, enter the quit command in the +CLI.

  10. +
  11. Kill the tunnel to the coordinator pod. The is only available while the +kubectl process is running, so you can just kill the kubectl process +that’s forwarding the port. In most cases that means pressing CTRL + +C in the terminal where the port-forward command is running.

  12. +
+
+
+

Configuration#

+

The Helm chart uses the Trino container image. +The Docker image already contains a default configuration to get started, and +some catalogs to allow you to explore Trino. Kubernetes allows you to mimic a +traditional deployment by supplying +configuration in YAML files. It’s important to understand how files such as the +Trino configuration, JVM, and various catalog properties are +configured in Trino before updating the values.

+
+

Creating your own YAML configuration#

+

When you use your own YAML Kubernetes configuration, you only override the values you specify. +The remaining properties use their default values. Add an example.yaml with +the following configuration:

+
image:
+  tag: "430"
+server:
+  workers: 3
+coordinator:
+  jvm:
+    maxHeapSize: "8G"
+worker:
+  jvm:
+    maxHeapSize: "8G"
+
+
+

These values are higher than the defaults and allow Trino to use more memory +and run more demanding queries. If the values are too high, Kubernetes might +not be able to schedule some Trino pods, depending on other applications +deployed in this cluster and the size of the cluster nodes.

+
    +
  1. .image.tag is set to the current version, 430. Set +this value if you need to use a specific version of Trino. The default is +latest, which is not recommended. Using latest will publish a new +version of Trino with each release and a following Kubernetes deployment.

  2. +
  3. .server.workers is set to 3. This value sets the number of +workers, in this case, a coordinator and three worker nodes are deployed.

  4. +
  5. .coordinator.jvm.maxHeapSize is set to 8GB. +This sets the maximum heap size in the JVM of the coordinator. See +JVM config.

  6. +
  7. .worker.jvm.maxHeapSize is set to 8GB. +This sets the maximum heap size in the JVM of the worker. See +JVM config.

  8. +
+
+

Warning

+

Some memory settings need to be tuned carefully as setting some values +outside of the range of the maximum heap size will cause Trino startup to +fail. See the warnings listed on Resource management properties.

+
+

Reference the full list of properties +that can be overridden in the Helm chart.

+
+

Note

+

Although example.yaml is used to refer to the Kubernetes configuration +file in this document, you should use clear naming guidelines for the cluster +and deployment you are managing. For example, +cluster-example-trino-etl.yaml might refer to a Trino deployment for a +cluster used primarily for extract-transform-load queries deployed on the +example Kubernetes cluster. See +Configuration Best Practices +for more tips on configuring Kubernetes deployments.

+
+
+
+

Adding catalogs#

+

A common use-case is to add custom catalogs. You can do this by adding values to +the additionalCatalogs property in the example.yaml file.

+
additionalCatalogs:
+  lakehouse: |-
+    connector.name=iceberg
+    hive.metastore.uri=thrift://example.net:9083
+  rdbms: |-
+    connector.name=postgresql
+    connection-url=jdbc:postgresql://example.net:5432/database
+    connection-user=root
+    connection-password=secret
+
+
+

This adds both lakehouse and rdbms catalogs to the Kubernetes deployment +configuration.

+
+
+
+

Running a local Kubernetes cluster with kind#

+

For local deployments, you can use +kind (Kubernetes in Docker). Follow the steps +below to run kind on your system.

+
    +
  1. kind runs on Docker, so first check if Docker +is installed:

    +
    docker --version
    +
    +
    +

    If this command fails, install Docker by following +Docker installation instructions.

    +
  2. +
  3. Install kind by following the +kind installation instructions.

  4. +
  5. Run a Kubernetes cluster in kind by running the command:

    +
    kind create cluster --name trino
    +
    +
    +
    +

    Note

    +

    The name parameter is optional but is used to showcase how the +namespace is applied in future commands. The cluster name defaults to +kind if no parameter is added. Use trino to make the application +on this cluster obvious.

    +
    +
  6. +
  7. Verify that kubectl is running against the correct Kubernetes cluster.

    +
    kubectl cluster-info --context kind-trino
    +
    +
    +

    If you have multiple Kubernetes clusters already configured within +~/.kube/config, you need to pass the context parameter to the +kubectl commands to operate with the local kind cluster. kubectl +uses the +default context +if this parameter isn’t supplied. Notice the context is the name of the +cluster with the kind- prefix added. Now you can look at all the +Kubernetes objects running on your kind cluster.

    +
  8. +
  9. Set up Trino by folling the Running Trino using Helm steps. When +running the kubectl get all command, add the context parameter.

    +
    kubectl get all --context kind-trino
    +
    +
    +
  10. +
  11. Run some queries by following the Executing queries steps.

  12. +
  13. Once you are done with the cluster using kind, you can delete the cluster.

    +
    kind delete cluster -n trino
    +
    +
    +
  14. +
+
+
+

Cleaning up#

+

To uninstall Trino from the Kubernetes cluster, run the following command:

+
helm uninstall my-trino-cluster
+
+
+

You should expect to see the following output:

+
release "my-trino-cluster" uninstalled
+
+
+

To validate that this worked, you can run this kubectl command to make sure +there are no remaining Kubernetes objects related to the Trino cluster.

+
kubectl get all
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/installation/query-resiliency.html b/430/installation/query-resiliency.html new file mode 100644 index 000000000..c85005aa0 --- /dev/null +++ b/430/installation/query-resiliency.html @@ -0,0 +1,563 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Improve query processing resilience — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+ +
+
+ +
+
+ +
+

Improve query processing resilience#

+

You can configure Trino to be more resilient against failures during query +processing by enabling fault-tolerant execution. This allows Trino to handle +larger queries such as batch operations without worker node interruptions +causing the query to fail.

+

When configured, the Trino cluster buffers data used by workers during query +processing. If processing on a worker node fails for any reason, such as a +network outage or running out of available resources, the coordinator +reschedules processing of the failed piece of work on another worker. This +allows query processing to continue using buffered data.

+
+

Architecture#

+

The coordinator node uses a configured exchange manager service that buffers +data during query processing in an external location, such as an S3 object +storage bucket. Worker nodes send data to the buffer as they execute their +query tasks.

+
+
+

Best practices and considerations#

+

A fault-tolerant cluster is best suited for large batch queries. Users may +experience latency or similar behavior if they issue a high volume of +short-running queries on a fault-tolerant cluster. As such, it is recommended to +run a dedicated fault-tolerant cluster for handling batch operations, separate +from a cluster that is designated for a higher query volume.

+

Catalogs using the following connectors support fault-tolerant execution of read +and write operations:

+ +

Catalogs using other connectors only support fault-tolerant execution of read +operations. When fault-tolerant execution is enabled on a cluster, write +operations fail on any catalogs that do not support fault-tolerant +execution of those operations.

+

The exchange manager may send a large amount of data to the exchange storage, +resulting in high I/O load on that storage. You can configure multiple storage +locations for use by the exchange manager to help balance the I/O load between +them.

+
+
+

Configuration#

+

The following steps describe how to configure a Trino cluster for +fault-tolerant execution with an S3-based exchange:

+
    +
  1. Set up an S3 bucket to use as the exchange storage. For this example we are +using an AWS S3 bucket, but other storage options are described in the +reference documentation +as well. You can use multiple S3 buckets for exchange storage.

    +

    For each bucket in AWS, collect the following information:

    +
      +
    • S3 URI location for the bucket, such as s3://exchange-spooling-bucket

    • +
    • Region that the bucket is located in, such as us-west-1

    • +
    • AWS access and secret keys for the bucket

    • +
    +
  2. +
  3. For a Kubernetes deployment of Trino, add +the following exchange manager configuration in the +server.exchangeManager and additionalExchangeManagerProperties +sections of the Helm chart, using the gathered S3 bucket information:

    +
    server:
    +  exchangeManager:
    +    name=filesystem
    +    base-directories=s3://exchange-spooling-bucket-1,s3://exchange-spooling-bucket-2
    +
    +additionalExchangeManagerProperties:
    +  s3.region=us-west-1
    +  s3.aws-access-key=example-access-key
    +  s3.aws-secret-key=example-secret-key
    +
    +
    +

    In non-Kubernetes installations, the same properties must be defined in an +exchange-manager.properties configuration file on the coordinator and +all worker nodes.

    +
  4. +
  5. Add the following configuration for fault-tolerant execution in the +additionalConfigProperties: section of the Helm chart:

    +
    additionalConfigProperties:
    +  retry-policy=TASK
    +
    +
    +

    In non-Kubernetes installations, the same property must be defined in the +config.properties file on the coordinator and all worker nodes.

    +
  6. +
  7. Re-deploy your instance of Trino or, for non-Kubernetes +installations, restart the cluster.

  8. +
+

Your Trino cluster is now configured with fault-tolerant query +execution. If a query run on the cluster would normally fail due to an +interruption of query processing, fault-tolerant execution now resumes the +query processing to ensure successful execution of the query.

+
+
+

Next steps#

+

For more information about fault-tolerant execution, including simple query +retries that do not require an exchange manager and advanced configuration +operations, see the reference documentation.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/installation/rpm.html b/430/installation/rpm.html new file mode 100644 index 000000000..51acb9933 --- /dev/null +++ b/430/installation/rpm.html @@ -0,0 +1,552 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + RPM package — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+ +
+ +
+
+ +
+

RPM package#

+

Users can install Trino using the RPM Package Manager (RPM) on some Linux +distributions that support RPM.

+

The RPM archive contains the application, all plugins, the necessary default +configuration files, default setups, and integration with the operating system +to start as a service.

+
+

Warning

+

It is recommended to deploy Trino with the Helm chart on +Kubernetes or manually with the Docker containers or the +tar archive. While the RPM is available for use, it is +discouraged in favor of the tarball or Docker containers.

+
+
+

Installing Trino#

+

Download the Trino server RPM package trino-server-rpm-430.rpm. Use the +rpm command to install the package:

+
rpm -i trino-server-rpm-*.rpm --nodeps
+
+
+

Installing the required Java and Python setup must be +managed separately.

+
+
+

Service script#

+

The RPM installation deploys a service script configured with systemctl so +that the service can be started automatically on operating system boot. After +installation, you can manage the Trino server with the service command:

+
service trino [start|stop|restart|status]
+
+
+ + ++++ + + + + + + + + + + + + + + + + + + + +
service commands#

Command

Action

start

Starts the server as a daemon and returns its process ID.

stop

Shuts down a server started with either start or run. Sends the +SIGTERM signal.

restart

Stops and then starts a running server, or starts a stopped server, +assigning a new process ID.

status

Prints a status line, either Stopped pid or Running as pid.

+
+
+

Installation directory structure#

+

The RPM installation places Trino files in accordance with the Linux Filesystem +Hierarchy Standard using the following directory structure:

+
    +
  • /usr/lib/trino/lib/: Contains the various libraries needed to run the +product. Plugins go in a plugin subdirectory.

  • +
  • /etc/trino: Contains the general Trino configuration files like +node.properties, jvm.config, config.properties. Catalog +configurations go in a catalog subdirectory.

  • +
  • /etc/trino/env.sh: Contains the Java installation path used by Trino, +allows configuring process environment variables, including secrets.

  • +
  • /var/log/trino: Contains the log files.

  • +
  • /var/lib/trino/data: The location of the data directory. Trino stores logs +and other data here.

  • +
  • /etc/rc.d/init.d/trino: Contains the service scripts for controlling the +server process, and launcher configuration for file paths.

  • +
+
+
+

Uninstalling#

+

Uninstalling the RPM is like uninstalling any other RPM, just run:

+
rpm -e trino-server-rpm-<version>
+
+
+

Note: During uninstall, all Trino related files are deleted except for +user-created configuration files, copies of the original configuration files +node.properties.rpmsave and env.sh.rpmsave located in the /etc/trino +directory, and the Trino logs directory /var/log/trino.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/language.html b/430/language.html new file mode 100644 index 000000000..0fd2061cf --- /dev/null +++ b/430/language.html @@ -0,0 +1,485 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SQL language — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

SQL language#

+

Trino is an ANSI SQL compliant query engine. This standard compliance allows +Trino users to integrate their favorite data tools, including BI and ETL tools +with any underlying data source.

+

Trino validates and translates the received SQL statements into the necessary +operations on the connected data source.

+

This section provides a reference to the supported SQL data types and other +general characteristics of the SQL support of Trino.

+

Refer to the following sections for further details:

+ + + +
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/language/comments.html b/430/language/comments.html new file mode 100644 index 000000000..60ddbefbb --- /dev/null +++ b/430/language/comments.html @@ -0,0 +1,476 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Comments — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Comments#

+
+

Synopsis#

+

Comments are part of a SQL statement or script that are ignored for processing. +Comments begin with double dashes and extend to the end of the line. Block +comments begin with /* and extend to the next occurrence of */, possibly +spanning over multiple lines.

+
+
+

Examples#

+

The following example displays a comment line, a comment after a valid +statement, and a block comment:

+
-- This is a comment.
+SELECT * FROM table; -- This comment is ignored.
+
+/* This is a block comment
+   that spans multiple lines
+   until it is closed. */
+
+
+
+
+

See also#

+

COMMENT

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/language/reserved.html b/430/language/reserved.html new file mode 100644 index 000000000..fb8e67a12 --- /dev/null +++ b/430/language/reserved.html @@ -0,0 +1,816 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Keywords and identifiers — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Keywords and identifiers#

+
+

Reserved keywords#

+

The following table lists all of the keywords that are reserved in Trino, +along with their status in the SQL standard. These reserved keywords must +be quoted (using double quotes) in order to be used as an identifier.

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Keyword

SQL:2016

SQL-92

ALTER

reserved

reserved

AND

reserved

reserved

AS

reserved

reserved

BETWEEN

reserved

reserved

BY

reserved

reserved

CASE

reserved

reserved

CAST

reserved

reserved

CONSTRAINT

reserved

reserved

CREATE

reserved

reserved

CROSS

reserved

reserved

CUBE

reserved

CURRENT_CATALOG

reserved

CURRENT_DATE

reserved

reserved

CURRENT_PATH

reserved

CURRENT_ROLE

reserved

reserved

CURRENT_SCHEMA

reserved

CURRENT_TIME

reserved

reserved

CURRENT_TIMESTAMP

reserved

reserved

CURRENT_USER

reserved

DEALLOCATE

reserved

reserved

DELETE

reserved

reserved

DESCRIBE

reserved

reserved

DISTINCT

reserved

reserved

DROP

reserved

reserved

ELSE

reserved

reserved

END

reserved

reserved

ESCAPE

reserved

reserved

EXCEPT

reserved

reserved

EXECUTE

reserved

reserved

EXISTS

reserved

reserved

EXTRACT

reserved

reserved

FALSE

reserved

reserved

FOR

reserved

reserved

FROM

reserved

reserved

FULL

reserved

reserved

GROUP

reserved

reserved

GROUPING

reserved

HAVING

reserved

reserved

IN

reserved

reserved

INNER

reserved

reserved

INSERT

reserved

reserved

INTERSECT

reserved

reserved

INTO

reserved

reserved

IS

reserved

reserved

JOIN

reserved

reserved

JSON_ARRAY

reserved

JSON_EXISTS

reserved

JSON_OBJECT

reserved

JSON_QUERY

reserved

JSON_TABLE

reserved

JSON_VALUE

reserved

LEFT

reserved

reserved

LIKE

reserved

reserved

LISTAGG

reserved

LOCALTIME

reserved

LOCALTIMESTAMP

reserved

NATURAL

reserved

reserved

NORMALIZE

reserved

NOT

reserved

reserved

NULL

reserved

reserved

ON

reserved

reserved

OR

reserved

reserved

ORDER

reserved

reserved

OUTER

reserved

reserved

PREPARE

reserved

reserved

RECURSIVE

reserved

RIGHT

reserved

reserved

ROLLUP

reserved

SELECT

reserved

reserved

SKIP

reserved

TABLE

reserved

reserved

THEN

reserved

reserved

TRIM

reserved

reserved

TRUE

reserved

reserved

UESCAPE

reserved

UNION

reserved

reserved

UNNEST

reserved

USING

reserved

reserved

VALUES

reserved

reserved

WHEN

reserved

reserved

WHERE

reserved

reserved

WITH

reserved

reserved

+
+
+

Identifiers#

+

Tokens that identify names of catalogs, schemas, tables, columns, functions, or +other objects, are identifiers.

+

Identifiers must start with a letter, and subsequently include alphanumeric +characters and underscores. Identifiers with other characters must be delimited +with double quotes ("). When delimited with double quotes, identifiers can use +any character. Escape a " with another preceding double quote in a delimited +identifier.

+

Identifiers are not treated as case sensitive.

+

Following are some valid examples:

+
tablename
+SchemaName
+example_catalog.a_schema."table$partitions"
+"identifierWith""double""quotes"
+
+
+

The following identifiers are invalid in Trino and must be quoted when used:

+
table-name
+123SchemaName
+colum$name@field
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/language/sql-support.html b/430/language/sql-support.html new file mode 100644 index 000000000..0c57825d4 --- /dev/null +++ b/430/language/sql-support.html @@ -0,0 +1,615 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SQL statement support — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

SQL statement support#

+

The SQL statement support in Trino can be categorized into several topics. Many +statements are part of the core engine and therefore available in all use cases. +For example, you can always set session properties or inspect an explain plan +and perform other actions with the globally available statements.

+

However, the details and architecture of the connected data sources can limit +some SQL functionality. For example, if the data source does not support any +write operations, then a DELETE statement cannot be executed against +the data source.

+

Similarly, if the underlying system does not have any security concepts, SQL +statements like CREATE ROLE cannot be supported by Trino and the +connector.

+

The categories of these different topics are related to read operations, write operations, +security operations and transactions.

+

Details of the support for specific statements is available with the +documentation for each connector.

+
+

Globally available statements#

+

The following statements are implemented in the core engine and available with +any connector:

+ +
+
+

Read operations#

+

The following statements provide read access to data and meta data exposed by a +connector accessing a data source. They are supported by all connectors:

+ +
+
+

Write operations#

+

The following statements provide write access to data and meta data exposed +by a connector accessing a data source. Availability varies widely from +connector to connector:

+
+

Data management#

+ +
+
+

Materialized view management#

+ +
+
+

Schema and table management#

+ +
+
+

View management#

+ +
+
+
+

Security operations#

+

The following statements provide security-related operations to security +configuration, data, and meta data exposed by a connector accessing a data +source. Most connectors do not support these operations:

+

Connector roles:

+ +

Grants management:

+ +
+
+

Transactions#

+

The following statements manage transactions. Most connectors do not support +transactions:

+ +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/language/types.html b/430/language/types.html new file mode 100644 index 000000000..00a3a6dc3 --- /dev/null +++ b/430/language/types.html @@ -0,0 +1,1060 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Data types — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+ +
+ +
+
+ +
+

Data types#

+

Trino has a set of built-in data types, described below. +Additional types can be provided by plugins.

+
+

Trino type support and mapping#

+

Connectors to data sources are not required to support all Trino data types +described on this page. If there are data types similar to Trino’s that are used +on the data source, the connector may map the Trino and remote data types to +each other as needed.

+

Depending on the connector and the data source, type mapping may apply +in either direction as follows:

+
    +
  • Data source to Trino mapping applies to any operation where columns in the +data source are read by Trino, such as a SELECT statement, and the +underlying source data type needs to be represented by a Trino data type.

  • +
  • Trino to data source mapping applies to any operation where the columns +or expressions in Trino need to be translated into data types or expressions +compatible with the underlying data source. For example, +CREATE TABLE AS statements specify Trino types that are then +mapped to types on the remote data source. Predicates like WHERE also use +these mappings in order to ensure that the predicate is translated to valid +syntax on the remote data source.

  • +
+

Data type support and mappings vary depending on the connector. Refer to the +connector documentation for more information.

+
+
+

Boolean#

+
+

BOOLEAN#

+

This type captures boolean values true and false.

+
+
+
+

Integer#

+

Integer numbers can be expressed as numeric literals in the following formats:

+
    +
  • Decimal integer. Examples are -7, 0, or 3.

  • +
  • Hexadecimal integer composed of 0X or 0x and the value. Examples are +0x0A for decimal 10 or 0x11 for decimal 17.

  • +
  • Octal integer composed of 0O or 0o and the value. Examples are 0o40 for +decimal 32 or 0o11 for decimal 9.

  • +
  • Binary integer composed of 0B or 0b and the value. Examples are 0b1001 +for decimal 9 or 0b101010 for decimal `42``.

  • +
+

Underscore characters are ignored within literal values, and can be used to +increase readability. For example, decimal integer 123_456.789_123 is +equivalent to 123456.789123. Preceding and trailing underscores are not +permitted.

+

Integers are supported by the following data types.

+
+

TINYINT#

+

A 8-bit signed two’s complement integer with a minimum value of +-2^7 or -0x80 and a maximum value of 2^7 - 1 or 0x7F.

+
+
+

SMALLINT#

+

A 16-bit signed two’s complement integer with a minimum value of +-2^15 or -0x8000 and a maximum value of 2^15 - 1 or 0x7FFF.

+
+
+

INTEGER or INT#

+

A 32-bit signed two’s complement integer with a minimum value of -2^31 or +-0x80000000 and a maximum value of 2^31 - 1 or 0x7FFFFFFF. The names +INTEGER and INT can both be used for this type.

+
+
+

BIGINT#

+

A 64-bit signed two’s complement integer with a minimum value of -2^63 or +-0x8000000000000000 and a maximum value of 2^63 - 1 or 0x7FFFFFFFFFFFFFFF.

+
+
+
+

Floating-point#

+

Floating-point, fixed-precision numbers can be expressed as numeric literal +using scientific notation such as 1.03e1 and are cast as DOUBLE data type. +Underscore characters are ignored within literal values, and can be used to +increase readability. For example, value 123_456.789e4 is equivalent to +123456.789e4. Preceding underscores, trailing underscores, and underscores +beside the comma (.) are not permitted.

+
+

REAL#

+

A real is a 32-bit inexact, variable-precision implementing the +IEEE Standard 754 for Binary Floating-Point Arithmetic.

+

Example literals: REAL '10.3', REAL '10.3e0', REAL '1.03e1'

+
+
+

DOUBLE#

+

A double is a 64-bit inexact, variable-precision implementing the +IEEE Standard 754 for Binary Floating-Point Arithmetic.

+

Example literals: DOUBLE '10.3', DOUBLE '1.03e1', 10.3e0, 1.03e1

+
+
+
+

Fixed-precision#

+

Fixed-precision numbers can be expressed as numeric literals such as 1.1, and +are supported by the DECIMAL data type.

+

Underscore characters are ignored within literal values, and can be used to +increase readability. For example, decimal 123_456.789_123 is equivalent to +123456.789123. Preceding underscores, trailing underscores, and underscores +beside the comma (.) are not permitted.

+

Leading zeros in literal values are permitted and ignored. For example, +000123.456 is equivalent to 123.456.

+
+

DECIMAL#

+

A fixed-precision decimal number. Precision up to 38 digits is supported +but performance is best up to 18 digits.

+

The decimal type takes two literal parameters:

+
    +
  • precision - total number of digits

  • +
  • scale - number of digits in fractional part. Scale is optional and defaults to 0.

  • +
+

Example type definitions: DECIMAL(10,3), DECIMAL(20)

+

Example literals: DECIMAL '10.3', DECIMAL '1234567890', 1.1

+
+
+
+

String#

+
+

VARCHAR#

+

Variable length character data with an optional maximum length.

+

Example type definitions: varchar, varchar(20)

+

SQL statements support simple literal, as well as Unicode usage:

+
    +
  • literal string : 'Hello winter !'

  • +
  • Unicode string with default escape character: U&'Hello winter \2603 !'

  • +
  • Unicode string with custom escape character: U&'Hello winter #2603 !' UESCAPE '#'

  • +
+

A Unicode string is prefixed with U& and requires an escape character +before any Unicode character usage with 4 digits. In the examples above +\2603 and #2603 represent a snowman character. Long Unicode codes +with 6 digits require usage of the plus symbol before the code. For example, +you need to use \+01F600 for a grinning face emoji.

+
+
+

CHAR#

+

Fixed length character data. A CHAR type without length specified has a default length of 1. +A CHAR(x) value always has x characters. For example, casting dog to CHAR(7) +adds 4 implicit trailing spaces. Leading and trailing spaces are included in comparisons of +CHAR values. As a result, two character values with different lengths (CHAR(x) and +CHAR(y) where x != y) will never be equal.

+

Example type definitions: char, char(20)

+
+
+

VARBINARY#

+

Variable length binary data.

+

SQL statements support usage of binary literal data with the prefix X or x. +The binary data has to use hexadecimal format. For example, the binary form of +eh? is X'65683F' as you can confirm with the following statement:

+
SELECT from_utf8(x'65683F');
+
+
+
+

Note

+

Binary strings with length are not yet supported: varbinary(n)

+
+
+
+

JSON#

+

JSON value type, which can be a JSON object, a JSON array, a JSON number, a JSON string, +true, false or null.

+
+
+
+

Date and time#

+

See also Date and time functions and operators

+
+

DATE#

+

Calendar date (year, month, day).

+

Example: DATE '2001-08-22'

+
+
+

TIME#

+

TIME is an alias for TIME(3) (millisecond precision).

+
+
+

TIME(P)#

+

Time of day (hour, minute, second) without a time zone with P digits of precision +for the fraction of seconds. A precision of up to 12 (picoseconds) is supported.

+

Example: TIME '01:02:03.456'

+
+
+

TIME WITH TIME ZONE#

+

Time of day (hour, minute, second, millisecond) with a time zone. +Values of this type are rendered using the time zone from the value. +Time zones are expressed as the numeric UTC offset value:

+
SELECT TIME '01:02:03.456 -08:00';
+-- 1:02:03.456-08:00
+
+
+
+
+

TIMESTAMP#

+

TIMESTAMP is an alias for TIMESTAMP(3) (millisecond precision).

+
+
+

TIMESTAMP(P)#

+

Calendar date and time of day without a time zone with P digits of precision +for the fraction of seconds. A precision of up to 12 (picoseconds) is supported. +This type is effectively a combination of the DATE and TIME(P) types.

+

TIMESTAMP(P) WITHOUT TIME ZONE is an equivalent name.

+

Timestamp values can be constructed with the TIMESTAMP literal +expression. Alternatively, language constructs such as +localtimestamp(p), or a number of date and time functions and operators can return timestamp values.

+

Casting to lower precision causes the value to be rounded, and not +truncated. Casting to higher precision appends zeros for the additional +digits.

+

The following examples illustrate the behavior:

+
SELECT TIMESTAMP '2020-06-10 15:55:23';
+-- 2020-06-10 15:55:23
+
+SELECT TIMESTAMP '2020-06-10 15:55:23.383345';
+-- 2020-06-10 15:55:23.383345
+
+SELECT typeof(TIMESTAMP '2020-06-10 15:55:23.383345');
+-- timestamp(6)
+
+SELECT cast(TIMESTAMP '2020-06-10 15:55:23.383345' as TIMESTAMP(1));
+ -- 2020-06-10 15:55:23.4
+
+SELECT cast(TIMESTAMP '2020-06-10 15:55:23.383345' as TIMESTAMP(12));
+-- 2020-06-10 15:55:23.383345000000
+
+
+
+
+

TIMESTAMP WITH TIME ZONE#

+

TIMESTAMP WITH TIME ZONE is an alias for TIMESTAMP(3) WITH TIME ZONE +(millisecond precision).

+
+
+

TIMESTAMP(P) WITH TIME ZONE#

+

Instant in time that includes the date and time of day with P digits of +precision for the fraction of seconds and with a time zone. Values of this +type are rendered using the time zone from the value. +Time zones can be expressed in the following ways:

+
    +
  • UTC, with GMT, Z, or UT usable as aliases for UTC.

  • +
  • +hh:mm or -hh:mm with hh:mm as an hour and minute offset from UTC. +Can be written with or without UTC, GMT, or UT as an alias for +UTC.

  • +
  • An IANA time zone name.

  • +
+

The following examples demonstrate some of these syntax options:

+
SELECT TIMESTAMP '2001-08-22 03:04:05.321 UTC';
+-- 2001-08-22 03:04:05.321 UTC
+
+SELECT TIMESTAMP '2001-08-22 03:04:05.321 -08:30';
+-- 2001-08-22 03:04:05.321 -08:30
+
+SELECT TIMESTAMP '2001-08-22 03:04:05.321 GMT-08:30';
+-- 2001-08-22 03:04:05.321 -08:30
+
+SELECT TIMESTAMP '2001-08-22 03:04:05.321 America/New_York';
+-- 2001-08-22 03:04:05.321 America/New_York
+
+
+
+
+

INTERVAL YEAR TO MONTH#

+

Span of years and months.

+

Example: INTERVAL '3' MONTH

+
+
+

INTERVAL DAY TO SECOND#

+

Span of days, hours, minutes, seconds and milliseconds.

+

Example: INTERVAL '2' DAY

+
+
+
+

Structural#

+
+

ARRAY#

+

An array of the given component type.

+

Example: ARRAY[1, 2, 3]

+
+
+

MAP#

+

A map between the given component types.

+

Example: MAP(ARRAY['foo', 'bar'], ARRAY[1, 2])

+
+
+

ROW#

+

A structure made up of fields that allows mixed types. +The fields may be of any SQL type.

+

By default, row fields are not named, but names can be assigned.

+

Example: CAST(ROW(1, 2e0) AS ROW(x BIGINT, y DOUBLE))

+

Named row fields are accessed with field reference operator (.).

+

Example: CAST(ROW(1, 2.0) AS ROW(x BIGINT, y DOUBLE)).x

+

Named or unnamed row fields are accessed by position with the subscript +operator ([]). The position starts at 1 and must be a constant.

+

Example: ROW(1, 2.0)[1]

+
+
+
+

Network address#

+
+

IPADDRESS#

+

An IP address that can represent either an IPv4 or IPv6 address. Internally, +the type is a pure IPv6 address. Support for IPv4 is handled using the +IPv4-mapped IPv6 address range (RFC 4291#section-2.5.5.2). +When creating an IPADDRESS, IPv4 addresses will be mapped into that range. +When formatting an IPADDRESS, any address within the mapped range will +be formatted as an IPv4 address. Other addresses will be formatted as IPv6 +using the canonical format defined in RFC 5952.

+

Examples: IPADDRESS '10.0.0.1', IPADDRESS '2001:db8::1'

+
+
+
+

UUID#

+
+

UUID#

+

This type represents a UUID (Universally Unique IDentifier), also known as a +GUID (Globally Unique IDentifier), using the format defined in RFC 4122.

+

Example: UUID '12151fd2-7586-11e9-8f9e-2a86e4085a59'

+
+
+
+

HyperLogLog#

+

Calculating the approximate distinct count can be done much more cheaply than an exact count using the +HyperLogLog data sketch. See HyperLogLog functions.

+
+

HyperLogLog#

+

A HyperLogLog sketch allows efficient computation of approx_distinct(). It starts as a +sparse representation, switching to a dense representation when it becomes more efficient.

+
+
+

P4HyperLogLog#

+

A P4HyperLogLog sketch is similar to HyperLogLog, but it starts (and remains) +in the dense representation.

+
+
+
+

SetDigest#

+
+

SetDigest#

+

A SetDigest (setdigest) is a data sketch structure used +in calculating Jaccard similarity coefficient +between two sets.

+

SetDigest encapsulates the following components:

+ +

The HyperLogLog structure is used for the approximation of the distinct elements +in the original set.

+

The MinHash structure is used to store a low memory footprint signature of the original set. +The similarity of any two sets is estimated by comparing their signatures.

+

SetDigests are additive, meaning they can be merged together.

+
+
+
+

Quantile digest#

+
+

QDigest#

+

A quantile digest (qdigest) is a summary structure which captures the approximate +distribution of data for a given input set, and can be queried to retrieve approximate +quantile values from the distribution. The level of accuracy for a qdigest +is tunable, allowing for more precise results at the expense of space.

+

A qdigest can be used to give approximate answer to queries asking for what value +belongs at a certain quantile. A useful property of qdigests is that they are +additive, meaning they can be merged together without losing precision.

+

A qdigest may be helpful whenever the partial results of approx_percentile +can be reused. For example, one may be interested in a daily reading of the 99th +percentile values that are read over the course of a week. Instead of calculating +the past week of data with approx_percentile, qdigests could be stored +daily, and quickly merged to retrieve the 99th percentile value.

+
+
+
+

T-Digest#

+
+

TDigest#

+

A T-digest (tdigest) is a summary structure which, similarly to qdigest, captures the +approximate distribution of data for a given input set. It can be queried to retrieve +approximate quantile values from the distribution.

+

TDigest has the following advantages compared to QDigest:

+
    +
  • higher performance

  • +
  • lower memory usage

  • +
  • higher accuracy at high and low percentiles

  • +
+

T-digests are additive, meaning they can be merged together.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/objects.inv b/430/objects.inv new file mode 100644 index 000000000..914fe96a3 Binary files /dev/null and b/430/objects.inv differ diff --git a/430/optimizer.html b/430/optimizer.html new file mode 100644 index 000000000..68caca955 --- /dev/null +++ b/430/optimizer.html @@ -0,0 +1,446 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Query optimizer — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+ +
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/optimizer/cost-based-optimizations.html b/430/optimizer/cost-based-optimizations.html new file mode 100644 index 000000000..42aa3ac91 --- /dev/null +++ b/430/optimizer/cost-based-optimizations.html @@ -0,0 +1,581 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Cost-based optimizations — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Cost-based optimizations#

+

Trino supports several cost based optimizations, described below.

+
+

Join enumeration#

+

The order in which joins are executed in a query can have a significant impact +on the query’s performance. The aspect of join ordering that has the largest +impact on performance is the size of the data being processed and transferred +over the network. If a join which produces a lot of data is performed early in +the query’s execution, then subsequent stages need to process large amounts of +data for longer than necessary, increasing the time and resources needed for +processing the query.

+

With cost-based join enumeration, Trino uses Table statistics +provided by connectors to estimate the costs for different join orders and +automatically picks the join order with the lowest computed costs.

+

The join enumeration strategy is governed by the join_reordering_strategy +session property, with the +optimizer.join-reordering-strategy configuration property providing the +default value.

+

The possible values are:

+
+
    +
  • AUTOMATIC (default) - enable full automatic join enumeration

  • +
  • ELIMINATE_CROSS_JOINS - eliminate unnecessary cross joins

  • +
  • NONE - purely syntactic join order

  • +
+
+

If you are using AUTOMATIC join enumeration and statistics are not +available or a cost can not be computed for any other reason, the +ELIMINATE_CROSS_JOINS strategy is used instead.

+
+
+

Join distribution selection#

+

Trino uses a hash-based join algorithm. For each join operator, a hash table +must be created from one join input, referred to as the build side. The other +input, called the probe side, is then iterated on. For each row, the hash table +is queried to find matching rows.

+

There are two types of join distributions:

+
+
    +
  • Partitioned: each node participating in the query builds a hash table from +only a fraction of the data

  • +
  • Broadcast: each node participating in the query builds a hash table from all +of the data. The data is replicated to each node.

  • +
+
+

Each type has advantages and disadvantages. Partitioned joins require +redistributing both tables using a hash of the join key. These joins can be much +slower than broadcast joins, but they allow much larger joins overall. Broadcast +joins are faster if the build side is much smaller than the probe side. However, +broadcast joins require that the tables on the build side of the join after +filtering fit in memory on each node, whereas distributed joins only need to fit +in distributed memory across all nodes.

+

With cost-based join distribution selection, Trino automatically chooses whether +to use a partitioned or broadcast join. With cost-based join enumeration, Trino +automatically chooses which sides are probe and build.

+

The join distribution strategy is governed by the join_distribution_type +session property, with the join-distribution-type configuration property +providing the default value.

+

The valid values are:

+
+
    +
  • AUTOMATIC (default) - join distribution type is determined automatically +for each join

  • +
  • BROADCAST - broadcast join distribution is used for all joins

  • +
  • PARTITIONED - partitioned join distribution is used for all join

  • +
+
+
+

Capping replicated table size#

+

The join distribution type is automatically chosen when the join reordering +strategy is set to AUTOMATIC or when the join distribution type is set to +AUTOMATIC. In both cases, it is possible to cap the maximum size of the +replicated table with the join-max-broadcast-table-size configuration +property or with the join_max_broadcast_table_size session property. This +allows you to improve cluster concurrency and prevent bad plans when the +cost-based optimizer misestimates the size of the joined tables.

+

By default, the replicated table size is capped to 100MB.

+
+
+
+

Syntactic join order#

+

If not using cost-based optimization, Trino defaults to syntactic join ordering. +While there is no formal way to optimize queries for this case, it is possible +to take advantage of how Trino implements joins to make them more performant.

+

Trino uses in-memory hash joins. When processing a join statement, Trino loads +the right-most table of the join into memory as the build side, then streams the +next right-most table as the probe side to execute the join. If a query has +multiple joins, the result of this first join stays in memory as the build side, +and the third right-most table is then used as the probe side, and so on for +additional joins. In the case where join order is made more complex, such as +when using parentheses to specify specific parents for joins, Trino may execute +multiple lower-level joins at once, but each step of that process follows the +same logic, and the same applies when the results are ultimately joined +together.

+

Because of this behavior, it is optimal to syntactically order joins in your SQL +queries from the largest tables to the smallest, as this minimizes memory usage.

+

As an example, if you have a small, medium, and large table and are using left +joins:

+
SELECT
+  *
+FROM
+  large_table l
+  LEFT JOIN medium_table m ON l.user_id = m.user_id
+  LEFT JOIN small_table s ON s.user_id = l.user_id
+
+
+
+

Warning

+

This means of optimization is not a feature of Trino. It is an artifact of +how joins are implemented, and therefore this behavior may change without +notice.

+
+
+
+

Connector implementations#

+

In order for the Trino optimizer to use the cost based strategies, +the connector implementation must provide Table statistics.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/optimizer/cost-in-explain.html b/430/optimizer/cost-in-explain.html new file mode 100644 index 000000000..589e621e1 --- /dev/null +++ b/430/optimizer/cost-in-explain.html @@ -0,0 +1,472 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Cost in EXPLAIN — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Cost in EXPLAIN#

+

During planning, the cost associated with each node of the plan is computed +based on the table statistics for the tables in the query. This calculated +cost is printed as part of the output of an EXPLAIN statement.

+

Cost information is displayed in the plan tree using the format {rows: XX (XX), cpu: XX, memory: XX, network: XX}. rows refers to the expected +number of rows output by each plan node during execution. The value in the +parentheses following the number of rows refers to the expected size of the data +output by each plan node in bytes. Other parameters indicate the estimated +amount of CPU, memory, and network utilized by the execution of a plan node. +These values do not represent any actual unit, but are numbers that are used to +compare the relative costs between plan nodes, allowing the optimizer to choose +the best plan for executing a query. If any of the values is not known, a ? +is printed.

+

For example:

+
EXPLAIN SELECT comment FROM tpch.sf1.nation WHERE nationkey > 3;
+
+
+
- Output[comment] => [[comment]]
+        Estimates: {rows: 22 (1.69kB), cpu: 6148.25, memory: 0.00, network: 1734.25}
+    - RemoteExchange[GATHER] => [[comment]]
+            Estimates: {rows: 22 (1.69kB), cpu: 6148.25, memory: 0.00, network: 1734.25}
+        - ScanFilterProject[table = tpch:nation:sf1.0, filterPredicate = ("nationkey" > BIGINT '3')] => [[comment]]
+                Estimates: {rows: 25 (1.94kB), cpu: 2207.00, memory: 0.00, network: 0.00}/{rows: 22 (1.69kB), cpu: 4414.00, memory: 0.00, network: 0.00}/{rows: 22 (1.69kB), cpu: 6148.25, memory: 0.00, network: 0.00}
+                nationkey := tpch:nationkey
+                comment := tpch:comment
+
+
+

Generally, there is only one cost printed for each plan node. However, when a +Scan operator is combined with a Filter and/or Project operator, +then multiple cost structures are printed, each corresponding to an +individual logical part of the combined operator. For example, three cost +structures are printed for a ScanFilterProject operator, corresponding +to the Scan, Filter, and Project parts of the operator, in that order.

+

Estimated cost is also printed in EXPLAIN ANALYZE in addition to actual +runtime statistics.

+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/optimizer/pushdown.html b/430/optimizer/pushdown.html new file mode 100644 index 000000000..61354876b --- /dev/null +++ b/430/optimizer/pushdown.html @@ -0,0 +1,785 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Pushdown — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Pushdown#

+

Trino can push down the processing of queries, or parts of queries, into the +connected data source. This means that a specific predicate, aggregation +function, or other operation, is passed through to the underlying database or +storage system for processing.

+

The results of this pushdown can include the following benefits:

+
    +
  • Improved overall query performance

  • +
  • Reduced network traffic between Trino and the data source

  • +
  • Reduced load on the remote data source

  • +
+

These benefits often result in significant cost reduction.

+

Support for pushdown is specific to each connector and the relevant underlying +database or storage system.

+
+

Predicate pushdown#

+

Predicate pushdown optimizes row-based filtering. It uses the inferred filter, +typically resulting from a condition in a WHERE clause to omit unnecessary +rows. The processing is pushed down to the data source by the connector and then +processed by the data source.

+

If predicate pushdown for a specific clause is succesful, the EXPLAIN plan +for the query does not include a ScanFilterProject operation for that +clause.

+
+
+

Projection pushdown#

+

Projection pushdown optimizes column-based filtering. It uses the columns +specified in the SELECT clause and other parts of the query to limit access +to these columns. The processing is pushed down to the data source by the +connector and then the data source only reads and returns the neccessary +columns.

+

If projection pushdown is succesful, the EXPLAIN plan for the query only +accesses the relevant columns in the Layout of the TableScan operation.

+
+
+

Dereference pushdown#

+

Projection pushdown and dereference pushdown limit access to relevant columns, +except dereference pushdown is more selective. It limits access to only read the +specified fields within a top level or nested ROW data type.

+

For example, consider a table in the Hive connector that has a ROW type +column with several fields. If a query only accesses one field, dereference +pushdown allows the file reader to read only that single field within the row. +The same applies to fields of a row nested within the top level row. This can +result in significant savings in the amount of data read from the storage +system.

+
+
+

Aggregation pushdown#

+

Aggregation pushdown can take place provided the following conditions are satisfied:

+
    +
  • If aggregation pushdown is generally supported by the connector.

  • +
  • If pushdown of the specific function or functions is supported by the connector.

  • +
  • If the query structure allows pushdown to take place.

  • +
+

You can check if pushdown for a specific query is performed by looking at the +EXPLAIN plan of the query. If an aggregate function is successfully +pushed down to the connector, the explain plan does not show that Aggregate operator. +The explain plan only shows the operations that are performed by Trino.

+

As an example, we loaded the TPCH data set into a PostgreSQL database and then +queried it using the PostgreSQL connector:

+
SELECT regionkey, count(*)
+FROM nation
+GROUP BY regionkey;
+
+
+

You can get the explain plan by prepending the above query with EXPLAIN:

+
EXPLAIN
+SELECT regionkey, count(*)
+FROM nation
+GROUP BY regionkey;
+
+
+

The explain plan for this query does not show any Aggregate operator with +the count function, as this operation is now performed by the connector. You +can see the count(*) function as part of the PostgreSQL TableScan +operator. This shows you that the pushdown was successful.

+
Fragment 0 [SINGLE]
+    Output layout: [regionkey_0, _generated_1]
+    Output partitioning: SINGLE []
+    Output[regionkey, _col1]
+    │   Layout: [regionkey_0:bigint, _generated_1:bigint]
+    │   Estimates: {rows: ? (?), cpu: ?, memory: 0B, network: ?}
+    │   regionkey := regionkey_0
+    │   _col1 := _generated_1
+    └─ RemoteSource[1]
+            Layout: [regionkey_0:bigint, _generated_1:bigint]
+
+Fragment 1 [SOURCE]
+    Output layout: [regionkey_0, _generated_1]
+    Output partitioning: SINGLE []
+    TableScan[postgresql:tpch.nation tpch.nation columns=[regionkey:bigint:int8, count(*):_generated_1:bigint:bigint] groupingSets=[[regionkey:bigint:int8]], gro
+        Layout: [regionkey_0:bigint, _generated_1:bigint]
+        Estimates: {rows: ? (?), cpu: ?, memory: 0B, network: 0B}
+        _generated_1 := count(*):_generated_1:bigint:bigint
+        regionkey_0 := regionkey:bigint:int8
+
+
+

A number of factors can prevent a push down:

+
    +
  • adding a condition to the query

  • +
  • using a different aggregate function that cannot be pushed down into the connector

  • +
  • using a connector without pushdown support for the specific function

  • +
+

As a result, the explain plan shows the Aggregate operation being performed +by Trino. This is a clear sign that now pushdown to the remote data source is not +performed, and instead Trino performs the aggregate processing.

+
Fragment 0 [SINGLE]
+    Output layout: [regionkey, count]
+    Output partitioning: SINGLE []
+    Output[regionkey, _col1]
+    │   Layout: [regionkey:bigint, count:bigint]
+    │   Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?}
+    │   _col1 := count
+    └─ RemoteSource[1]
+           Layout: [regionkey:bigint, count:bigint]
+
+Fragment 1 [HASH]
+    Output layout: [regionkey, count]
+    Output partitioning: SINGLE []
+    Aggregate(FINAL)[regionkey]
+    │   Layout: [regionkey:bigint, count:bigint]
+    │   Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?}
+    │   count := count("count_0")
+    └─ LocalExchange[HASH][$hashvalue] ("regionkey")
+       │   Layout: [regionkey:bigint, count_0:bigint, $hashvalue:bigint]
+       │   Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?}
+       └─ RemoteSource[2]
+              Layout: [regionkey:bigint, count_0:bigint, $hashvalue_1:bigint]
+
+Fragment 2 [SOURCE]
+    Output layout: [regionkey, count_0, $hashvalue_2]
+    Output partitioning: HASH [regionkey][$hashvalue_2]
+    Project[]
+    │   Layout: [regionkey:bigint, count_0:bigint, $hashvalue_2:bigint]
+    │   Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?}
+    │   $hashvalue_2 := combine_hash(bigint '0', COALESCE("$operator$hash_code"("regionkey"), 0))
+    └─ Aggregate(PARTIAL)[regionkey]
+       │   Layout: [regionkey:bigint, count_0:bigint]
+       │   count_0 := count(*)
+       └─ TableScan[tpch:nation:sf0.01, grouped = false]
+              Layout: [regionkey:bigint]
+              Estimates: {rows: 25 (225B), cpu: 225, memory: 0B, network: 0B}
+              regionkey := tpch:regionkey
+
+
+
+

Limitations#

+

Aggregation pushdown does not support a number of more complex statements:

+ +
+
+
+

Join pushdown#

+

Join pushdown allows the connector to delegate the table join operation to the +underlying data source. This can result in performance gains, and allows Trino +to perform the remaining query processing on a smaller amount of data.

+

The specifics for the supported pushdown of table joins varies for each data +source, and therefore for each connector.

+

However, there are some generic conditions that must be met in order for a join +to be pushed down:

+
    +
  • all predicates that are part of the join must be possible to be pushed down

  • +
  • the tables in the join must be from the same catalog

  • +
+

You can verify if pushdown for a specific join is performed by looking at the +EXPLAIN plan of the query. The explain plan does not +show a Join operator, if the join is pushed down to the data source by the +connector:

+
EXPLAIN SELECT c.custkey, o.orderkey
+FROM orders o JOIN customer c ON c.custkey = o.custkey;
+
+
+

The following plan results from the PostgreSQL connector querying TPCH +data in a PostgreSQL database. It does not show any Join operator as a +result of the successful join push down.

+
Fragment 0 [SINGLE]
+    Output layout: [custkey, orderkey]
+    Output partitioning: SINGLE []
+    Output[custkey, orderkey]
+    │   Layout: [custkey:bigint, orderkey:bigint]
+    │   Estimates: {rows: ? (?), cpu: ?, memory: 0B, network: ?}
+    └─ RemoteSource[1]
+           Layout: [orderkey:bigint, custkey:bigint]
+
+Fragment 1 [SOURCE]
+    Output layout: [orderkey, custkey]
+    Output partitioning: SINGLE []
+    TableScan[postgres:Query[SELECT l."orderkey" AS "orderkey_0", l."custkey" AS "custkey_1", r."custkey" AS "custkey_2" FROM (SELECT "orderkey", "custkey" FROM "tpch"."orders") l INNER JOIN (SELECT "custkey" FROM "tpch"."customer") r O
+        Layout: [orderkey:bigint, custkey:bigint]
+        Estimates: {rows: ? (?), cpu: ?, memory: 0B, network: 0B}
+        orderkey := orderkey_0:bigint:int8
+        custkey := custkey_1:bigint:int8
+
+
+

It is typically beneficial to push down a join. Pushing down a join can also +increase the row count compared to the size of the input to the join. This +may impact performance.

+
+
+

Limit pushdown#

+

A LIMIT or FETCH FIRST clause reduces the number of returned records for a statement. +Limit pushdown enables a connector to push processing of such queries of +unsorted record to the underlying data source.

+

A pushdown of this clause can improve the performance of the query and +significantly reduce the amount of data transferred from the data source to +Trino.

+

Queries include sections such as LIMIT N or FETCH FIRST N ROWS.

+

Implementation and support is connector-specific since different data sources have varying capabilities.

+
+
+

Top-N pushdown#

+

The combination of a LIMIT or FETCH FIRST clause with an ORDER BY clause creates +a small set of records to return out of a large sorted dataset. It relies on the +order to determine which records need to be returned, and is therefore quite +different to optimize compared to a Limit pushdown.

+

The pushdown for such a query is called a Top-N pushdown, since the operation is +returning the top N rows. It enables a connector to push processing of such +queries to the underlying data source, and therefore significantly reduces the +amount of data transferred to and processed by Trino.

+

Queries include sections such as ORDER BY ... LIMIT N or ORDER BY ... FETCH FIRST N ROWS.

+

Implementation and support is connector-specific since different data sources +support different SQL syntax and processing.

+

For example, you can find two queries to learn how to identify Top-N pushdown behavior in the following section.

+

First, a concrete example of a Top-N pushdown query on top of a PostgreSQL database:

+
SELECT id, name
+FROM postgresql.public.company
+ORDER BY id
+LIMIT 5;
+
+
+

You can get the explain plan by prepending the above query with EXPLAIN:

+
EXPLAIN SELECT id, name
+FROM postgresql.public.company
+ORDER BY id
+LIMIT 5;
+
+
+
Fragment 0 [SINGLE]
+    Output layout: [id, name]
+    Output partitioning: SINGLE []
+    Stage Execution Strategy: UNGROUPED_EXECUTION
+    Output[id, name]
+    │   Layout: [id:integer, name:varchar]
+    │   Estimates: {rows: ? (?), cpu: ?, memory: 0B, network: ?}
+    └─ RemoteSource[1]
+           Layout: [id:integer, name:varchar]
+
+Fragment 1 [SOURCE]
+    Output layout: [id, name]
+    Output partitioning: SINGLE []
+    Stage Execution Strategy: UNGROUPED_EXECUTION
+    TableScan[postgresql:public.company public.company sortOrder=[id:integer:int4 ASC NULLS LAST] limit=5, grouped = false]
+        Layout: [id:integer, name:varchar]
+        Estimates: {rows: ? (?), cpu: ?, memory: 0B, network: 0B}
+        name := name:varchar:text
+        id := id:integer:int4
+
+
+

Second, an example of a Top-N query on the tpch connector which does not support +Top-N pushdown functionality:

+
SELECT custkey, name
+FROM tpch.sf1.customer
+ORDER BY custkey
+LIMIT 5;
+
+
+

The related query plan:

+
Fragment 0 [SINGLE]
+    Output layout: [custkey, name]
+    Output partitioning: SINGLE []
+    Stage Execution Strategy: UNGROUPED_EXECUTION
+    Output[custkey, name]
+    │   Layout: [custkey:bigint, name:varchar(25)]
+    │   Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?}
+    └─ TopN[5 by (custkey ASC NULLS LAST)]
+       │   Layout: [custkey:bigint, name:varchar(25)]
+       └─ LocalExchange[SINGLE] ()
+          │   Layout: [custkey:bigint, name:varchar(25)]
+          │   Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?}
+          └─ RemoteSource[1]
+                 Layout: [custkey:bigint, name:varchar(25)]
+
+Fragment 1 [SOURCE]
+    Output layout: [custkey, name]
+    Output partitioning: SINGLE []
+    Stage Execution Strategy: UNGROUPED_EXECUTION
+    TopNPartial[5 by (custkey ASC NULLS LAST)]
+    │   Layout: [custkey:bigint, name:varchar(25)]
+    └─ TableScan[tpch:customer:sf1.0, grouped = false]
+           Layout: [custkey:bigint, name:varchar(25)]
+           Estimates: {rows: 150000 (4.58MB), cpu: 4.58M, memory: 0B, network: 0B}
+           custkey := tpch:custkey
+           name := tpch:name
+
+
+

In the preceding query plan, the Top-N operation TopN[5 by (custkey ASC NULLS LAST)] +is being applied in the Fragment 0 by Trino and not by the source database.

+

Note that, compared to the query executed on top of the tpch connector, +the explain plan of the query applied on top of the postgresql connector +is missing the reference to the operation TopN[5 by (id ASC NULLS LAST)] +in the Fragment 0. +The absence of the TopN Trino operator in the Fragment 0 from the query plan +demonstrates that the query benefits of the Top-N pushdown optimization.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/optimizer/statistics.html b/430/optimizer/statistics.html new file mode 100644 index 000000000..aa751edfb --- /dev/null +++ b/430/optimizer/statistics.html @@ -0,0 +1,475 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Table statistics — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Table statistics#

+

Trino supports statistics based optimizations for queries. For a query to take +advantage of these optimizations, Trino must have statistical information for +the tables in that query.

+

Table statistics are provided to the query planner by connectors.

+
+

Available statistics#

+

The following statistics are available in Trino:

+
    +
  • For a table:

    +
      +
    • row count: the total number of rows in the table

    • +
    +
  • +
  • For each column in a table:

    +
      +
    • data size: the size of the data that needs to be read

    • +
    • nulls fraction: the fraction of null values

    • +
    • distinct value count: the number of distinct values

    • +
    • low value: the smallest value in the column

    • +
    • high value: the largest value in the column

    • +
    +
  • +
+

The set of statistics available for a particular query depends on the connector +being used and can also vary by table. For example, the +Hive connector does not currently provide statistics on data size.

+

Table statistics can be displayed via the Trino SQL interface using the +SHOW STATS command. For the Hive connector, refer to the +Hive connector documentation to learn how to update table +statistics.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/overview.html b/430/overview.html new file mode 100644 index 000000000..a3664656d --- /dev/null +++ b/430/overview.html @@ -0,0 +1,432 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Overview — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Overview#

+

Trino is a distributed SQL query engine designed to query large data sets +distributed over one or more heterogeneous data sources.

+ +
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/overview/concepts.html b/430/overview/concepts.html new file mode 100644 index 000000000..faf554a76 --- /dev/null +++ b/430/overview/concepts.html @@ -0,0 +1,740 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Trino concepts — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Trino concepts#

+
+

Overview#

+

To understand Trino, you must first understand the terms and concepts +used throughout the Trino documentation.

+

While it is easy to understand statements and queries, as an end-user +you should have familiarity with concepts such as stages and splits to +take full advantage of Trino to execute efficient queries. As a +Trino administrator or a Trino contributor you should understand how +Trino’s concepts of stages map to tasks and how tasks contain a set +of drivers which process data.

+

This section provides a solid definition for the core concepts +referenced throughout Trino, and these sections are sorted from most +general to most specific.

+
+

Note

+

The book Trino: The Definitive Guide and the research +paper Presto: SQL on Everything can +provide further information about Trino and the concepts in use.

+
+
+
+

Architecture#

+

Trino is a distributed query engine that processes data in parallel across +multiple servers. There are two types of Trino servers, +coordinators and +workers. The following sections describe these +servers and other components of Trino’s architecture.

+
+

Cluster#

+

A Trino cluster consists of a coordinator and +many workers. Users connect to the coordinator +with their SQL query tool. The coordinator collaborates with the +workers. The coordinator and the workers access the connected +data sources. This access is configured in +catalogs.

+

Processing each query is a stateful operation. The workload is orchestrated by +the coordinator and spread parallel across all workers in the cluster. Each node +runs Trino in one JVM instance, and processing is parallelized further using +threads.

+
+
+

Coordinator#

+

The Trino coordinator is the server that is responsible for parsing +statements, planning queries, and managing Trino worker nodes. It is +the “brain” of a Trino installation and is also the node to which a +client connects to submit statements for execution. Every Trino +installation must have a Trino coordinator alongside one or more +Trino workers. For development or testing purposes, a single +instance of Trino can be configured to perform both roles.

+

The coordinator keeps track of the activity on each worker and +coordinates the execution of a query. The coordinator creates +a logical model of a query involving a series of stages, which is then +translated into a series of connected tasks running on a cluster of +Trino workers.

+

Coordinators communicate with workers and clients using a REST API.

+
+
+

Worker#

+

A Trino worker is a server in a Trino installation, which is responsible +for executing tasks and processing data. Worker nodes fetch data from +connectors and exchange intermediate data with each other. The coordinator +is responsible for fetching results from the workers and returning the +final results to the client.

+

When a Trino worker process starts up, it advertises itself to the discovery +server in the coordinator, which makes it available to the Trino coordinator +for task execution.

+

Workers communicate with other workers and Trino coordinators +using a REST API.

+
+
+
+

Data sources#

+

Throughout this documentation, you’ll read terms such as connector, +catalog, schema, and table. These fundamental concepts cover Trino’s +model of a particular data source and are described in the following +section.

+
+

Connector#

+

A connector adapts Trino to a data source such as Hive or a +relational database. You can think of a connector the same way you +think of a driver for a database. It is an implementation of Trino’s +SPI, which allows Trino to interact +with a resource using a standard API.

+

Trino contains several built-in connectors: a connector for +JMX, a System +connector which provides access to built-in system tables, +a Hive connector, and a +TPCH connector designed to serve TPC-H benchmark +data. Many third-party developers have contributed connectors so that +Trino can access data in a variety of data sources.

+

Every catalog is associated with a specific connector. If you examine +a catalog configuration file, you see that each contains a +mandatory property connector.name, which is used by the catalog +manager to create a connector for a given catalog. It is possible +to have more than one catalog use the same connector to access two +different instances of a similar database. For example, if you have +two Hive clusters, you can configure two catalogs in a single Trino +cluster that both use the Hive connector, allowing you to query data +from both Hive clusters, even within the same SQL query.

+
+
+

Catalog#

+

A Trino catalog contains schemas and references a data source via a +connector. For example, you can configure a JMX catalog to provide +access to JMX information via the JMX connector. When you run SQL +statements in Trino, you are running them against one or more catalogs. +Other examples of catalogs include the Hive catalog to connect to a +Hive data source.

+

When addressing a table in Trino, the fully-qualified table name is +always rooted in a catalog. For example, a fully-qualified table name +of hive.test_data.test refers to the test table in the +test_data schema in the hive catalog.

+

Catalogs are defined in properties files stored in the Trino +configuration directory.

+
+
+

Schema#

+

Schemas are a way to organize tables. Together, a catalog and schema +define a set of tables that can be queried. When accessing Hive or a +relational database such as MySQL with Trino, a schema translates to +the same concept in the target database. Other types of connectors may +choose to organize tables into schemas in a way that makes sense for +the underlying data source.

+
+
+

Table#

+

A table is a set of unordered rows, which are organized into named columns +with types. This is the same as in any relational database. The mapping +from source data to tables is defined by the connector.

+
+
+
+

Query execution model#

+

Trino executes SQL statements and turns these statements into queries, +that are executed across a distributed cluster of coordinator and workers.

+
+

Statement#

+

Trino executes ANSI-compatible SQL statements. When the Trino +documentation refers to a statement, it is referring to statements as +defined in the ANSI SQL standard, which consists of clauses, +expressions, and predicates.

+

Some readers might be curious why this section lists separate concepts +for statements and queries. This is necessary because, in Trino, +statements simply refer to the textual representation of a statement written +in SQL. When a statement is executed, Trino creates a query along +with a query plan that is then distributed across a series of Trino +workers.

+
+
+

Query#

+

When Trino parses a statement, it converts it into a query and creates +a distributed query plan, which is then realized as a series of +interconnected stages running on Trino workers. When you retrieve +information about a query in Trino, you receive a snapshot of every +component that is involved in producing a result set in response to a +statement.

+

The difference between a statement and a query is simple. A statement +can be thought of as the SQL text that is passed to Trino, while a query +refers to the configuration and components instantiated to execute +that statement. A query encompasses stages, tasks, splits, connectors, +and other components and data sources working in concert to produce a +result.

+
+
+

Stage#

+

When Trino executes a query, it does so by breaking up the execution +into a hierarchy of stages. For example, if Trino needs to aggregate +data from one billion rows stored in Hive, it does so by creating a +root stage to aggregate the output of several other stages, all of +which are designed to implement different sections of a distributed +query plan.

+

The hierarchy of stages that comprises a query resembles a tree. +Every query has a root stage, which is responsible for aggregating +the output from other stages. Stages are what the coordinator uses to +model a distributed query plan, but stages themselves don’t run on +Trino workers.

+
+
+

Task#

+

As mentioned in the previous section, stages model a particular +section of a distributed query plan, but stages themselves don’t +execute on Trino workers. To understand how a stage is executed, +you need to understand that a stage is implemented as a series of +tasks distributed over a network of Trino workers.

+

Tasks are the “work horse” in the Trino architecture as a distributed +query plan is deconstructed into a series of stages, which are then +translated to tasks, which then act upon or process splits. A Trino +task has inputs and outputs, and just as a stage can be executed in +parallel by a series of tasks, a task is executing in parallel with a +series of drivers.

+
+
+

Split#

+

Tasks operate on splits, which are sections of a larger data +set. Stages at the lowest level of a distributed query plan retrieve +data via splits from connectors, and intermediate stages at a higher +level of a distributed query plan retrieve data from other stages.

+

When Trino is scheduling a query, the coordinator queries a +connector for a list of all splits that are available for a table. +The coordinator keeps track of which machines are running which tasks, +and what splits are being processed by which tasks.

+
+
+

Driver#

+

Tasks contain one or more parallel drivers. Drivers act upon data and +combine operators to produce output that is then aggregated by a task +and then delivered to another task in another stage. A driver is a +sequence of operator instances, or you can think of a driver as a +physical set of operators in memory. It is the lowest level of +parallelism in the Trino architecture. A driver has one input and +one output.

+
+
+

Operator#

+

An operator consumes, transforms and produces data. For example, a table +scan fetches data from a connector and produces data that can be consumed +by other operators, and a filter operator consumes data and produces a +subset by applying a predicate over the input data.

+
+
+

Exchange#

+

Exchanges transfer data between Trino nodes for different stages of +a query. Tasks produce data into an output buffer and consume data +from other tasks using an exchange client.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/overview/use-cases.html b/430/overview/use-cases.html new file mode 100644 index 000000000..4ecf91dbc --- /dev/null +++ b/430/overview/use-cases.html @@ -0,0 +1,461 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Use cases — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Use cases#

+

This section puts Trino into perspective, so that prospective +administrators and end users know what to expect from Trino.

+
+

What Trino is not#

+

Since Trino is being called a database by many members of the community, +it makes sense to begin with a definition of what Trino is not.

+

Do not mistake the fact that Trino understands SQL with it providing +the features of a standard database. Trino is not a general-purpose +relational database. It is not a replacement for databases like MySQL, +PostgreSQL or Oracle. Trino was not designed to handle Online +Transaction Processing (OLTP). This is also true for many other +databases designed and optimized for data warehousing or analytics.

+
+
+

What Trino is#

+

Trino is a tool designed to efficiently query vast amounts of data +using distributed queries. If you work with terabytes or petabytes of +data, you are likely using tools that interact with Hadoop and HDFS. +Trino was designed as an alternative to tools that query HDFS +using pipelines of MapReduce jobs, such as Hive or Pig, but Trino +is not limited to accessing HDFS. Trino can be and has been extended +to operate over different kinds of data sources, including traditional +relational databases and other data sources such as Cassandra.

+

Trino was designed to handle data warehousing and analytics: data analysis, +aggregating large amounts of data and producing reports. These workloads +are often classified as Online Analytical Processing (OLAP).

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release.html b/430/release.html new file mode 100644 index 000000000..ae17cb454 --- /dev/null +++ b/430/release.html @@ -0,0 +1,2910 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release notes — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release notes#

+
+

2023#

+ +
+
+

2022#

+ +
+
+

2021#

+ +
+
+

2020#

+ +
+
+

2019#

+ +
+
+

Before 2019#

+
+ +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.100.html b/430/release/release-0.100.html new file mode 100644 index 000000000..179b39cc5 --- /dev/null +++ b/430/release/release-0.100.html @@ -0,0 +1,2574 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.100 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.100#

+
+

System connector#

+

The System connector now works like other connectors: global system +tables are only available in the system catalog, rather than in a special +schema that is available in every catalog. Additionally, connectors may now +provide system tables that are available within that connector’s catalog by +implementing the getSystemTables() method on the Connector interface.

+
+
+

General#

+
    +
  • Fix %f specifier in date_format() and date_parse().

  • +
  • Add WITH ORDINALITY support to UNNEST.

  • +
  • Add array_distinct() function.

  • +
  • Add split() function.

  • +
  • Add degrees() and radians() functions.

  • +
  • Add to_base() and from_base() functions.

  • +
  • Rename config property task.shard.max-threads to task.max-worker-threads. +This property sets the number of threads used to concurrently process splits. +The old property name is deprecated and will be removed in a future release.

  • +
  • Fix referencing NULL values in ROW.

  • +
  • Make MAP comparable.

  • +
  • Fix leak of tasks blocked during query teardown.

  • +
  • Improve query queue config validation.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.101.html b/430/release/release-0.101.html new file mode 100644 index 000000000..9164d5a2f --- /dev/null +++ b/430/release/release-0.101.html @@ -0,0 +1,2636 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.101 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.101#

+
+

General#

+
    +
  • Add support for CREATE TABLE (in addition to CREATE TABLE AS).

  • +
  • Add IF EXISTS support to DROP TABLE and DROP VIEW.

  • +
  • Add array_agg() function.

  • +
  • Add array_intersect() function.

  • +
  • Add array_position() function.

  • +
  • Add regexp_split() function.

  • +
  • Add support for millisecond to date_diff() and date_add().

  • +
  • Fix excessive memory usage in map_agg().

  • +
  • Fix excessive memory usage in queries that perform partitioned top-N operations +with row_number().

  • +
  • Optimize ARRAY comparison operators.

  • +
  • Fix analysis of UNION queries for tables with hidden columns.

  • +
  • Fix JOIN associativity to be left-associative instead of right-associative.

  • +
  • Add source column to runtime.queries table in System connector.

  • +
  • Add coordinator column to runtime.nodes table in System connector.

  • +
  • Add errorCode, errorName and errorType to error object in REST API +(errorCode previously existed but was always zero).

  • +
  • Fix DatabaseMetaData.getIdentifierQuoteString() in JDBC driver.

  • +
  • Handle thread interruption in JDBC driver ResultSet.

  • +
  • Add history command and support for running previous commands via !n to the CLI.

  • +
  • Change Driver to make as much progress as possible before blocking. This improves +responsiveness of some limit queries.

  • +
  • Add predicate push down support to JMX connector.

  • +
  • Add support for unary PLUS operator.

  • +
  • Improve scheduling speed by reducing lock contention.

  • +
  • Extend optimizer to understand physical properties such as local grouping and sorting.

  • +
  • Add support for streaming execution of window functions.

  • +
  • Make UNION run partitioned, if underlying plan is partitioned.

  • +
  • Add hash_partition_count session property to control hash partitions.

  • +
+
+
+

Web UI#

+

The main page of the web UI has been completely rewritten to use ReactJS. It also has +a number of new features, such as the ability to pause auto-refresh via the “Z” key and +also with a toggle in the UI.

+
+
+

Hive#

+
    +
  • Add support for connecting to S3 using EC2 instance credentials. +This feature is enabled by default. To disable it, set +hive.s3.use-instance-credentials=false in your Hive catalog properties file.

  • +
  • Treat ORC files as splittable.

  • +
  • Change PrestoS3FileSystem to use lazy seeks, which improves ORC performance.

  • +
  • Fix ORC DOUBLE statistic for columns containing NaN.

  • +
  • Lower the Hive metadata refresh interval from two minutes to one second.

  • +
  • Invalidate Hive metadata cache for failed operations.

  • +
  • Support s3a file system scheme.

  • +
  • Fix discovery of splits to correctly backoff when the queue is full.

  • +
  • Add support for non-canonical Parquet structs.

  • +
  • Add support for accessing Parquet columns by name. By default, columns in Parquet +files are accessed by their ordinal position in the Hive table definition. To access +columns based on the names recorded in the Parquet file, set +hive.parquet.use-column-names=true in your Hive catalog properties file.

  • +
  • Add JMX stats to PrestoS3FileSystem.

  • +
  • Add hive.recursive-directories config option to recursively scan +partition directories for data.

  • +
+
+
+

SPI#

+
    +
  • Add connector callback for rollback of INSERT and CREATE TABLE AS.

  • +
  • Introduce an abstraction for representing physical organizations of a table +and describing properties such as partitioning, grouping, predicate and columns. +ConnectorPartition and related interfaces are deprecated and will be removed +in a future version.

  • +
  • Rename ConnectorColumnHandle to ColumnHandle.

  • +
+
+

Note

+

This is a backwards incompatible change with the previous connector SPI. +If you have written a connector, you will need to update your code +before deploying this release.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.102.html b/430/release/release-0.102.html new file mode 100644 index 000000000..c76c96b40 --- /dev/null +++ b/430/release/release-0.102.html @@ -0,0 +1,2605 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.102 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.102#

+
+

Unicode support#

+

All string functions have been updated to support Unicode. The functions assume +that the string contains valid UTF-8 encoded code points. There are no explicit +checks for valid UTF-8, and the functions may return incorrect results on +invalid UTF-8. Invalid UTF-8 data can be corrected with from_utf8().

+

Additionally, the functions operate on Unicode code points and not user visible +characters (or grapheme clusters). Some languages combine multiple code points +into a single user-perceived character, the basic unit of a writing system for a +language, but the functions will treat each code point as a separate unit.

+
+
+

Regular expression functions#

+

All Regular expression functions have been rewritten to improve performance. +The new versions are often twice as fast and in some cases can be many +orders of magnitude faster (due to removal of quadratic behavior). +This change introduced some minor incompatibilities that are explained +in the documentation for the functions.

+
+
+

General#

+
    +
  • Add support for partitioned right outer joins, which allows for larger tables to +be joined on the inner side.

  • +
  • Add support for full outer joins.

  • +
  • Support returning booleans as numbers in JDBC driver

  • +
  • Fix contains() to return NULL if the value was not found, but a NULL was.

  • +
  • Fix nested ROW rendering in DESCRIBE.

  • +
  • Add array_join().

  • +
  • Optimize map subscript operator.

  • +
  • Add from_utf8() and to_utf8() functions.

  • +
  • Add task_writer_count session property to set task.writer-count.

  • +
  • Add cast from ARRAY(F) to ARRAY(T).

  • +
  • Extend implicit coercions to ARRAY element types.

  • +
  • Implement implicit coercions in VALUES expressions.

  • +
  • Fix potential deadlock in scheduler.

  • +
+
+
+

Hive#

+
    +
  • Collect more metrics from PrestoS3FileSystem.

  • +
  • Retry when seeking in PrestoS3FileSystem.

  • +
  • Ignore InvalidRange error in PrestoS3FileSystem.

  • +
  • Implement rename and delete in PrestoS3FileSystem.

  • +
  • Fix assertion failure when running SHOW TABLES FROM schema.

  • +
  • Fix S3 socket leak when reading ORC files.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.103.html b/430/release/release-0.103.html new file mode 100644 index 000000000..b3bdaa30a --- /dev/null +++ b/430/release/release-0.103.html @@ -0,0 +1,2607 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.103 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.103#

+
+

Cluster resource management#

+

There is a new cluster resource manager, which can be enabled via the +experimental.cluster-memory-manager-enabled flag. Currently, the only +resource that’s tracked is memory, and the cluster resource manager guarantees +that the cluster will not deadlock waiting for memory. However, in a low memory +situation it is possible that only one query will make progress. Memory limits can +now be configured via query.max-memory which controls the total distributed +memory a query may use and query.max-memory-per-node which limits the amount +of memory a query may use on any one node. On each worker, the +resources.reserved-system-memory flags controls how much memory is reserved +for internal Presto data structures and temporary allocations.

+
+
+

Task parallelism#

+

Queries involving a large number of aggregations or a large hash table for a +join can be slow due to single threaded execution in the intermediate stages. +This release adds experimental configuration and session properties to execute +this single threaded work in parallel. Depending on the exact query this may +reduce wall time, but will likely increase CPU usage.

+

Use the configuration parameter task.default-concurrency or the session +property task_default_concurrency to set the default number of parallel +workers to use for join probes, hash builds and final aggregations. +Additionally, the session properties task_join_concurrency, +task_hash_build_concurrency and task_aggregation_concurrency can be +used to control the parallelism for each type of work.

+

This is an experimental feature and will likely change in a future release. It +is also expected that this will eventually be handled automatically by the +query planner and these options will be removed entirely.

+
+
+

Hive#

+
    +
  • Removed the hive.max-split-iterator-threads parameter and renamed +hive.max-global-split-iterator-threads to hive.max-split-iterator-threads.

  • +
  • Fix excessive object creation when querying tables with a large number of partitions.

  • +
  • Do not retry requests when an S3 path is not found.

  • +
+
+
+

General#

+
    +
  • Add array_remove().

  • +
  • Fix NPE in max_by() and min_by() caused when few rows were present in the aggregation.

  • +
  • Reduce memory usage of map_agg().

  • +
  • Change HTTP client defaults: 2 second idle timeout, 10 second request +timeout and 250 connections per host.

  • +
  • Add SQL command autocompletion to CLI.

  • +
  • Increase CLI history file size.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.104.html b/430/release/release-0.104.html new file mode 100644 index 000000000..a1f50818f --- /dev/null +++ b/430/release/release-0.104.html @@ -0,0 +1,2577 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.104 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.104#

+
+

General#

+
    +
  • Handle thread interruption in StatementClient.

  • +
  • Fix CLI hang when server becomes unreachable during a query.

  • +
  • Add covar_pop(), covar_samp(), corr(), regr_slope(), +and regr_intercept() functions.

  • +
  • Fix potential deadlock in cluster memory manager.

  • +
  • Add a visualization of query execution timeline.

  • +
  • Allow mixed case in input to from_hex().

  • +
  • Display “BLOCKED” state in web UI.

  • +
  • Reduce CPU usage in coordinator.

  • +
  • Fix excess object retention in workers due to long running queries.

  • +
  • Reduce memory usage of array_distinct().

  • +
  • Add optimizer for projection push down which can +improve the performance of certain query shapes.

  • +
  • Improve query performance by storing pre-partitioned pages.

  • +
  • Support TIMESTAMP for first_value(), last_value(), +nth_value(), lead() and lag().

  • +
+
+
+

Hive#

+
    +
  • Upgrade to Parquet 1.6.0.

  • +
  • Collect request time and retry statistics in PrestoS3FileSystem.

  • +
  • Fix retry attempt counting for S3.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.105.html b/430/release/release-0.105.html new file mode 100644 index 000000000..9b06021c2 --- /dev/null +++ b/430/release/release-0.105.html @@ -0,0 +1,2569 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.105 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.105#

+
+

General#

+
    +
  • Fix issue which can cause queries to be blocked permanently.

  • +
  • Close connections correctly in JDBC connectors.

  • +
  • Add implicit coercions for values of equi-join criteria.

  • +
  • Fix detection of window function calls without an OVER clause.

  • +
+
+
+

SPI#

+
    +
  • Remove ordinalPosition from ColumnMetadata.

  • +
+
+

Note

+

This is a backwards incompatible change with the previous connector SPI. +If you have written a connector, you will need to update your code +before deploying this release.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.106.html b/430/release/release-0.106.html new file mode 100644 index 000000000..e40ed0c8d --- /dev/null +++ b/430/release/release-0.106.html @@ -0,0 +1,2557 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.106 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.106#

+
+

General#

+
    +
  • Parallelize startup of table scan task splits.

  • +
  • Fixed index join driver resource leak.

  • +
  • Improve memory accounting for JOINs and GROUP BYs.

  • +
  • Improve CPU efficiency of coordinator.

  • +
  • Added Asia/Chita, Asia/Srednekolymsk, and Pacific/Bougainville time zones.

  • +
  • Fix task leak caused by race condition in stage state machine.

  • +
  • Fix blocking in Hive split source.

  • +
  • Free resources sooner for queries that finish prematurely.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.107.html b/430/release/release-0.107.html new file mode 100644 index 000000000..6a18490c6 --- /dev/null +++ b/430/release/release-0.107.html @@ -0,0 +1,2558 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.107 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.107#

+
+

General#

+
    +
  • Added query_max_memory session property. Note: this session property cannot +increase the limit above the limit set by the query.max-memory configuration option.

  • +
  • Fixed task leak caused by queries that finish early, such as a LIMIT query +or cancelled query, when the cluster is under high load.

  • +
  • Added task.info-refresh-max-wait to configure task info freshness.

  • +
  • Add support for DELETE to language and connector SPI.

  • +
  • Reenable error classification code for syntax errors.

  • +
  • Fix out of bounds exception in lower() and upper() +when the string contains the code point U+10FFFF.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.108.html b/430/release/release-0.108.html new file mode 100644 index 000000000..60983bac6 --- /dev/null +++ b/430/release/release-0.108.html @@ -0,0 +1,2585 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.108 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.108#

+
+

General#

+
    +
  • Fix incorrect query results when a window function follows a row_number() +function and both are partitioned on the same column(s).

  • +
  • Fix planning issue where queries that apply a false predicate +to the result of a non-grouped aggregation produce incorrect results.

  • +
  • Fix exception when ORDER BY clause contains duplicate columns.

  • +
  • Fix issue where a query (read or write) that should fail can instead +complete successfully with zero rows.

  • +
  • Add normalize(), from_iso8601_timestamp(), from_iso8601_date() +and to_iso8601() functions.

  • +
  • Add support for position() syntax.

  • +
  • Add Teradata compatibility functions: index(), char2hexint(), +to_char(), to_date() and to_timestamp().

  • +
  • Make ctrl-C in CLI cancel the query (rather than a partial cancel).

  • +
  • Allow calling Connection.setReadOnly(false) in the JDBC driver. +The read-only status for the connection is currently ignored.

  • +
  • Add missing CAST from VARCHAR to TIMESTAMP WITH TIME ZONE.

  • +
  • Allow optional time zone in CAST from VARCHAR to TIMESTAMP and +TIMESTAMP WITH TIME ZONE.

  • +
  • Trim values when converting from VARCHAR to date/time types.

  • +
  • Add support for fixed time zones +00:00 and -00:00.

  • +
  • Properly account for query memory when using the row_number() function.

  • +
  • Skip execution of inner join when the join target is empty.

  • +
  • Improve query detail UI page.

  • +
  • Fix printing of table layouts in EXPLAIN.

  • +
  • Add Black Hole connector.

  • +
+
+
+

Cassandra#

+
    +
  • Randomly select Cassandra node for split generation.

  • +
  • Fix handling of UUID partition keys.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.109.html b/430/release/release-0.109.html new file mode 100644 index 000000000..0e6d2f712 --- /dev/null +++ b/430/release/release-0.109.html @@ -0,0 +1,2572 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.109 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.109#

+
+

General#

+
    +
  • Add slice(), md5(), array_min() and array_max() functions.

  • +
  • Fix bug that could cause queries submitted soon after startup to hang forever.

  • +
  • Fix bug that could cause JOIN queries to hang forever, if the right side of +the JOIN had too little data or skewed data.

  • +
  • Improve index join planning heuristics to favor streaming execution.

  • +
  • Improve validation of date/time literals.

  • +
  • Produce RPM package for Presto server.

  • +
  • Always redistribute data when writing tables to avoid skew. This can +be disabled by setting the session property redistribute_writes +or the config property redistribute-writes to false.

  • +
+
+
+

Remove “Big Query” support#

+

The experimental support for big queries has been removed in favor of +the new resource manager which can be enabled via the +experimental.cluster-memory-manager-enabled config option. +The experimental_big_query session property and the following config +options are no longer supported: experimental.big-query-initial-hash-partitions, +experimental.max-concurrent-big-queries and experimental.max-queued-big-queries.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.110.html b/430/release/release-0.110.html new file mode 100644 index 000000000..c655faa38 --- /dev/null +++ b/430/release/release-0.110.html @@ -0,0 +1,2559 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.110 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.110#

+
+

General#

+
    +
  • Fix result truncation bug in window function row_number() when performing a +partitioned top-N that chooses the maximum or minimum N rows. For example:

    +
    SELECT * FROM (
    +    SELECT row_number() OVER (PARTITION BY orderstatus ORDER BY orderdate) AS rn,
    +        custkey, orderdate, orderstatus
    +    FROM orders
    +) WHERE rn <= 5;
    +
    +
    +
  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.111.html b/430/release/release-0.111.html new file mode 100644 index 000000000..f516dc3c7 --- /dev/null +++ b/430/release/release-0.111.html @@ -0,0 +1,2556 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.111 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.111#

+
+

General#

+
    +
  • Add histogram() function.

  • +
  • Optimize CASE expressions on a constant.

  • +
  • Add basic support for IF NOT EXISTS for CREATE TABLE.

  • +
  • Semi-joins are hash-partitioned if distributed_join is turned on.

  • +
  • Add support for partial cast from JSON. For example, json can be cast to array(json), map(varchar, json), etc.

  • +
  • Add implicit coercions for UNION.

  • +
  • Expose query stats in the JDBC driver ResultSet.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.112.html b/430/release/release-0.112.html new file mode 100644 index 000000000..c3635d5c2 --- /dev/null +++ b/430/release/release-0.112.html @@ -0,0 +1,2568 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.112 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.112#

+
+

General#

+
    +
  • Fix incorrect handling of filters and limits in row_number() optimizer. +This caused certain query shapes to produce incorrect results.

  • +
  • Fix non-string object arrays in JMX connector.

  • +
+
+
+

Hive#

+
    +
  • Tables created using CREATE TABLE (not CREATE TABLE AS) +had invalid metadata and were not readable.

  • +
  • Improve performance of IN and OR clauses when reading ORC data. +Previously, the ranges for a column were always compacted into a single range +before being passed to the reader, preventing the reader from taking full +advantage of row skipping. The compaction only happens now if the number of +ranges exceeds the hive.domain-compaction-threshold config property.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.113.html b/430/release/release-0.113.html new file mode 100644 index 000000000..5382de9b2 --- /dev/null +++ b/430/release/release-0.113.html @@ -0,0 +1,2628 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.113 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.113#

+
+

Warning

+

The ORC reader in the Hive connector is broken in this release.

+
+
+

Cluster resource management#

+

The cluster resource manager announced in Release 0.103 is now enabled by default. +You can disable it with the experimental.cluster-memory-manager-enabled flag. +Memory limits can now be configured via query.max-memory which controls the total distributed +memory a query may use and query.max-memory-per-node which limits the amount +of memory a query may use on any one node. On each worker, the +resources.reserved-system-memory config property controls how much memory is reserved +for internal Presto data structures and temporary allocations.

+
+
+

Session properties#

+

All session properties have a type, default value, and description. +The value for SET SESSION can now be any constant expression, and +the SHOW SESSION command prints the current effective value and +default value for all session properties.

+

This type safety extends to the SPI where properties +can be validated and converted to any Java type using +SessionPropertyMetadata. For an example, see HiveSessionProperties.

+
+

Note

+

This is a backwards incompatible change with the previous connector SPI. +If you have written a connector that uses session properties, you will need +to update your code to declare the properties in the Connector +implementation and callers of ConnectorSession.getProperty() will now +need the expected Java type of the property.

+
+
+
+

General#

+ +
+
+

Hive#

+
    +
  • Fix the Hive metadata cache to properly handle negative responses. +This makes the background refresh work properly by clearing the cached +metadata entries when an object is dropped outside of Presto. +In particular, this fixes the common case where a table is dropped using +Hive but Presto thinks it still exists.

  • +
  • Fix metastore socket leak when SOCKS connect fails.

  • +
+
+
+

SPI#

+
    +
  • Changed the internal representation of structural types.

  • +
+
+

Note

+

This is a backwards incompatible change with the previous connector SPI. +If you have written a connector that uses structural types, you will need +to update your code to the new APIs.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.114.html b/430/release/release-0.114.html new file mode 100644 index 000000000..da8d8b24a --- /dev/null +++ b/430/release/release-0.114.html @@ -0,0 +1,2561 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.114 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.114#

+
+

General#

+ +
+
+

Hive#

+
    +
  • Fix ORC reader for Hive connector.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.115.html b/430/release/release-0.115.html new file mode 100644 index 000000000..f96cfb35a --- /dev/null +++ b/430/release/release-0.115.html @@ -0,0 +1,2565 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.115 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.115#

+
+

General#

+
    +
  • Fix an issue with hierarchical queue rules where queries could be rejected after being accepted.

  • +
  • Add sha1(), sha256() and sha512() functions.

  • +
  • Add power() as an alias for pow().

  • +
  • Add support for LIMIT ALL syntax.

  • +
+
+
+

Hive#

+
    +
  • Fix a race condition which could cause queries to finish without reading all the data.

  • +
  • Fix a bug in Parquet reader that causes failures while reading lists that has an element +schema name other than array_element in its Parquet-level schema.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.116.html b/430/release/release-0.116.html new file mode 100644 index 000000000..d257e7de4 --- /dev/null +++ b/430/release/release-0.116.html @@ -0,0 +1,2593 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.116 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 0.116#

+
+

Cast between JSON and VARCHAR#

+

Casts of both directions between JSON and VARCHAR have been removed. If you +have such casts in your scripts or views, they will fail with a message when +you move to release 0.116. To get the semantics of the current casts, use:

+
    +
  • JSON_PARSE(x) instead of CAST(x as JSON)

  • +
  • JSON_FORMAT(x) instead of CAST(x as VARCHAR)

  • +
+

In a future release, we intend to reintroduce casts between JSON and VARCHAR +along with other casts involving JSON. The semantics of the new JSON and +VARCHAR cast will be consistent with the other casts being introduced. But it +will be different from the semantics in 0.115 and before. When that comes, +cast between JSON and VARCHAR in old scripts and views will produce unexpected +result.

+
+
+

Cluster memory manager improvements#

+

The cluster memory manager now has a low memory killer. If the cluster runs low +on memory, the killer will kill queries to improve throughput. It can be enabled +with the query.low-memory-killer.enabled config flag, and the delay between +when the cluster runs low on memory and when the killer will be invoked can be +configured with the query.low-memory-killer.delay option.

+
+
+

General#

+
    +
  • Add multimap_agg() function.

  • +
  • Add checksum() function.

  • +
  • Add max() and min() that takes a second argument and produces +n largest or n smallest values.

  • +
  • Add query_max_run_time session property and query.max-run-time +config. Queries are failed after the specified duration.

  • +
  • Removed experimental.cluster-memory-manager-enabled config. The cluster +memory manager is now always enabled.

  • +
  • Removed task.max-memory config.

  • +
  • optimizer.optimize-hash-generation and distributed-joins-enabled are +both enabled by default now.

  • +
  • Add optimization for IF on a constant condition.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.117.html b/430/release/release-0.117.html new file mode 100644 index 000000000..a74e1c916 --- /dev/null +++ b/430/release/release-0.117.html @@ -0,0 +1,2554 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.117 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.117#

+
+

General#

+
    +
  • Add back casts between JSON and VARCHAR to provide an easier migration path +to json_parse() and json_format(). These will be removed in a +future release.

  • +
  • Fix bug in semi joins and group bys on a single BIGINT column where +0 could match NULL.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.118.html b/430/release/release-0.118.html new file mode 100644 index 000000000..0a90684fa --- /dev/null +++ b/430/release/release-0.118.html @@ -0,0 +1,2569 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.118 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.118#

+
+

General#

+
    +
  • Fix planning error for UNION queries that require implicit coercions.

  • +
  • Fix null pointer exception when using checksum().

  • +
  • Fix completion condition for SqlTask that can cause queries to be blocked.

  • +
+
+
+

Authorization#

+

We’ve added experimental support for authorization of SQL queries in Presto. +This is currently only supported by the Hive connector. You can enable Hive +checks by setting the hive.security property to none, read-only, +or sql-standard.

+
+

Note

+

The authentication support is experimental and only lightly tested. We are +actively working on this feature, so expect backwards incompatible changes. +See the ConnectorAccessControl interface the SPI for details.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.119.html b/430/release/release-0.119.html new file mode 100644 index 000000000..99549725b --- /dev/null +++ b/430/release/release-0.119.html @@ -0,0 +1,2646 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.119 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.119#

+
+

General#

+
    +
  • Add Redis connector.

  • +
  • Add geometric_mean() function.

  • +
  • Fix restoring interrupt status in StatementClient.

  • +
  • Support getting server version in JDBC driver.

  • +
  • Improve correctness and compliance of JDBC DatabaseMetaData.

  • +
  • Catalog and schema are now optional on the server. This allows connecting +and executing metadata commands or queries that use fully qualified names. +Previously, the CLI and JDBC driver would use a catalog and schema named +default if they were not specified.

  • +
  • Fix scheduler handling of partially canceled queries.

  • +
  • Execute views with the permissions of the view owner.

  • +
  • Replaced the task.http-notification-threads config option with two +independent options: task.http-response-threads and task.http-timeout-threads.

  • +
  • Improve handling of negated expressions in join criteria.

  • +
  • Fix arbitrary(), max_by() and min_by() functions when used +with an array, map or row type.

  • +
  • Fix union coercion when the same constant or column appears more than once on +the same side.

  • +
  • Support RENAME COLUMN in ALTER TABLE.

  • +
+
+
+

SPI#

+
    +
  • Add more system table distribution modes.

  • +
  • Add owner to view metadata.

  • +
+
+

Note

+

These are backwards incompatible changes with the previous connector SPI. +If you have written a connector, you may need to update your code to the +new APIs.

+
+
+
+

CLI#

+
    +
  • Fix handling of full width characters.

  • +
  • Skip printing query URL if terminal is too narrow.

  • +
  • Allow performing a partial query cancel using ctrl-P.

  • +
  • Allow toggling debug mode during query by pressing D.

  • +
  • Fix handling of query abortion after result has been partially received.

  • +
  • Fix handling of ctrl-C when displaying results without a pager.

  • +
+
+
+

Verifier#

+
    +
  • Add expected-double-precision config to specify the expected level of +precision when comparing double values.

  • +
  • Return non-zero exit code when there are failures.

  • +
+
+
+

Cassandra#

+
    +
  • Add support for Cassandra blob types.

  • +
+
+
+

Hive#

+
    +
  • Support adding and renaming columns using ALTER TABLE.

  • +
  • Automatically configure the S3 region when running in EC2.

  • +
  • Allow configuring multiple Hive metastores for high availability.

  • +
  • Add support for TIMESTAMP and VARBINARY in Parquet.

  • +
+
+
+

MySQL and PostgreSQL#

+
    +
  • Enable streaming results instead of buffering everything in memory.

  • +
  • Fix handling of pattern characters when matching table or column names.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.120.html b/430/release/release-0.120.html new file mode 100644 index 000000000..31204626d --- /dev/null +++ b/430/release/release-0.120.html @@ -0,0 +1,2542 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.120 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.120#

+
+

Warning

+

This release is broken and should not be used.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.121.html b/430/release/release-0.121.html new file mode 100644 index 000000000..be42781e9 --- /dev/null +++ b/430/release/release-0.121.html @@ -0,0 +1,2553 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.121 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.121#

+
+

General#

+
    +
  • Fix regression that causes task scheduler to not retry requests in some cases.

  • +
  • Throttle task info refresher on errors.

  • +
  • Fix planning failure that prevented the use of large IN lists.

  • +
  • Fix comparison of array(T) where T is a comparable, non-orderable type.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.122.html b/430/release/release-0.122.html new file mode 100644 index 000000000..8018753f3 --- /dev/null +++ b/430/release/release-0.122.html @@ -0,0 +1,2564 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.122 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.122#

+
+

Warning

+

There is a bug in this release that will cause queries to fail when the +optimizer.optimize-hash-generation config is disabled.

+
+
+

General#

+
    +
  • The deprecated casts between JSON and VARCHAR will now fail and provide the +user with instructions to migrate their query. For more details, see +Release 0.116.

  • +
  • Fix NoSuchElementException when cross join is used inside IN query.

  • +
  • Fix GROUP BY to support maps of structural types.

  • +
  • The web interface now displays a lock icon next to authenticated users.

  • +
  • The min_by() and max_by() aggregations now have an additional form +that return multiple values.

  • +
  • Fix incorrect results when using IN lists of more than 1000 elements of +timestamp with time zone, time with time zone or structural types.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.123.html b/430/release/release-0.123.html new file mode 100644 index 000000000..bf8e5566c --- /dev/null +++ b/430/release/release-0.123.html @@ -0,0 +1,2604 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.123 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.123#

+
+

General#

+
    +
  • Remove node-scheduler.location-aware-scheduling-enabled config.

  • +
  • Fixed query failures that occur when the optimizer.optimize-hash-generation +config is disabled.

  • +
  • Fix exception when using the ResultSet returned from the +DatabaseMetaData.getColumns method in the JDBC driver.

  • +
  • Increase default value of failure-detector.threshold config.

  • +
  • Fix race in queueing system which could cause queries to fail with +“Entering secondary queue failed”.

  • +
  • Fix issue with histogram() that can cause failures or incorrect results +when there are more than ten buckets.

  • +
  • Optimize execution of cross join.

  • +
  • Run Presto server as presto user in RPM init scripts.

  • +
+
+
+

Table properties#

+

When creating tables with CREATE TABLE or CREATE TABLE AS, +you can now add connector specific properties to the new table. For example, when +creating a Hive table you can specify the file format. To list all available table, +properties, run the following query:

+
SELECT * FROM system.metadata.table_properties
+
+
+
+
+

Hive#

+

We have implemented INSERT and DELETE for Hive. Both INSERT and CREATE +statements support partitioned tables. For example, to create a partitioned table +execute the following:

+
CREATE TABLE orders (
+   order_date VARCHAR,
+   order_region VARCHAR,
+   order_id BIGINT,
+   order_info VARCHAR
+) WITH (partitioned_by = ARRAY['order_date', 'order_region'])
+
+
+

To DELETE from a Hive table, you must specify a WHERE clause that matches +entire partitions. For example, to delete from the above table, execute the following:

+
DELETE FROM orders
+WHERE order_date = '2015-10-15' AND order_region = 'APAC'
+
+
+
+

Note

+

Currently, Hive deletion is only supported for partitioned tables. +Additionally, partition keys must be of type VARCHAR.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.124.html b/430/release/release-0.124.html new file mode 100644 index 000000000..a81b5ed12 --- /dev/null +++ b/430/release/release-0.124.html @@ -0,0 +1,2600 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.124 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.124#

+
+

General#

+
    +
  • Fix race in memory tracking of JOIN which could cause the cluster to become over +committed and possibly crash.

  • +
  • The approx_percentile() aggregation now also accepts an array of percentages.

  • +
  • Allow nested row type references.

  • +
  • Fix correctness for some queries with IN lists. When all constants in the +list are in the range of 32-bit signed integers but the test value can be +outside of the range, true may be produced when the correct result should +be false.

  • +
  • Fail queries submitted while coordinator is starting.

  • +
  • Add JMX stats to track authentication and authorization successes and failures.

  • +
  • Add configuration support for the system access control plugin. The system access +controller can be selected and configured using etc/access-control.properties. +Note that Presto currently does not ship with any system access controller +implementations.

  • +
  • Add support for WITH NO DATA syntax in CREATE TABLE ... AS SELECT.

  • +
  • Fix issue where invalid plans are generated for queries with multiple aggregations +that require input values to be cast in different ways.

  • +
  • Fix performance issue due to redundant processing in queries involving DISTINCT +and LIMIT.

  • +
  • Add optimization that can reduce the amount of data sent over the network +for grouped aggregation queries. This feature can be enabled by +optimizer.use-intermediate-aggregations config property or +task_intermediate_aggregation session property.

  • +
+
+
+

Hive#

+
    +
  • Do not count expected exceptions as errors in the Hive metastore client stats.

  • +
  • Improve performance when reading ORC files with many tiny stripes.

  • +
+
+
+

Verifier#

+
    +
  • Add support for pre and post control and test queries.

  • +
+

If you are upgrading, you need to alter your verifier_queries table:

+
ALTER TABLE verifier_queries ADD COLUMN test_postqueries text;
+ALTER TABLE verifier_queries ADD COLUMN test_prequeries text;
+ALTER TABLE verifier_queries ADD COLUMN control_postqueries text;
+ALTER TABLE verifier_queries ADD COLUMN control_prequeries text;
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.125.html b/430/release/release-0.125.html new file mode 100644 index 000000000..e35ce1af9 --- /dev/null +++ b/430/release/release-0.125.html @@ -0,0 +1,2553 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.125 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.125#

+
+

General#

+
    +
  • Fix an issue where certain operations such as GROUP BY, DISTINCT, etc. on the +output of a RIGHT or FULL OUTER JOIN can return incorrect results if they reference columns +from the left relation that are also used in the join clause, and not every row from the right relation +has a match.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.126.html b/430/release/release-0.126.html new file mode 100644 index 000000000..e51f44ff5 --- /dev/null +++ b/430/release/release-0.126.html @@ -0,0 +1,2598 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.126 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.126#

+
+

General#

+
    +
  • Add error location information (line and column number) for semantic errors.

  • +
  • Fix a CLI crash during tab-completion when no schema is currently selected.

  • +
  • Fix reset of session properties in CLI when running USE.

  • +
  • Fix occasional query planning failure due to a bug in the projection +push down optimizer.

  • +
  • Fix a parsing issue when expressions contain the form POSITION(x in (y)).

  • +
  • Add a new version of approx_percentile() that takes an accuracy +parameter.

  • +
  • Allow specifying columns names in INSERT queries.

  • +
  • Add field_length table property to blackhole connector to control the +size of generated VARCHAR and VARBINARY fields.

  • +
  • Bundle Teradata functions plugin in server package.

  • +
  • Improve handling of physical properties which can increase performance for +queries involving window functions.

  • +
  • Add ability to control whether index join lookups and caching are shared +within a task. This allows us to optimize for index cache hits or for more +CPU parallelism. This option is toggled by the task.share-index-loading +config property or the task_share_index_loading session property.

  • +
  • Add Tableau web connector.

  • +
  • Improve performance of queries that use an IN expression with a large +list of constant values.

  • +
  • Enable connector predicate push down for all comparable and equatable types.

  • +
  • Fix query planning failure when using certain operations such as GROUP BY, +DISTINCT, etc. on the output columns of UNNEST.

  • +
  • In ExchangeClient set maxResponseSize to be slightly smaller than +the configured value. This reduces the possibility of encountering +PageTooLargeException.

  • +
  • Fix memory leak in coordinator.

  • +
  • Add validation for names of table properties.

  • +
+
+
+

Hive#

+
    +
  • Fix reading structural types containing nulls in Parquet.

  • +
  • Fix writing DATE type when timezone offset is negative. Previous versions +would write the wrong date (off by one day).

  • +
  • Fix an issue where VARCHAR columns added to an existing table could not be +queried.

  • +
  • Fix over-creation of initial splits.

  • +
  • Fix hive.immutable-partitions config property to also apply to +unpartitioned tables.

  • +
  • Allow non-VARCHAR columns in DELETE query.

  • +
  • Support DATE columns as partition columns in parquet tables.

  • +
  • Improve error message for cases where partition columns are also table columns.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.127.html b/430/release/release-0.127.html new file mode 100644 index 000000000..2b29495ae --- /dev/null +++ b/430/release/release-0.127.html @@ -0,0 +1,2551 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.127 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.127#

+
+

General#

+
    +
  • Disable index join repartitioning when it disrupts streaming execution.

  • +
  • Fix memory accounting leak in some JOIN queries.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.128.html b/430/release/release-0.128.html new file mode 100644 index 000000000..a7c53abd4 --- /dev/null +++ b/430/release/release-0.128.html @@ -0,0 +1,2579 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.128 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.128#

+
+

Graceful shutdown#

+

Workers can now be instructed to shutdown. This is done by submiting a PUT +request to /v1/info/state with the body "SHUTTING_DOWN". Once instructed +to shutdown, the worker will no longer receive new tasks, and will exit once +all existing tasks have completed.

+
+
+

General#

+
    +
  • Fix cast from json to structural types when rows or maps have arrays, +rows, or maps nested in them.

  • +
  • Fix Example HTTP connector. +It would previously fail with a JSON deserialization error.

  • +
  • Optimize memory usage in TupleDomain.

  • +
  • Fix an issue that can occur when an INNER JOIN has equi-join clauses that +align with the grouping columns used by a preceding operation such as +GROUP BY, DISTINCT, etc. When this triggers, the join may fail to +produce some of the output rows.

  • +
+
+
+

MySQL#

+
    +
  • Fix handling of MySQL database names with underscores.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.129.html b/430/release/release-0.129.html new file mode 100644 index 000000000..797069a45 --- /dev/null +++ b/430/release/release-0.129.html @@ -0,0 +1,2624 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.129 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.129#

+
+

Warning

+

There is a performance regression in this release for GROUP BY and JOIN +queries when the length of the keys is between 16 and 31 bytes. This is fixed +in Release 0.130.

+
+
+

General#

+
    +
  • Fix a planner issue that could cause queries involving OUTER JOIN to +return incorrect results.

  • +
  • Some queries, particularly those using max_by() or min_by(), now +accurately reflect their true memory usage and thus appear to use more memory +than before.

  • +
  • Fix SHOW SESSION to not show hidden session properties.

  • +
  • Fix hang in large queries with ORDER BY and LIMIT.

  • +
  • Fix an issue when casting empty arrays or arrays containing only NULL to +other types.

  • +
  • Table property names are now properly treated as case-insensitive.

  • +
  • Minor UI improvements for query detail page.

  • +
  • Do not display useless stack traces for expected exceptions in verifier.

  • +
  • Improve performance of queries involving UNION ALL that write data.

  • +
  • Introduce the P4HyperLogLog type, which uses an implementation of the HyperLogLog data +structure that trades off accuracy and memory requirements when handling small sets for an +improvement in performance.

  • +
+
+
+

JDBC driver#

+
    +
  • Throw exception when using SET SESSION or RESET SESSION +rather than silently ignoring the command.

  • +
  • The driver now properly supports non-query statements. +The Statement interface supports all variants of the execute methods. +It also supports the getUpdateCount and getLargeUpdateCount methods.

  • +
+
+
+

CLI#

+
    +
  • Always clear screen when canceling query with ctrl-C.

  • +
  • Make client request timeout configurable.

  • +
+
+
+

Network topology aware scheduling#

+

The scheduler can now be configured to take network topology into account when +scheduling splits. This is set using the node-scheduler.network-topology +config. See Tuning Trino for more information.

+
+
+

Hive#

+
    +
  • The S3 region is no longer automatically configured when running in EC2. +To enable this feature, use hive.s3.pin-client-to-current-region=true +in your Hive catalog properties file. Enabling this feature is required +to access S3 data in the China isolated region, but prevents accessing +data outside the current region.

  • +
  • Server-side encryption is now supported for S3. To enable this feature, +use hive.s3.sse.enabled=true in your Hive catalog properties file.

  • +
  • Add support for the retention_days table property.

  • +
  • Add support for S3 EncryptionMaterialsProvider.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.130.html b/430/release/release-0.130.html new file mode 100644 index 000000000..26afed807 --- /dev/null +++ b/430/release/release-0.130.html @@ -0,0 +1,2559 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.130 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.130#

+
+

General#

+
    +
  • Fix a performance regression in GROUP BY and JOIN queries when the +length of the keys is between 16 and 31 bytes.

  • +
  • Add map_concat() function.

  • +
  • Performance improvements for filters, projections and dictionary encoded data. +This optimization is turned off by default. It can be configured via the +optimizer.columnar-processing-dictionary config property or the +columnar_processing_dictionary session property.

  • +
  • Improve performance of aggregation queries with large numbers of groups.

  • +
  • Improve performance for queries that use ARRAY type.

  • +
  • Fix querying remote views in MySQL and PostgreSQL connectors.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.131.html b/430/release/release-0.131.html new file mode 100644 index 000000000..683c59859 --- /dev/null +++ b/430/release/release-0.131.html @@ -0,0 +1,2551 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.131 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.131#

+
+

General#

+
    +
  • Fix poor performance of transporting dictionary encoded data over the network.

  • +
  • Fix code generator to prevent “Method code too large” error.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.132.html b/430/release/release-0.132.html new file mode 100644 index 000000000..ed172b9e9 --- /dev/null +++ b/430/release/release-0.132.html @@ -0,0 +1,2597 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.132 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.132#

+
+

Warning

+

concat() on ARRAY, or enabling columnar_processing_dictionary +may cause queries to fail in this release. This is fixed in Release 0.133.

+
+
+

General#

+
    +
  • Fix a correctness issue that can occur when any join depends on the output +of another outer join that has an inner side (or either side for the full outer +case) for which the connector declares that it has no data during planning.

  • +
  • Improve error messages for unresolved operators.

  • +
  • Add support for creating constant arrays with more than 255 elements.

  • +
  • Fix analyzer for queries with GROUP BY () such that errors are raised +during analysis rather than execution.

  • +
  • Add resource_overcommit session property. This disables all memory +limits for the query. Instead it may be killed at any time, if the coordinator +needs to reclaim memory.

  • +
  • Add support for transactional connectors.

  • +
  • Add support for non-correlated scalar sub-queries.

  • +
  • Add support for SQL binary literals.

  • +
  • Add variant of random() that produces an integer number between 0 and a +specified upper bound.

  • +
  • Perform bounds checks when evaluating abs().

  • +
  • Improve accuracy of memory accounting for map_agg() and array_agg(). +These functions will now appear to use more memory than before.

  • +
  • Various performance optimizations for functions operating on ARRAY.

  • +
  • Add server version to web UI.

  • +
+
+
+

CLI#

+
    +
  • Fix sporadic “Failed to disable interrupt character” error after exiting pager.

  • +
+
+
+

Hive#

+
    +
  • Report metastore and namenode latency in milliseconds rather than seconds in +JMX stats.

  • +
  • Fix NullPointerException when inserting a null value for a partition column.

  • +
  • Improve CPU efficiency when writing data.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.133.html b/430/release/release-0.133.html new file mode 100644 index 000000000..6d848b7b4 --- /dev/null +++ b/430/release/release-0.133.html @@ -0,0 +1,2562 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.133 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.133#

+
+

General#

+
    +
  • Add support for calling connector-defined procedures using CALL.

  • +
  • Add System connector procedure for killing running queries.

  • +
  • Properly expire idle transactions that consist of just the start transaction statement +and nothing else.

  • +
  • Fix possible deadlock in worker communication when task restart is detected.

  • +
  • Performance improvements for aggregations on dictionary encoded data. +This optimization is turned off by default. It can be configured via the +optimizer.dictionary-aggregation config property or the +dictionary_aggregation session property.

  • +
  • Fix race which could cause queries to fail when using concat() on +ARRAY, or when enabling columnar_processing_dictionary.

  • +
  • Add sticky headers and the ability to sort the tasks table on the query page +in the web interface.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.134.html b/430/release/release-0.134.html new file mode 100644 index 000000000..07c25c1d9 --- /dev/null +++ b/430/release/release-0.134.html @@ -0,0 +1,2581 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.134 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.134#

+
+

General#

+
    +
  • Add cumulative memory statistics tracking and expose the stat in the web interface.

  • +
  • Remove nullability and partition key flags from SHOW COLUMNS.

  • +
  • Remove non-standard is_partition_key column from information_schema.columns.

  • +
  • Fix performance regression in creation of DictionaryBlock.

  • +
  • Fix rare memory accounting leak in queries with JOIN.

  • +
+
+
+

Hive#

+
    +
  • The comment for partition keys is now prefixed with “Partition Key”.

  • +
+
+
+

SPI#

+
    +
  • Remove legacy partition API methods and classes.

  • +
+
+

Note

+

This is a backwards incompatible change with the previous connector SPI. +If you have written a connector and have not yet updated to the +TableLayout API, you will need to update your code before deploying +this release.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.135.html b/430/release/release-0.135.html new file mode 100644 index 000000000..7e9a4e7c9 --- /dev/null +++ b/430/release/release-0.135.html @@ -0,0 +1,2555 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.135 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.135#

+
+

General#

+
    +
  • Add summary of change in CPU usage to verifier output.

  • +
  • Add cast between JSON and VARCHAR, BOOLEAN, DOUBLE, BIGINT. For the old +behavior of cast between JSON and VARCHAR (pre-Release 0.122), +use json_parse() and json_format().

  • +
  • Fix bug in 0.134 that prevented query page in web UI from displaying in +Safari.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.136.html b/430/release/release-0.136.html new file mode 100644 index 000000000..95a9c8898 --- /dev/null +++ b/430/release/release-0.136.html @@ -0,0 +1,2554 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.136 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.136#

+
+

General#

+
    +
  • Add control.query-types and test.query-types to verifier, which can +be used to select the type of queries to run.

  • +
  • Fix issue where queries with ORDER BY LIMIT with a limit greater than +2147483647 could fail or return incorrect results.

  • +
  • Add query plan visualization with live stats to the web UI.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.137.html b/430/release/release-0.137.html new file mode 100644 index 000000000..76af6e140 --- /dev/null +++ b/430/release/release-0.137.html @@ -0,0 +1,2585 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.137 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.137#

+
+

General#

+
    +
  • Fix current_date to return correct results for all time zones.

  • +
  • Fix invalid plans when scalar subqueries use GROUP BY, DISTINCT or JOIN.

  • +
  • Do not allow creating views with a column type of UNKNOWN.

  • +
  • Improve expression optimizer to remove some redundant operations.

  • +
  • Add bit_count(), bitwise_not(), bitwise_and(), +bitwise_or(), and bitwise_xor() functions.

  • +
  • Add approx_distinct() aggregation support for VARBINARY input.

  • +
  • Add create time to query detail page in UI.

  • +
  • Add support for VARCHAR(length) type.

  • +
  • Track per-stage peak memory usage.

  • +
  • Allow using double input for approx_percentile() with an array of +percentiles.

  • +
  • Add API to JDBC driver to track query progress.

  • +
+
+
+

Hive#

+
    +
  • Do not allow inserting into tables when the Hive type does not match +the Presto type. Previously, Presto would insert data that did not +match the table or partition type and that data could not be read by +Hive. For example, Presto would write files containing BIGINT +data for a Hive column type of INT.

  • +
  • Add validation to CREATE TABLE and CREATE TABLE AS +to check that partition keys are the last columns in the table and in the same +order as the table properties.

  • +
  • Remove retention_days table property. This property is not used by Hive.

  • +
  • Fix Parquet decoding of MAP containing a null value.

  • +
  • Add support for accessing ORC columns by name. By default, columns in ORC +files are accessed by their ordinal position in the Hive table definition. +To access columns based on the names recorded in the ORC file, set +hive.orc.use-column-names=true in your Hive catalog properties file.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.138.html b/430/release/release-0.138.html new file mode 100644 index 000000000..7453061f6 --- /dev/null +++ b/430/release/release-0.138.html @@ -0,0 +1,2569 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.138 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.138#

+
+

General#

+
    +
  • Fix planning bug with NULL literal coercions.

  • +
  • Reduce query startup time by reducing lock contention in scheduler.

  • +
+
+
+

New Hive Parquet reader#

+

We have added a new Parquet reader implementation. The new reader supports vectorized +reads, lazy loading, and predicate push down, all of which make the reader more +efficient and typically reduces wall clock time for a query. Although the new +reader has been heavily tested, it is an extensive rewrite of the Apache Hive +Parquet reader, and may have some latent issues, so it is not enabled by default. +If you are using Parquet we suggest you test out the new reader on a per-query basis +by setting the <hive-catalog>.parquet_optimized_reader_enabled session property, +or you can enable the reader by default by setting the Hive catalog property +hive.parquet-optimized-reader.enabled=true. To enable Parquet predicate push down +there is a separate session property <hive-catalog>.parquet_predicate_pushdown_enabled +and configuration property hive.parquet-predicate-pushdown.enabled=true.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.139.html b/430/release/release-0.139.html new file mode 100644 index 000000000..fbaf08808 --- /dev/null +++ b/430/release/release-0.139.html @@ -0,0 +1,2579 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.139 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.139#

+
+

Dynamic split concurrency#

+

The number of running leaf splits per query is now dynamically adjusted to improve +overall cluster throughput. task.initial-splits-per-node can be used to set +the initial number of splits, and task.split-concurrency-adjustment-interval +can be used to change how frequently adjustments happen. The session properties +initial_splits_per_node and split_concurrency_adjustment_interval can +also be used.

+
+
+

General#

+
    +
  • Fix planning bug that causes some joins to not be redistributed when +distributed-joins-enabled is true.

  • +
  • Fix rare leak of stage objects and tasks for queries using LIMIT.

  • +
  • Add experimental task.join-concurrency config which can be used to increase +concurrency for the probe side of joins.

  • +
+
+
+

Hive#

+
    +
  • Remove cursor-based readers for ORC and DWRF file formats, as they have been +replaced by page-based readers.

  • +
  • Fix creating tables on S3 with CREATE TABLE AS.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.140.html b/430/release/release-0.140.html new file mode 100644 index 000000000..c8e8daf3c --- /dev/null +++ b/430/release/release-0.140.html @@ -0,0 +1,2594 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.140 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.140#

+
+

General#

+
    +
  • Add the TRY function to handle specific data exceptions. See +Conditional expressions.

  • +
  • Optimize predicate expressions to minimize redundancies.

  • +
  • Add environment name to UI.

  • +
  • Fix logging of failure_host and failure_task fields in +QueryCompletionEvent.

  • +
  • Fix race which can cause queries to fail with a REMOTE_TASK_ERROR.

  • +
  • Optimize array_distinct() for array(bigint).

  • +
  • Optimize > operator for ARRAY.

  • +
  • Fix an optimization issue that could result in non-deterministic functions +being evaluated more than once producing unexpected results.

  • +
  • Fix incorrect result for rare IN lists that contain certain combinations +of non-constant expressions that are null and non-null.

  • +
  • Improve performance of joins, aggregations, etc. by removing unnecessarily +duplicated columns.

  • +
  • Optimize NOT IN queries to produce more compact predicates.

  • +
+
+
+

Hive#

+
    +
  • Remove bogus “from deserializer” column comments.

  • +
  • Change categorization of Hive writer errors to be more specific.

  • +
  • Add date and timestamp support to new Parquet Reader

  • +
+
+
+

SPI#

+
    +
  • Remove partition key from ColumnMetadata.

  • +
  • Change return type of ConnectorTableLayout.getDiscretePredicates().

  • +
+
+

Note

+

This is a backwards incompatible change with the previous connector SPI. +If you have written a connector, you will need to update your code +before deploying this release.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.141.html b/430/release/release-0.141.html new file mode 100644 index 000000000..a44c6d251 --- /dev/null +++ b/430/release/release-0.141.html @@ -0,0 +1,2550 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.141 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.141#

+
+

General#

+
    +
  • Fix server returning an HTTP 500 response for queries with parse errors.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.142.html b/430/release/release-0.142.html new file mode 100644 index 000000000..747468725 --- /dev/null +++ b/430/release/release-0.142.html @@ -0,0 +1,2579 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.142 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.142#

+
+

General#

+
    +
  • Fix planning bug for JOIN criteria that optimizes to a FALSE expression.

  • +
  • Fix planning bug when the output of UNION doesn’t match the table column order +in INSERT queries.

  • +
  • Fix error when ORDER BY clause in window specification refers to the same column multiple times.

  • +
  • Add support for complex grouping operations +- CUBE, ROLLUP and GROUPING SETS.

  • +
  • Add support for IF NOT EXISTS in CREATE TABLE AS queries.

  • +
  • Add substring() function.

  • +
  • Add http.server.authentication.krb5.keytab config option to set the location of the Kerberos +keytab file explicitly.

  • +
  • Add optimize_metadata_queries session property to enable the metadata-only query optimization.

  • +
  • Improve support for non-equality predicates in JOIN criteria.

  • +
  • Add support for non-correlated subqueries in aggregation queries.

  • +
  • Improve performance of json_extract().

  • +
+
+
+

Hive#

+
    +
  • Change ORC input format to report actual bytes read as opposed to estimated bytes.

  • +
  • Fix cache invalidation when renaming tables.

  • +
  • Fix Parquet reader to handle uppercase column names.

  • +
  • Fix issue where the hive.respect-table-format config option was being ignored.

  • +
  • Add hive.compression-codec config option to control +compression used when writing. The default is now GZIP for all formats.

  • +
  • Collect and expose end-to-end execution time JMX metric for requests to AWS services.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.143.html b/430/release/release-0.143.html new file mode 100644 index 000000000..0b0ae5892 --- /dev/null +++ b/430/release/release-0.143.html @@ -0,0 +1,2577 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.143 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.143#

+
+

General#

+
    +
  • Fix race condition in output buffer that can cause a page to be lost.

  • +
  • Fix case-sensitivity issue when de-referencing row fields.

  • +
  • Fix bug in phased scheduler that could cause queries to block forever.

  • +
  • Fix DELETE for predicates that optimize to false.

  • +
  • Add support for scalar subqueries in DELETE queries.

  • +
  • Add config option query.max-cpu-time to limit CPU time used by a query.

  • +
  • Add loading indicator and error message to query detail page in UI.

  • +
  • Add query teardown to query timeline visualizer.

  • +
  • Add string padding functions lpad() and rpad().

  • +
  • Add width_bucket() function.

  • +
  • Add truncate() function.

  • +
  • Improve query startup time in large clusters.

  • +
  • Improve error messages for CAST and slice().

  • +
+
+
+

Hive#

+
    +
  • Fix native memory leak when reading or writing gzip compressed data.

  • +
  • Fix performance regression due to complex expressions not being applied +when pruning partitions.

  • +
  • Fix data corruption in CREATE TABLE AS when +hive.respect-table-format config is set to false and user-specified +storage format does not match default.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.144.1.html b/430/release/release-0.144.1.html new file mode 100644 index 000000000..b391a80a8 --- /dev/null +++ b/430/release/release-0.144.1.html @@ -0,0 +1,2550 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.144.1 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.144.1#

+
+

Hive#

+
    +
  • Fix bug when grouping on a bucketed column which causes incorrect results.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.144.2.html b/430/release/release-0.144.2.html new file mode 100644 index 000000000..a92f6a272 --- /dev/null +++ b/430/release/release-0.144.2.html @@ -0,0 +1,2552 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.144.2 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.144.2#

+
+

General#

+
    +
  • Fix potential memory leak in coordinator query history.

  • +
  • Add driver.max-page-partitioning-buffer-size config to control buffer size +used to repartition pages for exchanges.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.144.3.html b/430/release/release-0.144.3.html new file mode 100644 index 000000000..46735d02a --- /dev/null +++ b/430/release/release-0.144.3.html @@ -0,0 +1,2565 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.144.3 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.144.3#

+
+

General#

+
    +
  • Fix bugs in planner where coercions were not taken into account when computing +types.

  • +
  • Fix compiler failure when TRY is a sub-expression.

  • +
  • Fix compiler failure when TRY is called on a constant or an input reference.

  • +
  • Fix race condition that can cause queries that process data from non-columnar data +sources to fail.

  • +
+
+
+

Hive#

+
    +
  • Fix reading symlinks when the target is in a different HDFS instance.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.144.4.html b/430/release/release-0.144.4.html new file mode 100644 index 000000000..7f4ff9156 --- /dev/null +++ b/430/release/release-0.144.4.html @@ -0,0 +1,2550 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.144.4 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.144.4#

+
+

General#

+
    +
  • Fix incorrect results for grouping sets for some queries with filters.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.144.5.html b/430/release/release-0.144.5.html new file mode 100644 index 000000000..8bbaa5bfc --- /dev/null +++ b/430/release/release-0.144.5.html @@ -0,0 +1,2555 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.144.5 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.144.5#

+
+

General#

+
    +
  • Fix window functions to correctly handle empty frames between unbounded and +bounded in the same direction. For example, a frame such as +ROWS BETWEEN UNBOUNDED PRECEDING AND 2 PRECEDING +would incorrectly use the first row as the window frame for the first two +rows rather than using an empty frame.

  • +
  • Fix correctness issue when grouping on columns that are also arguments to aggregation functions.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.144.6.html b/430/release/release-0.144.6.html new file mode 100644 index 000000000..a724c63b6 --- /dev/null +++ b/430/release/release-0.144.6.html @@ -0,0 +1,2568 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.144.6 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.144.6#

+
+

General#

+

This release fixes several problems with large and negative intervals.

+
    +
  • Fix parsing of negative interval literals. Previously, the sign of each field was treated +independently instead of applying to the entire interval value. For example, the literal +INTERVAL '-2-3' YEAR TO MONTH was interpreted as a negative interval of 21 months +rather than 27 months (positive 3 months was added to negative 24 months).

  • +
  • Fix handling of INTERVAL DAY TO SECOND type in REST API. Previously, intervals greater than +2,147,483,647 milliseconds (about 24 days) were returned as the wrong value.

  • +
  • Fix handling of INTERVAL YEAR TO MONTH type. Previously, intervals greater than +2,147,483,647 months were returned as the wrong value from the REST API +and parsed incorrectly when specified as a literal.

  • +
  • Fix formatting of negative intervals in REST API. Previously, negative intervals +had a negative sign before each component and could not be parsed.

  • +
  • Fix formatting of negative intervals in JDBC PrestoInterval classes.

  • +
+
+

Note

+

Older versions of the JDBC driver will misinterpret most negative +intervals from new servers. Make sure to update the JDBC driver +along with the server.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.144.7.html b/430/release/release-0.144.7.html new file mode 100644 index 000000000..6d8c40d2d --- /dev/null +++ b/430/release/release-0.144.7.html @@ -0,0 +1,2552 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.144.7 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.144.7#

+
+

General#

+
    +
  • Fail queries with non-equi conjuncts in OUTER JOINs, instead of silently +dropping such conjuncts from the query and producing incorrect results.

  • +
  • Add cosine_similarity() function.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.144.html b/430/release/release-0.144.html new file mode 100644 index 000000000..2594bb8e6 --- /dev/null +++ b/430/release/release-0.144.html @@ -0,0 +1,2578 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.144 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.144#

+
+

Warning

+

Querying bucketed tables in the Hive connector may produce incorrect results. +This is fixed in Release 0.144.1, and Release 0.145.

+
+
+

General#

+
    +
  • Fix already exists check when adding a column to be case-insensitive.

  • +
  • Fix correctness issue when complex grouping operations have a partitioned source.

  • +
  • Fix missing coercion when using INSERT with NULL literals.

  • +
  • Fix regression that the queries fail when aggregation functions present in AT TIME ZONE.

  • +
  • Fix potential memory starvation when a query is run with resource_overcommit=true.

  • +
  • Queries run with resource_overcommit=true may now be killed before +they reach query.max-memory if the cluster is low on memory.

  • +
  • Discard output stage JSON from completion event when it is very long. +This limit can be configured with event.max-output-stage-size.

  • +
  • Add support for EXPLAIN ANALYZE.

  • +
  • Change infoUri field of /v1/statement to point to query HTML page instead of JSON.

  • +
  • Improve performance when processing results in CLI and JDBC driver.

  • +
  • Improve performance of GROUP BY queries.

  • +
+
+
+

Hive#

+
    +
  • Fix ORC reader to actually use hive.orc.stream-buffer-size configuration property.

  • +
  • Add support for creating and inserting into bucketed tables.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.145.html b/430/release/release-0.145.html new file mode 100644 index 000000000..51a8c911c --- /dev/null +++ b/430/release/release-0.145.html @@ -0,0 +1,2601 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.145 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.145#

+
+

General#

+
    +
  • Fix potential memory leak in coordinator query history.

  • +
  • Fix column resolution issue when qualified name refers to a view.

  • +
  • Fail arithmetic operations on overflow.

  • +
  • Fix bugs in planner where coercions were not taken into account when computing +types.

  • +
  • Fix compiler failure when TRY is a sub-expression.

  • +
  • Fix compiler failure when TRY is called on a constant or an input reference.

  • +
  • Add support for the integer type to the Presto engine and the Hive, +Raptor, Redis, Kafka, Cassandra and example-http connectors.

  • +
  • Add initial support for the decimal data type.

  • +
  • Add driver.max-page-partitioning-buffer-size config to control buffer size +used to repartition pages for exchanges.

  • +
  • Improve performance for distributed JOIN and GROUP BY queries with billions +of groups.

  • +
  • Improve reliability in highly congested networks by adjusting the default +connection idle timeouts.

  • +
+
+
+

Verifier#

+
    +
  • Change verifier to only run read-only queries by default. This behavior can be +changed with the control.query-types and test.query-types config flags.

  • +
+
+
+

CLI#

+
    +
  • Improve performance of output in batch mode.

  • +
  • Fix hex rendering in batch mode.

  • +
  • Abort running queries when CLI is terminated.

  • +
+
+
+

Hive#

+
    +
  • Fix bug when grouping on a bucketed column which causes incorrect results.

  • +
  • Add max_split_size and max_initial_split_size session properties to control +the size of generated splits.

  • +
  • Add retries to the metastore security calls.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.146.html b/430/release/release-0.146.html new file mode 100644 index 000000000..c67957048 --- /dev/null +++ b/430/release/release-0.146.html @@ -0,0 +1,2590 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.146 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.146#

+
+

General#

+
    +
  • Fix error in map_concat() when the second map is empty.

  • +
  • Require at least 4096 file descriptors to run Presto.

  • +
  • Support casting between map types.

  • +
  • Add MongoDB connector.

  • +
+
+
+

Hive#

+
    +
  • Fix incorrect skipping of data in Parquet during predicate push-down.

  • +
  • Fix reading of Parquet maps and lists containing nulls.

  • +
  • Fix reading empty ORC file with hive.orc.use-column-names enabled.

  • +
  • Fix writing to S3 when the staging directory is a symlink to a directory.

  • +
  • Legacy authorization properties, such as hive.allow-drop-table, are now +only enforced when hive.security=none is set, which is the default +security system. Specifically, the sql-standard authorization system +does not enforce these settings.

  • +
+
+
+

Black Hole#

+
    +
  • Add support for varchar(n).

  • +
+
+
+

Cassandra#

+
    +
  • Add support for Cassandra 3.0.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.147.html b/430/release/release-0.147.html new file mode 100644 index 000000000..1ed867354 --- /dev/null +++ b/430/release/release-0.147.html @@ -0,0 +1,2625 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.147 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.147#

+
+

General#

+
    +
  • Fix race condition that can cause queries that process data from non-columnar +data sources to fail.

  • +
  • Fix incorrect formatting of dates and timestamps before year 1680.

  • +
  • Fix handling of syntax errors when parsing EXTRACT.

  • +
  • Fix potential scheduling deadlock for connectors that expose node-partitioned data.

  • +
  • Fix performance regression that increased planning time.

  • +
  • Fix incorrect results for grouping sets for some queries with filters.

  • +
  • Add SHOW CREATE VIEW and SHOW CREATE TABLE.

  • +
  • Add support for column aliases in WITH clause.

  • +
  • Support LIKE clause for SHOW CATALOGS and SHOW SCHEMAS.

  • +
  • Add support for INTERSECT.

  • +
  • Add support for casting row types.

  • +
  • Add sequence() function.

  • +
  • Add sign() function.

  • +
  • Add flatten() function.

  • +
  • Add experimental implementation of resource groups.

  • +
  • Add Local file connector.

  • +
  • Remove experimental intermediate aggregation optimizer. The optimizer.use-intermediate-aggregations +config option and task_intermediate_aggregation session property are no longer supported.

  • +
  • Add support for colocated joins for connectors that expose node-partitioned data.

  • +
  • Improve the performance of array_intersect().

  • +
  • Generalize the intra-node parallel execution system to work with all query stages. +The task.concurrency configuration property replaces the old task.join-concurrency +and task.default-concurrency options. Similarly, the task_concurrency session +property replaces the task_join_concurrency, task_hash_build_concurrency, and +task_aggregation_concurrency properties.

  • +
+
+
+

Hive#

+
    +
  • Fix reading symlinks when the target is in a different HDFS instance.

  • +
  • Fix NoClassDefFoundError for SubnetUtils in HDFS client.

  • +
  • Fix error when reading from Hive tables with inconsistent bucketing metadata.

  • +
  • Correctly report read bytes when reading Parquet data.

  • +
  • Include path in unrecoverable S3 exception messages.

  • +
  • When replacing an existing Presto view, update the view data +in the Hive metastore rather than dropping and recreating it.

  • +
  • Rename table property clustered_by to bucketed_by.

  • +
  • Add support for varchar(n).

  • +
+
+
+

Kafka#

+
    +
  • Fix error code 6 when reading data from Kafka.

  • +
  • Add support for varchar(n).

  • +
+
+
+

Redis#

+
    +
  • Add support for varchar(n).

  • +
+
+
+

MySQL and PostgreSQL#

+
    +
  • Cleanup temporary data when a CREATE TABLE AS fails.

  • +
  • Add support for varchar(n).

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.148.html b/430/release/release-0.148.html new file mode 100644 index 000000000..d850df9db --- /dev/null +++ b/430/release/release-0.148.html @@ -0,0 +1,2705 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.148 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+ +
+
+ +
+
+ +
+

Release 0.148#

+
+

General#

+
    +
  • Fix issue where auto-commit transaction can be rolled back for a successfully +completed query.

  • +
  • Fix detection of colocated joins.

  • +
  • Fix planning bug involving partitioning with constants.

  • +
  • Fix window functions to correctly handle empty frames between unbounded and +bounded in the same direction. For example, a frame such as +ROWS BETWEEN UNBOUNDED PRECEDING AND 2 PRECEDING +would incorrectly use the first row as the window frame for the first two +rows rather than using an empty frame.

  • +
  • Fix correctness issue when grouping on columns that are also arguments to aggregation functions.

  • +
  • Fix failure when chaining AT TIME ZONE, e.g. +SELECT TIMESTAMP '2016-01-02 12:34:56' AT TIME ZONE 'America/Los_Angeles' AT TIME ZONE 'UTC'.

  • +
  • Fix data duplication when task.writer-count configuration mismatches between coordinator and worker.

  • +
  • Fix bug where node-scheduler.max-pending-splits-per-node-per-task config is not always +honored by node scheduler. This bug could stop the cluster from making further progress.

  • +
  • Fix incorrect results for grouping sets with partitioned source.

  • +
  • Add colocated-joins-enabled to enable colocated joins by default for +connectors that expose node-partitioned data.

  • +
  • Add support for colocated unions.

  • +
  • Reduce initial memory usage of array_agg() function.

  • +
  • Improve planning of co-partitioned JOIN and UNION.

  • +
  • Improve planning of aggregations over partitioned data.

  • +
  • Improve the performance of the array_sort() function.

  • +
  • Improve outer join predicate push down.

  • +
  • Increase default value for query.initial-hash-partitions to 100.

  • +
  • Change default value of query.max-memory-per-node to 10% of the Java heap.

  • +
  • Change default task.max-worker-threads to 2 times the number of cores.

  • +
  • Use HTTPS in JDBC driver when using port 443.

  • +
  • Warn if Presto server is not using G1 garbage collector.

  • +
  • Move interval types out of SPI.

  • +
+
+
+

Interval fixes#

+

This release fixes several problems with large and negative intervals.

+
    +
  • Fix parsing of negative interval literals. Previously, the sign of each field was treated +independently instead of applying to the entire interval value. For example, the literal +INTERVAL '-2-3' YEAR TO MONTH was interpreted as a negative interval of 21 months +rather than 27 months (positive 3 months was added to negative 24 months).

  • +
  • Fix handling of INTERVAL DAY TO SECOND type in REST API. Previously, intervals greater than +2,147,483,647 milliseconds (about 24 days) were returned as the wrong value.

  • +
  • Fix handling of INTERVAL YEAR TO MONTH type. Previously, intervals greater than +2,147,483,647 months were returned as the wrong value from the REST API +and parsed incorrectly when specified as a literal.

  • +
  • Fix formatting of negative intervals in REST API. Previously, negative intervals +had a negative sign before each component and could not be parsed.

  • +
  • Fix formatting of negative intervals in JDBC PrestoInterval classes.

  • +
+
+

Note

+

Older versions of the JDBC driver will misinterpret most negative +intervals from new servers. Make sure to update the JDBC driver +along with the server.

+
+
+
+

Functions and language features#

+
    +
  • Add element_at() function for map type.

  • +
  • Add split_to_map() function.

  • +
  • Add zip() function.

  • +
  • Add map_union() aggregation function.

  • +
  • Add ROW syntax for constructing row types.

  • +
  • Add support for REVOKE permission syntax.

  • +
  • Add support for SMALLINT and TINYINT types.

  • +
  • Add support for non-equi outer joins.

  • +
+
+
+

Verifier#

+
    +
  • Add skip-cpu-check-regex config property which can be used to skip the CPU +time comparison for queries that match the given regex.

  • +
  • Add check-cpu config property which can be used to disable CPU time comparison.

  • +
+
+
+

Hive#

+
    +
  • Fix NoClassDefFoundError for KMSClientProvider in HDFS client.

  • +
  • Fix creating tables on S3 in an empty database.

  • +
  • Implement REVOKE permission syntax.

  • +
  • Add support for SMALLINT and TINYINT

  • +
  • Support DELETE from unpartitioned tables.

  • +
  • Add support for Kerberos authentication when talking to Hive/HDFS.

  • +
  • Push down filters for columns of type DECIMAL.

  • +
  • Improve CPU efficiency when reading ORC files.

  • +
+
+
+

Cassandra#

+
    +
  • Allow configuring load balancing policy and no host available retry.

  • +
  • Add support for varchar(n).

  • +
+
+
+

Kafka#

+
    +
  • Update to Kafka client 0.8.2.2. This enables support for LZ4 data.

  • +
+
+
+

JMX#

+
    +
  • Add jmx.history schema with in-memory periodic samples of values from JMX MBeans.

  • +
+
+
+

MySQL and PostgreSQL#

+
    +
  • Push down predicates for VARCHAR, DATE, TIME and TIMESTAMP types.

  • +
+
+
+

Other connectors#

+
    +
  • Add support for varchar(n) to the Redis, TPCH, MongoDB, Local File +and Example HTTP connectors.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.149.html b/430/release/release-0.149.html new file mode 100644 index 000000000..f63f2664f --- /dev/null +++ b/430/release/release-0.149.html @@ -0,0 +1,2590 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.149 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.149#

+
+

General#

+
    +
  • Fix runtime failure for queries that use grouping sets over unions.

  • +
  • Do not ignore null values in array_agg().

  • +
  • Fix failure when casting row values that contain null fields.

  • +
  • Fix failure when using complex types as map keys.

  • +
  • Fix potential memory tracking leak when queries are cancelled.

  • +
  • Fix rejection of queries that do not match any queue/resource group rules. +Previously, a 500 error was returned to the client.

  • +
  • Fix trim() and rtrim() functions to produce more intuitive results +when the argument contains invalid UTF-8 sequences.

  • +
  • Add a new web interface with cluster overview, realtime stats, and improved sorting +and filtering of queries.

  • +
  • Add support for FLOAT type.

  • +
  • Rename query.max-age to query.min-expire-age.

  • +
  • optimizer.columnar-processing and optimizer.columnar-processing-dictionary +properties were merged to optimizer.processing-optimization with possible +values disabled, columnar and columnar_dictionary

  • +
  • columnar_processing and columnar_processing_dictionary session +properties were merged to processing_optimization with possible values +disabled, columnar and columnar_dictionary

  • +
  • Change %y (2-digit year) in date_parse() to evaluate to a year between +1970 and 2069 inclusive.

  • +
  • Add queued flag to StatementStats in REST API.

  • +
  • Improve error messages for math operations.

  • +
  • Improve memory tracking in exchanges to avoid running out of Java heap space.

  • +
  • Improve performance of subscript operator for the MAP type.

  • +
  • Improve performance of JOIN and GROUP BY queries.

  • +
+
+
+

Hive#

+
    +
  • Clean up empty staging directories after inserts.

  • +
  • Add hive.dfs.ipc-ping-interval config for HDFS.

  • +
  • Change default value of hive.dfs-timeout to 60 seconds.

  • +
  • Fix ORC/DWRF reader to avoid repeatedly fetching the same data when stripes +are skipped.

  • +
  • Fix force local scheduling for S3 or other non-HDFS file systems.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.150.html b/430/release/release-0.150.html new file mode 100644 index 000000000..364f5d113 --- /dev/null +++ b/430/release/release-0.150.html @@ -0,0 +1,2571 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.150 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.150#

+
+

Warning

+

The Hive bucketing optimizations are broken in this release. You should +disable them by adding hive.bucket-execution=false to your +Hive catalog properties.

+
+
+

General#

+
    +
  • Fix web UI bug that caused rendering to fail when a stage has no tasks.

  • +
  • Fix failure due to ambiguity when calling round() on tinyint arguments.

  • +
  • Fix race in exchange HTTP endpoint, which could cause queries to fail randomly.

  • +
  • Add support for parsing timestamps with nanosecond precision in date_parse().

  • +
  • Add CPU quotas to resource groups.

  • +
+
+
+

Hive#

+
    +
  • Add support for writing to bucketed tables.

  • +
  • Add execution optimizations for bucketed tables.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.151.html b/430/release/release-0.151.html new file mode 100644 index 000000000..45cd6d56a --- /dev/null +++ b/430/release/release-0.151.html @@ -0,0 +1,2587 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.151 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.151#

+
+

General#

+
    +
  • Fix issue where aggregations may produce the wrong result when task.concurrency is set to 1.

  • +
  • Fix query failure when array, map, or row type is used in non-equi JOIN.

  • +
  • Fix performance regression for queries using OUTER JOIN.

  • +
  • Fix query failure when using the arbitrary() aggregation function on integer type.

  • +
  • Add various math functions that operate directly on float type.

  • +
  • Add flag deprecated.legacy-array-agg to restore legacy array_agg() +behavior (ignore NULL input). This flag will be removed in a future release.

  • +
  • Add support for uncorrelated EXISTS clause.

  • +
  • Add cosine_similarity() function.

  • +
  • Allow Tableau web connector to use catalogs other than hive.

  • +
+
+
+

Verifier#

+
    +
  • Add shadow-writes.enabled option which can be used to transform CREATE TABLE AS SELECT +queries to write to a temporary table (rather than the originally specified table).

  • +
+
+
+

SPI#

+
    +
  • Remove getDataSourceName from ConnectorSplitSource.

  • +
  • Remove dataSourceName constructor parameter from FixedSplitSource.

  • +
+
+

Note

+

This is a backwards incompatible change with the previous connector SPI. +If you have written a connector, you will need to update your code +before deploying this release.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.152.1.html b/430/release/release-0.152.1.html new file mode 100644 index 000000000..e9193eb44 --- /dev/null +++ b/430/release/release-0.152.1.html @@ -0,0 +1,2551 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.152.1 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.152.1#

+
+

General#

+
    +
  • Fix race which could cause failed queries to have no error details.

  • +
  • Fix race in HTTP layer which could cause queries to fail.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.152.2.html b/430/release/release-0.152.2.html new file mode 100644 index 000000000..8c350db4e --- /dev/null +++ b/430/release/release-0.152.2.html @@ -0,0 +1,2550 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.152.2 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.152.2#

+
+

Hive#

+
    +
  • Improve performance of ORC reader when decoding dictionary encoded MAP.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.152.3.html b/430/release/release-0.152.3.html new file mode 100644 index 000000000..f1ebff54d --- /dev/null +++ b/430/release/release-0.152.3.html @@ -0,0 +1,2550 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.152.3 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.152.3#

+
+

General#

+
    +
  • Fix incorrect results for grouping sets when task.concurrency is greater than one.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.152.html b/430/release/release-0.152.html new file mode 100644 index 000000000..531605cf1 --- /dev/null +++ b/430/release/release-0.152.html @@ -0,0 +1,2663 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.152 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.152#

+
+

General#

+
    +
  • Add array_union() function.

  • +
  • Add reverse() function for arrays.

  • +
  • Fix issue that could cause queries with varchar literals to fail.

  • +
  • Fix categorization of errors from url_decode(), allowing it to be used with TRY.

  • +
  • Fix error reporting for invalid JSON paths provided to JSON functions.

  • +
  • Fix view creation for queries containing GROUPING SETS.

  • +
  • Fix query failure when referencing a field of a NULL row.

  • +
  • Improve query performance for multiple consecutive window functions.

  • +
  • Prevent web UI from breaking when query fails without an error code.

  • +
  • Display port on the task list in the web UI when multiple workers share the same host.

  • +
  • Add support for EXCEPT.

  • +
  • Rename FLOAT type to REAL for better compatibility with the SQL standard.

  • +
  • Fix potential performance regression when transporting rows between nodes.

  • +
+
+
+

JDBC driver#

+
    +
  • Fix sizes returned from DatabaseMetaData.getColumns() for +COLUMN_SIZE, DECIMAL_DIGITS, NUM_PREC_RADIX and CHAR_OCTET_LENGTH.

  • +
+
+
+

Hive#

+
    +
  • Fix resource leak in Parquet reader.

  • +
  • Rename JMX stat AllViews to GetAllViews in ThriftHiveMetastore.

  • +
  • Add file based security, which can be configured with the hive.security +and security.config-file config properties. See Hive connector security configuration +for more details.

  • +
  • Add support for custom S3 credentials providers using the +presto.s3.credentials-provider Hadoop configuration property.

  • +
+
+
+

MySQL#

+
    +
  • Fix reading MySQL tinyint(1) columns. Previously, these columns were +incorrectly returned as a boolean rather than an integer.

  • +
  • Add support for INSERT.

  • +
  • Add support for reading data as tinyint and smallint types rather than integer.

  • +
+
+
+

PostgreSQL#

+
    +
  • Add support for INSERT.

  • +
  • Add support for reading data as tinyint and smallint types rather than integer.

  • +
+
+
+

SPI#

+
    +
  • Remove owner from ConnectorTableMetadata.

  • +
  • Replace the generic getServices() method in Plugin with specific +methods such as getConnectorFactories(), getTypes(), etc. +Dependencies like TypeManager are now provided directly rather +than being injected into Plugin.

  • +
  • Add first-class support for functions in the SPI. This replaces the old +FunctionFactory interface. Plugins can return a list of classes from the +getFunctions() method:

    +
      +
    • Scalar functions are methods or classes annotated with @ScalarFunction.

    • +
    • Aggregation functions are methods or classes annotated with @AggregationFunction.

    • +
    • Window functions are an implementation of WindowFunction. Most implementations +should be a subclass of RankingWindowFunction or ValueWindowFunction.

    • +
    +
  • +
+
+

Note

+

This is a backwards incompatible change with the previous SPI. +If you have written a plugin, you will need to update your code +before deploying this release.

+
+
+
+

Verifier#

+
    +
  • Fix handling of shadow write queries with a LIMIT.

  • +
+
+
+

Local file#

+
    +
  • Fix file descriptor leak.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.153.html b/430/release/release-0.153.html new file mode 100644 index 000000000..0c4313b19 --- /dev/null +++ b/430/release/release-0.153.html @@ -0,0 +1,2760 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.153 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.153#

+
+

General#

+
    +
  • Fix incorrect results for grouping sets when task.concurrency is greater than one.

  • +
  • Fix silent numeric overflow when casting INTEGER to large DECIMAL types.

  • +
  • Fix issue where GROUP BY () would produce no results if the input had no rows.

  • +
  • Fix null handling in array_distinct() when applied to the array(bigint) type.

  • +
  • Fix handling of -2^63 as the element index for json_array_get().

  • +
  • Fix correctness issue when the input to TRY_CAST evaluates to null. +For types such as booleans, numbers, dates, timestamps, etc., rather than +returning null, a default value specific to the type such as +false, 0 or 1970-01-01 was returned.

  • +
  • Fix potential thread deadlock in coordinator.

  • +
  • Fix rare correctness issue with an aggregation on a single threaded right join when +task.concurrency is 1.

  • +
  • Fix query failure when casting a map with null values.

  • +
  • Fix failure when view column names contain upper-case letters.

  • +
  • Fix potential performance regression due to skew issue when +grouping or joining on columns of the following types: TINYINT, +SMALLINT, INTEGER, BIGINT, REAL, DOUBLE, +COLOR, DATE, INTERVAL, TIME, TIMESTAMP.

  • +
  • Fix potential memory leak for delete queries.

  • +
  • Fix query stats to not include queued time in planning time.

  • +
  • Fix query completion event to log final stats for the query.

  • +
  • Fix spurious log messages when queries are torn down.

  • +
  • Remove broken %w specifier for date_format() and date_parse().

  • +
  • Improve performance of ARRAY when underlying data is dictionary encoded.

  • +
  • Improve performance of outer joins with non-equality criteria.

  • +
  • Require task concurrency and task writer count to be a power of two.

  • +
  • Use nulls-last ordering for array_sort().

  • +
  • Validate that TRY is used with exactly one argument.

  • +
  • Allow running Presto with early-access Java versions.

  • +
  • Add Accumulo connector.

  • +
+
+
+

Functions and language features#

+ +
+
+

Pluggable resource groups#

+

Resource group management is now pluggable. A Plugin can +provide management factories via getResourceGroupConfigurationManagerFactories() +and the factory can be enabled via the etc/resource-groups.properties +configuration file by setting the resource-groups.configuration-manager +property. See the presto-resource-group-managers plugin for an example +and Resource groups for more details.

+
+
+

Web UI#

+
    +
  • Fix rendering failures due to null nested data structures.

  • +
  • Do not include coordinator in active worker count on cluster overview page.

  • +
  • Replace buffer skew indicators on query details page with scheduled time skew.

  • +
  • Add stage total buffer, pending tasks and wall time to stage statistics on query details page.

  • +
  • Add option to filter task lists by status on query details page.

  • +
  • Add copy button for query text, query ID, and user to query details page.

  • +
+
+
+

JDBC driver#

+
    +
  • Add support for real data type, which corresponds to the Java float type.

  • +
+
+
+

CLI#

+
    +
  • Add support for configuring the HTTPS Truststore.

  • +
+
+
+

Hive#

+
    +
  • Fix permissions for new tables when using SQL-standard authorization.

  • +
  • Improve performance of ORC reader when decoding dictionary encoded MAP.

  • +
  • Allow certain combinations of queries to be executed in a transaction-ish manner, +for example, when dropping a partition and then recreating it. Atomicity is not +guaranteed due to fundamental limitations in the design of Hive.

  • +
  • Support per-transaction cache for Hive metastore.

  • +
  • Fail queries that attempt to rename partition columns.

  • +
  • Add support for ORC bloom filters in predicate push down. +This is can be enabled using the hive.orc.bloom-filters.enabled +configuration property or the orc_bloom_filters_enabled session property.

  • +
  • Add new optimized RCFile reader. +This can be enabled using the hive.rcfile-optimized-reader.enabled +configuration property or the rcfile_optimized_reader_enabled session property.

  • +
  • Add support for the Presto real type, which corresponds to the Hive float type.

  • +
  • Add support for char(x) type.

  • +
  • Add support for creating, dropping and renaming schemas (databases). +The filesystem location can be specified when creating a schema, +which allows, for example, easily creating tables on S3.

  • +
  • Record Presto query ID for tables or partitions written by Presto +using the presto_query_id table or partition property.

  • +
  • Include path name in error message when listing a directory fails.

  • +
  • Rename allow-all authorization method to legacy. This +method is deprecated and will be removed in a future release.

  • +
  • Do not retry S3 requests that are aborted intentionally.

  • +
  • Set the user agent suffix for S3 requests to presto.

  • +
  • Allow configuring the user agent prefix for S3 requests +using the hive.s3.user-agent-prefix configuration property.

  • +
  • Add support for S3-compatible storage using the hive.s3.endpoint +and hive.s3.signer-type configuration properties.

  • +
  • Add support for using AWS KMS with S3 as an encryption materials provider +using the hive.s3.kms-key-id configuration property.

  • +
  • Allow configuring a custom S3 encryption materials provider using the +hive.s3.encryption-materials-provider configuration property.

  • +
+
+
+

JMX#

+
    +
  • Make name configuration for history tables case-insensitive.

  • +
+
+
+

MySQL#

+
    +
  • Optimize fetching column names when describing a single table.

  • +
  • Add support for char(x) and real data types.

  • +
+
+
+

PostgreSQL#

+
    +
  • Optimize fetching column names when describing a single table.

  • +
  • Add support for char(x) and real data types.

  • +
  • Add support for querying materialized views.

  • +
+
+
+

Blackhole#

+
    +
  • Add page_processing_delay table property.

  • +
+
+
+

SPI#

+
    +
  • Add schemaExists() method to ConnectorMetadata.

  • +
  • Add transaction to grant/revoke in ConnectorAccessControl.

  • +
  • Add isCoordinator() and getVersion() methods to Node.

  • +
  • Remove setOptionalConfig() method from Plugin.

  • +
  • Remove ServerInfo class.

  • +
  • Make NodeManager specific to a connector instance.

  • +
  • Replace ConnectorFactoryContext with ConnectorContext.

  • +
  • Use @SqlNullable for functions instead of @Nullable.

  • +
  • Prevent plugins from seeing classes that are not part of the JDK (bootstrap classes) or the SPI.

  • +
  • Update presto-maven-plugin, which provides a Maven packaging and +lifecycle for plugins, to validate that every SPI dependency is marked +as provided scope and that only SPI dependencies use provided +scope. This helps find potential dependency and class loader issues +at build time rather than at runtime.

  • +
+
+

Note

+

These are backwards incompatible changes with the previous SPI. +If you have written a plugin, you will need to update your code +before deploying this release.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.154.html b/430/release/release-0.154.html new file mode 100644 index 000000000..8f7313ca1 --- /dev/null +++ b/430/release/release-0.154.html @@ -0,0 +1,2577 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.154 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.154#

+
+

General#

+
    +
  • Fix planning issue that could cause JOIN queries involving functions +that return null on non-null input to produce incorrect results.

  • +
  • Fix regression that would cause certain queries involving uncorrelated +subqueries in IN predicates to fail during planning.

  • +
  • Fix potential “Input symbols do not match output symbols” +error when writing to bucketed tables.

  • +
  • Fix potential “Requested array size exceeds VM limit” error +that triggers the JVM’s OutOfMemoryError handling.

  • +
  • Improve performance of window functions with identical partitioning and +ordering but different frame specifications.

  • +
  • Add code-cache-collection-threshold config which controls when Presto +will attempt to force collection of the JVM code cache and reduce the +default threshold to 40%.

  • +
  • Add support for using LIKE with CREATE TABLE.

  • +
  • Add support for DESCRIBE INPUT to describe the requirements for +the input parameters to a prepared statement.

  • +
+
+
+

Hive#

+
    +
  • Fix handling of metastore cache TTL. With the introduction of the +per-transaction cache, the cache timeout was reset after each access, +which means cache entries might never expire.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.155.html b/430/release/release-0.155.html new file mode 100644 index 000000000..0939b94c0 --- /dev/null +++ b/430/release/release-0.155.html @@ -0,0 +1,2586 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.155 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.155#

+
+

General#

+
    +
  • Fix incorrect results when queries contain multiple grouping sets that +resolve to the same set.

  • +
  • Fix incorrect results when using map with IN predicates.

  • +
  • Fix compile failure for outer joins that have a complex join criteria.

  • +
  • Fix error messages for failures during commit.

  • +
  • Fix memory accounting for simple aggregation, top N and distinct queries. +These queries may now report higher memory usage than before.

  • +
  • Reduce unnecessary memory usage of map_agg(), multimap_agg() +and map_union().

  • +
  • Make INCLUDING, EXCLUDING and PROPERTIES non-reserved keywords.

  • +
  • Remove support for the experimental feature to compute approximate queries +based on sampled tables.

  • +
  • Properly account for time spent creating page source.

  • +
  • Various optimizations to reduce coordinator CPU usage.

  • +
+
+
+

Hive#

+
    +
  • Fix schema evolution support in new Parquet reader.

  • +
  • Fix NoClassDefFoundError when using Hadoop KMS.

  • +
  • Add support for Avro file format.

  • +
  • Always produce dictionary blocks for DWRF dictionary encoded streams.

  • +
+
+
+

SPI#

+
    +
  • Remove legacy connector API.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.156.html b/430/release/release-0.156.html new file mode 100644 index 000000000..32eeb19b1 --- /dev/null +++ b/430/release/release-0.156.html @@ -0,0 +1,2603 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.156 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.156#

+
+

Warning

+

Query may incorrectly produce NULL when no row qualifies for the aggregation +if the optimize_mixed_distinct_aggregations session property or +the optimizer.optimize-mixed-distinct-aggregations config option is enabled.

+
+
+

General#

+
    +
  • Fix potential correctness issue in queries that contain correlated scalar aggregation subqueries.

  • +
  • Fix query failure when using AT TIME ZONE in VALUES list.

  • +
  • Add support for quantified comparison predicates: ALL, ANY, and SOME.

  • +
  • Add support for ARRAY and ROW that contain NULL +in checksum() aggregation.

  • +
  • Add support for filtered aggregations. Example: SELECT sum(a) FILTER (WHERE b > 0) FROM ...

  • +
  • Add a variant of from_unixtime() function that takes a timezone argument.

  • +
  • Improve performance of GROUP BY queries that compute a mix of distinct +and non-distinct aggregations. This optimization can be turned on by setting +the optimizer.optimize-mixed-distinct-aggregations configuration option or +via the optimize_mixed_distinct_aggregations session property.

  • +
  • Change default task concurrency to 16.

  • +
+
+
+

Hive#

+
    +
  • Add support for legacy RCFile header version in new RCFile reader.

  • +
+
+
+

Redis#

+
    +
  • Support iso8601 data format for the hash row decoder.

  • +
+
+
+

SPI#

+
    +
  • Make ConnectorPageSink#finish() asynchronous.

  • +
+
+

Note

+

These are backwards incompatible changes with the previous SPI. +If you have written a plugin, you will need to update your code +before deploying this release.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.157.1.html b/430/release/release-0.157.1.html new file mode 100644 index 000000000..77622698f --- /dev/null +++ b/430/release/release-0.157.1.html @@ -0,0 +1,2551 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.157.1 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.157.1#

+
+

General#

+
    +
  • Fix regression that could cause high CPU and heap usage on coordinator, +when processing certain types of long running queries.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.157.html b/430/release/release-0.157.html new file mode 100644 index 000000000..6c461cfac --- /dev/null +++ b/430/release/release-0.157.html @@ -0,0 +1,2574 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.157 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.157#

+
+

General#

+
    +
  • Fix regression that could cause queries containing scalar subqueries to fail +during planning.

  • +
  • Reduce CPU usage of coordinator in large, heavily loaded clusters.

  • +
  • Add support for DESCRIBE OUTPUT.

  • +
  • Add bitwise_and_agg() and bitwise_or_agg() aggregation functions.

  • +
  • Add JMX stats for the scheduler.

  • +
  • Add query.min-schedule-split-batch-size config flag to set the minimum number of +splits to consider for scheduling per batch.

  • +
  • Remove support for scheduling multiple tasks in the same stage on a single worker.

  • +
  • Rename node-scheduler.max-pending-splits-per-node-per-stage to +node-scheduler.max-pending-splits-per-task. The old name may still be used, but is +deprecated and will be removed in a future version.

  • +
+
+
+

Hive#

+
    +
  • Fail attempts to create tables that are bucketed on non-existent columns.

  • +
  • Improve error message when trying to query tables that are bucketed on non-existent columns.

  • +
  • Add support for processing partitions whose schema does not match the table schema.

  • +
  • Add support for creating external Hive tables using the external_location table property.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.158.html b/430/release/release-0.158.html new file mode 100644 index 000000000..2ab247608 --- /dev/null +++ b/430/release/release-0.158.html @@ -0,0 +1,2588 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.158 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.158#

+
+

General#

+
    +
  • Fix regression that could cause high CPU and heap usage on coordinator +when processing certain types of long running queries.

  • +
  • Fix incorrect pruning of output columns in EXPLAIN ANALYZE.

  • +
  • Fix ordering of CHAR values so that trailing spaces are ordered after control characters.

  • +
  • Fix query failures for connectors that produce non-remotely accessible splits.

  • +
  • Fix non-linear performance issue when parsing certain SQL expressions.

  • +
  • Fix case-sensitivity issues when operating on columns of ROW data type.

  • +
  • Fix failure when creating views for tables names that need quoting.

  • +
  • Return NULL from element_at() for out-of-range indices instead of failing.

  • +
  • Remove redundancies in query plans, which can reduce data transfers over the network and reduce CPU requirements.

  • +
  • Validate resource groups configuration file on startup to ensure that all +selectors reference a configured resource group.

  • +
  • Add experimental on-disk merge sort for aggregations. This can be enabled with +the experimental.spill-enabled configuration flag.

  • +
  • Push down predicates for DECIMAL, TINYINT, SMALLINT and REAL data types.

  • +
+
+
+

Hive#

+
    +
  • Add hidden $bucket column for bucketed tables that +contains the bucket number for the current row.

  • +
  • Prevent inserting into non-managed (i.e., external) tables.

  • +
  • Add configurable size limit to Hive metastore cache to avoid using too much +coordinator memory.

  • +
+
+
+

Cassandra#

+
    +
  • Allow starting the server even if a contact point hostname cannot be resolved.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.159.html b/430/release/release-0.159.html new file mode 100644 index 000000000..0af38dca2 --- /dev/null +++ b/430/release/release-0.159.html @@ -0,0 +1,2562 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.159 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.159#

+
+

General#

+
    +
  • Improve predicate performance for JOIN queries.

  • +
+
+
+

Hive#

+
    +
  • Optimize filtering of partition names to reduce object creation.

  • +
  • Add limit on the number of partitions that can potentially be read per table scan. +This limit is configured using hive.max-partitions-per-scan and defaults to 100,000.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.160.html b/430/release/release-0.160.html new file mode 100644 index 000000000..005caf7e6 --- /dev/null +++ b/430/release/release-0.160.html @@ -0,0 +1,2569 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.160 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.160#

+
+

General#

+
    +
  • Fix planning failure when query has multiple unions with identical underlying columns.

  • +
  • Fix planning failure when multiple IN predicates contain an identical subquery.

  • +
  • Fix resource waste where coordinator floods rebooted workers if worker +comes back before coordinator times out the query.

  • +
  • Add Lambda expressions.

  • +
+
+
+

Hive#

+
    +
  • Fix planning failure when inserting into columns of struct types with uppercase field names.

  • +
  • Fix resource leak when using Kerberos authentication with impersonation.

  • +
  • Fix creating external tables so that they are properly recognized by the Hive metastore. +The Hive table property EXTERNAL is now set to TRUE in addition to the setting +the table type. Any previously created tables need to be modified to have this property.

  • +
  • Add bucket_execution_enabled session property.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.161.html b/430/release/release-0.161.html new file mode 100644 index 000000000..6bca2cb58 --- /dev/null +++ b/430/release/release-0.161.html @@ -0,0 +1,2589 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.161 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.161#

+
+

General#

+
    +
  • Fix correctness issue for queries involving multiple nested EXCEPT clauses. +A query such as a EXCEPT (b EXCEPT c) was incorrectly evaluated as +a EXCEPT b EXCEPT c and thus could return the wrong result.

  • +
  • Fix failure when executing prepared statements that contain parameters in the join criteria.

  • +
  • Fix failure when describing the output of prepared statements that contain aggregations.

  • +
  • Fix planning failure when a lambda is used in the context of an aggregation or subquery.

  • +
  • Fix column resolution rules for ORDER BY to match the behavior expected +by the SQL standard. This is a change in semantics that breaks +backwards compatibility. To ease migration of existing queries, the legacy +behavior can be restored by the deprecated.legacy-order-by config option +or the legacy_order_by session property.

  • +
  • Improve error message when coordinator responds with 403 FORBIDDEN.

  • +
  • Improve performance for queries containing expressions in the join criteria +that reference columns on one side of the join.

  • +
  • Improve performance of map_concat() when one argument is empty.

  • +
  • Remove /v1/execute resource.

  • +
  • Add new column to SHOW COLUMNS (and DESCRIBE) +to show extra information from connectors.

  • +
  • Add map() to construct an empty MAP.

  • +
+
+
+

Hive connector#

+
    +
  • Remove "Partition Key: " prefix from column comments and +replace it with the new extra information field described above.

  • +
+
+
+

JMX connector#

+
    +
  • Add support for escaped commas in jmx.dump-tables config property.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.162.html b/430/release/release-0.162.html new file mode 100644 index 000000000..929a2cc36 --- /dev/null +++ b/430/release/release-0.162.html @@ -0,0 +1,2591 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.162 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.162#

+
+

Warning

+

The xxhash64() function introduced in this release will return a +varbinary instead of a bigint in the next release.

+
+
+

General#

+
    +
  • Fix correctness issue when the type of the value in the IN predicate does +not match the type of the elements in the subquery.

  • +
  • Fix correctness issue when the value on the left-hand side of an IN +expression or a quantified comparison is NULL.

  • +
  • Fix correctness issue when the subquery of a quantified comparison produces no rows.

  • +
  • Fix correctness issue due to improper inlining of TRY arguments.

  • +
  • Fix correctness issue when the right side of a JOIN produces a very large number of rows.

  • +
  • Fix correctness issue for expressions with multiple nested AND and OR conditions.

  • +
  • Improve performance of window functions with similar PARTITION BY clauses.

  • +
  • Improve performance of certain multi-way JOINs by automatically choosing the +best evaluation order. This feature is turned off by default and can be enabled +via the reorder-joins config option or reorder_joins session property.

  • +
  • Add xxhash64() and to_big_endian_64() functions.

  • +
  • Add aggregated operator statistics to final query statistics.

  • +
  • Allow specifying column comments for CREATE TABLE.

  • +
+
+
+

Hive#

+
    +
  • Fix performance regression when querying Hive tables with large numbers of partitions.

  • +
+
+
+

SPI#

+
    +
  • Connectors can now return optional output metadata for write operations.

  • +
  • Add ability for event listeners to get connector-specific output metadata.

  • +
  • Add client-supplied payload field X-Presto-Client-Info to EventListener.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.163.html b/430/release/release-0.163.html new file mode 100644 index 000000000..38bbf7c99 --- /dev/null +++ b/430/release/release-0.163.html @@ -0,0 +1,2605 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.163 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.163#

+
+

General#

+
    +
  • Fix data corruption when transporting dictionary-encoded data.

  • +
  • Fix potential deadlock when resource groups are configured with memory limits.

  • +
  • Improve performance for OUTER JOIN queries.

  • +
  • Improve exchange performance by reading from buffers in parallel.

  • +
  • Improve performance when only a subset of the columns resulting from a JOIN are referenced.

  • +
  • Make ALL, SOME and ANY non-reserved keywords.

  • +
  • Add from_big_endian_64() function.

  • +
  • Change xxhash64() return type from BIGINT to VARBINARY.

  • +
  • Change subscript operator for map types to fail if the key is not present in the map. The former +behavior (returning NULL) can be restored by setting the deprecated.legacy-map-subscript +config option.

  • +
  • Improve EXPLAIN ANALYZE to render stats more accurately and to include input statistics.

  • +
  • Improve tolerance to communication errors for long running queries. This can be adjusted +with the query.remote-task.max-error-duration config option.

  • +
+
+
+

Accumulo#

+
    +
  • Fix issue that could cause incorrect results for large rows.

  • +
+
+
+

MongoDB#

+
    +
  • Fix NullPointerException when a field contains a null.

  • +
+
+
+

Cassandra#

+
    +
  • Add support for VARBINARY, TIMESTAMP and REAL data types.

  • +
+
+
+

Hive#

+
    +
  • Fix issue that would prevent predicates from being pushed into Parquet reader.

  • +
  • Fix Hive metastore user permissions caching when tables are dropped or renamed.

  • +
  • Add experimental file based metastore which stores information in HDFS or S3 instead of a database.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.164.html b/430/release/release-0.164.html new file mode 100644 index 000000000..c1ba0c4cf --- /dev/null +++ b/430/release/release-0.164.html @@ -0,0 +1,2584 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.164 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.164#

+
+

General#

+
    +
  • Fix correctness issue for queries that perform DISTINCT and LIMIT on the results of a JOIN.

  • +
  • Fix correctness issue when casting between maps where the key or value is the REAL type.

  • +
  • Fix correctness issue in min_by() and max_by() when nulls are present in the comparison column.

  • +
  • Fail queries when FILTER clause is specified for scalar functions.

  • +
  • Fix planning failure for certain correlated subqueries that contain aggregations.

  • +
  • Fix planning failure when arguments to selective aggregates are derived from other selective aggregates.

  • +
  • Fix boolean expression optimization bug that can cause long planning times, planning failures and coordinator instability.

  • +
  • Fix query failure when TRY or lambda expression with the exact same body is repeated in an expression.

  • +
  • Fix split source resource leak in coordinator that can occur when a query fails.

  • +
  • Improve array_join() performance.

  • +
  • Improve error message for map subscript operator when key is not present in the map.

  • +
  • Improve client error message for invalid session.

  • +
  • Add VALIDATE mode for EXPLAIN.

  • +
+
+
+

Web UI#

+
    +
  • Add resource group to query detail page.

  • +
+
+
+

Hive#

+
    +
  • Fix handling of ORC files containing extremely large metadata.

  • +
  • Fix failure when creating views in file based metastore.

  • +
  • Improve performance for queries that read bucketed tables by optimizing scheduling.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.165.html b/430/release/release-0.165.html new file mode 100644 index 000000000..e4a649aa0 --- /dev/null +++ b/430/release/release-0.165.html @@ -0,0 +1,2575 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.165 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.165#

+
+

General#

+
    +
  • Make AT a non-reserved keyword.

  • +
  • Improve performance of transform().

  • +
  • Improve exchange performance by deserializing in parallel.

  • +
  • Add support for compressed exchanges. This can be enabled with the exchange.compression-enabled +config option.

  • +
  • Add input and hash collision statistics to EXPLAIN ANALYZE output.

  • +
+
+
+

Hive#

+
    +
  • Add support for MAP and ARRAY types in optimized Parquet reader.

  • +
+
+
+

MySQL and PostgreSQL#

+
    +
  • Fix connection leak on workers.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.166.html b/430/release/release-0.166.html new file mode 100644 index 000000000..214050361 --- /dev/null +++ b/430/release/release-0.166.html @@ -0,0 +1,2585 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.166 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.166#

+
+

General#

+
    +
  • Fix failure due to implicit coercion issue in IN expressions for +certain combinations of data types (e.g., double and decimal).

  • +
  • Add query.max-length config flag to set the maximum length of a query. +The default maximum length is 1MB.

  • +
  • Improve performance of approx_percentile().

  • +
+
+
+

Hive#

+
    +
  • Include original exception from metastore for AlreadyExistsException when adding partitions.

  • +
  • Add support for the Hive JSON file format (org.apache.hive.hcatalog.data.JsonSerDe).

  • +
+
+
+

Cassandra#

+
    +
  • Add configuration properties for speculative execution.

  • +
+
+
+

SPI#

+
    +
  • Add peak memory reservation to SplitStatistics in split completion events.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.167.html b/430/release/release-0.167.html new file mode 100644 index 000000000..4338798af --- /dev/null +++ b/430/release/release-0.167.html @@ -0,0 +1,2642 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.167 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.167#

+
+

General#

+
    +
  • Fix planning failure when a window function depends on the output of another window function.

  • +
  • Fix planning failure for certain aggregation with both DISTINCT and GROUP BY.

  • +
  • Fix incorrect aggregation of operator summary statistics.

  • +
  • Fix a join issue that could cause joins that produce and filter many rows +to monopolize worker threads, even after the query has finished.

  • +
  • Expand plan predicate pushdown capabilities involving implicitly coerced types.

  • +
  • Short-circuit inner and right join when right side is empty.

  • +
  • Optimize constant patterns for LIKE predicates that use an escape character.

  • +
  • Validate escape sequences in LIKE predicates per the SQL standard.

  • +
  • Reduce memory usage of min_by() and max_by().

  • +
  • Add transform_keys(), transform_values() and zip_with() lambda functions.

  • +
  • Add levenshtein_distance() function.

  • +
  • Add JMX stat for the elapsed time of the longest currently active split.

  • +
  • Add JMX stats for compiler caches.

  • +
  • Raise required Java version to 8u92.

  • +
+
+
+

Security#

+
    +
  • The http.server.authentication.enabled config option that previously enabled +Kerberos has been replaced with http-server.authentication.type=KERBEROS.

  • +
  • Add support for LDAP authentication using username and password.

  • +
  • Add a read-only System access control named read-only.

  • +
  • Allow access controls to filter the results of listing catalogs, schemas and tables.

  • +
  • Add access control checks for SHOW SCHEMAS and SHOW TABLES.

  • +
+
+
+

Web UI#

+
    +
  • Add operator-level performance analysis.

  • +
  • Improve visibility of blocked and reserved query states.

  • +
  • Lots of minor improvements.

  • +
+
+
+

JDBC driver#

+
    +
  • Allow escaping in DatabaseMetaData patterns.

  • +
+
+
+

Hive#

+
    +
  • Fix write operations for ViewFileSystem by using a relative location.

  • +
  • Remove support for the hive-cdh4 and hive-hadoop1 connectors which +support CDH 4 and Hadoop 1.x, respectively.

  • +
  • Remove the hive-cdh5 connector as an alias for hive-hadoop2.

  • +
  • Remove support for the legacy S3 block-based file system.

  • +
  • Add support for KMS-managed keys for S3 server-side encryption.

  • +
+
+
+

Cassandra#

+
    +
  • Add support for Cassandra 3.x by removing the deprecated Thrift interface used to +connect to Cassandra. The following config options are now defunct and must be removed: +cassandra.thrift-port, cassandra.thrift-connection-factory-class, +cassandra.transport-factory-options and cassandra.partitioner.

  • +
+
+
+

SPI#

+
    +
  • Add methods to SystemAccessControl and ConnectorAccessControl to +filter the list of catalogs, schemas and tables.

  • +
  • Add access control checks for SHOW SCHEMAS and SHOW TABLES.

  • +
  • Add beginQuery and cleanupQuery notifications to ConnectorMetadata.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.168.html b/430/release/release-0.168.html new file mode 100644 index 000000000..d9a76f8ec --- /dev/null +++ b/430/release/release-0.168.html @@ -0,0 +1,2628 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.168 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.168#

+
+

General#

+
    +
  • Fix correctness issues for certain JOIN queries that require implicit coercions +for terms in the join criteria.

  • +
  • Fix invalid “No more locations already set” error.

  • +
  • Fix invalid “No more buffers already set” error.

  • +
  • Temporarily revert empty join short-circuit optimization due to issue with hanging queries.

  • +
  • Improve performance of DECIMAL type and operators.

  • +
  • Optimize window frame computation for empty frames.

  • +
  • json_extract() and json_extract_scalar() now support escaping double +quotes or backslashes using a backslash with a JSON path subscript. This changes +the semantics of any invocation using a backslash, as backslashes were previously +treated as normal characters.

  • +
  • Improve performance of filter() and map_filter() lambda functions.

  • +
  • Add Memory connector.

  • +
  • Add arrays_overlap() and array_except() functions.

  • +
  • Allow concatenating more than two arrays with concat() or maps with map_concat().

  • +
  • Add a time limit for the iterative optimizer. It can be adjusted via the iterative_optimizer_timeout +session property or experimental.iterative-optimizer-timeout configuration option.

  • +
  • ROW types are now orderable if all of the field types are orderable. +This allows using them in comparison expressions, ORDER BY and +functions that require orderable types (e.g., max()).

  • +
+
+
+

JDBC driver#

+
    +
  • Update DatabaseMetaData to reflect features that are now supported.

  • +
  • Update advertised JDBC version to 4.2, which part of Java 8.

  • +
  • Return correct driver and server versions rather than 1.0.

  • +
+
+
+

Hive#

+
    +
  • Fix reading decimals for RCFile text format using non-optimized reader.

  • +
  • Fix bug which prevented the file based metastore from being used.

  • +
  • Enable optimized RCFile reader by default.

  • +
  • Common user errors are now correctly categorized.

  • +
  • Add new, experimental, RCFile writer optimized for Presto. The new writer can be enabled with the +rcfile_optimized_writer_enabled session property or the hive.rcfile-optimized-writer.enabled +Hive catalog property.

  • +
+
+
+

Cassandra#

+
    +
  • Add predicate pushdown for clustering key.

  • +
+
+
+

MongoDB#

+
    +
  • Allow SSL connections using the mongodb.ssl.enabled config flag.

  • +
+
+
+

SPI#

+
    +
  • ConnectorIndex now returns ConnectorPageSource instead of RecordSet. Existing connectors +that support index join can use the RecordPageSource to adapt to the new API.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.169.html b/430/release/release-0.169.html new file mode 100644 index 000000000..19caab630 --- /dev/null +++ b/430/release/release-0.169.html @@ -0,0 +1,2584 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.169 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.169#

+
+

General#

+
    +
  • Fix regression that could cause queries involving JOIN and certain language features +such as current_date, current_time or extract to fail during planning.

  • +
  • Limit the maximum allowed input size to levenshtein_distance().

  • +
  • Improve performance of map_agg() and multimap_agg().

  • +
  • Improve memory accounting when grouping on a single BIGINT column.

  • +
+
+
+

JDBC driver#

+
    +
  • Return correct class name for ARRAY type from ResultSetMetaData.getColumnClassName().

  • +
+
+
+

CLI#

+
    +
  • Fix support for non-standard offset time zones (e.g., GMT+01:00).

  • +
+
+
+

Cassandra#

+
    +
  • Add custom error codes.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.170.html b/430/release/release-0.170.html new file mode 100644 index 000000000..e2c728d9e --- /dev/null +++ b/430/release/release-0.170.html @@ -0,0 +1,2613 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.170 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.170#

+
+

General#

+
    +
  • Fix race condition that could cause queries to fail with InterruptedException in rare cases.

  • +
  • Fix a performance regression for GROUP BY queries over UNION.

  • +
  • Fix a performance regression that occurs when a significant number of exchange +sources produce no data during an exchange (e.g., in a skewed hash join).

  • +
+
+
+

Web UI#

+
    +
  • Fix broken rendering when catalog properties are set.

  • +
  • Fix rendering of live plan when query is queued.

  • +
+
+
+

JDBC driver#

+
    +
  • Add support for DatabaseMetaData.getTypeInfo().

  • +
+
+
+

Hive#

+
    +
  • Improve decimal support for the Parquet reader.

  • +
  • Remove misleading “HDFS” string from error messages.

  • +
+
+
+

Cassandra#

+
    +
  • Fix an intermittent connection issue for Cassandra 2.1.

  • +
  • Remove support for selecting by partition key when the partition key is only partially specified. +The cassandra.limit-for-partition-key-select and cassandra.fetch-size-for-partition-key-select +config options are no longer supported.

  • +
  • Remove partition key cache to improve consistency and reduce load on the Cassandra cluster due to background cache refresh.

  • +
  • Reduce the number of connections opened to the Cassandra cluster. Now Presto opens a single connection from each node.

  • +
  • Use exponential backoff for retries when Cassandra hosts are down. The retry timeout can be controlled via the +cassandra.no-host-available-retry-timeout config option, which has a default value of 1m. +The cassandra.no-host-available-retry-count config option is no longer supported.

  • +
+
+
+

Verifier#

+
    +
  • Add support for INSERT queries.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.171.html b/430/release/release-0.171.html new file mode 100644 index 000000000..3e03e5c29 --- /dev/null +++ b/430/release/release-0.171.html @@ -0,0 +1,2581 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.171 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.171#

+
+

General#

+
    +
  • Fix planning regression for queries that compute a mix of distinct and non-distinct aggregations.

  • +
  • Fix casting from certain complex types to JSON when source type contains JSON or DECIMAL.

  • +
  • Fix issue for data definition queries that prevented firing completion events or purging them from +the coordinator’s memory.

  • +
  • Add support for capture in lambda expressions.

  • +
  • Add support for ARRAY and ROW type as the compared value in min_by() and max_by().

  • +
  • Add support for CHAR(n) data type to common string functions.

  • +
  • Add codepoint(), skewness() and kurtosis() functions.

  • +
  • Improve validation of resource group configuration.

  • +
  • Fail queries when casting unsupported types to JSON; see JSON functions and operators for supported types.

  • +
+
+
+

Web UI#

+
    +
  • Fix the threads UI (/ui/thread).

  • +
+
+
+

Hive#

+
    +
  • Fix issue where some files are not deleted on cancellation of INSERT or CREATE queries.

  • +
  • Allow writing to non-managed (external) Hive tables. This is disabled by default but can be +enabled via the hive.non-managed-table-writes-enabled configuration option.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.172.html b/430/release/release-0.172.html new file mode 100644 index 000000000..1f0afd0f7 --- /dev/null +++ b/430/release/release-0.172.html @@ -0,0 +1,2556 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.172 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.172#

+
+

General#

+
    +
  • Fix correctness issue in ORDER BY queries due to improper implicit coercions.

  • +
  • Fix planning failure when GROUP BY queries contain lambda expressions.

  • +
  • Fix planning failure when left side of IN expression contains subqueries.

  • +
  • Fix incorrect permissions check for SHOW TABLES.

  • +
  • Fix planning failure when JOIN clause contains lambda expressions that reference columns or variables from the enclosing scope.

  • +
  • Reduce memory usage of map_agg() and map_union().

  • +
  • Reduce memory usage of GROUP BY queries.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.173.html b/430/release/release-0.173.html new file mode 100644 index 000000000..26124f455 --- /dev/null +++ b/430/release/release-0.173.html @@ -0,0 +1,2551 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.173 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.173#

+
+

General#

+ +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.174.html b/430/release/release-0.174.html new file mode 100644 index 000000000..560c06153 --- /dev/null +++ b/430/release/release-0.174.html @@ -0,0 +1,2601 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.174 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.174#

+
+

General#

+
    +
  • Fix correctness issue for correlated subqueries containing a LIMIT clause.

  • +
  • Fix query failure when reduce() function is used with lambda expressions +containing array_sort(), shuffle(), reverse(), array_intersect(), +arrays_overlap(), concat() (for arrays) or map_concat().

  • +
  • Fix a bug that causes underestimation of the amount of memory used by max_by(), +min_by(), max(), min(), and arbitrary() aggregations over +varchar/varbinary columns.

  • +
  • Fix a memory leak in the coordinator that causes long-running queries in highly loaded +clusters to consume unnecessary memory.

  • +
  • Improve performance of aggregate window functions.

  • +
  • Improve parallelism of queries involving GROUPING SETS, CUBE or ROLLUP.

  • +
  • Improve parallelism of UNION queries.

  • +
  • Filter and projection operations are now always processed columnar if possible, and Presto +will automatically take advantage of dictionary encodings where effective. +The processing_optimization session property and optimizer.processing-optimization +configuration option have been removed.

  • +
  • Add support for escaped unicode sequences in string literals.

  • +
  • Add SHOW GRANTS and information_schema.table_privileges table.

  • +
+
+
+

Hive#

+
    +
  • Change default value of hive.metastore-cache-ttl and hive.metastore-refresh-interval to 0 +to disable cross-transaction metadata caching.

  • +
+
+
+

Web UI#

+
    +
  • Fix ES6 compatibility issue with older browsers.

  • +
  • Display buffered bytes for every stage in the live plan UI.

  • +
+
+
+

SPI#

+
    +
  • Add support for retrieving table grants.

  • +
  • Rename SPI access control check from checkCanShowTables to checkCanShowTablesMetadata, +which is used for both SHOW TABLES and SHOW GRANTS.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.175.html b/430/release/release-0.175.html new file mode 100644 index 000000000..c7685a21d --- /dev/null +++ b/430/release/release-0.175.html @@ -0,0 +1,2588 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.175 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.175#

+
+

General#

+
    +
  • Fix “position is not valid” query execution failures.

  • +
  • Fix memory accounting bug that can potentially cause OutOfMemoryError.

  • +
  • Fix regression that could cause certain queries involving UNION and +GROUP BY or JOIN to fail during planning.

  • +
  • Fix planning failure for GROUP BY queries containing correlated +subqueries in the SELECT clause.

  • +
  • Fix execution failure for certain DELETE queries.

  • +
  • Reduce occurrences of “Method code too large” errors.

  • +
  • Reduce memory utilization for certain queries involving ORDER BY.

  • +
  • Improve performance of map subscript from O(n) to O(1) when the map is +produced by an eligible operation, including the map constructor and +Hive readers (except ORC and optimized Parquet). More read and write +operations will take advantage of this in future releases.

  • +
  • Add enable_intermediate_aggregations session property to enable the +use of intermediate aggregations within un-grouped aggregations.

  • +
  • Add support for INTERVAL data type to avg() and sum() aggregation functions.

  • +
  • Add support for INT as an alias for the INTEGER data type.

  • +
  • Add resource group information to query events.

  • +
+
+
+

Hive#

+
    +
  • Make table creation metastore operations idempotent, which allows +recovery when retrying timeouts or other errors.

  • +
+
+
+

MongoDB#

+
    +
  • Rename mongodb.connection-per-host config option to mongodb.connections-per-host.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.176.html b/430/release/release-0.176.html new file mode 100644 index 000000000..638089a27 --- /dev/null +++ b/430/release/release-0.176.html @@ -0,0 +1,2586 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.176 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.176#

+
+

General#

+
    +
  • Fix an issue where a query (and some of its tasks) continues to +consume CPU/memory on the coordinator and workers after the query fails.

  • +
  • Fix a regression that cause the GC overhead and pauses to increase significantly when processing maps.

  • +
  • Fix a memory tracking bug that causes the memory to be overestimated for GROUP BY queries on bigint columns.

  • +
  • Improve the performance of the transform_values() function.

  • +
  • Add support for casting from JSON to REAL type.

  • +
  • Add parse_duration() function.

  • +
+
+
+

MySQL#

+
    +
  • Disallow having a database in the connection-url config property.

  • +
+
+
+

Accumulo#

+
    +
  • Decrease planning time by fetching index metrics in parallel.

  • +
+
+
+

MongoDB#

+
    +
  • Allow predicate pushdown for ObjectID.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.177.html b/430/release/release-0.177.html new file mode 100644 index 000000000..321d510ca --- /dev/null +++ b/430/release/release-0.177.html @@ -0,0 +1,2639 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.177 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.177#

+
+

Warning

+

Query may incorrectly produce NULL when no row qualifies for the aggregation +if the optimize_mixed_distinct_aggregations session property or +the optimizer.optimize-mixed-distinct-aggregations config option is enabled. +This optimization was introduced in Presto version 0.156.

+
+
+

General#

+
    +
  • Fix correctness issue when performing range comparisons over columns of type CHAR.

  • +
  • Fix correctness issue due to mishandling of nulls and non-deterministic expressions in +inequality joins unless fast_inequality_join is disabled.

  • +
  • Fix excessive GC overhead caused by lambda expressions. There are still known GC issues +with captured lambda expressions. This will be fixed in a future release.

  • +
  • Check for duplicate columns in CREATE TABLE before asking the connector to create +the table. This improves the error message for most connectors and will prevent errors +for connectors that do not perform validation internally.

  • +
  • Add support for null values on the left-hand side of a semijoin (i.e., IN predicate +with subqueries).

  • +
  • Add SHOW STATS to display table and query statistics.

  • +
  • Improve implicit coercion support for functions involving lambda. Specifically, this makes +it easier to use the reduce() function.

  • +
  • Improve plans for queries involving ORDER BY and LIMIT by avoiding unnecessary +data exchanges.

  • +
  • Improve performance of queries containing window functions with identical PARTITION BY +and ORDER BY clauses.

  • +
  • Improve performance of certain queries involving OUTER JOIN and aggregations, or +containing certain forms of correlated subqueries. This optimization is experimental +and can be turned on via the push_aggregation_through_join session property or the +optimizer.push-aggregation-through-join config option.

  • +
  • Improve performance of certain queries involving joins and aggregations. This optimization +is experimental and can be turned on via the push_partial_aggregation_through_join +session property.

  • +
  • Improve error message when a lambda expression has a different number of arguments than expected.

  • +
  • Improve error message when certain invalid GROUP BY expressions containing lambda expressions.

  • +
+
+
+

Hive#

+
    +
  • Fix handling of trailing spaces for the CHAR type when reading RCFile.

  • +
  • Allow inserts into tables that have more partitions than the partitions-per-scan limit.

  • +
  • Add support for exposing Hive table statistics to the engine. This option is experimental and +can be turned on via the statistics_enabled session property.

  • +
  • Ensure file name is always present for error messages about corrupt ORC files.

  • +
+
+
+

Cassandra#

+
    +
  • Remove caching of metadata in the Cassandra connector. Metadata caching makes Presto violate +the consistency defined by the Cassandra cluster. It’s also unnecessary because the Cassandra +driver internally caches metadata. The cassandra.max-schema-refresh-threads, +cassandra.schema-cache-ttl and cassandra.schema-refresh-interval config options have +been removed.

  • +
  • Fix intermittent issue in the connection retry mechanism.

  • +
+
+
+

Web UI#

+
    +
  • Change cluster HUD realtime statistics to be aggregated across all running queries.

  • +
  • Change parallelism statistic on cluster HUD to be averaged per-worker.

  • +
  • Fix bug that always showed indeterminate progress bar in query list view.

  • +
  • Change running drivers statistic to exclude blocked drivers.

  • +
  • Change unit of CPU and scheduled time rate sparklines to seconds on query details page.

  • +
  • Change query details page refresh interval to three seconds.

  • +
  • Add uptime and connected status indicators to every page.

  • +
+
+
+

CLI#

+
    +
  • Add support for preprocessing commands. When the PRESTO_PREPROCESSOR environment +variable is set, all commands are piped through the specified program before being sent to +the Presto server.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.178.html b/430/release/release-0.178.html new file mode 100644 index 000000000..018a9394c --- /dev/null +++ b/430/release/release-0.178.html @@ -0,0 +1,2588 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.178 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.178#

+
+

General#

+
    +
  • Fix various memory accounting bugs, which reduces the likelihood of full GCs/OOMs.

  • +
  • Fix a regression that causes queries that use the keyword “stats” to fail to parse.

  • +
  • Fix an issue where a query does not get cleaned up on the coordinator after query failure.

  • +
  • Add ability to cast to JSON from REAL, TINYINT or SMALLINT.

  • +
  • Add support for GROUPING operation to complex grouping operations.

  • +
  • Add support for correlated subqueries in IN predicates.

  • +
  • Add to_ieee754_32() and to_ieee754_64() functions.

  • +
+
+
+

Hive#

+
    +
  • Fix high CPU usage due to schema caching when reading Avro files.

  • +
  • Preserve decompression error causes when decoding ORC files.

  • +
+
+
+

Memory connector#

+
    +
  • Fix a bug that prevented creating empty tables.

  • +
+
+
+

SPI#

+
    +
  • Make environment available to resource group configuration managers.

  • +
  • Add additional performance statistics to query completion event.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.179.html b/430/release/release-0.179.html new file mode 100644 index 000000000..642ddaa76 --- /dev/null +++ b/430/release/release-0.179.html @@ -0,0 +1,2597 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.179 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.179#

+
+

General#

+
    +
  • Fix issue which could cause incorrect results when processing dictionary encoded data. +If the expression can fail on bad input, the results from filtered-out rows containing +bad input may be included in the query output (#8262).

  • +
  • Fix planning failure when similar expressions appear in the ORDER BY clause of a query that +contains ORDER BY and LIMIT.

  • +
  • Fix planning failure when GROUPING() is used with the legacy_order_by session property set to true.

  • +
  • Fix parsing failure when NFD, NFC, NFKD or NFKC are used as identifiers.

  • +
  • Fix a memory leak on the coordinator that manifests itself with canceled queries.

  • +
  • Fix excessive GC overhead caused by captured lambda expressions.

  • +
  • Reduce the memory usage of map/array aggregation functions.

  • +
  • Redact sensitive config property values in the server log.

  • +
  • Update timezone database to version 2017b.

  • +
  • Add repeat() function.

  • +
  • Add crc32() function.

  • +
  • Add file based global security, which can be configured with the etc/access-control.properties +and security.config-file config properties. See System access control +for more details.

  • +
  • Add support for configuring query runtime and queueing time limits to resource groups.

  • +
+
+
+

Hive#

+
    +
  • Fail queries that access encrypted S3 objects that do not have their unencrypted content lengths set in their metadata.

  • +
+
+
+

JDBC driver#

+
    +
  • Add support for setting query timeout through Statement.setQueryTimeout().

  • +
+
+
+

SPI#

+
    +
  • Add grantee and revokee to GRANT and REVOKE security checks.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.180.html b/430/release/release-0.180.html new file mode 100644 index 000000000..b80901fd0 --- /dev/null +++ b/430/release/release-0.180.html @@ -0,0 +1,2630 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.180 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.180#

+
+

General#

+
    +
  • Fix a rare bug where rows containing only null values are not returned +to the client. This only occurs when an entire result page contains only +null values. The only known case is a query over an ORC encoded Hive table +that does not perform any transformation of the data.

  • +
  • Fix incorrect results when performing comparisons between values of approximate +data types (REAL, DOUBLE) and columns of certain exact numeric types +(INTEGER, BIGINT, DECIMAL).

  • +
  • Fix memory accounting for min_by() and max_by() on complex types.

  • +
  • Fix query failure due to NoClassDefFoundError when scalar functions declared +in plugins are implemented with instance methods.

  • +
  • Improve performance of map subscript from O(n) to O(1) in all cases. Previously, only maps +produced by certain functions and readers could take advantage of this improvement.

  • +
  • Skip unknown costs in EXPLAIN output.

  • +
  • Support Secure internal communication between Presto nodes.

  • +
  • Add initial support for CROSS JOIN against LATERAL derived tables.

  • +
  • Add support for VARBINARY concatenation.

  • +
  • Add Thrift connector that makes it possible to use Presto with +external systems without the need to implement a custom connector.

  • +
  • Add experimental /v1/resourceGroupState REST endpoint on coordinator.

  • +
+
+
+

Hive#

+
    +
  • Fix skipping short decimal values in the optimized Parquet reader +when they are backed by the int32 or int64 types.

  • +
  • Ignore partition bucketing if table is not bucketed. This allows dropping +the bucketing from table metadata but leaving it for old partitions.

  • +
  • Improve error message for Hive partitions dropped during execution.

  • +
  • The optimized RCFile writer is enabled by default, but can be disabled +with the hive.rcfile-optimized-writer.enabled config option. +The writer supports validation which reads back the entire file after +writing. Validation is disabled by default, but can be enabled with the +hive.rcfile.writer.validate config option.

  • +
+
+
+

Cassandra#

+
    +
  • Add support for INSERT.

  • +
  • Add support for pushdown of non-equality predicates on clustering keys.

  • +
+
+
+

JDBC driver#

+
    +
  • Add support for authenticating using Kerberos.

  • +
  • Allow configuring SSL/TLS and Kerberos properties on a per-connection basis.

  • +
  • Add support for executing queries using a SOCKS or HTTP proxy.

  • +
+
+
+

CLI#

+
    +
  • Add support for executing queries using an HTTP proxy.

  • +
+
+
+

SPI#

+
    +
  • Add running time limit and queued time limit to ResourceGroupInfo.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.181.html b/430/release/release-0.181.html new file mode 100644 index 000000000..42554525b --- /dev/null +++ b/430/release/release-0.181.html @@ -0,0 +1,2633 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.181 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.181#

+
+

General#

+
    +
  • Fix query failure and memory usage tracking when query contains +transform_keys() or transform_values().

  • +
  • Prevent CREATE TABLE IF NOT EXISTS queries from ever failing with “Table already exists”.

  • +
  • Fix query failure when ORDER BY expressions reference columns that are used in +the GROUP BY clause by their fully-qualified name.

  • +
  • Fix excessive GC overhead caused by large arrays and maps containing VARCHAR elements.

  • +
  • Improve error handling when passing too many arguments to various +functions or operators that take a variable number of arguments.

  • +
  • Improve performance of count(*) aggregations over subqueries with known +constant cardinality.

  • +
  • Add VERBOSE option for EXPLAIN ANALYZE that provides additional +low-level details about query performance.

  • +
  • Add per-task distribution information to the output of EXPLAIN ANALYZE.

  • +
  • Add support for DROP COLUMN in ALTER TABLE.

  • +
  • Change local scheduler to prevent starvation of long running queries +when the cluster is under constant load from short queries. The new +behavior is disabled by default and can be enabled by setting the +config property task.level-absolute-priority=true.

  • +
  • Improve the fairness of the local scheduler such that long-running queries +which spend more time on the CPU per scheduling quanta (e.g., due to +slow connectors) do not get a disproportionate share of CPU. The new +behavior is disabled by default and can be enabled by setting the +config property task.legacy-scheduling-behavior=false.

  • +
  • Add a config option to control the prioritization of queries based on +elapsed scheduled time. The task.level-time-multiplier property +controls the target scheduled time of a level relative to the next +level. Higher values for this property increase the fraction of CPU +that will be allocated to shorter queries. This config property only +has an effect when task.level-absolute-priority=true and +task.legacy-scheduling-behavior=false.

  • +
+
+
+

Hive#

+
    +
  • Fix potential native memory leak when writing tables using RCFile.

  • +
  • Correctly categorize certain errors when writing tables using RCFile.

  • +
  • Decrease the number of file system metadata calls when reading tables.

  • +
  • Add support for dropping columns.

  • +
+
+
+

JDBC driver#

+
    +
  • Add support for query cancellation using Statement.cancel().

  • +
+
+
+

PostgreSQL#

+
    +
  • Add support for operations on external tables.

  • +
+
+
+

Accumulo#

+
    +
  • Improve query performance by scanning index ranges in parallel.

  • +
+
+
+

SPI#

+
    +
  • Fix regression that broke serialization for SchemaTableName.

  • +
  • Add access control check for DROP COLUMN.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.182.html b/430/release/release-0.182.html new file mode 100644 index 000000000..3b0137f45 --- /dev/null +++ b/430/release/release-0.182.html @@ -0,0 +1,2594 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.182 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.182#

+
+

General#

+
    +
  • Fix correctness issue that causes corr() to return positive numbers for inverse correlations.

  • +
  • Fix the EXPLAIN query plan for tables that are partitioned +on TIMESTAMP or DATE columns.

  • +
  • Fix query failure when using certain window functions that take arrays or maps as arguments (e.g., approx_percentile()).

  • +
  • Implement subtraction for all TIME and TIMESTAMP types.

  • +
  • Improve planning performance for queries that join multiple tables with +a large number columns.

  • +
  • Improve the performance of joins with only non-equality conditions by using +a nested loops join instead of a hash join.

  • +
  • Improve the performance of casting from JSON to ARRAY or MAP types.

  • +
  • Add a new IPADDRESS type to represent IP addresses.

  • +
  • Add to_milliseconds() function to convert intervals (day to second) to milliseconds.

  • +
  • Add support for column aliases in CREATE TABLE AS statements.

  • +
  • Add a config option to reject queries during cluster initialization. +Queries are rejected if the active worker count is less than the +query-manager.initialization-required-workers property while the +coordinator has been running for less than query-manager.initialization-timeout.

  • +
  • Add TPCDS connector. This connector provides a set of schemas to +support the TPC Benchmark™ DS (TPC-DS).

  • +
+
+
+

CLI#

+
    +
  • Fix an issue that would sometimes prevent queries from being cancelled when exiting from the pager.

  • +
+
+
+

Hive#

+
    +
  • Fix reading decimal values in the optimized Parquet reader when they are backed +by the int32 or int64 types.

  • +
  • Add a new experimental ORC writer implementation optimized for Presto. +We have some upcoming improvements, so we recommend waiting a few releases before +using this in production. The new writer can be enabled with the +hive.orc.optimized-writer.enabled configuration property or with the +orc_optimized_writer_enabled session property.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.183.html b/430/release/release-0.183.html new file mode 100644 index 000000000..b18f858a1 --- /dev/null +++ b/430/release/release-0.183.html @@ -0,0 +1,2628 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.183 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.183#

+
+

General#

+
    +
  • Fix planning failure for queries that use GROUPING and contain aggregation expressions +that require implicit coercions.

  • +
  • Fix planning failure for queries that contains a non-equi left join that is semantically +equivalent to an inner join.

  • +
  • Fix issue where a query may have a reported memory that is higher than actual usage when +an aggregation is followed by other non-trivial work in the same stage. This can lead to failures +due to query memory limit, or lower cluster throughput due to perceived insufficient memory.

  • +
  • Fix query failure for CHAR functions trim(), rtrim(), and substr() when +the return value would have trailing spaces under VARCHAR semantics.

  • +
  • Fix formatting in EXPLAIN ANALYZE output.

  • +
  • Improve error message when a query contains an unsupported form of correlated subquery.

  • +
  • Improve performance of CAST(json_parse(...) AS ...).

  • +
  • Add map_from_entries() and map_entries() functions.

  • +
  • Change spilling for aggregations to only occur when the cluster runs out of memory.

  • +
  • Remove the experimental.operator-memory-limit-before-spill config property +and the operator_memory_limit_before_spill session property.

  • +
  • Allow configuring the amount of memory that can be used for merging spilled aggregation data +from disk using the experimental.aggregation-operator-unspill-memory-limit config +property or the aggregation_operator_unspill_memory_limit session property.

  • +
+
+
+

Web UI#

+
    +
  • Add output rows, output size, written rows and written size to query detail page.

  • +
+
+
+

Hive#

+
    +
  • Work around ORC-222 which results in +invalid summary statistics in ORC or DWRF files when the input data contains invalid string data. +Previously, this would usually cause the query to fail, but in rare cases it could +cause wrong results by incorrectly skipping data based on the invalid statistics.

  • +
  • Fix issue where reported memory is lower than actual usage for table columns containing +string values read from ORC or DWRF files. This can lead to high GC overhead or out-of-memory crash.

  • +
  • Improve error message for small ORC files that are completely corrupt or not actually ORC.

  • +
  • Add predicate pushdown for the hidden column "$path".

  • +
+
+
+

TPCH#

+
    +
  • Add column statistics for schemas tiny and sf1.

  • +
+
+
+

TPCDS#

+
    +
  • Add column statistics for schemas tiny and sf1.

  • +
+
+
+

SPI#

+
    +
  • Map columns or values represented with ArrayBlock and InterleavedBlock are +no longer supported. They must be represented as MapBlock or SingleMapBlock.

  • +
  • Extend column statistics with minimal and maximal value.

  • +
  • Replace nullsCount with nullsFraction in column statistics.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.184.html b/430/release/release-0.184.html new file mode 100644 index 000000000..f4a2b0d21 --- /dev/null +++ b/430/release/release-0.184.html @@ -0,0 +1,2616 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.184 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.184#

+
+

General#

+
    +
  • Fix query execution failure for split_to_map(...)[...].

  • +
  • Fix issue that caused queries containing CROSS JOIN to continue using CPU resources +even after they were killed.

  • +
  • Fix planning failure for some query shapes containing count(*) and a non-empty +GROUP BY clause.

  • +
  • Fix communication failures caused by lock contention in the local scheduler.

  • +
  • Improve performance of element_at() for maps to be constant time rather than +proportional to the size of the map.

  • +
  • Improve performance of queries with gathering exchanges.

  • +
  • Require coalesce() to have at least two arguments, as mandated by the SQL standard.

  • +
  • Add hamming_distance() function.

  • +
+
+
+

JDBC driver#

+
    +
  • Always invoke the progress callback with the final stats at query completion.

  • +
+
+
+

Web UI#

+
    +
  • Add worker status page with information about currently running threads +and resource utilization (CPU, heap, memory pools). This page is accessible +by clicking a hostname on a query task list.

  • +
+
+
+

Hive#

+
    +
  • Fix partition filtering for keys of CHAR, DECIMAL, or DATE type.

  • +
  • Reduce system memory usage when reading table columns containing string values +from ORC or DWRF files. This can prevent high GC overhead or out-of-memory crashes.

  • +
+
+
+

TPCDS#

+
    +
  • Fix display of table statistics when running SHOW STATS FOR ....

  • +
+
+
+

SPI#

+
    +
  • Row columns or values represented with ArrayBlock and InterleavedBlock are +no longer supported. They must be represented as RowBlock or SingleRowBlock.

  • +
  • Add source field to ConnectorSession.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.185.html b/430/release/release-0.185.html new file mode 100644 index 000000000..cefa3cdd1 --- /dev/null +++ b/430/release/release-0.185.html @@ -0,0 +1,2589 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.185 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.185#

+
+

General#

+
    +
  • Fix incorrect column names in QueryCompletedEvent.

  • +
  • Fix excessive CPU usage in coordinator for queries that have +large string literals containing non-ASCII characters.

  • +
  • Fix potential infinite loop during query optimization when constant +expressions fail during evaluation.

  • +
  • Fix incorrect ordering when the same field appears multiple times +with different ordering specifications in a window function ORDER BY +clause. For example: OVER (ORDER BY x ASC, x DESC).

  • +
  • Do not allow dropping or renaming hidden columns.

  • +
  • When preparing to drop a column, ignore hidden columns when +checking if the table only has one column.

  • +
  • Improve performance of joins where the condition is a range over a function. +For example: a JOIN b ON b.x < f(a.x) AND b.x > g(a.x)

  • +
  • Improve performance of certain window functions (e.g., LAG) with similar specifications.

  • +
  • Extend substr() function to work on VARBINARY in addition to CHAR and VARCHAR.

  • +
  • Add cast from JSON to ROW.

  • +
  • Allow usage of TRY within lambda expressions.

  • +
+
+
+

Hive#

+
    +
  • Improve ORC reader efficiency by only reading small ORC streams when accessed in the query.

  • +
  • Improve RCFile IO efficiency by increasing the buffer size from 1 to 8 MB.

  • +
  • Fix native memory leak for optimized RCFile writer.

  • +
  • Fix potential native memory leak for optimized ORC writer.

  • +
+
+
+

Memory connector#

+
    +
  • Add support for views.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.186.html b/430/release/release-0.186.html new file mode 100644 index 000000000..f18010366 --- /dev/null +++ b/430/release/release-0.186.html @@ -0,0 +1,2628 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.186 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.186#

+
+

Warning

+

This release has a stability issue that may cause query failures in large deployments +due to HTTP requests timing out.

+
+
+

General#

+
    +
  • Fix excessive GC overhead caused by map to map cast.

  • +
  • Fix implicit coercions for ROW types, allowing operations between +compatible types such as ROW(INTEGER) and ROW(BIGINT).

  • +
  • Fix issue that may cause queries containing expensive functions, such as regular +expressions, to continue using CPU resources even after they are killed.

  • +
  • Fix performance issue caused by redundant casts.

  • +
  • Fix json_parse() to not ignore trailing characters. Previously, +input such as [1,2]abc would successfully parse as [1,2].

  • +
  • Fix leak in running query counter for failed queries. The counter would +increment but never decrement for queries that failed before starting.

  • +
  • Reduce coordinator HTTP thread usage for queries that are queued or waiting for data.

  • +
  • Reduce memory usage when building data of VARCHAR or VARBINARY types.

  • +
  • Estimate memory usage for GROUP BY more precisely to avoid out of memory errors.

  • +
  • Add queued time and elapsed time to the client protocol.

  • +
  • Add query_max_execution_time session property and query.max-execution-time config +property. Queries will be aborted after they execute for more than the specified duration.

  • +
  • Add inverse_normal_cdf() function.

  • +
  • Add Geospatial functions including functions for processing Bing tiles.

  • +
  • Add Spill to disk for joins.

  • +
  • Add Redshift connector.

  • +
+
+
+

Resource groups#

+
    +
  • Query Queues are deprecated in favor of Resource groups +and will be removed in a future release.

  • +
  • Rename the maxRunning property to hardConcurrencyLimit. The old +property name is deprecated and will be removed in a future release.

  • +
  • Fail on unknown property names when loading the JSON config file.

  • +
+
+
+

JDBC driver#

+
    +
  • Allow specifying an empty password.

  • +
  • Add getQueuedTimeMillis() and getElapsedTimeMillis() to QueryStats.

  • +
+
+
+

Hive#

+
    +
  • Fix FileSystem closed errors when using Kerberos authentication.

  • +
  • Add support for path style access to the S3 file system. This can be enabled +by setting the hive.s3.path-style-access=true config property.

  • +
+
+
+

SPI#

+
    +
  • Add an ignoreExisting flag to ConnectorMetadata::createTable().

  • +
  • Remove the getTotalBytes() method from RecordCursor and ConnectorPageSource.

  • +
+
+

Note

+

These are backwards incompatible changes with the previous SPI. +If you have written a connector, you will need to update your code +before deploying this release.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.187.html b/430/release/release-0.187.html new file mode 100644 index 000000000..88e551ee7 --- /dev/null +++ b/430/release/release-0.187.html @@ -0,0 +1,2551 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.187 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.187#

+
+

General#

+
    +
  • Fix a stability issue that may cause query failures due to a large number of HTTP requests timing out. +The issue has been observed in a large deployment under stress.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.188.html b/430/release/release-0.188.html new file mode 100644 index 000000000..c4fd48594 --- /dev/null +++ b/430/release/release-0.188.html @@ -0,0 +1,2608 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.188 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.188#

+
+

General#

+
    +
  • Fix handling of negative start indexes in array slice() function.

  • +
  • Fix inverted sign for time zones Etc/GMT-12, Etc/GMT-11, …, Etc/GMT-1, +Etc/GMT+1, … Etc/GMT+12.

  • +
  • Improve performance of server logging and HTTP request logging.

  • +
  • Reduce GC spikes by compacting join memory over time instead of all at once +when memory is low. This can increase reliability at the cost of additional +CPU. This can be enabled via the pages-index.eager-compaction-enabled +config property.

  • +
  • Improve performance of and reduce GC overhead for compaction of in-memory data structures, +primarily used in joins.

  • +
  • Mitigate excessive GC and degraded query performance by forcing expiration of +generated classes for functions and expressions one hour after generation.

  • +
  • Mitigate performance issue caused by JVM when generated code is used +for multiple hours or days.

  • +
+
+
+

CLI#

+
    +
  • Fix transaction support. Previously, after the first statement in the +transaction, the transaction would be abandoned and the session would +silently revert to auto-commit mode.

  • +
+
+
+

JDBC driver#

+
    +
  • Support using Statement.cancel() for all types of statements.

  • +
+
+
+

Resource group#

+
    +
  • Add environment support to the db resource groups manager. +Previously, configurations for different clusters had to be stored in separate databases. +With this change, different cluster configurations can be stored in the same table and +Presto will use the new environment column to differentiate them.

  • +
+
+
+

SPI#

+
    +
  • Add query plan to the query completed event.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.189.html b/430/release/release-0.189.html new file mode 100644 index 000000000..5f61cdc95 --- /dev/null +++ b/430/release/release-0.189.html @@ -0,0 +1,2666 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.189 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 0.189#

+
+

General#

+
    +
  • Fix query failure while logging the query plan.

  • +
  • Fix a bug that causes clients to hang when executing LIMIT queries when +optimizer.force-single-node-output is disabled.

  • +
  • Fix a bug in the bing_tile_at() and bing_tile_polygon() functions +where incorrect results were produced for points close to tile edges.

  • +
  • Fix variable resolution when lambda argument has the same name as a table column.

  • +
  • Improve error message when running SHOW TABLES on a catalog that does not exist.

  • +
  • Improve performance for queries with highly selective filters.

  • +
  • Execute USE on the server rather than in the CLI, allowing it +to be supported by any client. This requires clients to add support for +the protocol changes (otherwise the statement will be silently ignored).

  • +
  • Allow casting JSON to ROW even if the JSON does not contain every +field in the ROW.

  • +
  • Add support for dereferencing row fields in lambda expressions.

  • +
+
+
+

Security#

+
    +
  • Support configuring multiple authentication types, which allows supporting +clients that have different authentication requirements or gracefully +migrating between authentication types without needing to update all clients +at once. Specify multiple values for http-server.authentication.type, +separated with commas.

  • +
  • Add support for TLS client certificates as an authentication mechanism by +specifying CERTIFICATE for http-server.authentication.type. +The distinguished name from the validated certificate will be provided as a +javax.security.auth.x500.X500Principal. The certificate authority (CA) +used to sign client certificates will be need to be added to the HTTP server +KeyStore (should technically be a TrustStore but separating them out is not +yet supported).

  • +
  • Skip sending final leg of SPNEGO authentication when using Kerberos.

  • +
+
+
+

JDBC driver#

+
    +
  • Per the JDBC specification, close the ResultSet when Statement is closed.

  • +
  • Add support for TLS client certificate authentication by configuring the +SSLKeyStorePath and SSLKeyStorePassword parameters.

  • +
  • Add support for transactions using SQL statements or the standard JDBC mechanism.

  • +
  • Allow executing the USE statement. Note that this is primarily useful when +running arbitrary SQL on behalf of users. For programmatic use, continuing +to use setCatalog() and setSchema() on Connection is recommended.

  • +
  • Allow executing SET SESSION and RESET SESSION.

  • +
+
+
+

Resource group#

+
    +
  • Add WEIGHTED_FAIR resource group scheduling policy.

  • +
+
+
+

Hive#

+
    +
  • Do not require setting hive.metastore.uri when using the file metastore.

  • +
  • Reduce memory usage when reading string columns from ORC or DWRF files.

  • +
+
+
+

MySQL, PostgreSQL, Redshift, and SQL Server shanges#

+
    +
  • Change mapping for columns with DECIMAL(p,s) data type from Presto DOUBLE +type to the corresponding Presto DECIMAL type.

  • +
+
+
+

Kafka#

+
    +
  • Fix documentation for raw decoder.

  • +
+
+
+

Thrift connector#

+
    +
  • Add support for index joins.

  • +
+
+
+

SPI#

+
    +
  • Deprecate SliceArrayBlock.

  • +
  • Add SessionPropertyConfigurationManager plugin to enable overriding default +session properties dynamically.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.190.html b/430/release/release-0.190.html new file mode 100644 index 000000000..91ab74ade --- /dev/null +++ b/430/release/release-0.190.html @@ -0,0 +1,2658 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.190 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.190#

+
+

General#

+
    +
  • Fix correctness issue for array_min() and array_max() when arrays contain NaN.

  • +
  • Fix planning failure for queries involving GROUPING that require implicit coercions +in expressions containing aggregate functions.

  • +
  • Fix potential workload imbalance when using topology-aware scheduling.

  • +
  • Fix performance regression for queries containing DISTINCT aggregates over the same column.

  • +
  • Fix a memory leak that occurs on workers.

  • +
  • Improve error handling when a HAVING clause contains window functions.

  • +
  • Avoid unnecessary data redistribution when writing when the target table has +the same partition property as the data being written.

  • +
  • Ignore case when sorting the output of SHOW FUNCTIONS.

  • +
  • Improve rendering of the BingTile type.

  • +
  • The approx_distinct() function now supports a standard error +in the range of [0.0040625, 0.26000].

  • +
  • Add support for ORDER BY in aggregation functions.

  • +
  • Add dictionary processing for joins which can improve join performance up to 50%. +This optimization can be disabled using the dictionary-processing-joins-enabled +config property or the dictionary_processing_join session property.

  • +
  • Add support for casting to INTERVAL types.

  • +
  • Add ST_Buffer() geospatial function.

  • +
  • Allow treating decimal literals as values of the DECIMAL type rather than DOUBLE. +This behavior can be enabled by setting the parse-decimal-literals-as-double +config property or the parse_decimal_literals_as_double session property to false.

  • +
  • Add JMX counter to track the number of submitted queries.

  • +
+
+
+

Resource groups#

+
    +
  • Add priority column to the DB resource group selectors.

  • +
  • Add exact match source selector to the DB resource group selectors.

  • +
+
+
+

CLI#

+
    +
  • Add support for setting client tags.

  • +
+
+
+

JDBC driver#

+
    +
  • Add getPeakMemoryBytes() to QueryStats.

  • +
+
+
+

Accumulo#

+
    +
  • Improve table scan parallelism.

  • +
+
+
+

Hive#

+
    +
  • Fix query failures for the file-based metastore implementation when partition +column values contain a colon.

  • +
  • Improve performance for writing to bucketed tables when the data being written +is already partitioned appropriately (e.g., the output is from a bucketed join).

  • +
  • Add config property hive.max-outstanding-splits-size for the maximum +amount of memory used to buffer splits for a single table scan. Additionally, +the default value is substantially higher than the previous hard-coded limit, +which can prevent certain queries from failing.

  • +
+
+
+

Thrift connector#

+
    +
  • Make Thrift retry configurable.

  • +
  • Add JMX counters for Thrift requests.

  • +
+
+
+

SPI#

+
    +
  • Remove the RecordSink interface, which was difficult to use +correctly and had no advantages over the PageSink interface.

  • +
+
+

Note

+

This is a backwards incompatible change with the previous connector SPI. +If you have written a connector that uses the RecordSink interface, +you will need to update your code before deploying this release.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.191.html b/430/release/release-0.191.html new file mode 100644 index 000000000..4fa8a90c1 --- /dev/null +++ b/430/release/release-0.191.html @@ -0,0 +1,2630 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.191 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+ +
+ +
+
+ +
+

Release 0.191#

+
+

General#

+
    +
  • Fix regression that could cause high CPU usage for join queries when dictionary +processing for joins is enabled.

  • +
  • Fix bit_count() for bits between 33 and 63.

  • +
  • The query.low-memory-killer.enabled config property has been replaced +with query.low-memory-killer.policy. Use total-reservation to continue +using the previous policy of killing the largest query. There is also a new +policy, total-reservation-on-blocked-nodes, which kills the query that +is using the most memory on nodes that are out of memory (blocked).

  • +
  • Add support for grouped join execution. When both sides of a join have the +same table partitioning and the partitioning is addressable, partial data +can be loaded into memory at a time, making it possible to execute the join +with less peak memory usage. The colocated join feature must be enabled with +the colocated-joins-enabled config property or the colocated_join +session property, and the concurrent_lifespans_per_task session property +must be specified.

  • +
  • Allow connectors to report the amount of physical written data.

  • +
  • Add ability to dynamically scale out the number of writer tasks rather +than allocating a fixed number of tasks. Additional tasks are added when the +the average amount of physical data per writer is above a minimum threshold. +Writer scaling can be enabled with the scale-writers config property or +the scale_writers session property. The minimum size can be set with the +writer-min-size config property or the writer_min_size session property. +The tradeoff for writer scaling is that write queries can take longer to run +due to the decreased writer parallelism while the writer count ramps up.

  • +
+
+
+

Resource groups#

+
    +
  • Add query type to the exact match source selector in the DB resource group selectors.

  • +
+
+
+

CLI#

+
    +
  • Improve display of values of the Geometry type.

  • +
+
+
+

Hive#

+
    +
  • Add support for grouped join execution for Hive tables when both +sides of a join have the same bucketing property.

  • +
  • Report physical written data for the legacy RCFile writer, optimized RCFile +writer, and optimized ORC writer. These writers thus support writer scaling, +which can both reduce the number of written files and create larger files. +This is especially important for tables that have many small partitions, as +small files can take a disproportionately longer time to read.

  • +
+
+
+

Thrift connector#

+
    +
  • Add page size distribution metrics.

  • +
+
+
+

MySQL, PostgreSQL, Redshift, and SQL Server#

+
    +
  • Fix querying information_schema.columns if there are tables with +no columns or no supported columns.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.192.html b/430/release/release-0.192.html new file mode 100644 index 000000000..a835c7e48 --- /dev/null +++ b/430/release/release-0.192.html @@ -0,0 +1,2671 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.192 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.192#

+
+

General#

+
    +
  • Fix performance regression in split scheduling introduced in 0.191. If a query +scans a non-trivial number of splits (~1M splits in an hour), the coordinator +CPU utilization can be very high, leading to elevated communication failures.

  • +
  • Fix correctness issue in the geometry_to_bing_tiles() function that causes +it to return irrelevant tiles when bottom or right side of the bounding box of the +geometry is aligned with the tile border.

  • +
  • Fix handling of invalid WKT (well-known text) input in geospatial functions.

  • +
  • Fix an issue that can cause long-running queries to hang when writer scaling is enabled.

  • +
  • Fix cast from REAL or DOUBLE to DECIMAL to conform to the SQL standard. +For example, previously cast (double '100000000000000000000000000000000' as decimal(38)) +would return 100000000000000005366162204393472. Now it returns 100000000000000000000000000000000.

  • +
  • Fix bug in validation of resource groups that prevented use of the WEIGHTED_FAIR policy.

  • +
  • Fail queries properly when the coordinator fails to fetch data from workers. +Previously, it would return an HTTP 500 error to the client.

  • +
  • Improve memory tracking for queries involving DISTINCT or row_number() that could cause +over-committing memory resources for short time periods.

  • +
  • Improve performance for queries involving grouping().

  • +
  • Improve buffer utilization calculation for writer scaling.

  • +
  • Remove tracking of per-driver peak memory reservation.

  • +
  • Add resource-groups.max-refresh-interval config option to limit the maximum acceptable +staleness of resource group configuration.

  • +
  • Remove dictionary-processing-joins-enabled configuration option and dictionary_processing_join +session property.

  • +
+
+
+

Web UI#

+
    +
  • Fix incorrect reporting of input size and positions in live plan view.

  • +
+
+
+

CLI#

+
    +
  • Fix update of prompt after USE statement.

  • +
  • Fix correctness issue when rendering arrays of Bing tiles that causes +the first entry to be repeated multiple times.

  • +
+
+
+

Hive#

+
    +
  • Fix reading partitioned table statistics from newer Hive metastores.

  • +
  • Do not treat file system errors as corruptions for ORC.

  • +
  • Prevent reads from tables or partitions with object_not_readable attribute set.

  • +
  • Add support for validating ORC files after they have been written. This behavior can +be turned on via the hive.orc.writer.validate configuration property.

  • +
  • Expose ORC writer statistics via JMX.

  • +
  • Add configuration options to control ORC writer min/max rows per stripe and row group, +maximum stripe size, and memory limit for dictionaries.

  • +
  • Allow reading empty ORC files.

  • +
  • Handle ViewFs when checking file system cache expiration.

  • +
  • Improve error reporting when the target table of an insert query is dropped.

  • +
  • Remove retry when creating Hive record reader. This can help queries fail faster.

  • +
+
+
+

MySQL#

+
    +
  • Remove support for TIME WITH TIME ZONE and TIMESTAMP WITH TIME ZONE +types due to MySQL types not being able to store timezone information.

  • +
  • Add support for REAL type, which maps to MySQL’s FLOAT type.

  • +
+
+
+

PostgreSQL#

+
    +
  • Add support for VARBINARY type, which maps to PostgreSQL’s BYTEA type.

  • +
+
+
+

MongoDB#

+
    +
  • Fix support for pushing down inequality operators for string types.

  • +
  • Add support for reading documents as MAP values.

  • +
  • Add support for MongoDB’s Decimal128 type.

  • +
  • Treat document and array of documents as JSON instead of VARCHAR.

  • +
+
+
+

JMX#

+
    +
  • Allow nulls in history table values.

  • +
+
+
+

SPI#

+
    +
  • Remove SliceArrayBlock class.

  • +
  • Add offset and length parameters to Block.getPositions().

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.193.html b/430/release/release-0.193.html new file mode 100644 index 000000000..ce9108e1a --- /dev/null +++ b/430/release/release-0.193.html @@ -0,0 +1,2647 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.193 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.193#

+
+

General#

+
    +
  • Fix an infinite loop during planning for queries containing non-trivial predicates.

  • +
  • Fix row_number() optimization that causes query failure or incorrect results +for queries that constrain the result of row_number() to be less than one.

  • +
  • Fix failure during query planning when lambda expressions are used in UNNEST or VALUES clauses.

  • +
  • Fix Tried to free more revocable memory than is reserved error for queries that have spilling enabled +and run in the reserved memory pool.

  • +
  • Improve the performance of the ST_Contains() function.

  • +
  • Add map_zip_with() lambda function.

  • +
  • Add normal_cdf() function.

  • +
  • Add SET_DIGEST type and related functions.

  • +
  • Add query stat that tracks peak total memory.

  • +
  • Improve performance of queries that filter all data from a table up-front (e.g., due to partition pruning).

  • +
  • Turn on new local scheduling algorithm by default (see Release 0.181).

  • +
  • Remove the information_schema.__internal_partitions__ table.

  • +
+
+
+

Security#

+
    +
  • Apply the authentication methods in the order they are listed in the +http-server.authentication.type configuration.

  • +
+
+
+

CLI#

+
    +
  • Fix rendering of maps of Bing tiles.

  • +
  • Abort the query when the result pager exits.

  • +
+
+
+

JDBC driver#

+
    +
  • Use SSL by default for port 443.

  • +
+
+
+

Hive#

+
    +
  • Allow dropping any column in a table. Previously, dropping columns other +than the last one would fail with ConcurrentModificationException.

  • +
  • Correctly write files for text format tables that use non-default delimiters. +Previously, they were written with the default delimiter.

  • +
  • Fix reading data from S3 if the data is in a region other than us-east-1. +Previously, such queries would fail with +"The authorization header is malformed; the region 'us-east-1' is wrong; expecting '<region_name>'", +where <region_name> is the S3 region hosting the bucket that is queried.

  • +
  • Enable SHOW PARTITIONS FROM <table> WHERE <condition> to work for tables +that have more than hive.max-partitions-per-scan partitions as long as +the specified <condition> reduces the number of partitions to below this limit.

  • +
+
+
+

Blackhole#

+
    +
  • Do not allow creating tables in a nonexistent schema.

  • +
  • Add support for CREATE SCHEMA.

  • +
+
+
+

Memory connector#

+
    +
  • Allow renaming tables across schemas. Previously, the target schema was ignored.

  • +
  • Do not allow creating tables in a nonexistent schema.

  • +
+
+
+

MongoDB#

+
    +
  • Add INSERT support. It was previously removed in 0.155.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.194.html b/430/release/release-0.194.html new file mode 100644 index 000000000..d562f3352 --- /dev/null +++ b/430/release/release-0.194.html @@ -0,0 +1,2622 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.194 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.194#

+
+

General#

+
    +
  • Fix planning performance regression that can affect queries over Hive tables +with many partitions.

  • +
  • Fix deadlock in memory management logic introduced in the previous release.

  • +
  • Add word_stem() function.

  • +
  • Restrict n (number of result elements) to 10,000 or less for +min(col, n), max(col, n), min_by(col1, col2, n), and max_by(col1, col2, n).

  • +
  • Improve error message when a session property references an invalid catalog.

  • +
  • Reduce memory usage of histogram() aggregation function.

  • +
  • Improve coordinator CPU efficiency when discovering splits.

  • +
  • Include minimum and maximum values for columns in SHOW STATS.

  • +
+
+
+

Web UI#

+
    +
  • Fix previously empty peak memory display in the query details page.

  • +
+
+
+

CLI#

+
    +
  • Fix regression in CLI that makes it always print “query aborted by user” when +the result is displayed with a pager, even if the query completes successfully.

  • +
  • Return a non-zero exit status when an error occurs.

  • +
  • Add --client-info option for specifying client info.

  • +
  • Add --ignore-errors option to continue processing in batch mode when an error occurs.

  • +
+
+
+

JDBC driver#

+
    +
  • Allow configuring connection network timeout with setNetworkTimeout().

  • +
  • Allow setting client tags via the ClientTags client info property.

  • +
  • Expose update type via getUpdateType() on PrestoStatement.

  • +
+
+
+

Hive#

+
    +
  • Consistently fail queries that attempt to read partitions that are offline. +Previously, the query can have one of the following outcomes: fail as expected, +skip those partitions and finish successfully, or hang indefinitely.

  • +
  • Allow setting username used to access Hive metastore via the hive.metastore.username config property.

  • +
  • Add hive_storage_format and respect_table_format session properties, corresponding to +the hive.storage-format and hive.respect-table-format config properties.

  • +
  • Reduce ORC file reader memory consumption by allocating buffers lazily. +Buffers are only allocated for columns that are actually accessed.

  • +
+
+
+

Cassandra#

+
    +
  • Fix failure when querying information_schema.columns when there is no equality predicate on table_name.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.195.html b/430/release/release-0.195.html new file mode 100644 index 000000000..2ed77fc88 --- /dev/null +++ b/430/release/release-0.195.html @@ -0,0 +1,2619 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.195 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.195#

+
+

General#

+
    +
  • Fix histogram() for map type when type coercion is required.

  • +
  • Fix nullif for map type when type coercion is required.

  • +
  • Fix incorrect termination of queries when the coordinator to worker communication is under high load.

  • +
  • Fix race condition that causes queries with a right or full outer join to fail.

  • +
  • Change reference counting for varchar, varbinary, and complex types to be approximate. This +approximation reduces GC activity when computing large aggregations with these types.

  • +
  • Change communication system to be more resilient to issues such as long GC pauses or networking errors. +The min/max sliding scale of for timeouts has been removed and instead only max time is used. +The exchange.min-error-duration and query.remote-task.min-error-duration are now ignored and will be +removed in a future release.

  • +
  • Increase coordinator timeout for cleanup of worker tasks for failed queries. This improves the health of +the system when workers are offline for long periods due to GC or network errors.

  • +
  • Remove the compiler.interpreter-enabled config property.

  • +
+
+
+

Security#

+
    +
  • Presto now supports generic password authentication using a pluggable Password authenticator. +Enable password authentication by setting http-server.authentication.type to include PASSWORD as an +authentication type.

  • +
  • LDAP authentication is now implemented as a password authentication +plugin. You will need to update your configuration if you are using it.

  • +
+
+
+

CLI and JDBC#

+
    +
  • Provide a better error message when TLS client certificates are expired or not yet valid.

  • +
+
+
+

MySQL#

+
    +
  • Fix an error that can occur while listing tables if one of the listed tables is dropped.

  • +
+
+
+

Hive#

+
    +
  • Add support for LZ4 compressed ORC files.

  • +
  • Add support for reading Zstandard compressed ORC files.

  • +
  • Validate ORC compression block size when reading ORC files.

  • +
  • Set timeout of Thrift metastore client. This was accidentally removed in 0.191.

  • +
+
+
+

MySQL, Redis, Kafka, and MongoDB#

+
    +
  • Fix failure when querying information_schema.columns when there is no equality predicate on table_name.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.196.html b/430/release/release-0.196.html new file mode 100644 index 000000000..5d92cdc2c --- /dev/null +++ b/430/release/release-0.196.html @@ -0,0 +1,2604 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.196 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.196#

+
+

General#

+
    +
  • Fix behavior of JOIN ... USING to conform to standard SQL semantics. +The old behavior can be restored by setting the deprecated.legacy-join-using +configuration option or the legacy_join_using session property.

  • +
  • Fix memory leak for queries with ORDER BY.

  • +
  • Fix tracking of query peak memory usage.

  • +
  • Fix skew in dynamic writer scaling by eagerly freeing memory in the source output +buffers. This can be disabled by setting exchange.acknowledge-pages=false.

  • +
  • Fix planning failure for lambda with capture in rare cases.

  • +
  • Fix decimal precision of round(x, d) when x is a DECIMAL.

  • +
  • Fix returned value from round(x, d) when x is a DECIMAL with +scale 0 and d is a negative integer. Previously, no rounding was done +in this case.

  • +
  • Improve performance of the array_join() function.

  • +
  • Improve performance of the ST_Envelope() function.

  • +
  • Optimize min_by() and max_by() by avoiding unnecessary object +creation in order to reduce GC overhead.

  • +
  • Show join partitioning explicitly in EXPLAIN.

  • +
  • Add is_json_scalar() function.

  • +
  • Add regexp_replace() function variant that executes a lambda for +each replacement.

  • +
+
+
+

Security#

+
    +
  • Add rules to the file System access control +to enforce a specific matching between authentication credentials and a +executing username.

  • +
+
+
+

Hive#

+
    +
  • Fix a correctness issue where non-null values can be treated as null values +when writing dictionary-encoded strings to ORC files with the new ORC writer.

  • +
  • Fix invalid failure due to string statistics mismatch while validating ORC files +after they have been written with the new ORC writer. This happens when +the written strings contain invalid UTF-8 code points.

  • +
  • Add support for reading array, map, or row type columns from partitions +where the partition schema is different from the table schema. This can +occur when the table schema was updated after the partition was created. +The changed column types must be compatible. For rows types, trailing fields +may be added or dropped, but the corresponding fields (by ordinal) +must have the same name.

  • +
  • Add hive.non-managed-table-creates-enabled configuration option +that controls whether or not users may create non-managed (external) tables. +The default value is true.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.197.html b/430/release/release-0.197.html new file mode 100644 index 000000000..7ca113707 --- /dev/null +++ b/430/release/release-0.197.html @@ -0,0 +1,2627 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.197 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.197#

+
+

General#

+
    +
  • Fix query scheduling hang when the concurrent_lifespans_per_task session property is set.

  • +
  • Fix failure when a query contains a TIMESTAMP literal corresponding to a local time that +does not occur in the default time zone of the Presto JVM. For example, if Presto was running +in a CET zone (e.g., Europe/Brussels) and the client session was in UTC, an expression +such as TIMESTAMP '2017-03-26 02:10:00' would cause a failure.

  • +
  • Extend predicate inference and pushdown for queries using a <symbol> IN <subquery> predicate.

  • +
  • Support predicate pushdown for the <column> IN <values list> predicate +where values in the values list require casting to match the type of column.

  • +
  • Optimize min() and max() to avoid unnecessary object creation in order to reduce GC overhead.

  • +
  • Optimize the performance of ST_XMin(), ST_XMax(), ST_YMin(), and ST_YMax().

  • +
  • Add DATE variant for sequence() function.

  • +
  • Add ST_IsSimple() geospatial function.

  • +
  • Add support for broadcast spatial joins.

  • +
+
+
+

Resource groups#

+
    +
  • Change configuration check for weights in resource group policy to validate that +either all of the sub-groups or none of the sub-groups have a scheduling weight configured.

  • +
  • Add support for named variables in source and user regular expressions that can be +used to parameterize resource group names.

  • +
  • Add support for optional fields in DB resource group exact match selectors.

  • +
+
+
+

Hive#

+
    +
  • Fix reading of Hive partition statistics with unset fields. Previously, unset fields +were incorrectly interpreted as having a value of zero.

  • +
  • Fix integer overflow when writing a single file greater than 2GB with optimized ORC writer.

  • +
  • Fix system memory accounting to include stripe statistics size and +writer validation size for the optimized ORC writer.

  • +
  • Dynamically allocate the compression buffer for the optimized ORC writer +to avoid unnecessary memory allocation. Add config property +hive.orc.writer.max-compression-buffer-size to limit the maximum size of the buffer.

  • +
  • Add session property orc_optimized_writer_max_stripe_size to tune the +maximum stipe size for the optimized ORC writer.

  • +
  • Add session property orc_string_statistics_limit to drop the string +statistics when writing ORC files if they exceed the limit.

  • +
  • Use the view owner returned from the metastore at the time of the query rather than +always using the user who created the view. This allows changing the owner of a view.

  • +
+
+
+

CLI#

+
    +
  • Fix hang when CLI fails to communicate with Presto server.

  • +
+
+
+

SPI#

+
    +
  • Include connector session properties for the connector metadata calls made +when running SHOW statements or querying information_schema.

  • +
  • Add count and time of full GC that occurred while query was running to QueryCompletedEvent.

  • +
  • Change the ResourceGroupManager interface to include a match() method and +remove the getSelectors() method and the ResourceGroupSelector interface.

  • +
  • Rename the existing SelectionContext class to be SelectionCriteria and +create a new SelectionContext class that is returned from the match() method +and contains the resource group ID and a manager-defined context field.

  • +
  • Use the view owner from ConnectorViewDefinition when present.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.198.html b/430/release/release-0.198.html new file mode 100644 index 000000000..36d1a4618 --- /dev/null +++ b/430/release/release-0.198.html @@ -0,0 +1,2638 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.198 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.198#

+
+

General#

+
    +
  • Perform semantic analysis before enqueuing queries.

  • +
  • Add support for selective aggregates (FILTER) with DISTINCT argument qualifiers.

  • +
  • Support ESCAPE for LIKE predicate in SHOW SCHEMAS and SHOW TABLES queries.

  • +
  • Parse decimal literals (e.g. 42.0) as DECIMAL by default. Previously, they were parsed as +DOUBLE. This behavior can be turned off via the parse-decimal-literals-as-double config option or +the parse_decimal_literals_as_double session property.

  • +
  • Fix current_date failure when the session time zone has a “gap” at 1970-01-01 00:00:00. +The time zone America/Bahia_Banderas is one such example.

  • +
  • Add variant of sequence() function for DATE with an implicit one-day step increment.

  • +
  • Increase the maximum number of arguments for the zip() function from 4 to 5.

  • +
  • Add ST_IsValid(), geometry_invalid_reason(), simplify_geometry(), and +great_circle_distance() functions.

  • +
  • Support min() and max() aggregation functions when the input type is unknown at query analysis time. +In particular, this allows using the functions with NULL literals.

  • +
  • Add configuration property task.max-local-exchange-buffer-size for setting local exchange buffer size.

  • +
  • Add trace token support to the scheduler and exchange HTTP clients. Each HTTP request sent +by the scheduler and exchange HTTP clients will have a “trace token” (a unique ID) in their +headers, which will be logged in the HTTP request logs. This information can be used to +correlate the requests and responses during debugging.

  • +
  • Improve query performance when dynamic writer scaling is enabled.

  • +
  • Improve performance of ST_Intersects().

  • +
  • Improve query latency when tables are known to be empty during query planning.

  • +
  • Optimize array_agg() to avoid excessive object overhead and native memory usage with G1 GC.

  • +
  • Improve performance for high-cardinality aggregations with DISTINCT argument qualifiers. This +is an experimental optimization that can be activated by disabling the use_mark_distinct session +property or the optimizer.use-mark-distinct config option.

  • +
  • Improve parallelism of queries that have an empty grouping set.

  • +
  • Improve performance of join queries involving the ST_Distance() function.

  • +
+
+
+

Resource groups#

+
    +
  • Query Queues have been removed. Resource Groups are always enabled. The +config property experimental.resource-groups-enabled has been removed.

  • +
  • Change WEIGHTED_FAIR scheduling policy to select oldest eligible sub group +of groups where utilization and share are identical.

  • +
+
+
+

CLI#

+
    +
  • The --enable-authentication option has been removed. Kerberos authentication +is automatically enabled when --krb5-remote-service-name is specified.

  • +
  • Kerberos authentication now requires HTTPS.

  • +
+
+
+

Hive#

+
    +
  • Add support for using AWS Glue as the metastore. +Enable it by setting the hive.metastore config property to glue.

  • +
  • Fix a bug in the ORC writer that will write incorrect data of type VARCHAR or VARBINARY +into files.

  • +
+
+
+

JMX#

+
    +
  • Add wildcard character * which allows querying several MBeans with a single query.

  • +
+
+
+

SPI#

+
    +
  • Add performance statistics to query plan in QueryCompletedEvent.

  • +
  • Remove Page.getBlocks(). This call was rarely used and performed an expensive copy. +Instead, use Page.getBlock(channel) or the new helper Page.appendColumn().

  • +
  • Improve validation of ArrayBlock, MapBlock, and RowBlock during construction.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.199.html b/430/release/release-0.199.html new file mode 100644 index 000000000..1ff8f3c0a --- /dev/null +++ b/430/release/release-0.199.html @@ -0,0 +1,2659 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.199 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.199#

+
+

General#

+
    +
  • Allow users to create views for their own use when they do not have permission +to grant others access to the underlying tables or views. To enable this, +creation permission is now only checked at query time, not at creation time, +and the query time check is skipped if the user is the owner of the view.

  • +
  • Add support for spatial left join.

  • +
  • Add hmac_md5(), hmac_sha1(), hmac_sha256(), and hmac_sha512() functions.

  • +
  • Add array_sort() function that takes a lambda as a comparator.

  • +
  • Add line_locate_point() geospatial function.

  • +
  • Add support for ORDER BY clause in aggregations for queries that use grouping sets.

  • +
  • Add support for yielding when unspilling an aggregation.

  • +
  • Expand grouped execution support to GROUP BY and UNION ALL, making it possible +to execute aggregations with less peak memory usage.

  • +
  • Change the signature of round(x, d) and truncate(x, d) functions so that +d is of type INTEGER. Previously, d could be of type BIGINT. +This behavior can be restored with the deprecated.legacy-round-n-bigint config option +or the legacy_round_n_bigint session property.

  • +
  • Accessing anonymous row fields via .field0, .field1, etc., is no longer allowed. +This behavior can be restored with the deprecated.legacy-row-field-ordinal-access +config option or the legacy_row_field_ordinal_access session property.

  • +
  • Optimize the ST_Intersection() function for rectangles aligned with coordinate axes +(e.g., polygons produced by the ST_Envelope() and bing_tile_polygon() functions).

  • +
  • Finish joins early when possible if one side has no rows. This happens for +either side of an inner join, for the left side of a left join, and for the +right side of a right join.

  • +
  • Improve predicate evaluation performance during predicate pushdown in planning.

  • +
  • Improve the performance of queries that use LIKE predicates on the columns of information_schema tables.

  • +
  • Improve the performance of map-to-map cast.

  • +
  • Improve the performance of ST_Touches(), ST_Within(), ST_Overlaps(), ST_Disjoint(), +and ST_Crosses() functions.

  • +
  • Improve the serialization performance of geometry values.

  • +
  • Improve the performance of functions that return maps.

  • +
  • Improve the performance of joins and aggregations that include map columns.

  • +
+
+
+

Server RPM#

+
    +
  • Add support for installing on machines with OpenJDK.

  • +
+
+
+

Security#

+
    +
  • Add support for authentication with JWT access token.

  • +
+
+
+

JDBC driver#

+
    +
  • Make driver compatible with Java 9+. It previously failed with IncompatibleClassChangeError.

  • +
+
+
+

Hive#

+
    +
  • Fix ORC writer failure when writing NULL values into columns of type ROW, MAP, or ARRAY.

  • +
  • Fix ORC writers incorrectly writing non-null values as NULL for all types.

  • +
  • Support reading Hive partitions that have a different bucket count than the table, +as long as the ratio is a power of two (1:2^n or 2^n:1).

  • +
  • Add support for the skip.header.line.count table property.

  • +
  • Prevent reading from tables with the skip.footer.line.count table property.

  • +
  • Partitioned tables now have a hidden system table that contains the partition values. +A table named example will have a partitions table named example$partitions. +This provides the same functionality and data as SHOW PARTITIONS.

  • +
  • Partition name listings, both via the $partitions table and using +SHOW PARTITIONS, are no longer subject to the limit defined by the +hive.max-partitions-per-scan config option.

  • +
  • Allow marking partitions as offline via the presto_offline partition property.

  • +
+
+
+

Thrift connector#

+
    +
  • Most of the config property names are different due to replacing the +underlying Thrift client implementation. Please see Thrift connector +for details on the new properties.

  • +
+
+
+

SPI#

+
    +
  • Allow connectors to provide system tables dynamically.

  • +
  • Add resourceGroupId and queryType fields to SessionConfigurationContext.

  • +
  • Simplify the constructor of RowBlock.

  • +
  • Block.writePositionTo() now closes the current entry.

  • +
  • Replace the writeObject() method in BlockBuilder with appendStructure().

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.200.html b/430/release/release-0.200.html new file mode 100644 index 000000000..446b054a0 --- /dev/null +++ b/430/release/release-0.200.html @@ -0,0 +1,2588 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.200 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.200#

+
+

General#

+
    +
  • Disable early termination of inner or right joins when the right side +has zero rows. This optimization can cause indefinite query hangs +for queries that join against a small number of rows. +This regression was introduced in 0.199.

  • +
  • Fix query execution failure for bing_tile_coordinates().

  • +
  • Remove the log() function. The arguments to the function were in the +wrong order according to the SQL standard, resulting in incorrect results +when queries were translated to or from other SQL implementations. The +equivalent to log(x, b) is ln(x) / ln(b). The function can be +restored with the deprecated.legacy-log-function config option.

  • +
  • Allow including a comment when adding a column to a table with ALTER TABLE.

  • +
  • Add from_ieee754_32() and from_ieee754_64() functions.

  • +
  • Add ST_GeometryType() geospatial function.

  • +
+
+
+

Hive#

+
    +
  • Fix reading min/max statistics for columns of REAL type in partitioned tables.

  • +
  • Fix failure when reading Parquet files with optimized Parquet reader +related with the predicate push down for structural types. +Predicates on structural types are now ignored for Parquet files.

  • +
  • Fix failure when reading ORC files that contain UTF-8 Bloom filter streams. +Such Bloom filters are now ignored.

  • +
+
+
+

MySQL#

+
    +
  • Avoid reading extra rows from MySQL at query completion. +This typically affects queries with a LIMIT clause.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.201.html b/430/release/release-0.201.html new file mode 100644 index 000000000..e14faa9f7 --- /dev/null +++ b/430/release/release-0.201.html @@ -0,0 +1,2604 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.201 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.201#

+
+

General#

+
    +
  • Change grouped aggregations to use IS NOT DISTINCT FROM semantics rather than equality +semantics. This fixes incorrect results and degraded performance when grouping on NaN +floating point values, and adds support for grouping on structural types that contain nulls.

  • +
  • Fix planning error when column names are reused in ORDER BY query.

  • +
  • System memory pool is now unused by default and it will eventually be removed completely. +All memory allocations will now be served from the general/user memory pool. The old behavior +can be restored with the deprecated.legacy-system-pool-enabled config option.

  • +
  • Improve performance and memory usage for queries using row_number() followed by a +filter on the row numbers generated.

  • +
  • Improve performance and memory usage for queries using ORDER BY followed by a LIMIT.

  • +
  • Improve performance of queries that process structural types and contain joins, aggregations, +or table writes.

  • +
  • Add session property prefer-partial-aggregation to allow users to disable partial +aggregations for queries that do not benefit.

  • +
  • Add support for current_user (see Session information).

  • +
+
+
+

Security#

+
    +
  • Change rules in the System access control for enforcing matches +between authentication credentials and a chosen username to allow more fine-grained +control and ability to define superuser-like credentials.

  • +
+
+
+

Hive#

+
    +
  • Replace ORC writer stripe minimum row configuration hive.orc.writer.stripe-min-rows +with stripe minimum data size hive.orc.writer.stripe-min-size.

  • +
  • Change ORC writer validation configuration hive.orc.writer.validate to switch to a +sampling percentage hive.orc.writer.validation-percentage.

  • +
  • Fix optimized ORC writer writing incorrect data of type map or array.

  • +
  • Fix SHOW PARTITIONS and the $partitions table for tables that have null partition +values.

  • +
  • Fix impersonation for the simple HDFS authentication to use login user rather than current +user.

  • +
+
+
+

SPI#

+
    +
  • Support resource group selection based on resource estimates.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.202.html b/430/release/release-0.202.html new file mode 100644 index 000000000..9497cea56 --- /dev/null +++ b/430/release/release-0.202.html @@ -0,0 +1,2625 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.202 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.202#

+
+

General#

+
    +
  • Fix correctness issue for queries involving aggregations over the result of an outer join (#10592).

  • +
  • Fix map() to raise an error on duplicate keys rather than silently producing a corrupted map.

  • +
  • Fix map_from_entries() to raise an error when input array contains a null entry.

  • +
  • Fix out-of-memory error for bucketed execution by scheduling new splits on the same worker as +the recently finished one.

  • +
  • Fix query failure when performing a GROUP BY on json or ipaddress types.

  • +
  • Fix correctness issue in line_locate_point(), ST_IsValid(), and geometry_invalid_reason() +functions to not return values outside of the expected range.

  • +
  • Fix failure in geometry_to_bing_tiles() and ST_NumPoints() functions when +processing geometry collections.

  • +
  • Fix query failure in aggregation spilling (#10587).

  • +
  • Remove support for SHOW PARTITIONS statement.

  • +
  • Improve support for correlated subqueries containing equality predicates.

  • +
  • Improve performance of correlated EXISTS subqueries.

  • +
  • Limit the number of grouping sets in a GROUP BY clause. +The default limit is 2048 and can be set via the analyzer.max-grouping-sets +configuration property or the max_grouping_sets session property.

  • +
  • Allow coercion between row types regardless of field names. +Previously, a row type is coercible to another only if the field name in the source type +matches the target type, or when target type has anonymous field name.

  • +
  • Increase default value for experimental.filter-and-project-min-output-page-size to 500kB.

  • +
  • Improve performance of equals operator on array(bigint) and array(double) types.

  • +
  • Respect X-Forwarded-Proto header in client protocol responses.

  • +
  • Add support for column-level access control. +Connectors have not yet been updated to take advantage of this support.

  • +
  • Add support for correlated subqueries with correlated OR predicates.

  • +
  • Add multimap_from_entries() function.

  • +
  • Add bing_tiles_around(), ST_NumGeometries(), ST_GeometryN(), and ST_ConvexHull() geospatial functions.

  • +
  • Add wilson_interval_lower() and wilson_interval_upper() functions.

  • +
  • Add IS DISTINCT FROM for json and ipaddress type.

  • +
+
+
+

Hive#

+
    +
  • Fix optimized ORC writer encoding of TIMESTAMP before 1970-01-01. Previously, the +written value was off by one second.

  • +
  • Fix query failure when a Hive bucket has no splits. This commonly happens when a +predicate filters some buckets out entirely.

  • +
  • Remove the hive.bucket-writing config property.

  • +
  • Add support for creating and writing bucketed sorted tables. The list of +sorting columns may be specified using the sorted_by table property. +Writing to sorted tables can be disabled using the hive.sorted-writing +config property or the sorted_writing_enabled session property. The +maximum number of temporary files for can be controlled using the +hive.max-sort-files-per-bucket property.

  • +
  • Collect and store basic table statistics (rowCount, fileCount, rawDataSize, +totalSize) when writing.

  • +
  • Add hive.orc.tiny-stripe-threshold config property and orc_tiny_stripe_threshold +session property to control the stripe/file size threshold when ORC reader decides to +read multiple consecutive stripes or entire fires at once. Previously, this feature +piggybacks on other properties.

  • +
+
+
+

CLI#

+
    +
  • Add peak memory usage to --debug output.

  • +
+
+
+

SPI#

+
    +
  • Make PageSorter and PageIndexer supported interfaces.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.203.html b/430/release/release-0.203.html new file mode 100644 index 000000000..d90e93d81 --- /dev/null +++ b/430/release/release-0.203.html @@ -0,0 +1,2624 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.203 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.203#

+
+

General#

+
    +
  • Fix spurious duplicate key errors from map().

  • +
  • Fix planning failure when a correlated subquery containing a LIMIT +clause is used within EXISTS (#10696).

  • +
  • Fix out of memory error caused by missing pushback checks in data exchanges.

  • +
  • Fix execution failure for queries containing a cross join when using bucketed execution.

  • +
  • Fix execution failure for queries containing an aggregation function +with DISTINCT and a highly selective aggregation filter. +For example: sum(DISTINCT x) FILTER (WHERE y = 0)

  • +
  • Fix quoting in error message for SHOW PARTITIONS.

  • +
  • Eliminate redundant calls to check column access permissions.

  • +
  • Improve query creation reliability by delaying query start until the client +acknowledges the query ID by fetching the first response link. This eliminates +timeouts during the initial request for queries that take a long time to analyze.

  • +
  • Remove support for legacy ORDER BY semantics.

  • +
  • Distinguish between inner and left spatial joins in explain plans.

  • +
+
+
+

Security#

+
    +
  • Fix sending authentication challenge when at least two of the +KERBEROS, PASSWORD, or JWT authentication types are configured.

  • +
  • Allow using PEM encoded (PKCS #8) keystore and truststore with the HTTP server +and the HTTP client used for internal communication. This was already supported +for the CLI and JDBC driver.

  • +
+
+
+

Server RPM#

+
    +
  • Declare a dependency on uuidgen. The uuidgen program is required during +installation of the Presto server RPM package and lack of it resulted in an invalid +config file being generated during installation.

  • +
+
+
+

Hive connector#

+
    +
  • Fix complex type handling in the optimized Parquet reader. Previously, null values, +optional fields, and Parquet backward compatibility rules were not handled correctly.

  • +
  • Fix an issue that could cause the optimized ORC writer to fail with a LazyBlock error.

  • +
  • Improve error message for max open writers.

  • +
+
+
+

Thrift connector#

+
    +
  • Fix retry of requests when the remote Thrift server indicates that the +error is retryable.

  • +
+
+
+

Local file connector#

+
    +
  • Fix parsing of timestamps when the JVM time zone is UTC (#9601).

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.204.html b/430/release/release-0.204.html new file mode 100644 index 000000000..d86b4605b --- /dev/null +++ b/430/release/release-0.204.html @@ -0,0 +1,2620 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.204 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.204#

+
+

General#

+
    +
  • Use distributed join if one side is naturally partitioned on join keys.

  • +
  • Improve performance of correlated subqueries when filters from outer query +can be propagated to the subquery.

  • +
  • Improve performance for correlated subqueries that contain inequalities.

  • +
  • Add support for all geometry types in ST_Area().

  • +
  • Add ST_EnvelopeAsPts() function.

  • +
  • Add to_big_endian_32() and from_big_endian_32() functions.

  • +
  • Add cast between VARBINARY type and IPADDRESS type.

  • +
  • Make lpad() and rpad() functions support VARBINARY in addition to VARCHAR.

  • +
  • Allow using arrays of mismatched lengths with zip_with(). +The missing positions are filled with NULL.

  • +
  • Track execution statistics of AddExchanges and PredicatePushdown optimizer rules.

  • +
+
+
+

Event listener#

+
    +
  • Add resource estimates to query events.

  • +
+
+
+

Web UI#

+
    +
  • Fix kill query button.

  • +
  • Display resource estimates in Web UI query details page.

  • +
+
+
+

Resource group#

+
    +
  • Fix unnecessary queuing in deployments where no resource group configuration was specified.

  • +
+
+
+

Hive connector#

+
    +
  • Fix over-estimation of memory usage for scan operators when reading ORC files.

  • +
  • Fix memory accounting for sort buffer used for writing sorted bucketed tables.

  • +
  • Disallow creating tables with unsupported partition types.

  • +
  • Support overwriting partitions for insert queries. This behavior is controlled +by session property insert_existing_partitions_behavior.

  • +
  • Prevent the optimized ORC writer from writing excessively large stripes for +highly compressed, dictionary encoded columns.

  • +
  • Enable optimized Parquet reader and predicate pushdown by default.

  • +
+
+
+

Cassandra connector#

+
    +
  • Add support for reading from materialized views.

  • +
  • Optimize partition list retrieval for Cassandra 2.2+.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.205.html b/430/release/release-0.205.html new file mode 100644 index 000000000..6f5ba03f3 --- /dev/null +++ b/430/release/release-0.205.html @@ -0,0 +1,2655 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.205 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.205#

+
+

General#

+
    +
  • Fix parsing of row types where the field types contain spaces. +Previously, row expressions that included spaces would fail to parse. +For example: cast(row(timestamp '2018-06-01') AS row(timestamp with time zone)).

  • +
  • Fix distributed planning failure for complex queries when using bucketed execution.

  • +
  • Fix ST_ExteriorRing() to only accept polygons. +Previously, it erroneously accepted other geometries.

  • +
  • Add the task.min-drivers-per-task and task.max-drivers-per-task config options. +The former specifies the guaranteed minimum number of drivers a task will run concurrently +given that it has enough work to do. The latter specifies the maximum number of drivers +a task can run concurrently.

  • +
  • Add the concurrent-lifespans-per-task config property to control the default value +of the concurrent_lifespans_per_task session property.

  • +
  • Add the query_max_total_memory session property and the query.max-total-memory +config property. Queries will be aborted after their total (user + system) memory +reservation exceeds this threshold.

  • +
  • Improve stats calculation for outer joins and correlated subqueries.

  • +
  • Reduce memory usage when a Block contains all null or all non-null values.

  • +
  • Change the internal hash function used in approx_distinct. The result of approx_distinct +may change in this version compared to the previous version for the same set of values. However, +the standard error of the results should still be within the configured bounds.

  • +
  • Improve efficiency and reduce memory usage for scalar correlated subqueries with aggregations.

  • +
  • Remove the legacy local scheduler and associated configuration properties, +task.legacy-scheduling-behavior and task.level-absolute-priority.

  • +
  • Do not allow using the FILTER clause for the COALESCE, IF, or NULLIF functions. +The syntax was previously allowed but was otherwise ignored.

  • +
+
+
+

Security#

+
    +
  • Remove unnecessary check for SELECT privileges for DELETE queries. +Previously, DELETE queries could fail if the user only has DELETE +privileges but not SELECT privileges. +This only affected connectors that implement checkCanSelectFromColumns().

  • +
  • Add a check that the view owner has permission to create the view when +running SELECT queries against a view. This only affected connectors that +implement checkCanCreateViewWithSelectFromColumns().

  • +
  • Change DELETE FROM <table> WHERE <condition> to check that the user has SELECT +privileges on the objects referenced by the WHERE condition as is required by the SQL standard.

  • +
  • Improve the error message when access is denied when selecting from a view due to the +view owner having insufficient permissions to create the view.

  • +
+
+
+

JDBC driver#

+
    +
  • Add support for prepared statements.

  • +
  • Add partial query cancellation via partialCancel() on PrestoStatement.

  • +
  • Use VARCHAR rather than LONGNVARCHAR for the Presto varchar type.

  • +
  • Use VARBINARY rather than LONGVARBINARY for the Presto varbinary type.

  • +
+
+
+

Hive connector#

+
    +
  • Improve the performance of INSERT queries when all partition column values are constants.

  • +
  • Improve stripe size estimation for the optimized ORC writer. +This reduces the number of cases where tiny ORC stripes will be written.

  • +
  • Respect the skip.footer.line.count Hive table property.

  • +
+
+
+

CLI#

+
    +
  • Prevent the CLI from crashing when running on certain 256 color terminals.

  • +
+
+
+

SPI#

+
    +
  • Add a context parameter to the create() method in SessionPropertyConfigurationManagerFactory.

  • +
  • Disallow non-static methods to be annotated with @ScalarFunction. Non-static SQL function +implementations must now be declared in a class annotated with @ScalarFunction.

  • +
  • Disallow having multiple public constructors in @ScalarFunction classes. All non-static +implementations of SQL functions will now be associated with a single constructor. +This improves support for providing specialized implementations of SQL functions with generic arguments.

  • +
  • Deprecate checkCanSelectFromTable/checkCanSelectFromView and +checkCanCreateViewWithSelectFromTable/checkCanCreateViewWithSelectFromView in ConnectorAccessControl +and SystemAccessControl. checkCanSelectFromColumns and checkCanCreateViewWithSelectFromColumns +should be used instead.

  • +
+
+

Note

+

These are backwards incompatible changes with the previous SPI. +If you have written a plugin using these features, you will need +to update your code before deploying this release.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.206.html b/430/release/release-0.206.html new file mode 100644 index 000000000..e0a722d71 --- /dev/null +++ b/430/release/release-0.206.html @@ -0,0 +1,2603 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.206 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.206#

+
+

General#

+
    +
  • Fix execution failure for certain queries containing a join followed by an aggregation +when dictionary_aggregation is enabled.

  • +
  • Fix planning failure when a query contains a GROUP BY, but the cardinality of the +grouping columns is one. For example: SELECT c1, sum(c2) FROM t WHERE c1 = 'foo' GROUP BY c1

  • +
  • Fix high memory pressure on the coordinator during the execution of queries +using bucketed execution.

  • +
  • Add ST_Union(), ST_Geometries(), ST_PointN(), ST_InteriorRings(), +and ST_InteriorRingN() geospatial functions.

  • +
  • Add split_to_multimap() function.

  • +
  • Expand the approx_distinct() function to support the following types: +INTEGER, SMALLINT, TINYINT, DECIMAL, REAL, DATE, +TIMESTAMP, TIMESTAMP WITH TIME ZONE, TIME, TIME WITH TIME ZONE, IPADDRESS.

  • +
  • Add a resource group ID column to the system.runtime.queries table.

  • +
  • Add support for executing ORDER BY without LIMIT in a distributed manner. +This can be disabled with the distributed-sort configuration property +or the distributed_sort session property.

  • +
  • Add implicit coercion from VARCHAR(n) to CHAR(n), and remove implicit coercion the other way around. +As a result, comparing a CHAR with a VARCHAR will now follow +trailing space insensitive CHAR comparison semantics.

  • +
  • Improve query cost estimation by only including non-null rows when computing average row size.

  • +
  • Improve query cost estimation to better account for overhead when estimating data size.

  • +
  • Add new semantics that conform to the SQL standard for temporal types. +It affects the TIMESTAMP (aka TIMESTAMP WITHOUT TIME ZONE) type, +TIME (aka TIME WITHOUT TIME ZONE) type, and TIME WITH TIME ZONE type. +The legacy behavior remains default. +At this time, it is not recommended to enable the new semantics. +For any connector that supports temporal types, code changes are required before the connector +can work correctly with the new semantics. No connectors have been updated yet. +In addition, the new semantics are not yet stable as more breaking changes are planned, +particularly around the TIME WITH TIME ZONE type.

  • +
+
+
+

JDBC driver#

+
    +
  • Add applicationNamePrefix parameter, which is combined with +the ApplicationName property to construct the client source name.

  • +
+
+
+

Hive connector#

+
    +
  • Reduce ORC reader memory usage by reducing unnecessarily large internal buffers.

  • +
  • Support reading from tables with skip.footer.line.count and skip.header.line.count +when using HDFS authentication with Kerberos.

  • +
  • Add support for case-insensitive column lookup for Parquet readers.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.207.html b/430/release/release-0.207.html new file mode 100644 index 000000000..455d05714 --- /dev/null +++ b/430/release/release-0.207.html @@ -0,0 +1,2635 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.207 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.207#

+
+

General#

+
    +
  • Fix a planning issue for queries where correlated references were used in VALUES.

  • +
  • Remove support for legacy JOIN ... USING behavior.

  • +
  • Change behavior for unnesting an array of row type to produce multiple columns.

  • +
  • Deprecate the reorder_joins session property and the reorder-joins +configuration property. They are replaced by the join_reordering_strategy +session property and the optimizer.join-reordering-strategy configuration +property. NONE maintains the order of the joins as written and is equivalent +to reorder_joins=false. ELIMINATE_CROSS_JOINS will eliminate any +unnecessary cross joins from the plan and is equivalent to reorder_joins=true. +AUTOMATIC will use the new cost-based optimizer to select the best join order. +To simplify migration, setting the reorder_joins session property overrides the +new session and configuration properties.

  • +
  • Deprecate the distributed_joins session property and the +distributed-joins-enabled configuration property. They are replaced by the +join_distribution_type session property and the join-distribution-type +configuration property. PARTITIONED turns on hash partitioned joins and +is equivalent to distributed_joins-enabled=true. BROADCAST changes the +join strategy to broadcast and is equivalent to distributed_joins-enabled=false. +AUTOMATIC will use the new cost-based optimizer to select the best join +strategy. If no statistics are available, AUTOMATIC is the same as +REPARTITIONED. To simplify migration, setting the distributed_joins +session property overrides the new session and configuration properties.

  • +
  • Add support for column properties.

  • +
  • Add optimizer.max-reordered-joins configuration property to set the maximum number of joins that +can be reordered at once using cost-based join reordering.

  • +
  • Add support for char type to approx_distinct().

  • +
+
+
+

Security#

+
    +
  • Fail on startup when configuration for file based system access control is invalid.

  • +
  • Add support for securing communication between cluster nodes with Kerberos authentication.

  • +
+
+
+

Web UI#

+
    +
  • Add peak total (user + system) memory to query details UI.

  • +
+
+
+

Hive connector#

+
    +
  • Fix handling of VARCHAR(length) type in the optimized Parquet reader. Previously, predicate pushdown +failed with Mismatched Domain types: varchar(length) vs varchar.

  • +
  • Fail on startup when configuration for file based access control is invalid.

  • +
  • Add support for HDFS wire encryption.

  • +
  • Allow ORC files to have struct columns with missing fields. This allows the table schema to be changed +without rewriting the ORC files.

  • +
  • Change collector for columns statistics to only consider a sample of partitions. The sample size can be +changed by setting the hive.partition-statistics-sample-size property.

  • +
+
+
+

Memory connector#

+
    +
  • Add support for dropping schemas.

  • +
+
+
+

SPI#

+
    +
  • Remove deprecated table/view-level access control methods.

  • +
  • Change predicate in constraint for accessing table layout to be optional.

  • +
  • Change schema name in ConnectorMetadata to be optional rather than nullable.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.208.html b/430/release/release-0.208.html new file mode 100644 index 000000000..7c1135fe2 --- /dev/null +++ b/430/release/release-0.208.html @@ -0,0 +1,2608 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.208 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.208#

+
+

Warning

+

This release has the potential for data loss in the Hive connector +when writing bucketed sorted tables.

+
+
+

General#

+
    +
  • Fix an issue with memory accounting that would lead to garbage collection pauses +and out of memory exceptions.

  • +
  • Fix an issue that produces incorrect results when push_aggregation_through_join +is enabled (#10724).

  • +
  • Preserve field names when unnesting columns of type ROW.

  • +
  • Make the cluster out of memory killer more resilient to memory accounting leaks. +Previously, memory accounting leaks on the workers could effectively disable +the out of memory killer.

  • +
  • Improve planning time for queries over tables with high column count.

  • +
  • Add a limit on the number of stages in a query. The default is 100 and can +be changed with the query.max-stage-count configuration property and the +query_max_stage_count session property.

  • +
  • Add spooky_hash_v2_32() and spooky_hash_v2_64() functions.

  • +
  • Add a cluster memory leak detector that logs queries that have possibly accounted for +memory usage incorrectly on workers. This is a tool to for debugging internal errors.

  • +
  • Add support for correlated subqueries requiring coercions.

  • +
  • Add experimental support for running on Linux ppc64le.

  • +
+
+
+

CLI#

+
    +
  • Fix creation of the history file when it does not exist.

  • +
  • Add PRESTO_HISTORY_FILE environment variable to override location of history file.

  • +
+
+
+

Hive connector#

+
    +
  • Remove size limit for writing bucketed sorted tables.

  • +
  • Support writer scaling for Parquet.

  • +
  • Improve stripe size estimation for the optimized ORC writer. This reduces the +number of cases where tiny ORC stripes will be written.

  • +
  • Provide the actual size of CHAR, VARCHAR, and VARBINARY columns to the cost based optimizer.

  • +
  • Collect column level statistics when writing tables. This is disabled by default, +and can be enabled by setting the hive.collect-column-statistics-on-write property.

  • +
+
+
+

Thrift connector#

+
    +
  • Include error message from remote server in query failure message.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.209.html b/430/release/release-0.209.html new file mode 100644 index 000000000..ca2f2dad0 --- /dev/null +++ b/430/release/release-0.209.html @@ -0,0 +1,2649 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.209 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.209#

+
+

General#

+
    +
  • Fix incorrect predicate pushdown when grouping sets contain the empty grouping set (#11296).

  • +
  • Fix X-Forwarded-Proto header handling for requests to the / path (#11168).

  • +
  • Fix a regression that results in execution failure when at least one +of the arguments to min_by() or max_by() is a constant NULL.

  • +
  • Fix failure when some buckets are completely filtered out during bucket-by-bucket execution.

  • +
  • Fix execution failure of queries due to a planning deficiency involving +complex nested joins where a join that is not eligible for bucket-by-bucket +execution feeds into the build side of a join that is eligible.

  • +
  • Improve numerical stability for corr(), covar_samp(), +regr_intercept(), and regr_slope().

  • +
  • Do not include column aliases when checking column access permissions.

  • +
  • Eliminate unnecessary data redistribution for scalar correlated subqueries.

  • +
  • Remove table scan original constraint information from EXPLAIN output.

  • +
  • Introduce distinct error codes for global and per-node memory limit errors.

  • +
  • Include statistics and cost estimates for EXPLAIN (TYPE DISTRIBUTED) and EXPLAIN ANALYZE.

  • +
  • Support equality checks for ARRAY, MAP, and ROW values containing nulls.

  • +
  • Improve statistics estimation and fix potential negative nulls fraction +estimates for expressions that include NOT or OR.

  • +
  • Completely remove the SHOW PARTITIONS statement.

  • +
  • Add bing_tiles_around() variant that takes a radius.

  • +
  • Add the convex_hull_agg() and geometry_union_agg() geospatial aggregation functions.

  • +
  • Add (TYPE IO, FORMAT JSON) option for EXPLAIN that shows +input tables with constraints and the output table in JSON format.

  • +
  • Add Kudu connector.

  • +
  • Raise required Java version to 8u151. This avoids correctness issues for +map to map cast when running under some earlier JVM versions, including 8u92.

  • +
+
+
+

Web UI#

+
    +
  • Fix the kill query button on the live plan and stage performance pages.

  • +
+
+
+

CLI#

+
    +
  • Prevent spurious “No route to host” errors on macOS when using IPv6.

  • +
+
+
+

JDBC driver#

+
    +
  • Prevent spurious “No route to host” errors on macOS when using IPv6.

  • +
+
+
+

Hive connector#

+
    +
  • Fix data loss when writing bucketed sorted tables. Partitions would +be missing arbitrary rows if any of the temporary files for a bucket +had the same size. The numRows partition property contained the +correct number of rows and can be used to detect if this occurred.

  • +
  • Fix cleanup of temporary files when writing bucketed sorted tables.

  • +
  • Allow creating schemas when using file based security.

  • +
  • Reduce the number of cases where tiny ORC stripes will be written when +some columns are highly dictionary compressed.

  • +
  • Improve memory accounting when reading ORC files. Previously, buffer +memory and object overhead was not tracked for stream readers.

  • +
  • ORC struct columns are now mapped by name rather than ordinal. +This correctly handles missing or extra struct fields in the ORC file.

  • +
  • Add procedure system.create_empty_partition() for creating empty partitions.

  • +
+
+
+

Kafka connector#

+
    +
  • Support Avro formatted Kafka messages.

  • +
  • Support backward compatible Avro schema evolution.

  • +
+
+
+

SPI#

+
    +
  • Allow using Object as a parameter type or return type for SQL +functions when the corresponding SQL type is an unbounded generic.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.210.html b/430/release/release-0.210.html new file mode 100644 index 000000000..c15a0c492 --- /dev/null +++ b/430/release/release-0.210.html @@ -0,0 +1,2607 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.210 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.210#

+
+

General#

+
    +
  • Fix planning failure when aliasing columns of tables containing hidden +columns (#11385).

  • +
  • Fix correctness issue when GROUP BY DISTINCT terms contain references to +the same column using different syntactic forms (#11120).

  • +
  • Fix failures when querying information_schema tables using capitalized names.

  • +
  • Improve performance when converting between ROW types.

  • +
  • Remove user CPU time tracking as introduces non-trivial overhead.

  • +
  • Select join distribution type automatically for queries involving outer joins.

  • +
+
+
+

Hive connector#

+
    +
  • Fix a security bug introduced in 0.209 when using hive.security=file, +which would allow any user to create, drop, or rename schemas.

  • +
  • Prevent ORC writer from writing stripes larger than the max configured size +when converting a highly dictionary compressed column to direct encoding.

  • +
  • Support creating Avro tables with a custom schema using the avro_schema_url +table property.

  • +
  • Support backward compatible Avro schema evolution.

  • +
  • Support cross-realm Kerberos authentication for HDFS and Hive Metastore.

  • +
+
+
+

JDBC driver#

+
    +
  • Deallocate prepared statement when PreparedStatement is closed. Previously, +Connection became unusable after many prepared statements were created.

  • +
  • Remove getUserTimeMillis() from QueryStats and StageStats.

  • +
+
+
+

SPI#

+
    +
  • SystemAccessControl.checkCanSetUser() now takes an Optional<Principal> +rather than a nullable Principal.

  • +
  • Rename connectorId to catalogName in ConnectorFactory, +QueryInputMetadata, and QueryOutputMetadata.

  • +
  • Pass ConnectorTransactionHandle to ConnectorAccessControl.checkCanSetCatalogSessionProperty().

  • +
  • Remove getUserTime() from SplitStatistics (referenced in SplitCompletedEvent).

  • +
+
+

Note

+

These are backwards incompatible changes with the previous SPI. +If you have written a plugin, you will need to update your code +before deploying this release.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.211.html b/430/release/release-0.211.html new file mode 100644 index 000000000..6a4c22373 --- /dev/null +++ b/430/release/release-0.211.html @@ -0,0 +1,2619 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.211 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.211#

+
+

General#

+
    +
  • Fix missing final query plan in QueryCompletedEvent. Statistics and cost estimates +are removed from the plan text because they may not be available during event generation.

  • +
  • Update the default value of the http-server.https.excluded-cipher config +property to exclude cipher suites with a weak hash algorithm or without forward secrecy. +Specifically, this means all ciphers that use the RSA key exchange are excluded by default. +Consequently, TLS 1.0 or TLS 1.1 are no longer supported with the default configuration. +The http-server.https.excluded-cipher config property can be set to empty string +to restore the old behavior.

  • +
  • Add ST_GeomFromBinary() and ST_AsBinary() functions that convert +geometries to and from Well-Known Binary format.

  • +
  • Remove the verbose_stats session property, and rename the task.verbose-stats +configuration property to task.per-operator-cpu-timer-enabled.

  • +
  • Improve query planning performance for queries containing multiple joins +and a large number of columns (#11196).

  • +
  • Add built-in file based property manager +to automate the setting of session properties based on query characteristics.

  • +
  • Allow running on a JVM from any vendor that meets the functional requirements.

  • +
+
+
+

Hive connector#

+
    +
  • Fix regression in 0.210 that causes query failure when writing ORC or DWRF files +that occurs for specific patterns of input data. When the writer attempts to give up +using dictionary encoding for a column that is highly compressed, the process of +transitioning to use direct encoding instead can fail.

  • +
  • Fix coordinator OOM when a query scans many partitions of a Hive table (#11322).

  • +
  • Improve readability of columns, partitioning, and transactions in explain plains.

  • +
+
+
+

Thrift connector#

+
    +
  • Fix lack of retry for network errors while sending requests.

  • +
+
+
+

Resource group#

+
    +
  • Add documentation for new resource group scheduling policies.

  • +
  • Remove running and queue time limits from resource group configuration. +Legacy behavior can be replicated by using the +file based property manager +to set session properties.

  • +
+
+
+

SPI#

+
    +
  • Clarify semantics of predicate in ConnectorTableLayout.

  • +
  • Reduce flexibility of unenforcedConstraint that a connector can return in getTableLayouts. +For each column in the predicate, the connector must enforce the entire domain or none.

  • +
  • Make the null vector in ArrayBlock, MapBlock, and RowBlock optional. +When it is not present, all entries in the Block are non-null.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.212.html b/430/release/release-0.212.html new file mode 100644 index 000000000..0ddb3f592 --- /dev/null +++ b/430/release/release-0.212.html @@ -0,0 +1,2594 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.212 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.212#

+
+

General#

+
    +
  • Fix query failures when the ST_GeomFromBinary() function is run on multiple rows.

  • +
  • Fix memory accounting for the build side of broadcast joins.

  • +
  • Fix occasional query failures when running EXPLAIN ANALYZE.

  • +
  • Enhance ST_ConvexHull() and convex_hull_agg() functions to support geometry collections.

  • +
  • Improve performance for some queries using DISTINCT.

  • +
  • Improve performance for some queries that perform filtered global aggregations.

  • +
  • Remove round(x, d) and truncate(x, d) functions where d is a BIGINT (#11462).

  • +
  • Add ST_LineString() function to form a LineString from an array of points.

  • +
+
+
+

Hive connector#

+
    +
  • Prevent ORC writer from writing stripes larger than the max configured size for some rare data +patterns (#11526).

  • +
  • Restrict the maximum line length for text files. The default limit of 100MB can be changed +using the hive.text.max-line-length configuration property.

  • +
  • Add sanity checks that fail queries if statistics read from the metastore are corrupt. Corrupt +statistics can be ignored by setting the hive.ignore-corrupted-statistics +configuration property or the ignore_corrupted_statistics session property.

  • +
+
+
+

Thrift connector#

+
    +
  • Fix retry for network errors that occur while sending a Thrift request.

  • +
  • Remove failed connections from connection pool.

  • +
+
+
+

Verifier#

+
    +
  • Record the query ID of the test query regardless of query outcome.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.213.html b/430/release/release-0.213.html new file mode 100644 index 000000000..231065990 --- /dev/null +++ b/430/release/release-0.213.html @@ -0,0 +1,2682 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.213 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.213#

+
+

General#

+
    +
  • Fix split scheduling backpressure when plan contains colocated join. Previously, splits +for the second and subsequent scan nodes (in scheduling order) were scheduled continuously +until completion, rather than pausing due to sufficient pending splits.

  • +
  • Fix query execution failure or indefinite hang during grouped execution when all splits +for any lifespan are completely filtered out.

  • +
  • Fix grouped execution to respect the configured concurrent lifespans per task. +Previously, it always used a single lifespan per task.

  • +
  • Fix execution failure when using grouped execution with right or full outer joins +where the right side is not partitioned on the join key.

  • +
  • Fix a scenario where too many rows are returned to clients in a single response.

  • +
  • Do not allow setting invalid property values with SET SESSION.

  • +
  • Disable stats calculator by default as it can cause a planning failure for +certain complex queries. It can be enabled with the experimental.enable-stats-calculator +configuration property or the enable_stats_calculator session property.

  • +
  • Avoid making guesses when estimating filters for joins. Previously, if nothing +was known about the filter, a 0.9 coefficient was applied as a filter factor. +Now, if nothing is known about a filter, the estimate will be unknown. A 0.9 +coefficient will be applied for all additional conjuncts if at least a single +conjunct can be reasonably estimated.

  • +
  • Improve inference of predicates for inner joins.

  • +
  • Improve EXPLAIN ANALYZE output by adding CPU time and enhancing accuracy of CPU fraction.

  • +
  • Include stats and cost estimates in textual plans created on query completion.

  • +
  • Enhance SHOW STATS to support IN and BETWEEN predicates in the +WHERE condition of the SELECT clause.

  • +
  • Remove transaction from explain plan for indexes joins.

  • +
  • Add max_drivers_per_task session property, allowing users to limit concurrency by +specifying a number lower than the system configured maximum. This can cause the +query to run slower and consume less resources.

  • +
  • Add join-max-broadcast-table-size configuration property and +join_max_broadcast_table_size session property to control the maximum estimated size +of a table that can be broadcast when using AUTOMATIC join distribution type (#11667).

  • +
  • Add experimental config option experimental.reserved-pool-enabled to disable the reserved memory pool.

  • +
  • Add targetResultSize query parameter to /v1/statement endpoint to control response data size.

  • +
+
+
+

Geospatial#

+
    +
  • Fix ST_Distance() function to return NULL if any of the inputs is an +empty geometry as required by the SQL/MM specification.

  • +
  • Add ST_MultiPoint() function to construct multi-point geometry from an array of points.

  • +
  • Add geometry_union() function to efficiently union arrays of geometries.

  • +
  • Add support for distributed spatial joins (#11072).

  • +
+
+
+

Server RPM#

+
    +
  • Allow running on a JVM from any vendor.

  • +
+
+
+

Web UI#

+
    +
  • Remove legacy plan UI.

  • +
  • Add support for filtering queries by all error categories.

  • +
  • Add dialog to show errors refreshing data from coordinator.

  • +
  • Change worker thread list to not show thread stacks by default to improve page peformance.

  • +
+
+
+

Hive connector#

+
    +
  • Fix LZO and LZOP decompression to work with certain data compressed by Hadoop.

  • +
  • Fix ORC writer validation percentage so that zero does not result in 100% validation.

  • +
  • Fix potential out-of-bounds read for ZSTD on corrupted input.

  • +
  • Stop assuming no distinct values when column null fraction statistic is less than 1.0.

  • +
  • Treat -1 as an absent null count for compatibility with statistics written by +Impala.

  • +
  • Preserve original exception for metastore network errors.

  • +
  • Preserve exceptions from Avro deserializer

  • +
  • Categorize text line length exceeded error.

  • +
  • Remove the old Parquet reader. The hive.parquet-optimized-reader.enabled +configuration property and parquet_optimized_reader_enabled session property +no longer exist.

  • +
  • Remove the hive.parquet-predicate-pushdown.enabled configuration property +and parquet_predicate_pushdown_enabled session property. +Pushdown is always enabled now in the Parquet reader.

  • +
  • Enable optimized ORC writer by default. It can be disabled using the +hive.orc.optimized-writer.enabled configuration property or the +orc_optimized_writer_enabled session property.

  • +
  • Use ORC file format as the default for new tables or partitions.

  • +
  • Add support for Avro tables where the Avro schema URL is an HDFS location.

  • +
  • Add hive.parquet.writer.block-size and hive.parquet.writer.page-size +configuration properties and parquet_writer_block_size and +parquet_writer_page_size session properties for tuning Parquet writer options.

  • +
+
+
+

Memory connector#

+
    +
  • Improve table data size accounting.

  • +
+
+
+

Thrift connector#

+
    +
  • Include constraint in explain plan for index joins.

  • +
  • Improve readability of columns, tables, layouts, and indexes in explain plans.

  • +
+
+
+

Verifier#

+
    +
  • Rewrite queries in parallel when shadowing writes.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.214.html b/430/release/release-0.214.html new file mode 100644 index 000000000..1e5698712 --- /dev/null +++ b/430/release/release-0.214.html @@ -0,0 +1,2631 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.214 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.214#

+
+

General#

+
    +
  • Fix history leak in coordinator for failed or canceled queries.

  • +
  • Fix memory leak related to query tracking in coordinator that was introduced +in Release 0.213.

  • +
  • Fix planning failures when lambdas are used in join filter expression.

  • +
  • Fix responses to client for certain types of errors that are encountered +during query creation.

  • +
  • Improve error message when an invalid comparator is provided to the +array_sort() function.

  • +
  • Improve performance of lookup operations on map data types.

  • +
  • Improve planning and query performance for queries with TINYINT, +SMALLINT and VARBINARY literals.

  • +
  • Fix issue where queries containing distributed ORDER BY and aggregation +could sometimes fail to make progress when data was spilled.

  • +
  • Make top N row number optimization work in some cases when columns are pruned.

  • +
  • Add session property optimize-top-n-row-number and configuration property +optimizer.optimize-top-n-row-number to toggle the top N row number +optimization.

  • +
  • Add ngrams() function to generate N-grams from an array.

  • +
  • Add qdigest type and associated Quantile digest functions.

  • +
  • Add functionality to delay query execution until a minimum number of workers +nodes are available. The minimum number of workers can be set with the +query-manager.required-workers configuration property, and the max wait +time with the query-manager.required-workers-max-wait configuration property.

  • +
  • Remove experimental pre-allocated memory system, and the related configuration +property experimental.preallocate-memory-threshold.

  • +
+
+
+

Security#

+
    +
  • Add functionality to refresh the configuration of file-based access controllers. +The refresh interval can be set using the security.refresh-period +configuration property.

  • +
+
+
+

JDBC driver#

+
    +
  • Clear update count after calling Statement.getMoreResults().

  • +
+
+
+

Web UI#

+
    +
  • Show query warnings on the query detail page.

  • +
  • Allow selecting non-default sort orders in query list view.

  • +
+
+
+

Hive connector#

+
    +
  • Prevent ORC writer from writing stripes larger than the maximum configured size.

  • +
  • Add hive.s3.upload-acl-type configuration property to specify the type of +ACL to use while uploading files to S3.

  • +
  • Add Hive metastore API recording tool for remote debugging purposes.

  • +
  • Add support for retrying on metastore connection errors.

  • +
+
+
+

Verifier#

+
    +
  • Handle SQL execution timeouts while rewriting queries.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.215.html b/430/release/release-0.215.html new file mode 100644 index 000000000..ebd81325b --- /dev/null +++ b/430/release/release-0.215.html @@ -0,0 +1,2621 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.215 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.215#

+
+

General#

+
    +
  • Fix regression in 0.214 that could cause queries to produce incorrect results for queries +using map types.

  • +
  • Fix reporting of the processed input data for source stages in EXPLAIN ANALYZE.

  • +
  • Fail queries that use non-leaf resource groups. Previously, they would remain queued forever.

  • +
  • Improve CPU usage for specific queries (#11757).

  • +
  • Extend stats and cost model to support row_number() window function estimates.

  • +
  • Improve the join type selection and the reordering of join sides for cases where +the join output size cannot be estimated.

  • +
  • Add dynamic scheduling support to grouped execution. When a stage is executed +with grouped execution and the stage has no remote sources, table partitions can be +scheduled to tasks in a dynamic way, which can help mitigating skew for queries using +grouped execution. This feature can be enabled with the +dynamic_schedule_for_grouped_execution session property or the +dynamic-schedule-for-grouped-execution config property.

  • +
  • Add beta_cdf() and inverse_beta_cdf() functions.

  • +
  • Split the reporting of raw input data and processed input data for source operators.

  • +
  • Remove collection and reporting of raw input data statistics for the Values, +Local Exchange, and Local Merge Sort operators.

  • +
  • Simplify EXPLAIN (TYPE IO) output when there are too many discrete components. +This avoids large output at the cost of reduced granularity.

  • +
  • Add parse_presto_data_size() function.

  • +
  • Add support for UNION ALL to optimizer’s cost model.

  • +
  • Add support for estimating the cost of filters by using a default filter factor. +The default value for the filter factor can be configured with the default_filter_factor_enabled +session property or the optimizer.default-filter-factor-enabled.

  • +
+
+
+

Geospatial#

+
    +
  • Add input validation checks to ST_LineString() to conform with the specification.

  • +
  • Improve spatial join performance.

  • +
  • Enable spatial joins for join conditions expressed with the ST_Within() function.

  • +
+
+
+

Web UI#

+
    +
  • Fix Capture Snapshot button for showing current thread stacks.

  • +
  • Fix dropdown for expanding stage skew component on the query details page.

  • +
  • Improve the performance of the thread snapshot component on the worker status page.

  • +
  • Make the reporting of Cumulative Memory usage consistent on the query list and query details pages.

  • +
  • Remove legacy thread UI.

  • +
+
+
+

Hive#

+
    +
  • Add predicate pushdown support for the DATE type to the Parquet reader. This change also fixes +a bug that may cause queries with predicates on DATE columns to fail with type mismatch errors.

  • +
+
+
+

Redis#

+
    +
  • Prevent printing the value of the redis.password configuration property to log files.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.54.html b/430/release/release-0.54.html new file mode 100644 index 000000000..f3067fb2d --- /dev/null +++ b/430/release/release-0.54.html @@ -0,0 +1,2551 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.54 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.54#

+
    +
  • Restore binding for the node resource on the coordinator, which provides +the state of all nodes as seen by the coordinator’s failure detector. +Access /v1/node to see all nodes, or /v1/node/failed to see failed nodes.

  • +
  • Prevent the Command line interface from hanging when the server goes away.

  • +
  • Add Hive connector hive-hadoop1 for Apache Hadoop 1.x.

  • +
  • Add support for Snappy and LZ4 compression codecs for the hive-cdh4 connector.

  • +
  • Add Example HTTP connector example-http that reads CSV data via HTTP. +The connector requires a metadata URI that returns a JSON document +describing the table metadata and the CSV files to read.

    +

    Its primary purpose is to serve as an example of how to write a connector, +but it can also be used directly. Create etc/catalog/example.properties +with the following contents to mount the example-http connector as the +example catalog:

    +
    connector.name=example-http
    +metadata-uri=http://s3.amazonaws.com/presto-example/v1/example-metadata.json
    +
    +
    +
  • +
  • Show correct error message when a catalog or schema does not exist.

  • +
  • Verify JVM requirements on startup.

  • +
  • Log an error when the JVM code cache is full.

  • +
  • Upgrade the embedded Discovery server to allow using +non-UUID values for the node.id property.

  • +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.55.html b/430/release/release-0.55.html new file mode 100644 index 000000000..bf1632e42 --- /dev/null +++ b/430/release/release-0.55.html @@ -0,0 +1,2671 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.55 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 0.55#

+
+

RC binary 2-4x gain in CPU efficiency#

+

Presto uses custom fast-path decoding logic for specific Hive file +formats. In this release we have added a fast path for RCFile when using +the Binary SerDe (LazyBinaryColumnarSerDe). In our +micro benchmarks, we see a gain between 2x and 4x in CPU efficiency compared +to the generic (slow) path. Since Hive data decoding accounts for a +significant portion of the CPU time, this should +result in measurable gains for most queries over RC Binary encoded data. +Note that this optimization may not result in a reduction in latency +if your cluster is network or disk I/O bound.

+
+
+

Hash distributed aggregations#

+

GROUP BY aggregations are now distributed across a fixed number of machines. +This is controlled by the property query.initial-hash-partitions set in +etc/config.properties of the coordinator and workers. If the value is +larger than the number of machines available during query scheduling, Presto +will use all available machines. The default value is 8.

+

The maximum memory size of an aggregation is now +query.initial-hash-partitions times task.max-memory.

+
+
+

Simple distinct aggregations#

+

We have added support for the DISTINCT argument qualifier for aggregation +functions. This is currently limited to queries without a GROUP BY clause and +where all the aggregation functions have the same input expression. For example:

+
SELECT count(DISTINCT country)
+FROM users
+
+
+

Support for complete DISTINCT functionality is in our roadmap.

+
+
+

Range predicate pushdown#

+

We’ve modified the connector API to support range predicates in addition to simple equality predicates. +This lays the ground work for adding connectors to systems that support range +scans (e.g., HBase, Cassandra, JDBC, etc).

+

In addition to receiving range predicates, the connector can also communicate +back the ranges of each partition for use in the query optimizer. This can be a +major performance gain for JOIN queries where one side of the join has +only a few partitions. For example:

+
SELECT * FROM data_1_year JOIN data_1_week USING (ds)
+
+
+

If data_1_year and data_1_week are both partitioned on ds, the +connector will report back that one table has partitions for 365 days and the +other table has partitions for only 7 days. Then the optimizer will limit +the scan of the data_1_year table to only the 7 days that could possible +match. These constraints are combined with other predicates in the +query to further limit the data scanned.

+
+

Note

+

This is a backwards incompatible change with the previous connector SPI, +so if you have written a connector, you will need to update your code +before deploying this release.

+
+
+
+

json_array_get function#

+

The json_array_get() function makes it simple to fetch a single element from a +scalar json array.

+
+
+

Non-reserved keywords#

+

The keywords DATE, TIME, TIMESTAMP, and INTERVAL are no longer +reserved keywords in the grammar. This means that you can access a column +named date without quoting the identifier.

+
+
+

CLI source option#

+

The Presto CLI now has an option to set the query source. The source +value is shown in the UI and is recorded in events. When using the CLI in +shell scripts it is useful to set the --source option to distinguish shell +scripts from normal users.

+
+
+

SHOW SCHEMAS FROM#

+

Although the documentation included the syntax SHOW SCHEMAS [FROM catalog], +it was not implemented. This release now implements this statement correctly.

+
+
+

Hive bucketed table fixes#

+

For queries over Hive bucketed tables, Presto will attempt to limit scans to +the buckets that could possible contain rows that match the WHERE clause. +Unfortunately, the algorithm we were using to select the buckets was not +correct, and sometimes we would either select the wrong files or fail to +select any files. We have aligned +the algorithm with Hive and now the optimization works as expected.

+

We have also improved the algorithm for detecting tables that are not properly +bucketed. It is common for tables to declare bucketing in the Hive metadata, but +not actually be bucketed in HDFS. When Presto detects this case, it fallback to a full scan of the +partition. Not only does this change make bucketing safer, but it makes it easier +to migrate a table to use bucketing without rewriting all of the data.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.56.html b/430/release/release-0.56.html new file mode 100644 index 000000000..1cd38435e --- /dev/null +++ b/430/release/release-0.56.html @@ -0,0 +1,2580 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.56 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.56#

+
+

Table creation#

+

Tables can be created from the result of a query:

+
CREATE TABLE orders_by_date AS
+SELECT orderdate, sum(totalprice) AS price
+FROM orders
+GROUP BY orderdate
+
+
+

Tables are created in Hive without partitions (unpartitioned) and use +RCFile with the Binary SerDe (LazyBinaryColumnarSerDe) as this is +currently the best format for Presto.

+
+

Note

+

This is a backwards incompatible change to ConnectorMetadata in the SPI, +so if you have written a connector, you will need to update your code before +deploying this release. We recommend changing your connector to extend from +the new ReadOnlyConnectorMetadata abstract base class unless you want to +support table creation.

+
+
+
+

Cross joins#

+

Cross joins are supported using the standard ANSI SQL syntax:

+
SELECT *
+FROM a
+CROSS JOIN b
+
+
+

Inner joins that result in a cross join due to the join criteria evaluating +to true at analysis time are also supported.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.57.html b/430/release/release-0.57.html new file mode 100644 index 000000000..4153d36a4 --- /dev/null +++ b/430/release/release-0.57.html @@ -0,0 +1,2602 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.57 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.57#

+
+

Distinct aggregations#

+

The DISTINCT argument qualifier for aggregation functions is now +fully supported. For example:

+
SELECT country, count(DISTINCT city), count(DISTINCT age)
+FROM users
+GROUP BY country
+
+
+
+

Note

+

approx_distinct() should be used in preference to this +whenever an approximate answer is allowable as it is substantially +faster and does not have any limits on the number of distinct items it +can process. COUNT(DISTINCT ...) must transfer every item over the +network and keep each distinct item in memory.

+
+
+
+

Hadoop 2.x#

+

Use the hive-hadoop2 connector to read Hive data from Hadoop 2.x. +See Deploying Trino for details.

+
+
+

Amazon S3#

+

All Hive connectors support reading data from +Amazon S3. +This requires two additional catalog properties for the Hive connector +to specify your AWS Access Key ID and Secret Access Key:

+
hive.s3.aws-access-key=AKIAIOSFODNN7EXAMPLE
+hive.s3.aws-secret-key=wJalrXUtnFEMI/K7MDENG/bPxRfiCYEXAMPLEKEY
+
+
+
+
+

Miscellaneous#

+
    +
  • Allow specifying catalog and schema in the JDBC driver URL.

  • +
  • Implement more functionality in the JDBC driver.

  • +
  • Allow certain custom InputFormats to work by propagating +Hive serialization properties to the RecordReader.

  • +
  • Many execution engine performance improvements.

  • +
  • Fix optimizer performance regression.

  • +
  • Fix weird MethodHandle exception.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.58.html b/430/release/release-0.58.html new file mode 100644 index 000000000..f7404c1cb --- /dev/null +++ b/430/release/release-0.58.html @@ -0,0 +1,2549 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.58 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.58#

+
    +
  • Add first version of Cassandra connector. This plugin is still in +development and is not yet bundled with the server. See the README +in the plugin source directory for details.

  • +
  • Support UDFs for internal plugins. This is not yet part of the SPI +and is a stopgap feature intended for advanced users. UDFs must be +implemented using the internal Presto APIs which often change +substantially between releases.

  • +
  • Fix Hive connector semaphore release bug.

  • +
  • Fix handling of non-splittable files without blocks.

  • +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.59.html b/430/release/release-0.59.html new file mode 100644 index 000000000..6873d530c --- /dev/null +++ b/430/release/release-0.59.html @@ -0,0 +1,2542 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.59 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.59#

+
    +
  • Fix hang in HiveSplitSource. A query over a large table can hang +in split discovery due to a bug introduced in 0.57.

  • +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.60.html b/430/release/release-0.60.html new file mode 100644 index 000000000..d469f215a --- /dev/null +++ b/430/release/release-0.60.html @@ -0,0 +1,2715 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.60 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 0.60#

+
+

JDBC improvements#

+

The Presto version of the JDBC DatabaseMetaData interface now includes +proper implementations of getTables, getSchemas and getCatalogs.

+

The JDBC driver is now always packaged as a standalone jar without any +dependencies. Previously, this artifact was published with the Maven +classifier standalone. The new build does not publish this artifact +anymore.

+
+
+

USE CATALOG and USE SCHEMA#

+

The Command line interface now supports USE CATALOG and +USE SCHEMA.

+
+
+

TPCH connector#

+

We have added a new connector that will generate synthetic data following the +TPC-H specification. This connector makes it easy to generate large datasets for +testing and bug reports. When generating bug reports, we encourage users to use +this catalog since it eases the process of reproducing the issue. The data is +generated dynamically for each query, so no disk space is used by this +connector. To add the tpch catalog to your system, create the catalog +property file etc/catalog/tpch.properties on both the coordinator and workers +with the following contents:

+
connector.name=tpch
+
+
+

Additionally, update the datasources property in the config properties file, +etc/config.properties, for the workers to include tpch.

+
+
+

SPI#

+

The Connector interface now has explicit methods for supplying the services +expected by the query engine. Previously, this was handled by a generic +getService method.

+
+

Note

+

This is a backwards incompatible change to Connector in the SPI, +so if you have written a connector, you will need to update your code before +deploying this release.

+
+

Additionally, we have added the NodeManager interface to the SPI to allow a +plugin to detect all nodes in the Presto cluster. This is important for some +connectors that can divide a table evenly between all nodes as long as the +connector knows how many nodes exist. To access the node manager, simply add +the following to the Plugin class:

+
@Inject
+public void setNodeManager(NodeManager nodeManager)
+{
+    this.nodeManager = nodeManager;
+}
+
+
+
+
+

Optimizations#

+
+

DISTINCT LIMIT#

+

For queries with the following form:

+
SELECT DISTINCT ...
+FROM T
+LIMIT N
+
+
+

We have added an optimization that stops the query as soon as N distinct +rows are found.

+
+
+

Range predicates#

+

When optimizing a join, Presto analyzes the ranges of the partitions on each +side of a join and pushes these ranges to the other side. When tables have a +lot of partitions, this can result in a very large filter with one expression +for each partition. The optimizer now summarizes the predicate ranges to reduce +the complexity of the filters.

+
+
+

Compound filters#

+

Complex expressions involving AND, OR, or NOT are now optimized by +the expression optimizer.

+
+
+

Window functions#

+

Window functions with a PARTITION BY clause are now distributed based on the +partition key.

+
+
+
+

Bug fixes#

+
    +
  • Scheduling

    +

    In the changes to schedule splits in batches, we introduced two bugs that +resulted in an unbalanced workload across nodes which increases query latency. +The first problem was not inspecting the queued split count of the nodes while +scheduling the batch, and the second problem was not counting the splits +awaiting creation in the task executor.

    +
  • +
  • JSON conversion of complex Hive types

    +

    Presto converts complex Hive types (array, map, struct and union) into JSON. +Previously, numeric keys in maps were converted to numbers, not strings, +which is invalid as JSON only allows strings for object keys. This prevented +the JSON functions and operators from working.

    +
  • +
  • Hive hidden files

    +

    Presto will now ignore files in Hive that start with an underscore _ or +a dot .. This matches the behavior of Hadoop MapReduce / Hive.

    +
  • +
  • Failures incorrectly reported as no data

    +

    Certain types of failures would result in the query appearing to succeed and +return an incomplete result (often zero rows). There was a race condition +between the error propagation and query teardown. In some cases, the query +would be torn down before the exception made it to the coordinator. This was a +regression introduced during the query teardown optimization work. There are +now tests to catch this type of bug.

    +
  • +
  • Exchange client leak

    +

    When a query finished early (e.g., limit or failure) and the exchange operator +was blocked waiting for data from other nodes, the exchange was not be closed +properly. This resulted in continuous failing HTTP requests which leaked +resources and produced large log files.

    +
  • +
  • Hash partitioning

    +

    A query with many GROUP BY items could fail due to an overflow in the hash +function.

    +
  • +
  • Compiled NULL literal

    +

    In some cases queries with a select expression like CAST(NULL AS varchar) +would fail due to a bug in the output type detection code in expression +compiler.

    +
  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.61.html b/430/release/release-0.61.html new file mode 100644 index 000000000..7d7327fe3 --- /dev/null +++ b/430/release/release-0.61.html @@ -0,0 +1,2621 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.61 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 0.61#

+
+

Add support for table value constructors#

+

Presto now supports the SQL table value constructor syntax to create inline tables. +The VALUES clause can be used anywhere a SELECT statement is allowed. +For example, as a top-level query:

+
VALUES ('a', 1), ('b', 2);
+
+
+
 _col0 | _col1
+-------+-------
+ a     |     1
+ b     |     2
+(2 rows)
+
+
+

Alternatively, in the FROM clause:

+
SELECT *
+FROM (
+  VALUES
+    ('a', 'ape'),
+    ('b', 'bear')
+) AS animal (letter, animal)
+JOIN (
+  VALUES
+    ('a', 'apple'),
+    ('b', 'banana')
+) AS fruit (letter, fruit)
+USING (letter);
+
+
+
 letter | animal | letter |  fruit
+--------+--------+--------+---------
+ a      | ape    | a      | apple
+ b      | bear   | b      | banana
+(2 rows)
+
+
+
+
+

Cassandra#

+
    +
  • Add support for upper-case schema, table, and columns names.

  • +
  • Add support for DECIMAL type.

  • +
+
+
+

Amazon S3 support#

+
    +
  • Completely rewritten Hadoop FileSystem implementation for S3 using the Amazon AWS SDK, +with major performance and reliability improvements.

  • +
  • Add support for writing data to S3.

  • +
+
+
+

Miscellaneous#

+
    +
  • General improvements to the JDBC driver, specifically with respect to metadata handling.

  • +
  • Fix division by zero errors in variance aggregation functions (VARIANCE, STDDEV, etc.).

  • +
  • Fix a bug when using DISTINCT aggregations in the HAVING clause.

  • +
  • Fix an out of memory issue when writing large tables.

  • +
  • Fix a bug when using ORDER BY rand() in a JOIN query.

  • +
  • Fix handling of timestamps in maps and lists in Hive connector.

  • +
  • Add instrumentation for Hive metastore and HDFS API calls to track failures and latency. These metrics are exposed via JMX.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.62.html b/430/release/release-0.62.html new file mode 100644 index 000000000..14ca8be2c --- /dev/null +++ b/430/release/release-0.62.html @@ -0,0 +1,2545 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.62 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.62#

+
    +
  • Fix an issue with active queries JMX counter reporting incorrect numbers

  • +
  • Hive binary map keys were not being decoded correctly

  • +
  • Performance improvements for APPROX_DISTINCT

  • +
  • Fix performance regression when planning queries over a large number of partitions

  • +
  • Minor improvement to coordinator UI when displaying long SQL queries

  • +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.63.html b/430/release/release-0.63.html new file mode 100644 index 000000000..493745d00 --- /dev/null +++ b/430/release/release-0.63.html @@ -0,0 +1,2543 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.63 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.63#

+
    +
  • Minor improvements to coordinator UI

  • +
  • Minor planner optimization to avoid redundant computation in some cases

  • +
  • Error handling and classification improvements

  • +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.64.html b/430/release/release-0.64.html new file mode 100644 index 000000000..ee916e691 --- /dev/null +++ b/430/release/release-0.64.html @@ -0,0 +1,2546 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.64 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.64#

+
    +
  • Fix approximate aggregation error bound calculation

  • +
  • Error handling and classification improvements

  • +
  • Fix GROUP BY failure when keys are too large

  • +
  • Add thread visualization UI at /ui/thread

  • +
  • Fix regression in CREATE TABLE that can cause column data to be swapped. +This bug was introduced in version 0.57.

  • +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.65.html b/430/release/release-0.65.html new file mode 100644 index 000000000..260cbb015 --- /dev/null +++ b/430/release/release-0.65.html @@ -0,0 +1,2542 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.65 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.65#

+
    +
  • Fix NullPointerException when tearing down queries

  • +
  • Fix exposed third-party dependencies in JDBC driver JAR

  • +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.66.html b/430/release/release-0.66.html new file mode 100644 index 000000000..550dbe0d0 --- /dev/null +++ b/430/release/release-0.66.html @@ -0,0 +1,2748 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.66 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+ +
+
+ +
+
+ +
+

Release 0.66#

+
+

Type system#

+

In this release we have replaced the existing simple fixed type system +with a fully extensible type system and have added several new types. +We have also expanded the function system to support custom +arithmetic, comparison and cast operators. For example, the new date/time +types include an operator for adding an INTERVAL to a TIMESTAMP.

+

Existing functions have been updated to operate on and return the +newly added types. For example, the ANSI color functions now operate +on a COLOR type, and the date/time functions operate on standard +SQL date/time types (described below).

+

Finally, plugins can now provide custom types and operators in addition +to connectors and functions. This feature is highly experimental, so expect +the interfaces to change over the next few releases. Also, since in SQL +there is only one namespace for types, you should be careful to make names +for custom types unique as we will add other common SQL types to Presto +in the near future.

+
+
+

Date/time types#

+

Presto now supports all standard SQL date/time types: +DATE, TIME, TIMESTAMP and INTERVAL. +All of the date/time functions and language constructs now operate on these +types instead of BIGINT and perform temporal calculations correctly. +This was previously broken due to, for example, not being able to detect +whether an argument was a DATE or a TIMESTAMP. +This change comes at the cost of breaking existing queries that perform +arithmetic operations directly on the BIGINT value returned from +the date/time functions.

+

As part of this work, we have also added the date_trunc() function +which is convenient for grouping data by a time span. For example, you +can perform an aggregation by hour:

+
SELECT date_trunc('hour', timestamp_column), count(*)
+FROM ...
+GROUP BY 1
+
+
+
+

Time zones#

+

This release has full support for time zone rules, which are needed to +perform date/time calculations correctly. Typically, the session time +zone is used for temporal calculations. This is the time zone of the +client computer that submits the query, if available. Otherwise, it is +the time zone of the server running the Presto coordinator.

+

Queries that operate with time zones that follow daylight saving can +produce unexpected results. For example, if we run the following query +to add 24 hours using in the America/Los Angeles time zone:

+
SELECT date_add('hour', 24, TIMESTAMP '2014-03-08 09:00:00');
+-- 2014-03-09 10:00:00.000
+
+
+

The timestamp appears to only advance 23 hours. This is because on +March 9th clocks in America/Los Angeles are turned forward 1 hour, +so March 9th only has 23 hours. To advance the day part of the timestamp, +use the day unit instead:

+
SELECT date_add('day', 1, TIMESTAMP '2014-03-08 09:00:00');
+-- 2014-03-09 09:00:00.000
+
+
+

This works because the date_add() function treats the timestamp as +list of fields, adds the value to the specified field and then rolls any +overflow into the next higher field.

+

Time zones are also necessary for parsing and printing timestamps. +Queries that use this functionality can also produce unexpected results. +For example, on the same machine:

+
SELECT TIMESTAMP '2014-03-09 02:30:00';
+
+
+

The above query causes an error because there was no 2:30 AM on March 9th +in America/Los_Angeles due to a daylight saving time transition.

+

In addition to normal TIMESTAMP values, Presto also supports the +TIMESTAMP WITH TIME ZONE type, where every value has an explicit time zone. +For example, the following query creates a TIMESTAMP WITH TIME ZONE:

+
SELECT TIMESTAMP '2014-03-14 09:30:00 Europe/Berlin';
+-- 2014-03-14 09:30:00.000 Europe/Berlin
+
+
+

You can also change the time zone of an existing timestamp using the +AT TIME ZONE clause:

+
SELECT TIMESTAMP '2014-03-14 09:30:00 Europe/Berlin'
+     AT TIME ZONE 'America/Los_Angeles';
+-- 2014-03-14 01:30:00.000 America/Los_Angeles
+
+
+

Both timestamps represent the same instant in time; +they differ only in the time zone used to print them.

+

The time zone of the session can be set on a per-query basis using the +X-Presto-Time-Zone HTTP header, or via the +PrestoConnection.setTimeZoneId(String) method in the JDBC driver.

+
+
+

Localization#

+

In addition to time zones, the language of the user is important when +parsing and printing date/time types. This release adds localization +support to the Presto engine and functions that require it: +date_format() and date_parse(). +For example, if we set the language to Spanish:

+
SELECT date_format(TIMESTAMP '2001-01-09 09:04', '%M'); -- enero
+
+
+

If we set the language to Japanese:

+
SELECT date_format(TIMESTAMP '2001-01-09 09:04', '%M'); -- 1月
+
+
+

The language of the session can be set on a per-query basis using the +X-Presto-Language HTTP header, or via the +PrestoConnection.setLocale(Locale) method in the JDBC driver.

+
+
+
+

Optimizations#

+
    +
  • We have upgraded the Hive connector to Hive 0.12 which includes +performance improvements for RCFile.

  • +
  • GROUP BY and JOIN operators are now compiled to byte code +and are significantly faster.

  • +
  • Reduced memory usage of GROUP BY and SELECT DISTINCT, +which previously required several megabytes of memory +per operator, even when the number of groups was small.

  • +
  • The planner now optimizes function call arguments. This should improve +the performance of queries that contain complex expressions.

  • +
  • Fixed a performance regression in the HTTP client. The recent HTTP client +upgrade was using inadvertently GZIP compression and has a bug in the +buffer management resulting in high CPU usage.

  • +
+
+
+

SPI#

+

In this release we have made a number of backward incompatible changes to the SPI:

+
    +
  • Added Type and related interfaces

  • +
  • ConnectorType in metadata has been replaced with Type

  • +
  • Renamed TableHandle to ConnectorTableHandle

  • +
  • Renamed ColumnHandle to ConnectorColumnHandle

  • +
  • Renamed Partition to ConnectorPartition

  • +
  • Renamed PartitionResult to ConnectorPartitionResult

  • +
  • Renamed Split to ConnectorSplit

  • +
  • Renamed SplitSource to ConnectorSplitSource

  • +
  • Added a ConnectorSession parameter to most ConnectorMetadata methods

  • +
  • Removed most canHandle methods

  • +
+
+
+

General bug fixes#

+
    +
  • Fixed CLI hang after using USE CATALOG or USE SCHEMA

  • +
  • Implicit coercions in aggregations now work as expected

  • +
  • Nulls in expressions work as expected

  • +
  • Fixed memory leak in compiler

  • +
  • Fixed accounting bug in task memory usage

  • +
  • Fixed resource leak caused by abandoned queries

  • +
  • Fail queries immediately on unrecoverable data transport errors

  • +
+
+
+

Hive bug fixes#

+
    +
  • Fixed parsing of timestamps in the Hive RCFile Text SerDe (ColumnarSerDe) +by adding configuration to set the time zone originally used when writing data

  • +
+
+
+

Cassandra bug fixes#

+
    +
  • Auto-reconnect if Cassandra session dies

  • +
  • Format collection types as JSON

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.67.html b/430/release/release-0.67.html new file mode 100644 index 000000000..7eb4127de --- /dev/null +++ b/430/release/release-0.67.html @@ -0,0 +1,2559 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.67 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.67#

+
    +
  • Fix resource leak in Hive connector

  • +
  • Improve error categorization in event logging

  • +
  • Fix planning issue with certain queries using window functions

  • +
+
+

SPI#

+

The ConnectorSplitSource interface now extends Closeable.

+
+

Note

+

This is a backwards incompatible change to ConnectorSplitSource in the SPI, +so if you have written a connector, you will need to update your code before +deploying this release.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.68.html b/430/release/release-0.68.html new file mode 100644 index 000000000..e9f09587f --- /dev/null +++ b/430/release/release-0.68.html @@ -0,0 +1,2545 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.68 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.68#

+
    +
  • Fix a regression in the handling of Hive tables that are bucketed on a +string column. This caused queries that could take advantage of bucketing +on such tables to choose the wrong bucket and thus would not match any +rows for the table. This regression was introduced in 0.66.

  • +
  • Fix double counting of bytes and rows when reading records

  • +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.69.html b/430/release/release-0.69.html new file mode 100644 index 000000000..f1581ec66 --- /dev/null +++ b/430/release/release-0.69.html @@ -0,0 +1,2654 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.69 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 0.69#

+
+

Warning

+

The following config properties must be removed from the +etc/config.properties file on both the coordinator and workers:

+
    +
  • presto-metastore.db.type

  • +
  • presto-metastore.db.filename

  • +
+

Additionally, the datasources property is now deprecated and should also be +removed (see Datasource Configuration).

+
+
+

Prevent scheduling work on coordinator#

+

We have a new config property, node-scheduler.include-coordinator, +that allows or disallows scheduling work on the coordinator. +Previously, tasks like final aggregations could be scheduled on the +coordinator. For larger clusters, processing work on the coordinator +can impact query performance because the machine’s resources are not +available for the critical task of scheduling, managing and monitoring +query execution.

+

We recommend setting this property to false for the coordinator. +See Config properties for an example.

+
+
+

Datasource configuration#

+

The datasources config property has been deprecated. +Please remove it from your etc/config.properties file. +The datasources configuration is now automatically generated based +on the node-scheduler.include-coordinator property +(see [Prevent Scheduling Work on Coordinator]).

+
+
+

Raptor connector#

+

Presto has an extremely experimental connector that was previously called +the native connector and was intertwined with the main Presto code +(it was written before Presto had connectors). This connector is now +named raptor and lives in a separate plugin.

+

As part of this refactoring, the presto-metastore.db.type and +presto-metastore.db.filename config properties no longer exist +and must be removed from etc/config.properties.

+

The Raptor connector stores data on the Presto machines in a +columnar format using the same layout that Presto uses for in-memory +data. Currently, it has major limitations: lack of replication, +dropping a table does not reclaim the storage, etc. It is only +suitable for experimentation, temporary tables, caching of data from +slower connectors, etc. The metadata and data formats are subject to +change in incompatible ways between releases.

+

If you would like to experiment with the connector, create a catalog +properties file such as etc/catalog/raptor.properties on both the +coordinator and workers that contains the following:

+
connector.name=raptor
+metadata.db.type=h2
+metadata.db.filename=var/data/db/MetaStore
+
+
+
+
+

Machine learning functions#

+

Presto now has functions to train and use machine learning models +(classifiers and regressors). This is currently only a proof of concept +and is not ready for use in production. Example usage is as follows:

+
SELECT evaluate_classifier_predictions(label, classify(features, model))
+FROM (
+    SELECT learn_classifier(label, features) AS model
+    FROM training_data
+)
+CROSS JOIN validation_data
+
+
+

In the above example, the column label is a bigint and the column +features is a map of feature identifiers to feature values. The feature +identifiers must be integers (encoded as strings because JSON only supports +strings for map keys) and the feature values are numbers (floating point).

+
+
+

Variable length binary type#

+

Presto now supports the varbinary type for variable length binary data. +Currently, the only supported function is length(). +The Hive connector now maps the Hive BINARY type to varbinary.

+
+
+

General#

+
    +
  • Add missing operator: timestamp with time zone - interval year to month

  • +
  • Support explaining sampled queries

  • +
  • Add JMX stats for abandoned and canceled queries

  • +
  • Add javax.inject to parent-first class list for plugins

  • +
  • Improve error categorization in event logging

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.70.html b/430/release/release-0.70.html new file mode 100644 index 000000000..124f3d13c --- /dev/null +++ b/430/release/release-0.70.html @@ -0,0 +1,2653 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.70 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+ +
+
+ +
+
+ +
+

Release 0.70#

+
+

Warning

+

This release contained a packaging error that resulted in an +unusable server tarball. Do not use this release.

+
+
+

Views#

+

We have added support for creating views within Presto. +Views are defined using Presto syntax but are stored (as blobs) +by connectors. Currently, views are supported by the +Raptor and Hive connectors. For the Hive connector, views are +stored within the Hive metastore as Hive views, but they cannot +be queried by Hive, nor can Hive views be queried by Presto.

+

See CREATE VIEW and DROP VIEW +for details and examples.

+
+
+

DUAL table#

+

The synthetic DUAL table is no longer supported. As an alternative, please +write your queries without a FROM clause or use the VALUES syntax.

+
+
+

Presto Verifier#

+

There is a new project, Presto Verifier, which can be used to verify a set of +queries against two different clusters.

+
+
+

Connector improvements#

+
    +
  • Connectors can now add hidden columns to a table. Hidden columns are not +displayed in DESCRIBE or information_schema, and are not +considered for SELECT *. As an example, we have added a hidden +row_number column to the tpch connector.

  • +
  • Presto contains an extensive test suite to verify the correctness. This test +suite has been extracted into the presto-test module for use during +connector development. For an example, see TestRaptorDistributedQueries.

  • +
+
+
+

Machine learning functions#

+

We have added two new machine learning functions, which can be used +by advanced users familiar with LIBSVM. The functions are +learn_libsvm_classifier and learn_libsvm_regressor. Both take a +parameters string which has the form key=value,key=value

+
+
+

General#

+
    +
  • New comparison functions: greatest() and least()

  • +
  • New window functions: first_value(), last_value(), and nth_value()

  • +
  • We have added a config option to disable falling back to the interpreter when +expressions fail to be compiled to bytecode. To set this option, add +compiler.interpreter-enabled=false to etc/config.properties. +This will force certain queries to fail rather than running slowly.

  • +
  • DATE values are now implicitly coerced to TIMESTAMP and TIMESTAMP WITH TIME ZONE +by setting the hour/minute/seconds to 0 with respect to the session timezone.

  • +
  • Minor performance optimization when planning queries over tables with tens of +thousands of partitions or more.

  • +
  • Fixed a bug when planning ORDER BY ... LIMIT queries which could result in +duplicate and un-ordered results under rare conditions.

  • +
  • Reduce the size of stats collected from tasks, which dramatically reduces +garbage generation and improves coordinator stability.

  • +
  • Fix compiler cache for expressions.

  • +
  • Fix processing of empty or commented out statements in the CLI.

  • +
+
+
+

Hive#

+
    +
  • There are two new configuration options for the Hive connector, +hive.max-initial-split-size, which configures the size of the +initial splits, and hive.max-initial-splits, which configures +the number of initial splits. This can be useful for speeding up small +queries, which would otherwise have low parallelism.

  • +
  • The Hive connector will now consider all tables with a non-empty value +for the table property presto_offline to be offline. The value of the +property will be used in the error message.

  • +
  • We have added support for DROP TABLE in the hive connector. +By default, this feature is not enabled. To enable it, set +hive.allow-drop-table=true in your Hive catalog properties file.

  • +
  • Ignore subdirectories when generating splits +(this now matches the non-recursive behavior of Hive).

  • +
  • Fix handling of maps with null keys.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.71.html b/430/release/release-0.71.html new file mode 100644 index 000000000..febd8c68a --- /dev/null +++ b/430/release/release-0.71.html @@ -0,0 +1,2543 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.71 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.71#

+
    +
  • Fix packaging issue that resulted in an unusable server tarball +for the 0.70 release

  • +
  • Fix logging in Hive connector when using Amazon S3

  • +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.72.html b/430/release/release-0.72.html new file mode 100644 index 000000000..f670c624c --- /dev/null +++ b/430/release/release-0.72.html @@ -0,0 +1,2542 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.72 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.72#

+
    +
  • Fix infinite loop bug in Hive RCFile reader when decoding a Map +with a null key

  • +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.73.html b/430/release/release-0.73.html new file mode 100644 index 000000000..ec0b43717 --- /dev/null +++ b/430/release/release-0.73.html @@ -0,0 +1,2561 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.73 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.73#

+
+

Cassandra plugin#

+

The Cassandra connector now supports CREATE TABLE and DROP TABLE. Additionally, +the connector now takes into account Cassandra indexes when generating CQL. +This release also includes several bug fixes and performance improvements.

+
+
+

General#

+ +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.74.html b/430/release/release-0.74.html new file mode 100644 index 000000000..b3e1c9c14 --- /dev/null +++ b/430/release/release-0.74.html @@ -0,0 +1,2580 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.74 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.74#

+
+

Bytecode compiler#

+

This version includes new infrastructure for bytecode compilation, and lays the groundwork for future improvements. +There should be no impact in performance or correctness with the new code, but we have added a flag to revert to the +old implementation in case of issues. To do so, add compiler.new-bytecode-generator-enabled=false to +etc/config.properties in the coordinator and workers.

+
+
+

Hive storage format#

+

The storage format to use when writing data to Hive can now be configured via the hive.storage-format option +in your Hive catalog properties file. Valid options are RCBINARY, RCTEXT, SEQUENCEFILE and TEXTFILE. +The default format if the property is not set is RCBINARY.

+
+
+

General#

+
    +
  • Show column comments in DESCRIBE

  • +
  • Add try_cast() which works like cast() but returns null if the cast fails

  • +
  • nullif now correctly returns a value with the type of the first argument

  • +
  • Fix an issue with timezone_hour() returning results in milliseconds instead of hours

  • +
  • Show a proper error message when analyzing queries with non-equijoin clauses

  • +
  • Improve “too many failures” error message when coordinator can’t talk to workers

  • +
  • Minor optimization of json_size() function

  • +
  • Improve feature normalization algorithm for machine learning functions

  • +
  • Add exponential back-off to the S3 FileSystem retry logic

  • +
  • Improve CPU efficiency of semi-joins

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.75.html b/430/release/release-0.75.html new file mode 100644 index 000000000..46402b38d --- /dev/null +++ b/430/release/release-0.75.html @@ -0,0 +1,2667 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.75 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.75#

+
+

Hive#

+
    +
  • The Hive S3 file system has a new configuration option, +hive.s3.max-connections, which sets the maximum number of +connections to S3. The default has been increased from 50 to 500.

  • +
  • The Hive connector now supports renaming tables. By default, this feature +is not enabled. To enable it, set hive.allow-rename-table=true in +your Hive catalog properties file.

  • +
+
+
+

General#

+
    +
  • Optimize count() with a constant to execute as the much faster count(*)

  • +
  • Add support for binary types to the JDBC driver

  • +
  • The legacy byte code compiler has been removed

  • +
  • New aggregation framework (~10% faster)

  • +
  • Added max_by() aggregation function

  • +
  • The approx_avg() function has been removed. Use avg() instead.

  • +
  • Fixed parsing of UNION queries that use both DISTINCT and ALL

  • +
  • Fixed cross join planning error for certain query shapes

  • +
  • Added hex and base64 conversion functions for varbinary

  • +
  • Fix the LIKE operator to correctly match against values that contain +multiple lines. Previously, it would stop matching at the first newline.

  • +
  • Add support for renaming tables using the ALTER TABLE statement.

  • +
  • Add basic support for inserting data using the INSERT statement. +This is currently only supported for the Raptor connector.

  • +
+
+
+

JSON function#

+

The json_extract() and json_extract_scalar() functions now support +the square bracket syntax:

+
SELECT json_extract(json, '$.store[book]');
+SELECT json_extract(json, '$.store["book name"]');
+
+
+

As part of this change, the set of characters allowed in a non-bracketed +path segment has been restricted to alphanumeric, underscores and colons. +Additionally, colons cannot be used in a un-quoted bracketed path segment. +Use the new bracket syntax with quotes to match elements that contain +special characters.

+
+
+

Scheduler#

+

The scheduler now assigns splits to a node based on the current load on the node across all queries. +Previously, the scheduler load balanced splits across nodes on a per query level. Every node can have +node-scheduler.max-splits-per-node splits scheduled on it. To avoid starvation of small queries, +when the node already has the maximum allowable splits, every task can schedule at most +node-scheduler.max-pending-splits-per-node-per-task splits on the node.

+
+
+

Row number optimizations#

+

Queries that use the row_number() function are substantially faster +and can run on larger result sets for two types of queries.

+

Performing a partitioned limit that choses N arbitrary rows per +partition is a streaming operation. The following query selects +five arbitrary rows from orders for each orderstatus:

+
SELECT * FROM (
+    SELECT row_number() OVER (PARTITION BY orderstatus) AS rn,
+        custkey, orderdate, orderstatus
+    FROM orders
+) WHERE rn <= 5;
+
+
+

Performing a partitioned top-N that chooses the maximum or minimum +N rows from each partition now uses significantly less memory. +The following query selects the five oldest rows based on orderdate +from orders for each orderstatus:

+
SELECT * FROM (
+    SELECT row_number() OVER (PARTITION BY orderstatus ORDER BY orderdate) AS rn,
+        custkey, orderdate, orderstatus
+    FROM orders
+) WHERE rn <= 5;
+
+
+

Use the EXPLAIN statement to see if any of these optimizations +have been applied to your query.

+
+
+

SPI#

+

The core Presto engine no longer automatically adds a column for count(*) +queries. Instead, the RecordCursorProvider will receive an empty list of +column handles.

+

The Type and Block APIs have gone through a major refactoring in this +release. The main focus of the refactoring was to consolidate all type specific +encoding logic in the type itself, which makes types much easier to implement. +You should consider Type and Block to be a beta API as we expect +further changes in the near future.

+

To simplify the API, ConnectorOutputHandleResolver has been merged into +ConnectorHandleResolver. Additionally, ConnectorHandleResolver, +ConnectorRecordSinkProvider and ConnectorMetadata were modified to +support inserts.

+
+

Note

+

This is a backwards incompatible change with the previous connector and +type SPI, so if you have written a connector or type, you will need to update +your code before deploying this release. In particular, make sure your +connector can handle an empty column handles list (this can be verified +by running SELECT count(*) on a table from your connector).

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.76.html b/430/release/release-0.76.html new file mode 100644 index 000000000..155dadc0c --- /dev/null +++ b/430/release/release-0.76.html @@ -0,0 +1,2622 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.76 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.76#

+
+

Kafka connector#

+

This release adds a connector that allows querying of Apache Kafka topic data +from Presto. Topics can be live and repeated queries will pick up new data.

+

Apache Kafka 0.8+ is supported although Apache Kafka 0.8.1+ is recommended. +There is extensive documentation about configuring +the connector and a tutorial to get started.

+
+
+

MySQL and PostgreSQL connectors#

+

This release adds the MySQL connector and PostgreSQL connector +for querying and creating tables in external relational databases. These can +be used to join or copy data between different systems like MySQL and Hive, +or between two different MySQL or PostgreSQL instances, or any combination.

+
+
+

Cassandra#

+

The Cassandra connector configuration properties +cassandra.client.read-timeout and cassandra.client.connect-timeout +are now specified using a duration rather than milliseconds (this makes +them consistent with all other such properties in Presto). If you were +previously specifying a value such as 25, change it to 25ms.

+

The retry policy for the Cassandra client is now configurable via the +cassandra.retry-policy property. In particular, the custom BACKOFF +retry policy may be useful.

+
+
+

Hive#

+

The new Hive connector configuration property hive.s3.socket-timeout +allows changing the socket timeout for queries that read or write to Amazon S3. +Additionally, the previously added hive.s3.max-connections property +was not respected and always used the default of 500.

+

Hive allows the partitions in a table to have a different schema than the +table. In particular, it allows changing the type of a column without +changing the column type of existing partitions. The Hive connector does +not support this and could previously return garbage data for partitions +stored using the RCFile Text format if the column type was converted from +a non-numeric type such as STRING to a numeric type such as BIGINT +and the actual data in existing partitions was not numeric. The Hive +connector now detects this scenario and fails the query after the +partition metadata has been read.

+

The property hive.storage-format is broken and has been disabled. It +sets the storage format on the metadata but always writes the table using +RCBINARY. This will be implemented in a future release.

+
+
+

General#

+
    +
  • Fix hang in verifier when an exception occurs.

  • +
  • Fix chr() function to work with Unicode code points instead of ASCII code points.

  • +
  • The JDBC driver no longer hangs the JVM on shutdown (all threads are daemon threads).

  • +
  • Fix incorrect parsing of function arguments.

  • +
  • The bytecode compiler now caches generated code for join and group byqueries, +which should improve performance and CPU efficiency for these types of queries.

  • +
  • Improve planning performance for certain trivial queries over tables with lots of partitions.

  • +
  • Avoid creating large output pages. This should mitigate some cases of +“Remote page is too large” errors.

  • +
  • The coordinator/worker communication layer is now fully asynchronous. +Specifically, long-poll requests no longer tie up a thread on the worker. +This makes heavily loaded clusters more efficient.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.77.html b/430/release/release-0.77.html new file mode 100644 index 000000000..43efc4331 --- /dev/null +++ b/430/release/release-0.77.html @@ -0,0 +1,2609 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.77 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.77#

+
+

Parametric types#

+

Presto now has a framework for implementing parametric types and functions. +Support for ARRAY and MAP types has been added, including the element accessor +operator [], and new Array functions and operators.

+
+
+

Streaming index joins#

+

Index joins will now switch to use a key-by-key streaming join if index +results fail to fit in the allocated index memory space.

+
+
+

Distributed joins#

+

Joins where both tables are distributed are now supported. This allows larger tables to be joined, +and can be enabled with the distributed-joins-enabled flag. It may perform worse than the existing +broadcast join implementation because it requires redistributing both tables. +This feature is still experimental, and should be used with caution.

+
+
+

Hive#

+
    +
  • Handle spurious AbortedException when closing S3 input streams

  • +
  • Add support for ORC, DWRF and Parquet in Hive

  • +
  • Add support for DATE type in Hive

  • +
  • Fix performance regression in Hive when reading VARCHAR columns

  • +
+
+
+

Kafka#

+
    +
  • Fix Kafka handling of default port

  • +
  • Add support for Kafka messages with a null key

  • +
+
+
+

General#

+
    +
  • Fix race condition in scheduler that could cause queries to hang

  • +
  • Add ConnectorPageSource which is a more efficient interface for column-oriented sources

  • +
  • Add support for string partition keys in Cassandra

  • +
  • Add support for variable arity functions

  • +
  • Add support for count() for all types

  • +
  • Fix bug in HashAggregation that could cause the operator to go in an infinite loop

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.78.html b/430/release/release-0.78.html new file mode 100644 index 000000000..97fa0bc10 --- /dev/null +++ b/430/release/release-0.78.html @@ -0,0 +1,2609 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.78 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.78#

+
+

ARRAY and MAP types in Hive connector#

+

The Hive connector now returns arrays and maps instead of json encoded strings, +for columns whose underlying type is array or map. Please note that this is a backwards +incompatible change, and the JSON functions and operators will no longer work on these columns, +unless you cast() them to the json type.

+
+
+

Session properties#

+

The Presto session can now contain properties, which can be used by the Presto +engine or connectors to customize the query execution. There is a separate +namespace for the Presto engine and each catalog. A property for a catalog is +simplify prefixed with the catalog name followed by . (dot). A connector +can retrieve the properties for the catalog using +ConnectorSession.getProperties().

+

Session properties can be set using the --session command line argument to +the Presto CLI. For example:

+
presto-cli --session color=red --session size=large
+
+
+

For JDBC, the properties can be set by unwrapping the Connection as follows:

+
connection.unwrap(PrestoConnection.class).setSessionProperty("name", "value");
+
+
+
+

Note

+

This feature is a work in progress and will change in a future release. +Specifically, we are planning to require preregistration of properties so +the user can list available session properties and so the engine can verify +property values. Additionally, the Presto grammar will be extended to +allow setting properties via a query.

+
+
+
+

Hive#

+
    +
  • Add storage_format session property to override format used for creating tables.

  • +
  • Add write support for VARBINARY, DATE and TIMESTAMP.

  • +
  • Add support for partition keys of type TIMESTAMP.

  • +
  • Add support for partition keys with null values (__HIVE_DEFAULT_PARTITION__).

  • +
  • Fix hive.storage-format option (see Release 0.76).

  • +
+
+
+

General#

+
    +
  • Fix expression optimizer, so that it runs in linear time instead of exponential time.

  • +
  • Add cardinality() for maps.

  • +
  • Fix race condition in SqlTask creation which can cause queries to hang.

  • +
  • Fix node-scheduler.multiple-tasks-per-node-enabled option.

  • +
  • Fix an exception when planning a query with a UNION under a JOIN.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.79.html b/430/release/release-0.79.html new file mode 100644 index 000000000..c03fd85bf --- /dev/null +++ b/430/release/release-0.79.html @@ -0,0 +1,2567 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.79 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.79#

+
+

Hive#

+
    +
  • Add configuration option hive.force-local-scheduling and session property +force_local_scheduling to force local scheduling of splits.

  • +
  • Add new experimental optimized RCFile reader. The reader can be enabled by +setting the configuration option hive.optimized-reader.enabled or session +property optimized_reader_enabled.

  • +
+
+
+

General#

+
    +
  • Add support for UNNEST, which can be used as a replacement for the explode() function in Hive.

  • +
  • Fix a bug in the scan operator that can cause data to be missed. It currently only affects queries +over information_schema or sys tables, metadata queries such as SHOW PARTITIONS and connectors +that implement the ConnectorPageSource interface.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.80.html b/430/release/release-0.80.html new file mode 100644 index 000000000..37657c3da --- /dev/null +++ b/430/release/release-0.80.html @@ -0,0 +1,2659 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.80 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 0.80#

+
+

New Hive ORC reader#

+

We have added a new ORC reader implementation. The new reader supports vectorized +reads, lazy loading, and predicate push down, all of which make the reader more +efficient and typically reduces wall clock time for a query. Although the new +reader has been heavily tested, it is an extensive rewrite of the Apache Hive +ORC reader, and may have some latent issues. If you are seeing issues, you can +disable the new reader on a per-query basis by setting the +<hive-catalog>.optimized_reader_enabled session property, or you can disable +the reader by default by setting the Hive catalog property +hive.optimized-reader.enabled=false.

+
+
+

Hive#

+
    +
  • The maximum retry time for the Hive S3 file system can be configured +by setting hive.s3.max-retry-time.

  • +
  • Fix Hive partition pruning for null keys (i.e. __HIVE_DEFAULT_PARTITION__).

  • +
+
+
+

Cassandra#

+
    +
  • Update Cassandra driver to 2.1.0.

  • +
  • Map Cassandra TIMESTAMP type to Presto TIMESTAMP type.

  • +
+
+
+

“Big Query” support#

+

We’ve added experimental support for “big” queries. This provides a separate +queue controlled by the following properties:

+
    +
  • experimental.max-concurrent-big-queries

  • +
  • experimental.max-queued-big-queries

  • +
+

There are separate configuration options for queries that are submitted with +the experimental_big_query session property:

+
    +
  • experimental.big-query-initial-hash-partitions

  • +
  • experimental.big-query-max-task-memory

  • +
+

Queries submitted with this property will use hash distribution for all joins.

+
+
+

Metadata-only query optimization#

+

We now support an optimization that rewrites aggregation queries that are insensitive to the +cardinality of the input (e.g., max(), min(), DISTINCT aggregates) to execute +against table metadata.

+

For example, if key, key1 and key2 are partition keys, the following queries +will benefit:

+
SELECT min(key), max(key) FROM t;
+
+SELECT DISTINCT key FROM t;
+
+SELECT count(DISTINCT key) FROM t;
+
+SELECT count(DISTINCT key + 5) FROM t;
+
+SELECT count(DISTINCT key) FROM (SELECT key FROM t ORDER BY 1 LIMIT 10);
+
+SELECT key1, count(DISTINCT key2) FROM t GROUP BY 1;
+
+
+

This optimization is turned off by default. To turn it on, add optimizer.optimize-metadata-queries=true +to the coordinator config properties.

+
+

Warning

+

This optimization will cause queries to produce incorrect results if +the connector allows partitions to contain no data. For example, the +Hive connector will produce incorrect results if your Hive warehouse +contains partitions without data.

+
+
+
+

General#

+
    +
  • Add support implicit joins. The following syntax is now allowed:

    +
    SELECT * FROM a, b WHERE a.id = b.id;
    +
    +
    +
  • +
  • Add property task.verbose-stats to enable verbose statistics collection for +tasks. The default is false.

  • +
  • Format binary data in the CLI as a hex dump.

  • +
  • Add approximate numeric histogram function numeric_histogram().

  • +
  • Add array_sort() function.

  • +
  • Add map_keys() and map_values() functions.

  • +
  • Make row_number() completely streaming.

  • +
  • Add property task.max-partial-aggregation-memory to configure the memory limit +for the partial step of aggregations.

  • +
  • Fix exception when processing queries with an UNNEST operation where the output was not used.

  • +
  • Only show query progress in UI after the query has been fully scheduled.

  • +
  • Add query execution visualization to the coordinator UI. It can be accessed via the query details page.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.81.html b/430/release/release-0.81.html new file mode 100644 index 000000000..6282effac --- /dev/null +++ b/430/release/release-0.81.html @@ -0,0 +1,2562 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.81 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.81#

+
+

Hive#

+
    +
  • Fix ORC predicate pushdown.

  • +
  • Fix column selection in RCFile.

  • +
+
+
+

General#

+ +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.82.html b/430/release/release-0.82.html new file mode 100644 index 000000000..edf5340ca --- /dev/null +++ b/430/release/release-0.82.html @@ -0,0 +1,2546 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.82 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.82#

+
    +
  • Presto now supports the ROW type, and all Hive structs are +converted to ROWs, instead of JSON encoded VARCHARs.

  • +
  • Add current_timezone() function.

  • +
  • Improve planning performance for queries with thousands of columns.

  • +
  • Fix a regression that was causing excessive memory allocation and GC pressure +in the coordinator.

  • +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.83.html b/430/release/release-0.83.html new file mode 100644 index 000000000..a1b99990a --- /dev/null +++ b/430/release/release-0.83.html @@ -0,0 +1,2568 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.83 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.83#

+
+

Raptor#

+
    +
  • Raptor now enables specifying the backup storage location. This feature is highly experimental.

  • +
  • Fix the handling of shards not assigned to any node.

  • +
+
+
+

General#

+
    +
  • Fix resource leak in query queues.

  • +
  • Fix NPE when writing null ARRAY/MAP to Hive.

  • +
  • Fix json_array_get() to handle nested structures.

  • +
  • Fix UNNEST on null collections.

  • +
  • Fix a regression where queries that fail during parsing or analysis do not expire.

  • +
  • Make JSON type comparable.

  • +
  • Added an optimization for hash aggregations. This optimization is turned off by default. +To turn it on, add optimizer.optimize-hash-generation=true to the coordinator config properties.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.84.html b/430/release/release-0.84.html new file mode 100644 index 000000000..3de2d18d2 --- /dev/null +++ b/430/release/release-0.84.html @@ -0,0 +1,2546 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.84 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.84#

+
    +
  • Fix handling of NaN and infinity in ARRAYs

  • +
  • Fix approximate queries that use JOIN

  • +
  • Reduce excessive memory allocation and GC pressure in the coordinator

  • +
  • Fix an issue where setting node-scheduler.location-aware-scheduling-enabled=false +would cause queries to fail for connectors whose splits were not remotely accessible

  • +
  • Fix error when running COUNT(*) over tables in information_schema and sys

  • +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.85.html b/430/release/release-0.85.html new file mode 100644 index 000000000..c09f92004 --- /dev/null +++ b/430/release/release-0.85.html @@ -0,0 +1,2542 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.85 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.85#

+
    +
  • Improve query planning performance for tables with large numbers of partitions.

  • +
  • Fix issue when using JSON values in GROUP BY expressions.

  • +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.86.html b/430/release/release-0.86.html new file mode 100644 index 000000000..4c4eac3d5 --- /dev/null +++ b/430/release/release-0.86.html @@ -0,0 +1,2569 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.86 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.86#

+
+

General#

+
    +
  • Add support for inequality INNER JOIN when each term of the condition refers to only one side of the join.

  • +
  • Add ntile() function.

  • +
  • Add map() function to create a map from arrays of keys and values.

  • +
  • Add min_by() aggregation function.

  • +
  • Add support for concatenating arrays with the || operator.

  • +
  • Add support for = and != to JSON type.

  • +
  • Improve error message when DISTINCT is applied to types that are not comparable.

  • +
  • Perform type validation for IN expression where the right-hand side is a subquery expression.

  • +
  • Improve error message when ORDER BY ... LIMIT query exceeds its maximum memory allocation.

  • +
  • Improve error message when types that are not orderable are used in an ORDER BY clause.

  • +
  • Improve error message when the types of the columns for subqueries of a UNION query don’t match.

  • +
  • Fix a regression where queries could be expired too soon on a highly loaded cluster.

  • +
  • Fix scheduling issue for queries involving tables from information_schema, which could result in +inconsistent metadata.

  • +
  • Fix an issue with min_by() and max_by() that could result in an error when used with +a variable-length type (e.g., VARCHAR) in a GROUP BY query.

  • +
  • Fix rendering of array attributes in JMX connector.

  • +
  • Input rows/bytes are now tracked properly for JOIN queries.

  • +
  • Fix case-sensitivity issue when resolving names of constant table expressions.

  • +
  • Fix unnesting arrays and maps that contain the ROW type.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.87.html b/430/release/release-0.87.html new file mode 100644 index 000000000..a10b2866e --- /dev/null +++ b/430/release/release-0.87.html @@ -0,0 +1,2551 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.87 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.87#

+
+

General#

+
    +
  • Fixed a bug where ROW types could have the wrong field names.

  • +
  • Changed the minimum JDK version to 1.8.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.88.html b/430/release/release-0.88.html new file mode 100644 index 000000000..ed8fab2ae --- /dev/null +++ b/430/release/release-0.88.html @@ -0,0 +1,2560 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.88 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.88#

+
+

General#

+
    +
  • Added arbitrary() aggregation function.

  • +
  • Allow using all Aggregate functions as Window functions.

  • +
  • Support specifying window frames and correctly implement frames for all Window functions.

  • +
  • Allow approx_distinct() aggregation function to accept a standard error parameter.

  • +
  • Implement least() and greatest() with variable number of arguments.

  • +
  • ARRAY is now comparable and can be used as GROUP BY keys or in ORDER BY expressions.

  • +
  • Implement = and <> operators for ROW.

  • +
  • Fix excessive garbage creation in the ORC reader.

  • +
  • Fix an issue that could cause queries using row_number() and LIMIT to never terminate.

  • +
  • Fix an issue that could cause queries with row_number() and specific filters to produce incorrect results.

  • +
  • Fixed an issue that caused the Cassandra plugin to fail to load with a SecurityException.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.89.html b/430/release/release-0.89.html new file mode 100644 index 000000000..a6aeb123e --- /dev/null +++ b/430/release/release-0.89.html @@ -0,0 +1,2568 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.89 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.89#

+
+

DATE type#

+

The memory representation of dates is now the number of days since January 1, 1970 +using a 32-bit signed integer.

+
+

Note

+

This is a backwards incompatible change with the previous date +representation, so if you have written a connector, you will need to update +your code before deploying this release.

+
+
+
+

General#

+
    +
  • USE CATALOG and USE SCHEMA have been replaced with USE.

  • +
  • Fix issue where SELECT NULL incorrectly returns 0 rows.

  • +
  • Fix rare condition where JOIN queries could produce incorrect results.

  • +
  • Fix issue where UNION queries involving complex types would fail during planning.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.90.html b/430/release/release-0.90.html new file mode 100644 index 000000000..270ee5410 --- /dev/null +++ b/430/release/release-0.90.html @@ -0,0 +1,2623 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.90 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.90#

+
+

Warning

+

This release has a memory leak and should not be used.

+
+
+

General#

+
    +
  • Initial support for partition and placement awareness in the query planner. This can +result in better plans for queries involving JOIN and GROUP BY over the same +key columns.

  • +
  • Improve planning of UNION queries.

  • +
  • Add presto version to query creation and completion events.

  • +
  • Add property task.writer-count to configure the number of writers per task.

  • +
  • Fix a bug when optimizing constant expressions involving binary types.

  • +
  • Fix bug where a table writer commits partial results while cleaning up a failed query.

  • +
  • Fix a bug when unnesting an array of doubles containing NaN or Infinity.

  • +
  • Fix failure when accessing elements in an empty array.

  • +
  • Fix “Remote page is too large” errors.

  • +
  • Improve error message when attempting to cast a value to UNKNOWN.

  • +
  • Update the approx_distinct() documentation with correct standard error bounds.

  • +
  • Disable falling back to the interpreter when expressions fail to be compiled +to bytecode. To enable this option, add compiler.interpreter-enabled=true +to the coordinator and worker config properties. Enabling this option will +allow certain queries to run slowly rather than failing.

  • +
  • Improve JDBC driver conformance. In particular, all unimplemented +methods now throw SQLException rather than UnsupportedOperationException.

  • +
+
+
+

Functions and language features#

+ +
+
+

Hive#

+
    +
  • Disable optimized metastore partition fetching for non-string partition keys. +This fixes an issue were Presto might silently ignore data with non-canonical +partition values. To enable this option, add hive.assume-canonical-partition-keys=true +to the coordinator and worker config properties.

  • +
  • Don’t retry operations against S3 that fail due to lack of permissions.

  • +
+
+
+

SPI#

+
    +
  • Add getColumnTypes to RecordSink.

  • +
  • Use Slice for table writer fragments.

  • +
  • Add ConnectorPageSink which is a more efficient interface for column-oriented sources.

  • +
+
+

Note

+

This is a backwards incompatible change with the previous connector SPI. +If you have written a connector, you will need to update your code +before deploying this release.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.91.html b/430/release/release-0.91.html new file mode 100644 index 000000000..4e69caa55 --- /dev/null +++ b/430/release/release-0.91.html @@ -0,0 +1,2554 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.91 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.91#

+
+

Warning

+

This release has a memory leak and should not be used.

+
+
+

General#

+
    +
  • Clear LazyBlockLoader reference after load to free memory earlier.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.92.html b/430/release/release-0.92.html new file mode 100644 index 000000000..15956201c --- /dev/null +++ b/430/release/release-0.92.html @@ -0,0 +1,2550 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.92 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.92#

+
+

General#

+
    +
  • Fix buffer leak when a query fails.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.93.html b/430/release/release-0.93.html new file mode 100644 index 000000000..3f2827c1d --- /dev/null +++ b/430/release/release-0.93.html @@ -0,0 +1,2588 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.93 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.93#

+
+

ORC memory usage#

+

This release changes the Presto ORC reader to favor small buffers when reading +varchar and varbinary data. Some ORC files contain columns of data that are +hundreds of megabytes when decompressed. In the previous Presto ORC reader, we +would allocate a single large shared buffer for all values in the column. This +would cause heap fragmentation in CMS and G1, and it would cause OOMs since +each value of the column retains a reference to the shared buffer. In this +release the ORC reader uses a separate buffer for each value in the column. +This reduces heap fragmentation and excessive memory retention at the expense +of object creation.

+
+
+

Verifier#

+
    +
  • Add support for setting username and password per query

  • +
+

If you’re upgrading from 0.92, you need to alter your verifier_queries table

+
ALTER TABLE verifier_queries add test_username VARCHAR(256) NOT NULL default 'verifier-test';
+ALTER TABLE verifier_queries add test_password VARCHAR(256);
+ALTER TABLE verifier_queries add control_username VARCHAR(256) NOT NULL default 'verifier-test';
+ALTER TABLE verifier_queries add control_password VARCHAR(256);
+
+
+
+
+

General#

+
    +
  • Add optimizer for LIMIT 0

  • +
  • Fix incorrect check to disable string statistics in ORC

  • +
  • Ignore hidden columns in INSERT and CREATE TABLE AS queries

  • +
  • Add SOCKS support to CLI

  • +
  • Improve CLI output for update queries

  • +
  • Disable pushdown for non-deterministic predicates

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.94.html b/430/release/release-0.94.html new file mode 100644 index 000000000..48803e5fd --- /dev/null +++ b/430/release/release-0.94.html @@ -0,0 +1,2570 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.94 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.94#

+
+

ORC memory usage#

+

This release contains additional changes to the Presto ORC reader to favor +small buffers when reading varchar and varbinary data. Some ORC files contain +columns of data that are hundreds of megabytes compressed. When reading these +columns, Presto would allocate a single buffer for the compressed column data, +and this would cause heap fragmentation in CMS and G1 and eventually OOMs. +In this release, the hive.orc.max-buffer-size sets the maximum size for a +single ORC buffer, and for larger columns we instead stream the data. This +reduces heap fragmentation and excessive buffers in ORC at the expense of +HDFS IOPS. The default value is 8MB.

+
+
+

General#

+
    +
  • Update Hive CDH 4 connector to CDH 4.7.1

  • +
  • Fix ORDER BY with LIMIT 0

  • +
  • Fix compilation of try_cast

  • +
  • Group threads into Java thread groups to ease debugging

  • +
  • Add task.min-drivers config to help limit number of concurrent readers

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.95.html b/430/release/release-0.95.html new file mode 100644 index 000000000..281da1e69 --- /dev/null +++ b/430/release/release-0.95.html @@ -0,0 +1,2550 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.95 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.95#

+
+

General#

+
    +
  • Fix task and stage leak, caused when a stage finishes before its substages.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.96.html b/430/release/release-0.96.html new file mode 100644 index 000000000..de73e8c91 --- /dev/null +++ b/430/release/release-0.96.html @@ -0,0 +1,2569 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.96 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.96#

+
+

General#

+
    +
  • Fix try_cast() for TIMESTAMP and other types that +need access to session information.

  • +
  • Fix planner bug that could result in incorrect results for +tables containing columns with the same prefix, underscores and numbers.

  • +
  • MAP type is now comparable.

  • +
  • Fix output buffer leak in StatementResource.Query.

  • +
  • Fix leak in SqlTasks caused by invalid heartbeats .

  • +
  • Fix double logging of queries submitted while the queue is full.

  • +
  • Fixed “running queries” JMX stat.

  • +
  • Add distributed_join session property to enable/disable distributed joins.

  • +
+
+
+

Hive#

+
    +
  • Add support for tables partitioned by DATE.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.97.html b/430/release/release-0.97.html new file mode 100644 index 000000000..efda0799f --- /dev/null +++ b/430/release/release-0.97.html @@ -0,0 +1,2567 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.97 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.97#

+
+

General#

+
    +
  • The queueing policy in Presto may now be injected.

  • +
  • Speed up detection of ASCII strings in implementation of LIKE operator.

  • +
  • Fix NullPointerException when metadata-based query optimization is enabled.

  • +
  • Fix possible infinite loop when decompressing ORC data.

  • +
  • Fix an issue where NOT clause was being ignored in NOT BETWEEN predicates.

  • +
  • Fix a planning issue in queries that use SELECT *, window functions and implicit coercions.

  • +
  • Fix scheduler deadlock for queries with a UNION between VALUES and SELECT.

  • +
+
+
+

Hive#

+
    +
  • Fix decoding of STRUCT type from Parquet files.

  • +
  • Speed up decoding of ORC files with very small stripes.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.98.html b/430/release/release-0.98.html new file mode 100644 index 000000000..bd6896410 --- /dev/null +++ b/430/release/release-0.98.html @@ -0,0 +1,2592 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.98 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.98#

+
+

Array, map, and row types#

+

The memory representation of these types is now VariableWidthBlockEncoding +instead of JSON.

+
+

Note

+

This is a backwards incompatible change with the previous representation, +so if you have written a connector or function, you will need to update +your code before deploying this release.

+
+
+
+

Hive#

+
    +
  • Fix handling of ORC files with corrupt checkpoints.

  • +
+
+
+

SPI#

+
    +
  • Rename Index to ConnectorIndex.

  • +
+
+

Note

+

This is a backwards incompatible change, so if you have written a connector +that uses Index, you will need to update your code before deploying this release.

+
+
+
+

General#

+
    +
  • Fix bug in UNNEST when output is unreferenced or partially referenced.

  • +
  • Make max() and min() functions work on all orderable types.

  • +
  • Optimize memory allocation in max_by() and other places that Block is used.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-0.99.html b/430/release/release-0.99.html new file mode 100644 index 000000000..a35f3498e --- /dev/null +++ b/430/release/release-0.99.html @@ -0,0 +1,2553 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 0.99 — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 0.99#

+
+

General#

+
    +
  • Reduce lock contention in TaskExecutor.

  • +
  • Fix reading maps with null keys from ORC.

  • +
  • Fix precomputed hash optimization for nulls values.

  • +
  • Make contains() work for all comparable types.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-300.html b/430/release/release-300.html new file mode 100644 index 000000000..2573a9ef5 --- /dev/null +++ b/430/release/release-300.html @@ -0,0 +1,2726 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 300 (22 Jan 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 300 (22 Jan 2019)#

+
+

General#

+
    +
  • Fix array_intersect() and array_distinct() +skipping zeros when input also contains nulls.

  • +
  • Fix count(*) aggregation returning null on empty relation +when optimize_mixed_distinct_aggregation is enabled.

  • +
  • Improve table scan performance for structured types.

  • +
  • Improve performance for array_intersect().

  • +
  • Improve performance of window functions by filtering partitions early.

  • +
  • Add reduce_agg() aggregate function.

  • +
  • Add millisecond() function.

  • +
  • Remove ON keyword from SHOW STATS (use FOR instead).

  • +
  • Restrict WHERE clause in SHOW STATS +to filters that can be pushed down to connectors.

  • +
  • Return final results to clients immediately for failed queries.

  • +
+
+
+

JMX MBean naming#

+
    +
  • The base domain name for server MBeans is now presto. The old names can be +used by setting the configuration property jmx.base-name to com.facebook.presto.

  • +
  • The base domain name for the Hive, Raptor, and Thrift connectors is presto.plugin. +The old names can be used by setting the catalog configuration property +jmx.base-name to com.facebook.presto.hive, com.facebook.presto.raptor, +or com.facebook.presto.thrift, respectively.

  • +
+
+
+

Web UI#

+
    +
  • Fix rendering of live plan view for queries involving index joins.

  • +
+
+
+

JDBC driver#

+
    +
  • Change driver class name to io.prestosql.jdbc.PrestoDriver.

  • +
+
+
+

System connector#

+
    +
  • Remove node_id column from system.runtime.queries table.

  • +
+
+
+

Hive connector#

+
    +
  • Fix accounting of time spent reading Parquet data.

  • +
  • Fix corner case where the ORC writer fails with integer overflow when writing +highly compressible data using dictionary encoding (#11930).

  • +
  • Fail queries reading Parquet files if statistics in those files are corrupt +(e.g., min > max). To disable this behavior, set the configuration +property hive.parquet.fail-on-corrupted-statistics +or session property parquet_fail_with_corrupted_statistics to false.

  • +
  • Add support for S3 Select pushdown, which enables pushing down +column selection and range filters into S3 for text files.

  • +
+
+
+

Kudu connector#

+
    +
  • Add number_of_replicas table property to SHOW CREATE TABLE output.

  • +
+
+
+

Cassandra connector#

+
    +
  • Add cassandra.splits-per-node and cassandra.protocol-version configuration +properties to allow connecting to Cassandra servers older than 2.1.5.

  • +
+
+
+

MySQL connector#

+
    +
  • Add support for predicate pushdown for columns of char(x) type.

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Add support for predicate pushdown for columns of char(x) type.

  • +
+
+
+

Redshift connector#

+
    +
  • Add support for predicate pushdown for columns of char(x) type.

  • +
+
+
+

SQL Server connector#

+
    +
  • Add support for predicate pushdown for columns of char(x) type.

  • +
+
+
+

Raptor Legacy connector#

+
    +
  • Change name of connector to raptor-legacy.

  • +
+
+
+

Verifier#

+
    +
  • Add run-teardown-on-result-mismatch configuration property to facilitate debugging. +When set to false, temporary tables will not be dropped after checksum failures.

  • +
+
+
+

SPI#

+
    +
  • Change base package to io.prestosql.spi.

  • +
  • Move connector related classes to package io.prestosql.spi.connector.

  • +
  • Make ConnectorBucketNodeMap a top level class.

  • +
  • Use list instead of map for bucket-to-node mapping.

  • +
+
+

Note

+

These are backwards incompatible changes with the previous SPI. +If you have written a plugin, you will need to update your code +before deploying this release.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-301.html b/430/release/release-301.html new file mode 100644 index 000000000..cc6bc2a8e --- /dev/null +++ b/430/release/release-301.html @@ -0,0 +1,2633 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 301 (31 Jan 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 301 (31 Jan 2019)#

+
+

General#

+
    +
  • Fix reporting of aggregate input data size stats. (#100)

  • +
  • Add support for role management (see CREATE ROLE). Note, using SET ROLE +requires an up-to-date client library. (#90)

  • +
  • Add INVOKER security mode for CREATE VIEW. (#30)

  • +
  • Add ANALYZE SQL statement for collecting table statistics. (#99)

  • +
  • Add log() function with arbitrary base. (#36)

  • +
  • Remove the deprecated.legacy-log-function configuration option. The legacy behavior +(reverse argument order) for the log() function is no longer available. (#36)

  • +
  • Remove the deprecated.legacy-array-agg configuration option. The legacy behavior +(ignoring nulls) for array_agg() is no longer available. (#77)

  • +
  • Improve performance of COALESCE expressions. (#35)

  • +
  • Improve error message for unsupported reduce_agg() state type. (#55)

  • +
  • Improve performance of queries involving SYSTEM table sampling and computations over the +columns of the sampled table. (#29)

  • +
+
+
+

Server RPM#

+
    +
  • Do not allow uninstalling RPM while server is still running. (#67)

  • +
+
+
+

Security#

+
    +
  • Support LDAP with anonymous bind disabled. (#97)

  • +
+
+
+

Hive connector#

+
    +
  • Add procedure for dumping metastore recording to a file. (#54)

  • +
  • Add Metastore recorder support for Glue. (#61)

  • +
  • Add hive.temporary-staging-directory-enabled configuration property and +temporary_staging_directory_enabled session property to control whether a temporary staging +directory should be used for write operations. (#70)

  • +
  • Add hive.temporary-staging-directory-path configuration property and +temporary_staging_directory_path session property to control the location of temporary +staging directory that is used for write operations. The ${USER} placeholder can be used to +use a different location for each user (e.g., /tmp/${USER}). (#70)

  • +
+
+
+

Kafka connector#

+
    +
  • The minimum supported Kafka broker version is now 0.10.0. (#53)

  • +
+
+
+

Base-JDBC connector library#

+
    +
  • Add support for defining procedures. (#73)

  • +
  • Add support for providing table statistics. (#72)

  • +
+
+
+

SPI#

+
    +
  • Include session trace token in QueryCreatedEvent and QueryCompletedEvent. (#24)

  • +
  • Fix regression in NodeManager where node list was not being refreshed on workers. (#27)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-302.html b/430/release/release-302.html new file mode 100644 index 000000000..4cea409d1 --- /dev/null +++ b/430/release/release-302.html @@ -0,0 +1,2641 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 302 (6 Feb 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 302 (6 Feb 2019)#

+
+

General#

+
    +
  • Fix cluster starvation when wait for minimum number of workers is enabled. (#155)

  • +
  • Fix backup of queries blocked waiting for minimum number of workers. (#155)

  • +
  • Fix failure when preparing statements that contain a quoted reserved word as a table name. (#80)

  • +
  • Fix query failure when spilling is triggered during certain phases of query execution. (#164)

  • +
  • Fix SHOW CREATE VIEW output to preserve table name quoting. (#80)

  • +
  • Add Elasticsearch connector. (#118)

  • +
  • Add support for boolean type to approx_distinct(). (#82)

  • +
  • Add support for boolean columns to EXPLAIN with type IO. (#157)

  • +
  • Add SphericalGeography type and related geospatial functions. (#166)

  • +
  • Remove deprecated system memory pool. (#168)

  • +
  • Improve query performance for certain queries involving ROLLUP. (#105)

  • +
+
+
+

CLI#

+
    +
  • Add --trace-token option to set the trace token. (#117)

  • +
  • Display spilled data size as part of debug information. (#161)

  • +
+
+
+

Web UI#

+
    +
  • Add spilled data size to query details page. (#161)

  • +
+
+
+

Security#

+
    +
  • Add http.server.authentication.krb5.principal-hostname configuration option to set the hostname +for the Kerberos service principal. (#146, #153)

  • +
  • Add support for client-provided extra credentials that can be utilized by connectors. (#124)

  • +
+
+
+

Hive connector#

+
    +
  • Fix Parquet predicate pushdown for smallint, tinyint types. (#131)

  • +
  • Add support for Google Cloud Storage (GCS). Credentials can be provided globally using the +hive.gcs.json-key-file-path configuration property, or as a client-provided extra credential +named hive.gcs.oauth if the hive.gcs.use-access-token configuration property is enabled. (#124)

  • +
  • Allow creating tables with the external_location property pointing to an empty S3 directory. (#75)

  • +
  • Reduce GC pressure from Parquet reader by constraining the maximum column read size. (#58)

  • +
  • Reduce network utilization and latency for S3 when reading ORC or Parquet. (#142)

  • +
+
+
+

Kafka connector#

+
    +
  • Fix query failure when reading information_schema.columns without an equality condition on table_name. (#120)

  • +
+
+
+

Redis connector#

+
    +
  • Fix query failure when reading information_schema.columns without an equality condition on table_name. (#120)

  • +
+
+
+

SPI#

+
    +
  • Include query peak task user memory in QueryCreatedEvent and QueryCompletedEvent. (#163)

  • +
  • Include plan node cost and statistics estimates in QueryCompletedEvent. (#134)

  • +
  • Include physical and internal network input data size in QueryCompletedEvent. (#133)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-303.html b/430/release/release-303.html new file mode 100644 index 000000000..ded953287 --- /dev/null +++ b/430/release/release-303.html @@ -0,0 +1,2624 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 303 (13 Feb 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 303 (13 Feb 2019)#

+
+

General#

+
    +
  • Fix incorrect padding for CHAR values containing Unicode supplementary characters. +Previously, such values would be incorrectly padded with too few spaces. (#195)

  • +
  • Fix an issue where a union of a table with a VALUES statement would execute on a +single node, which could lead to out of memory errors. (#207)

  • +
  • Fix /v1/info to report started status after all plugins have been registered and initialized. (#213)

  • +
  • Improve performance of window functions by avoiding unnecessary data exchanges over the network. (#177)

  • +
  • Choose the distribution type for semi joins based on cost when the +join_distribution_type session property is set to AUTOMATIC. (#160)

  • +
  • Expand grouped execution support to window functions, making it possible +to execute them with less peak memory usage. (#169)

  • +
+
+
+

Web UI#

+
    +
  • Add additional details to and improve rendering of live plan. (#182)

  • +
+
+
+

CLI#

+
    +
  • Add --progress option to show query progress in batch mode. (#34)

  • +
+
+
+

Hive connector#

+
    +
  • Fix query failure when reading Parquet data with no columns selected. +This affects queries such as SELECT count(*). (#203)

  • +
+
+
+

Mongo connector#

+
    +
  • Fix failure for queries involving joins or aggregations on ObjectId type. (#215)

  • +
+
+
+

Base-JDBC connector library#

+
    +
  • Allow customizing how query predicates are pushed down to the underlying database. (#109)

  • +
  • Allow customizing how values are written to the underlying database. (#109)

  • +
+
+
+

SPI#

+
    +
  • Remove deprecated methods getSchemaName and getTableName from the SchemaTablePrefix +class. These were replaced by the getSchema and getTable methods. (#89)

  • +
  • Remove deprecated variants of methods listTables and listViews +from the ConnectorMetadata class. (#89)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-304.html b/430/release/release-304.html new file mode 100644 index 000000000..f7db11064 --- /dev/null +++ b/430/release/release-304.html @@ -0,0 +1,2629 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 304 (27 Feb 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 304 (27 Feb 2019)#

+
+

General#

+
    +
  • Fix wrong results for queries involving FULL OUTER JOIN and coalesce expressions +over the join keys. (#288)

  • +
  • Fix failure when a column is referenced using its fully qualified form. (#250)

  • +
  • Correctly report physical and internal network position count for operators. (#271)

  • +
  • Improve plan stability for repeated executions of the same query. (#226)

  • +
  • Remove deprecated datasources configuration property. (#306)

  • +
  • Improve error message when a query contains zero-length delimited identifiers. (#249)

  • +
  • Avoid opening an unnecessary HTTP listener on an arbitrary port. (#239)

  • +
  • Add experimental support for spilling for queries involving ORDER BY or window functions. (#228)

  • +
+
+
+

Server RPM#

+
    +
  • Preserve modified configuration files when the RPM is uninstalled. (#267)

  • +
+
+
+

Web UI#

+
    +
  • Fix broken timeline view. (#283)

  • +
  • Show data size and position count reported by connectors and by worker-to-worker data transfers +in detailed query view. (#271)

  • +
+
+
+

Hive connector#

+
    +
  • Fix authorization failure when using SQL Standard Based Authorization mode with user identifiers +that contain capital letters. (#289)

  • +
  • Fix wrong results when filtering on the hidden $bucket column for tables containing +partitions with different bucket counts. Instead, queries will now fail in this case. (#286)

  • +
  • Record the configured Hive time zone when writing ORC files. (#212)

  • +
  • Use the time zone recorded in ORC files when reading timestamps. +The configured Hive time zone, which was previously always used, is now +used only as a default when the writer did not record the time zone. (#212)

  • +
  • Support Parquet files written with Parquet 1.9+ that use DELTA_BINARY_PACKED +encoding with the Parquet INT64 type. (#334)

  • +
  • Allow setting the retry policy for the Thrift metastore client using the +hive.metastore.thrift.client.* configuration properties. (#240)

  • +
  • Reduce file system read operations when reading Parquet file footers. (#296)

  • +
  • Allow ignoring Glacier objects in S3 rather than failing the query. This is +disabled by default, as it may skip data that is expected to exist, but it can +be enabled using the hive.s3.skip-glacier-objects configuration property. (#305)

  • +
  • Add procedure system.sync_partition_metadata() to synchronize the partitions +in the metastore with the partitions that are physically on the file system. (#223)

  • +
  • Improve performance of ORC reader for columns that only contain nulls. (#229)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Map PostgreSQL json and jsonb types to Presto json type. (#81)

  • +
+
+
+

Cassandra connector#

+
    +
  • Support queries over tables containing partitioning columns of any type. (#252)

  • +
  • Support smallint, tinyint and date Cassandra types. (#141)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-305.html b/430/release/release-305.html new file mode 100644 index 000000000..856cbf4bc --- /dev/null +++ b/430/release/release-305.html @@ -0,0 +1,2621 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 305 (7 Mar 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 305 (7 Mar 2019)#

+
+

General#

+
    +
  • Fix failure of Regular expression functions for certain patterns and inputs +when using the default JONI library. (#350)

  • +
  • Fix a rare ClassLoader related problem for plugins providing an EventListenerFactory. (#299)

  • +
  • Expose join_max_broadcast_table_size session property, which was previously hidden. (#346)

  • +
  • Improve performance of queries when spill is enabled but not triggered. (#315)

  • +
  • Consider estimated query peak memory when making cost based decisions. (#247)

  • +
  • Include revocable memory in total memory stats. (#273)

  • +
  • Add peak revocable memory to operator stats. (#273)

  • +
  • Add ST_Points() function to access vertices of a linestring. (#316)

  • +
  • Add a system table system.metadata.analyze_properties +to list all ANALYZE properties. (#376)

  • +
+
+
+

Resource groups#

+
    +
  • Fix resource group selection when selector uses regular expression variables. (#373)

  • +
+
+
+

Web UI#

+
    +
  • Display peak revocable memory, current total memory, +and peak total memory in detailed query view. (#273)

  • +
+
+
+

CLI#

+
    +
  • Add option to output CSV without quotes. (#319)

  • +
+
+
+

Hive connector#

+
    +
  • Fix handling of updated credentials for Google Cloud Storage (GCS). (#398)

  • +
  • Fix calculation of bucket number for timestamps that contain a non-zero +milliseconds value. Previously, data would be written into the wrong bucket, +or could be incorrectly skipped on read. (#366)

  • +
  • Allow writing ORC files compatible with Hive 2.0.0 to 2.2.0 by identifying +the writer as an old version of Hive (rather than Presto) in the files. +This can be enabled using the hive.orc.writer.use-legacy-version-number +configuration property. (#353)

  • +
  • Support dictionary filtering for Parquet v2 files using RLE_DICTIONARY encoding. (#251)

  • +
  • Remove legacy writers for ORC and RCFile. (#353)

  • +
  • Remove support for the DWRF file format. (#353)

  • +
+
+
+

Base-JDBC connector library#

+
    +
  • Allow access to extra credentials when opening a JDBC connection. (#281)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-306.html b/430/release/release-306.html new file mode 100644 index 000000000..9480a1f02 --- /dev/null +++ b/430/release/release-306.html @@ -0,0 +1,2654 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 306 (16 Mar 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 306 (16 Mar 2019)#

+
+

General#

+
    +
  • Fix planning failure for queries containing a LIMIT after a global +aggregation. (#437)

  • +
  • Fix missing column types in EXPLAIN output. (#328)

  • +
  • Fix accounting of peak revocable memory reservation. (#413)

  • +
  • Fix double memory accounting for aggregations when spilling is active. (#413)

  • +
  • Fix excessive CPU usage that can occur when spilling for window functions. (#468)

  • +
  • Fix incorrect view name displayed by SHOW CREATE VIEW. (#433)

  • +
  • Allow specifying NOT NULL when creating tables or adding columns. (#418)

  • +
  • Add a config option (query.stage-count-warning-threshold) to specify a +per-query threshold for the number of stages. When this threshold is exceeded, +a TOO_MANY_STAGES warning is raised. (#330)

  • +
  • Support session property values with special characters (e.g., comma or equals sign). (#407)

  • +
  • Remove the deprecated.legacy-unnest-array-rows configuration option. +The legacy behavior for UNNEST of arrays containing ROW values is no +longer supported. (#430)

  • +
  • Remove the deprecated.legacy-row-field-ordinal-access configuration option. +The legacy mechanism for accessing fields of anonymous ROW types is no longer +supported. (#428)

  • +
  • Remove the deprecated.group-by-uses-equal configuration option. The legacy equality +semantics for GROUP BY are not longer supported. (#432)

  • +
  • Remove the deprecated.legacy-map-subscript. The legacy behavior for the map subscript +operator on missing keys is no longer supported. (#429)

  • +
  • Remove the deprecated.legacy-char-to-varchar-coercion configuration option. The +legacy coercion rules between CHAR and VARCHAR types are no longer +supported. (#431)

  • +
  • Remove deprecated distributed_join system property. Use join_distribution_type +instead. (#452)

  • +
+
+
+

Hive connector#

+
    +
  • Fix calling procedures immediately after startup, before any other queries are run. +Previously, the procedure call would fail and also cause all subsequent Hive queries +to fail. (#414)

  • +
  • Improve ORC reader performance for decoding REAL and DOUBLE types. (#465)

  • +
+
+
+

MySQL connector#

+
    +
  • Allow creating or renaming tables, and adding, renaming, or dropping columns. (#418)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Fix predicate pushdown for PostgreSQL ENUM type. (#408)

  • +
  • Allow creating or renaming tables, and adding, renaming, or dropping columns. (#418)

  • +
+
+
+

Redshift connector#

+
    +
  • Allow creating or renaming tables, and adding, renaming, or dropping columns. (#418)

  • +
+
+
+

SQL Server connector#

+
    +
  • Allow creating or renaming tables, and adding, renaming, or dropping columns. (#418)

  • +
+
+
+

Base-JDBC connector library#

+
    +
  • Allow mapping column type to Presto type based on Block. (#454)

  • +
+
+
+

SPI#

+
    +
  • Deprecate Table Layout APIs. Connectors can opt out of the legacy behavior by implementing +ConnectorMetadata.usesLegacyTableLayouts(). (#420)

  • +
  • Add support for limit pushdown into connectors via the ConnectorMetadata.applyLimit() +method. (#421)

  • +
  • Add time spent waiting for resources to QueryCompletedEvent. (#461)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-307.html b/430/release/release-307.html new file mode 100644 index 000000000..aae96e456 --- /dev/null +++ b/430/release/release-307.html @@ -0,0 +1,2656 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 307 (3 Apr 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 307 (3 Apr 2019)#

+
+

General#

+
    +
  • Fix cleanup of spill files for queries using window functions or ORDER BY. (#543)

  • +
  • Optimize queries containing ORDER BY together with LIMIT over an OUTER JOIN +by pushing ORDER BY and LIMIT to the outer side of the join. (#419)

  • +
  • Improve performance of table scans for data sources that produce tiny pages. (#467)

  • +
  • Improve performance of IN subquery expressions that contain a DISTINCT clause. (#551)

  • +
  • Expand support of types handled in EXPLAIN (TYPE IO). (#509)

  • +
  • Add support for outer joins involving lateral derived tables (i.e., LATERAL). (#390)

  • +
  • Add support for setting table comments via the COMMENT syntax. (#200)

  • +
+
+
+

Web UI#

+
    +
  • Allow UI to work when opened as /ui (no trailing slash). (#500)

  • +
+
+
+

Security#

+
    +
  • Make query result and cancellation URIs secure. Previously, an authenticated +user could potentially steal the result data of any running query. (#561)

  • +
+
+
+

Server RPM#

+
    +
  • Prevent JVM from allocating large amounts of native memory. The new configuration is applied +automatically when Presto is installed from RPM. When Presto is installed another way, or when +you provide your own jvm.config, we recommend adding -Djdk.nio.maxCachedBufferSize=2000000 +to your jvm.config. See Deploying Trino for details. (#542)

  • +
+
+
+

CLI#

+
    +
  • Always abort query in batch mode when CLI is killed. (#508, #580)

  • +
+
+
+

JDBC driver#

+
    +
  • Abort query synchronously when the ResultSet is closed or when the +Statement is cancelled. Previously, the abort was sent in the background, +allowing the JVM to exit before the abort was received by the server. (#580)

  • +
+
+
+

Hive connector#

+
    +
  • Add safety checks for Hive bucketing version. Hive 3.0 introduced a new +bucketing version that uses an incompatible hash function. The Hive connector +will treat such tables as not bucketed when reading and disallows writing. (#512)

  • +
  • Add support for setting table comments via the COMMENT syntax. (#200)

  • +
+
+
+

Other connectors#

+

These changes apply to the MySQL, PostgreSQL, Redshift, and SQL Server connectors.

+
    +
  • Fix reading and writing of timestamp values. Previously, an incorrect value +could be read, depending on the Presto JVM time zone. (#495)

  • +
  • Add support for using a client-provided username and password. The credential +names can be configured using the user-credential-name and password-credential-name +configuration properties. (#482)

  • +
+
+
+

SPI#

+
    +
  • LongDecimalType and IpAddressType now use Int128ArrayBlock instead +of FixedWithBlock. Any code that creates blocks directly, rather than using +the BlockBuilder returned from the Type, will need to be updated. (#492)

  • +
  • Remove FixedWidthBlock. Use one of the *ArrayBlock classes instead. (#492)

  • +
  • Add support for simple constraint pushdown into connectors via the +ConnectorMetadata.applyFilter() method. (#541)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-308.html b/430/release/release-308.html new file mode 100644 index 000000000..ede5e1412 --- /dev/null +++ b/430/release/release-308.html @@ -0,0 +1,2654 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 308 (11 Apr 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 308 (11 Apr 2019)#

+
+

General#

+
    +
  • Fix a regression that prevented the server from starting on Java 9+. (#610)

  • +
  • Fix correctness issue for queries involving FULL OUTER JOIN and coalesce. (#622)

  • +
+
+
+

Security#

+
    +
  • Add authorization for listing table columns. (#507)

  • +
+
+
+

CLI#

+
    +
  • Add option for specifying Kerberos service principal pattern. (#597)

  • +
+
+
+

JDBC driver#

+
    +
  • Correctly report precision and column display size in ResultSetMetaData +for char and varchar columns. (#615)

  • +
  • Add option for specifying Kerberos service principal pattern. (#597)

  • +
+
+
+

Hive connector#

+
    +
  • Fix regression that could cause queries to fail with Query can potentially read more than X partitions error. (#619)

  • +
  • Improve ORC read performance significantly. For TPC-DS, this saves about 9.5% of +total CPU when running over gzip-compressed data. (#555)

  • +
  • Require access to a table (any privilege) in order to list the columns. (#507)

  • +
  • Add directory listing cache for specific tables. The list of tables is specified +using the hive.file-status-cache-tables configuration property. (#343)

  • +
+
+
+

MySQL connector#

+
    +
  • Fix ALTER TABLE ... RENAME TO ... statement. (#586)

  • +
  • Push simple LIMIT queries into the external database. (#589)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Push simple LIMIT queries into the external database. (#589)

  • +
+
+
+

Redshift connector#

+
    +
  • Push simple LIMIT queries into the external database. (#589)

  • +
+
+
+

SQL Server connector#

+
    +
  • Fix writing varchar values with non-Latin characters in CREATE TABLE AS. (#573)

  • +
  • Support writing varchar and char values with length longer than 4000 +characters in CREATE TABLE AS. (#573)

  • +
  • Support writing boolean values in CREATE TABLE AS. (#573)

  • +
  • Push simple LIMIT queries into the external database. (#589)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Add support for Search Guard in Elasticsearch connector. Please refer to Elasticsearch connector +for the relevant configuration properties. (#438)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-309.html b/430/release/release-309.html new file mode 100644 index 000000000..51e735347 --- /dev/null +++ b/430/release/release-309.html @@ -0,0 +1,2662 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 309 (25 Apr 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 309 (25 Apr 2019)#

+
+

General#

+
    +
  • Fix incorrect match result for Regular expression functions when pattern ends +with a word boundary matcher. This only affects the default JONI library. +(#661)

  • +
  • Fix failures for queries involving spatial joins. (#652)

  • +
  • Add support for SphericalGeography to ST_Area(). (#383)

  • +
+
+
+

Security#

+
    +
  • Add option for specifying the Kerberos GSS name type. (#645)

  • +
+
+
+

Server RPM#

+ +
+
+

Hive connector#

+
    +
  • Fix rare failure when reading DECIMAL values from ORC files. (#664)

  • +
  • Add a hidden $properties table for each table that describes its Hive table +properties. For example, a table named example will have an associated +properties table named example$properties. (#268)

  • +
+
+
+

MySQL connector#

+
    +
  • Match schema and table names case insensitively. This behavior can be enabled by setting +the case-insensitive-name-matching catalog configuration option to true. (#614)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Add support for ARRAY type. (#317)

  • +
  • Add support writing TINYINT values. (#317)

  • +
  • Match schema and table names case insensitively. This behavior can be enabled by setting +the case-insensitive-name-matching catalog configuration option to true. (#614)

  • +
+
+
+

Redshift connector#

+
    +
  • Match schema and table names case insensitively. This behavior can be enabled by setting +the case-insensitive-name-matching catalog configuration option to true. (#614)

  • +
+
+
+

SQL Server connector#

+
    +
  • Match schema and table names case insensitively. This behavior can be enabled by setting +the case-insensitive-name-matching catalog configuration option to true. (#614)

  • +
+
+
+

Cassandra connector#

+
    +
  • Allow reading from tables which have Cassandra column types that are not supported by Presto. +These columns will not be visible in Presto. (#592)

  • +
+
+
+

SPI#

+
    +
  • Add session parameter to the applyFilter() and applyLimit() methods in +ConnectorMetadata. (#636)

  • +
+
+

Note

+

This is a backwards incompatible changes with the previous SPI. +If you have written a connector that implements these methods, +you will need to update your code before deploying this release.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-310.html b/430/release/release-310.html new file mode 100644 index 000000000..2cc1a1a60 --- /dev/null +++ b/430/release/release-310.html @@ -0,0 +1,2606 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 310 (3 May 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 310 (3 May 2019)#

+
+

General#

+
    +
  • Reduce compilation failures for expressions over types containing an extremely +large number of nested types. (#537)

  • +
  • Fix error reporting when query fails with due to running out of memory. (#696)

  • +
  • Improve performance of JOIN queries involving join keys of different types. +(#665)

  • +
  • Add initial and experimental support for late materialization. +This feature can be enabled via experimental.work-processor-pipelines +feature config or via work_processor_pipelines session config. +Simple select queries of type SELECT ... FROM table ORDER BY cols LIMIT n can +experience significant CPU and performance improvement. (#602)

  • +
  • Add support for FETCH FIRST syntax. (#666)

  • +
+
+
+

CLI#

+
    +
  • Make the final query time consistent with query stats. (#692)

  • +
+
+
+

Hive connector#

+
    +
  • Ignore boolean column statistics when the count is -1. (#241)

  • +
  • Prevent failures for information_schema queries when a table has an invalid +storage format. (#568)

  • +
  • Add support for assuming AWS role when accessing S3 or Glue. (#698)

  • +
  • Add support for coercions between DECIMAL, DOUBLE, and REAL for +partition and table schema mismatch. (#352)

  • +
  • Fix typo in Metastore recorder duration property name. (#711)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Support for the ARRAY type has been disabled by default. (#687)

  • +
+
+
+

Blackhole connector#

+
    +
  • Support having tables with same name in different Blackhole schemas. (#550)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-311.html b/430/release/release-311.html new file mode 100644 index 000000000..cd211f413 --- /dev/null +++ b/430/release/release-311.html @@ -0,0 +1,2600 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 311 (14 May 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+ +
+
+ +
+
+ +
+

Release 311 (14 May 2019)#

+
+

General#

+
    +
  • Fix incorrect results for aggregation query that contains a HAVING clause but no +GROUP BY clause. (#733)

  • +
  • Fix rare error when moving already completed query to a new memory pool. (#725)

  • +
  • Fix leak in operator peak memory computations (#764)

  • +
  • Improve consistency of reported query statistics. (#773)

  • +
  • Add support for OFFSET syntax. (#732)

  • +
  • Print cost metrics using appropriate units in the output of EXPLAIN. (#68)

  • +
  • Add combinations() function. (#714)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for static AWS credentials for the Glue metastore. (#748)

  • +
+
+
+

Cassandra connector#

+
    +
  • Support collections nested in other collections. (#657)

  • +
  • Automatically discover the Cassandra protocol version when the previously required +cassandra.protocol-version configuration property is not set. (#596)

  • +
+
+
+

Black Hole connector#

+
    +
  • Fix rendering of tables and columns in plans. (#728)

  • +
  • Add table and column statistics. (#728)

  • +
+
+
+

System connector#

+
    +
  • Add system.metadata.table_comments table that contains table comments. (#531)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-312.html b/430/release/release-312.html new file mode 100644 index 000000000..3856ee945 --- /dev/null +++ b/430/release/release-312.html @@ -0,0 +1,2640 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 312 (29 May 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 312 (29 May 2019)#

+
+

General#

+
    +
  • Fix incorrect results for queries using IS [NOT] DISTINCT FROM. (#795)

  • +
  • Fix array_distinct, array_intersect semantics with respect to indeterminate +values (i.e., NULL or structural types containing NULL). (#559)

  • +
  • Fix failure when the largest negative BIGINT value (-9223372036854775808) is used +as a constant in a query. (#805)

  • +
  • Improve reliability for network errors when using Kerberos with +Secure internal communication. (#838)

  • +
  • Improve performance of JOIN queries involving inline tables (VALUES). (#743)

  • +
  • Improve performance of queries containing duplicate expressions. (#730)

  • +
  • Improve performance of queries involving comparisons between values of different types. (#731)

  • +
  • Improve performance of queries containing redundant ORDER BY clauses in subqueries. This may +affect the semantics of queries that incorrectly rely on implementation-specific behavior. The +old behavior can be restored via the skip_redundant_sort session property or the +optimizer.skip-redundant-sort configuration property. (#818)

  • +
  • Improve performance of IN predicates that contain subqueries. (#767)

  • +
  • Improve support for correlated subqueries containing redundant LIMIT clauses. (#441)

  • +
  • Add a new UUID type to represent UUIDs. (#755)

  • +
  • Add uuid() function to generate random UUIDs. (#786)

  • +
  • Add Phoenix connector. (#672)

  • +
  • Make semantic error name available in client protocol. (#790)

  • +
  • Report operator statistics when experimental.work-processor-pipelines +is enabled. (#788)

  • +
+
+
+

Server#

+
    +
  • Raise required Java version to 8u161. This version allows unlimited strength crypto. (#779)

  • +
  • Show JVM configuration hint when JMX agent fails to start on Java 9+. (#838)

  • +
  • Skip starting JMX agent on Java 9+ if it is already configured via JVM properties. (#838)

  • +
  • Support configuring TrustStore for Secure internal communication using the +internal-communication.https.truststore.path and internal-communication.https.truststore.key +configuration properties. The path can point at a Java KeyStore or a PEM file. (#785)

  • +
  • Remove deprecated check for minimum number of workers before starting a coordinator. Use the +query-manager.required-workers and query-manager.required-workers-max-wait configuration +properties instead. (#95)

  • +
+
+
+

Hive connector#

+
    +
  • Fix SHOW GRANTS failure when metastore contains few tables. (#791)

  • +
  • Fix failure reading from information_schema.table_privileges table when metastore +contains few tables. (#791)

  • +
  • Use Hive naming convention for file names when writing to bucketed tables. (#822)

  • +
  • Support new Hive bucketing conventions by allowing any number of files per bucket. +This allows reading from partitions that were inserted into multiple times by Hive, +or were written to by Hive on Tez (which does not create files for empty buckets).

  • +
  • Allow disabling the creation of files for empty buckets when writing data. +This behavior is enabled by default for compatibility with previous versions of Presto, +but can be disabled using the hive.create-empty-bucket-files configuration property +or the create_empty_bucket_files session property. (#822)

  • +
+
+
+

MySQL connector#

+
    +
  • Map MySQL json type to Presto json type. (#824)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Add support for PostgreSQL’s TIMESTAMP WITH TIME ZONE data type. (#640)

  • +
+
+
+

SPI#

+
    +
  • Add support for pushing TABLESAMPLE into connectors via the +ConnectorMetadata.applySample() method. (#753)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-313.html b/430/release/release-313.html new file mode 100644 index 000000000..bc295a94e --- /dev/null +++ b/430/release/release-313.html @@ -0,0 +1,2583 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 313 (31 May 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 313 (31 May 2019)#

+
+

General#

+
    +
  • Fix leak in operator peak memory computations. (#843)

  • +
  • Fix incorrect results for queries involving GROUPING SETS and LIMIT. (#864)

  • +
  • Add compression and encryption support for Spill to disk. (#778)

  • +
+
+
+

CLI#

+
    +
  • Fix failure when selecting a value of type UUID. (#854)

  • +
+
+
+

JDBC driver#

+
    +
  • Fix failure when selecting a value of type UUID. (#854)

  • +
+
+
+

Phoenix connector#

+
    +
  • Allow matching schema and table names case insensitively. This can be enabled by setting +the case-insensitive-name-matching configuration property to true. (#872)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-314.html b/430/release/release-314.html new file mode 100644 index 000000000..08a29c0da --- /dev/null +++ b/430/release/release-314.html @@ -0,0 +1,2657 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 314 (7 Jun 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+ +
+ +
+
+ +
+

Release 314 (7 Jun 2019)#

+
+

General#

+
    +
  • Fix incorrect results for BETWEEN involving NULL values. (#877)

  • +
  • Fix query history leak in coordinator. (#939, #944)

  • +
  • Fix idle client timeout handling. (#947)

  • +
  • Improve performance of json_parse() function. (#904)

  • +
  • Visualize plan structure in EXPLAIN output. (#888)

  • +
  • Add support for positional access to ROW fields via the subscript +operator. (#860)

  • +
+
+
+

CLI#

+
    +
  • Add JSON output format. (#878)

  • +
+
+
+

Web UI#

+
    +
  • Fix queued queries counter in UI. (#894)

  • +
+
+
+

Server RPM#

+
    +
  • Change default location of the http-request.log to /var/log/presto. Previously, +the log would be located in /var/lib/presto/data/var/log by default. (#919)

  • +
+
+
+

Hive connector#

+
    +
  • Fix listing tables and views from Hive 2.3+ Metastore on certain databases, +including Derby and Oracle. This fixes SHOW TABLES, SHOW VIEWS and +reading from information_schema.tables table. (#833)

  • +
  • Fix handling of Avro tables with avro.schema.url defined in Hive +SERDEPROPERTIES. (#898)

  • +
  • Fix regression that caused ORC bloom filters to be ignored. (#921)

  • +
  • Add support for reading LZ4 and ZSTD compressed Parquet data. (#910)

  • +
  • Add support for writing ZSTD compressed ORC data. (#910)

  • +
  • Add support for configuring ZSTD and LZ4 as default compression methods via the +hive.compression-codec configuration option. (#910)

  • +
  • Do not allow inserting into text format tables that have a header or footer. (#891)

  • +
  • Add textfile_skip_header_line_count and textfile_skip_footer_line_count table properties +for text format tables that specify the number of header and footer lines. (#845)

  • +
  • Add hive.max-splits-per-second configuration property to allow throttling +the split discovery rate, which can reduce load on the file system. (#534)

  • +
  • Support overwriting unpartitioned tables for insert queries. (#924)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Support PostgreSQL arrays declared using internal type +name, for example _int4 (rather than int[]). (#659)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Add support for mixed-case field names. (#887)

  • +
+
+
+

Base-JDBC connector library#

+
    +
  • Allow connectors to customize how they store NULL values. (#918)

  • +
+
+
+

SPI#

+
    +
  • Expose the SQL text of the executed prepared statement to EventListener. (#908)

  • +
  • Deprecate table layouts for ConnectorMetadata.makeCompatiblePartitioning(). (#689)

  • +
  • Add support for delete pushdown into connectors via the ConnectorMetadata.applyDelete() +and ConnectorMetadata.executeDelete() methods. (#689)

  • +
  • Allow connectors without distributed tables. (#893)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-315.html b/430/release/release-315.html new file mode 100644 index 000000000..5a99d42c1 --- /dev/null +++ b/430/release/release-315.html @@ -0,0 +1,2625 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 315 (14 Jun 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 315 (14 Jun 2019)#

+
+

General#

+
    +
  • Fix incorrect results when dividing certain decimal numbers. (#958)

  • +
  • Add support for FETCH FIRST ... WITH TIES syntax. (#832)

  • +
  • Add locality awareness to default split scheduler. (#680)

  • +
  • Add format() function. (#548)

  • +
+
+
+

Server RPM#

+
    +
  • Require JDK version 8u161+ during installation, which is the version the server requires. (#983)

  • +
+
+
+

CLI#

+
    +
  • Fix alignment of nulls for numeric columns in aligned output format. (#871)

  • +
+
+
+

Hive connector#

+
    +
  • Fix regression in partition pruning for certain query shapes. (#984)

  • +
  • Correctly identify EMRFS as S3 when deciding to use a temporary location for writes. (#935)

  • +
  • Allow creating external tables on S3 even if the location does not exist. (#935)

  • +
  • Add support for UTF-8 ORC bloom filters. (#914)

  • +
  • Add support for DATE, TIMESTAMP and REAL in ORC bloom filters. (#967)

  • +
  • Disable usage of old, non UTF-8, ORC bloom filters for VARCHAR and CHAR. (#914)

  • +
  • Allow logging all calls to Hive Thrift metastore service. This can be enabled +by turning on DEBUG logging for +io.prestosql.plugin.hive.metastore.thrift.ThriftHiveMetastoreClient. (#946)

  • +
+
+
+

MongoDB connector#

+
    +
  • Fix query failure when ROW with an ObjectId field is used as a join key. (#933)

  • +
  • Add cast from ObjectId to VARCHAR. (#933)

  • +
+
+
+

SPI#

+
    +
  • Allow connectors to provide view definitions. ConnectorViewDefinition now contains +the real view definition rather than an opaque blob. Connectors that support view storage +can use the JSON representation of that class as a stable storage format. The JSON +representation is the same as the previous opaque blob, thus all existing view +definitions will continue to work. (#976)

  • +
  • Add getView() method to ConnectorMetadata as a replacement for getViews(). +The getViews() method now exists only as an optional method for connectors that +can efficiently support bulk retrieval of views and has a different signature. (#976)

  • +
+
+

Note

+

These are backwards incompatible changes with the previous SPI. +If you have written a connector that supports views, you will +need to update your code before deploying this release.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-316.html b/430/release/release-316.html new file mode 100644 index 000000000..8a433bd2e --- /dev/null +++ b/430/release/release-316.html @@ -0,0 +1,2653 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 316 (8 Jul 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 316 (8 Jul 2019)#

+
+

General#

+
    +
  • Fix date_format function failure when format string contains non-ASCII +characters. (#1056)

  • +
  • Improve performance of queries using UNNEST. (#901)

  • +
  • Improve error message when statement parsing fails. (#1042)

  • +
+
+
+

CLI#

+
    +
  • Fix refresh of completion cache when catalog or schema is changed. (#1016)

  • +
  • Allow reading password from console when stdout is a pipe. (#982)

  • +
+
+
+

Hive connector#

+
    +
  • Acquire S3 credentials from the default AWS locations if not configured explicitly. (#741)

  • +
  • Only allow using roles and grants with SQL standard based authorization. (#972)

  • +
  • Add support for CSV file format. (#920)

  • +
  • Support reading from and writing to Hadoop encryption zones (Hadoop KMS). (#997)

  • +
  • Collect column statistics on write by default. This can be disabled using the +hive.collect-column-statistics-on-write configuration property or the +collect_column_statistics_on_write session property. (#981)

  • +
  • Eliminate unused idle threads when using the metastore cache. (#1061)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Add support for columns of type UUID. (#1011)

  • +
  • Export JMX statistics for various JDBC and connector operations. (#906).

  • +
+
+
+

MySQL connector#

+
    +
  • Export JMX statistics for various JDBC and connector operations. (#906).

  • +
+
+
+

Redshift connector#

+
    +
  • Export JMX statistics for various JDBC and connector operations. (#906).

  • +
+
+
+

SQL Server connector#

+
    +
  • Export JMX statistics for various JDBC and connector operations. (#906).

  • +
+
+
+

TPC-H connector#

+
    +
  • Fix SHOW TABLES failure when used with a hidden schema. (#1005)

  • +
+
+
+

TPC-DS connector#

+
    +
  • Fix SHOW TABLES failure when used with a hidden schema. (#1005)

  • +
+
+
+

SPI#

+
    +
  • Add support for pushing simple column and row field reference expressions into +connectors via the ConnectorMetadata.applyProjection() method. (#676)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-317.html b/430/release/release-317.html new file mode 100644 index 000000000..966d2595c --- /dev/null +++ b/430/release/release-317.html @@ -0,0 +1,2660 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 317 (1 Aug 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+ +
+ +
+
+ +
+

Release 317 (1 Aug 2019)#

+
+

General#

+
    +
  • Fix url_extract_parameter() when the query string contains an encoded & or = character.

  • +
  • Export MBeans from the db resource group configuration manager. (#1151)

  • +
  • Add all_match(), any_match(), and none_match() functions. (#1045)

  • +
  • Add support for fractional weights in approx_percentile(). (#1168)

  • +
  • Add support for node dynamic filtering for semi-joins and filters when the experimental +WorkProcessor pipelines feature is enabled. (#1075, #1155, #1119)

  • +
  • Allow overriding session time zone for clients via the +sql.forced-session-time-zone configuration property. (#1164)

  • +
+
+
+

Web UI#

+
    +
  • Fix tooltip visibility on stage performance details page. (#1113)

  • +
  • Add planning time to query details page. (#1115)

  • +
+
+
+

Security#

+
    +
  • Allow schema owner to create, drop, and rename schema when using file-based +connector access control. (#1139)

  • +
  • Allow respecting the X-Forwarded-For header when retrieving the IP address +of the client submitting the query. This information is available in the +remoteClientAddress field of the QueryContext class for query events. +The behavior can be controlled via the dispatcher.forwarded-header +configuration property, as the header should only be used when the Presto +coordinator is behind a proxy. (#1033)

  • +
+
+
+

JDBC driver#

+
    +
  • Fix DatabaseMetaData.getURL() to include the jdbc: prefix. (#1211)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Add support for nested fields. (#1001)

  • +
+
+
+

Hive connector#

+
    +
  • Fix bucketing version safety check to correctly disallow writes +to tables that use an unsupported bucketing version. (#1199)

  • +
  • Fix metastore error handling when metastore debug logging is enabled. (#1152)

  • +
  • Improve performance of file listings in system.sync_partition_metadata procedure, +especially for S3. (#1093)

  • +
+
+
+

Kudu connector#

+
    +
  • Update Kudu client library version to 1.10.0. (#1086)

  • +
+
+
+

MongoDB connector#

+
    +
  • Allow passwords to contain the : or @ characters. (#1094)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Add support for reading hstore data type. (#1101)

  • +
+
+
+

SPI#

+
    +
  • Allow delete to be implemented for non-legacy connectors. (#1015)

  • +
  • Remove deprecated method from ConnectorPageSourceProvider. (#1095)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-318.html b/430/release/release-318.html new file mode 100644 index 000000000..784c7900c --- /dev/null +++ b/430/release/release-318.html @@ -0,0 +1,2684 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 318 (26 Aug 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+ +
+ +
+
+ +
+

Release 318 (26 Aug 2019)#

+
+

General#

+
    +
  • Fix query failure when using DISTINCT FROM with the UUID or +IPADDRESS types. (#1180)

  • +
  • Improve query performance when optimize_hash_generation is enabled. (#1071)

  • +
  • Improve performance of information schema tables. (#999, #1306)

  • +
  • Rename http.server.authentication.* configuration options to http-server.authentication.*. (#1270)

  • +
  • Change query CPU tracking for resource groups to update periodically while +the query is running. Previously, CPU usage would only update at query +completion. This improves resource management fairness when using +CPU-limited resource groups. (#1128)

  • +
  • Remove distributed_planning_time_ms column from system.runtime.queries. (#1084)

  • +
  • Add support for Asia/Qostanay time zone. (#1221)

  • +
  • Add session properties that allow overriding the query per-node memory limits: +query_max_memory_per_node and query_max_total_memory_per_node. These properties +can be used to decrease limits for a query, but not to increase them. (#1212)

  • +
  • Add Google Sheets connector. (#1030)

  • +
  • Add planning_time_ms column to the system.runtime.queries table that shows +the time spent on query planning. This is the same value that used to be in the +analysis_time_ms column, which was a misnomer. (#1084)

  • +
  • Add last_day_of_month() function. (#1295)

  • +
  • Add support for cancelling queries via the system.runtime.kill_query procedure when +they are in the queue or in the semantic analysis stage. (#1079)

  • +
  • Add queries that are in the queue or in the semantic analysis stage to the +system.runtime.queries table. (#1079)

  • +
+
+
+

Web UI#

+
    +
  • Display information about queries that are in the queue or in the semantic analysis +stage. (#1079)

  • +
  • Add support for cancelling queries that are in the queue or in the semantic analysis +stage. (#1079)

  • +
+
+
+

Hive connector#

+
    +
  • Fix query failure due to missing credentials while writing empty bucket files. (#1298)

  • +
  • Fix bucketing of NaN values of real type. Previously NaN values +could be assigned a wrong bucket. (#1336)

  • +
  • Fix reading RCFile collection delimiter set by Hive version earlier than 3.0. (#1321)

  • +
  • Return proper error when selecting "$bucket" column from a table using +Hive bucketing v2. (#1336)

  • +
  • Improve performance of S3 object listing. (#1232)

  • +
  • Improve performance when reading data from GCS. (#1200)

  • +
  • Add support for reading data from S3 Requester Pays buckets. This can be enabled +using the hive.s3.requester-pays.enabled configuration property. (#1241)

  • +
  • Allow inserting into bucketed, unpartitioned tables. (#1127)

  • +
  • Allow inserting into existing partitions of bucketed, partitioned tables. (#1347)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Add support for providing JDBC credential in a separate file. This can be enabled by +setting the credential-provider.type=FILE and connection-credential-file +config options in the catalog properties file. (#1124)

  • +
  • Allow logging all calls to JdbcClient. This can be enabled by turning +on DEBUG logging for io.prestosql.plugin.jdbc.JdbcClient. (#1274)

  • +
  • Add possibility to force mapping of certain types to varchar. This can be enabled +by setting jdbc-types-mapped-to-varchar to comma-separated list of type names. (#186)

  • +
  • Add support for PostgreSQL timestamp[] type. (#1023, #1262, #1328)

  • +
+
+
+

MySQL connector#

+
    +
  • Add support for providing JDBC credential in a separate file. This can be enabled by +setting the credential-provider.type=FILE and connection-credential-file +config options in the catalog properties file. (#1124)

  • +
  • Allow logging all calls to JdbcClient. This can be enabled by turning +on DEBUG logging for io.prestosql.plugin.jdbc.JdbcClient. (#1274)

  • +
  • Add possibility to force mapping of certain types to varchar. This can be enabled +by setting jdbc-types-mapped-to-varchar to comma-separated list of type names. (#186)

  • +
+
+
+

Redshift connector#

+
    +
  • Add support for providing JDBC credential in a separate file. This can be enabled by +setting the credential-provider.type=FILE and connection-credential-file +config options in the catalog properties file. (#1124)

  • +
  • Allow logging all calls to JdbcClient. This can be enabled by turning +on DEBUG logging for io.prestosql.plugin.jdbc.JdbcClient. (#1274)

  • +
  • Add possibility to force mapping of certain types to varchar. This can be enabled +by setting jdbc-types-mapped-to-varchar to comma-separated list of type names. (#186)

  • +
+
+
+

SQL Server connector#

+
    +
  • Add support for providing JDBC credential in a separate file. This can be enabled by +setting the credential-provider.type=FILE and connection-credential-file +config options in the catalog properties file. (#1124)

  • +
  • Allow logging all calls to JdbcClient. This can be enabled by turning +on DEBUG logging for io.prestosql.plugin.jdbc.JdbcClient. (#1274)

  • +
  • Add possibility to force mapping of certain types to varchar. This can be enabled +by setting jdbc-types-mapped-to-varchar to comma-separated list of type names. (#186)

  • +
+
+
+

SPI#

+
    +
  • Add Block.isLoaded() method. (#1216)

  • +
  • Update security APIs to accept the new ConnectorSecurityContext +and SystemSecurityContext classes. (#171)

  • +
  • Allow connectors to override minimal schedule split batch size. (#1251)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-319.html b/430/release/release-319.html new file mode 100644 index 000000000..1c94dd884 --- /dev/null +++ b/430/release/release-319.html @@ -0,0 +1,2646 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 319 (22 Sep 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 319 (22 Sep 2019)#

+
+

General#

+
    +
  • Fix planning failure for queries involving UNION and DISTINCT aggregates. (#1510)

  • +
  • Fix excessive runtime when parsing expressions involving CASE. (#1407)

  • +
  • Fix fragment output size in EXPLAIN ANALYZE output. (#1345)

  • +
  • Fix a rare failure when running EXPLAIN ANALYZE on a query containing +window functions. (#1401)

  • +
  • Fix failure when querying /v1/resourceGroupState endpoint for non-existing resource +group. (#1368)

  • +
  • Fix incorrect results when reading information_schema.table_privileges with +an equality predicate on table_name but without a predicate on table_schema. +(#1534)

  • +
  • Fix planning failure due to coercion handling for correlated subqueries. (#1453)

  • +
  • Improve performance of queries against information_schema tables. (#1329)

  • +
  • Reduce metadata querying during planning. (#1308, #1455)

  • +
  • Improve performance of certain queries involving coercions and complex expressions in JOIN +conditions. (#1390)

  • +
  • Include cost estimates in output of EXPLAIN (TYPE IO). (#806)

  • +
  • Improve support for correlated subqueries involving ORDER BY or LIMIT. (#1415)

  • +
  • Improve performance of certain JOIN queries when automatic join ordering is enabled. (#1431)

  • +
  • Allow setting the default session catalog and schema via the sql.default-catalog +and sql.default-schema configuration properties. (#1524)

  • +
  • Add support for IGNORE NULLS for window functions. (#1244)

  • +
  • Add support for INNER and OUTER joins involving UNNEST. (#1522)

  • +
  • Rename legacy and flat scheduler policies to +uniform and topology respectively. These can be configured via the node-scheduler.policy +configuration property. (#10491)

  • +
  • Add file network topology provider which can be configured +via the node-scheduler.network-topology.type configuration property. (#1500)

  • +
  • Add support for SphericalGeography to ST_Length(). (#1551)

  • +
+
+
+

Security#

+
    +
  • Allow configuring read-only access in System access control. (#1153)

  • +
  • Add missing checks for schema create, rename, and drop in file-based SystemAccessControl. (#1153)

  • +
  • Allow authentication over HTTP for forwarded requests containing the +X-Forwarded-Proto header. This is disabled by default, but can be enabled using the +http-server.authentication.allow-forwarded-https configuration property. (#1442)

  • +
+
+
+

Web UI#

+
    +
  • Fix rendering bug in Query Timeline resulting in inconsistency of presented information after +query finishes. (#1371)

  • +
  • Show total memory in Query Timeline instead of user memory. (#1371)

  • +
+
+
+

CLI#

+
    +
  • Add --insecure option to skip validation of server certificates for debugging. (#1484)

  • +
+
+
+

Hive connector#

+
    +
  • Fix reading from information_schema, as well as SHOW SCHEMAS, SHOW TABLES, and +SHOW COLUMNS when connecting to a Hive 3.x metastore that contains an information_schema +schema. (#1192)

  • +
  • Improve performance when reading data from GCS. (#1443)

  • +
  • Allow accessing tables in Glue metastore that do not have a table type. (#1343)

  • +
  • Add support for Azure Data Lake (adl) file system. (#1499)

  • +
  • Allow using custom S3 file systems by relying on the default Hadoop configuration by specifying +HADOOP_DEFAULT for the hive.s3-file-system-type configuration property. (#1397)

  • +
  • Add support for instance credentials for the Glue metastore via the +hive.metastore.glue.use-instance-credentials configuration property. (#1363)

  • +
  • Add support for custom credentials providers for the Glue metastore via the +hive.metastore.glue.aws-credentials-provider configuration property. (#1363)

  • +
  • Do not require setting the hive.metastore-refresh-interval configuration property +when enabling metastore caching. (#1473)

  • +
  • Add textfile_field_separator and textfile_field_separator_escape table properties +to support custom field separators for TEXTFILE format tables. (#1439)

  • +
  • Add $file_size and $file_modified_time hidden columns. (#1428)

  • +
  • The hive.metastore-timeout configuration property is now accepted only when using the +Thrift metastore. Previously, it was accepted for other metastore type, but was +ignored. (#1346)

  • +
  • Disallow reads from transactional tables. Previously, reads would appear to work, +but would not return any data. (#1218)

  • +
  • Disallow writes to transactional tables. Previously, writes would appear to work, +but the data would be written incorrectly. (#1218)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-320.html b/430/release/release-320.html new file mode 100644 index 000000000..be15fbdc2 --- /dev/null +++ b/430/release/release-320.html @@ -0,0 +1,2645 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 320 (10 Oct 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 320 (10 Oct 2019)#

+
+

General#

+
    +
  • Fix incorrect parameter binding order for prepared statement execution when +parameters appear inside a WITH clause. (#1191)

  • +
  • Fix planning failure for certain queries involving a mix of outer and +cross joins. (#1589)

  • +
  • Improve performance of queries containing complex predicates. (#1515)

  • +
  • Avoid unnecessary evaluation of redundant filters. (#1516)

  • +
  • Improve performance of certain window functions when using bounded window +frames (e.g., ROWS BETWEEN ... PRECEDING AND ... FOLLOWING). (#464)

  • +
  • Add Kinesis connector. (#476)

  • +
  • Add geometry_from_hadoop_shape(). (#1593)

  • +
  • Add at_timezone(). (#1612)

  • +
  • Add with_timezone(). (#1612)

  • +
+
+
+

JDBC driver#

+
    +
  • Only report warnings on Statement, not ResultSet, as warnings +are not associated with reads of the ResultSet. (#1640)

  • +
+
+
+

CLI#

+
    +
  • Add multi-line editing and syntax highlighting. (#1380)

  • +
+
+
+

Hive connector#

+
    +
  • Add impersonation support for calls to the Hive metastore. This can be enabled using the +hive.metastore.thrift.impersonation.enabled configuration property. (#43)

  • +
  • Add caching support for Glue metastore. (#1625)

  • +
  • Add separate configuration property hive.hdfs.socks-proxy for accessing HDFS via a +SOCKS proxy. Previously, it was controlled with the hive.metastore.thrift.client.socks-proxy +configuration property. (#1469)

  • +
+
+
+

MySQL connector#

+
    +
  • Add mysql.jdbc.use-information-schema configuration property to control whether +the MySQL JDBC driver should use the MySQL information_schema to answer metadata +queries. This may be helpful when diagnosing problems. (#1598)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Add support for reading PostgreSQL system tables, e.g., pg_catalog relations. +The functionality is disabled by default and can be enabled using the +postgresql.include-system-tables configuration property. (#1527)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Add support for VARBINARY, TIMESTAMP, TINYINT, SMALLINT, +and REAL data types. (#1639)

  • +
  • Discover available tables and their schema dynamically. (#1639)

  • +
  • Add support for special _id, _score and _source columns. (#1639)

  • +
  • Add support for full text queries. (#1662)

  • +
+
+
+

SPI#

+
    +
  • Introduce a builder for Identity and deprecate its public constructors. (#1624)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-321.html b/430/release/release-321.html new file mode 100644 index 000000000..0e074d8f5 --- /dev/null +++ b/430/release/release-321.html @@ -0,0 +1,2644 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 321 (15 Oct 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+ +
+ +
+
+ +
+

Release 321 (15 Oct 2019)#

+
+

Warning

+

The server RPM is broken in this release.

+
+
+

General#

+
    +
  • Fix incorrect result of round() when applied to a tinyint, smallint, +integer, or bigint type with negative decimal places. (#42)

  • +
  • Improve performance of queries with LIMIT over information_schema tables. (#1543)

  • +
  • Improve performance for broadcast joins by using dynamic filtering. This can be enabled +via the experimental.enable-dynamic-filtering configuration option or the +enable_dynamic_filtering session property. (#1686)

  • +
+
+
+

Security#

+
    +
  • Improve the security of query results with one-time tokens. (#1654)

  • +
+
+
+

Hive connector#

+
    +
  • Fix reading TEXT file collection delimiter set by Hive versions earlier +than 3.0. (#1714)

  • +
  • Fix a regression that prevented Presto from using the AWS Glue metastore. (#1698)

  • +
  • Allow skipping header or footer lines for CSV format tables via the +skip_header_line_count and skip_footer_line_count table properties. (#1090)

  • +
  • Rename table property textfile_skip_header_line_count to skip_header_line_count +and textfile_skip_footer_line_count to skip_footer_line_count. (#1090)

  • +
  • Add support for LZOP compressed (.lzo) files. Previously, queries accessing LZOP compressed +files would fail, unless all files were small. (#1701)

  • +
  • Add support for bucket-aware read of tables using bucketing version 2. (#538)

  • +
  • Add support for writing to tables using bucketing version 2. (#538)

  • +
  • Allow caching directory listings for all tables or schemas. (#1668)

  • +
  • Add support for dynamic filtering for broadcast joins. (#1686)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Support reading PostgreSQL arrays as the JSON data type. This can be enabled by +setting the postgresql.experimental.array-mapping configuration property or the +array_mapping catalog session property to AS_JSON. (#682)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Add support for Amazon Elasticsearch Service. (#1693)

  • +
+
+
+

Cassandra connector#

+
    +
  • Add TLS support. (#1680)

  • +
+
+
+

JMX connector#

+
    +
  • Add support for wildcards in configuration of history tables. (#1572)

  • +
+
+
+

SPI#

+
    +
  • Fix QueryStatistics.getWallTime() to report elapsed time rather than total +scheduled time. (#1719)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-322.html b/430/release/release-322.html new file mode 100644 index 000000000..5a025f6fa --- /dev/null +++ b/430/release/release-322.html @@ -0,0 +1,2577 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 322 (16 Oct 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 322 (16 Oct 2019)#

+
+

General#

+
    +
  • Improve performance of certain join queries by reducing the amount of data +that needs to be scanned. (#1673)

  • +
+
+
+

Server RPM#

+
    +
  • Fix a regression that caused zero-length files in the RPM. (#1767)

  • +
+
+
+

Other connectors#

+

These changes apply to MySQL, PostgreSQL, Redshift, and SQL Server.

+
    +
  • Add support for providing credentials using a keystore file. This can be enabled +by setting the credential-provider.type configuration property to KEYSTORE +and by setting the keystore-file-path, keystore-type, keystore-password, +keystore-user-credential-password, keystore-password-credential-password, +keystore-user-credential-name, and keystore-password-credential-name +configuration properties. (#1521)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-323.html b/430/release/release-323.html new file mode 100644 index 000000000..88c750adb --- /dev/null +++ b/430/release/release-323.html @@ -0,0 +1,2624 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 323 (23 Oct 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 323 (23 Oct 2019)#

+
+

General#

+
    +
  • Fix query failure when referencing columns from a table that contains +hidden columns. (#1796)

  • +
  • Fix a rare issue in which the server produces an extra row containing +the boolean value true as the last row in the result set. For most queries, +this will result in a client error, since this row does not match the result +schema, but is a correctness issue when the result schema is a single boolean +column. (#1732)

  • +
  • Allow using .* on expressions of type ROW in the SELECT clause to +convert the fields of a row into multiple columns. (#1017)

  • +
+
+
+

JDBC driver#

+
    +
  • Fix a compatibility issue when connecting to pre-321 servers. (#1785)

  • +
  • Fix reporting of views in DatabaseMetaData.getTables(). (#1488)

  • +
+
+
+

CLI#

+
    +
  • Fix a compatibility issue when connecting to pre-321 servers. (#1785)

  • +
+
+
+

Hive#

+
    +
  • Fix the ORC writer to correctly write the file footers. Previously written files were +sometimes unreadable in Hive 3.1 when querying the table for a second (or subsequent) +time. (#456)

  • +
  • Prevent writing to materialized views. (#1725)

  • +
  • Reduce metastore load when inserting data or analyzing tables. (#1783, #1793, #1794)

  • +
  • Allow using multiple Hive catalogs that use different Kerberos or other authentication +configurations. (#760, #978, #1820)

  • +
+
+
+

PostgreSQL#

+
    +
  • Support for PostgreSQL arrays is no longer considered experimental, therefore +the configuration property postgresql.experimental.array-mapping is now named +to postgresql.array-mapping. (#1740)

  • +
+
+
+

SPI#

+
    +
  • Add support for unnesting dictionary blocks duration compaction. (#1761)

  • +
  • Change LazyBlockLoader to directly return the loaded block. (#1744)

  • +
+
+

Note

+

This is a backwards incompatible changes with the previous SPI. +If you have written a plugin that instantiates LazyBlock, +you will need to update your code before deploying this release.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-324.html b/430/release/release-324.html new file mode 100644 index 000000000..9111b9b2a --- /dev/null +++ b/430/release/release-324.html @@ -0,0 +1,2613 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 324 (1 Nov 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 324 (1 Nov 2019)#

+
+

General#

+
    +
  • Fix query failure when CASE operands have different types. (#1825)

  • +
  • Add support for ESCAPE clause in SHOW CATALOGS LIKE .... (#1691)

  • +
  • Add line_interpolate_point() and line_interpolate_points(). (#1888)

  • +
  • Allow references to tables in the enclosing query when using .*. (#1867)

  • +
  • Configuration properties for optimizer and spill support no longer +have experimental. prefix. (#1875)

  • +
  • Configuration property experimental.reserved-pool-enabled was renamed to +experimental.reserved-pool-disabled (with meaning reversed). (#1916)

  • +
+
+
+

Security#

+
    +
  • Perform access control checks when displaying table or view definitions +with SHOW CREATE. (#1517)

  • +
+
+
+

Hive#

+
    +
  • Allow using SHOW GRANTS on a Hive view when using the sql-standard +security mode. (#1842)

  • +
  • Improve performance when filtering dictionary-encoded Parquet columns. (#1846)

  • +
+
+
+

PostgreSQL#

+
    +
  • Add support for inserting MAP(VARCHAR, VARCHAR) values into columns of +hstore type. (#1894)

  • +
+
+
+

Elasticsearch#

+
    +
  • Fix failure when reading datetime columns in Elasticsearch 5.x. (#1844)

  • +
  • Add support for mixed-case field names. (#1914)

  • +
+
+
+

SPI#

+
    +
  • Introduce a builder for ColumnMetadata. The various overloaded constructors +are now deprecated. (#1891)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-325.html b/430/release/release-325.html new file mode 100644 index 000000000..59b5e9de9 --- /dev/null +++ b/430/release/release-325.html @@ -0,0 +1,2605 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 325 (14 Nov 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 325 (14 Nov 2019)#

+
+

Warning

+

There is a performance regression in this release.

+
+
+

General#

+
    +
  • Fix incorrect results for certain queries involving FULL or RIGHT joins and +LATERAL. (#1952)

  • +
  • Fix incorrect results when using IS DISTINCT FROM on columns of DECIMAL type +with precision larger than 18. (#1985)

  • +
  • Fix query failure when row types contain a field named after a reserved SQL keyword. (#1963)

  • +
  • Add support for LIKE predicate to SHOW SESSION and SHOW FUNCTIONS. (#1688, #1692)

  • +
  • Add support for late materialization to join operations. (#1256)

  • +
  • Reduce number of metadata queries during planning. +This change disables stats collection for non-EXPLAIN queries. If you +want to have access to such stats and cost in query completion events, you +need to re-enable stats collection using the collect-plan-statistics-for-all-queries +configuration property. (#1866)

  • +
  • Add variant of strpos() that returns the Nth occurrence of a substring. (#1811)

  • +
  • Add to_encoded_polyline() and from_encoded_polyline() geospatial functions. (#1827)

  • +
+
+
+

Web UI#

+
    +
  • Show actual query for an EXECUTE statement. (#1980)

  • +
+
+
+

Hive#

+
    +
  • Fix incorrect behavior of CREATE TABLE when Hive metastore is configured +with metastore.create.as.acid set to true. (#1958)

  • +
  • Fix query failure when reading Parquet files that contain character data without statistics. (#1955)

  • +
  • Allow analyzing a subset of table columns (rather than all columns). (#1907)

  • +
  • Support overwriting unpartitioned tables for insert queries when using AWS Glue. (#1243)

  • +
  • Add support for reading Parquet files where the declared precision of decimal columns does not match +the precision in the table or partition schema. (#1949)

  • +
  • Improve performance when reading Parquet files with small row groups. (#1925)

  • +
+
+
+

Other connectors#

+

These changes apply to the MySQL, PostgreSQL, Redshift, and SQL Server connectors.

+
    +
  • Fix incorrect insertion of data when the target table has an unsupported type. (#1930)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-326.html b/430/release/release-326.html new file mode 100644 index 000000000..df62c098c --- /dev/null +++ b/430/release/release-326.html @@ -0,0 +1,2613 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 326 (27 Nov 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 326 (27 Nov 2019)#

+
+

General#

+
    +
  • Fix incorrect query results when query contains LEFT JOIN over UNNEST. (#2097)

  • +
  • Fix performance regression in queries involving JOIN. (#2047)

  • +
  • Fix accounting of semantic analysis time when queued queries are cancelled. (#2055)

  • +
  • Add SingleStore connector. (#1906)

  • +
  • Improve performance of INSERT and CREATE TABLE ... AS queries containing redundant +ORDER BY clauses. (#2044)

  • +
  • Improve performance when processing columns of map type. (#2015)

  • +
+
+
+

Server RPM#

+ +
+
+

Security#

+ +
+
+

Hive#

+
    +
  • Fix table creation error for tables with S3 location when using file metastore. (#1664)

  • +
  • Fix a compatibility issue with the CDH 5.x metastore which results in stats +not being recorded for ANALYZE. (#973)

  • +
  • Improve performance for Glue metastore by fetching partitions in parallel. (#1465)

  • +
  • Improve performance of sql-standard security. (#1922, #1929)

  • +
+
+
+

Phoenix connector#

+
    +
  • Collect statistics on the count and duration of each call to Phoenix. (#2024)

  • +
+
+
+

Other connectors#

+

These changes apply to the MySQL, PostgreSQL, Redshift, and SQL Server connectors.

+
    +
  • Collect statistics on the count and duration of operations to create +and destroy JDBC connections. (#2024)

  • +
  • Add support for showing column comments. (#1840)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-327.html b/430/release/release-327.html new file mode 100644 index 000000000..0b293f1dd --- /dev/null +++ b/430/release/release-327.html @@ -0,0 +1,2669 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 327 (20 Dec 2019) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 327 (20 Dec 2019)#

+
+

General#

+
    +
  • Fix join query failure when late materialization is enabled. (#2144)

  • +
  • Fix failure of word_stem() for certain inputs. (#2145)

  • +
  • Fix query failure when using transform_values() inside try() and the transformation fails +for one of the rows. (#2315)

  • +
  • Fix potential incorrect results for aggregations involving FILTER (WHERE ...) +when the condition is a reference to a table column. (#2267)

  • +
  • Allow renaming views with ALTER VIEW. (#1060)

  • +
  • Add error_type and error_code columns to system.runtime.queries. (#2249)

  • +
  • Rename experimental.work-processor-pipelines configuration property to experimental.late-materialization.enabled +and rename work_processor_pipelines session property to late_materialization. (#2275)

  • +
+
+
+

Security#

+ +
+
+

Hive connector#

+
    +
  • Fix incorrect query results when reading timestamp values from ORC files written by +Hive 3.1 or later. (#2099)

  • +
  • Fix a CDH 5.x metastore compatibility issue resulting in failure when analyzing or inserting +into a table with date columns. (#556)

  • +
  • Reduce number of metastore calls when fetching partitions. (#1921)

  • +
  • Support reading from insert-only transactional tables. (#576)

  • +
  • Deprecate parquet.fail-on-corrupted-statistics (previously known as hive.parquet.fail-on-corrupted-statistics). +Setting this configuration property to false may hide correctness issues, leading to incorrect query results. +Session property parquet_fail_with_corrupted_statistics is deprecated as well. +Both configuration and session properties will be removed in a future version. (#2129)

  • +
  • Improve concurrency when updating table or partition statistics. (#2154)

  • +
  • Add support for renaming views. (#2189)

  • +
  • Allow configuring the hive.orc.use-column-names config property on a per-session +basis using the orc_use_column_names session property. (#2248)

  • +
+
+
+

Kudu connector#

+
    +
  • Support predicate pushdown for the decimal type. (#2131)

  • +
  • Fix column position swap for delete operations that may result in deletion of the wrong records. (#2252)

  • +
  • Improve predicate pushdown for queries that match a column against +multiple values (typically using the IN operator). (#2253)

  • +
+
+
+

MongoDB connector#

+
    +
  • Add support for reading from views. (#2156)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Allow converting unsupported types to VARCHAR by setting the session property +unsupported_type_handling or configuration property unsupported-type-handling +to CONVERT_TO_VARCHAR. (#1182)

  • +
+
+
+

MySQL connector#

+
    +
  • Fix INSERT query failure when GTID mode is enabled. (#2251)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Improve performance for queries involving equality and range filters +over table columns. (#2310)

  • +
+
+
+

Google Sheets connector#

+
    +
  • Fix incorrect results when listing tables in information_schema. (#2118)

  • +
+
+
+

SPI#

+
    +
  • Add executionTime to QueryStatistics for event listeners. (#2247)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-328.html b/430/release/release-328.html new file mode 100644 index 000000000..9fb471ae2 --- /dev/null +++ b/430/release/release-328.html @@ -0,0 +1,2658 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 328 (10 Jan 2020) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 328 (10 Jan 2020)#

+
+

General#

+
    +
  • Fix correctness issue for certain correlated join queries when the correlated subquery on +the right produces no rows. (#1969)

  • +
  • Fix incorrect handling of multi-byte characters for Regular expression functions when +the pattern is empty. (#2313)

  • +
  • Fix failure when join criteria contains columns of different types. (#2320)

  • +
  • Fix failure for complex outer join queries when dynamic filtering is enabled. (#2363)

  • +
  • Improve support for correlated queries. (#1969)

  • +
  • Allow inserting values of a larger type into as smaller type when the values fit. For example, +BIGINT into SMALLINT, or VARCHAR(10) into VARCHAR(3). Values that don’t fit will +cause an error at runtime. (#2061)

  • +
  • Add regexp_count() and regexp_position() functions. (#2136)

  • +
  • Add support for interpolating Secrets in server and catalog configuration +files. (#2370)

  • +
+
+
+

Security#

+
    +
  • Fix a security issue allowing users to gain unauthorized access to Presto cluster +when using password authenticator with LDAP. (#2356)

  • +
  • Add support for LDAP referrals in LDAP password authenticator. (#2354)

  • +
+
+
+

JDBC driver#

+
    +
  • Fix behavior of java.sql.Connection#commit() and java.sql.Connection#rollback() +methods when no statements performed in a transaction. Previously, these methods +would fail. (#2339)

  • +
  • Fix failure when restoring autocommit mode with +java.sql.Connection#setAutocommit() (#2338)

  • +
+
+
+

Hive connector#

+
    +
  • Reduce query latency and Hive metastore load when using the +AUTOMATIC join reordering strategy. (#2184)

  • +
  • Allow configuring hive.max-outstanding-splits-size to values larger than 2GB. (#2395)

  • +
  • Avoid redundant file system stat call when writing Parquet files. (#1746)

  • +
  • Avoid retrying permanent errors for S3-related services such as STS. (#2331)

  • +
+
+
+

Kafka connector#

+
    +
  • Remove internal columns: _segment_start, _segment_end and +_segment_count. (#2303)

  • +
  • Add new configuration property kafka.messages-per-split to control how many Kafka +messages will be processed by a single Presto split. (#2303)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Fix query failure when an object in an Elasticsearch document +does not have any fields. (#2217)

  • +
  • Add support for querying index aliases. (#2324)

  • +
+
+
+

Phoenix connector#

+
    +
  • Add support for mapping unsupported data types to VARCHAR. This can be enabled by setting +the unsupported-type-handling configuration property or the unsupported_type_handling session +property to CONVERT_TO_VARCHAR. (#2427)

  • +
+
+
+

Other connectors#

+

These changes apply to the MySQL, PostgreSQL, Redshift and SQL Server connectors:

+
    +
  • Add support for creating schemas. (#1874)

  • +
  • Add support for caching metadata. The configuration property metadata.cache-ttl +controls how long to cache data (it defaults to 0ms which disables caching), +and metadata.cache-missing controls whether or not missing tables are cached. (#2290)

  • +
+

This change applies to the MySQL and PostgreSQL connectors:

+
    +
  • Add support for mapping DECIMAL types with precision larger than 38 +to Presto DECIMAL. (#2088)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-329.html b/430/release/release-329.html new file mode 100644 index 000000000..00e7dbf8c --- /dev/null +++ b/430/release/release-329.html @@ -0,0 +1,2634 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 329 (23 Jan 2020) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 329 (23 Jan 2020)#

+
+

General#

+
    +
  • Fix incorrect result for last_day_of_month() function for first day of month. (#2452)

  • +
  • Fix incorrect results when handling DOUBLE or REAL types with NaN values. (#2582)

  • +
  • Fix query failure when coordinator hostname contains underscores. (#2571)

  • +
  • Fix SHOW CREATE TABLE failure when row types contain a field named after a +reserved SQL keyword. (#2130)

  • +
  • Handle common disk failures during spill. When one disk fails but multiple +spill locations are configured, the healthy disks will be used for future queries. +(#2444)

  • +
  • Improve performance and reduce load on external systems when +querying information_schema. (#2488)

  • +
  • Improve performance of queries containing redundant scalar subqueries. (#2456)

  • +
  • Limit broadcasted table size to 100MB by default when using the AUTOMATIC +join type selection strategy. This avoids query failures or excessive memory usage when joining two or +more very large tables. (#2527)

  • +
  • Enable cost based join reordering and join type selection +optimizations by default. The previous behavior can be restored by +setting optimizer.join-reordering-strategy configuration property to ELIMINATE_CROSS_JOINS +and join-distribution-type to PARTITIONED. (#2528)

  • +
  • Hide non-standard columns comment and extra_info in the standard +information_schema.columns table. These columns can still be selected, +but will no longer appear when describing the table. (#2306)

  • +
+
+
+

Security#

+
    +
  • Add ldap.bind-dn and ldap.bind-password LDAP properties to allow LDAP authentication +access LDAP server using service account. (#1917)

  • +
+
+
+

Hive connector#

+
    +
  • Fix incorrect data returned when using S3 Select on uncompressed files. In our testing, S3 Select +was apparently returning incorrect results when reading uncompressed files, so S3 Select is disabled +for uncompressed files. (#2399)

  • +
  • Fix incorrect data returned when using S3 Select on a table with skip.header.line.count or +skip.footer.line.count property. S3 Select API does not support skipping footers or more than one +line of a header. In our testing, S3 Select was apparently sometimes returning incorrect results when +reading a compressed file with header skipping, so S3 Select is disabled when any of these table +properties is set to non-zero value. (#2399)

  • +
  • Fix query failure for writes when one of the inserted REAL or DOUBLE values +is infinite or NaN. (#2471)

  • +
  • Fix performance degradation reading from S3 when the Kinesis connector is installed. (#2496)

  • +
  • Allow reading data from Parquet files when the column type is declared as INTEGER +in the table or partition, but is a DECIMAL type in the file. (#2451)

  • +
  • Validate the scale of decimal types when reading Parquet files. This prevents +incorrect results when the decimal scale in the file does not match the declared +type for the table or partition. (#2451)

  • +
  • Delete storage location when dropping an empty schema. (#2463)

  • +
  • Improve performance when deleting multiple partitions by executing these actions concurrently. (#1812)

  • +
  • Improve performance for queries containing IN predicates over bucketing columns. (#2277)

  • +
  • Add procedure system.drop_stats() to remove the column statistics +for a table or selected partitions. (#2538)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Add support for Array types. (#2441)

  • +
  • Reduce load on Elasticsearch cluster and improve query performance. (#2561)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Fix mapping between PostgreSQL’s TIME and Presto’s TIME data types. +Previously the mapping was incorrect, shifting it by the relative offset between the session +time zone and the Presto server’s JVM time zone. (#2549)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-330.html b/430/release/release-330.html new file mode 100644 index 000000000..bdb77ccae --- /dev/null +++ b/430/release/release-330.html @@ -0,0 +1,2731 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 330 (18 Feb 2020) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+ +
+ +
+
+ +
+

Release 330 (18 Feb 2020)#

+
+

General#

+
    +
  • Fix incorrect behavior of format() for char values. Previously, the function +did not preserve trailing whitespace of the value being formatted. (#2629)

  • +
  • Fix query failure in some cases when aggregation uses inputs from both sides of a join. (#2560)

  • +
  • Fix query failure when dynamic filtering is enabled and the query contains complex +multi-level joins. (#2659)

  • +
  • Fix query failure for certain co-located joins when dynamic filtering is enabled. (#2685)

  • +
  • Fix failure of SHOW statements or queries that access information_schema schema tables +with an empty value used in a predicate. (#2575)

  • +
  • Fix query failure when EXECUTE is used with an expression containing a function call. (#2675)

  • +
  • Fix failure in SHOW CATALOGS when the user does not have permissions to see any catalogs. (#2593)

  • +
  • Improve query performance for some join queries when Cost-based optimizations +are enabled. (#2722)

  • +
  • Prevent uneven distribution of data that can occur when writing data with redistribution or writer +scaling enabled. (#2788)

  • +
  • Add support for CREATE VIEW with comment (#2557)

  • +
  • Add support for all major geometry types to ST_Points(). (#2535)

  • +
  • Add required_workers_count and required_workers_max_wait_time session properties +to control the number of workers that must be present in the cluster before query +processing starts. (#2484)

  • +
  • Add physical_input_bytes column to system.runtime.tasks table. (#2803)

  • +
  • Verify that the target schema exists for the USE statement. (#2764)

  • +
  • Verify that the session catalog exists when executing SET ROLE. (#2768)

  • +
+
+
+

Server#

+
    +
  • Require running on Java 11 or above. This requirement may be temporarily relaxed by adding +-Dpresto-temporarily-allow-java8=true to the Presto JVM config. +This fallback will be removed in future versions of Presto after March 2020. (#2751)

  • +
  • Add experimental support for running on Linux aarch64 (ARM64). (#2809)

  • +
+
+
+

Security#

+
    +
  • Principal rules are deprecated and will be removed in a future release. +These rules have been replaced with User mapping, which +specifies how a complex authentication user name is mapped to a simple +user name for Presto, and Impersonation rules which +control the ability of a user to impersonate another user. (#2215)

  • +
  • A shared secret is now required when using Secure internal communication. (#2202)

  • +
  • Kerberos for Secure internal communication has been replaced with the new shared secret mechanism. +The internal-communication.kerberos.enabled and internal-communication.kerberos.use-canonical-hostname +configuration properties must be removed. (#2202)

  • +
  • When authentication is disabled, the Presto user may now be set using standard +HTTP basic authentication with an empty password. (#2653)

  • +
+
+
+

Web UI#

+
    +
  • Display physical read time in detailed query view. (#2805)

  • +
+
+
+

JDBC driver#

+
    +
  • Fix a performance issue on JDK 11+ when connecting using HTTP/2. (#2633)

  • +
  • Implement PreparedStatement.setTimestamp() variant that takes a Calendar. (#2732)

  • +
  • Add roles property for catalog authorization roles. (#2780)

  • +
  • Add sessionProperties property for setting system and catalog session properties. (#2780)

  • +
  • Add clientTags property to set client tags for selecting resource groups. (#2468)

  • +
  • Allow using the : character within an extra credential value specified via the +extraCredentials property. (#2780)

  • +
+
+
+

CLI#

+
    +
  • Fix a performance issue on JDK 11+ when connecting using HTTP/2. (#2633)

  • +
+
+
+

Cassandra connector#

+
    +
  • Fix query failure when identifiers should be quoted. (#2455)

  • +
+
+
+

Hive connector#

+
    +
  • Fix reading symlinks from HDFS when using Kerberos. (#2720)

  • +
  • Reduce Hive metastore load when updating partition statistics. (#2734)

  • +
  • Allow redistributing writes for un-bucketed partitioned tables on the +partition keys, which results in a single writer per partition. This reduces +memory usage, results in a single file per partition, and allows writing a +large number of partitions (without hitting the open writer limit). However, +writing large partitions with a single writer can take substantially longer, so +this feature should only be enabled when required. To enable this feature, set the +use-preferred-write-partitioning system configuration property or the +use_preferred_write_partitioning system session property to true. (#2358)

  • +
  • Remove extra file status call after writing text-based, SequenceFile, or Avro file types. (#1748)

  • +
  • Allow using writer scaling with all file formats. Previously, it was not supported for +text-based, SequenceFile, or Avro formats. (#2657)

  • +
  • Add support for symlink-based tables with Avro files. (#2720)

  • +
  • Add support for ignoring partitions with a non-existent data directory. This can be configured +using the hive.ignore-absent-partitions=true configuration property or the +ignore_absent_partitions session property. (#2555)

  • +
  • Allow creation of external tables with data via CREATE TABLE AS when +both hive.non-managed-table-creates-enabled and hive.non-managed-table-writes-enabled +are set to true. Previously this required executing CREATE TABLE and INSERT +as separate statement (#2669)

  • +
  • Add support for Azure WASB, ADLS Gen1 (ADL) and ADLS Gen2 (ABFS) file systems. (#2494)

  • +
  • Add experimental support for executing basic Hive views. To enable this feature, the +hive.views-execution.enabled configuration property must be set to true. (#2715)

  • +
  • Add register_partition and unregister_partition +procedures for adding partitions to and removing partitions from a partitioned table. (#2692)

  • +
  • Allow running ANALYZE collecting only basic table statistics. (#2762)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Improve performance of queries containing a LIMIT clause. (#2781)

  • +
  • Add support for nested data type. (#754)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Add read support for PostgreSQL money data type. The type is mapped to varchar in Presto. +(#2601)

  • +
+
+
+

Other connectors#

+

These changes apply to the MySQL, PostgreSQL, Redshift, Phoenix and SQL Server connectors.

+
    +
  • Respect DEFAULT column clause when writing to a table. (#1185)

  • +
+
+
+

SPI#

+
    +
  • Allow procedures to have optional arguments with default values. (#2706)

  • +
  • SystemAccessControl.checkCanSetUser() is deprecated and has been replaced +with User mapping and SystemAccessControl.checkCanImpersonateUser(). (#2215)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-331.html b/430/release/release-331.html new file mode 100644 index 000000000..f45fbcbb8 --- /dev/null +++ b/430/release/release-331.html @@ -0,0 +1,2679 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 331 (16 Mar 2020) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 331 (16 Mar 2020)#

+
+

General#

+
    +
  • Prevent query failures when worker is shut down gracefully. (#2648)

  • +
  • Fix join failures for queries involving OR predicate with non-comparable functions. (#2861)

  • +
  • Ensure query completed event is fired when there is an error during analysis or planning. (#2842)

  • +
  • Fix memory accounting for ORDER BY queries. (#2612)

  • +
  • Fix last_day_of_month() for timestamp with time zone values. (#2851)

  • +
  • Fix excessive runtime when parsing deeply nested expressions with unmatched parenthesis. (#2968)

  • +
  • Correctly reject date literals that cannot be represented in Presto. (#2888)

  • +
  • Improve query performance by removing redundant data reshuffling. (#2853)

  • +
  • Improve performance of inequality joins involving BETWEEN. (#2859)

  • +
  • Improve join performance for dictionary encoded data. (#2862)

  • +
  • Enable dynamic filtering by default. (#2793)

  • +
  • Show reorder join cost in EXPLAIN ANALYZE VERBOSE (#2725)

  • +
  • Allow configuring resource groups selection based on user’s groups. (#3023)

  • +
  • Add SET AUTHORIZATION action to ALTER SCHEMA. (#2673)

  • +
  • Add BigQuery connector. (#2532)

  • +
  • Add support for large prepared statements. (#2719)

  • +
+
+
+

Security#

+
    +
  • Remove unused internal-communication.jwt.enabled configuration property. (#2709)

  • +
  • Rename JWT configuration properties from http.authentication.jwt.* to http-server.authentication.jwt.*. (#2712)

  • +
  • Add access control checks for query execution, view query, and kill query. This can be +configured using Query rules in File-based access control. (#2213)

  • +
  • Hide columns of tables for which the user has no privileges in File-based access control. (#2925)

  • +
+
+
+

JDBC driver#

+
    +
  • Implement PreparedStatement.getMetaData(). (#2770)

  • +
+
+
+

Web UI#

+
    +
  • Fix copying worker address to clipboard. (#2865)

  • +
  • Fix copying query ID to clipboard. (#2872)

  • +
  • Fix display of data size values. (#2810)

  • +
  • Fix redirect from / to /ui/ when Presto is behind a proxy. (#2908)

  • +
  • Fix display of prepared queries. (#2784)

  • +
  • Display physical input read rate. (#2873)

  • +
  • Add simple form based authentication that utilizes the configured password authenticator. (#2755)

  • +
  • Allow disabling the UI via the web-ui.enabled configuration property. (#2755)

  • +
+
+
+

CLI#

+
    +
  • Fix formatting of varbinary in nested data types. (#2858)

  • +
  • Add --timezone parameter. (#2961)

  • +
+
+
+

Hive connector#

+
    +
  • Fix incorrect results for reads from information_schema tables and +metadata queries when using a Hive 3.x metastore. (#3008)

  • +
  • Fix query failure when using Glue metastore and the table storage descriptor has no properties. (#2905)

  • +
  • Fix deadlock when Hive caching is enabled and has a refresh interval configured. (#2984)

  • +
  • Respect bucketing_version table property when using Glue metastore. (#2905)

  • +
  • Improve performance of partition fetching from Glue. (#3024)

  • +
  • Add support for bucket sort order in Glue when creating or updating a table or partition. (#1870)

  • +
  • Add support for Hive full ACID tables. (#2068, #1591, #2790)

  • +
  • Allow data conversion when reading decimal data from Parquet files and precision or scale in the file schema +is different from the precision or scale in partition schema. (#2823)

  • +
  • Add option to enforce that a filter on a partition key be present in the query. This can be enabled by setting the +hive.query-partition-filter-required configuration property or the query_partition_filter_required session property +to true. (#2334)

  • +
  • Allow selecting the Intelligent-Tiering S3 storage class when writing data to S3. This can be enabled by +setting the hive.s3.storage-class configuration property to INTELLIGENT_TIERING. (#3032)

  • +
  • Hide the Hive system schema sys for security reasons. (#3008)

  • +
  • Add support for changing the owner of a schema. (#2673)

  • +
+
+
+

MongoDB connector#

+
    +
  • Fix incorrect results when queries contain filters on certain data types, such +as real or decimal. (#1781)

  • +
+
+
+

Other connectors#

+

These changes apply to the MemSQL, MySQL, PostgreSQL, Redshift, Phoenix, and SQL Server connectors.

+
    +
  • Add support for dropping schemas. (#2956)

  • +
+
+
+

SPI#

+
    +
  • Remove deprecated Identity constructors. (#2877)

  • +
  • Introduce a builder for ConnectorIdentity and deprecate its public constructors. (#2877)

  • +
  • Add support for row filtering and column masking via the getRowFilter() and getColumnMask() APIs in +SystemAccessControl and ConnectorAccessControl. (#1480)

  • +
  • Add access control check for executing procedures. (#2924)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-332.html b/430/release/release-332.html new file mode 100644 index 000000000..143adf0fc --- /dev/null +++ b/430/release/release-332.html @@ -0,0 +1,2704 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 332 (08 Apr 2020) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 332 (08 Apr 2020)#

+
+

General#

+
    +
  • Fix query failure during planning phase for certain queries involving multiple joins. (#3149)

  • +
  • Fix execution failure for queries involving large IN predicates on decimal values with precision larger than 18. (#3191)

  • +
  • Fix prepared statements or view creation for queries containing certain nested aliases or TABLESAMPLE clauses. (#3250)

  • +
  • Fix rare query failure. (#2981)

  • +
  • Ignore trailing whitespace when loading configuration files such as +etc/event-listener.properties or etc/group-provider.properties. +Trailing whitespace in etc/config.properties and catalog properties +files was already ignored. (#3231)

  • +
  • Reduce overhead for internal communication requests. (#3215)

  • +
  • Include filters over all table columns in output of EXPLAIN (TYPE IO). (#2743)

  • +
  • Support configuring multiple event listeners. The properties files for all the event listeners +can be specified using the event-listener.config-files configuration property. (#3128)

  • +
  • Add CREATE SCHEMA ... AUTHORIZATION syntax to create a schema with specified owner. (#3066).

  • +
  • Add optimizer.push-partial-aggregation-through-join configuration property to control +pushing partial aggregations through inner joins. Previously, this was only available +via the push_partial_aggregation_through_join session property. (#3205)

  • +
  • Rename configuration property optimizer.push-aggregation-through-join +to optimizer.push-aggregation-through-outer-join. (#3205)

  • +
  • Add operator statistics for the number of splits processed with a dynamic filter applied. (#3217)

  • +
+
+
+

Security#

+
    +
  • Fix LDAP authentication when user belongs to multiple groups. (#3206)

  • +
  • Verify access to table columns when running SHOW STATS. (#2665)

  • +
  • Only return views accessible to the user from information_schema.views. (#3290)

  • +
+
+
+

JDBC driver#

+
    +
  • Add clientInfo property to set extra information about the client. (#3188)

  • +
  • Add traceToken property to set a trace token for correlating requests across systems. (#3188)

  • +
+
+
+

BigQuery connector#

+
    +
  • Extract parent project ID from service account before looking at the environment. (#3131)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Add support for ip type. (#3347)

  • +
  • Add support for keyword fields with numeric values. (#3381)

  • +
  • Remove unnecessary elasticsearch.aws.use-instance-credentials configuration property. (#3265)

  • +
+
+
+

Hive connector#

+
    +
  • Fix failure reading certain Parquet files larger than 2GB. (#2730)

  • +
  • Improve performance when reading gzip-compressed Parquet data. (#3175)

  • +
  • Explicitly disallow reading from Delta Lake tables. Previously, reading +from partitioned tables would return zero rows, and reading from +unpartitioned tables would fail with a cryptic error. (#3366)

  • +
  • Add hive.fs.new-directory-permissions configuration property for setting the permissions of new directories +created by Presto. Default value is 0777, which corresponds to previous behavior. (#3126)

  • +
  • Add hive.partition-use-column-names configuration property and matching partition_use_column_names catalog +session property that allows to match columns between table and partition schemas by names. By default they are mapped +by index. (#2933)

  • +
  • Add support for CREATE SCHEMA ... AUTHORIZATION to create a schema with specified owner. (#3066).

  • +
  • Allow specifying the Glue metastore endpoint URL using the +hive.metastore.glue.endpoint-url configuration property. (#3239)

  • +
  • Add experimental file system caching. This can be enabled with the hive.cache.enabled configuration property. (#2679)

  • +
  • Support reading files compressed with newer versions of LZO. (#3209)

  • +
  • Add support for Alluxio Catalog Service. (#2116)

  • +
  • Remove unnecessary hive.metastore.glue.use-instance-credentials configuration property. (#3265)

  • +
  • Remove unnecessary hive.s3.use-instance-credentials configuration property. (#3265)

  • +
  • Add flexible S3 security mapping, allowing for separate credentials +or IAM roles for specific users or buckets/paths. (#3265)

  • +
  • Add support for specifying an External ID for an IAM role trust policy using +the hive.metastore.glue.external-id configuration property (#3144)

  • +
  • Allow using configured S3 credentials with IAM role. Previously, +the configured IAM role was silently ignored. (#3351)

  • +
+
+
+

Kudu connector#

+
    +
  • Fix incorrect column mapping in Kudu connector. (#3170, #2963)

  • +
  • Fix incorrect query result for certain queries involving IS NULL predicates with OR. (#3274)

  • +
+
+
+

Memory connector#

+
    +
  • Include views in the list of tables returned to the JDBC driver. (#3208)

  • +
+
+
+

MongoDB connector#

+
    +
  • Add objectid_timestamp for extracting the timestamp from ObjectId. (#3089)

  • +
  • Delete document from _schema collection when DROP TABLE +is executed for a table that exists only in _schema. (#3234)

  • +
+
+
+

SQL Server connector#

+
    +
  • Disallow renaming tables between schemas. Previously, such renames were allowed +but the schema name was ignored when performing the rename. (#3284)

  • +
+
+
+

SPI#

+
    +
  • Expose row filters and column masks in QueryCompletedEvent. (#3183)

  • +
  • Expose referenced functions and procedures in QueryCompletedEvent. (#3246)

  • +
  • Allow Connector to provide EventListener instances. (#3166)

  • +
  • Deprecate the ConnectorPageSourceProvider.createPageSource() variant without the +dynamicFilter parameter. The method will be removed in a future release. (#3255)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-333.html b/430/release/release-333.html new file mode 100644 index 000000000..93adbc5f1 --- /dev/null +++ b/430/release/release-333.html @@ -0,0 +1,2687 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 333 (04 May 2020) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 333 (04 May 2020)#

+
+

General#

+
    +
  • Fix planning failure when lambda expressions are repeated in a query. (#3218)

  • +
  • Fix failure when input to TRY is a constant NULL. (#3408)

  • +
  • Fix failure for SHOW CREATE TABLE for tables with +row types that contain special characters. (#3380)

  • +
  • Fix failure when using max_by() or min_by() +where the second argument is of type varchar. (#3424)

  • +
  • Fix rare failure due to an invalid size estimation for T-Digests. (#3625)

  • +
  • Do not require coordinator to have spill paths setup when spill is enabled. (#3407)

  • +
  • Improve performance when dynamic filtering is enabled. (#3413)

  • +
  • Improve performance of queries involving constant scalar subqueries (#3432)

  • +
  • Allow overriding the count of available workers used for query cost +estimation via the cost_estimation_worker_count session property. (#2705)

  • +
  • Add data integrity verification for Presto internal communication. This can be configured +with the exchange.data-integrity-verification configuration property. (#3438)

  • +
  • Add support for LIKE predicate to SHOW COLUMNS. (#2997)

  • +
  • Add SHOW CREATE SCHEMA. (#3099)

  • +
  • Add starts_with() function. (#3392)

  • +
+
+
+

Server#

+ +
+
+

Server RPM#

+
    +
  • Reduce size of RPM and disk usage after installation. (#3595)

  • +
+
+
+

Security#

+
    +
  • Allow configuring trust certificate for LDAP password authenticator. (#3523)

  • +
+
+
+

JDBC driver#

+
    +
  • Fix hangs on JDK 8u252 when using secure connections. (#3444)

  • +
+
+
+

BigQuery connector#

+
    +
  • Improve performance for queries that contain filters on table columns. (#3376)

  • +
  • Add support for partitioned tables. (#3376)

  • +
+
+
+

Cassandra connector#

+ +
+
+

Elasticsearch connector#

+
    +
  • Fix failure when querying Elasticsearch 7.x clusters. (#3447)

  • +
+
+
+

Hive connector#

+
    +
  • Fix incorrect query results when reading Parquet data with a varchar column predicate +which is a comparison with a value containing non-ASCII characters. (#3517)

  • +
  • Ensure cleanup of resources (file descriptors, sockets, temporary files, etc.) +when an error occurs while writing an ORC file. (#3390)

  • +
  • Generate multiple splits for files in bucketed tables. (#3455)

  • +
  • Make file system caching honor Hadoop properties from hive.config.resources. (#3557)

  • +
  • Disallow enabling file system caching together with S3 security mapping or GCS access tokens. (#3571)

  • +
  • Disable file system caching parallel warmup by default. +It is currently broken and should not be enabled. (#3591)

  • +
  • Include metrics from S3 Select in the S3 JMX metrics. (#3429)

  • +
  • Report timings for request retries in S3 JMX metrics. +Previously, only the first request was reported. (#3429)

  • +
  • Add S3 JMX metric for client retry pause time (how long the thread was asleep +between request retries in the client itself). (#3429)

  • +
  • Add support for SHOW CREATE SCHEMA. (#3099)

  • +
  • Add hive.projection-pushdown-enabled configuration property and +projection_pushdown_enabled session property. (#3490)

  • +
  • Add support for connecting to the Thrift metastore using TLS. (#3440)

  • +
+
+
+

MongoDB connector#

+
    +
  • Skip unknown types in nested BSON object. (#2935)

  • +
  • Fix query failure when the user does not have access privileges for system.views. (#3355)

  • +
+
+
+

Other connectors#

+

These changes apply to the MemSQL, MySQL, PostgreSQL, Redshift, and SQL Server connectors.

+
    +
  • Export JMX statistics for various connector operations. (#3479).

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-334.html b/430/release/release-334.html new file mode 100644 index 000000000..849f5a87c --- /dev/null +++ b/430/release/release-334.html @@ -0,0 +1,2682 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 334 (29 May 2020) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 334 (29 May 2020)#

+
+

General#

+
    +
  • Fix incorrect query results for certain queries involving comparisons of real and double types +when values include negative zero. (#3745)

  • +
  • Fix failure when querying an empty table with late materialization enabled. (#3577)

  • +
  • Fix failure when the inputs to UNNEST are repeated. (#3587)

  • +
  • Fix failure when an aggregation is used in the arguments to format(). (#3829)

  • +
  • Fix localtime() and current_time() for session zones with DST or with historical offset changes +in legacy (default) timestamp semantics. (#3846, #3850)

  • +
  • Fix dynamic filter failures in complex spatial join queries. (#3694)

  • +
  • Improve performance of queries involving row_number(). (#3614)

  • +
  • Improve performance of queries containing LIKE predicate. (#3618)

  • +
  • Improve query performance when dynamic filtering is enabled. (#3632)

  • +
  • Improve performance for queries that read fields from nested structures. (#2672)

  • +
  • Add variant of random() function that produces a number in the provided range. (#1848)

  • +
  • Show distributed plan by default in EXPLAIN. (#3724)

  • +
  • Add Oracle connector. (#1959)

  • +
  • Add Pinot connector. (#2028)

  • +
  • Add Prometheus connector. (#2321)

  • +
  • Add support for standards compliant (RFC 7239) HTTP forwarded headers. Processing of HTTP forwarded headers is now controlled by the +http-server.process-forwarded configuration property, and the old http-server.authentication.allow-forwarded-https and +dispatcher.forwarded-header configuration properties are no longer supported. (#3714)

  • +
  • Add pluggable Certificate authenticator. (#3804)

  • +
+
+
+

JDBC driver#

+
    +
  • Implement toString() for java.sql.Array results. (#3803)

  • +
+
+
+

CLI#

+
    +
  • Improve rendering of elapsed time for short queries. (#3311)

  • +
+
+
+

Web UI#

+
    +
  • Add fixed, certificate, JWT, and Kerberos to UI authentication. (#3433)

  • +
  • Show join distribution type in Live Plan. (#1323)

  • +
+
+
+

JDBC driver#

+
    +
  • Improve performance of DatabaseMetaData.getColumns() when the +parameters contain unescaped % or _. (#1620)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Fix failure when executing SHOW CREATE TABLE. (#3718)

  • +
  • Improve performance for count(*) queries. (#3512)

  • +
  • Add support for raw Elasticsearch queries. (#3735)

  • +
+
+
+

Hive connector#

+
    +
  • Fix matching bucket filenames without leading zeros. (#3702)

  • +
  • Fix creation of external tables using CREATE TABLE AS. Previously, the +tables were created as managed and with the default location. (#3755)

  • +
  • Fix incorrect table statistics for newly created external tables. (#3819)

  • +
  • Prevent Presto from starting when cache fails to initialize. (#3749)

  • +
  • Fix race condition that could cause caching to be permanently disabled. (#3729, #3810)

  • +
  • Fix malformed reads when asynchronous read mode for caching is enabled. (#3772)

  • +
  • Fix eviction of cached data while still under size eviction threshold. (#3772)

  • +
  • Improve performance when creating unpartitioned external tables over large data sets. (#3624)

  • +
  • Leverage Parquet file statistics when reading decimal columns. (#3581)

  • +
  • Change type of $file_modified_time hidden column from bigint to timestamp with timezone type. (#3611)

  • +
  • Add caching support for HDFS and Azure file systems. (#3772)

  • +
  • Fix S3 connection pool depletion when asynchronous read mode for caching is enabled. (#3772)

  • +
  • Disable caching on coordinator by default. (#3820)

  • +
  • Use asynchronous read mode for caching by default. (#3799)

  • +
  • Cache delegation token for Hive thrift metastore. This can be configured with +the hive.metastore.thrift.delegation-token.cache-ttl and hive.metastore.thrift.delegation-token.cache-maximum-size +configuration properties. (#3771)

  • +
+
+
+

MemSQL connector#

+ +
+
+

MongoDB connector#

+
    +
  • Support case insensitive database and collection names. This can be enabled with the +mongodb.case-insensitive-name-matching configuration property. (#3453)

  • +
+
+
+

SPI#

+
    +
  • Allow a SystemAccessControl to provide an EventListener. (#3629).

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-335.html b/430/release/release-335.html new file mode 100644 index 000000000..601186606 --- /dev/null +++ b/430/release/release-335.html @@ -0,0 +1,2636 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 335 (14 Jun 2020) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 335 (14 Jun 2020)#

+
+

General#

+
    +
  • Fix failure when reduce_agg() is used as a window function. (#3883)

  • +
  • Fix incorrect cast from TIMESTAMP (without time zone) to TIME type. (#3848)

  • +
  • Fix incorrect query results when converting very large TIMESTAMP values into +TIMESTAMP WITH TIME ZONE, or when parsing very large +TIMESTAMP WITH TIME ZONE values. (#3956)

  • +
  • Return VARCHAR type when substr() argument is CHAR type. (#3599, #3456)

  • +
  • Improve optimized local scheduling with regard to non-uniform data distribution. (#3922)

  • +
  • Add support for variable-precision TIMESTAMP (without time zone) type. (#3783)

  • +
  • Add a variant of substring() that takes a CHAR argument. (#3949)

  • +
  • Add information_schema.role_authorization_descriptors table that returns information about the roles +granted to principals. (#3535)

  • +
+
+
+

Security#

+ +
+
+

Web UI#

+
    +
  • Fix the value displayed in the worker memory pools bar. (#3920)

  • +
+
+
+

Accumulo connector#

+
    +
  • The server-side iterators are now in a JAR file named presto-accumulo-iterators. (#3673)

  • +
+
+
+

Hive connector#

+
    +
  • Collect column statistics for inserts into empty tables. (#2469)

  • +
  • Add support for information_schema.role_authorization_descriptors table when using the sql-standard +security mode. (#3535)

  • +
  • Allow non-lowercase column names in system.sync_partition_metadata procedure. This can be enabled +by passing case_sensitive=false when invoking the procedure. (#3431)

  • +
  • Support caching with secured coordinator. (#3874)

  • +
  • Prevent caching from becoming disabled due to intermittent network failures. (#3874)

  • +
  • Ensure HDFS impersonation is not enabled when caching is enabled. (#3913)

  • +
  • Add hive.cache.ttl and hive.cache.disk-usage-percentage cache properties. (#3840)

  • +
  • Improve query performance when caching is enabled by scheduling work on nodes with cached data. (#3922)

  • +
  • Add support for UNIONTYPE. This is mapped to ROW containing a tag field and a field for each data type in the union. For +example, UNIONTYPE<INT, DOUBLE> is mapped to ROW(tag INTEGER, field0 INTEGER, field1 DOUBLE). (#3483)

  • +
  • Make partition_values argument to drop_stats procedure optional. (#3937)

  • +
  • Add support for dynamic partition pruning to improve performance of complex queries +over partitioned data. (#1072)

  • +
+
+
+

Phoenix connector#

+
    +
  • Allow configuring whether DROP TABLE is allowed. This is controlled by the new allow-drop-table +catalog configuration property and defaults to true, compatible with the previous behavior. (#3953)

  • +
+
+
+

SPI#

+
    +
  • Add support for aggregation pushdown into connectors via the +ConnectorMetadata.applyAggregation() method. (#3697)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-336.html b/430/release/release-336.html new file mode 100644 index 000000000..68ab49e95 --- /dev/null +++ b/430/release/release-336.html @@ -0,0 +1,2574 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 336 (16 Jun 2020) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 336 (16 Jun 2020)#

+
+

General#

+
    +
  • Fix failure when querying timestamp columns from older clients. (#4036)

  • +
  • Improve reporting of configuration errors. (#4050)

  • +
  • Fix rare failure when recording server stats in T-Digests. (#3965)

  • +
+
+
+

Security#

+ +
+
+

Hive connector#

+
    +
  • Fix incorrect query results when reading Parquet files with predicates +when hive.parquet.use-column-names is set to false (the default). (#3574)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-337.html b/430/release/release-337.html new file mode 100644 index 000000000..33f8bf6a4 --- /dev/null +++ b/430/release/release-337.html @@ -0,0 +1,2641 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 337 (25 Jun 2020) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 337 (25 Jun 2020)#

+
+

Note

+

This release fixes a potential security vulnerability when secure internal communication is enabled in a cluster. A malicious +attacker can take advantage of this vulnerability to escalate privileges to internal APIs. We encourage everyone to upgrade as soon +as possible.

+
+
+

General#

+
    +
  • Fix incorrect results for inequality join involving NaN. (#4120)

  • +
  • Fix peak non-revocable memory metric in event listener. (#4096)

  • +
  • Fix queued query JMX stats. (#4129)

  • +
  • Fix rendering of types in the output of DESCRIBE INPUT. (#4023)

  • +
  • Improve performance of queries involving comparisons between DOUBLE or REAL values and integer values. (#3533)

  • +
  • Reduce idle CPU consumption in coordinator. (#3990)

  • +
  • Add peak non-revocable memory metric to query stats. (#4096)

  • +
  • Add support for variable-precision TIMESTAMP WITH TIME ZONE type (#3947)

  • +
  • Add support for IN predicate with subqueries in outer join condition. (#4151)

  • +
  • Add support for quantified comparisons (e.g., > ALL (...)) in aggregation queries. (#4128)

  • +
  • Add Druid connector. (#3522)

  • +
  • Add translate() function. (#4080)

  • +
  • Reduce worker graceful shutdown duration. (#4192)

  • +
+
+
+

Security#

+
    +
  • Disable insecure authentication over HTTP by default when HTTPS with authentication is enabled. This +can be overridden via the http-server.authentication.allow-insecure-over-http configuration property. (#4199)

  • +
  • Add support for insecure authentication over HTTPS to the Web UI. (#4199)

  • +
  • Add System information rules which control the ability of a +user to access to read and write system management information. +(#4199)

  • +
  • Disable user impersonation in default system security. (#4082)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Add support for password authentication. (#4165)

  • +
+
+
+

Hive connector#

+
    +
  • Fix reading CSV tables with separatorChar, quoteChar or escapeChar table property +containing more than one character. For compatibility with Hive, only first character is considered +and remaining are ignored. (#3891)

  • +
  • Improve performance of INSERT queries writing to bucketed tables when some buckets do not contain any data. (#1375)

  • +
  • Improve performance of queries reading Parquet data with predicates on timestamp columns. (#4104)

  • +
  • Improve performance for join queries over partitioned tables. (#4156)

  • +
  • Add support for null_format table property for tables using TextFile storage format (#4056)

  • +
  • Add support for null_format table property for tables using RCText and SequenceFile +storage formats (#4143)

  • +
  • Add optimized Parquet writer. The new writer is disabled by default, and can be enabled with the +parquet_optimized_writer_enabled session property or the hive.parquet.optimized-writer.enabled configuration +property. (#3400)

  • +
  • Add support caching data in Azure Data Lake and AliyunOSS storage. (#4213)

  • +
  • Fix failures when caching data from Google Cloud Storage. (#4213)

  • +
  • Support ACID data files naming used when direct inserts are enabled in Hive (HIVE-21164). +Direct inserts is an upcoming feature in Hive 4. (#4049)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Improve performance of aggregation queries by computing aggregations within PostgreSQL database. +Currently, the following aggregate functions are eligible for pushdown: +count, min, max, sum and avg. (#3881)

  • +
+
+
+

Base-JDBC connector library#

+
    +
  • Implement framework for aggregation pushdown. (#3881)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-338.html b/430/release/release-338.html new file mode 100644 index 000000000..87af420d2 --- /dev/null +++ b/430/release/release-338.html @@ -0,0 +1,2645 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 338 (07 Jul 2020) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 338 (07 Jul 2020)#

+
+

General#

+
    +
  • Fix incorrect results when joining tables on a masked column. (#4251)

  • +
  • Fix planning failure when multiple columns have a mask. (#4322)

  • +
  • Fix incorrect comparison for TIMESTAMP WITH TIME ZONE values with precision larger than 3. (#4305)

  • +
  • Fix incorrect rounding for timestamps before 1970-01-01. (#4370)

  • +
  • Fix query failure when using VALUES with a floating point NaN value. (#4119)

  • +
  • Fix query failure when joining tables on a real or double column and one of the joined tables +contains NaN value. (#4272)

  • +
  • Fix unauthorized error for internal requests to management endpoints. (#4304)

  • +
  • Fix memory leak while using dynamic filtering. (#4228)

  • +
  • Improve dynamic partition pruning for broadcast joins. (#4262)

  • +
  • Add support for setting column comments via the COMMENT ON COLUMN syntax. (#2516)

  • +
  • Add compatibility mode for legacy clients when rendering datetime type names with default precision +in information_schema tables. This can be enabled via the deprecated.omit-datetime-type-precision +configuration property or omit_datetime_type_precision session property. (#4349, #4377)

  • +
  • Enforce NOT NULL column declarations when writing data. (#4144)

  • +
+
+
+

JDBC driver#

+
    +
  • Fix excessive CPU usage when reading query results. (#3928)

  • +
  • Implement DatabaseMetaData.getClientInfoProperties(). (#4318)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Add support for reading numeric values encoded as strings. (#4341)

  • +
+
+
+

Hive connector#

+
    +
  • Fix incorrect query results when Parquet file has no min/max statistics for an integral column. (#4200)

  • +
  • Fix query failure when reading from a table partitioned on a real or double column containing +a NaN value. (#4266)

  • +
  • Fix sporadic failure when writing to bucketed sorted tables on S3. (#2296)

  • +
  • Fix handling of strings when translating Hive views. (#3266)

  • +
  • Do not require cache directories to be configured on coordinator. (#3987, #4280)

  • +
  • Fix Azure ADL caching support. (#4240)

  • +
  • Add support for setting column comments. (#2516)

  • +
  • Add hidden $partition column for partitioned tables that contains the partition name. (#3582)

  • +
+
+
+

Kafka connector#

+
    +
  • Fix query failure when a column is projected and also referenced in a query predicate +when reading from Kafka topic using RAW decoder. (#4183)

  • +
+
+
+

MySQL connector#

+
    +
  • Fix type mapping for unsigned integer types. (#4187)

  • +
+
+
+

Oracle connector#

+
    +
  • Exclude internal schemas (e.g., sys) from schema listings. (#3784)

  • +
  • Add support for connection pooling. (#3770)

  • +
+
+
+

Base-JDBC connector library#

+
    +
  • Exclude the underlying database’s information_schema from schema listings. (#3834)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-339.html b/430/release/release-339.html new file mode 100644 index 000000000..ab66e77e7 --- /dev/null +++ b/430/release/release-339.html @@ -0,0 +1,2669 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 339 (21 Jul 2020) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 339 (21 Jul 2020)#

+
+

General#

+
    +
  • Add approx_most_frequent(). (#3425)

  • +
  • Physical bytes scan limit for queries can be configured via query.max-scan-physical-bytes configuration property +and query_max_scan_physical_bytes session property. (#4075)

  • +
  • Remove support for addition and subtraction between TIME and INTERVAL YEAR TO MONTH types. (#4308)

  • +
  • Fix planning failure when join criteria contains subqueries. (#4380)

  • +
  • Fix failure when subquery appear in window function arguments. (#4127)

  • +
  • Fix failure when subquery in WITH clause contains hidden columns. (#4423)

  • +
  • Fix failure when referring to type names with different case in a GROUP BY clause. (#2960)

  • +
  • Fix failure for queries involving DISTINCT when expressions in ORDER BY clause differ by case from expressions in SELECT clause. (#4233)

  • +
  • Fix incorrect type reporting for TIMESTAMP and TIMESTAMP WITH TIME ZONE for legacy clients. (#4408)

  • +
  • Fix failure when querying nested TIMESTAMP or TIMESTAMP WITH TIME ZONE for legacy clients. (#4475, #4425)

  • +
  • Fix failure when parsing timestamps with time zone with an offset of the form +NNNN. (#4490)

  • +
+
+
+

JDBC driver#

+
    +
  • Fix reading TIMESTAMP and TIMESTAMP WITH TIME ZONE values with a negative year +or a year higher than 9999. (#4364)

  • +
  • Fix incorrect column size metadata for TIMESTAMP and TIMESTAMP WITH TIME ZONE types. (#4411)

  • +
  • Return correct value from ResultSet.getDate(), ResultSet.getTime() and ResultSet.getTimestamp() methods +when session zone is set to a different zone than the default zone of the JVM the JDBC is run in. +The previous behavior can temporarily be restored using useSessionTimeZone JDBC connection +parameter. (#4017)

  • +
+
+
+

Druid connector#

+
    +
  • Fix handling of table and column names containing non-ASCII characters. (#4312)

  • +
+
+
+

Hive connector#

+
    +
  • Make location parameter optional for the system.register_partition procedure. (#4443)

  • +
  • Avoid creating tiny splits at the end of block boundaries. (#4485)

  • +
  • Remove requirement to configure metastore.storage.schema.reader.impl in Hive 3.x metastore +to let Presto access CSV tables. (#4457)

  • +
  • Fail query if there are bucket files outside of the bucket range. +Previously, these extra files were skipped. (#4378)

  • +
  • Fix a query failure when reading from Parquet file containing real or double NaN values, +if the file was written by a non-conforming writer. (#4267)

  • +
+
+
+

Kafka connector#

+
    +
  • Add insert support for Avro. (#4418)

  • +
  • Add insert support for CSV. (#4287)

  • +
+
+
+

Kudu connector#

+
    +
  • Add support for grouped execution. It can be enabled with the kudu.grouped-execution.enabled +configuration property or the grouped_execution session property. (#3715)

  • +
+
+
+

MongoDB connector#

+
    +
  • Allow querying Azure Cosmos DB. (#4415)

  • +
+
+
+

Oracle connector#

+
    +
  • Allow providing credentials via the connection-user and connection-password +configuration properties. These properties were previously ignored if connection pooling +was enabled. (#4430)

  • +
+
+
+

Phoenix connector#

+
    +
  • Fix handling of row key definition with white space. (#3251)

  • +
+
+
+

SPI#

+
    +
  • Allow connectors to wait for dynamic filters before splits are generated via the new +DynamicFilter object passed to ConnectorSplitManager.getSplits(). (#4224)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-340.html b/430/release/release-340.html new file mode 100644 index 000000000..1d1f833b9 --- /dev/null +++ b/430/release/release-340.html @@ -0,0 +1,2671 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 340 (8 Aug 2020) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 340 (8 Aug 2020)#

+
+

General#

+
    +
  • Add support for query parameters in LIMIT, OFFSET and FETCH FIRST clauses. (#4529, #4601)

  • +
  • Add experimental support for recursive queries. (#4250)

  • +
  • Add bitwise_left_shift(), bitwise_right_shift() and bitwise_right_shift_arithmetic(). (#740)

  • +
  • Add luhn_check(). (#4011)

  • +
  • Add IF EXISTS and IF NOT EXISTS syntax to ALTER TABLE. (#4651)

  • +
  • Include remote host in error info for page transport errors. (#4511)

  • +
  • Improve minimum latency for dynamic partition pruning. (#4388)

  • +
  • Reduce cluster load by cancelling query stages from which data is no longer required. (#4290)

  • +
  • Reduce query memory usage by improving retained size estimation for VARCHAR and CHAR types. (#4123)

  • +
  • Improve query performance for queries containing starts_with(). (#4669)

  • +
  • Improve performance of queries that use DECIMAL data type. (#4730)

  • +
  • Fix failure when GROUP BY clause contains duplicate expressions. (#4609)

  • +
  • Fix potential hang during query planning (#4635).

  • +
+
+
+

Security#

+
    +
  • Fix unprivileged access to table’s schema via CREATE TABLE LIKE. (#4472)

  • +
+
+
+

JDBC driver#

+
    +
  • Fix handling of dates before 1582-10-15. (#4563)

  • +
  • Fix handling of timestamps before 1900-01-01. (#4563)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Fix failure when index mapping is missing. (#4535)

  • +
+
+
+

Hive connector#

+
    +
  • Allow creating a table with external_location when schema’s location is not valid. (#4069)

  • +
  • Add read support for tables that were created as non-transactional and converted to be +transactional later. (#2293)

  • +
  • Allow creation of transactional tables. Note that writing to transactional tables +is not yet supported. (#4516)

  • +
  • Add hive.metastore.glue.max-error-retries configuration property for the +number of retries performed when accessing the Glue metastore. (#4611)

  • +
  • Support using Java KeyStore files for Thrift metastore TLS configuration. (#4432)

  • +
  • Expose hit rate statistics for Hive metastore cache via JMX. (#4458)

  • +
  • Improve performance when querying a table with large files and with skip.header.line.count property set to 1. (#4513)

  • +
  • Improve performance of reading JSON tables. (#4705)

  • +
  • Fix query failure when S3 data location contains a _$folder$ marker object. (#4552)

  • +
  • Fix failure when referencing nested fields of a ROW type when table and partition metadata differs. (#3967)

  • +
+
+
+

Kafka connector#

+
    +
  • Add insert support for Raw data format. (#4417)

  • +
  • Add insert support for JSON. (#4477)

  • +
  • Remove unused kafka.connect-timeout configuration properties. (#4664)

  • +
+
+
+

MongoDB connector#

+
    +
  • Add mongodb.max-connection-idle-time properties to limit the maximum idle time of a pooled connection. (#4483)

  • +
+
+
+

Phoenix connector#

+
    +
  • Add table level property to specify data block encoding when creating tables. (#4617)

  • +
  • Fix query failure when listing schemas. (#4560)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Push down count() aggregations over constant expressions. +For example, SELECT count(1). (#4362)

  • +
+
+
+

SPI#

+
    +
  • Expose information about query type in query Event Listener. (#4592)

  • +
  • Add support for TopN pushdown via the ConnectorMetadata.applyLimit() method. (#4249)

  • +
  • Deprecate the older variants of ConnectorSplitManager.getSplits(). (#4508)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-341.html b/430/release/release-341.html new file mode 100644 index 000000000..9c3ac9063 --- /dev/null +++ b/430/release/release-341.html @@ -0,0 +1,2833 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 341 (8 Sep 2020) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 341 (8 Sep 2020)#

+
+

General#

+
    +
  • Add support for variable-precision TIME type. (#4381)

  • +
  • Add support for variable precision TIME WITH TIME ZONE type. (#4905)

  • +
  • Add Iceberg connector.

  • +
  • Add human_readable_seconds() function. (#4344)

  • +
  • Add reverse() function for VARBINARY. (#4741)

  • +
  • Add support for extract() for timestamp(p) with time zone with values of p other than 3. (#4867)

  • +
  • Add support for correlated subqueries in recursive queries. (#4877)

  • +
  • Add runtime.optimizer_rule_stats system table. (#4659)

  • +
  • Report dynamic filters statistics. (#4440)

  • +
  • Improve query scalability when new nodes are added to cluster. (#4294)

  • +
  • Improve error message when JSON parsing fails. (#4616)

  • +
  • Reduce latency when dynamic filtering is in use. (#4924)

  • +
  • Remove support for political time zones in TIME WITH TIME ZONE type. (#191)

  • +
  • Remove deprecated reorder_joins session property. (#5027)

  • +
  • Remove the deprecated.legacy-timestamp configuration property and the legacy_timestamp session property. (#4799)

  • +
  • Change timestamp operations to match the SQL specification. The value of a TIMESTAMP +type is not affected by the session time zone. (#37)

  • +
  • Preserve precision when applying AT TIME ZONE to values of type TIMESTAMP. (#4866)

  • +
  • Fix serialization of NULL values in ROW, MAP and ARRAY types for old Presto clients. (#4778)

  • +
  • Fix failure when aggregation query contains duplicate expressions. (#4872)

  • +
  • Fix compiler failure when querying timestamps with a precision greater than 6. (#4824)

  • +
  • Fix parsing failure of timestamps due to daylight saving changes. (#37)

  • +
  • Fix failure when calling extract() with TIMEZONE_HOUR and TIMEZONE_MINUTE for +TIMESTAMP WITH TIME ZONE type. (#4867)

  • +
  • Fix query deadlock for connectors that wait for dynamic filters. (#4946)

  • +
  • Fix failure when TIME or TIMESTAMP subtraction returns a negative value. (#4847)

  • +
  • Fix failure when duplicate expressions appear in DISTINCT clause. (#4787)

  • +
  • Fix failure for certain join queries during spilling or when available memory is low. (#4994)

  • +
  • Fix issue where the query_max_scan_physical_bytes session property was ignored if +the query.max-scan-physical-bytes configuration property was not defined. (#5009)

  • +
  • Correctly compute sample ratio when TABLESAMPLE is used with a fractional percentage. (#5074)

  • +
  • Fail queries with a proper error message when TABLESAMPLE is used with a non-numeric sample ratio. (#5074)

  • +
  • Fail with an explicit error rather than OutOfMemoryError for certain operations. (#4890)

  • +
+
+
+

Security#

+ +
+
+

Web UI#

+
    +
  • Fix display of physical input read time in detailed query view. (#4962)

  • +
+
+
+

JDBC driver#

+
    +
  • Implement ResultSet.getStatement(). (#4957)

  • +
+
+
+

BigQuery connector#

+
    +
  • Add support for hourly partitioned tables. (#4968)

  • +
  • Redact the value of bigquery.credentials-key in the server log. (#4968)

  • +
+
+
+

Cassandra connector#

+
    +
  • Map Cassandra TIMESTAMP type to Presto TIMESTAMP(3) WITH TIME ZONE type. (#2269)

  • +
+
+
+

Hive connector#

+
    +
  • Skip stripes and row groups based on timestamp statistics for ORC files. (#1147)

  • +
  • Skip S3 objects with the DeepArchive storage class (in addition to the Glacier +storage class) when hive.s3.skip-glacier-objects is enabled. (#5002)

  • +
  • Use a temporary staging directory for temporary files when writing to sorted bucketed tables. +This allows using a more efficient file system for temporary files. (#3434)

  • +
  • Fix metastore cache invalidation for GRANT and REVOKE. (#4768)

  • +
  • Add Parquet and RCBinary configuration properties hive.parquet.time-zone +and hive.rcfile.time-zone to adjust binary timestamp values to a specific time zone. For Hive 3.1+, this +should be set to UTC. The default value is the JVM default time zone, for backwards compatibility with +earlier versions of Hive. (#4799)

  • +
  • Add ORC configuration property hive.orc.time-zone to set the default +time zone for legacy ORC files that did not declare a time zone. (#4799)

  • +
  • Replace the hive.time-zone configuration property with format specific properties: hive.orc.time-zone, +hive.parquet.time-zone, hive.rcfile.time-zone. (#4799)

  • +
  • Allow using the cluster default role with S3 security mapping. (#4931)

  • +
  • Remove support for bucketing on timestamp. The definition of the hash function for timestamp +incorrectly depends on the storage time zone and can result in incorrect results. (#4759)

  • +
  • Decrease the number of requests to the Glue metastore when fetching partitions. +This helps avoid hitting rate limits and decreases service costs. (#4938)

  • +
  • Match the existing user and group of the table or partition when creating new files on HDFS. (#4414)

  • +
  • Fix invalid timestamp values for nested data in Text, Avro, SequenceFile, JSON and CSV formats. (#4799)

  • +
  • Fix query failure when reading an ORC ACID table with a filter after the table +underwent a minor table compaction. (#4622)

  • +
  • Fix incorrect query results when reading an ORC ACID table that has deleted rows +and underwent a minor compaction. (#4623)

  • +
  • Fix query failure when storage caching is enabled and cached data is evicted during query execution. (#3580)

  • +
+
+
+

JMX connector#

+
    +
  • Change timestamp column type in history tables to TIMESTAMP WITH TIME ZONE. (#4753)

  • +
+
+
+

Kafka connector#

+
    +
  • Preserve time zone when parsing TIMESTAMP WITH TIME ZONE values. (#4799)

  • +
+
+
+

Kinesis connector#

+
    +
  • Preserve time zone when parsing TIMESTAMP WITH TIME ZONE values. (#4799)

  • +
+
+
+

Kudu connector#

+
    +
  • Fix delete when applied on table having primary key of decimal type. (#4683)

  • +
+
+
+

Local File connector#

+
    +
  • Change timestamp column type to TIMESTAMP WITH TIME ZONE. (#4752)

  • +
+
+
+

MySQL connector#

+
    +
  • Improve performance of aggregation queries by pushing the aggregation computation into the MySQL database. +Currently, the following aggregate functions are eligible for pushdown: count, min, max, +sum and avg. (#4138)

  • +
+
+
+

Oracle connector#

+
    +
  • Add oracle.connection-pool.inactive-timeout configuration property to specify how long +pooled connection can be inactive before it is closed. It defaults to 20 minutes. (#4779)

  • +
  • Add support for database internationalization. (#4775)

  • +
  • Add resilience to momentary connection authentication issues. (#4947)

  • +
  • Allowing forcing the mapping of certain types to VARCHAR. This can be enabled by +setting the jdbc-types-mapped-to-varchar configuration property to a comma-separated +list of type names. (#4955)

  • +
  • Prevent query failure for pushdown of predicates involving a large number of conjuncts. (#4918)

  • +
+
+
+

Phoenix connector#

+
    +
  • Fix overwriting of former value when insert is applied without specifying that column. (#4670)

  • +
+
+
+

Pinot connector#

+
    +
  • Add support for REAL and INTEGER types. (#4725)

  • +
  • Add support for functions in pass-through queries. (#4801)

  • +
  • Enforce a limit on the number of rows fetched from Pinot. This can be configured via the +pinot.max-rows-per-split-for-segment-queries configuration property. (#4723)

  • +
  • Fix incorrect results for count(*) queries. (#4802)

  • +
  • Fix incorrect results for queries involving avg() over columns of type long, int, or float. (#4802)

  • +
  • Fix incorrect results when columns in pass-through query do not match selected columns. (#4802)

  • +
+
+
+

Prometheus connector#

+
    +
  • Change the type of the timestamp column to TIMESTAMP(3) WITH TIME ZONE type. (#4799)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Improve performance of aggregation queries with predicates by pushing the computation +of both the filtering and aggregations into the PostgreSQL server where possible. (#4111)

  • +
  • Fix handling of PostgreSQL arrays when unsupported-type-handling is set to CONVERT_TO_VARCHAR. (#4981)

  • +
+
+
+

Raptor connector#

+
    +
  • Remove the storage.shard-day-boundary-time-zone configuration property, which was used to work +around legacy timestamp semantics in Presto. (#4799)

  • +
+
+
+

Redis connector#

+
    +
  • Preserve time zone when parsing TIMESTAMP WITH TIME ZONE values. (#4799)

  • +
+
+
+

SPI#

+
    +
  • The TIMESTAMP type is encoded as a number of fractional seconds from 1970-01-01 00:00:00 in the proleptic +Gregorian calendar. This value is no longer adjusted to the session time zone. Timestamps with precision less +than or equal to 3 are now represented in microseconds. (#4799)

  • +
  • Remove isLegacyTimestamp() from ConnectorSession. (#4799)

  • +
  • Enable connectors to wait for dynamic filters before producing data on worker nodes. (#3414)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-342.html b/430/release/release-342.html new file mode 100644 index 000000000..3852266dc --- /dev/null +++ b/430/release/release-342.html @@ -0,0 +1,2678 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 342 (24 Sep 2020) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 342 (24 Sep 2020)#

+
+

General#

+
    +
  • Add from_iso8601_timestamp_nanos() function. (#5048)

  • +
  • Improve performance of queries that use the DECIMAL type. (#4886)

  • +
  • Improve performance of queries involving IN with subqueries by extending support for dynamic filtering. (#5017)

  • +
  • Improve performance and latency of queries leveraging dynamic filters. (#4988)

  • +
  • Improve performance of queries joining tables with missing or incomplete column statistics when +cost based optimization is enabled (which is the default). (#5141)

  • +
  • Reduce latency for queries that perform a broadcast join of a large table. (#5237)

  • +
  • Allow collection of dynamic filters for joins with large build side using the +enable-large-dynamic-filters configuration property or the enable_large_dynamic_filters +session property. (#5262)

  • +
  • Fix query failure when lambda expression references a table column containing a dot. (#5087)

  • +
+
+
+

Atop connector#

+
    +
  • Fix incorrect query results when query contains predicates on start_time or end_time column. (#5125)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Allow reading boolean values stored as strings. (#5269)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for S3 encrypted files. (#2536)

  • +
  • Add support for ABFS OAuth authentication. (#5052)

  • +
  • Support reading timestamp with microsecond or nanosecond precision. This can be enabled with the +hive.timestamp-precision configuration property. (#4953)

  • +
  • Allow overwrite on insert by default using the hive.insert-existing-partitions-behavior configuration property. (#4999)

  • +
  • Allow delaying table scans until dynamic filtering can be performed more efficiently. This can be enabled +using the hive.dynamic-filtering-probe-blocking-timeout configuration property or the +dynamic_filtering_probe_blocking_timeout session property. (#4991)

  • +
  • Disable matching the existing user and group of the table or partition when creating new files on HDFS. +The functionality was added in 341 and is now disabled by default. It can be enabled using the +hive.fs.new-file-inherit-ownership configuration property. (#5187)

  • +
  • Improve performance when reading small files in RCTEXT or RCBINARY format. (#2536)

  • +
  • Improve planning time for queries with non-equality filters on partition columns when using the Glue metastore. (#5060)

  • +
  • Improve performance when reading JSON and CSV file formats. (#5142)

  • +
+
+
+

Iceberg connector#

+
    +
  • Fix partition transforms for temporal columns for dates before 1970. (#5273)

  • +
+
+
+

Kafka connector#

+
    +
  • Expose message headers as a _headers column of MAP(VARCHAR, ARRAY(VARBINARY)) type. (#4462)

  • +
  • Add write support for TIME, TIME WITH TIME ZONE, TIMESTAMP and TIMESTAMP WITH TIME ZONE +for Kafka connector when using the JSON encoder. (#4743)

  • +
  • Remove JSON decoder support for nonsensical combinations of input-format-type / data-type. The following +combinations are no longer supported: (#4743)

    +
      +
    • rfc2822: DATE, TIME, TIME WITH TIME ZONE

    • +
    • milliseconds-since-epoch: TIME WITH TIME ZONE, TIMESTAMP WITH TIME ZONE

    • +
    • seconds-since-epoch: TIME WITH TIME ZONE, TIMESTAMP WITH TIME ZONE

    • +
    +
  • +
+
+
+

MySQL connector#

+
    +
  • Improve performance of INSERT queries when GTID mode is disabled in MySQL. (#4995)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Add support for variable-precision TIMESTAMP and TIMESTAMP WITH TIME ZONE types. (#5124, #5105)

  • +
+
+
+

SQL Server connector#

+
    +
  • Fix failure when inserting NULL into a VARBINARY column. (#4846)

  • +
  • Improve performance of aggregation queries by computing aggregations within SQL Server database. +Currently, the following aggregate functions are eligible for pushdown: +count, min, max, sum and avg. (#4139)

  • +
+
+
+

SPI#

+
    +
  • Add DynamicFilter.isAwaitable() method that returns whether or not the dynamic filter is complete +and can be awaited for using the isBlocked() method. (#5043)

  • +
  • Enable connectors to wait for dynamic filters derived from replicated joins before generating splits. (#4685)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-343.html b/430/release/release-343.html new file mode 100644 index 000000000..52c409a25 --- /dev/null +++ b/430/release/release-343.html @@ -0,0 +1,2571 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 343 (25 Sep 2020) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 343 (25 Sep 2020)#

+
+

BigQuery connector#

+
    +
  • Add support for yearly partitioned tables. (#5298)

  • +
+
+
+

Hive connector#

+
    +
  • Fix query failure when read from or writing to a bucketed table containing a column of timestamp type. (#5295)

  • +
+
+
+

SQL Server connector#

+
    +
  • Improve performance of aggregation queries with stddev, stddev_samp, stddev_pop, variance, var_samp, var_pop +aggregate functions by computing aggregations within SQL Server database. (#5299)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-344.html b/430/release/release-344.html new file mode 100644 index 000000000..88833db44 --- /dev/null +++ b/430/release/release-344.html @@ -0,0 +1,2658 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 344 (9 Oct 2020) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 344 (9 Oct 2020)#

+
+

General#

+
    +
  • Add murmur3() function. (#5054)

  • +
  • Add from_unixtime_nanos() function. (#5046)

  • +
  • Add T-Digest type and functions. (#5158)

  • +
  • Improve performance and latency of queries leveraging dynamic filters. (#5081, #5340)

  • +
  • Add dynamic-filtering.service-thread-count config property to specify number of +threads used for processing dynamic filters on coordinator. (#5341)

  • +
  • Extend Secrets environment variable substitution to allow +multiple replacements in a single configuration property. (#4345)

  • +
  • Remove the fast-inequality-joins configuration property. This feature is always enabled. (#5375)

  • +
  • Use timestamp(3) with time zone rather than timestamp(3) for the queries, +transactions, and tasks tables in system.runtime. (#5464)

  • +
  • Improve performance and accuracy of approx_percentile(). (#5158)

  • +
  • Improve performance of certain cross join queries. (#5276)

  • +
  • Prevent potential query deadlock when query runs out of memory. (#5289)

  • +
  • Fix failure due to rounding error when casting between two timestamp types with precision +higher than 6. (#5310)

  • +
  • Fix failure due to rounding error when casting between two timestamp with time zone types +with precision higher than 3. (#5371)

  • +
  • Fix column pruning for EXPLAIN ANALYZE. (#4760)

  • +
  • Fix incorrect timestamp values returned by the queries, transactions, +and tasks tables in system.runtime. (#5462)

  • +
+
+
+

Security#

+
+

Warning

+

The file-based system and catalog access controls have changed in ways that reduce or increase permissions. +Please, read these release notes carefully.

+
+
    +
  • Change file-based catalog access control from deny to allow when table, schema, or session +property rules are not defined. (#5039)

  • +
  • Add missing table rule checks for table and view DDL in file-based system access control. (#5039)

  • +
  • Add missing schema rule checks for create schema in file-based system access control. (#5039)

  • +
  • Add session property rules to file-based system access control. (#5039)

  • +
  • Add catalog regex to table and schema rules in file-based system access control. (#5039)

  • +
  • Change create, rename, alter, and drop table in file-based system controls to only check for +table ownership. (#5039)

  • +
  • Change file-based system access control to support files without catalog rules defined. In this case, +all access to catalogs is allowed. (#5039)

  • +
  • Change file-based system and catalog access controls to only show catalogs, schemas, and tables a user +has permissions on. (#5039)

  • +
  • Change file-based catalog access control to deny permissions inspection and manipulation. (#5039)

  • +
  • Add file-based group provider. (#5028)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for hive.security=allow-all, which allows to skip all authorization checks. (#5416)

  • +
  • Support Kerberos authentication for Hudi tables. (#5472)

  • +
  • Allow hiding Delta Lake tables from table listings such as SHOW TABLES or information_schema.tables, +as these tables cannot be queried by the Hive connector. This be enabled using the hive.hide-delta-lake-tables +configuration property. (#5430)

  • +
  • Improve query concurrency by listing data files more efficiently. (#5260)

  • +
  • Fix Parquet encoding for timestamps before 1970-01-01. (#5364)

  • +
+
+
+

Kafka connector#

+
    +
  • Expose message timestamp via _timestamp internal column. (#4805)

  • +
  • Add predicate pushdown for _timestamp, _partition_offset and _partition_id columns. (#4805)

  • +
+
+
+

Phoenix connector#

+
    +
  • Fix query failure when a column name in CREATE TABLE requires quoting. (#3601)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Add support for setting a column comment. (#5307)

  • +
  • Add support for variable-precision time type. (#5342)

  • +
  • Allow CREATE TABLE and CREATE TABLE AS with timestamp and timestamp with time zone with precision higher than 6. +The resulting column will be declared with precision of 6, maximal supported by PostgreSQL. (#5342)

  • +
+
+
+

SQL Server connector#

+
    +
  • Improve performance of queries with aggregations and WHERE clause. (#5327)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-345.html b/430/release/release-345.html new file mode 100644 index 000000000..cce0a5675 --- /dev/null +++ b/430/release/release-345.html @@ -0,0 +1,2671 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 345 (23 Oct 2020) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 345 (23 Oct 2020)#

+
+

General#

+
    +
  • Add concat_ws() function. (#4680)

  • +
  • Add support for extract() for time with time zone values with precision other than 3. (#5539)

  • +
  • Add dynamic filtering support for right joins. (#5555)

  • +
  • Add support for catalog session properties to the file and database backed session property managers. (#5330)

  • +
  • Improve query scalability by increasing the default broadcast join buffer size to 200MB. +Broadcast join buffer size can be set using the sink.max-broadcast-buffer-size configuration property. (#5551)

  • +
  • Improve query performance by allowing larger dynamic filters. (#5529)

  • +
  • Improve performance for join queries where join keys are not of the same type. (#5461)

  • +
  • Improve performance of encrypted spilling. (#5557)

  • +
  • Improve performance of queries that use the decimal type. (#5181)

  • +
+
+
+

Security#

+
    +
  • Add support for JSON Web Key (JWK) to the existing JSON Web Token (JWT) authenticator. This is enabled by +setting the jwt.key-file configuration property to a http or https url. (#5419)

  • +
  • Add column security, column mask and row filter to file-based access controls. (#5460)

  • +
  • Enforce access control for column references in USING clause. (#5620)

  • +
+
+
+

JDBC driver#

+
    +
  • Add source parameter for directly setting the source name for a query. (#4739)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for INSERT and DELETE for ACID tables. (#5402)

  • +
  • Apply hive.domain-compaction-threshold to dynamic filters. (#5365)

  • +
  • Add support for reading Parquet timestamps encoded as microseconds. (#5483)

  • +
  • Improve translation of Hive views. (#4661)

  • +
  • Improve storage caching by better distributing files across workers. (#5621)

  • +
  • Fix disk space accounting for storage caching. (#5621)

  • +
  • Fix failure when reading Parquet timestamp columns encoded as int64. (#5443)

  • +
+
+
+

MongoDB connector#

+
    +
  • Add support for adding columns. (#5512)

  • +
  • Fix incorrect result for IS NULL predicates on fields that do not exist in the document. (#5615)

  • +
+
+
+

MemSQL connector#

+
    +
  • Fix representation for many MemSQL types. (#5495)

  • +
  • Prevent a query failure when table column name contains a semicolon by explicitly forbidding such names. (#5495)

  • +
  • Add support for case-insensitive table name matching. (#5495)

  • +
+
+
+

MySQL connector#

+
    +
  • Improve performance of queries with aggregations and LIMIT clause (but without ORDER BY). (#5261)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Improve performance of queries with aggregations and LIMIT clause (but without ORDER BY). (#5261)

  • +
+
+
+

Redshift connector#

+
    +
  • Add support for setting column comments. (#5397)

  • +
+
+
+

SQL Server connector#

+
    +
  • Improve performance of queries with aggregations and LIMIT clause (but without ORDER BY). (#5261)

  • +
+
+
+

Thrift connector#

+
    +
  • Fix handling of timestamp values. (#5596)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-346.html b/430/release/release-346.html new file mode 100644 index 000000000..2ba3e529d --- /dev/null +++ b/430/release/release-346.html @@ -0,0 +1,2753 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 346 (10 Nov 2020) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 346 (10 Nov 2020)#

+
+

General#

+
    +
  • Add support for RANGE BETWEEN <value> PRECEDING AND <value> FOLLOWING window frames. (#609)

  • +
  • Add support for window frames based on GROUPS. (#5713)

  • +
  • Add support for extract() with TIMEZONE_HOUR and TIMEZONE_MINUTE for time with time zone values. (#5668)

  • +
  • Add SQL syntax for GRANT and REVOKE on schema. This is not yet used by any connector. (#4396)

  • +
  • Add ALTER TABLE ... SET AUTHORIZATION syntax to allow changing the table owner. (#5717)

  • +
  • Make EXPLAIN more readable for queries containing timestamp or timestamp with time zone constants. (#5683)

  • +
  • Improve performance for queries with inequality conditions. (#2674)

  • +
  • Improve performance of queries with uncorrelated IN clauses. (#5582)

  • +
  • Use consistent NaN behavior for least(), greatest(), +array_min(), array_max(), min(), max(), +min_by(), and max_by(). +NaN is only returned when it is the only value (except for null +which are ignored for aggregation functions). (#5851)

  • +
  • Restore previous null handling for least() and greatest(). (#5787)

  • +
  • Restore previous null handling for array_min() and array_max(). (#5787)

  • +
  • Remove configuration properties arrayagg.implementation, +multimapagg.implementation, and histogram.implementation. (#4581)

  • +
  • Fix incorrect handling of negative offsets for the time with time zone type. (#5696)

  • +
  • Fix incorrect result when casting time(p) to timestamp(p) for precisions higher than 6. (#5736)

  • +
  • Fix incorrect query results when comparing a timestamp column with a timestamp with time zone constant. (#5685)

  • +
  • Fix improper table alias visibility for queries that select all fields. (#5660)

  • +
  • Fix failure when query parameter appears in a lambda expression. (#5640)

  • +
  • Fix failure for queries containing DISTINCT * and fully-qualified column names in the ORDER BY clause. (#5647)

  • +
  • Fix planning failure for certain queries involving INNER JOIN, GROUP BY and correlated subqueries. (#5846)

  • +
  • Fix recording of query completion event when query is aborted early. (#5815)

  • +
  • Fix exported JMX name for QueryManager. (#5702)

  • +
  • Fix failure when approx_distinct() is used with high precision timestamp(p)/timestamp(p) with time zone/time(p) with time zone +data types. (#5392)

  • +
+
+
+

Web UI#

+
    +
  • Fix “Capture Snapshot” button on the Worker page. (#5759)

  • +
+
+
+

JDBC driver#

+
    +
  • Support number accessor methods like ResultSet.getLong() or ResultSet.getDouble() +on decimal values, as well as char or varchar values that can be unambiguously interpreted as numbers. (#5509)

  • +
  • Add SSLVerification JDBC connection parameter that allows configuring SSL verification. (#5610)

  • +
  • Remove legacy useSessionTimeZone JDBC connection parameter. (#4521)

  • +
  • Implement ResultSet.getRow(). (#5769)

  • +
+
+
+

Server RPM#

+
    +
  • Remove leftover empty directories after RPM uninstall. (#5782)

  • +
+
+
+

BigQuery connector#

+
    +
  • Fix issue when query could return invalid results if some column references were pruned out during query optimization. (#5618)

  • +
+
+
+

Cassandra connector#

+
    +
  • Improve performance of INSERT queries with batch statement. The batch size can be configured via the cassandra.batch-size +configuration property. (#5047)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Fix failure when index mappings do not contain a properties section. (#5807)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for ALTER TABLE ... SET AUTHORIZATION SQL syntax to change the table owner. (#5717)

  • +
  • Add support for writing timestamps with microsecond or nanosecond precision, in addition to milliseconds. (#5283)

  • +
  • Export JMX statistics for Glue metastore client request metrics. (#5693)

  • +
  • Collect column statistics during ANALYZE and when data is inserted to table for columns of timestamp(p) +when precision is greater than 3. (#5392)

  • +
  • Improve query performance by adding support for dynamic bucket pruning. (#5634)

  • +
  • Remove deprecated parquet.fail-on-corrupted-statistics (previously known as hive.parquet.fail-on-corrupted-statistics). +A new configuration property, parquet.ignore-statistics, can be used to deal with Parquet files with incorrect metadata. (#3077)

  • +
  • Do not write min/max statistics for timestamp columns. (#5858)

  • +
  • If multiple metastore URIs are defined via hive.metastore.uri, prefer connecting to one which was seen operational most recently. +This prevents query failures when one or more metastores are misbehaving. (#5795)

  • +
  • Fix Hive view access when catalog name is other than hive. (#5785)

  • +
  • Fix failure when the declared length of a varchar(n) column in the partition schema differs from the table schema. (#5484)

  • +
  • Fix Glue metastore pushdown for complex expressions. (#5698)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add support for materialized views. (#4832)

  • +
  • Remove deprecated parquet.fail-on-corrupted-statistics (previously known as hive.parquet.fail-on-corrupted-statistics). +A new configuration property, parquet.ignore-statistics, can be used to deal with Parquet files with incorrect metadata. (#3077)

  • +
+
+
+

Kafka connector#

+
    +
  • Fix incorrect column comment. (#5751)

  • +
+
+
+

Kudu connector#

+
    +
  • Improve performance of queries having only LIMIT clause. (#3691)

  • +
+
+
+

MySQL connector#

+
    +
  • Improve performance for queries containing a predicate on a varbinary column. (#5672)

  • +
+
+
+

Oracle connector#

+
    +
  • Add support for setting column comments. (#5399)

  • +
  • Allow enabling remarks reporting via oracle.remarks-reporting.enabled configuration property. (#5720)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Improve performance of queries comparing a timestamp column with a timestamp with time zone constants +for timestamp with time zone precision higher than 3. (#5543)

  • +
+
+
+

Other connectors#

+
    +
  • Improve performance of queries with DISTINCT or LIMIT, or with GROUP BY and no aggregate functions and LIMIT, +when the computation can be pushed down to the underlying database for the PostgreSQL, MySQL, Oracle, Redshift and +SQL Server connectors. (#5522)

  • +
+
+
+

SPI#

+
    +
  • Fix propagation of connector session properties to ConnectorNodePartitioningProvider. (#5690)

  • +
  • Add user groups to query events. (#5643)

  • +
  • Add planning time to query completed event. (#5643)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-347.html b/430/release/release-347.html new file mode 100644 index 000000000..60fa74b5b --- /dev/null +++ b/430/release/release-347.html @@ -0,0 +1,2671 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 347 (25 Nov 2020) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 347 (25 Nov 2020)#

+
+

General#

+
    +
  • Add ALTER VIEW ... SET AUTHORIZATION syntax for changing owner of the view. (#5789)

  • +
  • Add support for INTERSECT ALL and EXCEPT ALL. (#2152)

  • +
  • Add contains_sequence() function. (#5593)

  • +
  • Support defining cluster topology (used for query scheduling) using network subnets. (#4862)

  • +
  • Improve query performance by reducing worker to worker communication overhead. (#5905, #5949)

  • +
  • Allow disabling client HTTP response compression, which can improve throughput over fast network links. +Compression can be disabled globally via the query-results.compression-enabled config property, for CLI via +the --disable-compression flag, and for the JDBC driver via the disableCompression driver property. (#5818)

  • +
  • Rename rewrite-filtering-semi-join-to-inner-join session property to rewrite_filtering_semi_join_to_inner_join. (#5954)

  • +
  • Throw a user error when session property value cannot be decoded. (#5731)

  • +
  • Fix query failure when expressions that produce values of type row are used in a VALUES clause. (#3398)

  • +
+
+
+

Server#

+
    +
  • A minimum Java version of 11.0.7 is now required for Presto to start. This is to mitigate JDK-8206955. (#5957)

  • +
+
+
+

Security#

+
    +
  • Add support for multiple LDAP bind patterns. (#5874)

  • +
  • Include groups for view owner when checking permissions for views. (#5945)

  • +
+
+
+

JDBC driver#

+
    +
  • Implement addBatch(), clearBatch() and executeBatch() methods in PreparedStatement. (#5507)

  • +
+
+
+

CLI#

+
    +
  • Add support for providing queries to presto-cli via shell redirection. (#5881)

  • +
+
+
+

Docker image#

+
    +
  • Update Presto docker image to use CentOS 8 as the base image. (#5920)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for ALTER VIEW  ... SET AUTHORIZATION SQL syntax to change the view owner. This supports Presto and Hive views. (#5789)

  • +
  • Allow configuring HDFS replication factor via the hive.dfs.replication config property. (#1829)

  • +
  • Add access checks for tables in Hive Procedures. (#1489)

  • +
  • Decrease latency of INSERT and CREATE TABLE AS ... queries by updating table and column statistics in parallel. (#3638)

  • +
  • Fix leaking S3 connections when querying Avro tables. (#5562)

  • +
+
+
+

Kudu connector#

+
    +
  • Add dynamic filtering support. It can be enabled by setting a non-zero duration value for kudu.dynamic-filtering.wait-timeout config property +or dynamic_filtering_wait_timeout session property. (#5594)

  • +
+
+
+

MongoDB connector#

+
    +
  • Improve performance of queries containing a LIMIT clause. (#5870)

  • +
+
+
+

Other connectors#

+
    +
  • Improve query performance by compacting large pushed down predicates for the PostgreSQL, MySQL, Oracle, +Redshift and SQL Server connectors. Compaction threshold can be changed using the domain-compaction-threshold +config property or domain_compaction_threshold session property. (#6057)

  • +
  • Improve performance for the PostgreSQL, MySQL, SQL Server connectors for certain complex queries involving +aggregation and predicates by pushing the aggregation and predicates computation into the remote database. (#4112)

  • +
+
+
+

SPI#

+
    +
  • Add support for connectors to redirect table scan operations to another connector. (#5792)

  • +
  • Add physical input bytes and rows for table scan operation to query completion event. (#5872)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-348.html b/430/release/release-348.html new file mode 100644 index 000000000..8adc1f9f1 --- /dev/null +++ b/430/release/release-348.html @@ -0,0 +1,2679 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 348 (14 Dec 2020) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 348 (14 Dec 2020)#

+
+

General#

+
    +
  • Add support for DISTINCT clause in aggregations within correlated subqueries. (#5904)

  • +
  • Support SHOW STATS for arbitrary queries. (#3109)

  • +
  • Improve query performance by reducing worker to worker communication overhead. (#6126)

  • +
  • Improve performance of ORDER BY ... LIMIT queries. (#6072)

  • +
  • Reduce memory pressure and improve performance of queries involving joins. (#6176)

  • +
  • Fix EXPLAIN ANALYZE for certain queries that contain broadcast join. (#6115)

  • +
  • Fix planning failures for queries that contain outer joins and aggregations using FILTER (WHERE <condition>) syntax. (#6141)

  • +
  • Fix incorrect results when correlated subquery in join contains aggregation functions such as array_agg or checksum. (#6145)

  • +
  • Fix incorrect query results when using timestamp with time zone constants with precision higher than 3 +describing same point in time but in different zones. (#6318)

  • +
  • Fix duplicate query completion events if query fails early. (#6103)

  • +
  • Fix query failure when views are accessed and current session does not +specify default schema and catalog. (#6294)

  • +
+
+
+

Web UI#

+
    +
  • Add support for OAuth2 authorization. (#5355)

  • +
  • Fix invalid operator stats in Stage Performance view. (#6114)

  • +
+
+
+

JDBC driver#

+
    +
  • Allow reading timestamp with time zone value as ZonedDateTime using ResultSet.getObject(int column, Class<?> type) method. (#307)

  • +
  • Accept java.time.LocalDate in PreparedStatement.setObject(int, Object). (#6301)

  • +
  • Extend PreparedStatement.setObject(int, Object, int) to allow setting time and timestamp values with precision higher than nanoseconds. (#6300) +This can be done via providing a String value representing a valid SQL literal.

  • +
  • Change representation of a row value. ResultSet.getObject now returns an instance of io.prestosql.jdbc.Row class, which better represents +the returned value. Previously a row value was represented as a Map instance, with unnamed fields being named like field0, field1, etc. +You can access the previous behavior by invoking getObject(column, Map.class) on the ResultSet object. (#4588)

  • +
  • Represent varbinary value using hex string representation in ResultSet.getString. Previously the return value was useless, similar to "B@2de82bf8". (#6247)

  • +
  • Report precision of the time(p), time(p) with time zone, timestamp(p) and timestamp(p) with time zone in the DECIMAL_DIGITS column +in the result set returned from DatabaseMetaData#getColumns. (#6307)

  • +
  • Fix the value of the DATA_TYPE column for time(p) and time(p) with time zone in the result set returned from DatabaseMetaData#getColumns. (#6307)

  • +
  • Fix failure when reading a timestamp or timestamp with time zone value with seconds fraction greater than or equal to 999999999500 picoseconds. (#6147)

  • +
  • Fix failure when reading a time value with seconds fraction greater than or equal to 999999999500 picoseconds. (#6204)

  • +
  • Fix element representation in arrays returned from ResultSet.getArray, making it consistent with ResultSet.getObject. +Previously the elements were represented using internal client representation (e.g. String). (#6048)

  • +
  • Fix ResultSetMetaData.getColumnType for timestamp with time zone. Previously the type was miscategorized as java.sql.Types.TIMESTAMP. (#6251)

  • +
  • Fix ResultSetMetaData.getColumnType for time with time zone. Previously the type was miscategorized as java.sql.Types.TIME. (#6251)

  • +
  • Fix failure when an instance of SphericalGeography geospatial type is returned in the ResultSet. (#6240)

  • +
+
+
+

CLI#

+
    +
  • Fix rendering of row values with unnamed fields. Previously they were printed using fake field names like field0, field1, etc. (#4587)

  • +
  • Fix query progress reporting. (#6119)

  • +
  • Fix failure when an instance of SphericalGeography geospatial type is returned to the client. (#6238)

  • +
+
+
+

Hive connector#

+
    +
  • Allow configuring S3 endpoint in security mapping. (#3869)

  • +
  • Add support for S3 streaming uploads. Data is uploaded to S3 as it is written, rather +than staged to a local temporary file. This feature is disabled by default, and can be enabled +using the hive.s3.streaming.enabled configuration property. (#3712, #6201)

  • +
  • Reduce load on metastore when background cache refresh is enabled. (#6101, #6156)

  • +
  • Verify that data is in the correct bucket file when reading bucketed tables. +This is enabled by default, as incorrect bucketing can cause incorrect query results, +but can be disabled using the hive.validate-bucketing configuration property +or the validate_bucketing session property. (#6012)

  • +
  • Allow fallback to legacy Hive view translation logic via hive.legacy-hive-view-translation config property or +legacy_hive_view_translation session property. (#6195)

  • +
  • Add deserializer class name to split information exposed to the event listener. (#6006)

  • +
  • Improve performance when querying tables that contain symlinks. (#6158, #6213)

  • +
+
+
+

Iceberg connector#

+
    +
  • Improve performance of queries containing filters on non-partition columns. Such filters are now used +for optimizing split generation and table scan. (#4932)

  • +
  • Add support for Google Cloud Storage and Azure Storage. (#6186)

  • +
+
+
+

Kafka connector#

+
    +
  • Allow writing timestamp with time zone values into columns using milliseconds-since-epoch or +seconds-since-epoch JSON encoders. (#6074)

  • +
+
+
+

Other connectors#

+
    +
  • Fix ineffective table metadata caching for PostgreSQL, MySQL, SQL Server, Redshift, MemSQL and Phoenix connectors. (#6081, #6167)

  • +
+
+
+

SPI#

+
    +
  • Change SystemAccessControl#filterColumns and ConnectorAccessControl#filterColumns methods to accept a set of +column names, and return a set of visible column names. (#6084)

  • +
  • Expose catalog names corresponding to the splits through the split completion event of the event listener. (#6006)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-349.html b/430/release/release-349.html new file mode 100644 index 000000000..9a47e274a --- /dev/null +++ b/430/release/release-349.html @@ -0,0 +1,2542 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 349 (28 Dec 2020) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 349 (28 Dec 2020)#

+
+

Warning

+

This release is broken and should not be used.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-350.html b/430/release/release-350.html new file mode 100644 index 000000000..b6679d2c0 --- /dev/null +++ b/430/release/release-350.html @@ -0,0 +1,2651 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 350 (28 Dec 2020) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 350 (28 Dec 2020)#

+
+

General#

+
    +
  • Add HTTP client JMX metrics. (#6453)

  • +
  • Improve query performance by reducing worker to worker communication overhead. (#6283, #6349)

  • +
  • Improve performance of queries that contain IS NOT DISTINCT FROM join predicates. (#6404)

  • +
  • Fix failure when restricted columns have column masks. (#6017)

  • +
  • Fix failure when try expressions reference columns that contain @ or : in their names. (#6380)

  • +
  • Fix memory management config handling to use query.max-total-memory-per-node +rather than only using query.max-memory-per-node for both values. (#6349)

  • +
+
+
+

Web UI#

+
    +
  • Fix truncation of query text in cluster overview page. (#6216)

  • +
+
+
+

JDBC driver#

+
    +
  • Accept java.time.OffsetTime in PreparedStatement.setObject(int, Object). (#6352)

  • +
  • Extend PreparedStatement.setObject(int, Object, int) to allow setting time with time zone and timestamp with time zone +values with precision higher than nanoseconds. This can be done via providing a String value representing a valid SQL literal. (#6352)

  • +
+
+
+

BigQuery connector#

+
    +
  • Fix incorrect results for count(*) queries with views. (#5635)

  • +
+
+
+

Cassandra connector#

+
    +
  • Support DELETE statement with primary key or partition key. (#4059)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Improve query analysis performance when Elasticsearch contains many index mappings. (#6368)

  • +
+
+
+

Kafka connector#

+
    +
  • Support Kafka Schema Registry for Avro topics. (#6137)

  • +
+
+
+

SQL Server connector#

+
    +
  • Add data_compression table property to control the target compression in SQL Server. +The allowed values are NONE, ROW or PAGE. (#4693)

  • +
+
+
+

Other connectors#

+

This change applies to the MySQL, Oracle, PostgreSQL, Redshift, and SQL Server connectors.

+
    +
  • Send shorter and potentially more performant queries to remote database when a Presto query has a NOT IN +predicate eligible for pushdown into the connector. (#6075)

  • +
+
+
+

SPI#

+
    +
  • Rename LongTimeWithTimeZone.getPicoSeconds() to LongTimeWithTimeZone.getPicoseconds(). (#6354)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-351.html b/430/release/release-351.html new file mode 100644 index 000000000..aeb8eb3c3 --- /dev/null +++ b/430/release/release-351.html @@ -0,0 +1,2668 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 351 (3 Jan 2021) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+ +
+
+ +
+
+ +
+

Release 351 (3 Jan 2021)#

+
+

General#

+
    +
  • Rename client protocol headers to start with X-Trino-. +Legacy clients can be supported by setting the configuration property +protocol.v1.alternate-header-name to Presto. This configuration +property is deprecated and will be removed in a future release.

  • +
+
+
+

JMX MBean naming#

+
    +
  • Rename base domain name for server MBeans to trino. The name can +be changed using the configuration property jmx.base-name.

  • +
  • Rename base domain name for the Elasticsearch, Hive, Iceberg, Raptor, +and Thrift connectors to trino.plugin. The name can be changed +using the catalog configuration property jmx.base-name.

  • +
+
+
+

Server RPM#

+
    +
  • Rename installation directories from presto to trino.

  • +
+
+
+

Docker image#

+
    +
  • Publish image as trinodb/trino.

  • +
  • Change base image to azul/zulu-openjdk-centos.

  • +
  • Change configuration directory to /etc/trino.

  • +
  • Rename CLI in image to trino.

  • +
+
+
+

CLI#

+
    +
  • Use new client protocol header names. The CLI is not compatible with older servers.

  • +
+
+
+

JDBC driver#

+
    +
  • Use new client protocol header names. The driver is not compatible with older servers.

  • +
  • Change driver URL prefix to jdbc:trino:. +The old prefix is deprecated and will be removed in a future release.

  • +
  • Change driver class to io.trino.jdbc.TrinoDriver. +The old class name is deprecated and will be removed in a future release.

  • +
  • Rename Java package for all driver classes to io.trino.jdbc and rename +various driver classes such as TrinoConnection to start with Trino.

  • +
+
+
+

Hive connector#

+
    +
  • Rename JMX name for PrestoS3FileSystem to TrinoS3FileSystem.

  • +
  • Change configuration properties +hive.hdfs.presto.principal to hive.hdfs.trino.principal and +hive.hdfs.presto.keytab to hive.hdfs.trino.keytab. +The old names are deprecated and will be removed in a future release.

  • +
+
+
+

Local file connector#

+
    +
  • Change configuration properties +presto-logs.http-request-log.location to trino-logs.http-request-log.location and +presto-logs.http-request-log.pattern to trino-logs.http-request-log.pattern. +The old names are deprecated and will be removed in a future release.

  • +
+
+
+

Thrift connector#

+
    +
  • Rename Thrift service method names starting with presto to trino.

  • +
  • Rename all classes in the Thrift IDL starting with Presto to Trino.

  • +
  • Rename configuration properties starting with presto to trino.

  • +
+
+
+

SPI#

+
    +
  • Rename Java package to io.trino.spi.

  • +
  • Rename PrestoException to TrinoException.

  • +
  • Rename PrestoPrincipal to TrinoPrincipal.

  • +
  • Rename PrestoWarning to TrinoWarning.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-352.html b/430/release/release-352.html new file mode 100644 index 000000000..ddd5393c2 --- /dev/null +++ b/430/release/release-352.html @@ -0,0 +1,2744 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 352 (9 Feb 2021) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 352 (9 Feb 2021)#

+
+

General#

+
    +
  • Add support for WINDOW clause. (#651)

  • +
  • Add support for UPDATE. (#5861)

  • +
  • Add version() function. (#4627)

  • +
  • Allow prepared statement parameters for SHOW STATS. (#6582)

  • +
  • Update tzdata version to 2020d. As a result, queries can no longer reference the +US/Pacific-New zone, as it has been removed. (#6660)

  • +
  • Add plan-with-table-node-partitioning feature config that corresponds to +existing plan_with_table_node_partitioning session property. (#6811)

  • +
  • Improve performance of queries using rank() window function. (#6333)

  • +
  • Improve performance of sum() and avg() for decimal types. (#6951)

  • +
  • Improve join performance. (#5981)

  • +
  • Improve query planning time for queries using range predicates or large IN lists. (#6544)

  • +
  • Fix window and streaming aggregation semantics regarding peer rows. Now peer rows are +grouped using IS NOT DISTINCT FROM instead of the = operator. (#6472)

  • +
  • Fix query failure when using an element of array(timestamp(p)) in a complex expression +for p greater than 6. (#6350)

  • +
  • Fix failure when using geospatial functions in a join clause and spatial_partitioning_table_name is set. (#6587)

  • +
  • Fix CREATE TABLE AS failure when source table has hidden columns. (#6835)

  • +
+
+
+

Security#

+
    +
  • Allow configuring HTTP client used for OAuth2 authentication. (#6600)

  • +
  • Add token polling client API for OAuth2 authentication. (#6625)

  • +
  • Support JWK with certificate chain for OAuth2 authorization. (#6428)

  • +
  • Add scopes to OAuth2 configuration. (#6580)

  • +
  • Optionally verify JWT audience (aud) field for OAuth2 authentication. (#6501)

  • +
  • Guard against replay attacks in OAuth2 by using nonce cookie when openid scope is requested. (#6580)

  • +
+
+
+

JDBC driver#

+
    +
  • Add OAuth2 authentication. (#6576)

  • +
  • Support user impersonation when using password-based authentication +using the new sessionUser parameter. (#6549)

  • +
+
+
+

Docker image#

+
    +
  • Remove support for configuration directory /usr/lib/trino/etc. The configuration +should be provided in /etc/trino. (#6497)

  • +
+
+
+

CLI#

+
    +
  • Support user impersonation when using password-based authentication using the +--session-user command line option. (#6567)

  • +
+
+
+

BigQuery connector#

+
    +
  • Add a view_definition system table which exposes BigQuery view definitions. (#3687)

  • +
  • Fix query failure when calculating count(*) aggregation on a view more than once, +without any filter. (#6706).

  • +
+
+
+

Hive connector#

+
    +
  • Add UPDATE support for ACID tables. (#5861)

  • +
  • Match columns by index rather than by name by default for ORC ACID tables. (#6479)

  • +
  • Match columns by name rather than by index by default for Parquet files. +This can be changed using hive.parquet.use-column-names configuration property and parquet_use_column_names +session property. (#6479)

  • +
  • Remove the hive.partition-use-column-names configuration property and the +partition_use_column_names session property. This is now determined automatically. (#6479)

  • +
  • Support timestamps with microsecond or nanosecond precision (as configured with +hive.timestamp-precision property) nested within array, map or struct data types. (#5195)

  • +
  • Support reading from table in Sequencefile format that uses LZO compression. (#6452)

  • +
  • Expose AWS HTTP Client stats via JMX. (#6503)

  • +
  • Allow specifying S3 KMS Key ID used for client side encryption via security mapping +config and extra credentials. (#6802)

  • +
  • Fix writing incorrect timestamp values within row, array or map when using Parquet file format. (#6760)

  • +
  • Fix possible S3 connection leak on query failure. (#6849)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add iceberg.max-partitions-per-writer config property to allow configuring the limit on partitions per writer. (#6650)

  • +
  • Optimize cardinality-insensitive aggregations (max(), min(), distinct(), approx_distinct()) +over identity partition columns with optimizer.optimize-metadata-queries config property +or optimize_metadata_queries session property. (#5199)

  • +
  • Provide use_file_size_from_metadata catalog session property and iceberg.use-file-size-from-metadata +config property to fix query failures on tables with wrong file sizes stored in the metadata. (#6369)

  • +
  • Fix the mapping of nested fields between table metadata and ORC file metadata. This +enables evolution of row typed columns for Iceberg tables stored in ORC. (#6520)

  • +
+
+
+

Kinesis connector#

+
    +
  • Support GZIP message compression. (#6442)

  • +
+
+
+

MySQL connector#

+
    +
  • Improve performance for certain complex queries involving aggregation and predicates (e.g. HAVING clause) +by pushing the aggregation and predicates computation into the remote database. (#6667)

  • +
  • Improve performance for certain queries using stddev_pop, stddev_samp, var_pop, var_samp aggregation +functions by pushing the aggregation and predicates computation into the remote database. (#6673)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Improve performance for certain complex queries involving aggregation and predicates (e.g. HAVING clause) +by pushing the aggregation and predicates computation into the remote database. (#6667)

  • +
  • Improve performance for certain queries using stddev_pop, stddev_samp, var_pop, var_samp, +covar_pop, covar_samp, corr, regr_intercept, regr_slope aggregation functions +by pushing the aggregation and predicates computation into the remote database. (#6731)

  • +
+
+
+

Redshift connector#

+
    +
  • Use the Redshift JDBC driver to access Redshift. As a result, connection-url in catalog +configuration files needs to be updated from jdbc:postgresql:... to jdbc:redshift:.... (#6465)

  • +
+
+
+

SQL Server connector#

+
    +
  • Avoid query failures due to transaction deadlocks in SQL Server by using transaction snapshot isolation. (#6274)

  • +
  • Honor precision of SQL Server’s datetime2 type . (#6654)

  • +
  • Add support for Trino timestamp type in CREATE TABLE statement, by mapping it to SQL Server’s datetime2 type. +Previously, it was incorrectly mapped to SQL Server’s timestamp type. (#6654)

  • +
  • Add support for the time type. (#6654)

  • +
  • Improve performance for certain complex queries involving aggregation and predicates (e.g. HAVING clause) +by pushing the aggregation and predicates computation into the remote database. (#6667)

  • +
  • Fix failure when querying tables having indexes and constraints. (#6464)

  • +
+
+
+

SPI#

+
    +
  • Add support for join pushdown via the ConnectorMetadata.applyJoin() method. (#6752)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-353.html b/430/release/release-353.html new file mode 100644 index 000000000..1d092aa69 --- /dev/null +++ b/430/release/release-353.html @@ -0,0 +1,2742 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 353 (5 Mar 2021) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 353 (5 Mar 2021)#

+
+

General#

+
    +
  • Add ClickHouse connector. (#4500)

  • +
  • Extend support for correlated subqueries including UNNEST. (#6326, #6925, #6951)

  • +
  • Add to_geojson_geometry() and from_geojson_geometry() functions. (#6355)

  • +
  • Add support for values of any integral type (tinyint, smallint, integer, bigint, decimal(p, 0)) +in window frame bound specification. (#6897)

  • +
  • Improve query planning time for queries containing IN predicates with many elements. (#7015)

  • +
  • Fix potential incorrect results when columns from WITH clause are exposed with aliases. (#6839)

  • +
  • Fix potential incorrect results for queries containing multiple < predicates. (#6896)

  • +
  • Always show SECURITY clause in SHOW CREATE VIEW. (#6913)

  • +
  • Fix reporting of column references for aliased tables in QueryCompletionEvent. (#6972)

  • +
  • Fix potential compiler failure when constructing an array with more than 128 elements. (#7014)

  • +
  • Fail SHOW COLUMNS when column metadata cannot be retrieved. (#6958)

  • +
  • Fix rendering of function references in EXPLAIN output. (#6703)

  • +
  • Fix planning failure when WITH clause contains hidden columns. (#6838)

  • +
  • Prevent client hangs when OAuth2 authentication fails. (#6659)

  • +
+
+
+

Server RPM#

+
    +
  • Allow configuring process environment variables through /etc/trino/env.sh. (#6635)

  • +
+
+
+

BigQuery connector#

+
    +
  • Add support for CREATE TABLE and DROP TABLE statements. (#3767)

  • +
  • Allow for case-insensitive identifiers matching via bigquery.case-insensitive-name-matching config property. (#6748)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for current_user() in Hive defined views. (#6720)

  • +
  • Add support for reading and writing column statistics from Glue metastore. (#6178)

  • +
  • Improve parallelism of bucketed tables inserts. Inserts into bucketed tables can now be parallelized +within task using task.writer-count feature config. (#6924, #6866)

  • +
  • Fix a failure when INSERT writes to a partition created by an earlier INSERT statement. (#6853)

  • +
  • Fix handling of folders created using the AWS S3 Console. (#6992)

  • +
  • Fix query failures on information_schema.views table when there are failures +translating hive view definitions. (#6370)

  • +
+
+
+

Iceberg connector#

+
    +
  • Fix handling of folders created using the AWS S3 Console. (#6992)

  • +
  • Fix query failure when reading nested columns with field names that may +contain upper case characters. (#7180)

  • +
+
+
+

Kafka connector#

+
    +
  • Fix failure when querying Schema Registry tables. (#6902)

  • +
  • Fix querying of Schema Registry tables with References in their schema. (#6907)

  • +
  • Fix listing of schema registry tables having ambiguous subject name in lower case. (#7048)

  • +
+
+
+

MySQL connector#

+
    +
  • Fix failure when reading a timestamp or datetime value with more than 3 decimal digits +in the fractional seconds part. (#6852)

  • +
  • Fix incorrect predicate pushdown for char and varchar column with operators +like <>, <, <=, > and >= due different case sensitivity between Trino +and MySQL. (#6746, #6671)

  • +
+
+
+

MemSQL connector#

+
    +
  • Fix failure when reading a timestamp or datetime value with more than 3 decimal digits +of the second fraction. (#6852)

  • +
  • Fix incorrect predicate pushdown for char and varchar column with operators +like <>, <, <=, > and >= due different case sensitivity between Trino +and MemSQL. (#6746, #6671)

  • +
+
+
+

Phoenix connector#

+
    +
  • Add support for Phoenix 5.1. This can be used by setting connector.name=phoenix5 in catalog +configuration properties. (#6865)

  • +
  • Fix failure when query contains a LIMIT exceeding 2147483647. (#7169)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Improve performance of queries with ORDER BY ... LIMIT clause, when the computation +can be pushed down to the underlying database. This can be enabled by setting topn-pushdown.enabled. +Enabling this feature can currently result in incorrect query results when sorting +on char or varchar columns. (#6847)

  • +
  • Fix incorrect predicate pushdown for char and varchar column with operators +like <>, <, <=, > and >= due different case collation between Trino +and PostgreSQL. (#3645)

  • +
+
+
+

Redshift connector#

+
    +
  • Fix failure when reading a timestamp value with more than 3 decimal digits of +the second fraction. (#6893)

  • +
+
+
+

SQL Server connector#

+
    +
  • Abort queries on the SQL Server side when the Trino query is finished. (#6637)

  • +
  • Fix incorrect predicate pushdown for char and varchar column with operators +like <>, <, <=, > and >= due different case sensitivity between Trino +and SQL Server. (#6753)

  • +
+
+
+

Other connectors#

+
    +
  • Reduce number of opened JDBC connections during planning for ClickHouse, Druid, MemSQL, MySQL, +Oracle, Phoenix, Redshift, and SQL Server connectors. (#7069)

  • +
  • Add experimental support for join pushdown in PostgreSQL, MySQL, MemSQL, Oracle, and SQL Server connectors. +It can be enabled with the experimental.join-pushdown.enabled=true catalog configuration property. (#6874)

  • +
+
+
+

SPI#

+
    +
  • Fix lazy blocks to call listeners that are registered after the top level block is already loaded. +Previously, such registered listeners were not called when the nested blocks were later loaded. (#6783)

  • +
  • Fix case where LazyBlock.getFullyLoadedBlock() would not load nested blocks +when the top level block was already loaded. (#6783)

  • +
  • Do not include coordinator node in the result of ConnectorAwareNodeManager.getWorkerNodes() +when node-scheduler.include-coordinator is false. (#7007)

  • +
  • The function name passed to ConnectorMetadata.applyAggregation() +is now the canonical function name. Previously, if query used function alias, the alias +name was passed. (#6189)

  • +
  • Add support for redirecting table scans to multiple tables that are unioned together. (#6679)

  • +
  • Change return type of Range.intersect(Range). The method now +returns Optional.empty() instead of throwing when ranges do not overlap. (#6976)

  • +
  • Change signature of ConnectorMetadata.applyJoin() to have an additional JoinStatistics argument. (#7000)

  • +
  • Deprecate io.trino.spi.predicate.Marker.

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-354.html b/430/release/release-354.html new file mode 100644 index 000000000..e086ba3e7 --- /dev/null +++ b/430/release/release-354.html @@ -0,0 +1,2699 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 354 (19 Mar 2021) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 354 (19 Mar 2021)#

+
+

General#

+
    +
  • Improve performance of certain queries involving LEFT, RIGHT and FULL JOIN +when one side of the join is known to produce a single row. (#7090)

  • +
  • Preferred write partitioning is now triggered automatically when the estimated number +of written partitions exceeds or is equal to preferred-write-partitioning-min-number-of-partitions. +This estimation requires that the input tables of the query have statistics. To enable +preferred partitioning unconditionally, set preferred-write-partitioning-min-number-of-partitions to 1. +To disable preferred write partitioning, set use-preferred-write-partitioning to false. +(#6920)

  • +
  • Fix incorrect results when multiple similar subqueries contain UNION clauses that differ +only in the ALL vs DISTINCT qualifier. (#7345)

  • +
  • Fix DELETE and UPDATE for connectors that subsume filters. (#7302)

  • +
  • Fix failure when subqueries contain EXCEPT or INTERSECT. (#7342)

  • +
  • Fix failure of queries that contain RIGHT JOIN when late materialization is enabled. (#6784)

  • +
+
+
+

Security#

+
    +
  • Fix retries for OAuth 2.0 authentication in case of token expiry. (#7172)

  • +
+
+
+

CLI#

+
    +
  • Support OAuth 2.0 authentication. (#7054)

  • +
+
+
+

ClickHouse connector#

+
    +
  • Use correct case for name of the schema in CREATE SCHEMA. (#7239)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Fix failure when reading single-valued fields for array types. (#7012)

  • +
+
+
+

Hive connector#

+
    +
  • Respect partition filter for DELETE and UPDATE of ACID tables. Previously, the partition +filter was ignored, resulting in the deletion or update of too many rows. (#7302)

  • +
  • Fix allocation of statement IDs for ACID tables, which could result in query failure or +data loss due to creating multiple delta files with the same name. (#7302)

  • +
  • Fix incorrect query results when reading from an incorrectly bucketed table created and registered +with the metastore by Spark. (#6848)

  • +
  • Avoid leaking file system connections or other resources when using the Avro file format. (#7178)

  • +
  • Fix query failure when columns of a CSV table are declared as a type other than varchar (string) in Glue +metastore. Columns are now interpreted as varchar values, instead. (#7059)

  • +
  • Rename hive.parallel-partitioned-bucketed-inserts configuration property to hive.parallel-partitioned-bucketed-writes. (#7259)

  • +
+
+
+

Iceberg connector#

+
    +
  • Fix queries on empty tables without a snapshot ID that were created by Spark. (#7233)

  • +
  • Update to Iceberg 0.11.0 behavior for transforms of dates and timestamps +before 1970. Data written by older versions of Trino and Iceberg will be +read correctly. New data will be written correctly, but may be read +incorrectly by older versions of Trino and Iceberg. (#7049)

  • +
+
+
+

MemSQL connector#

+
    +
  • Add support for MemSQL 3.2. (#7179)

  • +
  • Use correct case for name of the schema in CREATE SCHEMA. (#7239)

  • +
  • Improve performance of queries with ORDER BY ... LIMIT clause when the computation +can be pushed down to the underlying database. (#7326)

  • +
+
+
+

MySQL connector#

+
    +
  • Use proper column type (datetime(3)) in MySQL when creating a table with timestamp(3) column. +Previously, the second fraction was being truncated. (#6909)

  • +
  • Use correct case for name of the schema in CREATE SCHEMA. (#7239)

  • +
  • Improve performance of queries with ORDER BY ... LIMIT clause when the computation +can be pushed down to the underlying database. (#7326)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Fix incorrect query results for ORDER BY ... LIMIT clause when sorting on char or varchar columns +and topn-pushdown.enabled configuration property is enabled. The optimization is now enabled by default. +(#7170, #7314)

  • +
  • Use correct case for name of the schema in CREATE SCHEMA. (#7239)

  • +
+
+
+

Redshift connector#

+
    +
  • Fix failure when query contains a LIMIT exceeding 2147483647. (#7236)

  • +
  • Use correct case for name of the schema in CREATE SCHEMA. (#7239)

  • +
+
+
+

SQL Server connector#

+
    +
  • Add support for parametric time type. (#7122)

  • +
  • Use correct case for name of the schema in CREATE SCHEMA. (#7239)

  • +
  • Improve performance of queries with ORDER BY ... LIMIT clause when the computation +can be pushed down to the underlying database. (#7324)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-355.html b/430/release/release-355.html new file mode 100644 index 000000000..68a4d65bf --- /dev/null +++ b/430/release/release-355.html @@ -0,0 +1,2638 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 355 (8 Apr 2021) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+ +
+
+ +
+
+ +
+

Release 355 (8 Apr 2021)#

+
+

General#

+
    +
  • Report tables that are directly referenced by a query in QueryCompletedEvent. (#7330)

  • +
  • Report columns that are the target of INSERT or UPDATE queries in QueryCompletedEvent. This includes +information about which input columns they are derived from. (#7425, #7465)

  • +
  • Rename optimizer.plan-with-table-node-partitioning config property to optimizer.use-table-scan-node-partitioning. (#7257)

  • +
  • Improve query parallelism when table bucket count is small compared to number of nodes. +This optimization is now triggered automatically when the ratio between table buckets and +possible table scan tasks exceeds or is equal to optimizer.table-scan-node-partitioning-min-bucket-to-task-ratio. (#7257)

  • +
  • Include information about Spill to disk in EXPLAIN ANALYZE. (#7427)

  • +
  • Disallow inserting data into tables that have row filters. (#7346)

  • +
  • Improve performance of queries that can benefit from both Cost-based optimizations and join pushdown +by giving precedence to cost-based optimizations. (#7331)

  • +
  • Fix inconsistent behavior for to_unixtime() with values of type timestamp(p). (#7450)

  • +
  • Change return type of from_unixtime() and from_unixtime_nanos() to timestamp(p) with time zone. (#7460)

  • +
+
+
+

Security#

+
    +
  • Add support for configuring multiple password authentication plugins. (#7151)

  • +
+
+
+

JDBC driver#

+
    +
  • Add assumeLiteralNamesInMetadataCallsForNonConformingClients parameter for use as a workaround when +applications do not properly escape schema or table names in calls to DatabaseMetaData methods. (#7438)

  • +
+
+
+

ClickHouse connector#

+
    +
  • Support creating tables with MergeTree storage engine. (#7135)

  • +
+
+
+

Hive connector#

+
    +
  • Support Hive views containing LATERAL VIEW json_tuple(...) AS ... syntax. (#7242)

  • +
  • Fix incorrect results when reading from a Hive view that uses array subscript operators. (#7271)

  • +
  • Fix incorrect results when querying the $file_modified_time hidden column. (#7511)

  • +
+
+
+

Phoenix connector#

+
    +
  • Improve performance when fetching table metadata during query analysis. (#6975)

  • +
  • Improve performance of queries with ORDER BY ... LIMIT clause when the computation +can be pushed down to the underlying database. (#7490)

  • +
+
+
+

SQL Server connector#

+
    +
  • Improve performance when fetching table metadata during query analysis. (#6975)

  • +
+
+
+

SPI#

+
    +
  • Engine now uses ConnectorMaterializedViewDefinition#storageTable +to determine materialized view storage table. (#7319)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-356.html b/430/release/release-356.html new file mode 100644 index 000000000..df9a086ff --- /dev/null +++ b/430/release/release-356.html @@ -0,0 +1,2752 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 356 (30 Apr 2021) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 356 (30 Apr 2021)#

+
+

General#

+
    +
  • Add support for MATCH_RECOGNIZE. (#6111)

  • +
  • Add soundex() function. (#4022)

  • +
  • Introduce system.metadata.materialized_view_properties table for listing available materialized view properties. (#7615)

  • +
  • Add support for limiting the maximum planning time via the query.max-planning-time configuration property. (#7213)

  • +
  • Allow redirecting clients to an alternative location to fetch query information. This can be configured +via the query.info-url-template configuration property. (#7678)

  • +
  • Allow cancellation of queries during planning phase. (#7213)

  • +
  • Improve performance of ORDER BY ... LIMIT queries over a LEFT JOIN. (#7028)

  • +
  • Improve performance of queries with predicates on boolean columns. (#7263)

  • +
  • Improve planning time for queries with large IN predicates. (#7556)

  • +
  • Improve performance of queries that contain joins on varchar keys of different length. (#7644)

  • +
  • Improve performance of queries when late materialization is enabled. (#7695)

  • +
  • Reduce coordinator network overhead when scheduling queries. (#7351)

  • +
  • Fix possible deadlock for JOIN queries when spilling is enabled. (#7455)

  • +
  • Fix incorrect results for queries containing full outer join with an input that is known to produce one row. (#7629)

  • +
  • Fix failure when quantified comparison expressions contain scalar subqueries. (#7792)

  • +
+
+
+

Security#

+
    +
  • Materialized views require UPDATE privilege to be refreshed. (#7707)

  • +
  • Add dedicated access control for creating and dropping materialized views. (#7645)

  • +
  • Add dedicated access control for refreshing materialized views. Insert privilege on storage table is no +longer required. (#7707)

  • +
  • Fix authentication failure when providing multiple scope values for http-server.authentication.oauth2.scopes. (#7706)

  • +
+
+
+

JDBC driver#

+
    +
  • Add support for caching OAuth2 credentials in memory to avoid unnecessary authentication flows. (#7309)

  • +
+
+
+

BigQuery connector#

+
    +
  • Add support for CREATE SCHEMA and DROP SCHEMA statements. (#7543)

  • +
  • Improve table listing performance when case insensitive matching is enabled. (#7628)

  • +
+
+
+

Cassandra connector#

+
    +
  • Fix NullPointerException when reading an empty timestamp value. (#7433)

  • +
+
+
+

Hive connector#

+
    +
  • Improve performance when reading dictionary-encoded Parquet files. (#7754)

  • +
  • Fix incorrect results when referencing nested fields with non-lowercase names from ORC files. (#7350)

  • +
  • Always use row-by-row deletes for ACID tables rather than dropping partitions. (#7621)

  • +
  • Allow reading from ORC ACID transactional tables when _orc_acid_version metadata files are missing. (#7579)

  • +
  • Add UPDATE support for ACID tables that were originally created as non-transactional. (#7622)

  • +
  • Add support for connection proxying for Azure ADLS endpoints. (#7509)

  • +
+
+
+

Iceberg connector#

+
    +
  • Show Iceberg tables created by other engines in SHOW TABLES output. (#1592)

  • +
  • Improve performance when reading dictionary-encoded Parquet files. (#7754)

  • +
  • Improve query planning through table metadata caching. (#7336)

  • +
  • Fix failure querying materialized views that were created using the session catalog and schema. (#7711)

  • +
  • Fix listing of materialized views when using SHOW TABLES query. (#7790)

  • +
+
+
+

Kafka connector#

+
    +
  • Add support for TLS security protocol. (#6929)

  • +
+
+
+

MemSQL connector#

+
    +
  • Improve metadata caching hit rate. (#7039)

  • +
+
+
+

MongoDB connector#

+
    +
  • Fix handling of non-lowercase MongoDB views. (#7491)

  • +
+
+
+

MySQL connector#

+
    +
  • Improve metadata caching hit rate. (#7039)

  • +
  • Exclude an internal sys schema from schema listings. (#6337)

  • +
+
+
+

Oracle connector#

+
    +
  • Improve metadata caching hit rate. (#7039)

  • +
+
+
+

Phoenix connector#

+
    +
  • Improve metadata caching hit rate. (#7039)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Improve metadata caching hit rate. (#7039)

  • +
  • Cancel query on PostgreSQL when the Trino query is cancelled. (#7306)

  • +
  • Discontinue support for PostgreSQL 9.5, which has reached end of life. (#7676)

  • +
+
+
+

Redshift connector#

+
    +
  • Improve metadata caching hit rate. (#7039)

  • +
+
+
+

SQL Server connector#

+
    +
  • Improve metadata caching hit rate. (#7039)

  • +
  • Fix query failure when snapshot isolation is disabled in target SQL Server database, +but READ_COMMITTED_SNAPSHOT is still enabled. (#7548)

  • +
  • Fix reading date values before 1583-10-14. (#7634)

  • +
+
+
+

SPI#

+
    +
  • Require that ConnectorMaterializedViewDefinition provides a view owner. (#7489)

  • +
  • Add Connector#getMaterializedViewPropertyManager for specifying materialized view properties. (#7615)

  • +
  • Add ConnectorAccessControl.checkCanCreateMaterializedView() and +ConnectorAccessControl.checkCanDropMaterializedView() for authorizing +creation and removal of materialized views. (#7645)

  • +
  • Allow a materialized view to return a storage table in a different catalog or schema. (#7638)

  • +
  • Add ConnectorAccessControl.checkCanRefreshMaterializedView() for authorizing +refresh of materialized views. (#7707)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-357.html b/430/release/release-357.html new file mode 100644 index 000000000..76e4f638b --- /dev/null +++ b/430/release/release-357.html @@ -0,0 +1,2709 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 357 (21 May 2021) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 357 (21 May 2021)#

+
+

General#

+
    +
  • Add support for subquery expressions that return multiple columns. +Example: SELECT x = (VALUES (1, 'a')) (#7773, #7863)

  • +
  • Allow aggregation pushdown when COUNT(1) is used with GROUP BY. (#7251)

  • +
  • Add support for CURRENT_CATALOG and CURRENT_SCHEMA. (#7824)

  • +
  • Add format_number() function. (#1878)

  • +
  • Change row to json cast to produce JSON objects instead of JSON arrays. This behavior can be restored +with the deprecated.legacy-row-to-json-cast configuration option. (#3536)

  • +
  • Print dynamic filters summary in EXPLAIN ANALYZE. (#7874)

  • +
  • Improve performance for queries using IN predicate with a short list of constants. (#7840)

  • +
  • Release memory immediately when queries involving window functions fail. (#7947)

  • +
  • Fix incorrect handling of row expressions for IN predicates, quantified comparisons and scalar subqueries. Previously, +the queries would succeed where they should have failed with a type mismatch error. (#7797)

  • +
  • Fix failure when using PREPARE with a GRANT statement that contains quoted SQL keywords. (#7941)

  • +
  • Fix cluster instability after executing certain large EXPLAIN queries. (#8017)

  • +
+
+
+

Security#

+
    +
  • Enforce materialized view creator security policies when view is fresh. (#7618)

  • +
  • Use system truststore for OAuth2 and JWK for JWT authentication. Previously, the truststore +configured for internal communication was used. This means that globally trusted certificates +will work by default. (#7936)

  • +
  • Fix handling of SNI for multiple TLS certificates. (#8007)

  • +
+
+
+

Web UI#

+
    +
  • Make the UI aware of principal-field (configured with http-server.authentication.oauth2.principal-field) when +web-ui.authentication.type is set to oauth2. (#7526)

  • +
+
+
+

JDBC driver#

+
    +
  • Cancel Trino query execution when JDBC statement is closed. (#7819)

  • +
  • Close statement when connection is closed. (#7819)

  • +
+
+
+

CLI#

+
    +
  • Add clear command to clear the screen. (#7632)

  • +
+
+
+

BigQuery connector#

+
    +
  • Fix failures for queries accessing information_schema.columns when case-insensitive-name-matching is disabled. (#7830)

  • +
  • Fix query failure when a predicate on a BigQuery string column contains a value with a single quote ('). (#7784)

  • +
+
+
+

ClickHouse connector#

+
    +
  • Improve performance of aggregation queries by computing aggregations within ClickHouse. Currently, the following aggregate functions +are eligible for pushdown: count, min, max, sum and avg. (#7434)

  • +
  • Map ClickHouse UUID columns as UUID type in Trino instead of VARCHAR. (#7097)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Support decoding timestamp columns encoded as strings containing milliseconds since epoch values. (#7838)

  • +
  • Retry requests with backoff when Elasticsearch is overloaded. (#7423)

  • +
+
+
+

Kinesis connector#

+
    +
  • Add kinesis.table-description-refresh-interval configuration property to set the +refresh interval for fetching table descriptions from S3. (#1609)

  • +
+
+
+

Kudu connector#

+
    +
  • Fix query failures for grouped execution on range partitioned tables. (#7738)

  • +
+
+
+

MongoDB connector#

+
    +
  • Redact the value of mongodb.credentials in the server log. (#7862)

  • +
  • Add support for dropping columns. (#7853)

  • +
+
+
+

Pinot connector#

+
    +
  • Add support for complex filter expressions in passthrough queries. (#7161)

  • +
+
+
+

Other connectors#

+

This change applies to the Druid, MemSQL, MySQL, Oracle, Phoenix, PosgreSQL, Redshift, and SQL Server connectors.

+
    +
  • Add rule support for identifier mapping. The rules can be configured via the +case-insensitive-name-matching.config-file configuration property. (#7841)

  • +
+
+
+

SPI#

+
    +
  • Make ConnectorMaterializedViewDefinition non-serializable. It is the responsibility of the connector to serialize +and store the materialized view definitions in an appropriate format. (#7762)

  • +
  • Deprecate TupleDomain.transform. (#7980)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-358.html b/430/release/release-358.html new file mode 100644 index 000000000..b5278d36c --- /dev/null +++ b/430/release/release-358.html @@ -0,0 +1,2617 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 358 (1 Jun 2021) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 358 (1 Jun 2021)#

+
+

General#

+
    +
  • Support arbitrary queries in SHOW STATS. (#8026)

  • +
  • Improve performance of complex queries involving joins and TABLESAMPLE. (#8094)

  • +
  • Improve performance of ORDER BY ... LIMIT queries on sorted data. (#6634)

  • +
  • Reduce graceful shutdown time for worker nodes. (#8149)

  • +
  • Fix query failure columns of non-orderable types (e.g. HyperLogLog, tdigest, etc.), are involved in a join. (#7723)

  • +
  • Fix failure for queries containing repeated ordinals in a GROUP BY clause. +Example: SELECT x FROM t GROUP BY 1, 1. (#8023)

  • +
  • Fix failure for queries containing repeated expressions in the ORDER BY clause of an aggregate function. +Example: SELECT array_agg(x ORDER BY y, y) FROM (VALUES ('a', 2)) t(x, y). (#8080)

  • +
+
+
+

JDBC Driver#

+
    +
  • Remove legacy JDBC URL prefix jdbc:presto:. (#8042)

  • +
  • Remove legacy driver classes io.prestosql.jdbc.PrestoDriver +and com.facebook.presto.jdbc.PrestoDriver. (#8042)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for reading from Hive views that use LATERAL VIEW EXPLODE +or LATERAL VIEW OUTER EXPLODE over array of STRUCT. (#8120)

  • +
  • Improve performance of ORDER BY ... LIMIT queries on sorted data. (#6634)

  • +
+
+
+

Iceberg connector#

+
    +
  • Fix failure when listing materialized views in information_schema.tables or via the +java.sql.DatabaseMetaData.getTables() JDBC API. (#8151)

  • +
+
+
+

Memory connector#

+
    +
  • Improve performance of certain complex queries involving joins. (#8095)

  • +
+
+
+

SPI#

+
    +
  • Remove deprecated ConnectorPageSourceProvider.createPageSource() method overrides. (#8077)

  • +
  • Add support for casting the columns of a redirected table scan when source column types don’t match. (#6066)

  • +
  • Add ConnectorMetadata.redirectTable() to allow connectors to redirect table reads and metadata listings. (#7606)

  • +
  • Add ConnectorMetadata.streamTableColumns() for streaming column metadata in a redirection-aware manner. The +alternate method for column listing ConnectorMetadata.listTableColumns() is now deprecated. (#7606)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-359.html b/430/release/release-359.html new file mode 100644 index 000000000..0465c49db --- /dev/null +++ b/430/release/release-359.html @@ -0,0 +1,2693 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 359 (1 Jul 2021) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 359 (1 Jul 2021)#

+
+

General#

+
    +
  • Raise minimum required Java version for running Trino server to 11.0.11. (#8103)

  • +
  • Add support for row pattern recognition in window specification. (#8141)

  • +
  • Add support for SET TIME ZONE. (#8112)

  • +
  • Add geometry_nearest_points(). (#8280)

  • +
  • Add current_groups(). (#8446)

  • +
  • Add support for varchar, varbinary and date types to make_set_digest(). (#8295)

  • +
  • Add support for granting UPDATE privileges. (#8279)

  • +
  • List materialized view columns in the information_schema.columns table. (#8113)

  • +
  • Expose comments in views and materialized views in system.metadata.table_comments correctly. (#8327)

  • +
  • Fix query failure for certain queries with ORDER BY ... LIMIT on sorted data. (#8184)

  • +
  • Fix incorrect query results for certain queries using LIKE with pattern against +char columns in the WHERE clause. (#8311)

  • +
  • Fix planning failure when using hash_counts(). (#8248)

  • +
  • Fix error message when grouping expressions in GROUP BY queries contain aggregations, window functions or grouping operations. (#8247)

  • +
+
+
+

Security#

+
    +
  • Fix spurious impersonation check when applying user mapping for password authentication. (#7027)

  • +
  • Fix handling of multiple LDAP user bind patterns. (#8134)

  • +
+
+
+

Web UI#

+
    +
  • Show session timezone in query details page. (#4196)

  • +
+
+
+

Docker image#

+
    +
  • Add support for ARM64. (#8397)

  • +
+
+
+

CLI#

+
    +
  • Add support for logging of network traffic via the --network-logging command line option. (#8329)

  • +
+
+
+

BigQuery connector#

+
    +
  • Add bigquery.views-cache-ttl config property to allow configuring the cache expiration for BigQuery views. (#8236)

  • +
  • Fix incorrect results when accessing BigQuery records with wrong index. (#8183)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Fix potential incorrect results when queries contain an IS NULL predicate. (#3605)

  • +
  • Fix failure when multiple indexes share the same alias. (#8158)

  • +
+
+
+

Hive connector#

+
    +
  • Rename hive-hadoop2 connector to hive. (#8166)

  • +
  • Add support for Hive views which use GROUP BY over a subquery that also uses GROUP BY on matching columns. (#7635)

  • +
  • Add support for granting UPDATE privileges when hive.security=sql-standard is used. (#8279)

  • +
  • Add support for inserting data into CSV and TEXT tables with skip_header_line_count table property set to 1. +The same applies to creating tables with data using CREATE TABLE ... AS SELECT syntax. (#8390)

  • +
  • Disallow creating CSV and TEXT tables with data if skip_header_line_count is set to a value +greater than 0. (#8373)

  • +
  • Fix query failure when reading from a non-ORC insert-only transactional table. (#8259)

  • +
  • Fix incorrect results when reading ORC ACID tables containing deleted rows. (#8208)

  • +
  • Respect hive.metastore.glue.get-partition-threads configuration property. (#8320)

  • +
+
+
+

Iceberg connector#

+
    +
  • Do not include Hive views in SHOW TABLES query results. (#8153)

  • +
+
+
+

MongoDB connector#

+
    +
  • Skip creating an index for the _schema collection if it already exists. (#8264)

  • +
+
+
+

MySQL connector#

+
    +
  • Support reading and writing timestamp values with precision higher than 3. (#6910)

  • +
  • Support predicate pushdown on timestamp columns. (#7413)

  • +
  • Handle timestamp values during forward offset changes (‘gaps’ in DST) correctly. (#5449)

  • +
+
+
+

SPI#

+
    +
  • Introduce ConnectorMetadata#listMaterializedViews for listing materialized view names. (#8113)

  • +
  • Introduce ConnectorMetadata#getMaterializedViews for getting materialized view definitions. (#8113)

  • +
  • Enable connector to delegate materialized view refresh to itself. (#7960)

  • +
  • Allow computing HyperLogLog based approximate set summary as a column statistic during ConnectorMetadata +driven statistics collection flow. (#8355)

  • +
  • Report output column types through EventListener. (#8405)

  • +
  • Report input column information for queries involving set operations (UNION, INTERSECT and EXCEPT). (#8371)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-360.html b/430/release/release-360.html new file mode 100644 index 000000000..56cfd0e8e --- /dev/null +++ b/430/release/release-360.html @@ -0,0 +1,2767 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 360 (30 Jul 2021) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 360 (30 Jul 2021)#

+
+

General#

+
    +
  • Improve support for correlated subqueries with GROUP BY or LIMIT and +complex correlated filter conditions. (#8554)

  • +
  • Report cumulative query system memory usage. (#8615)

  • +
  • Fix DROP SCHEMA and DROP SCHEMA RESTRICT not to drop then schema if it is not empty. (#8660)

  • +
  • Fix query failure when there is a constant predicate on some +column col (e.g col=1), followed by ORDER BY col and LIMIT. (#8535)

  • +
  • Fix SHOW CREATE SCHEMA failure. (#8598)

  • +
  • Fix query failure when running SHOW CREATE SCHEMA information_schema. (#8600)

  • +
  • Improve performance of WHERE clause evaluation. (#8624)

  • +
  • Reduce coordinator network load. (#8460)

  • +
  • Improve query performance by sending collected dynamic filters from coordinator to workers. (#5183)

  • +
  • Improve performance of inequality joins where join condition sides have different type. (#8500)

  • +
  • Improve performance of IN (<subquery>) expressions. (#8639)

  • +
+
+
+

Security#

+ +
+
+

CLI#

+
    +
  • Fix auto completion when pressing the tab button. (#8529)

  • +
+
+
+

ClickHouse connector#

+
    +
  • Add write.batch-size connector configuration property to control JDBC batch size used during writes. +It can also be controlled using the write_batch_size session property. (#8434)

  • +
  • Add new configuration property insert.non-transactional-insert.enabled to allow INSERT queries to write directly +to the target table. This can improve performance in some cases by sacrificing transactional behaviour. It can also +be controlled using non_transactional_insert session property. (#8496)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Add support for assuming an IAM role. (#8714)

  • +
+
+
+

Hive connector#

+
    +
  • Fix data corruption when performing UPDATE or INSERT on ORC ACID transactional table. (#8268, #8452)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add support for Trino views. (#8540)

  • +
  • Prevent incorrect query results by failing a query when Iceberg table has row-level deletes. (#8450)

  • +
  • Fix query failure when joining with a bucketed Iceberg table. (#7502)

  • +
  • Fix query failure when showing stats for a bucketed Iceberg table. (#8616)

  • +
  • Fix query failure when joining with a partitioned table that has structural columns (array, map or row). (#8647)

  • +
  • Fix failures for queries that write tables in Parquet format. (#5201)

  • +
  • Improve query planning time by reducing calls to the metastore. (#8676, #8689)

  • +
+
+
+

MemSQL connector#

+
    +
  • Add write.batch-size connector configuration property to control JDBC batch size used during writes. +It can also be controlled using the write_batch_size session property. (#8434)

  • +
  • Add new configuration property insert.non-transactional-insert.enabled to allow INSERT queries to write directly +to the target table. This can improve performance in some cases by sacrificing transactional behaviour. It can also +be controlled using non_transactional_insert session property. (#8496)

  • +
  • Partial support for DELETE statement where predicate can be fully pushed down to the remote datasource. (#6287)

  • +
  • Fix performance regression of writes. (#8559)

  • +
+
+
+

MongoDB connector#

+
    +
  • Add support for json type. (#8352)

  • +
  • Support reading MongoDB DBRef type. (#3134)

  • +
+
+
+

MySQL connector#

+
    +
  • Add write.batch-size connector configuration property to control JDBC batch size used during writes. +It can also be controlled using the write_batch_size session property. (#8434)

  • +
  • Add new configuration property insert.non-transactional-insert.enabled to allow INSERT queries to write directly +to the target table. This can improve performance in some cases by sacrificing transactional behaviour. It can also +be controlled using non_transactional_insert session property. (#8496)

  • +
  • Partial support for DELETE statement where predicate can be fully pushed down to the remote datasource. (#6287)

  • +
  • Fix performance regression of writes. (#8559)

  • +
+
+
+

Oracle connector#

+
    +
  • Add write.batch-size connector configuration property to control JDBC batch size used during writes. +It can also be controlled using the write_batch_size session property. (#8434)

  • +
  • Add new configuration property insert.non-transactional-insert.enabled to allow INSERT queries to write directly +to the target table. This can improve performance in some cases by sacrificing transactional behaviour. It can also +be controlled using non_transactional_insert session property. (#8496)

  • +
  • Partial support for DELETE statement where predicate can be fully pushed down to the remote datasource. (#6287)

  • +
+
+
+

Phoenix connector#

+
    +
  • Add write.batch-size connector configuration property to control JDBC batch size used during writes. +It can also be controlled using the write_batch_size session property. (#8434)

  • +
  • Add new configuration property insert.non-transactional-insert.enabled to allow INSERT queries to write directly +to the target table. This can improve performance in some cases by sacrificing transactional behaviour. It can also +be controlled using non_transactional_insert session property. (#8496)

  • +
  • Improve performance of ORDER BY ... LIMIT queries on sorted data for Phoenix 5. (#8171)

  • +
  • Partial support for DELETE statement where predicate can be fully pushed down to the remote datasource. (#6287)

  • +
  • Fix performance regression of writes. (#8559)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Add write.batch-size connector configuration property to control JDBC batch size used during writes. +It can also be controlled using the write_batch_size session property. (#8434)

  • +
  • Add new configuration property insert.non-transactional-insert.enabled to allow INSERT queries to write directly +to the target table. This can improve performance in some cases by sacrificing transactional behaviour. It can also +be controlled using non_transactional_insert session property. (#8496)

  • +
  • Partial support for DELETE statement where predicate can be fully pushed down to the remote datasource. (#6287)

  • +
+
+
+

Redshift connector#

+
    +
  • Add write.batch-size connector configuration property to control JDBC batch size used during writes. +It can also be controlled using the write_batch_size session property. (#8434)

  • +
  • Add new configuration property insert.non-transactional-insert.enabled to allow INSERT queries to write directly +to the target table. This can improve performance in some cases by sacrificing transactional behaviour. It can also +be controlled using non_transactional_insert session property. (#8496)

  • +
  • Partial support for DELETE statement where predicate can be fully pushed down to the remote datasource. (#6287)

  • +
+
+
+

SQL Server connector#

+
    +
  • Add write.batch-size connector configuration property to control JDBC batch size used during writes. +It can also be controlled using the write_batch_size session property. (#8434)

  • +
  • Add new configuration property insert.non-transactional-insert.enabled to allow INSERT queries to write directly +to the target table. This can improve performance in some cases by sacrificing transactional behaviour. It can also +be controlled using non_transactional_insert session property. (#8496)

  • +
  • Partial support for DELETE statement where predicate can be fully pushed down to the remote datasource. (#6287)

  • +
  • Fix performance regression of writes. (#8559)

  • +
+
+
+

SPI#

+
    +
  • Cast materialized view storage table columns to match view column types. (#8408)

  • +
  • Remove deprecated ConnectorSplitManager#getSplits method overrides. (#8569)

  • +
  • Introduce ConnectorPageSource#getCompletedPositions for tracing physically read positions. (#8524)

  • +
  • Remove deprecated TupleDomain.transform. (#8056)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-361.html b/430/release/release-361.html new file mode 100644 index 000000000..0b7d4f3b7 --- /dev/null +++ b/430/release/release-361.html @@ -0,0 +1,2776 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 361 (27 Aug 2021) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 361 (27 Aug 2021)#

+
+

General#

+
    +
  • Add support for subqueries in MATCH_RECOGNIZE and WINDOW clause. (#8736)

  • +
  • Add system.metadata.materialized_views table that contains detailed information about materialized views. (#8796)

  • +
  • Support table redirection for INSERT, UPDATE and DELETE operations. (#8683)

  • +
  • Improve performance of sum() and avg() aggregations on decimal values. (#8878)

  • +
  • Improve performance for queries using IN predicate with moderate to large number of constants. (#8833)

  • +
  • Fix failures of specific queries accessing row columns with +with field names that would require quoting when used as an identifier. (#8845)

  • +
  • Fix incorrect results for queries with a comparison between a varchar column and a char constant. (#8984)

  • +
  • Fix invalid result when two decimals are added together. This happened in certain +queries where decimals had different precision. (#8973)

  • +
  • Prevent dropping or renaming objects with an incompatible SQL command. For example, DROP TABLE no longer allows dropping a view. (#8869)

  • +
+
+
+

Security#

+
    +
  • Add support for OAuth2/OIDC opaque access tokens. The property +http-server.authentication.oauth2.audience has been removed in favor of +using http-server.authentication.oauth2.client-id, as expected by OIDC. +The new property http-server.authentication.oauth2.additional-audiences +supports audiences which are not the client-id. Additionally, the new +property http-server.authentication.oauth2.issuer is now required; +tokens which are not issued by this URL will be rejected. (#8641)

  • +
+
+
+

JDBC driver#

+
    +
  • Implement the PreparedStatement.getParameterMetaData() method. (#2978)

  • +
  • Fix listing columns where table or schema name pattern contains an upper case value. +Note that this fix is on the server, not in the JDBC driver. (#8978)

  • +
+
+
+

BigQuery connector#

+
    +
  • Fix incorrect result when using BigQuery time type. (#8999)

  • +
+
+
+

Cassandra connector#

+
    +
  • Add support for predicate pushdown of smallint, tinyint and date types on partition columns. (#3763)

  • +
  • Fix incorrect results for queries containing inequality predicates on a clustering key in the WHERE clause. (#401)

  • +
+
+
+

ClickHouse connector#

+
    +
  • Allow limiting the size of the metadata cache via the metadata.cache-maximum-size configuration property. (#8652)

  • +
  • Fix incorrect results for aggregation functions applied to columns of type varchar and char. (#7320)

  • +
+
+
+

Druid connector#

+
    +
  • Allow limiting the size of the metadata cache via the metadata.cache-maximum-size configuration property. (#8652)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Add support for reading fields as json values. (#7308)

  • +
+
+
+

Hive connector#

+
    +
  • Expose <view>$properties system table for Trino and Hive views. (#8805)

  • +
  • Add support for translating Hive views which contain common table expressions. (#5977)

  • +
  • Add support for translating Hive views which contain outer parentheses. (#8789)

  • +
  • Add support for translating Hive views which use the from_utc_timestamp function. (#8502)

  • +
  • Add support for translating Hive views which use the date function. (#8789)

  • +
  • Add support for translating Hive views which use the pmod function. (#8935)

  • +
  • Prevent creating of tables that have column names containing commas, or leading or trailing spaces. (#8954)

  • +
  • Improve performance of updating Glue table statistics for partitioned tables. (#8839)

  • +
  • Change default Glue statistics read/write parallelism from 1 to 5. (#8839)

  • +
  • Improve performance of querying Parquet data for files containing column indexes. (#7349)

  • +
  • Fix query failure when inserting data into a Hive ACID table which is not explicitly bucketed. (#8899)

  • +
+
+
+

Iceberg connector#

+
    +
  • Fix reading or writing Iceberg tables that previously contained a +partition field that was later dropped. (#8730)

  • +
  • Allow reading from Iceberg tables which specify the Iceberg +write.object-storage.path table property. (#8573)

  • +
  • Allow using randomized location when creating a table, so that future table +renames or drops do not interfere with new tables created with the same name. +This can be enabled using the iceberg.unique-table-location configuration +property. (#6063)

  • +
  • Return proper query results for queries accessing multiple snapshots of single Iceberg table. (#8868)

  • +
+
+
+

MemSQL connector#

+
    +
  • Allow limiting the size of the metadata cache via the metadata.cache-maximum-size configuration property. (#8652)

  • +
+
+
+

MongoDB connector#

+ +
+
+

MySQL connector#

+
    +
  • Allow limiting the size of the metadata cache via the metadata.cache-maximum-size configuration property. (#8652)

  • +
  • Fix incorrect results for aggregation functions applied to columns of type varchar and char. (#7320)

  • +
+
+
+

Oracle connector#

+
    +
  • Allow limiting the size of the metadata cache via the metadata.cache-maximum-size configuration property. (#8652)

  • +
+
+
+

Phoenix connector#

+
    +
  • Allow limiting the size of the metadata cache via the metadata.cache-maximum-size configuration property. (#8652)

  • +
+
+
+

Pinot connector#

+
    +
  • Implement aggregation pushdown for count, avg, min, max, sum, count(DISTINCT) and approx_distinct. +It is enabled by default and can be disabled using the configuration property pinot.aggregation-pushdown.enabled +or the catalog session property aggregation_pushdown_enabled. (#4140)

  • +
  • Allow https URLs in pinot.controller-urls. (#8617)

  • +
  • Fix failures when querying information_schema.columns with a filter on the table name. (#8307)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Allow limiting the size of the metadata cache via the metadata.cache-maximum-size configuration property. (#8652)

  • +
  • Fix incorrect results for aggregation functions applied to columns of type varchar and char. (#7320)

  • +
+
+
+

Redshift connector#

+
    +
  • Allow limiting the size of the metadata cache via the metadata.cache-maximum-size configuration property. (#8652)

  • +
+
+
+

SQL Server connector#

+
    +
  • Allow limiting the size of the metadata cache via the metadata.cache-maximum-size configuration property. (#8652)

  • +
  • Fix incorrect results for aggregation functions applied to columns of type varchar and char. (#7320)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-362.html b/430/release/release-362.html new file mode 100644 index 000000000..0a88110dd --- /dev/null +++ b/430/release/release-362.html @@ -0,0 +1,2715 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 362 (20 Sep 2021) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 362 (20 Sep 2021)#

+
+

General#

+
    +
  • Add support for listagg(). (#4835)

  • +
  • Improve join performance. (#8974)

  • +
  • Improve performance of queries that contain a DISTINCT clause. (#8967, #9194)

  • +
  • Improve query performance by reducing CPU overhead of repartitioning data across worker nodes. (#9102)

  • +
  • Split metrics that are reported in operator stats into connectorMetrics and metrics. (#9156)

  • +
  • Prevent failure when querying system.materialized_views and a materialized view is dropped concurrently. (#9050)

  • +
  • Fix incorrect results for aggregations that contain DISTINCT and FILTER clauses. (#9265)

  • +
  • Fix incorrect query results when row pattern contains back references. (#9109)

  • +
  • Fix ALTER SCHEMA ... SET AUTHORIZATION to resolve roles using the catalog of the schema instead of the session catalog. (#9066)

  • +
  • Fix query failure when query contains a cast from varchar to a shorter char. (#9036)

  • +
  • Fix planning failure of INSERT statement when source table has hidden columns. (#9150)

  • +
  • Fix planning of recursive queries when the recursion, the base plan, or the recursion step plan produce duplicate outputs. (#9153)

  • +
  • Fix failure when querying the optimizer_rule_stats system table. (#8700)

  • +
  • Fix failure for queries that push projections into connectors. (#6200)

  • +
  • Fix planning timeout for queries containing IS NULL, AND, and OR predicates in the WHERE clause. (#9250)

  • +
  • Fix failure for queries containing ORDER BY ... LIMIT when columns in the subquery are known to be constant. (#9171)

  • +
+
+
+

Security#

+
    +
  • Add IN catalog clause to CREATE ROLE, DROP ROLE, GRANT ROLE, REVOKE ROLE, and SET ROLE to specify +the target catalog of the statement instead of using the current session catalog. This change is necessary to +support system roles in the future. The old behavior of these statements can be restored by setting the +deprecated.legacy-catalog-roles config property. (#9087)

  • +
+
+
+

Web UI#

+
    +
  • Add query error info to cluster overview page. (#8762)

  • +
+
+
+

JDBC driver#

+
    +
  • Improve performance of listing table columns via java.sql.DatabaseMetaData API when filtering on schema name. (#9214)

  • +
+
+
+

Server RPM#

+
    +
  • Change RPM architecture to noarch to allow installing on any machine type. (#9187)

  • +
+
+
+

BigQuery connector#

+
    +
  • Support case insensitive name matching for BigQuery views. (#9164)

  • +
  • Change type mapping of BigQuery datetime from timestamp(3) to timestamp(6) in Trino. (#9052)

  • +
  • Change type mapping of BigQuery time from time with time zone to time(6) in Trino. (#9052)

  • +
  • Change type mapping of BigQuery timestamp from timestamp(3) with time zone to timestamp(6) with time zone in Trino. (#9052)

  • +
  • Fix failure for queries where predicate on datetime column is pushed down to BigQuery. (#9005)

  • +
  • Fix incorrect results when using parameterized numeric type with non-default precision and scale. (#9060)

  • +
  • Fix query failure when accessing tables with unsupported data type columns. (#9086)

  • +
  • Fix failure for queries where predicate on float64 column involving +infinity or +-infinity values is pushed down to BigQuery. (#9122)

  • +
+
+
+

Cassandra connector#

+
    +
  • Change minimum number of speculative executions from 2 to 1. (#9096)

  • +
+
+
+

Hive connector#

+
    +
  • Support reading Parquet timestamp stored with millisecond or microsecond precision as INT64 with +logical type annotations when Hive timestamp precision is NANOSECONDS. (#9139)

  • +
  • Support reading Parquet timestamp stored as INT64 with nanosecond precision. (#9188)

  • +
  • Support writing Parquet timestamp stored as INT64 with nanosecond precision when experimental Parquet writer is enabled. +To use this, the Hive timestamp precision should be set to NANOSECONDS. (#9188)

  • +
  • Support loading of S3 mappings via HTTP(S) url. The hive.s3.security-mapping.config-file property can now +either point to a local file or a URL. (#6210)

  • +
  • Allow reading from tables bucketed on a column that uses type for which bucketing is not natively +supported by Trino. Writing to such tables is still not allowed. (#8986)

  • +
  • Extend set of statistics reported by JMX to include metrics for calls made to the Glue statistics API. (#9100)

  • +
  • Limit maximum file size generated by write operations to 1 GB by default. The limit is not exact and is applied on a best-effort basis. +The limit can be set with the hive.target-max-file-size configuration property or the target_max_file_size session property. (#7991)

  • +
  • Allow specifying the list of schemas for which Trino will enforce that queries use a filter on partition keys for source tables. +The list can be specified using the hive.query-partition-filter-required-schemas, or the query_partition_filter_required_schemas session property. +The list is taken into consideration only if the hive.query-partition-filter-required configuration property or the query_partition_filter_required +session property is set to true. (#9106)

  • +
  • Fix failure when writing Parquet files with Snappy compression on ARM64. (#9148)

  • +
  • Fix performance regression when reading Parquet files that contain dictionaries. (#9161)

  • +
  • Fix incorrect table locking in Hive metastore when querying unpartitioned non-transactional tables. (#9070)

  • +
  • Fix ArrayIndexOutOfBoundsException when using the experimental Parquet writer. (#5518)

  • +
  • Fix reading Avro tables written with older Avro versions that had certain semi-invalid schemas. (#9114)

  • +
  • Fix possible INSERT/UPDATE/ANALYZE query failure when Glue metastore is in use and table statistics collection is enabled. (#9297)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add support for Iceberg uuid type. (#6663)

  • +
  • Fix the mapping of nested fields between table metadata and Parquet file metadata. This +enables evolution of row typed columns for Iceberg tables stored in Parquet. (#9124)

  • +
  • Fix failure for queries filtering on columns with array, map, or row type. (#8822)

  • +
  • Fix ArrayIndexOutOfBoundsException when using the experimental Parquet writer. (#5518)

  • +
  • Fix query failures caused by errors reading certain Avro metadata files. (#9114)

  • +
+
+
+

Pinot connector#

+
    +
  • Support pushdown of filters on varbinary columns to Pinot. (#9180)

  • +
  • Fix incorrect results for queries that contain aggregations and IN and NOT IN filters over varchar columns. (#9133)

  • +
  • Fix failure for queries with filters on real or double columns having +Infinity or -Infinity values. (#9180)

  • +
+
+
+

TPC-H connector#

+
    +
  • Add support for switching the mapping of floating point values between SQL double and decimal types. The mapping +can be set via the tpch.double-type-mapping configuration property. (#7990)

  • +
+
+
+

SPI#

+
    +
  • Change Connector.isSingleStatementWritesOnly() to return true by default. (#8872)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-363.html b/430/release/release-363.html new file mode 100644 index 000000000..3ce05bbc1 --- /dev/null +++ b/430/release/release-363.html @@ -0,0 +1,2676 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 363 (6 Oct 2021) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 363 (6 Oct 2021)#

+
+

General#

+
    +
  • Add HTTP event listener implementation which sends JSON serialized events to a remote HTTP endpoint. (#8821)

  • +
  • Improve performance of queries that group by bigint columns. (#9510)

  • +
  • Improve performance of queries that process row or array data. (#9402)

  • +
  • Improve query planning performance. (#9462)

  • +
  • Reduce query memory usage when spilling occurs. (#9270, #9275)

  • +
  • Reduce memory usage for processing JOIN clauses. (#9327)

  • +
  • Fix potential data loss in query results when clients retry requests to the coordinator. (#9453)

  • +
  • Fix incorrect result for comparisons between zero-valued decimals. (#8985)

  • +
  • Fix SHOW ROLES failure when there are no roles to display. (#9319)

  • +
  • Fix EXPLAIN ANALYZE to show estimates. (#9396)

  • +
  • Fix incorrect result for round() with precision set to 0. (#9371)

  • +
  • Respect deprecated.legacy-catalog-roles=true configuration property in SHOW ROLES, +SHOW CURRENT ROLES and SHOW ROLE GRANTS statements. (#9496)

  • +
+
+
+

Python client#

+
    +
  • Fix column type reported in cursor.description for time with time zone column. (#9460)

  • +
+
+
+

BigQuery connector#

+
    +
  • Fix failure for queries where predicates on geography, array or struct column are pushed down to BigQuery. (#9391)

  • +
+
+
+

Cassandra connector#

+
    +
  • Add support for Cassandra tuple type. (#8570)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Add support for scaled_float type. (#9358)

  • +
+
+
+

Hive connector#

+
    +
  • Support usage of avro_schema_url table property in partitioned tables. (#9370}

  • +
  • Add support for insert overwrite operations on S3-backed tables. (#9234)

  • +
  • Improve query performance when reading Parquet data with predicate on a decimal column. (#9338)

  • +
  • Fix Failed reading parquet data: Socket is closed by peer query failure when reading from Parquet table with a predicate. (#9097)

  • +
  • Fix query failure when updating or deleting from an ORC ACID transactional table that has some rows deleted since the last major compaction. (#9354)

  • +
  • Fix failure when reading large Parquet files. (#9469)

  • +
  • Fix failures for some UPDATE queries, such as those where the SET clause contains the same constant more than once. (#9295)

  • +
  • Fix incorrect results when filtering on Parquet columns containing a dot in their name. (#9516)

  • +
+
+
+

Iceberg connector#

+
    +
  • Improve query performance when reading Parquet data with predicate on a decimal column. (#9338)

  • +
  • Fix support for comments when adding a new column. Previously, they were silently ignored. (#9123)

  • +
  • Fix page and block sizing when writing Parquet data. (#9326)

  • +
  • Fix failure when reading large Parquet files. (#9469)

  • +
+
+
+

MySQL connector#

+
    +
  • Add support for variable precision time type. (#9339)

  • +
  • Support CREATE TABLE and CREATE TABLE AS statements for time type. (#9339)

  • +
+
+
+

Phoenix connector#

+
    +
  • Allowing forcing the mapping of certain types to varchar. This can be enabled by +setting the jdbc-types-mapped-to-varchar configuration property to a comma-separated +list of type names. (#2084)

  • +
+
+
+

Pinot connector#

+
    +
  • Fix failure when a column name is a reserved keyword. (#9373)

  • +
+
+
+

SQL Server connector#

+
    +
  • Add support for SQL Server datetimeoffset type. (#9329)

  • +
  • Fix failure for queries where predicates on text or ntext typed columns are pushed down to SQL Server. (#9387)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-364.html b/430/release/release-364.html new file mode 100644 index 000000000..dfbe75440 --- /dev/null +++ b/430/release/release-364.html @@ -0,0 +1,2673 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 364 (1 Nov 2021) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 364 (1 Nov 2021)#

+
+

General#

+
    +
  • Add support for ALTER MATERIALIZED VIEW ... RENAME TO .... (#9492)

  • +
  • Improve performance of GROUP BY with single grouping column. (#9514)

  • +
  • Improve performance of decimal aggregations. (#9640)

  • +
  • Improve performance when evaluating the WHERE and SELECT clause. (#9610)

  • +
  • Improve performance when computing the product of decimal values with precision larger than 19. (#9744)

  • +
  • Improve CPU coordinator utilization. (#8650)

  • +
  • Remove support for the unwrap_casts session property and optimizer.unwrap-casts configuration option. (#9550)

  • +
  • Fix incorrect results for queries with nested joins and IS NOT DISTINCT FROM join clauses. (#9805)

  • +
  • Fix displaying character type dynamic filter values in EXPLAIN ANALYZE. (#9673)

  • +
  • Fix query failure for update operation if it has a correlated subquery. (#8286)

  • +
  • Fix decimal division when result is between -1 and 0. (#9696)

  • +
  • Fix SHOW STATS failure for a query projecting a boolean column. (#9710)

  • +
+
+
+

Web UI#

+
    +
  • Improve responsiveness of Web UI when query history contains queries with long query text. (#8892)

  • +
+
+
+

JDBC driver#

+
    +
  • Allow using token from existing Kerberos context. This allows the client to perform Kerberos authentication without +passing the Keytab or credential cache to the driver. (#4826)

  • +
+
+
+

Cassandra connector#

+
    +
  • Map Cassandra uuid type to Trino uuid. (#5231)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Fix failure when documents contain fields of unsupported types. (#9552)

  • +
+
+
+

Hive connector#

+
    +
  • Allow to skip setting permissions on new directories by setting hive.fs.new-directory-permissions=skip in connector properties file. (#9539)

  • +
  • Allow translating Hive views which cast timestamp columns to decimal. (#9530)

  • +
  • Add optimize table procedure for merging small files in non-transactional Hive table. +Procedure can be executed using ALTER TABLE <table> EXECUTE optimize(file_size_threshold => ...) syntax. (#9665)

  • +
  • Restrict partition overwrite on insert to auto-commit context only. (#9559)

  • +
  • Reject execution of CREATE TABLE when bucketing is requested on columns with unsupported column types. +Previously CREATE was allowed but it was not possible to insert data to such a table. (#9793)

  • +
  • Improve performance of querying Parquet data for files containing column indexes. (#9633)

  • +
  • Fix Hive 1 and Hive 3 incompatibility with Parquet files containing char or varchar data produced by the experimental Parquet writer. +Hive 2 or newer should now read such files correctly, while Hive 1.x is still known not to read them. (#9515, (#6377))

  • +
  • Fix ArrayIndexOutOfBoundsException when inserting into a partitioned table with hive.target-max-file-size set. (#9557)

  • +
  • Fix reading Avro schema written by Avro 1.8.2 with non-spec-compliant default values. (#9243)

  • +
  • Fix failure when querying nested Parquet data if column indexes are enabled. (#9587)

  • +
  • Fix incorrect results when querying Parquet data. (#9587)

  • +
  • Fix query failure when writing to a partitioned table with target max file size set. (#9557)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add support for renaming materialized views. (#9492)

  • +
  • Create Parquet files that can be read more efficiently. (#9569)

  • +
  • Improve query performance when dynamic filtering can be leveraged. (#4115)

  • +
  • Return value with UTC zone for table partitioned on timestamp with time zone. (#9704)

  • +
  • Fix data loss in case of concurrent inserts to a table. (#9583)

  • +
  • Fix query failure when reading from $partitions table for a table partitioned on timestamp with time zone or uuid (#9703, #9757)

  • +
  • Fix query failure when reading Iceberg table statistics. (#9714)

  • +
+
+
+

MemSQL connector#

+
    +
  • Support reading and writing MemSQL datetime(6) and timestamp(6) types as Trino timestamp(6). (#9725)

  • +
+
+
+

SQL Server connector#

+
    +
  • Fix query failure when count(*) aggregation is pushed down to SQL Server database and the table has more than 2147483647 rows. (#9549)

  • +
+
+
+

SPI#

+
    +
  • Expose which columns are covered by dynamic filters. (#9644)

  • +
  • Add SPI for table procedures that can process table data in a distributed manner. Table procedures can be run via ALTER TABLE ... EXECUTE ... syntax. (#9665)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-365.html b/430/release/release-365.html new file mode 100644 index 000000000..7eda00425 --- /dev/null +++ b/430/release/release-365.html @@ -0,0 +1,2814 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 365 (3 Dec 2021) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 365 (3 Dec 2021)#

+
+

General#

+
    +
  • Add support for TRUNCATE TABLE. (#8921)

  • +
  • Add support for aggregate functions in row pattern recognition context. (#8738)

  • +
  • Add support for time travel queries. (#8773)

  • +
  • Add support for spilling aggregations containing ORDER BY or DISTINCT clauses. (#9723)

  • +
  • Add contains function to check whether a CIDR contains an IP address. (#9654)

  • +
  • Report connector metrics in EXPLAIN ANALYZE VERBOSE. (#9858)

  • +
  • Report operator input row count distribution in EXPLAIN ANALYZE VERBOSE. (#10133)

  • +
  • Allow executing INSERT or DELETE statements on tables restricted with a row filter. (#8856)

  • +
  • Remove owner column from the system.metadata.materialized_views table. (#9961)

  • +
  • Remove the optimizer.iterative-rule-based-column-pruning config property. The legacy column pruning optimizer is no longer available. (#9564)

  • +
  • Improve performance of inequality joins. (#9307)

  • +
  • Improve performance of joins involving a small table on one side. (#9851)

  • +
  • Improve CPU utilization by adjusting task.concurrency automatically based on the number of physical cores. (#10088)

  • +
  • Make query final query statistics more accurate. (#9888, #9913)

  • +
  • Improve query planning performance for queries containing large IN predicates. (#9874)

  • +
  • Reduce peak memory usage for queries involving the rank, dense_rank, or row_number window functions. (#10056)

  • +
  • Fix incorrect results when casting bigint values to varchar(n) type. (#552)

  • +
  • Fix query failure when the PREPARE statement is used with DROP or INSERT +and the table or schema name contains special characters. (#9822)

  • +
  • Fix minor memory leak when queries are abandoned during the initial query submission phase. (#9962)

  • +
  • Collect connector metrics after ConnectorPageSource is closed. (#9615)

  • +
+
+
+

Security#

+
    +
  • Allow configuring HTTP proxy for OAuth2 authentication. (#9920, #10069)

  • +
  • Add group-based and owner-based query access rules to file based system access control. (#9811)

  • +
  • Use internal names for discovery client when automatic TLS is enabled for internal communications. This allows discovery.uri +to be configured using a normal DNS name like https://coordinator.trino and still use automatic TLS certificates. (#9821)

  • +
  • Use Kerberos operating system ticket cache if keytab file is not provided to JDBC and CLI for Kerberos authentication. (#8987)

  • +
  • Fix internal communication automatic TLS on Java 17. (#9821)

  • +
+
+
+

CLI#

+
    +
  • Automatically use HTTPS when port is set to 443. (#8798)

  • +
+
+
+

BigQuery connector#

+
    +
  • Support reading bignumeric type whose precision is less than or equal to 38. (#9882)

  • +
  • Fix failure when a schema is dropped while listing tables. (#9954)

  • +
+
+
+

Cassandra connector#

+
    +
  • Support reading user defined types in Cassandra. (#147)

  • +
+
+
+

ClickHouse connector#

+
    +
  • Add support for truncating tables. (#8921)

  • +
  • Fix incorrect query results when query contains predicates on real type columns. (#9998)

  • +
+
+
+

Druid connector#

+ +
+
+

Elasticsearch connector#

+
    +
  • Add support for additional Elastic Cloud node roles. (#9892)

  • +
  • Fix failure when empty values exist in numeric fields. (#9939)

  • +
+
+
+

Hive connector#

+
    +
  • Allow reading empty files of type Parquet, RCFile, SequenceFile. (#9929)

  • +
  • Enable hive.s3.streaming by default. (#9715)

  • +
  • Improve performance by not generating splits for empty files. (#9929)

  • +
  • Improve performance of decimal avg aggregation. (#9738)

  • +
  • Improve performance when reading Parquet files with timestamps encoded using int64 representation. (#9414)

  • +
  • Improve dynamic partition pruning efficiency. (#9866, #9869)

  • +
  • Improve query performance on partitioned tables or tables with small files +by increasing hive.split-loader-concurrency from 4 to 64. (#9979)

  • +
  • Fix reporting of number of read bytes for tables using ORC file format. (#10048)

  • +
  • Account for memory used for deleted row information when reading from ACID tables. (#9914, #10070)

  • +
  • Fix REVOKE GRANT OPTION to revoke only the grant option instead of revoking the entire privilege. (#10094)

  • +
  • Fix bug where incorrect rows were deleted when deleting from a transactional table that has original files (before the first major compaction). (#10095)

  • +
  • Fix delete and update failure when changing a table after a major compaction. (#10120)

  • +
  • Fix incorrect results when decoding decimal values in Parquet reader. (#9971)

  • +
  • Fix hive.dynamic-filtering.wait-timeout not having any effect. (#10106)

  • +
  • Fix failure when reading Parquet data if column indexes are enabled. (#9890, #10076)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add support for storing and reading UUID nested in row, array or map type. (#9918)

  • +
  • Use Iceberg’s schema.name-mapping.default table property for scanning files +with missing Iceberg IDs. This aligns Trino behavior on migrated files with the +Iceberg spec. (#9959)

  • +
  • Use ZSTD compression by default. (#10058)

  • +
  • Add read-only security option which can be enabled by setting the configuration iceberg.security=read-only. (#9974)

  • +
  • Change schema of $partitions system table to avoid conflicts when table name contains a column named row_count, file_count +or total_size, or when a column is used for partitioning for part of table data, and it not used for partitioning in some other +part of the table data. (#9519, #8729).

  • +
  • Improve performance when reading timestamps from Parquet files. (#9414)

  • +
  • Improve query performance for certain queries with complex predicates. (#9309)

  • +
  • Reduce resource consumption and create bigger files when writing to an Iceberg table with partitioning. Bigger files are more efficient to query later. (#9826)

  • +
  • Improve performance for queries on nested data through dereference pushdown. (#8129)

  • +
  • Write correct file_size_in_bytes in manifest when creating new ORC files. (#9810)

  • +
  • Fix query failures that could appear when reading Parquet files which contained ROW columns that were subject to schema evolution. (#9264)

  • +
  • Fix failure caused by stale metadata in the rollback_to_snapshot procedure. (#9921)

  • +
+
+
+

Kudu connector#

+
    +
  • Avoid scanner time-out issues when reading Kudu tables. (#7250)

  • +
+
+
+

MemSQL connector#

+
    +
  • Add support for truncating tables. (#8921)

  • +
  • Fix incorrect query results when query contains predicates on real type columns. (#9998)

  • +
+
+
+

MongoDB connector#

+
    +
  • Support connecting to MongoDB clusters via mongodb.connection-url config property. mongodb.seeds and mongodb.credentials properties are now deprecated. (#9819)

  • +
+
+
+

MySQL connector#

+
    +
  • Add support for truncating tables. (#8921)

  • +
  • Fix incorrect query results when query contains predicates on real type columns. (#9998)

  • +
+
+
+

Oracle connector#

+ +
+
+

Phoenix connector#

+
    +
  • Support reading decimal columns from Phoenix with unspecified precision or scale. (#9795)

  • +
  • Fix query failures when reading Phoenix tables. (#9151)

  • +
+
+
+

Pinot connector#

+
    +
  • Update Pinot connector to be compatible with versions >= 0.8.0 and drop support for older versions. (#9098)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Add support for truncating tables. (#8921)

  • +
  • Add experimental support for range predicate pushdown on string columns. It can be enabled +by setting the postgresql.experimental.enable-string-pushdown-with-collate catalog configuration +property or the corresponding enable_string_pushdown_with_collate session property to true. (#9746)

  • +
+
+
+

Redshift connector#

+ +
+
+

SQL Server connector#

+ +
+
+

SPI#

+
    +
  • Allow split manager to filter splits based on a predicate not expressible as a TupleDomain. (#7608)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-366.html b/430/release/release-366.html new file mode 100644 index 000000000..75d9a75fa --- /dev/null +++ b/430/release/release-366.html @@ -0,0 +1,2645 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 366 (14 Dec 2021) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 366 (14 Dec 2021)#

+
+

General#

+
    +
  • Add support for automatic query retries. This feature can be turned on by setting the retry-policy +config property or the retry_policy session property to retry. (#9361)

  • +
  • Add CREATE privilege kind to SQL grammar. Note that this permission is not used by any +existing security systems, but is available for plugins. (#10206)

  • +
  • Add support for DENY statement in the engine. Note that this statement is not supported by any +existing security systems, but is available for plugins. (#10205)

  • +
  • Reduce lock contention during query execution. (#10246, #10239)

  • +
  • Improve query performance through optimizations to in-memory representations. (#10225)

  • +
  • Reduce query latency for contended clusters or complex queries with multiple stages. (#10249)

  • +
  • Fix incorrect results or failure when casting numeric values to varchar(n) type. (#552)

  • +
  • Remove support for spilling aggregations containing ORDER BY or DISTINCT clauses and associated +configuration properties spill-distincting-aggregations-enabled, spill-ordering-aggregations-enabled. +(#10183)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Read extended metadata from the _meta.trino index mapping attribute. _meta.presto is still +supported for backward compatibility. (#8383)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for redirects from Hive to Iceberg. This can be configured with hive.iceberg-catalog-name +catalog configuration property. (#10173)

  • +
  • Improve performance of uploading data into tables that use S3 filesystem. (#10180)

  • +
+
+
+

Iceberg connector#

+
    +
  • Fix incorrect query results for tables partitioned on columns of type binary. (#9755)

  • +
+
+
+

MemSQL connector#

+
    +
  • Fix incorrect result when a date value is older than or equal to 1582-10-14. (#10054)

  • +
+
+
+

MySQL connector#

+
    +
  • Fix incorrect result when a date value is older than or equal to 1582-10-14. (#10054)

  • +
+
+
+

Phoenix connector#

+
    +
  • Avoid running into out of memory errors with certain types of queries. (#10143)

  • +
+
+
+

Prometheus connector#

+
    +
  • Support configuring a read timeout via the prometheus.read-timeout config property. (#10101)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Fix incorrect result when a date value is older than or equal to 1582-10-14. (#10054)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-367.html b/430/release/release-367.html new file mode 100644 index 000000000..8a1da141e --- /dev/null +++ b/430/release/release-367.html @@ -0,0 +1,2613 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 367 (22 Dec 2021) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 367 (22 Dec 2021)#

+
+

General#

+
    +
  • Capture lineage details for columns from WITH clauses and subqueries. (#10272)

  • +
  • Improve CREATE VIEW error message when table or materialized view already exists. (#10186)

  • +
  • Disallow query retries when connectors cannot perform them safely. (#10064)

  • +
  • Improve performance when query retries are enabled by adding support for dynamic filtering. (#10274)

  • +
  • Fix failure in min_by and max_by aggregation execution. (#10347)

  • +
  • Fix planning failure for queries that access fields of row types by index (ROW(...)[n]) or that +select all row fields (ROW(..).*). (#10321)

  • +
  • Fix bug where certain queries which use broadcast joins could hang and never complete. (#10344)

  • +
  • Fix failure when row or array in VALUES clause contains nulls. (#10141)

  • +
+
+
+

Security#

+
    +
  • Hide inaccessible columns from SELECT * statement when
    +the hide-inaccessible-columns configuration property is set to true. (#9991)

  • +
  • Disable SET AUTHORIZATION when VIEW runs as DEFINER. (#10351)

  • +
+
+
+

Web UI#

+
    +
  • Improve user experience by introducing a new landing page for logout flow when +Oauth2 authentication is used. (#10299)

  • +
+
+
+

Hive connector#

+
    +
  • Add procedure system.flush_metadata_cache() to flush metadata caches. (#10251)

  • +
  • Prevent data loss during DROP SCHEMA when schema location contains files but not tables. (#10146)

  • +
  • Ensure no duplicate rows are created if query which writes data to Hive table is retried. (#10252, #10064)

  • +
+
+
+

Iceberg connector#

+
    +
  • Prevent data loss during DROP SCHEMA when schema location contains files but not tables. (#9767)

  • +
+
+
+

SPI#

+
    +
  • Fix ClassNotFoundException when using aggregation with a custom state type. (#10341)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-368.html b/430/release/release-368.html new file mode 100644 index 000000000..aba96b863 --- /dev/null +++ b/430/release/release-368.html @@ -0,0 +1,2656 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 368 (11 Jan 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 368 (11 Jan 2022)#

+
+

General#

+
    +
  • Allow setting per task memory limits via query.max-total-memory-per-task +config property or via query_max_total_memory_per_task session property. +(#10308)

  • +
  • Improve wall time for query processing with the phased scheduling policy. +The previous behavior can be restored by setting the query.execution-policy +configuration property to legacy-phased. (#10350)

  • +
  • Enable phased scheduling policy by default. The previous behavior can be +restored by setting the query.execution-policy configuration property to +all-at-once. (#10455)

  • +
  • Improve performance of arithmetic operations involving decimals with precision +larger than 18. (#10051)

  • +
  • Reduce risk of out-of-memory failure on congested clusters with high memory +usage. (#10475)

  • +
  • Fix queries not being unblocked when placed in reserved memory pool. +(#10475)

  • +
  • Prevent execution of REFRESH MATERIALIZED VIEW from getting stuck. +(#10360)

  • +
  • Fix double reporting of scheduled time for scan operators in +EXPLAIN ANALYZE. (#10472)

  • +
  • Fix issue where the length of log file names grow indefinitely upon log +rotation. (#10394)

  • +
+
+
+

Hive connector#

+
    +
  • Improve performance of decoding decimal values with precision larger than 18 +in ORC, Parquet and RCFile data. (#10051)

  • +
  • Disallow querying the properties system table for Delta Lake tables, since +Delta Lake tables are not supported. This fixes the previous behavior of +silently returning incorrect values. (#10447)

  • +
  • Reduce risk of worker out-of-memory exception when scanning ORC files. +(#9949)

  • +
+
+
+

Iceberg connector#

+
    +
  • Fix Iceberg table creation with location when schema location inaccessible. +(#9732)

  • +
  • Support file based access control. (#10493)

  • +
  • Display the Iceberg table location in SHOW CREATE TABLE output. +(#10459)

  • +
+
+
+

SingleStore (MemSQL) connector#

+
    +
  • Add support for time type. (#10332)

  • +
+
+
+

Oracle connector#

+
    +
  • Fix incorrect result when a date value is older than or equal to +1582-10-14. (#10380)

  • +
+
+
+

Phoenix connector#

+
    +
  • Add support for reading binary type. (#10539)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Add support for accessing tables created with declarative partitioning in +PostgreSQL. (#10400)

  • +
+
+
+

SPI#

+
    +
  • Encode long decimal values using two’s complement representation and change +their carrier type to io.trino.type.Int128 instead of +io.airlift.slice.Slice. (#10051)

  • +
  • Fix ClassNotFoundException when using aggregation with a custom state type. +(#10408)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-369.html b/430/release/release-369.html new file mode 100644 index 000000000..1c1cca780 --- /dev/null +++ b/430/release/release-369.html @@ -0,0 +1,2805 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 369 (24 Jan 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 369 (24 Jan 2022)#

+
+

General#

+
    +
  • Add support for Pacific/Kanton time zone. (#10679)

  • +
  • Display Physical input read time using most succinct time unit in EXPLAIN ANALYZE VERBOSE. (#10576)

  • +
  • Fine tune request retry mechanism in HTTP event listener. (#10566)

  • +
  • Add support for using PostgreSQL and Oracle as backend database for resource +groups. (#9812)

  • +
  • Remove unnecessary spilling configuration properties spill-order-by and +spill-window-operator. (#10591)

  • +
  • Remove distinction between system and user memory to simplify +cluster configuration. The configuration property +query.max-total-memory-per-node is removed. Use query.max-memory-per-node +instead. (#10574)

  • +
  • Use formatting specified in the SQL standard when casting double and real +values to varchar type. (#552)

  • +
  • Add support for ALTER MATERIALIZED VIEW ... SET PROPERTIES. (#9613)

  • +
  • Add experimental implementation of task level retries. This can be enabled by +setting the retry-policy configuration property or the retry_policy +session property to task. (#9818)

  • +
  • Improve query wall time by splitting workload between nodes in a more balanced +way. Previous workload balancing policy can be restored via +node-scheduler.splits-balancing-policy=node. (#10660)

  • +
  • Prevent hanging query execution on failures with phased execution policy. +(#10656)

  • +
  • Catch overflow in decimal multiplication. (#10732)

  • +
  • Fix UnsupportedOperationException in max_by and min_by aggregation. +(#10599)

  • +
  • Fix incorrect results or failure when casting date to varchar(n) type. +(#552)

  • +
  • Fix issue where the length of log file names grow indefinitely upon log +rotation. (#10738)

  • +
+
+
+

Security#

+
    +
  • Allow extracting groups from OAuth2 claims from +http-server.authentication.oauth2.groups-field. (#10262)

  • +
+
+
+

JDBC driver#

+
    +
  • Fix memory leak when using DatabaseMetaData. (#10584, +#10632)

  • +
+
+
+

BigQuery connector#

+
    +
  • Remove bigquery.case-insensitive-name-matching.cache-ttl configuration +option. It was previously ignored. (#10697)

  • +
  • Fix query failure when reading columns with numeric or bignumeric type. +(#10564)

  • +
+
+
+

ClickHouse connector#

+
    +
  • Upgrade minimum required version to 21.3. (#10703)

  • +
  • Add support for renaming schemas. (#10558)

  • +
  • Add support for setting column comments. (#10641)

  • +
  • Map ClickHouse ipv4 and ipv6 types to Trino ipaddress type. +(#7098)

  • +
  • Allow mapping ClickHouse fixedstring or string as Trino varchar via the +map_string_as_varchar session property. (#10601)

  • +
  • Disable avg pushdown on decimal types to avoid incorrect results. +(#10650)

  • +
  • Fix spurious errors when metadata cache is enabled. (#10544, +#10512)

  • +
+
+
+

Druid connector#

+
    +
  • Fix spurious errors when metadata cache is enabled. (#10544, +#10512)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for writing Bloom filters in ORC files. (#3939)

  • +
  • Allow flushing the metadata cache for specific schemas, tables, or partitions +with the flush_metadata_cache system procedure. +(#10385)

  • +
  • Add support for long lived AWS Security Token Service (STS) credentials for +authentication with Glue catalog. (#10735)

  • +
  • Ensure transaction locks in the Hive Metastore are released in case of query +failure when querying Hive ACID tables. (#10401)

  • +
  • Disallow reading from Iceberg tables when redirects from Hive to Iceberg are +not enabled. (#8693, #10441)

  • +
  • Improve performance of queries using range predicates when reading ORC files +with Bloom filters. (#4108)

  • +
  • Support writing Parquet files greater than 2GB. (#10722)

  • +
  • Fix spurious errors when metadata cache is enabled. (#10646, +#10512)

  • +
  • Prevent data loss during DROP SCHEMA when the schema location contains files +that are not part of existing tables. (#10485)

  • +
  • Fix inserting into transactional table when task_writer_count > 1. +(#9149)

  • +
  • Fix possible data corruption when writing data to S3 with streaming enabled. +(#10710)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add $properties system table which can be queried to inspect Iceberg table +properties. (#10480)

  • +
  • Add support for ALTER TABLE .. EXECUTE OPTIMIZE statement. (#10497)

  • +
  • Respect Iceberg column metrics mode when writing. (#9938)

  • +
  • Add support for long lived AWS Security Token Service (STS) credentials for +authentication with Glue catalog. (#10735)

  • +
  • Improve performance of queries using range predicates when reading ORC files +with Bloom filters. (#4108)

  • +
  • Improve select query planning performance after write operations from Trino. +(#9340)

  • +
  • Ensure table statistics are accumulated in a deterministic way from Iceberg +column metrics. (#9716)

  • +
  • Prevent data loss during DROP SCHEMA when the schema location contains files +that are not part of existing tables. (#10485)

  • +
  • Support writing Parquet files greater than 2GB. (#10722)

  • +
  • Fix materialized view refresh when view a query references the same table +multiple times. (#10570)

  • +
  • Fix possible data corruption when writing data to S3 with streaming enabled. +(#10710)

  • +
+
+
+

MySQL connector#

+
    +
  • Fix spurious errors when metadata cache is enabled. (#10544, +#10512)

  • +
+
+
+

Oracle connector#

+
    +
  • Map Oracle date to Trino timestamp(0) type. (#10626)

  • +
  • Fix performance regression of predicate pushdown on indexed date columns. +(#10626)

  • +
  • Fix spurious errors when metadata cache is enabled. (#10544, +#10512)

  • +
+
+
+

Phoenix connector#

+
    +
  • Fix spurious errors when metadata cache is enabled. (#10544, +#10512)

  • +
+
+
+

Pinot connector#

+
    +
  • Add support for basic authentication. (#9531)

  • +
+
+
+

PostgreSQL connector#

+ +
+
+

Redshift connector#

+ +
+
+

SingleStore (MemSQL) connector#

+
    +
  • Fix spurious errors when metadata cache is enabled. (#10544, +#10512)

  • +
+
+
+

SQL Server connector#

+
    +
  • Fix spurious errors when metadata cache is enabled. (#10544, +#10512)

  • +
+
+
+

SPI#

+
    +
  • Remove support for the ConnectorMetadata.getTableLayout() API. +(#781)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-370.html b/430/release/release-370.html new file mode 100644 index 000000000..a5857cabf --- /dev/null +++ b/430/release/release-370.html @@ -0,0 +1,2786 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 370 (3 Feb 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 370 (3 Feb 2022)#

+
+

General#

+
    +
  • Add support for DEFAULT keyword in ALTER TABLE...SET PROPERTIES.... +(#10331)

  • +
  • Improve performance of map and row types. (#10469)

  • +
  • Improve performance when evaluating expressions in WHERE and SELECT +clauses. (#10322)

  • +
  • Prevent queries deadlock when using phased execution policy with dynamic +filters in multi-join queries. (#10868)

  • +
  • Fix query scheduling regression introduced in Trino 360 that caused +coordinator slowness in assigning splits to workers. (#10839)

  • +
  • Fix information_schema query failure when an IS NOT NULL predicate is +used. (#10861)

  • +
  • Fix failure when nested subquery contains a TABLESAMPLE clause. +(#10764)

  • +
+
+
+

Security#

+
    +
  • Reduced the latency of successful OAuth 2.0 authentication. (#10929)

  • +
  • Fix server start failure when using JWT and OAuth 2.0 authentication together +(http-server.authentication.type=jwt,oauth2). (#10811)

  • +
+
+
+

CLI#

+
    +
  • Add support for ARM64 processors. (#10177)

  • +
  • Allow to choose the way how external authentication is handled with the +--external-authentication-redirect-handler parameter. (#10248)

  • +
+
+
+

RPM package#

+
    +
  • Fix failure when operating system open file count is set too low. +(#8819)

  • +
+
+
+

Docker image#

+
    +
  • Change base image to registry.access.redhat.com/ubi8/ubi, since CentOS 8 has +reached end-of-life. (#10866)

  • +
+
+
+

Cassandra connector#

+
    +
  • Fix query failure when pushing predicates on uuid partitioned columns. +(#10799)

  • +
+
+
+

ClickHouse connector#

+
    +
  • Support creating tables with Trino timestamp(0) type columns.

  • +
  • Drop support for ClickHouse servers older than version 20.7 to avoid using a +deprecated driver. You can continue to use the deprecated driver with the +clickhouse.legacy-driver flag when connecting to old servers. +(#10541)

  • +
  • Remove the legacy allow-drop-table configuration property. This defaulted to +false, which disallowed dropping tables, but other modifications were still +allowed. Use System access control instead, if +desired. (#588)

  • +
+
+
+

Druid connector#

+
    +
  • Remove the legacy allow-drop-table configuration property. This defaulted to +false, which disallowed dropping tables, but other modifications were still +allowed. Use System access control instead, if +desired. (#588)

  • +
+
+
+

Hive connector#

+
    +
  • Improve query performance when reading ORC data. (#10575)

  • +
  • Add configuration property hive.single-statement-writes to require +auto-commit for writes. This can be used to disallow multi-statement write +transactions. (#10820)

  • +
  • Fix sporadic query failure Partition no longer exists when working with wide +tables using a AWS Glue catalog as metastore. (#10696)

  • +
  • Fix SHOW TABLES failure when hive.hide-delta-lake-tables is enabled, and +Glue metastore references the table with no properties. (#10864)

  • +
+
+
+

Iceberg connector#

+
    +
  • Fix query failure when reading from a table that underwent partitioning +evolution. (#10770)

  • +
  • Fix data corruption when writing Parquet files. (#9749)

  • +
+
+
+

MySQL connector#

+
    +
  • Remove the legacy allow-drop-table configuration property. This defaulted to +false, which disallowed dropping tables, but other modifications were still +allowed. Use System access control instead, if +desired. (#588)

  • +
+
+
+

Oracle connector#

+
    +
  • Remove the legacy allow-drop-table configuration property. This defaulted to +false, which disallowed dropping tables, but other modifications were still +allowed. Use System access control instead, if +desired. (#588)

  • +
+
+
+

Phoenix connector#

+
    +
  • Fix incorrect result when a date value is older than or equal to +1899-12-31. (#10749)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Remove the legacy allow-drop-table configuration property. This defaulted to +false, which disallowed dropping tables, but other modifications were still +allowed. Use System access control instead, if +desired. (#588)

  • +
+
+
+

Redshift connector#

+
    +
  • Remove the legacy allow-drop-table configuration property. This defaulted to +false, which disallowed dropping tables, but other modifications were still +allowed. Use System access control instead, if +desired. (#588)

  • +
+
+
+

SingleStore (MemSQL) connector#

+
    +
  • Remove the legacy allow-drop-table configuration property. This defaulted to +false, which disallowed dropping tables, but other modifications were still +allowed. Use System access control instead, if +desired. (#588)

  • +
+
+
+

SQL Server connector#

+
    +
  • Remove the legacy allow-drop-table configuration property. This defaulted to +false, which disallowed dropping tables, but other modifications were still +allowed. Use System access control instead, if +desired. (#588)

  • +
+
+
+

SPI#

+
    +
  • Allow null property names in ConnetorMetadata#setTableProperties. +(#10331)

  • +
  • Rename ConnectorNewTableLayout to ConnectorTableLayout. (#10587)

  • +
  • Connectors no longer need to explicitly declare handle classes. The +ConnectorFactory.getHandleResolver and Connector.getHandleResolver methods +are removed. (#10858, #10872)

  • +
  • Remove unnecessary Block.writePositionTo and BlockBuilder.appendStructure +methods. Use of these methods can be replaced with the existing +Type.appendTo or writeObject methods. (#10602)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-371.html b/430/release/release-371.html new file mode 100644 index 000000000..ec76f8438 --- /dev/null +++ b/430/release/release-371.html @@ -0,0 +1,2653 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 371 (16 Feb 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+ +
+ +
+
+ +
+

Release 371 (16 Feb 2022)#

+
+

General#

+
    +
  • Add support for using secrets in database backed resource group manager +configuration. (#10996)

  • +
  • Add support for the user group selector rule in database backed resource group +manager. (#10914)

  • +
  • Remove deprecated.disable-set-properties-security-check-for-create-ddl +configuration toggle. (#10923)

  • +
  • Prevent infinite planning loop by removing redundant predicates above table +scan. (#10532)

  • +
  • Prevent time outs for planning of certain queries. (#10985)

  • +
  • Fix performance regression in internal communication authentication +processing. (#10954)

  • +
  • Fix failure when casting values to decimal(38, 38). (#10946)

  • +
  • Enforce timeout for idle transactions. (#10923)

  • +
  • Ensure removal of all catalog session properties when using session property +defaults with transactions. (#10923)

  • +
+
+
+

Security#

+
    +
  • Invoke correct authorization check when table is created via CREATE TABLE AS. (#10939)

  • +
+
+
+

ClickHouse connector#

+
    +
  • Remove support for ClickHouse connector in Altinity distribution 20.3. +(#10975)

  • +
  • Add missing output of table properties for SHOW CREATE TABLE statements. +(#11027)

  • +
+
+
+

Hive connector#

+
    +
  • Allow specifying AWS role session name via S3 security mapping config. +(#10714)

  • +
  • Disallow writes to bucketed tables recognized as created by Spark to prevent +data corruption. Spark uses a custom bucketing hash function that is not +compatible with Hive and Trino. (#10815)

  • +
  • Fix failure when reading Hive tables that contain symlinks that are text +files. (#10910)

  • +
  • Fix metastore impersonation for Avro tables. (#11035)

  • +
+
+
+

Iceberg connector#

+
    +
  • Allow running queries performing DML on Iceberg tables with fault-tolerant +execution. (#10622)

  • +
  • Create files of up to approximately 1GB of size when writing. This can be +configured using hive.target-max-file-size catalog property or +target_max_file_size session property. (#10957)

  • +
+
+
+

Kudu connector#

+
    +
  • Drop support for Kudu versions older than 1.13.0. (#10940)

  • +
+
+
+

SQL Server connector#

+
    +
  • Fix incorrect results when negative dates are specified in predicates. +(#10263)

  • +
  • Fix incorrect results when writing negative dates. (#10263)

  • +
+
+
+

SPI#

+
    +
  • Add ConnectorSession to the Connector getMetadata method. The former +signature is deprecated and should be updated. (#9482)

  • +
  • Remove deprecated checkCanCreateTable and checkCanCreateMaterializedView +methods not taking parameters. (#10939)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-372.html b/430/release/release-372.html new file mode 100644 index 000000000..4a1822ac4 --- /dev/null +++ b/430/release/release-372.html @@ -0,0 +1,2805 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 372 (2 Mar 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 372 (2 Mar 2022)#

+
+

General#

+
    +
  • Add trim_array() function. (#11238)

  • +
  • Improve handling of prepared statements with long query text by compressing +them within HTTP headers. This can be configured or disabled using the +protocol.v1.prepared-statement-compression.length-threshold configuration +property. (#11098)

  • +
  • Improve performance of specific queries which compare table columns of type +timestamp with date literals. (#11170)

  • +
  • Add redirection awareness for ADD COLUMN, DROP TABLE, COMMENT tasks. (#11072)

  • +
  • Remove support for reserved memory pool. Configuration property +experimental.reserved-pool-disabled can no longer be used. (#6677)

  • +
  • Ensure memory is released completely after query completion. (#11030)

  • +
  • Fix certain queries failing due to dictionary compacting error. (#11080)

  • +
  • Fix SET SESSION and RESET SESSION not working for catalogs which include +special characters in their name. (#11171)

  • +
  • Fix bug where queries were not transitioned to RUNNING state when task-level +retries were enabled. (#11198)

  • +
+
+
+

Security#

+
    +
  • Allow configuration of connect and read timeouts for LDAP authentication. (#10925)

  • +
+
+
+

Docker image#

+
    +
  • Add a health check to the Docker container image. (#10413)

  • +
+
+
+

JDBC driver#

+
    +
  • Fix DatabaseMetaData#getTables and DatabaseMetaData#getColumns to include +views for Iceberg, Raptor, Accumulo and Blackhole connectors. (#11063, #11060)

  • +
+
+
+

Base-JDBC connector library#

+
    +
  • Fix spurious query failures when metadata cache is not enabled and data +access depends on the session state. (#11068)

  • +
+
+
+

Accumulo connector#

+
    +
  • Fix incorrect results when querying date type columns. (#11055)

  • +
+
+
+

Cassandra connector#

+
    +
  • Fix incorrect results when filtering partition keys without projections. +(#11001)

  • +
+
+
+

ClickHouse connector#

+
    +
  • Fix spurious query failures when metadata cache is not enabled and extra +credentials with user-credential-name or password-credential-name are used +to access data. (#11068)

  • +
+
+
+

Druid connector#

+
    +
  • Fix spurious query failures when metadata cache is not enabled and extra +credentials with user-credential-name or password-credential-name are used +to access data. (#11068)

  • +
+
+
+

Hive connector#

+
    +
  • Allow redirects of Iceberg or Delta tables which have no StorageDescriptor +in the Glue metastore. (#11092)

  • +
  • Stop logging spurious failures when storage caching +is used. (#11101)

  • +
  • Allow reading Zstandard compressed Avro files. (#11090)

  • +
  • Fix incorrect query results after writes to a table when directory caching is +enabled enabled with the hive.file-status-cache-tables configuration +property. (#10621)

  • +
  • Fix potential query failures for queries writing data to tables backed by S3. +(#11089)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add support for COMMENT ON COLUMN statement. (#11143)

  • +
  • Improve query performance after table schema evolved, by fixing the connector +to support table stats in such case. (#11091)

  • +
  • Fix potential query failures for queries writing data to tables backed by S3. (#11089)

  • +
  • Prevent query failure from dereference pushdown when a column has a comment. (#11104)

  • +
+
+
+

Kudu connector#

+
    +
  • Add support for Kerberos authentication. (#10953)

  • +
+
+
+

MongoDB connector#

+
    +
  • Map MongoDB bindata type to Trino varbinary type if explicit schema does +not exist. (#11122)

  • +
+
+
+

MySQL connector#

+
    +
  • Fix spurious query failures when metadata cache is not enabled and extra +credentials with user-credential-name or password-credential-name are used +to access data. (#11068)

  • +
+
+
+

Oracle connector#

+
    +
  • Fix spurious query failures when metadata cache is not enabled and extra +credentials with user-credential-name or password-credential-name are used +to access data. (#11068)

  • +
+
+
+

Phoenix connector#

+
    +
  • Fix query failures when applying predicates on array(char) type columns. (#10451)

  • +
  • Fix metadata listing failure in case of concurrent table deletion. (#10904)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Add support for pushing down joins on character string type columns. (#10059)

  • +
  • Fix spurious query failures when metadata cache is not enabled and extra +credentials with user-credential-name or password-credential-name are used +to access data. (#11068)

  • +
+
+
+

Redshift connector#

+
    +
  • Fix spurious query failures when metadata cache is not enabled and extra +credentials with user-credential-name or password-credential-name are used +to access data. (#11068)

  • +
+
+
+

SingleStore (MemSQL) connector#

+
    +
  • Fix spurious query failures when metadata cache is not enabled, and extra +credentials with user-credential-name or password-credential-name are used +to access data. (#11068)

  • +
+
+
+

SQL Server connector#

+
    +
  • Update JDBC driver to 10.2.0. The new version automatically enables TLS and +certificate verification. Update the TLS configuration to +keep the old behavior. (#10898)

  • +
  • Fix spurious query failures when metadata cache is not enabled and extra +credentials with user-credential-name or password-credential-name are used +to access data. (#11068)

  • +
+
+
+

SPI#

+
    +
  • Pass more information about predicates in ConnectorMetadata#applyFilter +invocation. The predicates that cannot be represented with a TupleDomain are +available via Constraint.getExpression(). (#7994)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-373.html b/430/release/release-373.html new file mode 100644 index 000000000..f20bb87b8 --- /dev/null +++ b/430/release/release-373.html @@ -0,0 +1,2681 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 373 (9 Mar 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 373 (9 Mar 2022)#

+
+

General#

+
    +
  • Add Delta Lake connector. (#11296, #10897)

  • +
  • Improve query performance by reducing overhead of cluster internal +communication. (#11146)

  • +
  • Handle varchar to timestamp conversion errors in try(). (#11259)

  • +
  • Add redirection awareness for DROP COLUMN task. (#11304)

  • +
  • Add redirection awareness for RENAME COLUMN task. (#11226)

  • +
  • Disallow table redirections in SHOW GRANTS statement. (#11270)

  • +
  • Allow low memory killer to abort individual tasks when retry-mode is set to +TASK. This requires query.low-memory-killer.policy set to +total-reservation-on-blocked-nodes. (#11129)

  • +
  • Fix incorrect results when distinct or ordered aggregation are used and +spilling is enabled. (#11353)

  • +
+
+
+

Web UI#

+
    +
  • Add CPU time, scheduled time, and cumulative memory statistics regarding +failed tasks in a query. (#10754)

  • +
+
+
+

BigQuery connector#

+
    +
  • Allow configuring view expiration time via the bigquery.view-expire-duration +config property. (#11272)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Improve performance of queries involving LIKE by pushing predicate +computation to the Elasticsearch cluster. (#7994, #11308)

  • +
+
+
+

Hive connector#

+
    +
  • Support access to S3 via a HTTP proxy. (#11255)

  • +
  • Improve query performance by better estimating partitioned tables statistics. (#11333)

  • +
  • Prevent failure for queries with the final number of partitions +below HIVE_EXCEEDED_PARTITION_LIMIT. (#10215)

  • +
  • Fix issue where duplicate rows could be inserted into a partition when +insert_existing_partitions_behavior was set to OVERWRITE and +retry-policy was TASK. (#11196)

  • +
  • Fix failure when querying Hive views containing column aliases that differ in +case only. (#11159)

  • +
+
+
+

Iceberg connector#

+
    +
  • Support access to S3 via a HTTP proxy. (#11255)

  • +
  • Delete table data when dropping table. (#11062)

  • +
  • Fix SHOW TABLES failure when a materialized view is removed during query +execution. (#10976)

  • +
  • Fix query failure when reading from information_schema.tables or +information_schema.columns and a materialized view is removed during +query execution. (#10976)

  • +
+
+
+

Oracle connector#

+
    +
  • Fix query failure when performing concurrent write operations. (#11318)

  • +
+
+
+

Phoenix connector#

+
    +
  • Prevent writing incorrect results when arrays contain null values. (#11351)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Improve performance of queries involving LIKE by pushing predicate +computation to the underlying database. (#11045)

  • +
+
+
+

SQL Server connector#

+
    +
  • Fix incorrect results when querying SQL Server tinyint columns by mapping +them to Trino smallint. (#11209)

  • +
+
+
+

SPI#

+
    +
  • Add CPU time, scheduled time, and cumulative memory statistics regarding +failed tasks in a query to query-completion events. (#10734)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-374.html b/430/release/release-374.html new file mode 100644 index 000000000..f5812a3c7 --- /dev/null +++ b/430/release/release-374.html @@ -0,0 +1,2684 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 374 (17 Mar 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 374 (17 Mar 2022)#

+
+

General#

+
    +
  • Add support for query parameters in CREATE SCHEMA. (#11485)

  • +
  • Improve performance when reading from S3-based spool for +fault-tolerant execution. (#11050)

  • +
  • Improve performance of queries with GROUP BY clauses. (#11392)

  • +
  • Improve performance of GROUP BY with a large number of groups. (#11011)

  • +
  • Improve handling of queries where individual tasks require lots of memory when +retry-policy is set to TASK. (#10432)

  • +
  • Produce better query plans by improving cost-based-optimizer estimates in the +presence of correlated columns. (#11324)

  • +
  • Fix memory accounting and improve performance for queries involving certain +variable-width data types such as varchar or varbinary. (#11315)

  • +
  • Fix performance regression for GROUP BY queries. (#11234)

  • +
  • Fix trim, ltrim and rtim function results when the argument is char +type. Previously, it returned padded results as char type. It returns +varchar type without padding now. (#11440)

  • +
+
+
+

JDBC driver#

+
    +
  • Add support for DatabaseMetaData.getImportedKeys. (#8708)

  • +
  • Fix Driver.getPropertyInfo(), and validate allowed properties. (#10624)

  • +
+
+
+

CLI#

+
    +
  • Add support for selecting Vim or Emacs editing modes with the --editing-mode +command line argument. (#3377)

  • +
+
+
+

Cassandra connector#

+ +
+
+

ClickHouse connector#

+
    +
  • Add support for uint8, uint16, uint32 and uint64 types. (#11490)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Allow specifying STS endpoint to be used when connecting to S3. (#10169)

  • +
  • Fix query failures due to exhausted file system resources after DELETE or +UPDATE. (#11418)

  • +
+
+
+

Hive connector#

+
    +
  • Allow specifying STS endpoint to be used when connecting to S3. (#10169)

  • +
  • Fix shared metadata caching with Hive ACID tables. (#11443)

  • +
+
+
+

Iceberg connector#

+
    +
  • Allow specifying STS endpoint to be used when connecting to S3. (#10169)

  • +
  • Add support for using Glue metastore as Iceberg catalog. (#10845)

  • +
+
+
+

MongoDB connector#

+ +
+
+

PostgreSQL connector#

+
    +
  • Improve performance of queries involving OR with simple comparisons and +LIKE predicates by pushing predicate computation to the PostgreSQL database. +(#11086)

  • +
  • Improve performance of aggregation queries with certain complex predicates by +computing predicates and aggregations within PostgreSQL. (#11083)

  • +
  • Fix possible connection leak when connecting to PostgreSQL failed. (#11449)

  • +
+
+
+

SingleStore (MemSQL) connector#

+
    +
  • The connector now uses the official Single Store JDBC Driver. As a result, +connection-url in catalog configuration files needs to be updated from +jdbc:mariadb:... to jdbc:singlestore:.... (#10669)

  • +
  • Deprecate memsql as the connector name. We recommend using singlestore in +the connector.name configuration property. (#11459)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-375.html b/430/release/release-375.html new file mode 100644 index 000000000..c985d5af2 --- /dev/null +++ b/430/release/release-375.html @@ -0,0 +1,2694 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 375 (28 Mar 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 375 (28 Mar 2022)#

+
+

General#

+
    +
  • Change behavior of ALTER TABLE qualified_name TO unqualified_name to leave +the table in the schema. This is backwards-incompatible behavioral change. (#11282)

  • +
  • Disallow table redirections for the GRANT, REVOKE, and +SET TABLE AUTHORIZATION table tasks. (#11302)

  • +
  • Improve performance of queries that contain filter predicates on aggregation +results. (#11469)

  • +
  • Improve performance of queries that contain array_distinct(array_sort(…)) +expressions. (#8777)

  • +
  • Fix SHOW CREATE TABLE to show actual table name in case of table +redirections, so that the statement can be used to recreate the table. (#11604)

  • +
  • Fix scheduling for non-remotely accessible splits in fault-tolerant execution. (#11581)

  • +
  • Fix incorrect is_nullable value in information_schema.columns table. (#11613)

  • +
+
+
+

JDBC driver#

+
    +
  • Allow PreparedStatement.close() to be called multiple times. (#11620)

  • +
  • Fix incorrect is_nullable value in DatabaseMetaData.getColumns() method. (#11613)

  • +
+
+
+

Cassandra connector#

+
    +
  • Return 0 instead of row count in completed bytes. (#11644)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Add access control to drop_extended_stats and vacuum procedures. (#11633)

  • +
  • Fix incorrect query results when query executes concurrent with DROP TABLE. (#11562)

  • +
+
+
+

Hive connector#

+
    +
  • Fix infinite loop in the query optimizer when query contains predicates on a +struct field. (#11559)

  • +
  • Fix query failure when reading a partitioned table with a predicate on a +partition column with a specific name such as table, key, order, and +others. (#11512)

  • +
+
+
+

Iceberg connector#

+
    +
  • Fix failure when query contains predicates on a struct field. (#11560)

  • +
  • Fix query failure when reading from $files system table after a table column +has been dropped. (#11576)

  • +
+
+
+

Kudu connector#

+
    +
  • Improve write performance by flushing operations in batches. (#11264)

  • +
  • Fix failure when multiple queries run concurrently and schema emulation is enabled. (#11264)

  • +
+
+
+

MongoDB connector#

+
    +
  • Support predicate pushdown on boolean columns. (#11536)

  • +
  • Return 0 instead of row count in completed bytes. (#11679)

  • +
+
+
+

MySQL connector#

+
    +
  • Add support for table comments. (#11211)

  • +
+
+
+

Pinot connector#

+
    +
  • Fix handling of passthrough queries that contain aggregation functions. (#9137)

  • +
  • Fix incorrect results when aggregation functions on columns having +non-lowercase names are pushed down to Pinot. (#9137, #10148)

  • +
  • Fix possible incorrect results when grouping on columns of array types. (#9781)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Improve performance of queries involving OR with IS NULL, IS NOT NULL +predicates, or involving NOT expression by pushing predicate computation to +the PostgreSQL database. (#11514)

  • +
  • Improve performance of queries with predicates involving nullif function by +pushing predicate computation to the PostgreSQL database. (#11532)

  • +
  • Improve performance of queries involving joins by pushing computation to the +PostgreSQL database. (#11635)

  • +
  • Improve performance of queries involving predicates with arithmetic +expressions by pushing predicate computation to the PostgreSQL database. (#11510)

  • +
  • Fix deletion of too much data when delete query involves a LIKE predicate. (#11615)

  • +
+
+
+

SPI#

+
    +
  • Add processed input bytes and rows to query events in event listener. (#11623)

  • +
  • Remove deprecated constructors from ColumnMetadata. (#11621)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-376.html b/430/release/release-376.html new file mode 100644 index 000000000..6c567200c --- /dev/null +++ b/430/release/release-376.html @@ -0,0 +1,2720 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 376 (7 Apr 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 376 (7 Apr 2022)#

+
+

General#

+
    +
  • Add table redirection awareness for RENAME table operations. (#11277)

  • +
  • Deny adding column with comment if the connector does not support this feature. (#11486)

  • +
  • Improve performance for queries that contain inequality expressions. (#11518)

  • +
  • Consider null values as identical values in array_except, array_union, +map_concat, map_from_entries, multimap_from_entries, and multimap_agg +functions. (#560)

  • +
  • Fix failure of DISTINCT .. LIMIT operator when input data is dictionary +encoded. (#11776)

  • +
  • Fix returning of invalid results for distinct aggregation when input data is +dictionary encoded. (#11776)

  • +
  • Fix query failure when performing joins with connectors that support index lookups. (#11758)

  • +
  • Fix incorrect stage memory statistics reporting for queries running with +retry-policy set to TASK. (#11801)

  • +
+
+
+

Security#

+
    +
  • Add support to use two-way TLS/SSL certificate validation with LDAP authentication. +Additionally ldap.ssl-trust-certificate config is replaced by ldap.ssl.truststore.path. (#11070).

  • +
  • Fix failures in information schema role tables for catalogs using system roles. (#11694)

  • +
+
+
+

Web UI#

+
    +
  • Add new page to display the runtime information of all workers in the cluster. (#11653)

  • +
+
+
+

JDBC driver#

+
    +
  • Add support for using the system truststore with the SSLUseSystemTrustStore parameter. (#10482)

  • +
  • Add support for ResultSet.getAsciiStream() and ResultSet.getBinaryStream(). (#11753)

  • +
  • Remove user property requirement. (#11350)

  • +
+
+
+

CLI#

+
    +
  • Add support for using the system truststore with the --use-system-truststore option. (#10482)

  • +
+
+
+

Accumulo connector#

+
    +
  • Add support for adding and dropping schemas. (#11808)

  • +
  • Disallow creating tables in a schema that doesn’t exist. (#11808)

  • +
+
+
+

ClickHouse connector#

+
    +
  • Add support for column comments when creating new tables. (#11606)

  • +
  • Add support for column comments when adding new columns. (#11606)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Add support for INSERT, UPDATE, and DELETE queries on Delta Lake tables +with fault-tolerant execution. (#11591)

  • +
  • Allow setting duration for completion of dynamic filtering +with the delta.dynamic-filtering.wait-timeout configuration property. (#11600)

  • +
  • Improve query planning time after ALTER TABLE ... EXECUTE optimize by always +creating a transaction log checkpoint. (#11721)

  • +
  • Add support for reading Delta Lake tables in with auto-commit mode disabled. (#11792)

  • +
+
+
+

Hive connector#

+
    +
  • Store file min/max ORC statistics for string columns even when actual min or +max value exceeds 64 bytes. This improves query performance when filtering on +such column. (#11652)

  • +
  • Improve performance when reading Parquet data. (#11675)

  • +
  • Improve query performance when the same table is referenced multiple times +within a query. (#11650)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add support for views when using Iceberg Glue catalog. (#11499)

  • +
  • Add support for reading Iceberg v2 tables containing deletion files. (#11642)

  • +
  • Add support for table redirections to the Hive connector. (#11356)

  • +
  • Include non-Iceberg tables when listing tables from Hive catalogs. (#11617)

  • +
  • Expose nan_count in the $partitions metadata table. (#10709)

  • +
  • Store file min/max ORC statistics for string columns even when actual min or +max value exceeds 64 bytes. This improves query performance when filtering on +such column. (#11652)

  • +
  • Improve performance when reading Parquet data. (#11675)

  • +
  • Fix NPE when an Iceberg data file is missing null count statistics. (#11832)

  • +
+
+
+

Kudu connector#

+
    +
  • Add support for adding columns with comment. (#11486)

  • +
+
+
+

MySQL connector#

+
    +
  • Improve performance of queries involving joins by pushing computation to the +MySQL database. (#11638)

  • +
+
+
+

Oracle connector#

+
    +
  • Improve query performance of queries involving aggregation by pushing +aggregation computation to the Oracle database. (#11657)

  • +
+
+
+

SPI#

+
    +
  • Add support for table procedures that execute on the coordinator only. (#11750)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-377.html b/430/release/release-377.html new file mode 100644 index 000000000..1050bf0dc --- /dev/null +++ b/430/release/release-377.html @@ -0,0 +1,2598 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 377 (13 Apr 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 377 (13 Apr 2022)#

+
+

General#

+
    +
  • Add support for standard SQL trim syntax. (#11236)

  • +
  • Fix incorrect results when queries execute in fault-tolerant mode. (#11870)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for date type partition names with timestamp formatting. (#11873)

  • +
  • Improve performance of queries that use Glue metadata. (#11869)

  • +
  • Fix failure of the sync_partition_metadata procedure when partition names +differ from partition paths on the file system. (#11864)

  • +
+
+
+

Iceberg connector#

+
    +
  • Support setting Glue metastore catalog identifier with the +hive.metastore.glue.catalogid catalog configuration property. (#11520)

  • +
  • Add support for materialized views when using Glue metastore. (#11780)

  • +
+
+
+

Kafka connector#

+
    +
  • Add support for additional Kafka client properties specified with the +kafka.config.resources catalog configuration property. (#8743)

  • +
+
+
+

SQL Server connector#

+
    +
  • Improve performance of queries involving joins by pushing computation to the +SQL Server database. (#11637)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-378.html b/430/release/release-378.html new file mode 100644 index 000000000..a417a203e --- /dev/null +++ b/430/release/release-378.html @@ -0,0 +1,2675 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 378 (21 Apr 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 378 (21 Apr 2022)#

+
+

General#

+ +
+
+

Security#

+ +
+
+

CLI#

+
    +
  • Allow disabling progress reporting during query executing in the CLI client by +specifying --no-progress (#11894)

  • +
  • Reduce latency for very short queries. (#11768)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Improve query planning performance. (#11858)

  • +
  • Fix failure when reading from information_schema.columns when metastore +contains views. (#11946)

  • +
  • Add support for dropping tables with invalid metadata. (#11924)

  • +
  • Fix query failure when partition column has a null value and query has a +complex predicate on that partition column. (#12056)

  • +
+
+
+

Hive connector#

+
    +
  • Improve query planning performance. (#11858)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add support for hidden $path columns. (#8769)

  • +
  • Add support for creating tables with either Iceberg format version 1, or 2. (#11880)

  • +
  • Add the expire_snapshots table procedure. (#10810)

  • +
  • Add the delete_orphan_files table procedure. (#10810)

  • +
  • Allow reading Iceberg tables written by Glue that have locations containing +double slashes. (#11964)

  • +
  • Improve query planning performance. (#11858)

  • +
  • Fix query failure with a dynamic filter prunes a split on a worker node. (#11976)

  • +
  • Include missing format_version property in SHOW CREATE TABLE output. (#11980)

  • +
+
+
+

MySQL connector#

+
    +
  • Improve query planning performance. (#11858)

  • +
+
+
+

Pinot connector#

+
    +
  • Support querying tables having non-lowercase names in Pinot. (#6789)

  • +
  • Fix handling of hybrid tables in Pinot and stop returning duplicate data. (#10125)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Improve query planning performance. (#11858)

  • +
+
+
+

SQL Server connector#

+
    +
  • Improve query planning performance. (#11858)

  • +
+
+
+

SPI#

+
    +
  • Deprecate passing constraints to ConnectorMetadata.getTableStatistics(). +Constraints can be associated with the table handle in +ConnectorMetadata.applyFilter(). (#11877)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-379.html b/430/release/release-379.html new file mode 100644 index 000000000..ee36be82d --- /dev/null +++ b/430/release/release-379.html @@ -0,0 +1,2629 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 379 (28 Apr 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+ +
+ +
+
+ +
+

Release 379 (28 Apr 2022)#

+
+

General#

+
    +
  • Add MariaDB connector. (#10046)

  • +
  • Improve performance of queries that contain JOIN and UNION clauses. (#11935)

  • +
  • Improve performance of queries that contain GROUP BY clauses. (#12095)

  • +
  • Fail DROP TABLE IF EXISTS when deleted entity is not a table. Previously the +statement did not delete anything. (#11555)

  • +
  • Fail DROP VIEW IF EXISTS when deleted entity is not a view. Previously the +statement did not delete anything. (#11555)

  • +
  • Fail DROP MATERIALIZED VIEW IF EXISTS when deleted entity is not a +materialized view. Previously the statement did not delete anything. +(#11555)

  • +
+
+
+

Web UI#

+
    +
  • Group information about tasks by stage. (#12099)

  • +
  • Show aggregated statistics for failed tasks of queries that are executed with +retry-policy set to TASK. (#12099)

  • +
  • Fix reporting of physical input read time. (#12135)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Add support for Google Cloud Storage. (#12144)

  • +
  • Fix failure when reading from information_schema.columns when non-Delta +tables are present in the metastore. (#12122)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add support for DELETE with arbitrary predicates. (#11886)

  • +
  • Improve compatibility when Glue storage properties are used. (#12164)

  • +
  • Prevent data loss when queries modify a table concurrently when Glue catalog +is used. (#11713)

  • +
  • Enable commit retries when conflicts occur writing a transaction to a Hive Metastore. (#12419)

  • +
  • Always return the number of deleted rows for DELETE statements. (#12055)

  • +
+
+
+

Pinot connector#

+
    +
  • Add support for Pinot 0.10. (#11475)

  • +
+
+
+

Redis connector#

+
    +
  • Improve performance when reading data from Redis. (#12108)

  • +
+
+
+

SQL Server connector#

+
    +
  • Properly apply snapshot isolation to all connections when it is enabled. (#11662)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-380.html b/430/release/release-380.html new file mode 100644 index 000000000..b557827f6 --- /dev/null +++ b/430/release/release-380.html @@ -0,0 +1,2620 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 380 (6 May 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 380 (6 May 2022)#

+
+

General#

+
    +
  • Enable automatic writer scaling by default. (#10614)

  • +
  • Improve performance of joins involving comparisons with the <,<=, >,>= operators. (#12236)

  • +
+
+
+

Cassandra connector#

+
    +
  • Add support for the v5 and v6 protocols. (#7729)

  • +
  • Removes support for v2 protocol. (#7729)

  • +
  • Make the cassandra.load-policy.use-dc-aware and cassandra.load-policy.dc-aware.local-dc +catalog configuration properties mandatory. (#7729)

  • +
+
+
+

Hive connector#

+
    +
  • Support table redirections from Hive to Delta Lake. (#11550)

  • +
  • Allow configuring a default value for the auto_purge table property with the +hive.auto-purge catalog property. (#11749)

  • +
  • Allow configuration of the Hive views translation security semantics with the +hive.hive-views.run-as-invoker catalog configuration property. (#9227)

  • +
  • Rename catalog configuration property hive.translate-hive-views to +hive.hive-views.enabled. The former name is still accepted. (#12238)

  • +
  • Rename catalog configuration property hive.legacy-hive-view-translation +to hive.hive-views.legacy-translation. The former name is still accepted. (#12238)

  • +
  • Rename session property legacy_hive_view_translation to +hive_views_legacy_translation. (#12238)

  • +
+
+
+

Iceberg connector#

+
    +
  • Allow updating tables from the Iceberg v1 table format to v2 with +ALTER TABLE ... SET PROPERTIES. (#12161)

  • +
  • Allow changing the default file format for a table +with ALTER TABLE ... SET PROPERTIES. (#12161)

  • +
  • Prevent potential corruption when a table change is interrupted by networking +or timeout failures. (#10462)

  • +
+
+
+

MongoDB connector#

+ +
+
+

SQL Server connector#

+
    +
  • Add support for bulk data insertion. (#12176)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-381.html b/430/release/release-381.html new file mode 100644 index 000000000..74517a3d7 --- /dev/null +++ b/430/release/release-381.html @@ -0,0 +1,2774 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 381 (16 May 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 381 (16 May 2022)#

+
+

General#

+
    +
  • Add support for fault-tolerant execution with exchange spooling on Azure Blob Storage. (#12211)

  • +
  • Add experimental support for Table functions. (#1839)

  • +
  • Increase the default number of stages allowed for a query from 100 to 150, +specified with query.max-stage-count. (#12292)

  • +
  • Allow configuring the number of partitions for distributed joins and +aggregations when task-based fault-tolerant execution is enabled. This can be +set with the fault-tolerant-execution-partition-count configuration property +or the fault_tolerant_execution_partition_count session property. (#12263)

  • +
  • Introduce the least-waste low memory task killer policy. This policy avoids +killing tasks that are already executing for a long time, so the amount of +wasted work is minimized. It can be enabled with the +task.low-memory-killer.policy configuration property. (#12393)

  • +
  • Fix potential planning failure of queries with multiple subqueries. (#12199)

  • +
+
+
+

Security#

+
    +
  • Add support for automatic discovery of OpenID Connect metadata with OAuth 2.0 +authentication. (#9788)

  • +
  • Re-introduce ldap.ssl-trust-certificate as legacy configuration to avoid +failures when updating Trino version. (#12187)

  • +
  • Fix potential query failure when a table has multiple column masks defined. (#12262)

  • +
  • Fix incorrect masking of columns when multiple rules in file-based system and +connector access controls match. (#12203)

  • +
  • Fix authentication failure when using the LDAP password authenticator with +ActiveDirectory. (#12321)

  • +
+
+
+

Web UI#

+
    +
  • Ensure consistent sort order in the list of workers. (#12290)

  • +
+
+
+

Docker image#

+
    +
  • Improve Advanced Encryption Standard (AES) processing performance on ARM64 +processors. This is used for operations such as accessing object storage +systems via TLS/SSL. (#12251)

  • +
+
+
+

CLI#

+
    +
  • Add automatic suggestions from command history. This can be disabled with the +--disable-auto-suggestion option. (#11671)

  • +
+
+
+

BigQuery connector#

+
    +
  • Support reading materialized views. (#12352)

  • +
  • Allow skipping view materialization via bigquery.skip-view-materialization +configuration property. (#12210)

  • +
  • Support reading snapshot tables. (#12380)

  • +
+
+
+

ClickHouse connector#

+ +
+
+

Druid connector#

+
    +
  • Prevent data loss when non-transactional insert fails. (#12229)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Improve query performance by simplifying filters sent to Elasticsearch. (#10717)

  • +
  • Fix failure when reading nested timestamp values that are not ISO 8601 formatted. (#12250)

  • +
+
+
+

Hive connector#

+
    +
  • Fix query failure when the table and partition bucket counts do not match. (#11885)

  • +
+
+
+

Iceberg connector#

+ +
+
+

MariaDB connector#

+
    +
  • Prevent data loss when non-transactional insert fails. (#12229)

  • +
+
+
+

MySQL connector#

+
    +
  • Prevent data loss when non-transactional insert fails. (#12229)

  • +
+
+
+

Oracle connector#

+
    +
  • Prevent data loss when non-transactional insert fails. (#12229)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Prevent data loss when non-transactional insert fails. (#12225)

  • +
+
+
+

Redis connector#

+
    +
  • Allow specifying the refresh interval for fetching the table description with +the redis.table-description-cache-ttl configuration property. (#12240)

  • +
  • Support setting username for the connection with the redis.user +configuration property. (#12279)

  • +
+
+
+

Redshift connector#

+
    +
  • Prevent data loss when non-transactional insert fails. (#12229)

  • +
+
+
+

SingleStore (MemSQL) connector#

+
    +
  • Prevent data loss when non-transactional insert fails. (#12229)

  • +
+
+
+

SQL Server connector#

+
    +
  • Prevent data loss when non-transactional insert fails. (#12229)

  • +
+
+
+

SPI#

+
    +
  • Remove deprecated ConnectorMetadata methods without the retry mode parameter. (#12342)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-382.html b/430/release/release-382.html new file mode 100644 index 000000000..5f546c9cd --- /dev/null +++ b/430/release/release-382.html @@ -0,0 +1,2659 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 382 (25 May 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 382 (25 May 2022)#

+
+

General#

+
    +
  • Add support for fault-tolerant execution with exchange spooling on Google Cloud Storage. (#12360)

  • +
  • Drop support for exchange spooling on S3 with for the legacy schemes s3n:// and s3a://. (#12360)

  • +
  • Improve join performance when one side of the join is small. (#12257)

  • +
  • Fix potential query failures due to EXCEEDED_TASK_DESCRIPTOR_STORAGE_CAPACITY +errors with task-based fault-tolerant execution. (#12478)

  • +
+
+
+

BigQuery connector#

+ +
+
+

Delta Lake connector#

+
    +
  • Improve performance of queries that include filters on columns of timestamp with time zone type. (#12007)

  • +
  • Add support for adding columns with ALTER TABLE. (#12371)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for disabling partition caching in the Hive metastore with the +hive.metastore-cache.cache-partitions catalog configuration property. (#12343)

  • +
  • Fix potential query failure when metastore caching is enabled. (#12513)

  • +
  • Fix query failure when a transactional table contains a column named +operation, originalTransaction, bucket, rowId, row, or +currentTransaction. (#12401)

  • +
  • Fix sync_partition_metadata procedure failure when table has a large number of partitions. (#12525)

  • +
+
+
+

Iceberg connector#

+
    +
  • Support updating Iceberg table partitioning using ALTER TABLE ... SET PROPERTIES. (#12174)

  • +
  • Improves the performance of queries using equality and IN predicates when +reading ORC data that contains Bloom filters. (#11732)

  • +
  • Rename the delete_orphan_files table procedure to remove_orphan_files. (#12468)

  • +
  • Improve query performance of reads after DELETE removes all rows from a file. (#12197)

  • +
+
+
+

MySQL connector#

+
    +
  • Improve INSERT performance. (#12411)

  • +
+
+
+

Oracle connector#

+
    +
  • Improve INSERT performance when data includes NULL values. (#12400)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Improve INSERT performance. (#12417)

  • +
+
+
+

Prometheus connector#

+
    +
  • Add support for Basic authentication. (#12302)

  • +
+
+
+

SPI#

+
    +
  • Change ConnectorTableFunction into an interface and add +AbstractConnectorTableFunction class as the base implementation of table +functions. (#12531)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-383.html b/430/release/release-383.html new file mode 100644 index 000000000..eac94af7b --- /dev/null +++ b/430/release/release-383.html @@ -0,0 +1,2782 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 383 (1 Jun 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 383 (1 Jun 2022)#

+
+

Warning

+

This release has a regression that may cause queries to fail.

+
+
+

General#

+
    +
  • Introduce json_exists, json_query, and json_value JSON functions. (#9081)

  • +
  • Add AWS IAM role support for exchange spooling on S3. (#12444)

  • +
  • Improve query performance by reducing worker-to-worker communication overhead. (#11289)

  • +
  • Improve performance and reduce memory usage of queries that contain aggregations. (#12336)

  • +
  • Improve performance of correlated queries involving distinct aggregations. (#12564)

  • +
+
+
+

Web UI#

+
    +
  • Clarify format of cumulative user memory on query details page. (#12596)

  • +
+
+
+

Accumulo connector#

+
    +
  • Fail creating a new table if a table comment is specified. Previously, the +comment was ignored. (#12452)

  • +
+
+
+

BigQuery connector#

+
    +
  • Fail creating a new table if a table comment or a column comment is specified. +Previously, the comment was ignored. (#12452, #12574)

  • +
+
+
+

Cassandra connector#

+
    +
  • Fail creating a new table if a table comment or a column comment is specified. +Previously, the comment was ignored. (#12452, #12574)

  • +
+
+
+

ClickHouse connector#

+
    +
  • Fix incorrect results for certain aggregation queries when aggregations are +pushed down to the underlying database. (#12598)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Add support for table comments during table creation.. (#12452)

  • +
  • Fix incorrect table already exists error caused by a client timeout when +creating a new table. (#12300)

  • +
  • Fail creating a new table if a column comment is specified. Previously, the +comment was ignored. (#12574)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add support for v2 tables for the optimize table procedure. (#12351)

  • +
  • Rename hive.target-max-file-size to iceberg.target-max-file-size and +hive.delete-schema-locations-fallback to iceberg.delete-schema-locations-fallback. (#12330)

  • +
+
+
+

Kudu connector#

+
    +
  • Fail creating a new table if a table comment or a column comment is specified. +Previously, the comment was ignored. (#12452, #12574)

  • +
+
+
+

MariaDB connector#

+
    +
  • Fix incorrect results for certain queries involving aggregations that are +pushed down to the underlying database. (#12598)

  • +
  • Fail creating a new table if a column comment is specified. Previously, the +comment was ignored. (#12574)

  • +
+
+
+

Memory connector#

+
    +
  • Fail creating a new table if a table comment or a column comment is specified. +Previously, the comment was ignored. (#12452, #12574)

  • +
+
+
+

MySQL connector#

+
    +
  • Fix incorrect results for certain aggregation queries when aggregations are +pushed down to the underlying database. (#12598)

  • +
  • Fail creating a new table if a column comment is specified. Previously, the +comment was ignored. (#12574)

  • +
+
+
+

Oracle connector#

+
    +
  • Fail creating a new table if a table comment or a column comment is specified. +Previously, the comment was ignored. (#12452, #12574)

  • +
  • Fix incorrect results for certain aggregation queries when aggregations are +pushed down to the underlying database. (#12598)

  • +
+
+
+

Phoenix connector#

+
    +
  • Fail creating a new table if a table comment or a column comment is specified. +Previously, the comment was ignored. (#12452, #12574)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Fail creating a new table if a table comment or a column comment is specified. +Previously, the comment was ignored. (#12452, #12574)

  • +
  • Fix incorrect results for certain aggregation queries when aggregations are +pushed down to the underlying database. (#12598)

  • +
+
+
+

Raptor connector#

+
    +
  • Fail creating a new table if a table comment or a column comment is specified. +Previously, the comment was ignored. (#12452, #12574)

  • +
+
+
+

Redshift connector#

+
    +
  • Fail creating a new table if a table comment or a column comment is specified. +Previously, the comment was ignored. (#12452, #12574)

  • +
+
+
+

SingleStore (MemSQL) connector#

+
    +
  • Fail creating a new table if a table comment or a column comment is specified. +Previously, the comment was ignored. (#12452, #12574)

  • +
  • Fix incorrect results for certain aggregation queries when aggregations are +pushed down to the underlying database. (#12598)

  • +
+
+
+

SQL Server connector#

+
    +
  • Fail creating a new table if a table comment or a column comment is specified. +Previously, the comment was ignored. (#12452, #12574)

  • +
  • Fix incorrect results for certain aggregation queries when aggregations are +pushed down to the underlying database. (#12598)

  • +
+
+
+

SPI#

+
    +
  • Allow limiting access to functions based on whether they are scalar, +aggregation, window, or table functions. (#12544)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-384.html b/430/release/release-384.html new file mode 100644 index 000000000..655295f06 --- /dev/null +++ b/430/release/release-384.html @@ -0,0 +1,2631 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 384 (3 Jun 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 384 (3 Jun 2022)#

+
+

General#

+
    +
  • Fix potential query failure due to a regression introduced in version 383. (#12667)

  • +
  • Fix potential failure or incorrect result of the json_query JSON function. (#12681)

  • +
+
+
+

Druid connector#

+
    +
  • Add query table function for full query pass-through to the connector. (#9163)

  • +
+
+
+

MariaDB connector#

+
    +
  • Add query table function for full query pass-through to the connector. (#9163)

  • +
+
+
+

MySQL connector#

+
    +
  • Add query table function for full query pass-through to the connector. (#9163)

  • +
+
+
+

Oracle connector#

+
    +
  • Add query table function for full query pass-through to the connector. (#9163)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Add query table function for full query pass-through to the connector. (#9163)

  • +
+
+
+

Redshift connector#

+
    +
  • Add query table function for full query pass-through to the connector. (#9163)

  • +
+
+
+

SingleStore (MemSQL) connector#

+
    +
  • Add query table function for full query pass-through to the connector. (#9163)

  • +
+
+
+

SQL Server connector#

+
    +
  • Add query table function for full query pass-through to the connector. (#9163)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-385.html b/430/release/release-385.html new file mode 100644 index 000000000..a882caace --- /dev/null +++ b/430/release/release-385.html @@ -0,0 +1,2613 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 385 (8 Jun 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+ +
+ +
+
+ +
+

Release 385 (8 Jun 2022)#

+
+

General#

+
    +
  • Add the json_array and json_object JSON functions. (#9081)

  • +
  • Support all types that can be cast to varchar as parameters for the JSON +path. (#12682)

  • +
  • Allow CREATE TABLE LIKE clause on a table from a different catalog if +explicitly excluding table properties. (#3171)

  • +
  • Reduce Exceeded limit of N open writers for partitions errors when +fault-tolerant execution is enabled. (#12721)

  • +
+
+
+

Delta Lake connector#

+ +
+
+

Hive connector#

+
    +
  • Allow cancelling a query on a transactional table if it is waiting for a lock. (#11798)

  • +
  • Add support for selecting a compression scheme when writing Avro files via the +hive.compression-codec config property or the compression_codec session +property. (#12639)

  • +
+
+
+

Iceberg connector#

+
    +
  • Improve query performance when a table consists of many small files. (#12579)

  • +
  • Improve query performance when performing a delete or update. (#12671)

  • +
  • Add support for the [VERSION | TIMESTAMP] AS OF clause. (#10258)

  • +
  • Show Iceberg location and format_version in SHOW CREATE MATERIALIZED VIEW. (#12504)

  • +
+
+
+

MariaDB connector#

+
    +
  • Add support for timestamp(p) type. (#12200)

  • +
+
+
+

TPC-H connector#

+
    +
  • Fix query failure when reading the dbgen_version table. (#12673)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-386.html b/430/release/release-386.html new file mode 100644 index 000000000..83c54cd1b --- /dev/null +++ b/430/release/release-386.html @@ -0,0 +1,2659 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 386 (15 Jun 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 386 (15 Jun 2022)#

+
+

General#

+
    +
  • Improve out-of-the-box performance of queries when task retry policy is +enabled. (#12646)

  • +
  • Improve query latency when task retry policy is enabled. (#12615)

  • +
+
+
+

JDBC driver#

+
    +
  • Add configuration property +assumeLiteralUnderscoreInMetadataCallsForNonConformingClients for situations +where applications do not properly escape schema or table names in calls to +DatabaseMetaData. (#12672)

  • +
+
+
+

Accumulo connector#

+
    +
  • Disallow creating a view in a non-existent schema. (#12475)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Improve query performance on tables with many small files. (#12755)

  • +
  • Disallow reading tables if delta.columnMapping.mode table property is +specified. (#12621)

  • +
  • Set a target maximum file size during table writes. The default is 1 GB and +can be configured with the target_max_file_size session property or the +target-max-file-size configuration property. (#12820)

  • +
+
+
+

Hive connector#

+
    +
  • Fix incompatibility with Apache Hive when writing decimal values with +precision of 18 or less with the experimental Parquet writer. (#12658)

  • +
  • Fix potential query failure when using schema evolution with union-typed +columns. (#12520)

  • +
  • Fix potential query failure when reading timestamp(6) with time zone values. (#12804)

  • +
+
+
+

Iceberg connector#

+
    +
  • Disallow creating a table with a pre-existing destination location. (#12573)

  • +
  • Fix NoClassDefFoundError query failure when using Google Cloud Storage. (#12674)

  • +
  • Fix ClassNotFoundException: Class io.trino.plugin.hive.s3.TrinoS3FileSystem +error when querying information_schema.columns. (#12676)

  • +
  • Avoid creating a table snapshot when a write statement does not change the +table state. (#12319)

  • +
  • Fix incorrect query results when filtering on $path synthetic column and on +at least one other column. (#12790)

  • +
  • Fix potential query failure when reading timestamp(6) with time zone values. (#12804)

  • +
  • Fix query failure when using the [VERSION | TIMESTAMP] AS OF clause on a +table with redirection. (#12542)

  • +
  • Fix query failure when reading a timestamp(p) with time zone value before +1970 from a Parquet file. (#12852)

  • +
+
+
+

Kafka connector#

+
    +
  • Fix failure when decoding a float value to real type. (#12784)

  • +
+
+
+

Phoenix connector#

+
    +
  • Remove support for Phoenix 4. (#12772)

  • +
+
+
+

SPI#

+
    +
  • Add new version of getStatisticsCollectionMetadata() to ConnectorMetadata +which returns ConnectorAnalyzeMetadata. Deprecate the existing method and +getTableHandleForStatisticsCollection(). (#12388)

  • +
  • Remove deprecated ConnectorMetadata.getTableStatistics method. (#12489)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-387.html b/430/release/release-387.html new file mode 100644 index 000000000..fb55c6157 --- /dev/null +++ b/430/release/release-387.html @@ -0,0 +1,2649 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 387 (22 Jun 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+ +
+
+ +
+
+ +
+

Release 387 (22 Jun 2022)#

+
+

General#

+
    +
  • Add support for query parameters in table function arguments. (#12910)

  • +
  • Update minimum required Java version to 11.0.15. (#12841)

  • +
  • Fix incorrect result for to_iso8601() when the timestamp is in the +daylight savings transition region. (#11619)

  • +
+
+
+

CLI#

+
    +
  • Fix query history not being stored when a query starts with whitespace. (#12847)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Record table size when analyzing a table. (#12814)

  • +
  • Enable the optimized Parquet writer by default. This can be disabled via the +parquet.experimental-optimized-writer.enabled configuration property. (#12757)

  • +
  • Disallow adding a new column to a table that has been written with an +unsupported writer. (#12883)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for ORC bloom filters on varchar columns. (#11757)

  • +
+
+
+

Iceberg connector#

+
    +
  • Allow OPTIMIZE on a table partitioned on a timestamp with time zone column +when using CAST(timestamp_col AS date) >= DATE '...' syntax. (#12362)

  • +
  • Allow OPTIMIZE with a predicate on a table that does not have identity +partitioning. (#12795)

  • +
  • Improve performance of DELETE when deleting whole partitions from a table +that does not have identity partitioning. (#7905)

  • +
  • Fix incorrect results when a query contains a filter on a UUID column. (#12834)

  • +
  • Fail queries that attempt to modify old snapshots. (#12860)

  • +
  • Deprecate using synthetic @-based syntax for Iceberg snapshot access in +favor of the AS OF syntax. The old behavior can be restored by setting the +allow_legacy_snapshot_syntax session property or +iceberg.allow-legacy-snapshot-syntax configuration property. (#10768)

  • +
+
+
+

Kudu connector#

+
    +
  • Fix failure when inserting into a table with a row_uuid column. (#12915)

  • +
+
+
+

Pinot connector#

+
    +
  • Add support for querying Pinot via the gRPC endpoint. (#9296)

  • +
+
+
+

Redis connector#

+
    +
  • Add support for predicate pushdown on columns of type string. (#12218)

  • +
+
+
+

SPI#

+
    +
  • Add information about query retry policy to QueryCompletedEvent and +QueryCreatedEvent. (#12898)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-388.html b/430/release/release-388.html new file mode 100644 index 000000000..e2cea6e03 --- /dev/null +++ b/430/release/release-388.html @@ -0,0 +1,2623 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 388 (29 Jun 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+ +
+ +
+
+ +
+

Release 388 (29 Jun 2022)#

+
+

General#

+
    +
  • Add support for EXPLAIN (TYPE LOGICAL, FORMAT JSON). (#12694)

  • +
  • Add use_exact_partitioning session property to re-partition data when the +upstream stage’s partitioning does not exactly match what the downstream stage +expects. (#12495)

  • +
  • Improve read performance for row data types. (#12926)

  • +
  • Remove the grouped execution mechanism, including the +grouped-execution-enabled, dynamic-schedule-for-grouped-execution, +and concurrent-lifespans-per-task configuration properties and the +grouped_execution, dynamic_schedule_for_grouped_execution, and +concurrent_lifespans_per_task session properties. (#12916)

  • +
+
+
+

Security#

+ +
+
+

Delta Lake connector#

+
    +
  • Add support for setting table and column comments with the COMMENT +statement. (#12971)

  • +
  • Support reading tables with the property delta.columnMapping.mode=name. (#12675)

  • +
  • Allow renaming tables with an explicitly set location. (#11400)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Remove support for Elasticsearch versions below 6.6.0. (#11263)

  • +
+
+
+

Hive connector#

+
    +
  • Improve performance of listing files and generating splits when recursive +directory listings are enabled and tables are stored in S3. (#12443)

  • +
  • Fix incompatibility that prevents Apache Hive 3 and older from reading +timestamp columns in files produced by Trino’s optimized Parquet +writer. (#12857)

  • +
  • Prevent reading from a table that was modified within the same Trino +transaction. Previously, this returned incorrect query results. (#11769)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add support for reading tinyint columns from ORC files. (#8919)

  • +
  • Add the ability to configure the schema for materialized view storage tables. (#12591)

  • +
  • Remove old deletion-tracking files when running optimize. (#12617)

  • +
  • Fix failure when invoking the rollback_to_snapshot procedure. (#12887)

  • +
  • Fix query failure when reading the $partitions table after table +partitioning changed. (#12874)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-389.html b/430/release/release-389.html new file mode 100644 index 000000000..da482b878 --- /dev/null +++ b/430/release/release-389.html @@ -0,0 +1,2646 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 389 (7 Jul 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 389 (7 Jul 2022)#

+
+

General#

+
    +
  • Improve performance of queries involving row type or certain aggregations +such as sum, avg, etc. (#12762)

  • +
  • Improve performance when spilling to disk is disabled. (#12618)

  • +
  • Fix potential incorrect results for queries involving non-equality filters on +top of an outer join. (#13109)

  • +
  • Fix query failure when no arguments are passed to a table function. (#12951)

  • +
  • Fix potential failure when using EXPLAIN with queries involving table +functions. (#13106)

  • +
  • Fix potential resource leaks when calling long-running regular expression +functions. (#13064)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Improve optimized Parquet writer performance for +non-structural data types. (#13030)

  • +
  • Prevent failure when starting the server if the internal table snapshots cache +is disabled. (#13086)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Add raw_query table function for full query pass-through to the connector. (#12324)

  • +
+
+
+

Hive connector#

+ +
+
+

Iceberg connector#

+ +
+
+

MongoDB connector#

+
    +
  • Create a collection when creating a new table. Previously, it was created when +the data was written to the table for the first time. (#12892)

  • +
+
+
+

Phoenix connector#

+
    +
  • Add support for Java 17. (#13108)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Prevent creating a new table with a name longer than the max length. +Previously, the name was truncated to the max length. (#12892)

  • +
+
+
+

SPI#

+
    +
  • Remove deprecated version of ConnectorRecordSetProvider#getRecordSet. (#13084)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-390.html b/430/release/release-390.html new file mode 100644 index 000000000..9e626529c --- /dev/null +++ b/430/release/release-390.html @@ -0,0 +1,2635 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 390 (13 Jul 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 390 (13 Jul 2022)#

+
+

General#

+
    +
  • Update minimum required Java version to 17.0.3. (#13014)

  • +
  • Add support for setting comments on views. (#8349)

  • +
  • Improve performance of queries with an UNNEST clause. (#10506)

  • +
  • Fix potential query failure when spilling to disk is enabled by the +force-spilling-join-operator configuration property or the +force_spilling_join session property. (#13123)

  • +
  • Fix incorrect results for certain join queries containing filters involving +explicit or implicit casts. (#13145)

  • +
+
+
+

Cassandra connector#

+
    +
  • Change mapping for Cassandra inet type to Trino ipaddress type. +Previously, inet was mapped to varchar. (#851)

  • +
  • Remove support for the +cassandra.load-policy.use-token-aware, +cassandra.load-policy.shuffle-replicas, and +cassandra.load-policy.allowed-addresses configuration properties. (#12223)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Add support for filtering splits based on $path column predicates. (#13169)

  • +
  • Add support for Databricks runtime 10.4 LTS. (#13081)

  • +
  • Expose AWS Glue metastore statistics via JMX. (#13087)

  • +
  • Fix failure when using the Glue metastore and queries contain IS NULL or +IS NOT NULL filters on numeric partition columns. (#13124)

  • +
+
+
+

Hive connector#

+
    +
  • Expose AWS Glue metastore statistics via JMX. (#13087)

  • +
  • Add support for setting comments on views. (#13147)

  • +
  • Fix failure when using the Glue metastore and queries contain IS NULL or +IS NOT NULL filters on numeric partition columns. (#13124)

  • +
  • Fix and re-enable usage of Amazon S3 Select for uncompressed files. (#12633)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add added_rows_count, existing_rows_count, and deleted_rows_count +columns to the $manifests table. (#10809)

  • +
  • Add support for setting comments on views. (#13147)

  • +
  • Expose AWS Glue metastore statistics via JMX. (#13087)

  • +
  • Fix failure when using the Glue metastore and queries contain IS NULL or +IS NOT NULL filters on numeric partition columns. (#13124)

  • +
+
+
+

Memory connector#

+ +
+
+

Prometheus connector#

+
    +
  • Fix failure when reading a table without specifying a labels column. (#12510)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-391.html b/430/release/release-391.html new file mode 100644 index 000000000..a33757164 --- /dev/null +++ b/430/release/release-391.html @@ -0,0 +1,2699 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 391 (22 Jul 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 391 (22 Jul 2022)#

+
+

General#

+
    +
  • Improve performance of repeated aggregations with CASE expressions. (#12548)

  • +
  • Improve query latency when there is high concurrency. (#13213)

  • +
  • Improve planning performance for join queries when tables have statistics. (#13047)

  • +
  • Fail queries that get stuck in long-running regular expression functions. (#12392)

  • +
  • Fix potential query failure when the UUID type is used. (#13265)

  • +
  • Set the default value of the optimizer.force-single-node-output +configuration property to false. (#13217)

  • +
+
+
+

BigQuery connector#

+
    +
  • Add support for reading external tables. (#13164)

  • +
  • Add support for specifying table and column comments when creating a table. (#13105)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Improve optimized Parquet writer performance. (#13203, #13208)

  • +
  • Store query ID when creating a new schema. (#13242)

  • +
  • Fix incorrect schema already exists error caused by a client timeout when +creating a new schema. (#13242)

  • +
  • Fix incorrect query results when reading a table with an outdated cached +representation of its active data files. (#13181)

  • +
+
+
+

Druid connector#

+
    +
  • Fix potential query failure when using the query table function with +metadata caching and the underlying table schema is changed via Trino. (#12526)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for AWS Athena partition projection. (#11305)

  • +
  • Improve optimized Parquet writer performance. (#13203, #13208)

  • +
  • Fix potential failure when creating empty ORC bucket files while using ZSTD +compression. (#9775)

  • +
  • Fix query failure or potentially incorrect statistics when running concurrent +CREATE TABLE AS queries with the IF NOT EXISTS clause for the same +non-existent table. (#12895)

  • +
  • Fix incorrect results when using the Glue metastore with queries that contain +IS NULL and additional filters. (#13122)

  • +
+
+
+

Iceberg connector#

+
    +
  • Improve performance when writing Parquet writer data. (#13203, #13208)

  • +
  • Fix query failure when reading an Iceberg table with deletion-tracking files. (#13035)

  • +
+
+
+

MariaDB connector#

+
    +
  • Fix potential query failure when using the query table function with +metadata caching and the underlying table schema is changed via Trino. (#12526)

  • +
+
+
+

MySQL connector#

+
    +
  • Fix potential query failure when using the query table function with +metadata caching and the underlying table schema is changed via Trino. (#12526)

  • +
+
+
+

Oracle connector#

+
    +
  • Fix potential query failure when using the query table function with +metadata caching and the underlying table schema is changed via Trino. (#12526)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Fix potential query failure when using the query table function with +metadata caching and the underlying table schema is changed via Trino. (#12526)

  • +
+
+
+

Redshift connector#

+
    +
  • Fix potential query failure when using the query table function with +metadata caching and the underlying table schema is changed via Trino. (#12526)

  • +
+
+
+

SQL Server connector#

+
    +
  • Fix potential query failure when using the query table function with +metadata caching and the underlying table schema is changed via Trino. (#12526)

  • +
+
+
+

SPI#

+
    +
  • Removed deprecated methods and classes related to the grouped execution +feature. (#13125)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-392.html b/430/release/release-392.html new file mode 100644 index 000000000..41ee78aaf --- /dev/null +++ b/430/release/release-392.html @@ -0,0 +1,2740 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 392 (3 Aug 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 392 (3 Aug 2022)#

+
+

General#

+
    +
  • Add support for dynamic filtering when task-based fault-tolerant execution is enabled. (#9935)

  • +
  • Add support for correlated sub-queries in DELETE queries. (#9447)

  • +
  • Fix potential query failure in certain complex queries with multiple joins and +aggregations. (#13315)

  • +
+
+
+

JDBC driver#

+
    +
  • Add the assumeLiteralUnderscoreInMetadataCallsForNonConformingClients +configuration property as a replacement for +assumeLiteralNamesInMetadataCallsForNonConformingClients, which is +deprecated and planned to be removed in a future release. (#12761)

  • +
+
+
+

ClickHouse connector#

+
    +
  • Report the total time spent reading data from the data source. (#13132)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Add support for using a randomized location when creating a table, so that +future table renames or drops do not interfere with new tables created with +the same name. This can be disabled by setting the +delta.unique-table-location configuration property to false. (#12980)

  • +
  • Add delta.metadata.live-files.cache-ttl configuration property for the +caching duration of active data files. (#13316)

  • +
  • Retain metadata properties and column metadata after schema changes. (#13368, #13418)

  • +
  • Prevent writing to a table with NOT NULL or +column invariants +columns. (#13353)

  • +
  • Fix incorrect min and max column statistics when writing NULL values. (#13389)

  • +
+
+
+

Druid connector#

+
    +
  • Add support for timestamp(p) predicate pushdown. (#8404)

  • +
  • Report the total time spent reading data from the data source. (#13132)

  • +
  • Change mapping for the Druid float type to the Trino real type instead of +the double type. (#13412)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for short timezone IDs when translating Hive views. For example, +JST now works as an alias for Asia/Tokyo. (#13179)

  • +
  • Add support for Amazon S3 Select pushdown for JSON files. (#13354)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add support for hidden $file_modified_time columns. (#13082)

  • +
  • Add support for the Avro file format. (#12125)

  • +
  • Add support for filtering splits based on $path column predicates. (#12785)

  • +
  • Improve query performance for tables with updated or deleted rows. (#13092)

  • +
  • Improve performance of the expire_snapshots command for tables with many +snapshots. (#13399)

  • +
  • Use unique table locations by default. This can be disabled by setting the +iceberg.unique-table-location configuration property to false. (#12941)

  • +
  • Use the correct table schema when reading a past version of a table. (#12786)

  • +
  • Return the $path column without encoding when the path contains double +slashes on S3. (#13012)

  • +
  • Fix failure when inserting into a Parquet table with columns that have +quotation marks in their names. (#13074)

  • +
+
+
+

MariaDB connector#

+
    +
  • Report the total time spent reading data from the data source. (#13132)

  • +
+
+
+

MySQL connector#

+
    +
  • Report the total time spent reading data from the data source. (#13132)

  • +
  • Change mapping for the MySQL enum type to the Trino varchar type instead +of the char type. (#13303)

  • +
  • Fix failure when reading table statistics while the +information_schema.column_statistics table doesn’t exist. (#13323)

  • +
+
+
+

Oracle connector#

+
    +
  • Report the total time spent reading data from the data source. (#13132)

  • +
+
+
+

Phoenix connector#

+
    +
  • Report the total time spent reading data from the data source. (#13132)

  • +
+
+
+

Pinot connector#

+
    +
  • Redact the values of pinot.grpc.tls.keystore-password and +pinot.grpc.tls.truststore-password in the server log. (#13422)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Report the total time spent reading data from the data source. (#13132)

  • +
  • Improve performance of queries with an IN expression within a complex +expression. (#13136)

  • +
+
+
+

Redshift connector#

+
    +
  • Report the total time spent reading data from the data source. (#13132)

  • +
+
+
+

SingleStore (MemSQL) connector#

+
    +
  • Report the total time spent reading data from the data source. (#13132)

  • +
+
+
+

SQL Server connector#

+
    +
  • Report the total time spent reading data from the data source. (#13132)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-393.html b/430/release/release-393.html new file mode 100644 index 000000000..2d6a5f67d --- /dev/null +++ b/430/release/release-393.html @@ -0,0 +1,2810 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 393 (17 Aug 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 393 (17 Aug 2022)#

+
+

General#

+
    +
  • Add support for MERGE. (#7933)

  • +
  • Add query state and error code to the timeline log message. (#13698)

  • +
  • Improve performance of highly selective LIMIT queries by making them finish +as soon as the required number of rows is produced. (#13616)

  • +
  • Disallow inserting NULL into non-nullable columns.. (#13462)

  • +
  • Fix queries over bucketed tables never completing. (#13655)

  • +
  • Fix potential query failure for GROUP BY queries involving aggregations with +ORDER BY and FILTER clauses. (#13627)

  • +
+
+
+

Security#

+
    +
  • Fix potential incorrect rejection of OAuth 2.0 refresh tokens. (#13168)

  • +
  • Fix incorrectly showing outdated OAuth 2.0 refresh tokens in the web UI. (#13168)

  • +
+
+
+

Docker image#

+
    +
  • Add experimental image for ppc64le. (#13522)

  • +
+
+
+

ClickHouse connector#

+
    +
  • Add support for the unsupported-type-handling catalog configuration +property. (#13542)

  • +
  • Improve performance for queries with selective joins. (#13334)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Add support for MERGE. (#7933)

  • +
  • Add support for the NOT NULL column constraint. (#13436)

  • +
  • Fix writing incorrect results when the order of partition columns is different +from the order in the table definition. (#13505)

  • +
  • Fix failure when reading a table which has partition columns renamed by +another engine. (#13521)

  • +
+
+
+

Druid connector#

+
    +
  • Improve performance for queries with selective joins. (#13334)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for MERGE. (#7933)

  • +
  • Add support for bucket filtering on bucketed columns of float, double, +date, list, map and bounded varchar data types. (#13553)

  • +
  • Add exchange.azure.max-error-retries configuration property for the number +of retries performed when accessing Azure blob storage. (#13663)

  • +
  • Improve performance of queries with S3 Select pushdown by not utilizing +pushdown when it is unnecessary. (#13477)

  • +
  • Reduce Thrift metastore communication overhead when impersonation is enabled. (#13606)

  • +
  • Improve performance when retrieving table statistics from the metastore. (#13488)

  • +
  • Fix error when writing to a table with only date columns while using the +Hive metastore. (#13502)

  • +
  • Fix error when reading a Hive view which has a column names with a reserved +keyword. (#13450)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add support for MERGE. (#7933)

  • +
  • Improve performance when filtering on $file_modified_time column. (#13504)

  • +
  • Improve performance of read queries on Iceberg v2 tables with +deletion-tracking files. (#13395)

  • +
  • Allow partitioning over columns which use whitespace in their names. (#12226)

  • +
  • Disallow specifying a NOT NULL constraint when adding a new column. +Previously, the option was ignored. (#13673)

  • +
  • Fix error when querying tables which are empty and contain no table history. (#13576)

  • +
  • Prevent truncation of the table history in the $snapshots system table by +certain DELETE queries. (#12843)

  • +
  • Prevent errors when optimizing an Iceberg table which is empty and contains +no table history. (#13582)

  • +
  • Fix incorrect query results when reading from a materialized view that was +created on a table which was empty and contained no history. (#13574)

  • +
+
+
+

Kafka connector#

+
    +
  • Fix query failure when applying a negative timestamp predicate on the +_timestamp column. (#13167)

  • +
+
+
+

Kudu connector#

+ +
+
+

MariaDB connector#

+
    +
  • Improve performance for queries with selective joins. (#13334)

  • +
+
+
+

MongoDB connector#

+
    +
  • Prevent renaming a table with a name longer than the max length supported by +MongoDB. Previously, the name was truncated to the max length. (#13073)

  • +
+
+
+

MySQL connector#

+
    +
  • Improve performance for queries with selective joins. (#13334)

  • +
+
+
+

Oracle connector#

+
    +
  • Improve performance for queries with selective joins. (#13334)

  • +
+
+
+

Phoenix connector#

+
    +
  • Improve performance for queries with selective joins. (#13334)

  • +
+
+
+

Pinot connector#

+
    +
  • Add support for the Pinot bytes type. (#13427)

  • +
  • Add support for the json type. (#13428)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Improve performance for queries with selective joins. (#13334)

  • +
  • Prevent using schema names or renaming a table with a name which is longer +than the max length supported by PostgreSQL. Previously, long names were +truncated to the max length. (#13307, #13073)

  • +
+
+
+

Raptor connector#

+ +
+
+

Redshift connector#

+
    +
  • Improve performance for queries with selective joins. (#13334)

  • +
+
+
+

SingleStore (MemSQL) connector#

+
    +
  • Improve performance for queries with selective joins. (#13334)

  • +
+
+
+

SQL Server connector#

+
    +
  • Improve performance for queries with selective joins. (#13334)

  • +
  • Prevent renaming a table with a name longer than the max length supported by +SQL Server. Previously, the name was truncated to the max length. (#13073)

  • +
+
+
+

SPI#

+
    +
  • Add @Experimental annotation to designate SPIs that are still under active +development. (#13302)

  • +
  • Deprecate io.trino.spi.block.MethodHandleUtil. (#13245)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-394.html b/430/release/release-394.html new file mode 100644 index 000000000..58bcd5dc3 --- /dev/null +++ b/430/release/release-394.html @@ -0,0 +1,2711 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 394 (29 Aug 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 394 (29 Aug 2022)#

+
+

General#

+
    +
  • Add support for JSON as an output format of EXPLAIN. (#12968)

  • +
  • Improve performance of queries involving joins on a single bigint column. (#13432)

  • +
  • Improve performance of LIKE expressions. (#13479)

  • +
  • Ensure UPDATE queries cannot insert null values into columns with a +NOT NULL constraint. (#13435)

  • +
  • Fix failure when an UPDATE query contains a WHERE clause which always +evaluates to false. (#12422)

  • +
  • Fix potential failure for queries involving joins and implicit or explicit +casts of null to a concrete type. (#13565)

  • +
+
+
+

Docker image#

+
    +
  • Ensure Trino stops running with insufficient resources to avoid partial +failures. (#13736)

  • +
+
+
+

BigQuery connector#

+
    +
  • Add query table function for full query pass-through to the connector. (#12502)

  • +
  • Add support for the INSERT statement. (#6868)

  • +
  • Add support for the CREATE TABLE ... AS SELECT ... statement. (#6869)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Disallow adding a column with a NOT NULL constraint to a table which is not +empty. (#13785)

  • +
  • Fix failure when reading Parquet data that contains only null values. (#9424)

  • +
  • Fix potential failure of unrelated queries after dropping a schema. (#13810)

  • +
+
+
+

Druid connector#

+
    +
  • Improve performance of queries that perform filtering on varchar columns that +contain temporal data with the format YYYY-MM-DD. (#12925)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Add support for multiple hosts in the elasticsearch.host configuration +property. (#12530)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for a Kerberos credential cache. (#13482)

  • +
  • Fix failure when reading Parquet data that contains only null values. (#9424)

  • +
  • Fix failure when the metastore returns duplicated column statistics. (#13787)

  • +
  • Fix potential failure of unrelated queries after dropping a schema. (#13810)

  • +
+
+
+

Iceberg connector#

+
    +
  • Improve query planning performance when a varchar partitioning column +contains date values in the YYYY-MM-DD format. (#12925)

  • +
  • Fix query failure when using the [VERSION | TIMESTAMP] AS OF clause on a +table created with Iceberg versions older than 0.12. (#13613)

  • +
  • Fix failure when reading Parquet data that contains only null values. (#9424)

  • +
+
+
+

Oracle connector#

+
    +
  • Improve performance of queries that perform filtering on varchar columns that +contain temporal data with the format YYYY-MM-DD. (#12925)

  • +
+
+
+

Phoenix connector#

+
    +
  • Improve performance of queries that perform filtering on varchar columns that +contain temporal data with the format YYYY-MM-DD. (#12925)

  • +
+
+
+

Pinot connector#

+
    +
  • Add support for TLS when connecting to the Pinot controllers and brokers. (#13410)

  • +
  • Fix query failure when using the HAVING clause. (#13429)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Improve performance of queries that perform filtering on varchar columns +that contain temporal data with the format YYYY-MM-DD. (#12925)

  • +
  • Prevent using a column name which is longer than the maximum length supported +by PostgreSQL. Previously, long names were truncated. (#13742)

  • +
+
+
+

SQL Server connector#

+
    +
  • Prevent renaming a column to a name which is longer than the maximum length +supported by SQL Server. Previously, long names were truncated. (#13742)

  • +
+
+
+

SPI#

+
    +
  • Add the query plan in JSON format to QueryCompletedEvent, and allow +connectors to request anonymized query plans in the QueryCompletedEvent. (#12968)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-395.html b/430/release/release-395.html new file mode 100644 index 000000000..1d6d090fc --- /dev/null +++ b/430/release/release-395.html @@ -0,0 +1,2743 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 395 (7 Sep 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 395 (7 Sep 2022)#

+
+

General#

+
    +
  • Reduce memory consumption when fault-tolerant execution is enabled. (#13855)

  • +
  • Reduce memory consumption of aggregations. (#12512)

  • +
  • Improve performance of aggregations with decimals. (#13573)

  • +
  • Improve concurrency for large clusters. (#13934, 13986)

  • +
  • Remove information_schema.role_authorization_descriptors table. (#11341)

  • +
  • Fix SHOW CREATE TABLE or SHOW COLUMNS showing an invalid type for columns +that use a reserved keyword as column name. (#13483)

  • +
+
+
+

ClickHouse connector#

+
    +
  • Fix query failure when renaming or dropping a column with a name which matches +a reserved keyword or has special characters which require it to be quoted. (#13839)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Add support for the ALTER TABLE ... RENAME TO statement with a Glue +metastore. (#12985)

  • +
  • Improve performance of inserts by automatically scaling the number of writers +within a worker node. (#13111)

  • +
  • Enforce delta.checkpoint.writeStatsAsJson and +delta.checkpoint.writeStatsAsStruct table properties to ensure table +statistics are written in the correct format. (#12031)

  • +
+
+
+

Hive connector#

+
    +
  • Improve performance of inserts by automatically scaling the number of writers +within a worker node. (#13111)

  • +
  • Improve performance of S3 Select when using CSV files as an input. (#13754)

  • +
  • Fix error where the S3 KMS key is not searched in the proper AWS region when +S3 client-side encryption is used. (#13715)

  • +
+
+
+

Iceberg connector#

+
    +
  • Improve performance of inserts by automatically scaling the number of writers +within a worker node. (#13111)

  • +
  • Fix creating metadata and manifest files with a URL-encoded name on S3 when +the metadata location has trailing slashes. (#13759)

  • +
+
+
+

MariaDB connector#

+
    +
  • Fix query failure when renaming or dropping a column with a name which matches +a reserved keyword or has special characters which require it to be quoted. (#13839)

  • +
+
+
+

Memory connector#

+
    +
  • Add support for table and column comments. (#13936)

  • +
+
+
+

MongoDB connector#

+
    +
  • Fix query failure when filtering on columns of json type. (#13536)

  • +
+
+
+

MySQL connector#

+
    +
  • Fix query failure when renaming or dropping a column with a name which matches +a reserved keyword or has special characters which require it to be quoted. (#13839)

  • +
+
+
+

Oracle connector#

+
    +
  • Fix query failure when renaming or dropping a column with a name which matches +a reserved keyword or has special characters which require it to be quoted. (#13839)

  • +
+
+
+

Phoenix connector#

+
    +
  • Fix query failure when adding, renaming, or dropping a column with a name +which matches a reserved keyword or has special characters which require it to +be quoted. (#13839)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Fix query failure when renaming or dropping a column with a name which matches +a reserved keyword or has special characters which require it to be quoted. (#13839)

  • +
+
+
+

Prometheus connector#

+
    +
  • Add support for case-insensitive table name matching with the +prometheus.case-insensitive-name-matching configuration property. (#8740)

  • +
+
+
+

Redshift connector#

+
    +
  • Fix query failure when renaming or dropping a column with a name which matches +a reserved keyword or has special characters which require it to be quoted. (#13839)

  • +
+
+
+

SingleStore (MemSQL) connector#

+
    +
  • Fix query failure when renaming or dropping a column with a name which matches +a reserved keyword or has special characters which require it to be quoted. (#13839)

  • +
+
+
+

SQL Server connector#

+
    +
  • Fix query failure when renaming or dropping a column with a name which matches +a reserved keyword or has special characters which require it to be quoted. (#13839)

  • +
+
+
+

SPI#

+
    +
  • Add support for dynamic function resolution. (#8)

  • +
  • Rename LIKE_PATTERN_FUNCTION_NAME to LIKE_FUNCTION_NAME in +StandardFunctions. (#13965)

  • +
  • Remove the listAllRoleGrants method from ConnectorMetadata. (#11341)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-396.html b/430/release/release-396.html new file mode 100644 index 000000000..a551f0576 --- /dev/null +++ b/430/release/release-396.html @@ -0,0 +1,2706 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 396 (15 Sep 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 396 (15 Sep 2022)#

+
+

General#

+
    +
  • Improve performance of queries that process string data. (#12798)

  • +
  • Fix failure when querying views that use table functions. (#13944)

  • +
+
+
+

BigQuery connector#

+ +
+
+

ClickHouse connector#

+
    +
  • Improve performance when pushing down large lists of predicates by raising +the default threshold before the predicate is compacted. (#14029)

  • +
  • Fix mapping to the ClickHouse Date and DateTime types to include the full +range of possible values. (#11116)

  • +
  • Fix failure when specifying a table or column comment that contains special +characters. (#14058)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Add support for writing to tables using version 3 of the +writer. +This does not yet include support for CHECK constraints. (#14068)

  • +
  • Add support for reading tables with the table property +delta.columnMapping.mode set to id. (#13629)

  • +
  • Improve performance when writing +structural data types to Parquet files. (#13714)

  • +
  • Attempt to undo the operation when writing a checkpoint file fails. (#14108)

  • +
  • Fix performance regression when reading Parquet data. (#14094)

  • +
+
+
+

Hive connector#

+ +
+
+

Iceberg connector#

+
    +
  • Improve performance when writing +structural data types to Parquet files. (#13714)

  • +
  • Improve performance of queries that contain predicates involving date_trunc +on date, timestamp or timestamp with time zone partition columns. (#14011)

  • +
  • Fix incorrect results from using the [VERSION | TIMESTAMP] AS OF clause when +the snapshot’s schema differs from the current schema of the table. (#14064)

  • +
  • Prevent No bucket node map failures when inserting data. (#13960)

  • +
  • Fix performance regression when reading Parquet data introduced in +Trino version 394. (#14094)

  • +
+
+
+

MariaDB connector#

+
    +
  • Fix failure when using special characters in a table or column comment when +creating a table. (#14058)

  • +
+
+
+

MySQL connector#

+
    +
  • Fix failure when using special characters in a table or column comment when +creating a table. (#14058)

  • +
+
+
+

Oracle connector#

+
    +
  • Fix failure when setting a column comment with special characters. (#14058)

  • +
+
+
+

Phoenix connector#

+
    +
  • Improve performance when pushing down large lists of predicates by raising +the default threshold before the predicate is compacted. (#14029)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Fix failure when setting a column comment with special characters. (#14058)

  • +
+
+
+

Redshift connector#

+
    +
  • Fix failure when setting a column comment with special characters. (#14058)

  • +
+
+
+

SPI#

+
    +
  • Add the SystemAccessControl.checkCanGrantExecuteFunctionPrivilege overload, +which needs to be implemented to allow views that use table functions. (#13944)

  • +
  • Add the ConnectorMetadata.applyJoin overload. It provides the connector +with a join condition that is as complete as possible to represent using +ConnectorExpression. Deprecate the previous version of +ConnectorMetadata.applyJoin. (#13943)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-397.html b/430/release/release-397.html new file mode 100644 index 000000000..bb1772738 --- /dev/null +++ b/430/release/release-397.html @@ -0,0 +1,2638 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 397 (21 Sep 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 397 (21 Sep 2022)#

+
+

General#

+
    +
  • Fix incorrect parsing of invalid values in cast from varchar to timestamp. (#14164)

  • +
  • Fix potential incorrect results for queries with a partitioned output which +doesn’t depend on any column data. (#14168)

  • +
  • Fix EXPLAIN (TYPE IO) query failure for certain queries on empty tables. (#10398)

  • +
+
+
+

Security#

+
    +
  • Add support for multiple recipients with JWT authentication. (#13442)

  • +
  • Fix OAuth 2.0 token refresh causing JWT authentication failure. (#13575)

  • +
+
+
+

JDBC driver#

+
    +
  • Fix potential memory leak when cancelling statements. (#14176)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Rename the parquet.experimental-optimized-writer.enabled configuration +property and experimental_parquet_optimized_writer_enabled session property +to parquet.optimized-writer.enabled and parquet_optimized_writer_enabled, +respectively. (#14137)

  • +
+
+
+

Hive connector#

+
    +
  • Rename the parquet.experimental-optimized-writer.enabled configuration +property and experimental_parquet_optimized_writer_enabled session property +to parquet.optimized-writer.enabled and parquet_optimized_writer_enabled, +respectively. (#14137)

  • +
  • Improve performance when querying JSON data and Hive S3 Select pushdown is +enabled. (#14040)

  • +
  • Improve planning performance when querying tables in the Glue catalog that +contain a large number of columns. (#14206)

  • +
  • Allow reading from a partitioned table after a column’s data type was changed +from decimal to varchar or string. (#2817)

  • +
  • Fix query failure when reading from a Hive view and +hive.hive-views.run-as-invoker and hive.hive-views.legacy-translation are +both enabled. (#14077)

  • +
+
+
+

Iceberg connector#

+
    +
  • Improve performance of queries that contain predicates involving date_trunc +with an hour unit on date, timestamp, or timestamp with time zone +partition columns. (#14161)

  • +
  • Improve performance of reads after a DELETE removes all rows from a file. (#14198)

  • +
  • Reduce query latency when using a Glue catalog for metadata. (#13875)

  • +
  • Fix materialized views temporarily appearing empty when a refresh is about to +complete. (#14145)

  • +
  • Fix potential table corruption when changing a table before it is known if +committing to the Hive metastore has failed or succeeded. (#14174)

  • +
+
+
+

SPI#

+
    +
  • Replace DictionaryBlock constructors with a factory method. (#14092)

  • +
  • Replace RunLengthEncodedBlock constructors with a factory method. (#14092)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-398.html b/430/release/release-398.html new file mode 100644 index 000000000..3a0cf2ddd --- /dev/null +++ b/430/release/release-398.html @@ -0,0 +1,2661 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 398 (28 Sep 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 398 (28 Sep 2022)#

+
+

General#

+
    +
  • Add Hudi connector. (#10228)

  • +
  • Add metrics for the execution time of filters and projections to EXPLAIN ANALYZE VERBOSE. (#14135)

  • +
  • Show local cost estimates when using EXPLAIN. (#14268)

  • +
  • Fix timeouts happening too early because of improper handling of the +node-scheduler.allowed-no-matching-node-period configuration property. (#14256)

  • +
  • Fix query failure for MERGE queries when task_writer_count is greater +than one. (#14306)

  • +
+
+
+

Accumulo connector#

+
    +
  • Add support for column comments when creating a new table. (#14114)

  • +
  • Move column mapping and index information into the output of DESCRIBE +instead of a comment. (#14095)

  • +
+
+
+

BigQuery connector#

+
    +
  • Fix improper escaping of backslash and newline characters. (#14254)

  • +
  • Fix query failure when the predicate involves a varchar value with a +backslash. (#14254)

  • +
+
+
+

ClickHouse connector#

+
    +
  • Upgrade minimum required Clickhouse version to 21.8. (#14112)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Improve performance when reading Parquet files for queries with predicates. (#14247)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Deprecate support for query pass-through using the special +<index>$query:<es-query> dynamic tables in favor of the raw_query table +function. Legacy behavior can be re-enabled with the +elasticsearch.legacy-pass-through-query.enabled configuration property. (#14015)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for partitioned views when legacy mode for view translation is +enabled. (#14028)

  • +
  • Extend the flush_metadata_cache procedure to be able to flush table-related +caches instead of only partition-related caches. (#14219)

  • +
  • Improve performance when reading Parquet files for queries with predicates. (#14247)

  • +
+
+
+

Iceberg connector#

+
    +
  • Improve performance when reading Parquet files for queries with predicates. (#14247)

  • +
  • Fix potential table corruption when changing a table before it is known if +committing to the Glue metastore has failed or succeeded. (#14174)

  • +
+
+
+

Pinot connector#

+
    +
  • Add support for the timestamp type. (#10199)

  • +
+
+
+

SPI#

+
    +
  • Extend ConnectorMetadata.getStatisticsCollectionMetadata to allow the +connector to request the computation of any aggregation function during stats +collection. (#14233)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-399.html b/430/release/release-399.html new file mode 100644 index 000000000..498f9b4dd --- /dev/null +++ b/430/release/release-399.html @@ -0,0 +1,2680 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 399 (6 Oct 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 399 (6 Oct 2022)#

+
+

General#

+
    +
  • Add operator CPU and wall time distribution to EXPLAIN ANALYZE VERBOSE. (#14370)

  • +
  • Improve performance of joins. (#13352)

  • +
  • Remove support for the deprecated row to json cast behavior, and remove the +deprecated.legacy-row-to-json-cast configuration property. (#14388)

  • +
  • Fix error when using PREPARE with DROP VIEW when the view name is quoted. (#14196)

  • +
  • Fix potential planning failure for queries involving UNION. (#14472)

  • +
  • Fix error when using aggregations in window expressions when the function +loaded from a plugin. (#14486)

  • +
+
+
+

Accumulo connector#

+
    +
  • Change the default value of the accumulo.zookeeper.metadata.root +configuration property to /trino-accumulo from /presto-accumulo. (#14326)

  • +
+
+
+

BigQuery connector#

+
    +
  • Add support for writing array, row, and timestamp columns. (#14418, #14473)

  • +
+
+
+

ClickHouse connector#

+
    +
  • Fix bug where the intended default value of the domain-compaction-threshold +configuration property was incorrectly used as a maximum limit. (#14350)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Improve performance of reading decimal columns from Parquet files. (#14260)

  • +
  • Allow setting the AWS Security Token Service endpoint and region when using a +Glue metastore. (#14412)

  • +
+
+
+

Hive connector#

+
    +
  • Add max-partition-drops-per-query configuration property to limit the number +of partition drops. (#12386)

  • +
  • Add hive.s3.region configuration property to force S3 to connect to a +specific region. (#14398)

  • +
  • Improve performance of reading decimal columns from Parquet files. (#14260)

  • +
  • Reduce memory usage on the coordinator. (#14408)

  • +
  • Reduce query memory usage during inserts to S3. (#14212)

  • +
  • Change the name of the partition_column and partition_value arguments for +the flush_metadata_cache procedure to partition_columns and +partition_values, respectively, for parity with other procedures. (#13566)

  • +
  • Change field name matching to be case insensitive. (#13423)

  • +
  • Allow setting the AWS STS endpoint and region when using a Glue metastore. (#14412)

  • +
+
+
+

Hudi connector#

+
    +
  • Fix failure when reading hidden columns. (#14341)

  • +
+
+
+

Iceberg connector#

+
    +
  • Improve performance of reading decimal columns from Parquet files. (#14260)

  • +
  • Reduce planning time for complex queries. (#14443)

  • +
  • Store metastore table_type property value in uppercase for compatibility +with other Iceberg catalog implementations. (#14384)

  • +
  • Allow setting the AWS STS endpoint and region when using a Glue metastore. (#14412)

  • +
+
+
+

Phoenix connector#

+
    +
  • Fix bug where the intended default value of the domain-compaction-threshold +configuration property was incorrectly used as a maximum limit. (#14350)

  • +
+
+
+

SQL Server connector#

+
    +
  • Fix error when querying or listing tables with names that contain special +characters. (#14286)

  • +
+
+
+

SPI#

+
    +
  • Add stage output buffer distribution to EventListener. (#14400)

  • +
  • Remove deprecated TimeType.TIME, TimestampType.TIMESTAMP and +TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE constants. (#14414)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-400.html b/430/release/release-400.html new file mode 100644 index 000000000..8c9eace81 --- /dev/null +++ b/430/release/release-400.html @@ -0,0 +1,2653 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 400 (13 Oct 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 400 (13 Oct 2022)#

+
+

General#

+
    +
  • Add output buffer utilization to EXPLAIN ANALYZE VERBOSE. (#14396)

  • +
  • Increase concurrency for large clusters. (#14395)

  • +
  • Fix JSON serialization failure for QueryCompletedEvent in event listener. +(#14604)

  • +
  • Fix occasional maximum pending connection acquisitions exceeded failure +when fault-tolerant execution is enabled. (#14580)

  • +
  • Fix incorrect results when calling the round function on large real and +double values. (#14613)

  • +
  • Fix query failure when using the merge(qdigest) function. (#14616)

  • +
+
+
+

BigQuery connector#

+ +
+
+

Delta Lake connector#

+
    +
  • Prevent coordinator out-of-memory failure when querying a large number of +tables in a short period of time. (#14571)

  • +
+
+
+

Hive connector#

+
    +
  • Reduce memory usage when scanning a large number of partitions, and add the +hive.max-partitions-for-eager-load configuration property to manage the +number of partitions that can be loaded into memory. (#14225)

  • +
  • Increase the default value of the hive.max-partitions-per-scan +configuration property to 1000000 from 100000. (#14225)

  • +
  • Utilize the hive.metastore.thrift.delete-files-on-drop configuration +property when dropping partitions and tables. Previously, it was only used +when dropping tables. (#13545)

  • +
+
+
+

Hudi connector#

+
    +
  • Hide Hive system schemas. (#14510)

  • +
+
+
+

Iceberg connector#

+
    +
  • Reduce query latency when querying tables with a large number of files. (#14504)

  • +
  • Prevent table corruption when changing a table fails due to an inability to +release the table lock from the Hive metastore. (#14386)

  • +
  • Fix query failure when reading from a table with a leading double slash in the +metadata location. (#14299)

  • +
+
+
+

Pinot connector#

+
    +
  • Add support for the Pinot proxy for controller/broker and server gRPC +requests. (#13015)

  • +
  • Update minimum required version to 0.10.0. (#14090)

  • +
+
+
+

SQL Server connector#

+
    +
  • Allow renaming column names containing special characters. (#14272)

  • +
+
+
+

SPI#

+
    +
  • Add ConnectorAccessControl.checkCanGrantExecuteFunctionPrivilege overload +which must be implemented to allow views that use table functions. (#13944)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-401.html b/430/release/release-401.html new file mode 100644 index 000000000..b9b3848ba --- /dev/null +++ b/430/release/release-401.html @@ -0,0 +1,2692 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 401 (26 Oct 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 401 (26 Oct 2022)#

+
+

General#

+
    +
  • Add support for using path-style access for all requests to S3 when using +fault-tolerant execution with exchange spooling. This can be enabled with the +exchange.s3.path-style-access configuration property. (#14655)

  • +
  • Add support for table functions in file-based access control. (#13713)

  • +
  • Add output buffer utilization distribution to EXPLAIN ANALYZE VERBOSE. (#14596)

  • +
  • Add operator blocked time distribution to EXPLAIN ANALYZE VERBOSE. (#14640)

  • +
  • Improve performance and reliability of INSERT and MERGE. (#14553)

  • +
  • Fix query failure caused by a +com.google.common.base.VerifyException: cannot unset noMoreSplits error. (#14668)

  • +
  • Fix underestimation of CPU usage and scheduled time statistics for joins in +EXPLAIN ANALYZE. (#14572)

  • +
+
+
+

Cassandra connector#

+
    +
  • Upgrade minimum required Cassandra version to 3.0. (#14562)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Add support for writing to tables with Delta Lake writer protocol version 4. +This does not yet include support for change data feeds +or generated columns. (#14573)

  • +
  • Add support for writes on Google Cloud Storage. (#12264)

  • +
  • Avoid overwriting the reader and writer versions when executing a COMMENT or +ALTER TABLE ... ADD COLUMN statement. (#14611)

  • +
  • Fix failure when listing tables from the Glue metastore and one of the tables +has no properties. (#14577)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for IBM Cloud Object Storage. (#14625)

  • +
  • Allow creating tables with an Avro schema literal using the new table property +avro_schema_literal. (#14426)

  • +
  • Fix potential query failure or incorrect results when reading from a table +with the avro.schema.literal Hive table property set. (#14426)

  • +
  • Fix failure when listing tables from the Glue metastore and one of the tables +has no properties. (#14577)

  • +
+
+
+

Iceberg connector#

+
    +
  • Improve performance of the remove_orphan_files table procedure. (#13691)

  • +
  • Fix query failure when analyzing a table that contains a column with a +non-lowercase name. (#14583)

  • +
  • Fix failure when listing tables from the Glue metastore and one of the tables +has no properties. (#14577)

  • +
+
+
+

Kafka connector#

+
    +
  • Add support for configuring the prefix for internal column names with the +kafka.internal-column-prefix catalog configuration property. The default +value is _ to maintain current behavior. (#14224)

  • +
+
+
+

MongoDB connector#

+
    +
  • Add query table function for query pass-through to the connector. (#14535)

  • +
+
+
+

MySQL connector#

+ +
+
+

Pinot connector#

+
    +
  • Fix failure when executing SHOW CREATE TABLE. (#14071)

  • +
+
+
+

PostgreSQL connector#

+ +
+
+

SQL Server connector#

+ +
+
+

SPI#

+
    +
  • Add stage output buffer distribution to EventListener. (#14638)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-402.html b/430/release/release-402.html new file mode 100644 index 000000000..264c83b1e --- /dev/null +++ b/430/release/release-402.html @@ -0,0 +1,2755 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 402 (2 Nov 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 402 (2 Nov 2022)#

+
+

General#

+ +
+
+

Blackhole connector#

+
    +
  • Add support for column comments on view columns. (#10705)

  • +
+
+
+

Clickhouse connector#

+
    +
  • Reuse JDBC connections for metadata queries. This can be disabled with the +query.reuse-connection configuration property. (#14653)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Remove the deprecated hive.parquet.fail-on-corrupted-statistics and +parquet.fail-on-corrupted-statistics configuration properties. The +parquet.ignore-statistics property can be used to allow querying Parquet +files with corrupted or incorrect statistics. (#14777)

  • +
  • Fix memory leak and improve memory tracking during large INSERT queries. (#14823)

  • +
+
+
+

Druid connector#

+
    +
  • Reuse JDBC connections for metadata queries. This can be disabled with the +query.reuse-connection configuration property. (#14653)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for column comments on view columns. (#10705)

  • +
  • Remove the deprecated hive.parquet.fail-on-corrupted-statistics and +parquet.fail-on-corrupted-statistics configuration properties. The +parquet.ignore-statistics property can be used to allow querying Parquet +files with corrupted or incorrect statistics. (#14777)

  • +
  • Fix memory leak and improve memory tracking during large INSERT queries. (#14823)

  • +
+
+
+

Hudi connector#

+
    +
  • Remove the deprecated hive.parquet.fail-on-corrupted-statistics and +parquet.fail-on-corrupted-statistics configuration properties. The +parquet.ignore-statistics property can be used to allow querying Parquet +files with corrupted or incorrect statistics. (#14777)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add support to skip archiving when committing to a table in the Glue +metastore and the iceberg.glue.skip-archive configuration property is set +to true. (#13413)

  • +
  • Add support for column comments on view columns. (#10705)

  • +
  • Remove the deprecated hive.parquet.fail-on-corrupted-statistics and +parquet.fail-on-corrupted-statistics configuration properties. The +parquet.ignore-statistics property can be used to allow querying Parquet +files with corrupted or incorrect statistics. (#14777)

  • +
  • Fix incorrect results when the column order in the equality delete filter is +different from the table definition. (#14693)

  • +
  • Fix memory leak and improve memory tracking during large INSERT queries. (#14823)

  • +
+
+
+

MariaDB connector#

+
    +
  • Reuse JDBC connections for metadata queries. This can be disabled with the +query.reuse-connection configuration property. (#14653)

  • +
+
+
+

Memory connector#

+
    +
  • Add support for column comments on view columns. (#10705)

  • +
+
+
+

MySQL connector#

+
    +
  • Reuse JDBC connections for metadata queries. This can be disabled with the +query.reuse-connection configuration property. (#14653)

  • +
+
+
+

MongoDB connector#

+
    +
  • Support predicate pushdown on date, time(3), timestamp(3) and +timestamp(3) with time zone columns. (#14795)

  • +
+
+
+

Oracle connector#

+
    +
  • Reuse JDBC connections for metadata queries. This can be disabled with the +query.reuse-connection configuration property. (#14653)

  • +
+
+
+

Phoenix connector#

+
    +
  • Reuse JDBC connections for metadata queries. This can be disabled with the +query.reuse-connection configuration property. (#14653)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Reuse JDBC connections for metadata queries. This can be disabled with the +query.reuse-connection configuration property. (#14653)

  • +
+
+
+

Redshift connector#

+
    +
  • Reuse JDBC connections for metadata queries. This can be disabled with the +query.reuse-connection configuration property. (#14653)

  • +
+
+
+

SingleStore (MemSQL) connector#

+
    +
  • Reuse JDBC connections for metadata queries. This can be disabled with the +query.reuse-connection configuration property. (#14653)

  • +
+
+
+

SQL Server connector#

+
    +
  • Improve performance of certain queries which use the OR operator. (#14570)

  • +
  • Improve performance of queries with predicates involving the nullif function +or arithmetic expressions. (#14570)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-403.html b/430/release/release-403.html new file mode 100644 index 000000000..25455e4ef --- /dev/null +++ b/430/release/release-403.html @@ -0,0 +1,2670 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 403 (15 Nov 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 403 (15 Nov 2022)#

+
+

General#

+
    +
  • Include the amount of data read from external sources in the output of +EXPLAIN ANALYZE. (#14907)

  • +
  • Improve performance of worker-to-worker data transfer encryption when +fault-tolerant execution is enabled. (#14941)

  • +
  • Improve performance of aggregations when input data does not contain nulls. (#14567)

  • +
  • Fix potential failure when clients do not support variable precision temporal +types. (#14950)

  • +
  • Fix query deadlock in multi-join queries where broadcast join size is +underestimated. (#14948)

  • +
  • Fix incorrect results when min(x, n) or max(x, n) is used as a window +function. (#14886)

  • +
  • Fix failure for certain queries involving joins over partitioned tables. (#14317)

  • +
  • Fix incorrect order of parameters in DESCRIBE INPUT when they appear in a +WITH clause. (#14738)

  • +
  • Fix failure for queries involving BETWEEN predicates over varchar columns +that contain temporal data. (#14954)

  • +
+
+
+

Security#

+
    +
  • Allow access token passthrough when using OAuth 2.0 authentication with +refresh tokens enabled. (#14949)

  • +
+
+
+

BigQuery connector#

+
    +
  • Improve performance of SHOW SCHEMAS by adding a metadata cache. This can be +configured with the bigquery.metadata.cache-ttl catalog property, which is +disabled by default. (#14729)

  • +
  • Fix failure when a row access policy +returns an empty result. (#14760)

  • +
+
+
+

ClickHouse connector#

+
    +
  • Add mapping for the ClickHouse DateTime(timezone) type to the Trino +timestamp(0) with time zone type for read-only operations. (#13541)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Fix statistics for DATE columns. (#15005)

  • +
+
+
+

Hive connector#

+
    +
  • Avoid showing the unsupported AUTHORIZATION ROLE property in the result of +SHOW CREATE SCHEMA when the access control doesn’t support roles. (#8817)

  • +
+
+
+

Iceberg connector#

+
    +
  • Improve performance and storage requirements when running the +expire_snapshots table procedure on S3-compatible storage. (#14434)

  • +
  • Allow registering existing table files in the metastore with the new +register_table procedure. (#13552)

  • +
+
+
+

MongoDB connector#

+
    +
  • Add support for DELETE. (#14864)

  • +
  • Fix incorrect results when predicates over varchar and char columns are +pushed into the connector and MongoDB collections have a collation specified. (#14900)

  • +
+
+
+

SQL Server connector#

+
    +
  • Fix incorrect results when non-transactional INSERT is disabled and bulk +INSERT is enabled. (#14856)

  • +
+
+
+

SPI#

+
    +
  • Enhance ConnectorTableLayout to allow the connector to specify that multiple +writers per partition are allowed. (#14956)

  • +
  • Remove deprecated methods from ConnectorPageSinkProvider. (#14959)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-404.html b/430/release/release-404.html new file mode 100644 index 000000000..3e03c07e7 --- /dev/null +++ b/430/release/release-404.html @@ -0,0 +1,2543 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 404 (???) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 404 (???)#

+
+

Note

+

Where’s release 404? Not found. (We skipped it on purpose because we think we’re +funny.)

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-405.html b/430/release/release-405.html new file mode 100644 index 000000000..699b74e3a --- /dev/null +++ b/430/release/release-405.html @@ -0,0 +1,2877 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 405 (28 Dec 2022) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 405 (28 Dec 2022)#

+
+

General#

+
    +
  • Add Trino version to the output of EXPLAIN. (#15317)

  • +
  • Add task input/output size distribution to the output of +EXPLAIN ANALYZE VERBOSE. (#15286)

  • +
  • Add stage skewness warnings to the output of EXPLAIN ANALYZE. (#15286)

  • +
  • Add support for ALTER COLUMN ... SET DATA TYPE statement. (#11608)

  • +
  • Allow configuring a refresh interval for the database resource group manager +with the resource-groups.refresh-interval configuration property. (#14514)

  • +
  • Improve performance of queries that compare date columns with +timestamp(n) with time zone literals. (#5798)

  • +
  • Improve performance and resource utilization when inserting into tables. (#14718, #14874)

  • +
  • Improve performance for INSERT queries when fault-tolerant execution is +enabled. (#14735)

  • +
  • Improve planning performance for queries with many GROUP BY clauses. (#15292)

  • +
  • Improve query performance for large clusters and skewed queries. (#15369)

  • +
  • Rename the node-scheduler.max-pending-splits-per-task configuration property +to node-scheduler.min-pending-splits-per-task. (#15168)

  • +
  • Ensure that the configured number of task retries is not larger than 126. (#14459)

  • +
  • Fix incorrect rounding of time(n) and time(n) with time zone values near +the top of the range of allowed values. (#15138)

  • +
  • Fix incorrect results for queries involving window functions without a +PARTITION BY clause followed by the evaluation of window functions with a +PARTITION BY and ORDER BY clause. (#15203)

  • +
  • Fix incorrect results when adding or subtracting an interval from a +timestamp with time zone. (#15103)

  • +
  • Fix potential incorrect results when joining tables on indexed and non-indexed +columns at the same time. (#15334)

  • +
  • Fix potential failure of queries involving MATCH_RECOGNIZE. (#15343)

  • +
  • Fix incorrect reporting of Projection CPU time in the output of EXPLAIN ANALYZE VERBOSE. (#15364)

  • +
  • Fix SET TIME ZONE LOCAL to correctly reset to the initial time zone of the +client session. (#15314)

  • +
+
+
+

Security#

+
    +
  • Add support for string replacement as part of +impersonation rules. (#14962)

  • +
  • Add support for fetching access control rules via HTTPS. (#14008)

  • +
  • Fix some system.metadata tables improperly showing the names of catalogs +which the user cannot access. (#14000)

  • +
  • Fix USE statement improperly disclosing the names of catalogs and schemas +which the user cannot access. (#14208)

  • +
  • Fix improper HTTP redirect after OAuth 2.0 token refresh. (#15336)

  • +
+
+
+

Web UI#

+
    +
  • Display operator CPU time in the “Stage Performance” tab. (#15339)

  • +
+
+
+

JDBC driver#

+
    +
  • Return correct values in NULLABLE columns of the +DatabaseMetaData.getColumns result. (#15214)

  • +
+
+
+

BigQuery connector#

+
    +
  • Improve read performance with experimental support for Apache Arrow +serialization when reading from BigQuery. This can be enabled with the +bigquery.experimental.arrow-serialization.enabled catalog configuration +property. (#14972)

  • +
  • Fix queries incorrectly executing with the project ID specified in the +credentials instead of the project ID specified in the bigquery.project-id +catalog property. (#14083)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Add support for views. (#11609)

  • +
  • Add support for configuring batch size for reads on Parquet files using the +parquet.max-read-block-row-count configuration property or the +parquet_max_read_block_row_count session property. (#15474)

  • +
  • Improve performance and reduce storage requirements when running the vacuum +procedure on S3-compatible storage. (#15072)

  • +
  • Improve memory accounting for INSERT, MERGE, and +CREATE TABLE ... AS SELECT queries. (#14407)

  • +
  • Improve performance of reading Parquet files for boolean, tinyint, +short, int, long, float, double, short decimal, UUID, time, +decimal, varchar, and char data types. This optimization can be disabled +with the parquet.optimized-reader.enabled catalog configuration property. (#14423, #14667)

  • +
  • Improve query performance when the nulls fraction statistic is not available +for some columns. (#15132)

  • +
  • Improve performance when reading Parquet files. (#15257, #15474)

  • +
  • Improve performance of reading Parquet files for queries with filters. (#15268)

  • +
  • Improve DROP TABLE performance for tables stored on AWS S3. (#13974)

  • +
  • Improve performance of reading Parquet files for timestamp and +timestamp with timezone data types. (#15204)

  • +
  • Improve performance of queries that read a small number of columns and queries +that process tables with large Parquet row groups or ORC stripes. (#15168)

  • +
  • Improve stability and reduce peak memory requirements when reading from +Parquet files. (#15374)

  • +
  • Allow registering existing table files in the metastore with the new +register_table procedure. (#13568)

  • +
  • Deprecate creating a new table with existing table content. This can be +re-enabled using the delta.legacy-create-table-with-existing-location.enabled +configuration property or the +legacy_create_table_with_existing_location_enabled session property. (#13568)

  • +
  • Fix query failure when reading Parquet files with large row groups. (#5729)

  • +
  • Fix DROP TABLE leaving files behind when using managed tables stored on S3 +and created by the Databricks runtime. (#13017)

  • +
  • Fix query failure when the path contains special characters. (#15183)

  • +
  • Fix potential INSERT failure for tables stored on S3. (#15476)

  • +
+
+
+

Google Sheets connector#

+
    +
  • Add support for setting a read timeout with the gsheets.read-timeout +configuration property. (#15322)

  • +
  • Add support for base64-encoded credentials using the +gsheets.credentials-key configuration property. (#15477)

  • +
  • Rename the credentials-path configuration property to +gsheets.credentials-path, metadata-sheet-id to +gsheets.metadata-sheet-id, sheets-data-max-cache-size to +gsheets.max-data-cache-size, and sheets-data-expire-after-write to +gsheets.data-cache-ttl. (#15042)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for referencing nested fields in columns with the UNIONTYPE Hive +type. (#15278)

  • +
  • Add support for configuring batch size for reads on Parquet files using the +parquet.max-read-block-row-count configuration property or the +parquet_max_read_block_row_count session property. (#15474)

  • +
  • Improve memory accounting for INSERT, MERGE, and CREATE TABLE AS SELECT +queries. (#14407)

  • +
  • Improve performance of reading Parquet files for boolean, tinyint, +short, int, long, float, double, short decimal, UUID, time, +decimal, varchar, and char data types. This optimization can be disabled +with the parquet.optimized-reader.enabled catalog configuration property. (#14423, #14667)

  • +
  • Improve performance for queries which write data into multiple partitions. (#15241, #15066)

  • +
  • Improve performance when reading Parquet files. (#15257, #15474)

  • +
  • Improve performance of reading Parquet files for queries with filters. (#15268)

  • +
  • Improve DROP TABLE performance for tables stored on AWS S3. (#13974)

  • +
  • Improve performance of reading Parquet files for timestamp and +timestamp with timezone data types. (#15204)

  • +
  • Improve performance of queries that read a small number of columns and queries +that process tables with large Parquet row groups or ORC stripes. (#15168)

  • +
  • Improve stability and reduce peak memory requirements when reading from +Parquet files. (#15374)

  • +
  • Disallow creating transactional tables when not using the Hive metastore. (#14673)

  • +
  • Fix query failure when reading Parquet files with large row groups. (#5729)

  • +
  • Fix incorrect schema already exists error caused by a client timeout when +creating a new schema. (#15174)

  • +
  • Fix failure when an access denied exception happens while listing tables or +views in a Glue metastore. (#14746)

  • +
  • Fix INSERT failure on ORC ACID tables when Apache Hive 3.1.2 is used as a +metastore. (#7310)

  • +
  • Fix failure when reading Hive views with char types. (#15470)

  • +
  • Fix potential INSERT failure for tables stored on S3. (#15476)

  • +
+
+
+

Hudi connector#

+
    +
  • Improve performance of reading Parquet files for boolean, tinyint, +short, int, long, float, double, short decimal, UUID, time, +decimal, varchar, and char data types. This optimization can be disabled +with the parquet.optimized-reader.enabled catalog configuration property. (#14423, #14667)

  • +
  • Improve performance of reading Parquet files for queries with filters. (#15268)

  • +
  • Improve performance of reading Parquet files for timestamp and +timestamp with timezone data types. (#15204)

  • +
  • Improve performance of queries that read a small number of columns and queries +that process tables with large Parquet row groups or ORC stripes. (#15168)

  • +
  • Improve stability and reduce peak memory requirements when reading from +Parquet files. (#15374)

  • +
  • Fix query failure when reading Parquet files with large row groups. (#5729)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add support for configuring batch size for reads on Parquet files using the +parquet.max-read-block-row-count configuration property or the +parquet_max_read_block_row_count session property. (#15474)

  • +
  • Add support for the Iceberg REST catalog. (#13294)

  • +
  • Improve memory accounting for INSERT, MERGE, and CREATE TABLE AS SELECT +queries. (#14407)

  • +
  • Improve performance of reading Parquet files for boolean, tinyint, +short, int, long, float, double, short decimal, UUID, time, +decimal, varchar, and char data types. This optimization can be disabled +with the parquet.optimized-reader.enabled catalog configuration property. (#14423, #14667)

  • +
  • Improve performance when reading Parquet files. (#15257, #15474)

  • +
  • Improve performance of reading Parquet files for queries with filters. (#15268)

  • +
  • Improve DROP TABLE performance for tables stored on AWS S3. (#13974)

  • +
  • Improve performance of reading Parquet files for timestamp and +timestamp with timezone data types. (#15204)

  • +
  • Improve performance of queries that read a small number of columns and queries +that process tables with large Parquet row groups or ORC stripes. (#15168)

  • +
  • Improve stability and reduce peak memory requirements when reading from +Parquet files. (#15374)

  • +
  • Fix incorrect results when predicates over row columns on Parquet files are +pushed into the connector. (#15408)

  • +
  • Fix query failure when reading Parquet files with large row groups. (#5729)

  • +
  • Fix REFRESH MATERIALIZED VIEW failure when the materialized view is based on +non-Iceberg tables. (#13131)

  • +
  • Fix failure when an access denied exception happens while listing tables or +views in a Glue metastore. (#14971)

  • +
  • Fix potential INSERT failure for tables stored on S3. (#15476)

  • +
+
+
+

Kafka connector#

+ +
+
+

MongoDB connector#

+
    +
  • Add support for fault-tolerant execution. (#15062)

  • +
  • Add support for setting a file path and password for the truststore and +keystore. (#15240)

  • +
  • Add support for case-insensitive name-matching in the query table function. (#15329)

  • +
  • Rename the mongodb.ssl.enabled configuration property to +mongodb.tls.enabled. (#15240)

  • +
  • Upgrade minimum required MongoDB version to +4.2. (#15062)

  • +
  • Delete a MongoDB field from collections when dropping a column. +Previously, the connector deleted only metadata. (#15226)

  • +
  • Remove deprecated mongodb.seeds and mongodb.credentials configuration +properties. (#15263)

  • +
  • Fix failure when an unauthorized exception happens while listing schemas or +tables. (#1398)

  • +
  • Fix NullPointerException when a column name contains uppercase characters in +the query table function. (#15294)

  • +
  • Fix potential incorrect results when the objectid function is used more than +once within a single query. (#15426)

  • +
+
+
+

MySQL connector#

+
    +
  • Fix failure when the query table function contains a WITH clause. (#15332)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Fix query failure when a FULL JOIN is pushed down. (#14841)

  • +
+
+
+

Redshift connector#

+
    +
  • Add support for aggregation, join, and ORDER BY ... LIMIT pushdown. (#15365)

  • +
  • Add support for DELETE. (#15365)

  • +
  • Add schema, table, and column name length checks. (#15365)

  • +
  • Add full type mapping for Redshift types. The previous behavior can be +restored via the redshift.use-legacy-type-mapping configuration property. (#15365)

  • +
+
+
+

SPI#

+
    +
  • Remove deprecated ConnectorNodePartitioningProvider.getBucketNodeMap() +method. (#14067)

  • +
  • Use the MERGE APIs in the engine to execute DELETE and UPDATE. +Require connectors to implement beginMerge() and related APIs. +Deprecate beginDelete(), beginUpdate() and UpdatablePageSource, which +are unused and do not need to be implemented. (#13926)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-406.html b/430/release/release-406.html new file mode 100644 index 000000000..31912dcd4 --- /dev/null +++ b/430/release/release-406.html @@ -0,0 +1,2716 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 406 (25 Jan 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 406 (25 Jan 2023)#

+
+

General#

+
    +
  • Add support for exchange spooling on HDFS when +fault-tolerant execution is enabled. (#15160)

  • +
  • Add support for CHECK constraints in an INSERT statement. (#14964)

  • +
  • Improve planner estimates for queries containing outer joins over a subquery +involving ORDER BY and LIMIT. (#15428)

  • +
  • Improve accuracy of memory usage reporting for table scans. (#15711)

  • +
  • Improve performance of queries parsing date values in ISO 8601 format. (#15548)

  • +
  • Improve performance of queries with selective joins. (#15569)

  • +
  • Remove legacy-phased execution scheduler as an option for the +query.execution-policy configuration property. (#15657)

  • +
  • Fix failure when WHERE or JOIN clauses contain a LIKE expression with a +non-constant pattern or escape. (#15629)

  • +
  • Fix inaccurate planner estimates for queries with filters on columns without +statistics. (#15642)

  • +
  • Fix queries with outer joins failing when fault-tolerant execution is +enabled. (#15608)

  • +
  • Fix potential query failure when using MATCH_RECOGNIZE. (#15461)

  • +
  • Fix query failure when using group-based access control with column masks or +row filters. (#15583)

  • +
  • Fix potential hang during shutdown. (#15675)

  • +
  • Fix incorrect results when referencing a field resulting from the application +of a column mask expression that produces a row type. (#15659)

  • +
  • Fix incorrect application of column masks when a mask expression references a +different column in the underlying table. (#15680)

  • +
+
+
+

BigQuery connector#

+ +
+
+

Cassandra connector#

+
    +
  • Fix incorrect results when the Cassandra list, map, or set types contain +user-defined types. (#15771)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Reduce latency for INSERT queries on unpartitioned tables. (#15708)

  • +
  • Improve performance of reading Parquet files. (#15498)

  • +
  • Improve memory accounting of the Parquet reader. (#15554)

  • +
  • Improve performance of queries with filters or projections on low-cardinality +string columns stored in Parquet files. (#15269)

  • +
  • Fix reading more data than necessary from Parquet files for queries with +filters. (#15552)

  • +
  • Fix potential query failure when writing to Parquet from a table with an +INTEGER range on a BIGINT column. (#15496)

  • +
  • Fix query failure due to missing null counts in Parquet column indexes. (#15706)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for table redirections to catalogs using the Hudi connector. (#14750)

  • +
  • Reduce latency for INSERT queries on unpartitioned tables. (#15708)

  • +
  • Improve performance of caching. (#13243)

  • +
  • Improve performance of reading Parquet files. (#15498)

  • +
  • Improve memory accounting of the Parquet reader. (#15554)

  • +
  • Improve performance of queries with filters or projections on low-cardinality +string columns stored in Parquet files. (#15269)

  • +
  • Improve performance of queries with filters when Bloom filter indexes are +present in Parquet files. Use of Bloom filters from Parquet files can be +disabled with the parquet.use-bloom-filter configuration property or the +parquet_use_bloom_filter session property. (#14428)

  • +
  • Allow coercion between Hive UNIONTYPE and Hive STRUCT-typed columns. (#15017)

  • +
  • Fix reading more data than necessary from Parquet files for queries with +filters. (#15552)

  • +
  • Fix query failure due to missing null counts in Parquet column indexes. (#15706)

  • +
  • Fix incorrect schema already exists error caused by a client timeout when +creating a new schema. (#15174)

  • +
+
+
+

Hudi connector#

+
    +
  • Improve performance of reading Parquet files. (#15498)

  • +
  • Improve memory accounting of the Parquet reader. (#15554)

  • +
  • Improve performance of queries with filters or projections on low-cardinality +string columns stored in Parquet files. (#15269)

  • +
  • Fix reading more data than necessary from Parquet files for queries with +filters. (#15552)

  • +
  • Fix query failure due to missing null counts in Parquet column indexes. (#15706)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add support for changing column types. (#15515)

  • +
  • Add support for the JDBC catalog. (#9968)

  • +
  • Reduce latency for INSERT queries on unpartitioned tables. (#15708)

  • +
  • Improve performance of reading Parquet files. (#15498)

  • +
  • Improve memory accounting of the Parquet reader. (#15554)

  • +
  • Improve performance of queries with filters or projections on low-cardinality +string columns stored in Parquet files. (#15269)

  • +
  • Fix reading more data than necessary from Parquet files for queries with +filters. (#15552)

  • +
  • Fix query failure due to missing null counts in Parquet column indexes. (#15706)

  • +
  • Fix query failure when a subquery contains time travel. (#15607)

  • +
  • Fix failure when reading columns that had their type changed from float to +double by other query engines. (#15650)

  • +
  • Fix incorrect results when reading or writing NaN with real or double +types on partitioned columns. (#15723)

  • +
+
+
+

MongoDB connector#

+
    +
  • Fix schemas not being dropped when trying to drop schemas with the +mongodb.case-insensitive-name-matching configuration property enabled. (#15716)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Add support for changing column types. (#15515)

  • +
+
+
+

SPI#

+
    +
  • Remove the getDeleteRowIdColumnHandle(), beginDelete(), finishDelete(), +getUpdateRowIdColumnHandle(), beginUpdate(), and finishUpdate() methods +from ConnectorMetadata. (#15161)

  • +
  • Remove the UpdatablePageSource interface. (#15161)

  • +
  • Remove support for multiple masks on a single column. (#15680)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-407.html b/430/release/release-407.html new file mode 100644 index 000000000..3a37c4058 --- /dev/null +++ b/430/release/release-407.html @@ -0,0 +1,2718 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 407 (16 Feb 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 407 (16 Feb 2023)#

+
+

General#

+
    +
  • Add support for correlated queries involving a VALUES clause with a single +row. (#15989)

  • +
  • Reduce memory usage for large schemas. This behavior can be enabled with the +query.remote-task.enable-adaptive-request-size configuration property and +configured with the query.remote-task.max-request-size, +query.remote-task.request-size-headroom, and +query.remote-task.guaranteed-splits-per-task configuration properties or +their respective session properties. (#15721)

  • +
  • Improve concurrency when small, concurrent queries are run on a large cluster +by automatically determining how many nodes to use for distributed joins +and aggregations. This can be managed with the query.max-hash-partition-count +(renamed from query.hash-partition-count) and query.min-hash-partition-count +configuration properties. (#15489)

  • +
  • Improve query memory tracking. (#15983)

  • +
  • Improve memory usage accounting for queries with dynamic filters. (#16110)

  • +
  • Improve query performance when a predicate evaluates to a null value. (#15744)

  • +
  • Improve performance of queries with joins on the output of global +aggregations. (#15858)

  • +
  • Improve performance of selective queries, queries that read a small number of +columns, and queries that process tables with large Parquet row groups or ORC +stripes. (#15579)

  • +
  • Improve performance of queries with window functions. (#15994)

  • +
  • Return an exit code of 100 when Trino crashes during startup. (#16113)

  • +
  • Fix precision loss when converting time values with a precision higher than +three and time with time zone values with lower precision. (#15861)

  • +
  • Fix potential incorrect results due to a query reporting normal completion +instead of failing. (#15917)

  • +
  • Fix connection errors caused by a reusable connection being closed. (#16121)

  • +
  • Fix incorrect results for queries involving an equality predicate in a WHERE +clause that is equal to a term of a SELECT clause in one of the branches of +a JOIN. (#16101)

  • +
+
+
+

Cassandra connector#

+
    +
  • Add query table function for full query pass-through to the connector. (#15973)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Add support for the unregister_table procedure. (#15784)

  • +
  • Add support for inserting into tables that have CHECK constraints. (#15396)

  • +
  • Add support for writing to the change data feed. +This can be enabled with the delta.enableChangeDataFeed table property. (#15453)

  • +
  • Add a $history system table which can be queried to inspect Delta Lake table +history. (#15683)

  • +
  • Improve performance of reading decimal types from Parquet files. (#15713)

  • +
  • Improve performance of reading numeric types from Parquet files. (#15850)

  • +
  • Improve performance of reading string types from Parquet files. (#15897, #15923)

  • +
  • Improve performance of reading timestamp and boolean types from Parquet files. (#15954)

  • +
  • Improve query performance on tables created by Trino with CREATE TABLE AS. (#15878)

  • +
  • Remove support for the legacy Parquet writer. (#15436)

  • +
  • Fix query failure when reading Parquet files written by Apache Impala. (#15942)

  • +
  • Fix listing relations failure when a Glue table has no table type set. (#15909)

  • +
+
+
+

Hive connector#

+
    +
  • Reduce query latency. (#15811)

  • +
  • Improve performance of reading decimal types from Parquet files. (#15713)

  • +
  • Improve performance of reading numeric types from Parquet files. (#15850)

  • +
  • Improve performance of reading string types from Parquet files. (#15897, #15923)

  • +
  • Improve performance of reading timestamp and boolean types from Parquet files. (#15954)

  • +
  • Improve performance of predicate pushdown to partitioned columns in tables +with a high number of partitions. (#16113)

  • +
  • Reduce server errors in high-load scenarios. This can be enabled with the +hive.s3.connect-ttl configuration property. (#16005)

  • +
  • Allow setting the hive.max-partitions-per-scan configuration property to a +value lower than the value set in hive.max-partitions-for-eager-load. (#16111)

  • +
  • Fix query failure when reading Parquet files written by Apache Impala. (#15942)

  • +
  • Fix listing relations failure when a Glue table has no table type set. (#15909)

  • +
+
+
+

Hudi connector#

+
    +
  • Improve performance of reading decimal types from Parquet files. (#15713)

  • +
  • Improve performance of reading numeric types from Parquet files. (#15850)

  • +
  • Improve performance of reading string types from Parquet files. (#15897, #15923)

  • +
  • Improve performance of reading timestamp and boolean types from Parquet files. (#15954)

  • +
  • Fix query failure when reading Parquet files written by Apache Impala. (#15942)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add support for the unregister_table procedure. (#15784)

  • +
  • Add support for register_table procedures in the JDBC catalog. (#15853)

  • +
  • Add support for specifying a user and password when connecting to the JDBC +catalog via the iceberg.jdbc-catalog.connection-user and +iceberg.jdbc-catalog.connection-password configuration properties. (#16040)

  • +
  • Add support for compacting manifests asynchronously, which can be enabled by +setting the iceberg.merge_manifests_on_write session property to false. (#14822)

  • +
  • Improve performance of DROP TABLE. (#15981)

  • +
  • Improve performance of reading position delete files +with ORC data (#15969).

  • +
  • Improve performance of reading decimal columns from Parquet files. (#15713)

  • +
  • Improve performance of reading numeric types from Parquet files. (#15850)

  • +
  • Improve performance of reading string types from Parquet files. (#15897, #15923)

  • +
  • Improve performance of reading timestamp and boolean types from Parquet files. (#15954)

  • +
  • Prevent creating a table when the specified schema does not exist. (#15779)

  • +
  • Fix query failure when reading Parquet files written by Apache Impala. (#15942)

  • +
  • Fix listing relations failure when a Glue table has no table type set. (#15909)

  • +
  • Fix failure when encountering access denied exceptions while listing +materialized views in the Glue metastore. (#15893)

  • +
+
+
+

Kudu connector#

+
    +
  • Fix authentication failure when Kerberos tickets expire. (#14372)

  • +
+
+
+

Memory connector#

+
    +
  • Fix potential failure when reading table column metadata with concurrent +CREATE TABLE or DROP TABLE operations. (#16062)

  • +
+
+
+

MongoDB connector#

+
    +
  • Add support for changing column types. (#15515)

  • +
+
+
+

MySQL connector#

+
    +
  • Fix potential failure when zeroDateTimeBehavior is set to convertToNull. (#16027)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-408.html b/430/release/release-408.html new file mode 100644 index 000000000..9f107b830 --- /dev/null +++ b/430/release/release-408.html @@ -0,0 +1,2684 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 408 (23 Feb 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 408 (23 Feb 2023)#

+
+

General#

+
    +
  • Add physical input read time to query statistics and the output of EXPLAIN ANALYZE. (#16190)

  • +
  • Fix query failure for queries involving joins or aggregations with a +structural type that contains NULL elements. (#16140)

  • +
+
+
+

Security#

+
    +
  • Deprecate using groups with OAuth 2.0 authentication, and rename the +http-server.authentication.oauth2.groups-field configuration property to +deprecated.http-server.authentication.oauth2.groups-field. (#15669)

  • +
+
+
+

CLI#

+
    +
  • Add AUTO output format which switches from ALIGNED to VERTICAL if +the output doesn’t fit the current terminal. (#12208)

  • +
  • Add --pager and --history-file options to match the existing TRINO_PAGER +and TRINO_HISTORY_FILE environmental variables. Also allow setting these +options in a configuration file. (#16151)

  • +
+
+
+

BigQuery connector#

+
    +
  • Add support for writing decimal types to BigQuery. (#16145)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Rename the connector to delta_lake. The old name delta-lake is now +deprecated and will be removed in a future release. (#13931)

  • +
  • Add support for creating tables with the Trino change_data_feed_enabled +table property. (#16129)

  • +
  • Improve query performance on tables that Trino has written to with INSERT. (#16026)

  • +
  • Improve performance of reading structural types from +Parquet files. This optimization can be disabled with the +parquet_optimized_nested_reader_enabled catalog session property or the +parquet.optimized-nested-reader.enabled catalog configuration property. (#16177)

  • +
  • Retry dropping Delta tables registered in the Glue catalog to avoid failures +due to concurrent modifications. (#13199)

  • +
  • Allow updating the reader_version and writer_version table properties. (#15932)

  • +
  • Fix inaccurate change data feed entries for MERGE queries. (#16127)

  • +
  • Fix performance regression when writing to partitioned tables if table +statistics are absent. (#16152)

  • +
+
+
+

Hive connector#

+
    +
  • Remove support for the deprecated hive-hadoop2 connector name, requiring the +connector.name property to be set to hive. (#16166)

  • +
  • Retry dropping Delta tables registered in the Glue catalog to avoid failures +due to concurrent modifications. (#13199)

  • +
  • Fix performance regression when writing to partitioned tables if table +statistics are absent. (#16152)

  • +
+
+
+

Iceberg connector#

+
    +
  • Reduce memory usage when reading $files system tables. (#15991)

  • +
  • Require the iceberg.jdbc-catalog.driver-class configuration property to be +set to prevent a “driver not found” error after initialization. (#16196)

  • +
  • Fix performance regression when writing to partitioned tables if table +statistics are absent. (#16152)

  • +
+
+
+

Ignite connector#

+ +
+
+

SingleStore connector#

+
    +
  • Remove support for the deprecated memsql connector name, requiring the +connector.name property to be set to singlestore. (#16180)

  • +
+
+
+

SQL Server connector#

+
    +
  • Add support for pushing down =, <> and IN predicates over text columns +if the column uses a case-sensitive collation within SQL Server. (#15714)

  • +
+
+
+

Thrift connector#

+
    +
  • Rename the connector to trino_thrift. The old name trino-thrift is now +deprecated and will be removed in a future release. (#13931)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-409.html b/430/release/release-409.html new file mode 100644 index 000000000..1363b5ac5 --- /dev/null +++ b/430/release/release-409.html @@ -0,0 +1,2689 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 409 (3 Mar 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 409 (3 Mar 2023)#

+
+

General#

+
    +
  • Add support for dropping nested fields with a DROP COLUMN statement. (#15975)

  • +
  • Add queue, analysis, planning, and execution time to the output of +EXPLAIN ANALYZE. (#16329)

  • +
  • Add support for executing table functions with any number of table arguments. (#1839)

  • +
  • Improve output of EXPLAIN queries to show statistics when a query contains +aggregations. (#16201)

  • +
  • Improve performance of queries with aggregations containing a DISTINCT +clause using table statistics. This can be configured with the +optimizer.mark-distinct-strategyconfiguration property, and the +optimizer.use-mark-distinct configuration property is now deprecated in +favor of it. (#15927)

  • +
  • Improve performance of LIKE expressions with patterns constructed dynamically. (#15999)

  • +
  • Remove the distributed-index-joins-enabled configuration property and +related functionality. (#15375)

  • +
  • Fix failure when using non-comparable and non-sortable values as part of a +VALUES expression. (#16242)

  • +
+
+
+

BigQuery connector#

+
    +
  • Add support for using default values when inserting data. (#16327)

  • +
  • Fix failure when non-lowercase column names exist in the query table +function. (#16075)

  • +
+
+
+

Cassandra connector#

+
    +
  • Add support for TIME type. (#13063)

  • +
+
+
+

ClickHouse connector#

+
    +
  • Remove support for the ru.yandex.clickhouse.ClickHouseDriver legacy JDBC +driver in the clickhouse.legacy-driver configuration property. (#16188)

  • +
  • Remove support for specifying expressions in the sample_by table property to +prevent SQL injection. (#16261)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Avoid query failure by inferring required Delta Lake version when creating new +tables or configuring table features. (#16310)

  • +
  • Fix query failure when reading Parquet files generated by Kafka Connect. (#16264)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for the Hadoop DefaultCodec to Hive formats. (#16250)

  • +
  • Add a native CSV file format reader and writer. These can be disabled with the +csv_native_reader_enabled and csv_native_writer_enabled session properties +or the csv.native-reader.enabled and csv.native-writer.enabled +configuration properties. (#15918)

  • +
  • Add a native JSON file format reader and writer. These can be disabled with +the json_native_reader_enabled and json_native_writer_enabled session +properties or the json.native-reader.enabled and +json.native-writer.enabled configuration properties. (#15918)

  • +
  • Add a native text file format reader and writer. These can be disabled with +the text_file_native_reader_enabled and text_file_native_writer_enabled +session properties or the text-file.native-reader.enabled and +text-file.native-writer.enabled configuration properties. (#15918)

  • +
  • Add a native sequence file format reader and writer. These can be disabled +with the sequence_file_native_reader_enabled and +sequence_file_native_writer_enabled session properties or the +sequence-file.native-reader.enabled and +sequence-file.native-writer.enabled configuration properties. (#15918)

  • +
  • Add a native regex file format reader. The reader can be disabled with the +regex_native_reader_enabled session property or the +regex.native-reader.enabled configuration property. (#15918)

  • +
  • Add regex and regex_case_insensitive table properties for the REGEX +format. (#16271)

  • +
  • Improve performance of queries which read from partitioned Hive tables and +write to partitioned tables when statistics are not available for the source +tables. (#16229)

  • +
  • Improve query performance when only table statistics generated by Apache Spark +are available. This can be disabled via the +hive.metastore.thrift.use-spark-table-statistics-fallback configuration +property. (#16120)

  • +
  • Fix incorrectly ignoring computed table statistics in ANALYZE. (#15995)

  • +
  • Fix query failure when reading Parquet files generated by Kafka Connect. (#16264)

  • +
+
+
+

Hudi connector#

+
    +
  • Fix query failure when reading Parquet files generated by Kafka Connect. (#16264)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add support for dropping nested fields with a DROP COLUMN statement. (#15975)

  • +
  • Add support for Iceberg table sort orders. Tables can have a list of +sorted_by columns which are used to order files written to the table. (#14891)

  • +
  • Fix query failure when reading nested columns on a table with +equality delete files. (#14836)

  • +
  • Fix query failure when reading Parquet files generated by Kafka Connect. (#16264)

  • +
+
+
+

SQL Server connector#

+
    +
  • Add support for pushing down joins using = and != predicates over text +columns if the column uses a case-sensitive collation within SQL Server. (#16185)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-410.html b/430/release/release-410.html new file mode 100644 index 000000000..0d4df1951 --- /dev/null +++ b/430/release/release-410.html @@ -0,0 +1,2621 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 410 (8 Mar 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 410 (8 Mar 2023)#

+
+

General#

+
    +
  • Improve performance of certain queries with unions. (#16019)

  • +
  • Fix potential query failure when using an ORDER BY clause while spilling to +disk is enabled. (#16406)

  • +
  • Fix potential query failure when performing a JOIN with a comparison between +two different types. (#16444)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Fix query failure when reading Parquet files generated by Airbyte. (#16362)

  • +
+
+
+

Google Sheets connector#

+ +
+
+

Hive connector#

+
    +
  • Add support for AWS Security Token Support with S3 Select. (#5947)

  • +
  • Fix incorrectly ignoring the hive.timestamp-precision configuration property +in Hive views. (#6295)

  • +
  • Fix query failure when reading Parquet files generated by Airbyte. (#16362)

  • +
  • Fix incorrect results when using a value with uppercase characters for the +partition_projection_location_template table property. (#16349)

  • +
+
+
+

Hudi connector#

+
    +
  • Fix query failure when reading Parquet files generated by Airbyte. (#16362)

  • +
+
+
+

Iceberg connector#

+
    +
  • Improve file pruning when generating Iceberg table statistics. (#16244)

  • +
  • Fix query failure when reading Parquet files generated by Airbyte. (#16362)

  • +
+
+
+

Ignite connector#

+ +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-411.html b/430/release/release-411.html new file mode 100644 index 000000000..8ed4e7ea0 --- /dev/null +++ b/430/release/release-411.html @@ -0,0 +1,2786 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 411 (29 Mar 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 411 (29 Mar 2023)#

+
+

General#

+
    +
  • Add spilled data size to query statistics. (#16442)

  • +
  • Add sinh() function. (#16494)

  • +
  • Add quantile_at_value() function. (#16736)

  • +
  • Add support for a GRACE PERIOD clause in the CREATE MATERIALIZED VIEW +task. For backwards compatibility, the existing materialized views are +interpreted as having a GRACE PERIOD of zero, however, new materialized +views have an unlimited grace period by default. This is a backwards +incompatible change, and the previous behavior can be restored with the +legacy.materialized-view-grace-period configuration property or the +legacy_materialized_view_grace_period session property. (#15842)

  • +
  • Fix potential incorrect query stats when tasks are waiting on running drivers +to fully terminate. (#15478)

  • +
  • Add support for specifying the number of nodes that will write data during +INSERT, CREATE TABLE ... AS SELECT, or EXECUTE queries with the +query.max-writer-tasks-count configuration property. (#16238)

  • +
  • Improve performance of queries that contain predicates involving the year +function. (#14078)

  • +
  • Improve performance of queries that contain a sum aggregation. (#16624)

  • +
  • Improve performance of filter function on arrays. (#16681)

  • +
  • Reduce coordinator memory usage. (#16668, #16669)

  • +
  • Reduce redundant data exchanges for queries with multiple aggregations. (#16328)

  • +
  • Fix incorrect query results when using keyvalue() methods in the +JSON path. (#16482)

  • +
  • Fix potential incorrect results in queries involving joins and a +non-deterministic value. (#16512)

  • +
  • Fix potential query failure when exchange compression is enabled. (#16541)

  • +
  • Fix query failure when calling a function with a large number of parameters. (#15979)

  • +
+
+
+

BigQuery connector#

+
    +
  • Fix failure of aggregation queries when executed against a materialized view, +external table, or snapshot table. (#15546)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Add support for inserting into tables that have +simple invariants. (#16136)

  • +
  • Add generated column expressions +to the Extra column in the results of DESCRIBE and SHOW COLUMNS. (#16631)

  • +
  • Expand the flush_metadata_cache table procedure to also flush the internal +caches of table snapshots and active data files. (#16466)

  • +
  • Collect statistics for newly-created columns. (#16109)

  • +
  • Remove the $data system table. (#16650)

  • +
  • Fix query failure when evaluating a WHERE clause on a partition column. (#16388)

  • +
+
+
+

Druid connector#

+
    +
  • Fix failure when the query passed to the query table function contains a +column alias. (#16225)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Remove the deprecated pass-through query, which has been replaced with the +raw_query table function. (#13050)

  • +
+
+
+

Hive connector#

+
    +
  • Add a native OpenX JSON file format reader and writer. These can be disabled +with the openx_json_native_reader_enabled and +openx_json_native_writer_enabled session properties or the +openx-json.native-reader.enabled and openx-json.native-writer.enabled +configuration properties. (#16073)

  • +
  • Add support for implicit coercions between char types of different lengths. (#16402)

  • +
  • Improve performance of queries with joins where both sides of a join have keys +with the same table bucketing definition. (#16381)

  • +
  • Improve query planning performance for queries scanning tables with a large +number of columns. (#16203)

  • +
  • Improve scan performance for COUNT(*) queries on row-oriented formats. (#16595)

  • +
  • Ensure the value of the hive.metastore-stats-cache-ttl configuration +property always is greater than or equal to the value specified in the +hive.metastore-cache-ttl configuration property. (#16625)

  • +
  • Skip listing Glue metastore tables with invalid column types. (#16677)

  • +
  • Fix query failure when a file that is using a text file format with a +single header row that is large enough to be split into multiple files. (#16492)

  • +
  • Fix potential query failure when Kerberos is enabled and the query execution +takes longer than a Kerberos ticket’s lifetime. (#16680)

  • +
+
+
+

Hudi connector#

+
    +
  • Add a $timeline system table which can be queried to inspect the Hudi table +timeline. (#16149)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add a migrate procedure that converts a Hive table to an Iceberg table. (#13196)

  • +
  • Add support for materialized views with a freshness grace period. (#15842)

  • +
  • Add a $refs system table which can be queried to inspect snapshot +references. (#15649)

  • +
  • Add support for creation of materialized views partitioned with a temporal +partitioning function on a timestamp with time zone column. (#16637)

  • +
  • Improve performance of queries run after data was written by Trino. (#15441)

  • +
  • Remove the $data system table. (#16650)

  • +
  • Fix failure when the $files system table contains non-null values in the +key_metadata, split_offsets, and equality_ids columns. (#16473)

  • +
  • Fix failure when partitioned column names contain uppercase characters. (#16622)

  • +
+
+
+

Ignite connector#

+
    +
  • Add support for predicate pushdown with a LIKE clause. (#16396)

  • +
  • Add support for pushdown of joins. (#16428)

  • +
  • Add support for DELETE. (#16720)

  • +
+
+
+

MariaDB connector#

+
    +
  • Fix failure when the query passed to the query table function contains a +column alias. (#16225)

  • +
+
+
+

MongoDB connector#

+
    +
  • Fix incorrect results when the query passed to the MongoDB query table +function contains helper functions such as ISODate. (#16626)

  • +
+
+
+

MySQL connector#

+
    +
  • Fix failure when the query passed to the query table function contains a +column alias. (#16225)

  • +
+
+
+

Oracle connector#

+
    +
  • Improve performance of queries when the network latency between Trino and +Oracle is high, or when selecting a small number of columns. (#16644)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Improve performance of queries when the network latency between Trino and +PostgreSQL is high, or when selecting a small number of columns. (#16644)

  • +
+
+
+

Redshift connector#

+
    +
  • Improve performance of queries when the network latency between Trino and +Redshift is high, or when selecting a small number of columns. (#16644)

  • +
+
+
+

SingleStore connector#

+
    +
  • Fix failure when the query passed to the query table function contains a +column alias. (#16225)

  • +
+
+
+

SQL Server connector#

+
    +
  • Add support for executing stored procedures using the procedure table +function. (#16696)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-412.html b/430/release/release-412.html new file mode 100644 index 000000000..d28d90634 --- /dev/null +++ b/430/release/release-412.html @@ -0,0 +1,2708 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 412 (5 Apr 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 412 (5 Apr 2023)#

+
+

General#

+
    +
  • Add support for aggregate functions and parameters as arguments for the +json_object() and json_array() +functions. (#16489, #16523, #16525)

  • +
  • Expose optimizer rule execution statistics in query statistics. The number of +rules for which statistics are collected can be limited with the +query.reported-rule-stats-limit configuration property. (#2578)

  • +
  • Add the exclude_columns() table function. (#16584)

  • +
  • Allow disabling the use of the cost-based optimizer to determine partitioning +of a stage with the optimizer.use-cost-based-partitioningconfiguration +property or the use_cost_based_partitioning session property. (#16781)

  • +
  • Improve performance of queries involving table functions with table arguments. (#16012)

  • +
  • Improve latency for small queries when fault-tolerant execution is enabled. (#16103)

  • +
  • Fix failure when querying a nested field of a row type in queries involving +ORDER BY ... LIMIT. (#16768)

  • +
+
+
+

JDBC driver#

+
    +
  • Allow configuring a custom DNS resolver. (#16647)

  • +
+
+
+

ClickHouse connector#

+
    +
  • Improve performance of queries involving sum(DISTINCT ...) by pushing +computation down to ClickHouse. (#16452)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Add support for arithmetic binary expressions in table check constraints. (#16721)

  • +
  • Improve performance of queries that only read partition columns. (#16788)

  • +
+
+
+

Hive connector#

+
    +
  • Fix query failure when bucketing or sorting column names are registered in +a metastore in uppercase. (#16796)

  • +
  • Fix query failure when reading transactional tables with locations containing +hidden directories. (#16773)

  • +
+
+
+

Iceberg connector#

+
    +
  • Fix incorrect results for the migrate procedure when the table location +contains a hidden directory. (#16779)

  • +
+
+
+

Ignite connector#

+
    +
  • Add support for ALTER TABLE ... ADD COLUMN. (#16755)

  • +
  • Improve performance of queries involving sum(DISTINCT ...) by pushing +computation to Ignite. (#16452)

  • +
+
+
+

MariaDB connector#

+
    +
  • Improve performance of queries involving sum(DISTINCT ...) by pushing +computation down to MariaDB. (#16452)

  • +
+
+
+

MySQL connector#

+
    +
  • Improve performance of queries involving sum(DISTINCT ...) by pushing +computation down to MySQL. (#16452)

  • +
+
+
+

Oracle connector#

+
    +
  • Improve performance of queries involving sum(DISTINCT ...) by pushing +computation down to Oracle. (#16452)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Add support for table comments. (#16135)

  • +
  • Improve performance of queries involving sum(DISTINCT ...) by pushing +computation down to PostgreSQL. (#16452)

  • +
+
+
+

Redshift connector#

+
    +
  • Improve performance of queries involving sum(DISTINCT ...) by pushing +computation down to Redshift. (#16452)

  • +
+
+
+

SQL Server connector#

+
    +
  • Improve performance of queries involving sum(DISTINCT ...) by pushing +computation down to SQL Server. (#16452)

  • +
+
+
+

SPI#

+
    +
  • Allow table functions to return anonymous columns. (#16584)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-413.html b/430/release/release-413.html new file mode 100644 index 000000000..42a5401cd --- /dev/null +++ b/430/release/release-413.html @@ -0,0 +1,2642 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 413 (12 Apr 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+ +
+ +
+
+ +
+

Release 413 (12 Apr 2023)#

+
+

General#

+ +
+
+

Delta Lake connector#

+
    +
  • Add support for nested timestamp with time zone values in +structural data types. (#16826)

  • +
  • Disallow using _change_type, _commit_version, and _commit_timestamp as +column names when creating a table or adding a column with +change data feed. (#16913)

  • +
  • Disallow enabling change data feed when the table contains +_change_type, _commit_version and _commit_timestamp columns. (#16913)

  • +
  • Fix incorrect results when reading INT32 values without a decimal logical +annotation in Parquet files. (#16938)

  • +
+
+
+

Hive connector#

+
    +
  • Fix incorrect results when reading INT32 values without a decimal logical +annotation in Parquet files. (#16938)

  • +
  • Fix incorrect results when the file path contains hidden characters. (#16386)

  • +
+
+
+

Hudi connector#

+
    +
  • Fix incorrect results when reading INT32 values without a decimal logical +annotation in Parquet files. (#16938)

  • +
+
+
+

Iceberg connector#

+
    +
  • Fix incorrect results when reading INT32 values without a decimal logical +annotation in Parquet files. (#16938)

  • +
  • Fix failure when creating a schema with a username containing uppercase +characters in the Iceberg Glue catalog. (#16116)

  • +
+
+
+

Oracle connector#

+ +
+
+

Phoenix connector#

+ +
+
+

SPI#

+
    +
  • Deprecate the getSchemaProperties() and getSchemaOwner() methods in +ConnectorMetadata in favor of versions that accept a String for the schema +name rather than CatalogSchemaName. (#16862)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-414.html b/430/release/release-414.html new file mode 100644 index 000000000..fb010f058 --- /dev/null +++ b/430/release/release-414.html @@ -0,0 +1,2654 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 414 (19 Apr 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 414 (19 Apr 2023)#

+
+

General#

+ +
+
+

Security#

+ +
+
+

Delta Lake connector#

+ +
+
+

Iceberg connector#

+
    +
  • Add support for Hive external tables in the migrate table procedure. (#16704)

  • +
+
+
+

Kafka connector#

+
    +
  • Fix query failure when a Kafka topic contains tombstones (messages with a +NULL value). (#16962)

  • +
+
+
+

Kudu connector#

+
    +
  • Fix query failure when merging two tables that were created by +CREATE TABLE ... AS SELECT .... (#16848)

  • +
+
+
+

Pinot connector#

+
    +
  • Fix incorrect results due to incorrect pushdown of aggregations. (#12655)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Fix failure when fetching table statistics for PostgreSQL 14.0 and later. (#17061)

  • +
+
+
+

Redshift connector#

+ +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-415.html b/430/release/release-415.html new file mode 100644 index 000000000..a455f5f38 --- /dev/null +++ b/430/release/release-415.html @@ -0,0 +1,2617 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 415 (28 Apr 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 415 (28 Apr 2023)#

+
+

General#

+
    +
  • Improve performance of aggregations with variable file sizes. (#11361)

  • +
  • Perform missing permission checks for table arguments to table functions. (#17279)

  • +
+
+
+

Web UI#

+
    +
  • Add CPU planning time to the query details page. (#15318)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Add support for commenting on tables and columns with an id and name +column mapping mode. (#17139)

  • +
  • Add support for BETWEEN predicates in table check constraints. (#17120)

  • +
+
+
+

Hive connector#

+
    +
  • Improve performance of queries with selective filters on primitive fields in +row columns. (#15163)

  • +
+
+
+

Iceberg connector#

+
    +
  • Improve performance of queries with filters when Bloom filter indexes are +present in Parquet files. (#17192)

  • +
  • Fix failure when trying to use DROP TABLE on a corrupted table. (#12318)

  • +
+
+
+

Kafka connector#

+
    +
  • Add support for Protobuf oneof types when using the Confluent table +description provider. (#16836)

  • +
+
+
+

SPI#

+
    +
  • Expose planningCpuTime in QueryStatistics. (#15318)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-416.html b/430/release/release-416.html new file mode 100644 index 000000000..afad3b7c0 --- /dev/null +++ b/430/release/release-416.html @@ -0,0 +1,2565 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 416 (3 May 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 416 (3 May 2023)#

+
+

General#

+
    +
  • Improve performance of partitioned INSERT, CREATE TABLE AS .. SELECT, and +EXECUTE statements when the source table statistics are missing or +inaccurate. (#16802)

  • +
  • Improve performance of LIKE expressions that contain %. (#16167)

  • +
  • Remove the deprecated preferred-write-partitioning-min-number-of-partitions +configuration property. (#16802)

  • +
+
+
+

Hive connector#

+
    +
  • Reduce coordinator memory usage when file metadata caching is enabled. (#17270)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-417.html b/430/release/release-417.html new file mode 100644 index 000000000..0af7b0950 --- /dev/null +++ b/430/release/release-417.html @@ -0,0 +1,2602 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 417 (10 May 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 417 (10 May 2023)#

+
+

General#

+
    +
  • Improve performance of UNION ALL queries. (#17265)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Add support for COMMENT ON VIEW. (#17089)

  • +
  • Improve performance when reading Parquet data written by Trino. (#17373, #17404)

  • +
  • Improve read performance for tables with row columns when only a subset of +fields is needed for a query. (#17085)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for specifying arbitrary table properties via the +extra_properties table property. (#954)

  • +
  • Improve performance when reading Parquet data written by Trino. (#17373, #17404)

  • +
  • Improve performance when reading text files that contain more columns in the +file than are mapped in the schema. (#17364)

  • +
  • Limit file listing cache based on in-memory size instead of number of entries. +This is configured via the hive.file-status-cache.max-retained-size and +hive.per-transaction-file-status-cache.max-retained-size configuration +properties. The hive.per-transaction-file-status-cache-maximum-size and +hive.file-status-cache-size configuration properties are deprecated. (#17285)

  • +
+
+
+

Hudi connector#

+
    +
  • Improve performance when reading Parquet data written by Trino. (#17373, #17404)

  • +
+
+
+

Iceberg connector#

+
    +
  • Improve performance when reading Parquet data written by Trino. (#17373, #17404)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-418.html b/430/release/release-418.html new file mode 100644 index 000000000..f4e2e9657 --- /dev/null +++ b/430/release/release-418.html @@ -0,0 +1,2661 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 418 (17 May 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 418 (17 May 2023)#

+
+

General#

+ +
+
+

BigQuery connector#

+
    +
  • Add support for adding labels to BigQuery jobs started by Trino as part of +query processing. The name and value of the label can be configured via the +bigquery.job.label-name and bigquery.job.label-format catalog +configuration properties, respectively. (#16187)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Add support for INSERT, UPDATE, DELETE, and MERGE statements for +tables with an id column mapping. (#16600)

  • +
  • Add the table_changes table function. (#16205)

  • +
  • Improve performance of joins on partition columns. (#14493)

  • +
+
+
+

Hive connector#

+
    +
  • Improve performance of querying information_schema.tables when using the +Hive metastore. (#17127)

  • +
  • Improve performance of joins on partition columns. (#14493)

  • +
  • Improve performance of writing Parquet files by enabling the optimized Parquet +writer by default. (#17393)

  • +
  • Remove the temporary_staging_directory_enabled and +temporary_staging_directory_path session properties. (#17390)

  • +
  • Fix failure when querying text files in S3 if the native reader is enabled. (#16546)

  • +
+
+
+

Hudi connector#

+
    +
  • Improve performance of joins on partition columns. (#14493)

  • +
+
+
+

Iceberg connector#

+
    +
  • Improve planning time for SELECT queries. (#17347)

  • +
  • Improve performance of joins on partition columns. (#14493)

  • +
  • Fix incorrect results when querying the $history table if the REST catalog +is used. (#17470)

  • +
+
+
+

Kafka connector#

+
    +
  • Fix query failure when a Kafka key or message cannot be de-serialized, and +instead correctly set the _key_corrupt and _message_corrupt columns. (#17479)

  • +
+
+
+

Kinesis connector#

+
    +
  • Fix query failure when a Kinesis message cannot be de-serialized, and +instead correctly set the _message_valid column. (#17479)

  • +
+
+
+

Oracle connector#

+ +
+
+

Redis connector#

+
    +
  • Fix query failure when a Redis key or value cannot be de-serialized, and +instead correctly set the _key_corrupt and _value_corrupt columns. (#17479)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-419.html b/430/release/release-419.html new file mode 100644 index 000000000..df271646a --- /dev/null +++ b/430/release/release-419.html @@ -0,0 +1,2651 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 419 (5 Jun 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+ +
+
+ +
+
+ +
+

Release 419 (5 Jun 2023)#

+
+

General#

+
    +
  • Add the array_histogram() function to find the number of occurrences of +the unique elements in an array. (#14725)

  • +
  • Improve planning performance for queries involving joins. (#17458)

  • +
  • Fix query failure when the server JSON response exceeds the 5MB limit for +string values. (#17557)

  • +
+
+
+

Web UI#

+
    +
  • Allow uppercase or mixed case values for the web-ui.authentication.type +configuration property. (#17334)

  • +
+
+
+

BigQuery connector#

+
    +
  • Add support for proxying BigQuery APIs via an HTTP(S) proxy. (#17508)

  • +
  • Improve performance of retrieving metadata from BigQuery. (#16064)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Support the id and name mapping modes when adding new columns. (#17236)

  • +
  • Improve performance of reading Parquet files. (#17612)

  • +
  • Improve performance when writing Parquet files with +structural data types. (#17665)

  • +
  • Properly display the schema, table name, and location of tables being inserted +into in the output of EXPLAIN queries. (#17590)

  • +
  • Fix query failure when writing to a file location with a trailing / in its +name. (#17552)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for reading ORC files with shorthand timezone ids in the Stripe +footer metadata. You can set the hive.orc.read-legacy-short-zone-id +configuration property to true to enable this behavior. (#12303)

  • +
  • Improve performance of reading ORC files with Bloom filter indexes. (#17530)

  • +
  • Improve performance of reading Parquet files. (#17612)

  • +
  • Improve optimized Parquet writer performance for +structural data types. (#17665)

  • +
  • Fix query failure for tables with file paths that contain non-alphanumeric +characters. (#17621)

  • +
+
+
+

Hudi connector#

+ +
+
+

Iceberg connector#

+
    +
  • Add support for the Nessie catalog. (#11701)

  • +
  • Disallow use of the migrate table procedure on Hive tables with array, +map and row types. Previously, this returned incorrect results after the +migration. (#17587)

  • +
  • Improve performance of reading ORC files with Bloom filter indexes. (#17530)

  • +
  • Improve performance of reading Parquet files. (#17612)

  • +
  • Improve performance when writing Parquet files with +structural data types. (#17665)

  • +
  • Improve performance of reading table statistics. (#16745)

  • +
+
+
+

SPI#

+
    +
  • Remove unused NullAdaptationPolicy from ScalarFunctionAdapter. (#17706)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-420.html b/430/release/release-420.html new file mode 100644 index 000000000..7c68fb8d1 --- /dev/null +++ b/430/release/release-420.html @@ -0,0 +1,2685 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 420 (22 Jun 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 420 (22 Jun 2023)#

+
+

General#

+
    +
  • Add support for the any_value() aggregation function. (#17777)

  • +
  • Add support for underscores in numeric literals. (#17776)

  • +
  • Add support for hexadecimal, binary, and octal numeric literals. (#17776)

  • +
  • Deprecate the dynamic-filtering.small-broadcast.* and +dynamic-filtering.large-broadcast.* configuration properties in favor of +dynamic-filtering.small.* and dynamic-filtering.large.*. (#17831)

  • +
+
+
+

Security#

+
    +
  • Add support for configuring authorization rules for +ALTER ... SET AUTHORIZATION... statements in file-based access control. (#16691)

  • +
  • Remove the deprecated legacy.allow-set-view-authorization configuration +property. (#16691)

  • +
+
+
+

BigQuery connector#

+
    +
  • Fix direct download of access tokens, and correctly use the proxy when it +is enabled with the bigquery.rpc-proxy.enabled configuration property. (#17783)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Add support for comments on view columns. (#17773)

  • +
  • Add support for recalculating all statistics with an ANALYZE statement. (#15968)

  • +
  • Disallow using the root directory of a bucket (scheme://authority) as a +table location without a trailing slash in the location name. (#17921)

  • +
  • Fix Parquet writer incompatibility with Apache Spark and Databricks Runtime. (#17978)

  • +
+
+
+

Druid connector#

+
    +
  • Add support for tables with uppercase characters in their names. (#7197)

  • +
+
+
+

Hive connector#

+
    +
  • Add a native Avro file format reader. This can be disabled with the +avro.native-reader.enabled configuration property or the +avro_native_reader_enabled session property. (#17221)

  • +
  • Require admin role privileges to perform ALTER ... SET AUTHORIZATION... +statements when the hive-security configuration property is set to +sql-standard. (#16691)

  • +
  • Improve query performance on partitioned Hive tables when table statistics are +not available. (#17677)

  • +
  • Disallow using the root directory of a bucket (scheme://authority) as a +table location without a trailing slash in the location name. (#17921)

  • +
  • Fix Parquet writer incompatibility with Apache Spark and Databricks Runtime. (#17978)

  • +
  • Fix reading from a Hive table when its location is the root directory of an S3 +bucket. (#17848)

  • +
+
+
+

Hudi connector#

+
    +
  • Disallow using the root directory of a bucket (scheme://authority) as a +table location without a trailing slash in the location name. (#17921)

  • +
  • Fix Parquet writer incompatibility with Apache Spark and Databricks Runtime. (#17978)

  • +
  • Fix failure when fetching table metadata for views. (#17901)

  • +
+
+
+

Iceberg connector#

+
    +
  • Disallow using the root directory of a bucket (scheme://authority) as a +table location without a trailing slash in the location name. (#17921)

  • +
  • Fix Parquet writer incompatibility with Apache Spark and Databricks Runtime. (#17978)

  • +
  • Fix scheduling failure when dynamic filtering is enabled. (#17871)

  • +
+
+
+

Kafka connector#

+
    +
  • Fix server startup failure when a Kafka catalog is present. (#17299)

  • +
+
+
+

MongoDB connector#

+
    +
  • Add support for ALTER TABLE ... RENAME COLUMN. (#17874)

  • +
  • Fix incorrect results when the order of the +dbref type +fields is different from databaseName, collectionName, and id. (#17883)

  • +
+
+
+

SPI#

+
    +
  • Move table function infrastructure to the io.trino.spi.function.table +package. (#17774)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-421.html b/430/release/release-421.html new file mode 100644 index 000000000..fc6d50160 --- /dev/null +++ b/430/release/release-421.html @@ -0,0 +1,2667 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 421 (6 Jul 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 421 (6 Jul 2023)#

+
+

General#

+
    +
  • Add support for check constraints in an UPDATE statement. (#17195)

  • +
  • Improve performance for queries involving a year function within an IN +predicate. (#18092)

  • +
  • Fix failure when cancelling a query with a window function. (#18061)

  • +
  • Fix failure for queries involving the concat_ws function on arrays with more +than 254 values. (#17816)

  • +
  • Fix query failure or incorrect results when coercing a +structural data type that contains a timestamp. (#17900)

  • +
+
+
+

JDBC driver#

+
    +
  • Add support for using an alternative hostname with the hostnameInCertificate +property when SSL verification is set to FULL. (#17939)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Add support for check constraints and column invariants in UPDATE +statements. (#17195)

  • +
  • Add support for creating tables with the column mapping mode. (#12638)

  • +
  • Add support for using the OPTIMIZE procedure on column mapping tables. (#17527)

  • +
  • Add support for DROP COLUMN. (#15792)

  • +
+
+
+

Google Sheets connector#

+ +
+
+

Hive connector#

+
    +
  • Add Hive partition projection column properties to the output of +SHOW CREATE TABLE. (#18076)

  • +
  • Fix incorrect query results when using S3 Select with IS NULL or +IS NOT NULL predicates. (#17563)

  • +
  • Fix incorrect query results when using S3 Select and a table’s null_format +field is set. (#17563)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add support for migrating a bucketed Hive table into a non-bucketed Iceberg +table. (#18103)

  • +
+
+
+

Kafka connector#

+
    +
  • Add support for reading Protobuf messages containing the Any Protobuf type. +This is disabled by default and can be enabled by setting the +kafka.protobuf-any-support-enabled configuration property to true. (#17394)

  • +
+
+
+

MongoDB connector#

+
    +
  • Improve query performance on tables with row columns when only a subset of +fields is needed for the query. (#17710)

  • +
+
+
+

Redshift connector#

+ +
+
+

SPI#

+
    +
  • Add the BLOCK_AND_POSITION_NOT_NULL argument convention. (#18035)

  • +
  • Add the BLOCK_BUILDER return convention that writes function results +directly to a BlockBuilder. (#18094)

  • +
  • Add the READ_VALUE operator that can read a value from any argument +convention to any return convention. (#18094)

  • +
  • Remove write methods from the BlockBuilder interface. (#17342)

  • +
  • Change array, map, and row build to use a single writeEntry. (#17342)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-422.html b/430/release/release-422.html new file mode 100644 index 000000000..f4e9786f0 --- /dev/null +++ b/430/release/release-422.html @@ -0,0 +1,2650 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 422 (13 Jul 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 422 (13 Jul 2023)#

+
+

General#

+
    +
  • Add support for adding nested fields with an ADD COLUMN statement. (#16248)

  • +
  • Improve performance of INSERT and CREATE TABLE AS ... SELECT queries. (#18005)

  • +
  • Prevent queries from hanging when worker nodes fail and the +task.retry-policy configuration property is set to TASK. (#18175)

  • +
+
+
+

Security#

+
    +
  • Add support for validating JWT types with OAuth 2.0 authentication. (#17640)

  • +
  • Fix error when the http-server.authentication.type configuration property +is set to oauth2 or jwt and the principal-field property’s value +differs. (#18210)

  • +
+
+
+

BigQuery connector#

+
    +
  • Add support for writing to columns with a timestamp(p) with time zone type. (#17793)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Add support for renaming columns. (#15821)

  • +
  • Improve performance of reading from tables with a large number of +checkpoints. (#17405)

  • +
  • Disallow using the vacuum procedure when the max +writer version +is above 5. (#18095)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for reading the timestamp with local time zone Hive type. (#1240)

  • +
  • Add a native Avro file format writer. This can be disabled with the +avro.native-writer.enabled configuration property or the +avro_native_writer_enabled session property. (#18064)

  • +
  • Fix query failure when the hive.recursive-directories configuration property +is set to true and partition names contain non-alphanumeric characters. (#18167)

  • +
  • Fix incorrect results when reading text and RCTEXT files with a value that +contains the character that separates fields. (#18215)

  • +
  • Fix incorrect results when reading concatenated GZIP compressed text files. (#18223)

  • +
  • Fix incorrect results when reading large text and sequence files with a single +header row. (#18255)

  • +
  • Fix incorrect reporting of bytes read for compressed text files. (#1828)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add support for adding nested fields with an ADD COLUMN statement. (#16248)

  • +
  • Add support for the register_table procedure to register Hadoop tables. (#16363)

  • +
  • Change the default file format to Parquet. The iceberg.file-format +catalog configuration property can be used to specify a different default file +format. (#18170)

  • +
  • Improve performance of reading row types from Parquet files. (#17387)

  • +
  • Fix failure when writing to tables sorted on UUID or TIME types. (#18136)

  • +
+
+
+

Kudu connector#

+
    +
  • Add support for table comments when creating tables. (#17945)

  • +
+
+
+

Redshift connector#

+
    +
  • Prevent returning incorrect results by throwing an error when encountering +unsupported types. Previously, the query would fall back to the legacy type +mapping. (#18209)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-423.html b/430/release/release-423.html new file mode 100644 index 000000000..ea2c116f1 --- /dev/null +++ b/430/release/release-423.html @@ -0,0 +1,2801 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 423 (10 Aug 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 423 (10 Aug 2023)#

+
+

General#

+
    +
  • Add support for renaming nested fields in a column via RENAME COLUMN. (#16757)

  • +
  • Add support for setting the type of a nested field in a column via SET DATA TYPE. (#16959)

  • +
  • Add support for comments on materialized view columns. (#18016)

  • +
  • Add support for displaying all Unicode characters in string literals. (#5061)

  • +
  • Improve performance of INSERT and CREATE TABLE AS ... SELECT queries. (#18212)

  • +
  • Improve performance when planning queries involving multiple window functions. (#18491)

  • +
  • Improve performance of queries involving BETWEEN clauses. (#18501)

  • +
  • Improve performance of queries containing redundant ORDER BY clauses in +views or WITH clauses. This may affect the semantics of queries that +incorrectly rely on implementation-specific behavior. The old behavior can be +restored via the skip_redundant_sort session property or the +optimizer.skip-redundant-sort configuration property. (#18159)

  • +
  • Reduce default values for the task.partitioned-writer-count and +task.scale-writers.max-writer-count configuration properties to reduce the +memory requirements of queries that write data. (#18488)

  • +
  • Remove the deprecated optimizer.use-mark-distinct configuration property, +which has been replaced with optimizer.mark-distinct-strategy. (#18540)

  • +
  • Fix query planning failure due to dynamic filters in +fault tolerant execution mode. (#18383)

  • +
  • Fix EXPLAIN failure when a query contains WHERE ... IN (NULL). (#18328)

  • +
+
+
+

JDBC driver#

+ +
+
+

CLI#

+
    +
  • Add support for accepting a single Trino JDBC URL with parameters as an +alternative to passing command line arguments. (#12587)

  • +
+
+
+

ClickHouse connector#

+
    +
  • Add support for CASCADE option in DROP SCHEMA statements. (#18305)

  • +
+
+
+

Blackhole connector#

+
    +
  • Add support for the COMMENT ON VIEW statement. (#18516)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Add $properties system table which can be queried to inspect Delta Lake +table properties. (#17294)

  • +
  • Add support for reading the timestamp_ntz type. (#17502)

  • +
  • Add support for writing the timestamp with time zone type on partitioned +columns. (#16822)

  • +
  • Add option to enforce that a filter on a partition key is present for +query processing. This can be enabled by setting the +delta.query-partition-filter-required configuration property or the +query_partition_filter_required session property to true. +(#18345)

  • +
  • Improve performance of the $history system table. (#18427)

  • +
  • Improve memory accounting of the Parquet writer. (#18564)

  • +
  • Allow metadata changes on Delta Lake tables with +identity columns. (#18200)

  • +
  • Fix incorrectly creating files smaller than the configured +file_size_threshold as part of OPTIMIZE. (#18388)

  • +
  • Fix query failure when a table has a file with a location ending with +whitespace. (#18206)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for varchar to timestamp coercion in Hive tables. (#18014)

  • +
  • Improve memory accounting of the Parquet writer. (#18564)

  • +
  • Remove the legacy Parquet writer, along with the +parquet.optimized-writer.enabled configuration property and the +parquet_optimized_writer_enabled session property. Replace the +parquet.optimized-writer.validation-percentage configuration property with +parquet.writer.validation-percentage. (#18420)

  • +
  • Disallow coercing Hive timestamp types to varchar for dates before 1900. (#18004)

  • +
  • Fix loss of data precision when coercing Hive timestamp values. (#18003)

  • +
  • Fix incorrectly creating files smaller than the configured +file_size_threshold as part of OPTIMIZE. (#18388)

  • +
  • Fix query failure when a table has a file with a location ending with +whitespace. (#18206)

  • +
  • Fix incorrect results when using S3 Select and a query predicate includes a +quote character (") or a decimal column. (#17775)

  • +
  • Add the hive.s3select-pushdown.experimental-textfile-pushdown-enabled +configuration property to enable S3 Select pushdown for TEXTFILE tables. (#17775)

  • +
+
+
+

Hudi connector#

+
    +
  • Fix query failure when a table has a file with a location ending with +whitespace. (#18206)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add support for renaming nested fields in a column via RENAME COLUMN. (#16757)

  • +
  • Add support for setting the type of a nested field in a column via +SET DATA TYPE. (#16959)

  • +
  • Add support for comments on materialized view columns. (#18016)

  • +
  • Add support for tinyint and smallint types in the migrate procedure. (#17946)

  • +
  • Add support for reading Parquet files with time stored in millisecond precision. (#18535)

  • +
  • Improve performance of information_schema.columns queries for tables managed +by Trino with AWS Glue as metastore. (#18315)

  • +
  • Improve performance of system.metadata.table_comments when querying Iceberg +tables backed by AWS Glue as metastore. (#18517)

  • +
  • Improve performance of information_schema.columns when using the Glue +catalog. (#18586)

  • +
  • Improve memory accounting of the Parquet writer. (#18564)

  • +
  • Fix incorrectly creating files smaller than the configured +file_size_threshold as part of OPTIMIZE. (#18388)

  • +
  • Fix query failure when a table has a file with a location ending with +whitespace. (#18206)

  • +
  • Fix failure when creating a materialized view on a table which has been +rolled back. (#18205)

  • +
  • Fix query failure when reading ORC files with nullable time columns. (#15606)

  • +
  • Fix failure to calculate query statistics when referring to $path as part of +a WHERE clause. (#18330)

  • +
  • Fix write conflict detection for UPDATE, DELETE, and MERGE operations. +In rare situations this issue may have resulted in duplicate rows when +multiple operations were run at the same time, or at the same time as an +optimize procedure. (#18533)

  • +
+
+
+

Kafka connector#

+
    +
  • Rename the ADD_DUMMY value for the kafka.empty-field-strategy +configuration property and the empty_field_strategy session property to +MARK (#18485).

  • +
+
+
+

Kudu connector#

+
    +
  • Add support for optimized local scheduling of splits. (#18121)

  • +
+
+
+

MariaDB connector#

+
    +
  • Add support for CASCADE option in DROP SCHEMA statements. (#18305)

  • +
+
+
+

MongoDB connector#

+
    +
  • Add support for predicate pushdown on char and decimal type. (#18382)

  • +
+
+
+

MySQL connector#

+
    +
  • Add support for predicate pushdown for =, <>, IN, NOT IN, and LIKE +operators on case-sensitive varchar and nvarchar columns. (#18140, #18441)

  • +
  • Add support for CASCADE option in DROP SCHEMA statements. (#18305)

  • +
+
+
+

Oracle connector#

+
    +
  • Add support for Oracle timestamp types with non-millisecond precision. (#17934)

  • +
  • Add support for CASCADE option in DROP SCHEMA statements. (#18305)

  • +
+
+
+

SingleStore connector#

+
    +
  • Add support for CASCADE option in DROP SCHEMA statements. (#18305)

  • +
+
+
+

SPI#

+
    +
  • Deprecate the ConnectorMetadata.getTableHandle(ConnectorSession, SchemaTableName) +method signature. Connectors should implement +ConnectorMetadata.getTableHandle(ConnectorSession, SchemaTableName, Optional, Optional) +instead. (#18596)

  • +
  • Remove the deprecated supportsReportingWrittenBytes method from +ConnectorMetadata. (#18617)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-424.html b/430/release/release-424.html new file mode 100644 index 000000000..163ba4f59 --- /dev/null +++ b/430/release/release-424.html @@ -0,0 +1,2651 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 424 (17 Aug 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 424 (17 Aug 2023)#

+
+

General#

+
    +
  • Reduce coordinator overhead on large clusters. (#18542)

  • +
  • Require the JVM default charset to be UTF-8. This can be set with the JVM +command line option -Dfile.encoding=UTF-8. (#18657)

  • +
+
+
+

JDBC driver#

+
    +
  • Add the number of bytes that have been written to the query results response. (#18651)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Remove the legacy Parquet reader, along with the +parquet.optimized-reader.enabled and +parquet.optimized-nested-reader.enabled configuration properties. (#18639)

  • +
+
+
+

Hive connector#

+
    +
  • Improve performance for line-oriented Hive formats. (#18703)

  • +
  • Improve performance of reading JSON files. (#18709)

  • +
  • Remove the legacy Parquet reader, along with the +parquet.optimized-reader.enabled and +parquet.optimized-nested-reader.enabled configuration properties. (#18639)

  • +
  • Fix incorrect reporting of written bytes for uncompressed text files, which +prevented the target_max_file_size session property from working. (#18701)

  • +
+
+
+

Hudi connector#

+
    +
  • Remove the legacy Parquet reader, along with the +parquet.optimized-reader.enabled and +parquet.optimized-nested-reader.enabled configuration properties. (#18639)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add support for CASCADE option in DROP SCHEMA statements. (#18689)

  • +
  • Remove the legacy Parquet reader, along with the +parquet.optimized-reader.enabled and +parquet.optimized-nested-reader.enabled configuration properties. (#18639)

  • +
  • Fix potential incorrect query results when a query involves a predicate on a +timestamp with time zone column. (#18588)

  • +
+
+
+

Memory connector#

+
    +
  • Add support for CASCADE option in DROP SCHEMA statements. (#18668)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Add support for CASCADE option in DROP SCHEMA statements. (#18663)

  • +
  • Remove support for Postgres versions older than +version 11. (#18696)

  • +
+
+
+

SPI#

+
    +
  • Introduce the getNewTableWriterScalingOptions and +getInsertWriterScalingOptions methods to ConnectorMetadata, which enable +connectors to limit writer scaling. (#18561)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-425.html b/430/release/release-425.html new file mode 100644 index 000000000..dd56a7072 --- /dev/null +++ b/430/release/release-425.html @@ -0,0 +1,2621 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 425 (24 Aug 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 425 (24 Aug 2023)#

+
+

General#

+
    +
  • Improve performance of GROUP BY. (#18106)

  • +
  • Fix incorrect reporting of cumulative memory usage. (#18714)

  • +
+
+
+

BlackHole connector#

+
    +
  • Remove support for materialized views. (#18628)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Add support for check constraints in MERGE statements. (#15411)

  • +
  • Improve performance when statistics are missing from the transaction log. (#16743)

  • +
  • Improve memory usage accounting of the Parquet writer. (#18756)

  • +
  • Improve performance of DELETE statements when they delete the whole table or +when the filters only apply to partition columns. (#18332)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for CASCADE option in DROP SCHEMA statements. (#18320)

  • +
  • Create a new directory if the specified external location for a new table does +not exist. (#17920)

  • +
  • Improve memory usage accounting of the Parquet writer. (#18756)

  • +
  • Improve performance of writing to JSON files. (#18683)

  • +
+
+
+

Iceberg connector#

+
    +
  • Improve memory usage accounting of the Parquet writer. (#18756)

  • +
+
+
+

Kudu connector#

+
    +
  • Add support for CASCADE option in DROP SCHEMA statements. (#18629)

  • +
+
+
+

MongoDB connector#

+
    +
  • Add support for the Decimal128 MongoDB type. (#18722)

  • +
  • Add support for CASCADE option in DROP SCHEMA statements. (#18629)

  • +
  • Fix query failure when reading the value of -0 as a decimal type. (#18777)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-426.html b/430/release/release-426.html new file mode 100644 index 000000000..4b753552d --- /dev/null +++ b/430/release/release-426.html @@ -0,0 +1,2624 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 426 (5 Sep 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Release 426 (5 Sep 2023)#

+
+

General#

+
    +
  • Add support for SET SESSION AUTHORIZATION and RESET SESSION AUTHORIZATION. (#16067)

  • +
  • Add support for automatic type coercion when creating tables. (#13994)

  • +
  • Improve performance of aggregations over decimal values. (#18868)

  • +
  • Fix event listener incorrectly reporting output columns for UPDATE +statements with subqueries. (#18815)

  • +
  • Fix failure when performing an outer join involving geospatial functions in +the join clause. (#18860)

  • +
  • Fix failure when querying partitioned tables with a WHERE clause that +contains lambda expressions. (#18865)

  • +
  • Fix failure for GROUP BY queries over map and array types. (#18863)

  • +
+
+
+

Security#

+
    +
  • Fix authentication failure with OAuth 2.0 when authentication tokens are +larger than 4 KB. (#18836)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Add support for the TRUNCATE TABLE statement. (#18786)

  • +
  • Add support for the CASCADE option in DROP SCHEMA statements. (#18333)

  • +
  • Add support for +Databricks 13.3 LTS. (#18888)

  • +
  • Fix writing an incorrect transaction log for partitioned tables with an id +or name column mapping mode. (#18661)

  • +
+
+
+

Hive connector#

+
    +
  • Add the hive.metastore.thrift.batch-fetch.enabled configuration property, +which can be set to false to disable batch metadata fetching from the Hive +metastore. (#18111)

  • +
  • Fix ANALYZE failure when row count stats are missing. (#18798)

  • +
  • Fix the hive.target-max-file-size configuration property being ignored +when writing to sorted tables. (#18653)

  • +
  • Fix query failure when reading large SequenceFile, RCFile, or Avro files. (#18837)

  • +
+
+
+

Iceberg connector#

+
    +
  • Fix the iceberg.target-max-file-size configuration property being ignored +when writing to sorted tables. (#18653)

  • +
+
+
+

SPI#

+
    +
  • Remove the deprecated +ConnectorMetadata#dropSchema(ConnectorSession session, String schemaName) +method. (#18839)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-427.html b/430/release/release-427.html new file mode 100644 index 000000000..8bbe1cbbd --- /dev/null +++ b/430/release/release-427.html @@ -0,0 +1,2747 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 427 (26 Sep 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Release 427 (26 Sep 2023)#

+
+

General#

+
    +
  • Add support for comparing IPv4 and IPv6 addresses and CIDRs with contains. (#18497)

  • +
  • Improve performance of GROUP BY and DISTINCT. (#19059)

  • +
  • Reduce coordinator memory footprint when scannning tables. (#19009)

  • +
  • Fix failure due to exceeding node memory limits with INSERT statements. (#18771)

  • +
  • Fix query hang for certain LIKE patterns involving a mix of % and _. (#19146)

  • +
+
+
+

Security#

+
    +
  • Ensure authorization is checked when accessing table comments with table redirections. (#18514)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Add support for reading tables with +Deletion Vectors. (#16903)

  • +
  • Add support for Delta Lake writer +version 7. (#15873)

  • +
  • Add support for writing columns with the timestamp(p) type. (#16927)

  • +
  • Reduce data read from Parquet files for queries with filters. (#19032)

  • +
  • Improve performance of writing to Parquet files. (#19122)

  • +
  • Fix error reading Delta Lake table history when the initial transaction logs +have been removed. (#18845)

  • +
+
+
+

Elasticsearch connector#

+
    +
  • Fix query failure when a LIKE clause contains multi-byte characters. (#18966)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for changing column comments when using the Glue catalog. (#19076)

  • +
  • Reduce data read from Parquet files for queries with filters. (#19032)

  • +
  • Improve performance of reading text files. (#18959)

  • +
  • Allow changing a column’s type from double to varchar in Hive tables. (#18930)

  • +
  • Remove legacy Hive readers and writers. The *_native_reader_enabled and +*_native_writer_enabled session properties and *.native-reader.enabled and +*.native-writer.enabled configuration properties are removed. (#18241)

  • +
  • Remove support for S3 Select. The s3_select_pushdown_enabled session +property and the hive.s3select* configuration properties are removed. (#18241)

  • +
  • Remove support for disabling optimized symlink listing. The +optimize_symlink_listing session property and +hive.optimize-symlink-listing configuration property are removed. (#18241)

  • +
  • Fix incompatibility with Hive OpenCSV deserialization. As a result, when the +escape character is explicitly set to ", a \ (backslash) must be used +instead. (#18918)

  • +
  • Fix performance regression when reading CSV files on AWS S3. (#18976)

  • +
  • Fix failure when creating a table with a varchar(0) column. (#18811)

  • +
+
+
+

Hudi connector#

+
    +
  • Fix query failure when reading from Hudi tables with +instants that have been +replaced. (#18213)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add support for usage of date and timestamp arguments in FOR TIMESTAMP AS OF expressions. (#14214)

  • +
  • Add support for using tags with AS OF VERSION queries. (#19111)

  • +
  • Reduce data read from Parquet files for queries with filters. (#19032)

  • +
  • Improve performance of writing to Parquet files. (#19090)

  • +
  • Improve performance of reading tables with many equality delete files. (#17114)

  • +
+
+
+

Ignite connector#

+
    +
  • Add support for UPDATE. (#16445)

  • +
+
+
+

MariaDB connector#

+
    +
  • Add support for UPDATE. (#16445)

  • +
+
+
+

MongoDB connector#

+
    +
  • Fix query failure when mapping MongoDB Decimal128 values with leading zeros. (#19068)

  • +
+
+
+

MySQL connector#

+
    +
  • Add support for UPDATE. (#16445)

  • +
  • Change mapping for MySQL TIMESTAMP types from timestamp(n) to +timestamp(n) with time zone. (#18470)

  • +
+
+
+

Oracle connector#

+
    +
  • Add support for UPDATE. (#16445)

  • +
  • Fix potential query failure when joins are pushed down to Oracle. (#18924)

  • +
+
+
+

PostgreSQL connector#

+
    +
  • Add support for UPDATE. (#16445)

  • +
+
+
+

Redshift connector#

+
    +
  • Add support for UPDATE. (#16445)

  • +
+
+
+

SingleStore connector#

+
    +
  • Add support for UPDATE. (#16445)

  • +
+
+
+

SQL Server connector#

+
    +
  • Add support for UPDATE. (#16445)

  • +
+
+
+

SPI#

+
    +
  • Change BlockBuilder to no longer extend Block. (#18738)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-428.html b/430/release/release-428.html new file mode 100644 index 000000000..b68e9e6b7 --- /dev/null +++ b/430/release/release-428.html @@ -0,0 +1,2634 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 428 (4 Oct 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+ +
+
+ +
+
+ +
+

Release 428 (4 Oct 2023)#

+
+

General#

+
    +
  • Reduce memory usage for queries involving GROUP BY clauses. (#19187)

  • +
  • Simplify writer count configuration. Add the new task.min-writer-count +and task.max-writer-count configuration properties along with the +task_min_writer_count and task_max_writer_count session properties, which +control the number of writers depending on scenario. Deprecate the +task.writer-count, task.scale-writers.max-writer-count, and +task.partitioned-writer-count configuration properties, which will be +removed in the future. Remove the task_writer_count, +task_partitioned_writer_count, and task_scale_writers_max_writer_count +session properties. (#19135)

  • +
  • Remove support for the parse-decimal-literals-as-double legacy configuration +property. (#19166)

  • +
  • Fix out of memory error when running queries with GROUP BY clauses. (#19119)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Reduce the number of read requests for scanning small Parquet files. Add the +parquet.small-file-threshold configuration property and the +parquet_small_file_threshold session property to change the default size of +3MB, below which, files will be read in their entirety. Setting this +configuration to 0B disables the feature. (#19127)

  • +
  • Fix potential data duplication when running OPTIMIZE coincides with +updates to a table. (#19128)

  • +
  • Fix error when deleting rows in tables that have partitions with certain +non-alphanumeric characters in their names. (#18922)

  • +
+
+
+

Hive connector#

+
    +
  • Reduce the number of read requests for scanning small Parquet files. Add the +parquet.small-file-threshold configuration property and the +parquet_small_file_threshold session property to change the default size of +3MB, below which, files will be read in their entirety. Setting this +configuration to 0B disables the feature. (#19127)

  • +
+
+
+

Hudi connector#

+
    +
  • Reduce the number of read requests for scanning small Parquet files. Add the +parquet.small-file-threshold configuration property and the +parquet_small_file_threshold session property to change the default size of +3MB, below which, files will be read in their entirety. Setting this +configuration to 0B disables the feature. (#19127)

  • +
+
+
+

Iceberg connector#

+
    +
  • Reduce the number of read requests for scanning small Parquet files. Add the +parquet.small-file-threshold configuration property and the +parquet_small_file_threshold session property to change the default size of +3MB, below which, files will be read in their entirety. Setting this +configuration to 0B disables the feature. (#19127)

  • +
  • Fix incorrect column statistics for the Parquet file format in manifest files. (#19052)

  • +
+
+
+

Pinot connector#

+ +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-429.html b/430/release/release-429.html new file mode 100644 index 000000000..88d889404 --- /dev/null +++ b/430/release/release-429.html @@ -0,0 +1,2645 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 429 (11 Oct 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+ +
+
+ +
+
+ +
+

Release 429 (11 Oct 2023)#

+
+

General#

+ +
+
+

Security#

+
    +
  • Stop performing security checks for functions in the system.builtin schema. (#19160)

  • +
  • Remove support for using function kind as a rule in file-based access control. (#19160)

  • +
+
+
+

Web UI#

+
    +
  • Log out from a Trino OAuth session when logging out from the Web UI. (#13060)

  • +
+
+
+

Delta Lake connector#

+
    +
  • Allow using the # and ? characters in S3 location paths or URLs. (#19296)

  • +
+
+
+

Hive connector#

+
    +
  • Add support for changing a column’s type from varchar to date. (#19201)

  • +
  • Add support for changing a column’s type from decimal to tinyint, +smallint, integer, or bigint in partitioned Hive tables. (#19201)

  • +
  • Improve performance of reading ORC files. (#19295)

  • +
  • Allow using the # and ? characters in S3 location paths or URLs. (#19296)

  • +
  • Fix error reading Avro files when a schema has uppercase characters in its +name. (#19249)

  • +
+
+
+

Hudi connector#

+
    +
  • Allow using the # and ? characters in S3 location paths or URLs. (#19296)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add support for specifying timestamp precision as part of +CREATE TABLE AS .. SELECT statements. (#13981)

  • +
  • Improve performance of reading ORC files. (#19295)

  • +
  • Allow using the # and ? characters in S3 location paths or URLs. (#19296)

  • +
+
+
+

MongoDB connector#

+
    +
  • Fix mixed case schema names being inaccessible when using custom roles and +the case-insensitive-name-matching configuration property is enabled. (#19218)

  • +
+
+
+

SPI#

+
    +
  • Change function security checks to return a boolean instead of throwing an +exception. (#19160)

  • +
  • Add SQL path field to ConnectorViewDefinition, +ConnectorMaterializedViewDefinition, and ViewExpression. (#19160)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/release/release-430.html b/430/release/release-430.html new file mode 100644 index 000000000..121dedb1e --- /dev/null +++ b/430/release/release-430.html @@ -0,0 +1,2640 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Release 430 (20 Oct 2023) — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+ +
+ +
+
+ +
+

Release 430 (20 Oct 2023)#

+
+

General#

+
    +
  • Improve performance of queries with GROUP BY. (#19302)

  • +
  • Fix incorrect results for queries involving ORDER BY and window functions +with ordered frames. (#19399)

  • +
  • Fix incorrect results for query involving an aggregation in a correlated +subquery. (#19002)

  • +
+
+
+

Security#

+
    +
  • Enforce authorization capability of client when receiving commands RESET and +SET for SESSION AUTHORIZATION. (#19217)

  • +
+
+
+

JDBC driver#

+
    +
  • Add support for a timezone parameter to set the session timezone. (#19102)

  • +
+
+
+

Iceberg connector#

+
    +
  • Add an option to require filters on partition columns. This can be enabled by +setting the iceberg.query-partition-filter-required configuration property +or the query_partition_filter_required session property. (#17263)

  • +
  • Improve performance when reading partition columns. (#19303)

  • +
+
+
+

Ignite connector#

+
    +
  • Fix failure when a query contains LIKE with ESCAPE. (#19464)

  • +
+
+
+

MariaDB connector#

+
    +
  • Add support for table statistics. (#19408)

  • +
+
+
+

MongoDB connector#

+
    +
  • Fix incorrect results when a query contains several <> or NOT IN +predicates. (#19404)

  • +
+
+
+

Oracle connector#

+
    +
  • Improve reliability of connecting to the source database. (#19191)

  • +
+
+
+

SPI#

+
    +
  • Change the Java stack type for a map value to SqlMap and a row value to +SqlRow, which do not implement Block. (#18948)

  • +
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/search.html b/430/search.html new file mode 100644 index 000000000..dd917b6fb --- /dev/null +++ b/430/search.html @@ -0,0 +1,388 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Search — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +

Search

+
+ +

+ Please activate JavaScript to enable the search + functionality. +

+
+ +
+ +
+ +
+
+
+
+
+
+ + +
+ + + + \ No newline at end of file diff --git a/430/searchindex.js b/430/searchindex.js new file mode 100644 index 000000000..1618ca3e2 --- /dev/null +++ b/430/searchindex.js @@ -0,0 +1 @@ +Search.setIndex({"docnames": ["admin", "admin/dist-sort", "admin/dynamic-filtering", "admin/event-listeners-http", "admin/event-listeners-mysql", "admin/fault-tolerant-execution", "admin/graceful-shutdown", "admin/jmx", "admin/properties", "admin/properties-exchange", "admin/properties-general", "admin/properties-http-client", "admin/properties-logging", "admin/properties-node-scheduler", "admin/properties-optimizer", "admin/properties-query-management", "admin/properties-regexp-function", "admin/properties-resource-management", "admin/properties-spilling", "admin/properties-task", "admin/properties-web-interface", "admin/properties-write-partitioning", "admin/properties-writer-scaling", "admin/resource-groups", "admin/session-property-managers", "admin/spill", "admin/tuning", "admin/web-interface", "appendix", "appendix/from-hive", "appendix/legal-notices", "client", "client/cli", "client/jdbc", "connector", "connector/accumulo", "connector/atop", "connector/bigquery", "connector/blackhole", "connector/cassandra", "connector/clickhouse", "connector/delta-lake", "connector/druid", "connector/elasticsearch", "connector/googlesheets", "connector/hive", "connector/hive-alluxio", "connector/hive-azure", "connector/hive-caching", "connector/hive-cos", "connector/hive-gcs-tutorial", "connector/hive-s3", "connector/hive-security", "connector/hudi", "connector/iceberg", "connector/ignite", "connector/jmx", "connector/kafka", "connector/kafka-tutorial", "connector/kinesis", "connector/kudu", "connector/localfile", "connector/mariadb", "connector/memory", "connector/metastores", "connector/mongodb", "connector/mysql", "connector/object-storage-file-formats", "connector/oracle", "connector/phoenix", "connector/pinot", "connector/postgresql", "connector/prometheus", "connector/redis", "connector/redshift", "connector/singlestore", "connector/sqlserver", "connector/system", "connector/thrift", "connector/tpcds", "connector/tpch", "develop", "develop/certificate-authenticator", "develop/client-protocol", "develop/connectors", "develop/event-listener", "develop/example-http", "develop/example-jdbc", "develop/functions", "develop/group-provider", "develop/header-authenticator", "develop/insert", "develop/password-authenticator", "develop/spi-overview", "develop/supporting-merge", "develop/system-access-control", "develop/table-functions", "develop/types", "functions", "functions/aggregate", "functions/array", "functions/binary", "functions/bitwise", "functions/color", "functions/comparison", "functions/conditional", "functions/conversion", "functions/datetime", "functions/decimal", "functions/geospatial", "functions/hyperloglog", "functions/ipaddress", "functions/json", "functions/lambda", "functions/list", "functions/list-by-topic", "functions/logical", "functions/map", "functions/math", "functions/ml", "functions/qdigest", "functions/regexp", "functions/session", "functions/setdigest", "functions/string", "functions/system", "functions/table", "functions/tdigest", "functions/teradata", "functions/url", "functions/uuid", "functions/window", "glossary", "index", "installation", "installation/containers", "installation/deployment", "installation/kubernetes", "installation/query-resiliency", "installation/rpm", "language", "language/comments", "language/reserved", "language/sql-support", "language/types", "optimizer", "optimizer/cost-based-optimizations", "optimizer/cost-in-explain", "optimizer/pushdown", "optimizer/statistics", "overview", "overview/concepts", "overview/use-cases", "release", "release/release-0.100", "release/release-0.101", "release/release-0.102", "release/release-0.103", "release/release-0.104", "release/release-0.105", "release/release-0.106", "release/release-0.107", "release/release-0.108", "release/release-0.109", "release/release-0.110", "release/release-0.111", "release/release-0.112", "release/release-0.113", "release/release-0.114", "release/release-0.115", "release/release-0.116", "release/release-0.117", "release/release-0.118", "release/release-0.119", "release/release-0.120", "release/release-0.121", "release/release-0.122", "release/release-0.123", "release/release-0.124", "release/release-0.125", "release/release-0.126", "release/release-0.127", "release/release-0.128", "release/release-0.129", "release/release-0.130", "release/release-0.131", "release/release-0.132", "release/release-0.133", "release/release-0.134", "release/release-0.135", "release/release-0.136", "release/release-0.137", "release/release-0.138", "release/release-0.139", "release/release-0.140", "release/release-0.141", "release/release-0.142", "release/release-0.143", "release/release-0.144", "release/release-0.144.1", "release/release-0.144.2", "release/release-0.144.3", "release/release-0.144.4", "release/release-0.144.5", "release/release-0.144.6", "release/release-0.144.7", "release/release-0.145", "release/release-0.146", "release/release-0.147", "release/release-0.148", "release/release-0.149", "release/release-0.150", "release/release-0.151", "release/release-0.152", "release/release-0.152.1", "release/release-0.152.2", "release/release-0.152.3", "release/release-0.153", "release/release-0.154", "release/release-0.155", "release/release-0.156", "release/release-0.157", "release/release-0.157.1", "release/release-0.158", "release/release-0.159", "release/release-0.160", "release/release-0.161", "release/release-0.162", "release/release-0.163", "release/release-0.164", "release/release-0.165", "release/release-0.166", "release/release-0.167", "release/release-0.168", "release/release-0.169", "release/release-0.170", "release/release-0.171", "release/release-0.172", "release/release-0.173", "release/release-0.174", "release/release-0.175", "release/release-0.176", "release/release-0.177", "release/release-0.178", "release/release-0.179", "release/release-0.180", "release/release-0.181", "release/release-0.182", "release/release-0.183", "release/release-0.184", "release/release-0.185", "release/release-0.186", "release/release-0.187", "release/release-0.188", "release/release-0.189", "release/release-0.190", "release/release-0.191", "release/release-0.192", "release/release-0.193", "release/release-0.194", "release/release-0.195", "release/release-0.196", "release/release-0.197", "release/release-0.198", "release/release-0.199", "release/release-0.200", "release/release-0.201", "release/release-0.202", "release/release-0.203", "release/release-0.204", "release/release-0.205", "release/release-0.206", "release/release-0.207", "release/release-0.208", "release/release-0.209", "release/release-0.210", "release/release-0.211", "release/release-0.212", "release/release-0.213", "release/release-0.214", "release/release-0.215", "release/release-0.54", "release/release-0.55", "release/release-0.56", "release/release-0.57", "release/release-0.58", "release/release-0.59", "release/release-0.60", "release/release-0.61", "release/release-0.62", "release/release-0.63", "release/release-0.64", "release/release-0.65", "release/release-0.66", "release/release-0.67", "release/release-0.68", "release/release-0.69", "release/release-0.70", "release/release-0.71", "release/release-0.72", "release/release-0.73", "release/release-0.74", "release/release-0.75", "release/release-0.76", "release/release-0.77", "release/release-0.78", "release/release-0.79", "release/release-0.80", "release/release-0.81", "release/release-0.82", "release/release-0.83", "release/release-0.84", "release/release-0.85", "release/release-0.86", "release/release-0.87", "release/release-0.88", "release/release-0.89", "release/release-0.90", "release/release-0.91", "release/release-0.92", "release/release-0.93", "release/release-0.94", "release/release-0.95", "release/release-0.96", "release/release-0.97", "release/release-0.98", "release/release-0.99", "release/release-300", "release/release-301", "release/release-302", "release/release-303", "release/release-304", "release/release-305", "release/release-306", "release/release-307", "release/release-308", "release/release-309", "release/release-310", "release/release-311", "release/release-312", "release/release-313", "release/release-314", "release/release-315", "release/release-316", "release/release-317", "release/release-318", "release/release-319", "release/release-320", "release/release-321", "release/release-322", "release/release-323", "release/release-324", "release/release-325", "release/release-326", "release/release-327", "release/release-328", "release/release-329", "release/release-330", "release/release-331", "release/release-332", "release/release-333", "release/release-334", "release/release-335", "release/release-336", "release/release-337", "release/release-338", "release/release-339", "release/release-340", "release/release-341", "release/release-342", "release/release-343", "release/release-344", "release/release-345", "release/release-346", "release/release-347", "release/release-348", "release/release-349", "release/release-350", "release/release-351", "release/release-352", "release/release-353", "release/release-354", "release/release-355", "release/release-356", "release/release-357", "release/release-358", "release/release-359", "release/release-360", "release/release-361", "release/release-362", "release/release-363", "release/release-364", "release/release-365", "release/release-366", "release/release-367", "release/release-368", "release/release-369", "release/release-370", "release/release-371", "release/release-372", "release/release-373", "release/release-374", "release/release-375", "release/release-376", "release/release-377", "release/release-378", "release/release-379", "release/release-380", "release/release-381", "release/release-382", "release/release-383", "release/release-384", "release/release-385", "release/release-386", "release/release-387", "release/release-388", "release/release-389", "release/release-390", "release/release-391", "release/release-392", "release/release-393", "release/release-394", "release/release-395", "release/release-396", "release/release-397", "release/release-398", "release/release-399", "release/release-400", "release/release-401", "release/release-402", "release/release-403", "release/release-404", "release/release-405", "release/release-406", "release/release-407", "release/release-408", "release/release-409", "release/release-410", "release/release-411", "release/release-412", "release/release-413", "release/release-414", "release/release-415", "release/release-416", "release/release-417", "release/release-418", "release/release-419", "release/release-420", "release/release-421", "release/release-422", "release/release-423", "release/release-424", "release/release-425", "release/release-426", "release/release-427", "release/release-428", "release/release-429", "release/release-430", "security", "security/authentication-types", "security/built-in-system-access-control", "security/certificate", "security/file-system-access-control", "security/group-file", "security/inspect-jks", "security/inspect-pem", "security/internal-communication", "security/jwt", "security/kerberos", "security/ldap", "security/oauth2", "security/overview", "security/password-file", "security/salesforce", "security/secrets", "security/tls", "security/user-mapping", "sql", "sql/alter-materialized-view", "sql/alter-schema", "sql/alter-table", "sql/alter-view", "sql/analyze", "sql/call", "sql/comment", "sql/commit", "sql/create-materialized-view", "sql/create-role", "sql/create-schema", "sql/create-table", "sql/create-table-as", "sql/create-view", "sql/deallocate-prepare", "sql/delete", "sql/deny", "sql/describe", "sql/describe-input", "sql/describe-output", "sql/drop-materialized-view", "sql/drop-role", "sql/drop-schema", "sql/drop-table", "sql/drop-view", "sql/execute", "sql/execute-immediate", "sql/explain", "sql/explain-analyze", "sql/grant", "sql/grant-roles", "sql/insert", "sql/match-recognize", "sql/merge", "sql/pattern-recognition-in-window", "sql/prepare", "sql/refresh-materialized-view", "sql/reset-session", "sql/reset-session-authorization", "sql/revoke", "sql/revoke-roles", "sql/rollback", "sql/select", "sql/set-path", "sql/set-role", "sql/set-session", "sql/set-session-authorization", "sql/set-time-zone", "sql/show-catalogs", "sql/show-columns", "sql/show-create-materialized-view", "sql/show-create-schema", "sql/show-create-table", "sql/show-create-view", "sql/show-functions", "sql/show-grants", "sql/show-role-grants", "sql/show-roles", "sql/show-schemas", "sql/show-session", "sql/show-stats", "sql/show-tables", "sql/start-transaction", "sql/truncate", "sql/update", "sql/use", "sql/values"], "filenames": ["admin.md", "admin/dist-sort.md", "admin/dynamic-filtering.md", "admin/event-listeners-http.md", "admin/event-listeners-mysql.md", "admin/fault-tolerant-execution.md", "admin/graceful-shutdown.md", "admin/jmx.md", "admin/properties.md", "admin/properties-exchange.md", "admin/properties-general.md", "admin/properties-http-client.md", "admin/properties-logging.md", "admin/properties-node-scheduler.md", "admin/properties-optimizer.md", "admin/properties-query-management.md", "admin/properties-regexp-function.md", "admin/properties-resource-management.md", "admin/properties-spilling.md", "admin/properties-task.md", "admin/properties-web-interface.md", "admin/properties-write-partitioning.md", "admin/properties-writer-scaling.md", "admin/resource-groups.md", "admin/session-property-managers.md", "admin/spill.md", "admin/tuning.md", "admin/web-interface.md", "appendix.md", "appendix/from-hive.md", "appendix/legal-notices.md", "client.md", "client/cli.md", "client/jdbc.md", "connector.md", "connector/accumulo.md", "connector/atop.md", "connector/bigquery.md", "connector/blackhole.md", "connector/cassandra.md", "connector/clickhouse.md", "connector/delta-lake.md", "connector/druid.md", "connector/elasticsearch.md", "connector/googlesheets.md", "connector/hive.md", "connector/hive-alluxio.md", "connector/hive-azure.md", "connector/hive-caching.md", "connector/hive-cos.md", "connector/hive-gcs-tutorial.md", "connector/hive-s3.md", "connector/hive-security.md", "connector/hudi.md", "connector/iceberg.md", "connector/ignite.md", "connector/jmx.md", "connector/kafka.md", "connector/kafka-tutorial.md", "connector/kinesis.md", "connector/kudu.md", "connector/localfile.md", "connector/mariadb.md", "connector/memory.md", "connector/metastores.md", "connector/mongodb.md", "connector/mysql.md", "connector/object-storage-file-formats.md", "connector/oracle.md", "connector/phoenix.md", "connector/pinot.md", "connector/postgresql.md", "connector/prometheus.md", "connector/redis.md", "connector/redshift.md", "connector/singlestore.md", "connector/sqlserver.md", "connector/system.md", "connector/thrift.md", "connector/tpcds.md", "connector/tpch.md", "develop.md", "develop/certificate-authenticator.md", "develop/client-protocol.md", "develop/connectors.md", "develop/event-listener.md", "develop/example-http.md", "develop/example-jdbc.md", "develop/functions.md", "develop/group-provider.md", "develop/header-authenticator.md", "develop/insert.md", "develop/password-authenticator.md", "develop/spi-overview.md", "develop/supporting-merge.md", "develop/system-access-control.md", "develop/table-functions.md", "develop/types.md", "functions.md", "functions/aggregate.md", "functions/array.md", "functions/binary.md", "functions/bitwise.md", "functions/color.md", "functions/comparison.md", "functions/conditional.md", "functions/conversion.md", "functions/datetime.md", "functions/decimal.md", "functions/geospatial.md", "functions/hyperloglog.md", "functions/ipaddress.md", "functions/json.md", "functions/lambda.md", "functions/list.md", "functions/list-by-topic.md", "functions/logical.md", "functions/map.md", "functions/math.md", "functions/ml.md", "functions/qdigest.md", "functions/regexp.md", "functions/session.md", "functions/setdigest.md", "functions/string.md", "functions/system.md", "functions/table.md", "functions/tdigest.md", "functions/teradata.md", "functions/url.md", "functions/uuid.md", "functions/window.md", "glossary.md", "index.md", "installation.md", "installation/containers.md", "installation/deployment.md", "installation/kubernetes.md", "installation/query-resiliency.md", "installation/rpm.md", "language.md", "language/comments.md", "language/reserved.md", "language/sql-support.md", "language/types.md", "optimizer.md", "optimizer/cost-based-optimizations.md", "optimizer/cost-in-explain.md", "optimizer/pushdown.md", "optimizer/statistics.md", "overview.md", "overview/concepts.md", "overview/use-cases.md", "release.md", "release/release-0.100.md", "release/release-0.101.md", "release/release-0.102.md", "release/release-0.103.md", "release/release-0.104.md", "release/release-0.105.md", "release/release-0.106.md", "release/release-0.107.md", "release/release-0.108.md", "release/release-0.109.md", "release/release-0.110.md", "release/release-0.111.md", "release/release-0.112.md", "release/release-0.113.md", "release/release-0.114.md", "release/release-0.115.md", "release/release-0.116.md", "release/release-0.117.md", "release/release-0.118.md", "release/release-0.119.md", "release/release-0.120.md", "release/release-0.121.md", "release/release-0.122.md", "release/release-0.123.md", "release/release-0.124.md", "release/release-0.125.md", "release/release-0.126.md", "release/release-0.127.md", "release/release-0.128.md", "release/release-0.129.md", "release/release-0.130.md", "release/release-0.131.md", "release/release-0.132.md", "release/release-0.133.md", "release/release-0.134.md", "release/release-0.135.md", "release/release-0.136.md", "release/release-0.137.md", "release/release-0.138.md", "release/release-0.139.md", "release/release-0.140.md", "release/release-0.141.md", "release/release-0.142.md", "release/release-0.143.md", "release/release-0.144.md", "release/release-0.144.1.md", "release/release-0.144.2.md", "release/release-0.144.3.md", "release/release-0.144.4.md", "release/release-0.144.5.md", "release/release-0.144.6.md", "release/release-0.144.7.md", "release/release-0.145.md", "release/release-0.146.md", "release/release-0.147.md", "release/release-0.148.md", "release/release-0.149.md", "release/release-0.150.md", "release/release-0.151.md", "release/release-0.152.md", "release/release-0.152.1.md", "release/release-0.152.2.md", "release/release-0.152.3.md", "release/release-0.153.md", "release/release-0.154.md", "release/release-0.155.md", "release/release-0.156.md", "release/release-0.157.md", "release/release-0.157.1.md", "release/release-0.158.md", "release/release-0.159.md", "release/release-0.160.md", "release/release-0.161.md", "release/release-0.162.md", "release/release-0.163.md", "release/release-0.164.md", "release/release-0.165.md", "release/release-0.166.md", "release/release-0.167.md", "release/release-0.168.md", "release/release-0.169.md", "release/release-0.170.md", "release/release-0.171.md", "release/release-0.172.md", "release/release-0.173.md", "release/release-0.174.md", "release/release-0.175.md", "release/release-0.176.md", "release/release-0.177.md", "release/release-0.178.md", "release/release-0.179.md", "release/release-0.180.md", "release/release-0.181.md", "release/release-0.182.md", "release/release-0.183.md", "release/release-0.184.md", "release/release-0.185.md", "release/release-0.186.md", "release/release-0.187.md", "release/release-0.188.md", "release/release-0.189.md", "release/release-0.190.md", "release/release-0.191.md", "release/release-0.192.md", "release/release-0.193.md", "release/release-0.194.md", "release/release-0.195.md", "release/release-0.196.md", "release/release-0.197.md", "release/release-0.198.md", "release/release-0.199.md", "release/release-0.200.md", "release/release-0.201.md", "release/release-0.202.md", "release/release-0.203.md", "release/release-0.204.md", "release/release-0.205.md", "release/release-0.206.md", "release/release-0.207.md", "release/release-0.208.md", "release/release-0.209.md", "release/release-0.210.md", "release/release-0.211.md", "release/release-0.212.md", "release/release-0.213.md", "release/release-0.214.md", "release/release-0.215.md", "release/release-0.54.md", "release/release-0.55.md", "release/release-0.56.md", "release/release-0.57.md", "release/release-0.58.md", "release/release-0.59.md", "release/release-0.60.md", "release/release-0.61.md", "release/release-0.62.md", "release/release-0.63.md", "release/release-0.64.md", "release/release-0.65.md", "release/release-0.66.md", "release/release-0.67.md", "release/release-0.68.md", "release/release-0.69.md", "release/release-0.70.md", "release/release-0.71.md", "release/release-0.72.md", "release/release-0.73.md", "release/release-0.74.md", "release/release-0.75.md", "release/release-0.76.md", "release/release-0.77.md", "release/release-0.78.md", "release/release-0.79.md", "release/release-0.80.md", "release/release-0.81.md", "release/release-0.82.md", "release/release-0.83.md", "release/release-0.84.md", "release/release-0.85.md", "release/release-0.86.md", "release/release-0.87.md", "release/release-0.88.md", "release/release-0.89.md", "release/release-0.90.md", "release/release-0.91.md", "release/release-0.92.md", "release/release-0.93.md", "release/release-0.94.md", "release/release-0.95.md", "release/release-0.96.md", "release/release-0.97.md", "release/release-0.98.md", "release/release-0.99.md", "release/release-300.md", "release/release-301.md", "release/release-302.md", "release/release-303.md", "release/release-304.md", "release/release-305.md", "release/release-306.md", "release/release-307.md", "release/release-308.md", "release/release-309.md", "release/release-310.md", "release/release-311.md", "release/release-312.md", "release/release-313.md", "release/release-314.md", "release/release-315.md", "release/release-316.md", "release/release-317.md", "release/release-318.md", "release/release-319.md", "release/release-320.md", "release/release-321.md", "release/release-322.md", "release/release-323.md", "release/release-324.md", "release/release-325.md", "release/release-326.md", "release/release-327.md", "release/release-328.md", "release/release-329.md", "release/release-330.md", "release/release-331.md", "release/release-332.md", "release/release-333.md", "release/release-334.md", "release/release-335.md", "release/release-336.md", "release/release-337.md", "release/release-338.md", "release/release-339.md", "release/release-340.md", "release/release-341.md", "release/release-342.md", "release/release-343.md", "release/release-344.md", "release/release-345.md", "release/release-346.md", "release/release-347.md", "release/release-348.md", "release/release-349.md", "release/release-350.md", "release/release-351.md", "release/release-352.md", "release/release-353.md", "release/release-354.md", "release/release-355.md", "release/release-356.md", "release/release-357.md", "release/release-358.md", "release/release-359.md", "release/release-360.md", "release/release-361.md", "release/release-362.md", "release/release-363.md", "release/release-364.md", "release/release-365.md", "release/release-366.md", "release/release-367.md", "release/release-368.md", "release/release-369.md", "release/release-370.md", "release/release-371.md", "release/release-372.md", "release/release-373.md", "release/release-374.md", "release/release-375.md", "release/release-376.md", "release/release-377.md", "release/release-378.md", "release/release-379.md", "release/release-380.md", "release/release-381.md", "release/release-382.md", "release/release-383.md", "release/release-384.md", "release/release-385.md", "release/release-386.md", "release/release-387.md", "release/release-388.md", "release/release-389.md", "release/release-390.md", "release/release-391.md", "release/release-392.md", "release/release-393.md", "release/release-394.md", "release/release-395.md", "release/release-396.md", "release/release-397.md", "release/release-398.md", "release/release-399.md", "release/release-400.md", "release/release-401.md", "release/release-402.md", "release/release-403.md", "release/release-404.md", "release/release-405.md", "release/release-406.md", "release/release-407.md", "release/release-408.md", "release/release-409.md", "release/release-410.md", "release/release-411.md", "release/release-412.md", "release/release-413.md", "release/release-414.md", "release/release-415.md", "release/release-416.md", "release/release-417.md", "release/release-418.md", "release/release-419.md", "release/release-420.md", "release/release-421.md", "release/release-422.md", "release/release-423.md", "release/release-424.md", "release/release-425.md", "release/release-426.md", "release/release-427.md", "release/release-428.md", "release/release-429.md", "release/release-430.md", "security.md", "security/authentication-types.md", "security/built-in-system-access-control.md", "security/certificate.md", "security/file-system-access-control.md", "security/group-file.md", "security/inspect-jks.md", "security/inspect-pem.md", "security/internal-communication.md", "security/jwt.md", "security/kerberos.md", "security/ldap.md", "security/oauth2.md", "security/overview.md", "security/password-file.md", "security/salesforce.md", "security/secrets.md", "security/tls.md", "security/user-mapping.md", "sql.md", "sql/alter-materialized-view.md", "sql/alter-schema.md", "sql/alter-table.md", "sql/alter-view.md", "sql/analyze.md", "sql/call.md", "sql/comment.md", "sql/commit.md", "sql/create-materialized-view.md", "sql/create-role.md", "sql/create-schema.md", "sql/create-table.md", "sql/create-table-as.md", "sql/create-view.md", "sql/deallocate-prepare.md", "sql/delete.md", "sql/deny.md", "sql/describe.md", "sql/describe-input.md", "sql/describe-output.md", "sql/drop-materialized-view.md", "sql/drop-role.md", "sql/drop-schema.md", "sql/drop-table.md", "sql/drop-view.md", "sql/execute.md", "sql/execute-immediate.md", "sql/explain.md", "sql/explain-analyze.md", "sql/grant.md", "sql/grant-roles.md", "sql/insert.md", "sql/match-recognize.md", "sql/merge.md", "sql/pattern-recognition-in-window.md", "sql/prepare.md", "sql/refresh-materialized-view.md", "sql/reset-session.md", "sql/reset-session-authorization.rst", "sql/revoke.md", "sql/revoke-roles.md", "sql/rollback.md", "sql/select.md", "sql/set-path.md", "sql/set-role.md", "sql/set-session.md", "sql/set-session-authorization.rst", "sql/set-time-zone.md", "sql/show-catalogs.md", "sql/show-columns.md", "sql/show-create-materialized-view.md", "sql/show-create-schema.md", "sql/show-create-table.md", "sql/show-create-view.md", "sql/show-functions.md", "sql/show-grants.md", "sql/show-role-grants.md", "sql/show-roles.md", "sql/show-schemas.md", "sql/show-session.md", "sql/show-stats.md", "sql/show-tables.md", "sql/start-transaction.md", "sql/truncate.md", "sql/update.md", "sql/use.md", "sql/values.md"], "titles": ["Administration", "Distributed sort", "Dynamic filtering", "HTTP event listener", "MySQL event listener", "Fault-tolerant execution", "Graceful shutdown", "Monitoring with JMX", "Properties reference", "Exchange properties", "General properties", "HTTP client properties", "Logging properties", "Node scheduler properties", "Optimizer properties", "Query management properties", "Regular expression function properties", "Resource management properties", "Spilling properties", "Task properties", "Web UI properties", "Write partitioning properties", "Writer scaling properties", "Resource groups", "Session property managers", "Spill to disk", "Tuning Trino", "Web UI", "Appendix", "Migrating from Hive", "Legal notices", "Clients", "Command line interface", "JDBC driver", "Connectors", "Accumulo connector", "Atop connector", "BigQuery connector", "Black Hole connector", "Cassandra connector", "ClickHouse connector", "Delta Lake connector", "Druid connector", "Elasticsearch connector", "Google Sheets connector", "Hive connector", "Hive connector with Alluxio", "Hive connector with Azure Storage", "Hive connector storage caching", "Hive connector with IBM Cloud Object Storage", "Google Cloud Storage", "Hive connector with Amazon S3", "Hive connector security configuration", "Hudi connector", "Iceberg connector", "Ignite connector", "JMX connector", "Kafka connector", "Kafka connector tutorial", "Kinesis connector", "Kudu connector", "Local file connector", "MariaDB connector", "Memory connector", "Metastores", "MongoDB connector", "MySQL connector", "Object storage file formats", "Oracle connector", "Phoenix connector", "Pinot connector", "PostgreSQL connector", "Prometheus connector", "Redis connector", "Redshift connector", "SingleStore connector", "SQL Server connector", "System connector", "Thrift connector", "TPCDS connector", "TPCH connector", "Developer guide", "Certificate authenticator", "Trino client REST API", "Connectors", "Event listener", "Example HTTP connector", "Example JDBC connector", "Functions", "Group provider", "Header authenticator", "Supporting INSERT and CREATE TABLE AS", "Password authenticator", "SPI overview", "Supporting MERGE", "System access control", "Table functions", "Types", "Functions and operators", "Aggregate functions", "Array functions and operators", "Binary functions and operators", "Bitwise functions", "Color functions", "Comparison functions and operators", "Conditional expressions", "Conversion functions", "Date and time functions and operators", "Decimal functions and operators", "Geospatial functions", "HyperLogLog functions", "IP Address Functions", "JSON functions and operators", "Lambda expressions", "List of functions and operators", "List of functions by topic", "Logical operators", "Map functions and operators", "Mathematical functions and operators", "Machine learning functions", "Quantile digest functions", "Regular expression functions", "Session information", "Set Digest functions", "String functions and operators", "System information", "Table functions", "T-Digest functions", "Teradata functions", "URL functions", "UUID functions", "Window functions", "Glossary", "Trino documentation", "Installation", "Trino in a Docker container", "Deploying Trino", "Trino on Kubernetes with Helm", "Improve query processing resilience", "RPM package", "SQL language", "Comments", "Keywords and identifiers", "SQL statement support", "Data types", "Query optimizer", "Cost-based optimizations", "Cost in EXPLAIN", "Pushdown", "Table statistics", "Overview", "Trino concepts", "Use cases", "Release notes", "Release 0.100", "Release 0.101", "Release 0.102", "Release 0.103", "Release 0.104", "Release 0.105", "Release 0.106", "Release 0.107", "Release 0.108", "Release 0.109", "Release 0.110", "Release 0.111", "Release 0.112", "Release 0.113", "Release 0.114", "Release 0.115", "Release 0.116", "Release 0.117", "Release 0.118", "Release 0.119", "Release 0.120", "Release 0.121", "Release 0.122", "Release 0.123", "Release 0.124", "Release 0.125", "Release 0.126", "Release 0.127", "Release 0.128", "Release 0.129", "Release 0.130", "Release 0.131", "Release 0.132", "Release 0.133", "Release 0.134", "Release 0.135", "Release 0.136", "Release 0.137", "Release 0.138", "Release 0.139", "Release 0.140", "Release 0.141", "Release 0.142", "Release 0.143", "Release 0.144", "Release 0.144.1", "Release 0.144.2", "Release 0.144.3", "Release 0.144.4", "Release 0.144.5", "Release 0.144.6", "Release 0.144.7", "Release 0.145", "Release 0.146", "Release 0.147", "Release 0.148", "Release 0.149", "Release 0.150", "Release 0.151", "Release 0.152", "Release 0.152.1", "Release 0.152.2", "Release 0.152.3", "Release 0.153", "Release 0.154", "Release 0.155", "Release 0.156", "Release 0.157", "Release 0.157.1", "Release 0.158", "Release 0.159", "Release 0.160", "Release 0.161", "Release 0.162", "Release 0.163", "Release 0.164", "Release 0.165", "Release 0.166", "Release 0.167", "Release 0.168", "Release 0.169", "Release 0.170", "Release 0.171", "Release 0.172", "Release 0.173", "Release 0.174", "Release 0.175", "Release 0.176", "Release 0.177", "Release 0.178", "Release 0.179", "Release 0.180", "Release 0.181", "Release 0.182", "Release 0.183", "Release 0.184", "Release 0.185", "Release 0.186", "Release 0.187", "Release 0.188", "Release 0.189", "Release 0.190", "Release 0.191", "Release 0.192", "Release 0.193", "Release 0.194", "Release 0.195", "Release 0.196", "Release 0.197", "Release 0.198", "Release 0.199", "Release 0.200", "Release 0.201", "Release 0.202", "Release 0.203", "Release 0.204", "Release 0.205", "Release 0.206", "Release 0.207", "Release 0.208", "Release 0.209", "Release 0.210", "Release 0.211", "Release 0.212", "Release 0.213", "Release 0.214", "Release 0.215", "Release 0.54", "Release 0.55", "Release 0.56", "Release 0.57", "Release 0.58", "Release 0.59", "Release 0.60", "Release 0.61", "Release 0.62", "Release 0.63", "Release 0.64", "Release 0.65", "Release 0.66", "Release 0.67", "Release 0.68", "Release 0.69", "Release 0.70", "Release 0.71", "Release 0.72", "Release 0.73", "Release 0.74", "Release 0.75", "Release 0.76", "Release 0.77", "Release 0.78", "Release 0.79", "Release 0.80", "Release 0.81", "Release 0.82", "Release 0.83", "Release 0.84", "Release 0.85", "Release 0.86", "Release 0.87", "Release 0.88", "Release 0.89", "Release 0.90", "Release 0.91", "Release 0.92", "Release 0.93", "Release 0.94", "Release 0.95", "Release 0.96", "Release 0.97", "Release 0.98", "Release 0.99", "Release 300 (22 Jan 2019)", "Release 301 (31 Jan 2019)", "Release 302 (6 Feb 2019)", "Release 303 (13 Feb 2019)", "Release 304 (27 Feb 2019)", "Release 305 (7 Mar 2019)", "Release 306 (16 Mar 2019)", "Release 307 (3 Apr 2019)", "Release 308 (11 Apr 2019)", "Release 309 (25 Apr 2019)", "Release 310 (3 May 2019)", "Release 311 (14 May 2019)", "Release 312 (29 May 2019)", "Release 313 (31 May 2019)", "Release 314 (7 Jun 2019)", "Release 315 (14 Jun 2019)", "Release 316 (8 Jul 2019)", "Release 317 (1 Aug 2019)", "Release 318 (26 Aug 2019)", "Release 319 (22 Sep 2019)", "Release 320 (10 Oct 2019)", "Release 321 (15 Oct 2019)", "Release 322 (16 Oct 2019)", "Release 323 (23 Oct 2019)", "Release 324 (1 Nov 2019)", "Release 325 (14 Nov 2019)", "Release 326 (27 Nov 2019)", "Release 327 (20 Dec 2019)", "Release 328 (10 Jan 2020)", "Release 329 (23 Jan 2020)", "Release 330 (18 Feb 2020)", "Release 331 (16 Mar 2020)", "Release 332 (08 Apr 2020)", "Release 333 (04 May 2020)", "Release 334 (29 May 2020)", "Release 335 (14 Jun 2020)", "Release 336 (16 Jun 2020)", "Release 337 (25 Jun 2020)", "Release 338 (07 Jul 2020)", "Release 339 (21 Jul 2020)", "Release 340 (8 Aug 2020)", "Release 341 (8 Sep 2020)", "Release 342 (24 Sep 2020)", "Release 343 (25 Sep 2020)", "Release 344 (9 Oct 2020)", "Release 345 (23 Oct 2020)", "Release 346 (10 Nov 2020)", "Release 347 (25 Nov 2020)", "Release 348 (14 Dec 2020)", "Release 349 (28 Dec 2020)", "Release 350 (28 Dec 2020)", "Release 351 (3 Jan 2021)", "Release 352 (9 Feb 2021)", "Release 353 (5 Mar 2021)", "Release 354 (19 Mar 2021)", "Release 355 (8 Apr 2021)", "Release 356 (30 Apr 2021)", "Release 357 (21 May 2021)", "Release 358 (1 Jun 2021)", "Release 359 (1 Jul 2021)", "Release 360 (30 Jul 2021)", "Release 361 (27 Aug 2021)", "Release 362 (20 Sep 2021)", "Release 363 (6 Oct 2021)", "Release 364 (1 Nov 2021)", "Release 365 (3 Dec 2021)", "Release 366 (14 Dec 2021)", "Release 367 (22 Dec 2021)", "Release 368 (11 Jan 2022)", "Release 369 (24 Jan 2022)", "Release 370 (3 Feb 2022)", "Release 371 (16 Feb 2022)", "Release 372 (2 Mar 2022)", "Release 373 (9 Mar 2022)", "Release 374 (17 Mar 2022)", "Release 375 (28 Mar 2022)", "Release 376 (7 Apr 2022)", "Release 377 (13 Apr 2022)", "Release 378 (21 Apr 2022)", "Release 379 (28 Apr 2022)", "Release 380 (6 May 2022)", "Release 381 (16 May 2022)", "Release 382 (25 May 2022)", "Release 383 (1 Jun 2022)", "Release 384 (3 Jun 2022)", "Release 385 (8 Jun 2022)", "Release 386 (15 Jun 2022)", "Release 387 (22 Jun 2022)", "Release 388 (29 Jun 2022)", "Release 389 (7 Jul 2022)", "Release 390 (13 Jul 2022)", "Release 391 (22 Jul 2022)", "Release 392 (3 Aug 2022)", "Release 393 (17 Aug 2022)", "Release 394 (29 Aug 2022)", "Release 395 (7 Sep 2022)", "Release 396 (15 Sep 2022)", "Release 397 (21 Sep 2022)", "Release 398 (28 Sep 2022)", "Release 399 (6 Oct 2022)", "Release 400 (13 Oct 2022)", "Release 401 (26 Oct 2022)", "Release 402 (2 Nov 2022)", "Release 403 (15 Nov 2022)", "Release 404 (???)", "Release 405 (28 Dec 2022)", "Release 406 (25 Jan 2023)", "Release 407 (16 Feb 2023)", "Release 408 (23 Feb 2023)", "Release 409 (3 Mar 2023)", "Release 410 (8 Mar 2023)", "Release 411 (29 Mar 2023)", "Release 412 (5 Apr 2023)", "Release 413 (12 Apr 2023)", "Release 414 (19 Apr 2023)", "Release 415 (28 Apr 2023)", "Release 416 (3 May 2023)", "Release 417 (10 May 2023)", "Release 418 (17 May 2023)", "Release 419 (5 Jun 2023)", "Release 420 (22 Jun 2023)", "Release 421 (6 Jul 2023)", "Release 422 (13 Jul 2023)", "Release 423 (10 Aug 2023)", "Release 424 (17 Aug 2023)", "Release 425 (24 Aug 2023)", "Release 426 (5 Sep 2023)", "Release 427 (26 Sep 2023)", "Release 428 (4 Oct 2023)", "Release 429 (11 Oct 2023)", "Release 430 (20 Oct 2023)", "Security", "Authentication types", "System access control", "Certificate authentication", "File-based access control", "File group provider", "JKS files", "PEM files", "Secure internal communication", "JWT authentication", "Kerberos authentication", "LDAP authentication", "OAuth 2.0 authentication", "Security overview", "Password file authentication", "Salesforce authentication", "Secrets", "TLS and HTTPS", "User mapping", "SQL statement syntax", "ALTER MATERIALIZED VIEW", "ALTER SCHEMA", "ALTER TABLE", "ALTER VIEW", "ANALYZE", "CALL", "COMMENT", "COMMIT", "CREATE MATERIALIZED VIEW", "CREATE ROLE", "CREATE SCHEMA", "CREATE TABLE", "CREATE TABLE AS", "CREATE VIEW", "DEALLOCATE PREPARE", "DELETE", "DENY", "DESCRIBE", "DESCRIBE INPUT", "DESCRIBE OUTPUT", "DROP MATERIALIZED VIEW", "DROP ROLE", "DROP SCHEMA", "DROP TABLE", "DROP VIEW", "EXECUTE", "EXECUTE IMMEDIATE", "EXPLAIN", "EXPLAIN ANALYZE", "GRANT", "GRANT ROLES", "INSERT", "MATCH_RECOGNIZE", "MERGE", "Row pattern recognition in window structures", "PREPARE", "REFRESH MATERIALIZED VIEW", "RESET SESSION", "RESET SESSION AUTHORIZATION", "REVOKE", "REVOKE ROLES", "ROLLBACK", "SELECT", "SET PATH", "SET ROLE", "SET SESSION", "SET SESSION AUTHORIZATION", "SET TIME ZONE", "SHOW CATALOGS", "SHOW COLUMNS", "SHOW CREATE MATERIALIZED VIEW", "SHOW CREATE SCHEMA", "SHOW CREATE TABLE", "SHOW CREATE VIEW", "SHOW FUNCTIONS", "SHOW GRANTS", "SHOW ROLE GRANTS", "SHOW ROLES", "SHOW SCHEMAS", "SHOW SESSION", "SHOW STATS", "SHOW TABLES", "START TRANSACTION", "TRUNCATE", "UPDATE", "USE", "VALUES"], "terms": {"web": [0, 2, 8, 15, 32, 33, 37, 40, 45, 54, 57, 59, 62, 66, 68, 69, 71, 73, 74, 75, 76, 123, 133, 136, 158, 176, 180, 186, 187, 188, 189, 190, 210, 211, 212, 213, 364, 372, 420, 466, 467, 470, 471, 472, 473, 475, 476, 479, 482, 488, 500], "ui": [0, 2, 8, 15, 33, 133, 136, 158, 162, 183, 186, 189, 190, 191, 194, 197, 211, 213, 282, 289, 290, 291, 307, 364, 420, 466, 470, 471, 472, 473, 475], "tune": [0, 8, 9, 16, 73, 133, 137, 183, 262, 278, 396], "trino": [0, 1, 2, 3, 4, 5, 6, 8, 9, 10, 11, 12, 13, 14, 17, 19, 22, 23, 24, 25, 27, 29, 30, 31, 32, 33, 34, 36, 38, 45, 47, 48, 49, 51, 52, 53, 55, 56, 57, 59, 61, 63, 64, 73, 74, 77, 79, 80, 81, 82, 84, 85, 86, 87, 88, 89, 90, 91, 92, 93, 94, 95, 96, 97, 98, 104, 106, 109, 110, 112, 119, 120, 123, 125, 126, 127, 132, 134, 138, 140, 142, 143, 146, 148, 149, 150, 183, 284, 334, 336, 378, 379, 380, 381, 383, 384, 386, 387, 388, 389, 391, 392, 393, 395, 396, 397, 398, 399, 400, 408, 413, 415, 417, 418, 419, 420, 421, 423, 426, 430, 432, 434, 435, 438, 444, 445, 447, 450, 456, 459, 460, 461, 462, 463, 464, 465, 466, 467, 471, 472, 473, 474, 476, 477, 494, 505, 506, 516, 520, 523, 524], "monitor": [0, 13, 19, 23, 25, 27, 36, 56, 133, 136, 296, 475], "jmx": [0, 19, 23, 34, 41, 48, 133, 136, 151, 155, 166, 167, 178, 186, 196, 213, 221, 232, 255, 288, 289, 296, 313, 323, 339, 343, 360, 364, 367, 373, 377, 379, 389, 405, 417], "properti": [0, 1, 2, 7, 25, 27, 29, 32, 33, 38, 48, 49, 56, 58, 63, 77, 79, 80, 82, 83, 84, 85, 87, 89, 90, 92, 93, 95, 96, 121, 126, 133, 135, 137, 138, 139, 143, 144, 146, 151, 154, 155, 156, 157, 161, 163, 166, 170, 172, 178, 180, 183, 184, 186, 187, 191, 192, 193, 196, 198, 206, 207, 208, 209, 210, 211, 213, 217, 219, 220, 221, 225, 226, 227, 231, 233, 235, 239, 240, 241, 242, 244, 245, 246, 247, 248, 251, 253, 254, 255, 256, 257, 259, 260, 261, 262, 263, 264, 266, 267, 269, 270, 271, 272, 273, 274, 275, 276, 277, 278, 279, 280, 281, 282, 284, 287, 296, 297, 301, 302, 303, 306, 307, 310, 317, 323, 327, 328, 329, 330, 331, 332, 333, 334, 335, 336, 337, 338, 339, 340, 341, 343, 344, 345, 346, 347, 348, 349, 350, 351, 352, 354, 355, 356, 357, 358, 359, 360, 361, 362, 364, 365, 366, 367, 368, 369, 371, 372, 373, 374, 375, 377, 378, 379, 380, 381, 382, 383, 384, 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, 403, 404, 405, 406, 407, 408, 409, 412, 413, 414, 415, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, 432, 433, 434, 435, 436, 437, 438, 439, 441, 443, 444, 445, 446, 447, 448, 449, 450, 451, 453, 454, 455, 456, 457, 459, 460, 463, 464, 466, 471, 472, 473, 474, 475, 476, 482, 486, 488, 489, 490, 515, 520, 525, 537], "refer": [0, 5, 12, 26, 32, 35, 37, 39, 40, 41, 42, 44, 45, 46, 51, 52, 53, 54, 55, 57, 59, 60, 62, 64, 65, 66, 68, 69, 70, 71, 73, 74, 75, 76, 83, 86, 97, 98, 112, 126, 128, 131, 132, 133, 135, 136, 137, 138, 140, 144, 146, 147, 148, 149, 151, 178, 179, 196, 201, 206, 223, 226, 237, 246, 259, 260, 272, 275, 313, 318, 320, 335, 343, 351, 354, 366, 369, 372, 373, 377, 379, 380, 389, 396, 397, 433, 438, 450, 462, 464, 465, 469, 474, 475, 477, 478, 512, 520], "spill": [0, 5, 8, 133, 223, 248, 251, 258, 267, 279, 329, 331, 332, 333, 334, 340, 351, 356, 360, 368, 372, 382, 383, 390, 392, 393, 396, 400, 405, 416, 417, 437, 438], "disk": [0, 11, 18, 27, 45, 48, 52, 133, 223, 248, 251, 282, 287, 340, 356, 360, 362, 372, 382, 416, 417, 437, 520], "resourc": [0, 5, 8, 13, 15, 16, 19, 24, 27, 32, 33, 45, 47, 51, 57, 69, 77, 83, 85, 93, 94, 98, 132, 133, 136, 137, 138, 146, 151, 160, 163, 208, 210, 211, 213, 223, 225, 226, 228, 229, 236, 240, 243, 244, 249, 257, 266, 271, 278, 280, 281, 287, 293, 294, 296, 310, 333, 344, 345, 346, 357, 358, 360, 381, 392, 396, 398, 401, 404, 416, 421, 432, 462, 463], "group": [0, 2, 13, 14, 19, 24, 25, 32, 33, 35, 37, 45, 51, 53, 54, 57, 58, 59, 60, 66, 67, 70, 71, 81, 83, 88, 93, 99, 100, 110, 112, 121, 122, 131, 133, 136, 142, 148, 155, 160, 162, 171, 176, 178, 179, 180, 182, 183, 184, 186, 191, 196, 198, 199, 202, 203, 206, 208, 209, 210, 211, 213, 216, 219, 220, 223, 228, 229, 232, 234, 235, 236, 237, 239, 240, 241, 242, 243, 244, 246, 248, 249, 257, 264, 266, 267, 271, 274, 275, 278, 280, 282, 283, 284, 287, 291, 293, 303, 307, 312, 313, 315, 317, 321, 330, 333, 338, 340, 344, 345, 346, 352, 357, 358, 359, 366, 367, 368, 369, 371, 373, 374, 379, 384, 385, 386, 387, 390, 391, 392, 396, 398, 401, 402, 406, 415, 418, 420, 432, 433, 434, 435, 452, 453, 454, 455, 457, 458, 462, 464, 468, 470, 471, 475, 476, 486, 490, 491, 505, 506, 512, 532], "session": [0, 1, 2, 5, 8, 10, 14, 15, 18, 19, 21, 22, 23, 25, 29, 32, 33, 40, 42, 45, 51, 52, 54, 55, 57, 62, 64, 66, 67, 68, 69, 70, 71, 74, 75, 76, 83, 84, 85, 93, 94, 96, 98, 107, 133, 137, 143, 146, 155, 156, 157, 161, 163, 170, 178, 180, 183, 184, 186, 187, 192, 193, 196, 206, 208, 210, 217, 220, 225, 226, 227, 229, 233, 239, 240, 242, 244, 247, 248, 251, 253, 254, 255, 256, 257, 259, 261, 262, 263, 264, 266, 267, 269, 270, 271, 272, 273, 276, 277, 278, 279, 280, 293, 297, 306, 307, 317, 323, 327, 328, 330, 332, 333, 336, 337, 339, 343, 344, 345, 346, 348, 352, 354, 355, 356, 357, 358, 359, 360, 361, 364, 365, 366, 368, 369, 371, 372, 373, 374, 375, 379, 383, 386, 387, 388, 389, 391, 392, 393, 395, 396, 398, 399, 407, 408, 412, 413, 414, 415, 417, 423, 424, 432, 433, 434, 435, 436, 438, 439, 445, 447, 449, 450, 451, 453, 454, 455, 456, 457, 460, 468, 470, 477, 492, 513, 520, 521, 522, 525, 532, 534, 540, 543], "manag": [0, 7, 8, 13, 27, 37, 40, 42, 43, 44, 47, 48, 51, 56, 59, 64, 66, 71, 74, 76, 84, 89, 93, 133, 136, 137, 138, 139, 151, 158, 163, 217, 223, 232, 236, 243, 247, 253, 259, 261, 262, 276, 279, 287, 293, 296, 328, 339, 344, 345, 357, 361, 364, 365, 372, 377, 392, 398, 427, 432, 434, 450, 459, 462, 463, 464, 465, 466, 470, 474, 475, 487, 499, 508, 518, 522, 542], "distribut": [0, 2, 5, 13, 14, 15, 19, 21, 23, 32, 45, 46, 52, 54, 59, 60, 84, 86, 94, 99, 119, 131, 132, 133, 136, 139, 144, 150, 151, 152, 157, 167, 170, 173, 193, 206, 246, 256, 269, 270, 271, 272, 274, 275, 278, 279, 287, 307, 317, 323, 330, 341, 356, 357, 361, 362, 372, 391, 392, 398, 408, 426, 428, 432, 434, 436, 466, 475, 506], "sort": [0, 17, 18, 25, 40, 45, 71, 84, 94, 100, 118, 133, 148, 151, 155, 187, 210, 223, 255, 267, 269, 271, 273, 274, 279, 280, 339, 358, 365, 368, 380, 381, 385, 386, 387, 408, 436, 439, 449, 450, 453, 510, 520], "dynam": [0, 13, 15, 22, 23, 37, 40, 41, 42, 54, 55, 57, 62, 66, 68, 69, 71, 73, 74, 75, 76, 96, 126, 133, 254, 256, 261, 262, 263, 264, 280, 287, 344, 347, 348, 355, 357, 358, 359, 360, 361, 362, 365, 366, 367, 368, 369, 371, 372, 373, 374, 384, 387, 391, 392, 394, 397, 403, 405, 415, 419, 422, 425, 434, 436, 447, 450, 455], "filter": [0, 10, 23, 29, 35, 37, 39, 40, 41, 42, 44, 54, 55, 62, 64, 66, 67, 68, 69, 70, 71, 73, 74, 75, 76, 84, 100, 114, 115, 133, 146, 147, 148, 151, 166, 184, 202, 208, 209, 210, 217, 220, 224, 229, 232, 233, 238, 239, 244, 249, 254, 258, 263, 265, 266, 267, 268, 269, 270, 274, 277, 278, 279, 280, 315, 327, 331, 332, 341, 342, 344, 347, 348, 351, 354, 355, 357, 358, 359, 360, 361, 365, 366, 368, 369, 371, 372, 374, 375, 379, 381, 382, 384, 387, 388, 389, 390, 391, 392, 394, 396, 397, 399, 402, 403, 405, 408, 409, 413, 414, 416, 417, 418, 419, 420, 421, 422, 425, 429, 432, 433, 434, 438, 442, 446, 447, 450, 452, 454, 457, 491, 506, 520, 526, 527, 532, 536, 537, 539], "grace": [0, 35, 133, 364, 385, 438, 441, 460, 462, 486], "shutdown": [0, 133, 303, 364, 385, 433, 441, 460, 462], "fault": [0, 2, 15, 17, 133, 136, 138, 398, 401, 402, 403, 404, 408, 409, 412, 419, 422, 427, 428, 429, 430, 432, 433, 439, 441, 445, 450], "toler": [0, 2, 15, 17, 133, 136, 138, 228, 398, 401, 402, 403, 404, 408, 409, 412, 419, 422, 427, 428, 429, 430, 432, 433, 439, 441, 445, 450], "execut": [0, 1, 2, 7, 13, 14, 17, 18, 19, 23, 24, 25, 27, 32, 36, 39, 40, 42, 43, 47, 52, 55, 56, 62, 69, 75, 77, 83, 84, 85, 87, 91, 94, 95, 97, 98, 119, 122, 126, 133, 136, 138, 142, 143, 146, 147, 148, 155, 157, 158, 162, 163, 173, 177, 181, 183, 186, 196, 208, 211, 217, 226, 231, 240, 245, 249, 251, 254, 256, 261, 264, 265, 267, 268, 269, 270, 271, 274, 278, 279, 280, 284, 296, 302, 305, 307, 329, 330, 331, 341, 347, 352, 356, 357, 358, 359, 361, 366, 368, 384, 389, 391, 392, 393, 394, 395, 396, 397, 398, 400, 401, 402, 403, 404, 405, 406, 408, 409, 412, 415, 418, 419, 422, 425, 427, 428, 429, 430, 432, 433, 436, 438, 439, 441, 443, 445, 450, 462, 477, 491, 492, 499, 505, 506, 507, 508, 511, 513, 517, 518, 520, 522, 524], "http": [0, 5, 6, 8, 10, 15, 27, 30, 33, 37, 40, 42, 45, 47, 49, 51, 55, 57, 58, 59, 61, 64, 69, 70, 72, 73, 78, 81, 82, 84, 90, 92, 93, 129, 132, 133, 135, 136, 137, 157, 173, 182, 195, 196, 206, 209, 211, 214, 217, 232, 245, 251, 252, 253, 254, 257, 258, 260, 263, 268, 276, 281, 287, 293, 329, 331, 339, 341, 345, 346, 357, 358, 361, 364, 368, 372, 374, 377, 378, 379, 383, 384, 388, 389, 390, 392, 396, 397, 399, 400, 432, 435, 446, 449, 458, 459, 461, 462, 464, 466, 467, 468, 469, 470, 471, 472, 473, 476], "mysql": [0, 5, 23, 29, 34, 45, 54, 133, 138, 151, 152, 184, 334, 349, 352, 353, 355, 357, 358, 360, 374, 375, 377, 384, 462], "allow": [1, 2, 3, 4, 5, 6, 8, 9, 10, 11, 12, 13, 14, 15, 16, 18, 19, 20, 22, 23, 25, 27, 29, 32, 33, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 48, 51, 52, 54, 55, 56, 57, 58, 59, 60, 61, 62, 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 76, 78, 83, 84, 86, 93, 94, 95, 96, 97, 98, 99, 104, 110, 112, 123, 126, 131, 132, 135, 136, 137, 138, 139, 140, 144, 146, 147, 148, 151, 156, 158, 162, 167, 173, 178, 180, 191, 207, 209, 212, 213, 217, 223, 227, 232, 233, 234, 236, 240, 241, 242, 245, 248, 250, 251, 254, 255, 256, 257, 258, 259, 262, 263, 264, 265, 266, 267, 268, 269, 270, 272, 274, 275, 276, 278, 279, 281, 284, 287, 288, 296, 297, 302, 303, 304, 305, 307, 315, 317, 327, 328, 329, 330, 331, 332, 333, 334, 336, 339, 340, 341, 342, 343, 344, 345, 346, 348, 350, 351, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 366, 367, 368, 369, 371, 372, 373, 374, 375, 377, 379, 380, 383, 384, 385, 386, 387, 388, 389, 390, 391, 392, 395, 396, 397, 398, 399, 400, 401, 402, 403, 405, 407, 408, 410, 412, 414, 415, 417, 420, 421, 423, 424, 425, 426, 427, 428, 429, 430, 432, 433, 434, 435, 439, 441, 446, 447, 450, 454, 456, 459, 460, 461, 462, 469, 470, 471, 472, 473, 474, 475, 476, 479, 480, 488, 489, 491, 507, 510, 512, 520, 526, 527, 532, 536, 537, 539], "data": [1, 2, 3, 4, 5, 7, 10, 11, 12, 13, 14, 15, 17, 18, 19, 23, 24, 25, 27, 31, 34, 36, 37, 38, 39, 43, 44, 46, 48, 50, 52, 53, 55, 57, 59, 63, 64, 65, 67, 69, 70, 72, 73, 75, 77, 78, 79, 80, 83, 84, 86, 87, 93, 94, 96, 98, 99, 101, 104, 105, 107, 108, 112, 119, 124, 126, 132, 133, 135, 136, 137, 138, 139, 140, 146, 147, 148, 149, 150, 152, 155, 156, 157, 163, 166, 167, 169, 178, 183, 184, 185, 186, 187, 191, 194, 197, 201, 206, 207, 208, 209, 210, 213, 217, 220, 223, 228, 231, 235, 236, 240, 242, 244, 245, 248, 251, 253, 254, 255, 256, 257, 258, 263, 264, 266, 268, 271, 273, 274, 276, 277, 278, 279, 280, 281, 282, 284, 287, 288, 291, 293, 296, 301, 302, 303, 306, 307, 317, 320, 321, 324, 327, 328, 329, 330, 331, 332, 334, 335, 339, 341, 344, 345, 346, 347, 348, 349, 350, 352, 355, 356, 357, 358, 359, 360, 361, 362, 364, 365, 367, 368, 369, 371, 373, 375, 379, 381, 382, 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, 405, 406, 407, 408, 409, 415, 416, 418, 419, 420, 421, 423, 424, 428, 429, 430, 432, 433, 434, 435, 436, 438, 440, 444, 446, 448, 450, 454, 455, 460, 462, 466, 469, 473, 477, 480, 483, 486, 488, 489, 490, 491, 505, 506, 510, 512, 514, 520, 527, 538], "which": [1, 2, 3, 5, 6, 8, 9, 10, 12, 14, 15, 16, 18, 19, 20, 23, 24, 27, 29, 32, 33, 35, 36, 37, 39, 40, 41, 42, 43, 44, 45, 47, 48, 51, 52, 53, 54, 55, 56, 57, 58, 59, 62, 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 82, 84, 85, 86, 88, 89, 90, 91, 92, 93, 94, 95, 96, 97, 99, 100, 103, 104, 107, 109, 110, 112, 113, 117, 120, 123, 124, 126, 127, 131, 132, 135, 136, 137, 144, 146, 148, 151, 155, 156, 157, 158, 159, 163, 167, 169, 177, 178, 180, 183, 186, 187, 190, 192, 193, 194, 199, 206, 207, 209, 211, 212, 213, 214, 217, 218, 223, 228, 232, 233, 235, 239, 240, 243, 244, 245, 246, 248, 254, 255, 256, 257, 263, 271, 275, 280, 281, 285, 287, 293, 297, 301, 302, 303, 304, 305, 306, 307, 313, 317, 327, 330, 331, 332, 336, 339, 341, 342, 345, 346, 350, 353, 355, 357, 358, 359, 360, 364, 367, 368, 369, 371, 373, 374, 375, 379, 381, 382, 383, 386, 388, 389, 390, 391, 392, 394, 396, 397, 399, 413, 419, 420, 421, 422, 423, 424, 427, 429, 430, 432, 434, 435, 436, 438, 439, 450, 451, 453, 455, 457, 462, 464, 466, 468, 469, 470, 471, 473, 475, 476, 478, 480, 489, 491, 506, 510, 511, 512, 520], "exce": [1, 2, 5, 13, 17, 23, 25, 99, 166, 218, 262, 270, 313, 381, 382, 403, 446, 464, 465, 520], "queri": [1, 2, 3, 4, 6, 8, 9, 13, 19, 22, 23, 24, 25, 31, 32, 33, 35, 38, 41, 45, 46, 48, 51, 52, 53, 57, 59, 61, 63, 64, 72, 73, 78, 79, 80, 84, 85, 86, 87, 88, 93, 94, 95, 96, 98, 99, 100, 104, 105, 106, 107, 110, 112, 119, 122, 125, 126, 129, 131, 132, 133, 134, 136, 140, 144, 146, 147, 148, 149, 150, 152, 154, 155, 157, 158, 159, 160, 161, 162, 165, 166, 167, 169, 170, 172, 173, 176, 177, 178, 180, 181, 183, 184, 186, 187, 188, 189, 190, 191, 192, 193, 194, 195, 196, 197, 198, 200, 201, 202, 205, 206, 208, 209, 210, 211, 212, 213, 214, 217, 218, 219, 220, 221, 222, 223, 224, 225, 226, 227, 228, 229, 231, 232, 233, 234, 235, 236, 237, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, 276, 277, 278, 279, 280, 282, 283, 286, 287, 288, 289, 292, 293, 294, 295, 296, 297, 301, 302, 303, 304, 305, 306, 309, 310, 311, 312, 313, 315, 316, 317, 319, 320, 323, 324, 327, 328, 329, 330, 331, 332, 333, 334, 335, 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, 377, 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, 408, 409, 410, 411, 412, 413, 414, 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, 442, 444, 445, 446, 447, 448, 449, 450, 451, 452, 453, 454, 455, 457, 466, 467, 469, 471, 472, 473, 475, 482, 483, 486, 488, 489, 490, 491, 496, 497, 503, 504, 505, 506, 509, 510, 511, 512, 513, 514, 520, 521, 524, 526, 527, 532, 536, 537, 538, 539, 544], "max": [1, 2, 3, 5, 22, 23, 27, 36, 37, 40, 41, 43, 44, 45, 51, 53, 54, 55, 56, 62, 63, 64, 66, 67, 68, 69, 70, 71, 72, 76, 99, 104, 108, 113, 114, 115, 146, 154, 157, 161, 163, 167, 170, 197, 198, 200, 206, 209, 210, 221, 224, 228, 231, 233, 239, 242, 251, 255, 257, 258, 259, 260, 262, 263, 264, 265, 267, 268, 270, 272, 273, 275, 277, 278, 279, 282, 297, 302, 303, 307, 321, 325, 327, 339, 341, 355, 364, 365, 366, 367, 368, 369, 372, 373, 377, 379, 383, 384, 388, 389, 391, 395, 396, 398, 403, 408, 410, 413, 416, 419, 420, 426, 427, 430, 432, 434, 438, 444, 449, 450, 453, 455, 472, 506, 520], "memori": [1, 2, 3, 4, 5, 7, 9, 10, 13, 16, 23, 27, 33, 34, 38, 48, 56, 84, 99, 109, 110, 123, 133, 136, 137, 144, 146, 147, 148, 151, 155, 157, 158, 160, 161, 162, 163, 167, 173, 178, 180, 181, 182, 183, 186, 188, 191, 197, 198, 200, 206, 209, 210, 217, 219, 223, 228, 231, 232, 233, 234, 236, 237, 239, 240, 241, 244, 245, 246, 248, 249, 251, 253, 254, 255, 256, 257, 259, 261, 262, 263, 264, 266, 267, 268, 269, 270, 271, 273, 274, 277, 279, 280, 282, 284, 288, 293, 296, 302, 304, 307, 309, 311, 313, 316, 317, 318, 325, 329, 330, 332, 333, 334, 337, 338, 340, 345, 346, 356, 357, 358, 362, 364, 365, 367, 368, 371, 375, 377, 383, 384, 387, 390, 392, 393, 395, 396, 399, 400, 401, 403, 408, 424, 426, 427, 432, 433, 435, 438, 443, 444, 450, 452, 454, 455, 505, 506], "per": [1, 2, 5, 8, 9, 12, 21, 22, 23, 25, 32, 35, 36, 37, 39, 41, 45, 52, 53, 54, 57, 63, 69, 70, 83, 84, 86, 99, 112, 120, 126, 127, 131, 136, 157, 167, 191, 192, 193, 209, 217, 218, 221, 224, 232, 240, 242, 245, 246, 254, 256, 257, 258, 264, 267, 270, 274, 276, 278, 293, 302, 305, 307, 317, 320, 327, 333, 339, 341, 345, 354, 355, 357, 368, 377, 379, 387, 395, 396, 415, 426, 427, 430, 432, 434, 444, 462, 463, 472, 480, 506, 512, 523], "node": [1, 2, 8, 9, 10, 15, 19, 21, 23, 25, 32, 35, 36, 39, 41, 43, 45, 48, 52, 56, 58, 59, 61, 63, 69, 70, 84, 93, 94, 112, 125, 132, 133, 135, 137, 138, 139, 146, 147, 151, 155, 157, 162, 167, 177, 183, 193, 208, 209, 213, 217, 221, 235, 245, 254, 256, 272, 274, 278, 279, 281, 287, 296, 302, 305, 310, 311, 327, 328, 329, 330, 344, 345, 346, 362, 368, 377, 379, 380, 382, 385, 389, 392, 396, 400, 405, 418, 422, 423, 425, 432, 434, 438, 449, 454, 460, 462, 466, 472, 475, 505, 506], "enabl": [1, 2, 3, 5, 6, 7, 10, 17, 21, 23, 24, 25, 27, 32, 33, 35, 37, 39, 40, 41, 42, 43, 44, 45, 48, 51, 52, 53, 54, 55, 56, 60, 62, 63, 64, 66, 67, 68, 69, 70, 71, 74, 75, 76, 82, 84, 85, 90, 92, 99, 119, 121, 136, 138, 146, 148, 155, 157, 163, 167, 170, 172, 173, 177, 178, 180, 183, 186, 187, 192, 193, 196, 207, 209, 212, 217, 220, 223, 227, 230, 232, 233, 236, 240, 242, 245, 246, 247, 251, 253, 254, 255, 256, 257, 258, 260, 261, 263, 264, 266, 269, 271, 272, 273, 276, 278, 280, 297, 301, 302, 304, 305, 306, 307, 310, 311, 317, 323, 324, 327, 328, 329, 331, 332, 336, 337, 339, 340, 342, 344, 345, 346, 347, 348, 349, 351, 352, 354, 355, 356, 357, 358, 359, 360, 361, 362, 364, 365, 366, 368, 369, 371, 372, 373, 374, 375, 379, 380, 381, 383, 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 399, 400, 402, 405, 406, 407, 408, 409, 412, 413, 414, 415, 417, 419, 420, 422, 424, 425, 427, 428, 429, 430, 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, 443, 445, 446, 447, 448, 449, 450, 451, 453, 454, 456, 457, 461, 462, 463, 467, 468, 469, 470, 471, 472, 473, 475, 476, 522, 523, 524, 533, 535], "via": [1, 4, 7, 10, 12, 19, 23, 27, 33, 35, 37, 38, 45, 47, 48, 51, 54, 56, 57, 59, 64, 66, 68, 69, 70, 71, 73, 75, 77, 82, 83, 84, 86, 88, 90, 92, 113, 121, 136, 149, 151, 155, 157, 163, 167, 184, 187, 217, 220, 227, 233, 235, 236, 242, 253, 257, 259, 263, 264, 267, 270, 281, 288, 293, 301, 303, 305, 307, 333, 334, 337, 339, 341, 343, 344, 345, 346, 347, 348, 357, 358, 359, 360, 362, 364, 365, 366, 367, 368, 371, 373, 374, 375, 377, 379, 380, 383, 384, 385, 386, 388, 389, 391, 392, 393, 395, 396, 398, 399, 400, 408, 412, 414, 417, 418, 432, 434, 436, 444, 445, 446, 450, 462, 466, 473, 483, 517, 520], "distributed_sort": [1, 271], "configur": [1, 2, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 20, 23, 24, 25, 46, 58, 86, 87, 93, 112, 132, 134, 139, 143, 146, 151, 157, 161, 167, 170, 173, 178, 180, 183, 184, 187, 192, 198, 208, 209, 213, 217, 220, 223, 224, 228, 231, 233, 236, 239, 243, 244, 245, 247, 248, 253, 254, 255, 257, 258, 259, 260, 261, 262, 263, 266, 267, 268, 269, 270, 271, 272, 273, 275, 276, 277, 278, 279, 280, 293, 297, 301, 302, 303, 306, 307, 317, 327, 328, 329, 331, 332, 333, 334, 335, 336, 338, 339, 340, 341, 343, 344, 345, 346, 347, 348, 349, 350, 351, 352, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, 371, 372, 373, 374, 375, 378, 379, 380, 381, 382, 383, 384, 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, 403, 404, 405, 407, 408, 409, 413, 414, 415, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, 432, 433, 434, 435, 436, 437, 438, 439, 441, 443, 444, 445, 446, 447, 448, 449, 450, 451, 453, 454, 455, 456, 457, 459, 460, 464, 465, 474, 476, 510, 523, 525], "set": [1, 2, 3, 4, 5, 7, 8, 9, 10, 11, 12, 13, 14, 15, 17, 18, 19, 20, 21, 23, 24, 27, 29, 32, 33, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 47, 50, 51, 52, 53, 55, 57, 58, 59, 60, 62, 64, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 83, 84, 85, 87, 88, 89, 94, 98, 99, 102, 107, 109, 110, 112, 119, 120, 126, 133, 136, 137, 138, 143, 144, 146, 148, 149, 150, 151, 154, 155, 156, 157, 161, 163, 167, 172, 180, 183, 191, 192, 193, 196, 197, 202, 207, 208, 209, 210, 212, 213, 216, 217, 219, 220, 221, 225, 228, 231, 233, 235, 239, 242, 244, 246, 247, 251, 254, 255, 256, 257, 259, 260, 261, 262, 263, 264, 267, 270, 272, 273, 274, 276, 277, 278, 279, 282, 293, 296, 297, 301, 302, 303, 305, 306, 307, 311, 317, 320, 321, 327, 328, 329, 330, 331, 334, 336, 338, 340, 345, 346, 348, 349, 350, 352, 354, 355, 356, 357, 358, 359, 361, 363, 365, 366, 367, 368, 371, 372, 373, 374, 375, 377, 378, 379, 380, 381, 384, 386, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, 406, 407, 408, 409, 413, 414, 415, 417, 418, 419, 423, 426, 428, 429, 432, 433, 434, 435, 441, 445, 446, 447, 448, 449, 450, 451, 453, 454, 455, 457, 460, 461, 462, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 479, 481, 484, 486, 487, 488, 489, 490, 499, 505, 508, 510, 511, 512, 515, 516, 518, 533, 537, 542], "etc": [1, 3, 4, 5, 17, 23, 24, 27, 32, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 49, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 78, 79, 80, 82, 83, 84, 85, 89, 90, 92, 95, 109, 112, 135, 136, 139, 165, 178, 179, 180, 182, 194, 213, 217, 244, 253, 264, 281, 282, 287, 288, 296, 297, 301, 359, 360, 375, 378, 379, 380, 385, 416, 459, 460, 461, 462, 463, 466, 467, 468, 469, 472, 473, 474, 475, 476], "config": [1, 3, 4, 5, 7, 17, 23, 24, 27, 32, 35, 37, 40, 41, 42, 45, 47, 49, 51, 52, 54, 55, 57, 58, 60, 62, 66, 68, 69, 71, 72, 74, 75, 76, 84, 85, 95, 137, 138, 139, 154, 155, 163, 166, 167, 170, 173, 176, 177, 178, 180, 183, 184, 187, 193, 196, 197, 200, 206, 208, 209, 210, 213, 218, 220, 221, 226, 227, 228, 230, 231, 232, 233, 235, 240, 241, 242, 244, 245, 246, 247, 248, 251, 253, 255, 256, 257, 259, 260, 262, 263, 264, 265, 266, 267, 268, 270, 276, 278, 280, 282, 287, 296, 297, 301, 307, 310, 317, 321, 333, 334, 337, 345, 354, 357, 359, 360, 371, 374, 375, 377, 379, 380, 382, 384, 386, 388, 389, 392, 393, 395, 398, 400, 403, 404, 412, 459, 460, 461, 462, 464, 466, 467, 470, 471, 472, 473, 474, 475, 476, 523], "coordin": [1, 2, 4, 5, 6, 8, 15, 19, 20, 23, 27, 32, 33, 35, 37, 38, 39, 40, 41, 42, 44, 45, 48, 52, 53, 54, 55, 57, 60, 62, 63, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 78, 82, 85, 89, 90, 92, 93, 95, 109, 134, 136, 137, 138, 155, 158, 160, 178, 180, 186, 200, 206, 209, 217, 219, 221, 222, 223, 225, 226, 229, 236, 239, 241, 243, 244, 245, 247, 250, 251, 257, 259, 260, 264, 271, 276, 278, 279, 281, 282, 287, 289, 290, 293, 297, 301, 303, 307, 309, 310, 311, 317, 339, 341, 344, 356, 360, 361, 362, 364, 365, 371, 380, 383, 387, 390, 391, 392, 397, 403, 426, 427, 438, 443, 451, 454, 460, 461, 463, 464, 465, 466, 467, 470, 471, 472, 473, 476], "default": [1, 2, 3, 4, 5, 6, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 29, 32, 33, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 47, 48, 49, 50, 51, 53, 54, 55, 56, 58, 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, 72, 74, 75, 76, 78, 80, 83, 84, 87, 88, 93, 94, 95, 96, 99, 103, 105, 107, 112, 126, 131, 132, 135, 136, 137, 139, 144, 146, 155, 157, 167, 170, 173, 177, 184, 187, 191, 192, 196, 197, 206, 207, 208, 209, 210, 217, 218, 220, 224, 227, 231, 233, 235, 236, 239, 245, 246, 254, 255, 258, 261, 262, 263, 266, 267, 269, 270, 271, 273, 276, 277, 278, 279, 280, 282, 297, 301, 302, 303, 304, 307, 310, 320, 321, 331, 332, 336, 337, 339, 341, 342, 343, 346, 347, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, 368, 369, 372, 375, 379, 381, 384, 388, 389, 391, 392, 395, 397, 398, 407, 408, 413, 414, 418, 419, 423, 426, 427, 428, 430, 436, 438, 441, 445, 448, 449, 450, 451, 455, 460, 462, 463, 464, 466, 467, 468, 469, 470, 471, 472, 473, 475, 476, 478, 480, 482, 486, 489, 491, 494, 505, 510, 512, 515, 520, 525], "when": [1, 2, 3, 5, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 21, 22, 23, 25, 27, 32, 33, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 47, 48, 51, 52, 53, 54, 55, 56, 57, 59, 60, 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 83, 84, 87, 88, 91, 93, 94, 95, 99, 100, 102, 104, 105, 106, 107, 108, 110, 112, 119, 121, 124, 126, 135, 136, 137, 138, 142, 144, 146, 147, 151, 155, 156, 157, 158, 161, 162, 163, 164, 166, 167, 170, 172, 173, 176, 177, 178, 180, 181, 182, 183, 184, 186, 187, 191, 193, 196, 197, 198, 199, 201, 203, 204, 206, 207, 208, 209, 210, 211, 212, 213, 215, 216, 217, 218, 219, 220, 221, 222, 223, 225, 226, 227, 228, 229, 231, 232, 234, 235, 236, 237, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, 276, 277, 278, 279, 280, 281, 282, 287, 288, 289, 291, 292, 293, 295, 297, 298, 299, 300, 301, 302, 303, 304, 305, 307, 310, 311, 312, 313, 317, 319, 320, 321, 322, 324, 325, 327, 329, 330, 331, 332, 333, 334, 335, 336, 337, 338, 339, 340, 342, 343, 344, 345, 346, 347, 348, 350, 351, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, 377, 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, 412, 413, 414, 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, 442, 443, 444, 445, 446, 447, 448, 449, 450, 451, 452, 453, 454, 455, 456, 457, 461, 462, 464, 465, 466, 467, 468, 469, 470, 471, 473, 474, 475, 476, 480, 506, 510, 511, 512, 516, 520, 524, 542], "oper": [1, 2, 11, 14, 15, 19, 32, 33, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 47, 48, 52, 53, 54, 55, 56, 57, 58, 59, 60, 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 83, 84, 87, 88, 91, 94, 96, 99, 115, 121, 132, 133, 135, 137, 138, 139, 140, 144, 146, 147, 148, 152, 155, 156, 179, 180, 182, 186, 191, 194, 196, 198, 206, 210, 212, 223, 227, 228, 229, 232, 233, 236, 239, 240, 243, 246, 248, 251, 257, 267, 269, 276, 279, 280, 287, 293, 296, 302, 304, 305, 306, 307, 313, 315, 317, 324, 328, 331, 332, 333, 338, 339, 340, 341, 343, 352, 353, 354, 359, 360, 363, 368, 373, 374, 375, 379, 380, 382, 386, 388, 389, 390, 391, 392, 395, 396, 397, 400, 402, 403, 407, 408, 409, 417, 423, 426, 428, 429, 430, 432, 434, 440, 441, 448, 450, 460, 462, 468, 471, 477, 480, 486, 505, 506, 510, 511, 512, 514], "parallel": [1, 5, 14, 19, 21, 22, 27, 37, 41, 45, 53, 57, 64, 70, 99, 136, 151, 160, 180, 208, 228, 230, 239, 241, 242, 246, 255, 256, 263, 278, 297, 353, 360, 374, 380, 381, 382, 388], "multipl": [1, 5, 8, 14, 18, 21, 25, 32, 41, 42, 47, 48, 49, 51, 52, 56, 59, 60, 62, 63, 77, 84, 96, 99, 104, 109, 110, 112, 117, 118, 123, 124, 132, 135, 136, 137, 138, 141, 146, 147, 151, 156, 173, 176, 178, 196, 213, 219, 221, 225, 226, 227, 247, 250, 253, 254, 257, 267, 270, 272, 276, 277, 302, 305, 339, 350, 354, 356, 359, 360, 365, 371, 373, 374, 380, 381, 382, 383, 384, 386, 388, 393, 396, 402, 403, 408, 419, 421, 424, 430, 432, 433, 438, 450, 460, 464, 467, 469, 476, 478, 480, 486, 489, 505, 509, 510, 523], "cluster": [1, 4, 5, 6, 9, 10, 13, 14, 15, 17, 19, 22, 23, 25, 26, 27, 32, 33, 36, 40, 41, 42, 43, 47, 51, 52, 55, 56, 59, 60, 61, 66, 68, 69, 70, 71, 73, 75, 76, 77, 82, 83, 85, 89, 90, 92, 93, 95, 124, 125, 132, 134, 136, 138, 146, 156, 158, 161, 163, 178, 193, 197, 198, 209, 210, 217, 221, 233, 235, 239, 242, 245, 246, 247, 248, 253, 272, 273, 282, 287, 296, 297, 303, 313, 329, 355, 356, 357, 360, 364, 367, 368, 374, 377, 384, 388, 389, 392, 393, 395, 396, 400, 403, 422, 427, 432, 434, 451, 460, 461, 462, 464, 465, 466, 467, 472, 475, 476, 523], "partial": [1, 2, 63, 84, 88, 112, 144, 148, 162, 165, 173, 235, 256, 266, 270, 307, 317, 325, 359, 387, 421, 505, 506, 510], "from": [1, 2, 3, 4, 5, 7, 9, 10, 11, 13, 14, 15, 16, 23, 25, 27, 28, 32, 33, 34, 36, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 51, 52, 53, 54, 55, 56, 57, 60, 61, 62, 63, 64, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 82, 83, 84, 86, 87, 88, 90, 91, 93, 94, 95, 96, 97, 99, 100, 101, 103, 105, 107, 109, 110, 113, 117, 119, 120, 121, 123, 124, 126, 127, 129, 131, 132, 133, 135, 136, 137, 138, 141, 142, 143, 144, 146, 147, 148, 151, 152, 155, 156, 159, 162, 164, 165, 166, 167, 170, 177, 179, 182, 188, 189, 194, 198, 201, 204, 205, 208, 209, 212, 213, 217, 220, 223, 226, 228, 229, 231, 233, 234, 235, 236, 237, 239, 240, 241, 243, 244, 245, 246, 247, 248, 249, 250, 251, 254, 255, 257, 258, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, 276, 277, 278, 279, 281, 283, 284, 287, 288, 293, 296, 297, 302, 303, 307, 313, 320, 324, 326, 327, 329, 330, 334, 335, 336, 337, 339, 341, 342, 343, 344, 345, 346, 348, 350, 352, 354, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, 373, 375, 377, 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, 412, 413, 414, 415, 419, 420, 422, 423, 424, 426, 427, 428, 429, 430, 432, 433, 434, 435, 436, 446, 447, 448, 449, 450, 451, 452, 453, 454, 456, 459, 461, 462, 464, 465, 466, 467, 468, 469, 470, 471, 472, 473, 475, 476, 478, 480, 482, 483, 486, 488, 489, 490, 491, 492, 493, 496, 497, 503, 504, 505, 506, 509, 510, 511, 512, 513, 514, 517, 518, 520, 521, 527, 532, 534, 535, 536, 539, 541, 542, 544], "each": [1, 2, 5, 10, 12, 13, 14, 15, 19, 21, 23, 24, 25, 27, 32, 35, 37, 39, 40, 41, 42, 43, 44, 45, 47, 48, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63, 64, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 80, 83, 84, 86, 88, 93, 94, 96, 99, 100, 104, 105, 109, 112, 117, 119, 121, 123, 124, 126, 131, 132, 136, 137, 138, 143, 144, 146, 147, 148, 149, 151, 156, 157, 167, 204, 209, 218, 235, 261, 263, 276, 282, 284, 287, 302, 305, 313, 320, 328, 336, 353, 362, 461, 462, 464, 465, 466, 469, 471, 476, 486, 496, 505, 506, 509, 510, 511, 512, 520, 521, 523, 532, 538], "worker": [1, 2, 5, 6, 8, 13, 14, 17, 21, 27, 35, 36, 37, 39, 40, 41, 42, 44, 45, 48, 52, 53, 54, 55, 57, 60, 61, 62, 63, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 78, 84, 86, 88, 93, 94, 124, 134, 136, 137, 138, 154, 157, 158, 167, 182, 187, 209, 213, 217, 221, 225, 230, 232, 241, 242, 247, 249, 255, 257, 260, 267, 273, 278, 279, 280, 282, 287, 296, 301, 303, 317, 328, 329, 331, 339, 357, 358, 360, 362, 364, 368, 372, 373, 374, 375, 377, 385, 387, 389, 395, 397, 403, 405, 408, 410, 422, 430, 449, 460, 466, 470, 471, 505], "stream": [1, 3, 4, 25, 37, 45, 51, 59, 84, 96, 99, 136, 146, 155, 163, 173, 181, 198, 219, 250, 265, 274, 302, 307, 321, 375, 379, 385, 392, 396], "singl": [1, 2, 5, 13, 14, 15, 18, 19, 21, 23, 25, 29, 32, 35, 37, 39, 41, 43, 45, 50, 53, 56, 57, 59, 61, 64, 69, 71, 73, 74, 76, 82, 84, 86, 88, 90, 92, 93, 94, 95, 96, 99, 100, 101, 103, 104, 109, 112, 117, 119, 120, 123, 124, 126, 127, 132, 136, 137, 144, 148, 151, 156, 157, 166, 171, 217, 221, 234, 235, 254, 255, 262, 263, 270, 278, 282, 320, 321, 330, 350, 355, 357, 371, 381, 384, 388, 391, 397, 401, 418, 421, 432, 433, 434, 438, 448, 449, 450, 462, 465, 467, 475, 476, 489, 505, 506, 509, 510, 511, 520, 522, 524, 544], "final": [1, 58, 62, 66, 69, 71, 74, 75, 76, 88, 91, 99, 100, 119, 148, 151, 157, 217, 227, 249, 254, 276, 293, 296, 327, 337, 392, 400, 459, 505, 506, 512, 520], "merg": [1, 25, 38, 41, 45, 53, 54, 60, 69, 81, 88, 99, 100, 110, 114, 115, 117, 120, 123, 127, 133, 143, 144, 210, 223, 248, 280, 302, 391, 420, 425, 427, 428, 432, 435, 440, 441, 445, 450, 452, 477, 505], "thi": [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 12, 13, 14, 15, 16, 17, 18, 19, 20, 23, 24, 25, 27, 29, 30, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, 91, 92, 93, 94, 95, 96, 97, 98, 99, 100, 101, 103, 104, 106, 107, 108, 109, 110, 112, 113, 118, 119, 120, 121, 122, 123, 124, 126, 127, 128, 129, 131, 132, 135, 136, 137, 138, 140, 141, 144, 146, 147, 148, 151, 152, 154, 155, 156, 157, 159, 161, 163, 166, 167, 172, 173, 174, 176, 178, 180, 182, 183, 184, 186, 187, 188, 191, 194, 198, 204, 206, 209, 211, 212, 213, 217, 220, 223, 224, 225, 226, 227, 228, 230, 233, 236, 240, 242, 245, 246, 247, 248, 249, 251, 253, 254, 255, 256, 257, 258, 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, 276, 278, 280, 282, 283, 284, 285, 287, 291, 293, 294, 295, 296, 297, 300, 301, 302, 303, 304, 305, 307, 310, 316, 317, 318, 320, 321, 325, 327, 330, 331, 332, 333, 335, 336, 337, 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, 349, 350, 352, 354, 355, 356, 357, 358, 359, 360, 361, 362, 364, 365, 368, 369, 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, 384, 387, 388, 389, 390, 391, 392, 393, 395, 396, 397, 398, 399, 400, 402, 403, 408, 409, 410, 414, 415, 419, 423, 428, 429, 430, 432, 434, 435, 436, 438, 441, 444, 446, 447, 448, 449, 450, 451, 455, 457, 460, 461, 462, 464, 465, 466, 467, 468, 469, 470, 471, 473, 474, 475, 476, 477, 478, 480, 486, 491, 503, 504, 510, 512, 520, 538], "techniqu": [1, 119, 123], "util": [1, 2, 5, 9, 13, 14, 15, 18, 19, 36, 41, 45, 58, 85, 86, 88, 109, 147, 240, 249, 257, 263, 329, 358, 391, 392, 420, 427, 428, 432, 466, 472, 475], "The": [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 19, 20, 22, 23, 25, 26, 27, 29, 30, 31, 32, 33, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 82, 83, 84, 85, 86, 87, 88, 89, 90, 91, 92, 93, 94, 95, 97, 99, 100, 101, 103, 104, 105, 106, 107, 108, 109, 110, 112, 113, 116, 117, 118, 119, 120, 121, 123, 124, 126, 127, 128, 129, 131, 132, 135, 136, 137, 138, 139, 141, 142, 143, 144, 146, 147, 148, 149, 151, 154, 155, 156, 162, 163, 166, 167, 170, 172, 176, 178, 183, 188, 192, 193, 196, 208, 211, 217, 221, 225, 227, 228, 231, 232, 233, 235, 239, 242, 245, 246, 247, 251, 252, 254, 255, 256, 258, 260, 261, 263, 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, 276, 277, 278, 279, 280, 281, 282, 284, 287, 288, 293, 294, 296, 297, 300, 301, 302, 303, 305, 306, 307, 316, 321, 324, 325, 327, 328, 331, 333, 334, 335, 339, 342, 344, 346, 347, 348, 351, 355, 356, 357, 359, 362, 364, 366, 368, 369, 371, 373, 377, 378, 379, 380, 381, 384, 385, 386, 388, 389, 392, 395, 396, 397, 398, 399, 401, 407, 413, 414, 428, 429, 432, 435, 436, 439, 444, 445, 449, 450, 454, 459, 461, 462, 463, 464, 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 478, 480, 482, 484, 486, 487, 488, 489, 490, 491, 494, 498, 499, 500, 501, 502, 505, 506, 507, 508, 510, 512, 513, 514, 516, 518, 520, 521, 522, 523, 524, 525, 533, 538, 542], "primari": [1, 35, 39, 40, 47, 55, 60, 65, 69, 86, 281, 368, 377, 510, 512], "purpos": [1, 5, 66, 71, 76, 83, 101, 151, 152, 279, 281, 431, 470], "don": [1, 2, 25, 32, 35, 47, 78, 84, 88, 100, 112, 137, 151, 313, 317, 355, 385], "t": [1, 2, 6, 25, 29, 32, 35, 38, 41, 47, 48, 51, 54, 68, 77, 78, 84, 88, 94, 98, 99, 100, 104, 107, 108, 112, 113, 119, 120, 123, 133, 137, 140, 151, 156, 175, 196, 271, 287, 301, 307, 313, 317, 355, 360, 363, 371, 385, 403, 419, 424, 430, 435, 511, 520, 526, 542, 544], "normal": [1, 12, 27, 32, 33, 41, 54, 57, 77, 99, 114, 115, 118, 124, 138, 142, 162, 233, 282, 293, 301, 392, 434, 464, 486, 520], "fit": [1, 9, 10, 25, 32, 57, 65, 66, 68, 69, 71, 75, 103, 108, 112, 146, 304, 355, 435, 510], "perform": [1, 2, 3, 4, 5, 6, 9, 10, 13, 14, 15, 19, 25, 26, 29, 33, 36, 37, 38, 39, 42, 43, 48, 52, 53, 55, 57, 60, 64, 65, 67, 69, 70, 74, 77, 79, 80, 84, 85, 88, 89, 90, 94, 95, 96, 101, 107, 108, 109, 110, 112, 118, 121, 124, 131, 136, 143, 144, 146, 148, 151, 155, 156, 158, 164, 166, 173, 178, 180, 183, 184, 185, 186, 187, 188, 194, 196, 197, 198, 206, 208, 209, 210, 212, 213, 215, 217, 218, 220, 223, 224, 226, 227, 228, 229, 230, 231, 232, 233, 234, 235, 239, 240, 241, 242, 243, 245, 246, 247, 248, 249, 250, 251, 253, 254, 255, 257, 258, 259, 261, 262, 263, 264, 266, 267, 269, 270, 274, 275, 276, 277, 279, 280, 282, 284, 288, 289, 293, 296, 297, 300, 301, 302, 303, 304, 309, 312, 313, 327, 328, 329, 330, 331, 332, 333, 334, 335, 337, 339, 341, 343, 344, 345, 346, 347, 348, 349, 351, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 364, 367, 368, 369, 370, 371, 372, 373, 374, 375, 377, 379, 380, 381, 382, 383, 384, 385, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, 412, 413, 414, 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 428, 429, 430, 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, 442, 443, 444, 445, 446, 447, 448, 449, 450, 451, 452, 453, 454, 456, 457, 459, 461, 462, 469, 473, 474, 483, 486, 510, 512, 514, 520], "improv": [1, 2, 5, 9, 10, 13, 14, 19, 37, 39, 40, 41, 42, 43, 45, 48, 53, 54, 55, 60, 62, 64, 65, 66, 68, 69, 70, 71, 73, 74, 75, 76, 99, 133, 134, 136, 146, 148, 154, 155, 156, 158, 160, 162, 163, 166, 173, 178, 180, 183, 184, 186, 187, 191, 193, 194, 196, 197, 198, 206, 208, 209, 210, 213, 215, 217, 218, 220, 221, 224, 226, 227, 228, 229, 230, 231, 232, 233, 234, 235, 236, 239, 240, 241, 242, 245, 246, 247, 248, 249, 250, 253, 254, 255, 256, 257, 258, 259, 260, 261, 263, 264, 266, 267, 268, 269, 270, 271, 273, 274, 275, 276, 277, 278, 279, 280, 282, 284, 288, 289, 290, 291, 293, 294, 296, 300, 301, 303, 309, 312, 313, 317, 320, 327, 328, 329, 330, 331, 332, 333, 334, 335, 337, 338, 339, 341, 343, 344, 345, 346, 347, 348, 349, 351, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 367, 368, 369, 370, 371, 372, 373, 374, 375, 377, 379, 380, 381, 382, 383, 384, 385, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, 412, 413, 414, 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 428, 429, 430, 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, 442, 443, 444, 445, 446, 447, 448, 449, 450, 451, 452, 453, 454, 456, 457, 466, 469], "can": [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 22, 23, 24, 25, 26, 27, 29, 31, 32, 33, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 83, 84, 86, 87, 88, 89, 91, 93, 94, 95, 96, 97, 99, 100, 103, 104, 105, 106, 107, 108, 109, 110, 112, 113, 117, 119, 121, 123, 124, 126, 127, 129, 131, 132, 134, 135, 136, 137, 138, 139, 142, 143, 144, 146, 148, 149, 151, 152, 156, 157, 158, 159, 162, 163, 165, 167, 170, 172, 177, 178, 179, 180, 182, 183, 184, 186, 187, 190, 192, 193, 194, 197, 198, 201, 206, 208, 209, 212, 213, 217, 220, 223, 224, 226, 227, 228, 229, 230, 233, 235, 236, 240, 242, 244, 245, 246, 247, 248, 249, 251, 253, 255, 256, 257, 259, 260, 261, 262, 263, 264, 265, 266, 267, 269, 270, 271, 272, 273, 274, 276, 277, 278, 279, 280, 281, 282, 283, 284, 286, 287, 288, 291, 293, 296, 297, 301, 302, 303, 304, 305, 306, 307, 315, 317, 327, 328, 329, 331, 332, 333, 334, 335, 336, 337, 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, 349, 355, 356, 357, 358, 359, 360, 361, 362, 364, 365, 366, 368, 369, 372, 373, 374, 375, 377, 378, 379, 380, 381, 382, 383, 384, 387, 388, 389, 390, 391, 392, 393, 395, 396, 397, 398, 399, 402, 405, 408, 409, 412, 413, 414, 419, 423, 425, 427, 428, 429, 430, 432, 433, 434, 435, 436, 438, 439, 441, 445, 446, 447, 448, 449, 450, 451, 453, 457, 459, 460, 461, 462, 463, 464, 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 480, 482, 483, 484, 486, 488, 489, 490, 491, 505, 506, 510, 511, 512, 513, 514, 516, 520, 521, 523, 524, 525, 530, 544], "expect": [1, 10, 14, 23, 33, 35, 37, 39, 41, 42, 43, 45, 51, 62, 66, 68, 71, 74, 76, 78, 84, 96, 97, 106, 112, 123, 137, 147, 152, 157, 167, 172, 173, 178, 183, 226, 242, 258, 259, 267, 282, 287, 293, 302, 331, 388, 415, 461, 467, 475, 505, 510], "won": [1, 47], "scale": [1, 5, 8, 9, 14, 15, 37, 55, 64, 66, 68, 69, 71, 75, 78, 79, 80, 108, 123, 133, 136, 137, 144, 256, 257, 260, 261, 263, 273, 356, 357, 358, 389, 392, 407, 422, 450, 451, 455], "linearli": 1, "number": [1, 2, 3, 5, 7, 8, 9, 12, 13, 14, 15, 16, 18, 19, 22, 23, 25, 27, 32, 33, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 48, 51, 53, 54, 55, 56, 57, 58, 59, 60, 62, 64, 65, 66, 67, 69, 70, 71, 73, 74, 75, 76, 78, 80, 83, 84, 88, 94, 96, 97, 99, 100, 102, 106, 107, 108, 109, 112, 113, 118, 120, 121, 124, 127, 129, 131, 132, 134, 135, 136, 137, 144, 147, 148, 149, 154, 155, 156, 157, 166, 167, 180, 184, 186, 193, 209, 217, 221, 223, 224, 227, 235, 242, 246, 247, 252, 255, 256, 257, 258, 259, 263, 265, 266, 267, 270, 272, 273, 274, 276, 278, 279, 282, 284, 287, 289, 293, 296, 297, 312, 315, 316, 317, 321, 323, 329, 332, 333, 337, 339, 341, 342, 352, 354, 357, 359, 361, 367, 368, 371, 373, 380, 381, 382, 388, 389, 392, 400, 401, 406, 408, 409, 420, 422, 424, 426, 427, 432, 434, 436, 438, 439, 443, 444, 446, 449, 451, 455, 466, 469, 472, 473, 478, 480, 505, 510, 511, 520, 538], "sinc": [1, 2, 7, 15, 19, 23, 32, 40, 41, 45, 52, 54, 55, 57, 58, 59, 62, 63, 66, 68, 71, 73, 75, 77, 78, 84, 88, 94, 97, 99, 104, 107, 119, 148, 152, 282, 287, 293, 316, 320, 350, 369, 375, 384, 390, 395, 397, 470, 486, 514, 520], "need": [1, 2, 3, 4, 5, 6, 10, 14, 22, 23, 25, 32, 33, 35, 36, 37, 39, 40, 41, 42, 43, 44, 45, 47, 48, 49, 50, 51, 52, 53, 54, 55, 57, 59, 60, 62, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 83, 84, 88, 90, 93, 94, 96, 104, 112, 119, 121, 124, 132, 136, 137, 139, 144, 146, 148, 149, 151, 155, 159, 167, 173, 178, 186, 188, 194, 212, 213, 217, 220, 223, 225, 245, 251, 254, 255, 260, 270, 275, 282, 283, 287, 293, 294, 302, 316, 317, 320, 323, 325, 327, 334, 336, 342, 349, 350, 352, 379, 387, 397, 401, 423, 432, 444, 448, 461, 462, 464, 467, 468, 469, 470, 474, 475, 486, 504, 512], "optim": [2, 8, 10, 15, 35, 53, 62, 63, 65, 66, 71, 76, 77, 84, 96, 119, 126, 133, 147, 148, 149, 152, 155, 156, 158, 165, 166, 170, 176, 177, 178, 180, 182, 184, 186, 187, 191, 192, 194, 196, 197, 208, 210, 211, 217, 219, 220, 224, 229, 230, 232, 233, 239, 240, 242, 245, 247, 250, 254, 255, 256, 258, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 272, 273, 278, 279, 280, 282, 284, 290, 297, 301, 305, 306, 310, 317, 320, 324, 325, 326, 334, 339, 351, 356, 357, 359, 362, 364, 369, 373, 375, 379, 381, 382, 391, 392, 393, 396, 401, 402, 403, 410, 414, 415, 416, 418, 420, 423, 424, 432, 435, 436, 439, 445, 446, 448, 450, 451, 454, 455, 470, 480, 520, 523], "significantli": [2, 14, 45, 48, 112, 148, 241, 293, 302, 335, 466], "select": [2, 5, 14, 15, 23, 24, 27, 29, 32, 33, 35, 37, 38, 39, 40, 41, 42, 43, 44, 45, 47, 48, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 62, 63, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 84, 94, 99, 100, 101, 102, 104, 105, 106, 107, 110, 111, 112, 113, 116, 117, 118, 119, 121, 123, 124, 126, 129, 131, 133, 135, 137, 141, 142, 143, 144, 147, 148, 162, 164, 177, 178, 180, 190, 209, 212, 220, 229, 235, 240, 254, 263, 266, 268, 270, 271, 272, 275, 278, 279, 280, 282, 283, 284, 287, 288, 293, 296, 297, 302, 307, 308, 316, 324, 327, 330, 332, 337, 340, 345, 350, 356, 357, 358, 360, 366, 367, 368, 373, 384, 385, 386, 391, 394, 396, 397, 401, 412, 417, 419, 420, 421, 422, 424, 432, 433, 434, 437, 438, 441, 442, 443, 445, 448, 449, 450, 454, 456, 460, 462, 466, 471, 472, 474, 475, 477, 482, 486, 488, 489, 490, 491, 493, 494, 496, 497, 503, 504, 505, 506, 507, 509, 510, 511, 512, 513, 517, 521, 542, 544], "join": [2, 5, 9, 15, 18, 19, 23, 29, 37, 40, 41, 42, 45, 54, 55, 57, 62, 63, 69, 73, 74, 78, 84, 94, 99, 123, 142, 155, 156, 157, 159, 160, 162, 163, 165, 170, 171, 173, 176, 177, 178, 179, 180, 181, 182, 183, 184, 186, 188, 191, 193, 194, 196, 205, 206, 208, 209, 210, 212, 217, 218, 219, 224, 226, 227, 228, 229, 232, 233, 234, 235, 237, 240, 242, 245, 247, 248, 249, 250, 251, 253, 254, 255, 256, 257, 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, 274, 275, 276, 277, 278, 279, 280, 282, 287, 288, 293, 296, 301, 302, 303, 305, 307, 311, 313, 316, 317, 323, 327, 330, 331, 334, 335, 336, 337, 339, 342, 344, 346, 347, 348, 349, 352, 353, 354, 355, 356, 357, 358, 359, 361, 364, 365, 366, 368, 369, 371, 372, 373, 374, 375, 377, 379, 380, 381, 382, 383, 385, 387, 389, 390, 391, 392, 394, 397, 399, 402, 403, 404, 406, 407, 408, 409, 416, 417, 418, 419, 420, 421, 423, 426, 428, 430, 432, 433, 434, 435, 436, 437, 438, 445, 446, 453, 454, 466, 486], "avoid": [2, 3, 4, 10, 14, 15, 18, 32, 39, 40, 41, 42, 44, 45, 48, 55, 62, 64, 66, 68, 70, 71, 74, 75, 76, 88, 109, 123, 136, 137, 163, 210, 223, 242, 251, 255, 261, 262, 263, 265, 274, 278, 280, 290, 302, 303, 330, 331, 347, 355, 356, 366, 368, 379, 381, 383, 392, 393, 396, 397, 408, 413, 421, 428, 430, 435, 436, 471, 486, 520], "read": [2, 5, 14, 15, 23, 24, 25, 32, 33, 35, 36, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 62, 63, 64, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 84, 86, 87, 132, 138, 140, 144, 148, 149, 151, 156, 162, 166, 169, 172, 178, 180, 191, 192, 196, 197, 201, 206, 207, 208, 209, 213, 224, 228, 229, 232, 233, 240, 242, 243, 245, 246, 247, 248, 249, 250, 254, 256, 257, 258, 259, 260, 261, 262, 264, 265, 267, 269, 271, 274, 277, 278, 281, 284, 295, 303, 304, 307, 320, 321, 326, 327, 329, 330, 331, 332, 334, 335, 336, 339, 341, 343, 344, 345, 346, 347, 348, 351, 352, 354, 356, 357, 358, 359, 360, 361, 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, 375, 379, 380, 381, 382, 383, 385, 386, 387, 388, 389, 390, 391, 392, 393, 395, 396, 397, 398, 399, 400, 401, 402, 403, 405, 406, 407, 408, 409, 412, 413, 415, 417, 418, 419, 420, 421, 423, 424, 425, 426, 427, 428, 430, 432, 433, 434, 435, 436, 437, 439, 440, 441, 444, 446, 447, 448, 449, 450, 451, 452, 453, 454, 455, 456, 457, 462, 467, 469, 470, 475, 486, 514, 520, 540], "would": [2, 32, 41, 42, 47, 51, 84, 93, 95, 100, 131, 136, 138, 173, 180, 182, 191, 203, 209, 217, 218, 228, 247, 248, 251, 253, 257, 258, 262, 270, 273, 274, 275, 280, 282, 287, 295, 296, 297, 302, 311, 316, 320, 321, 330, 332, 333, 341, 345, 346, 348, 355, 359, 380, 384, 388, 449, 462], "condit": [2, 14, 24, 37, 41, 45, 54, 68, 76, 84, 94, 98, 99, 104, 108, 112, 133, 148, 160, 169, 170, 172, 194, 197, 201, 208, 227, 235, 247, 250, 258, 260, 270, 278, 280, 287, 297, 304, 305, 313, 316, 329, 346, 354, 361, 364, 373, 375, 387, 423, 462, 466, 493, 510, 511, 512, 520, 542], "consid": [2, 3, 5, 10, 23, 24, 25, 32, 35, 39, 41, 45, 64, 84, 104, 107, 112, 136, 148, 221, 272, 297, 302, 332, 350, 364, 403, 461, 462, 467, 471, 473, 475, 476, 510, 512, 520], "follow": [2, 3, 4, 5, 6, 8, 11, 13, 15, 20, 23, 24, 25, 26, 27, 29, 30, 31, 32, 33, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 47, 48, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 83, 84, 85, 86, 88, 93, 94, 96, 98, 99, 103, 104, 105, 106, 107, 112, 113, 116, 119, 121, 123, 124, 126, 128, 129, 131, 132, 134, 135, 136, 137, 138, 139, 140, 141, 142, 143, 144, 146, 147, 148, 149, 151, 162, 163, 177, 217, 232, 248, 259, 266, 271, 281, 287, 293, 296, 302, 305, 307, 347, 364, 368, 369, 373, 384, 387, 432, 459, 460, 461, 462, 463, 464, 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 480, 482, 488, 489, 490, 506, 510, 512, 520, 521, 523, 524, 526, 527, 532, 536, 537, 538, 539], "captur": [2, 3, 4, 12, 103, 113, 121, 136, 144, 236, 242, 244, 261, 280, 373, 394, 462, 476], "common": [2, 16, 27, 32, 45, 48, 52, 80, 88, 99, 100, 104, 108, 109, 132, 137, 167, 233, 236, 282, 293, 356, 388, 428, 468], "pattern": [2, 3, 16, 32, 33, 49, 57, 59, 61, 73, 107, 121, 124, 173, 232, 276, 277, 332, 335, 336, 355, 374, 378, 386, 388, 389, 392, 433, 436, 440, 454, 461, 467, 468, 469, 470, 520, 526, 527, 532, 536, 537, 539], "fact": [2, 40, 42, 45, 55, 62, 66, 68, 69, 71, 74, 75, 76, 112, 123, 152], "store_sal": 2, "date_dim": 2, "count": [2, 3, 5, 7, 22, 25, 32, 38, 40, 41, 42, 53, 54, 55, 56, 57, 58, 59, 62, 65, 66, 67, 68, 70, 71, 76, 88, 94, 99, 100, 102, 114, 115, 121, 123, 135, 136, 137, 144, 148, 149, 156, 158, 178, 209, 217, 235, 238, 246, 247, 249, 256, 260, 262, 264, 270, 271, 273, 278, 279, 282, 284, 287, 293, 295, 302, 304, 307, 311, 317, 327, 330, 331, 333, 337, 353, 356, 360, 361, 364, 367, 368, 369, 371, 377, 379, 380, 382, 384, 388, 391, 392, 397, 402, 403, 408, 432, 433, 434, 437, 438, 450, 453, 455, 466, 472, 497, 505, 506, 520], "ON": [2, 23, 52, 66, 76, 94, 99, 112, 123, 142, 146, 148, 250, 327, 365, 399, 401, 408, 444, 450, 462, 484, 486, 494, 507, 511, 517, 520, 533], "ss_sold_date_sk": 2, "d_date_sk": 2, "where": [2, 5, 8, 13, 14, 18, 23, 25, 27, 29, 33, 35, 37, 39, 40, 41, 42, 43, 45, 47, 48, 51, 53, 54, 55, 57, 58, 60, 61, 62, 64, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 83, 84, 88, 99, 100, 104, 105, 106, 107, 110, 112, 113, 124, 126, 129, 131, 137, 142, 144, 146, 147, 148, 162, 164, 167, 169, 171, 175, 177, 178, 179, 180, 190, 196, 201, 206, 209, 212, 217, 220, 225, 229, 236, 238, 239, 241, 243, 245, 248, 250, 254, 258, 261, 262, 263, 268, 269, 270, 271, 272, 273, 274, 277, 278, 279, 280, 282, 293, 302, 304, 307, 310, 311, 313, 314, 316, 317, 324, 327, 328, 330, 352, 354, 360, 368, 371, 372, 375, 380, 384, 386, 387, 388, 389, 390, 391, 392, 394, 395, 396, 397, 399, 400, 401, 413, 421, 422, 426, 430, 431, 433, 434, 438, 450, 453, 462, 467, 468, 469, 471, 476, 480, 486, 493, 496, 503, 504, 505, 506, 510, 511, 512, 513, 520, 540, 542, 544], "d_following_holidai": 2, "y": [2, 60, 68, 84, 99, 100, 102, 107, 108, 109, 112, 113, 117, 118, 123, 144, 180, 210, 268, 385, 478, 480, 520], "AND": [2, 23, 45, 55, 62, 66, 68, 70, 71, 74, 75, 76, 84, 94, 99, 102, 104, 114, 116, 123, 142, 177, 203, 209, 227, 250, 287, 347, 373, 389, 496, 503, 505, 510, 511, 512, 513, 520], "d_year": 2, "2000": [2, 5, 13, 29, 38, 53, 58, 68], "without": [2, 6, 15, 19, 27, 32, 33, 37, 39, 40, 41, 42, 43, 44, 45, 48, 51, 54, 55, 57, 58, 59, 62, 64, 66, 68, 69, 70, 71, 73, 74, 75, 76, 78, 79, 80, 84, 86, 87, 88, 96, 97, 99, 100, 105, 112, 120, 121, 124, 126, 132, 135, 138, 144, 146, 148, 159, 169, 173, 213, 245, 254, 271, 272, 276, 282, 283, 285, 287, 297, 303, 307, 329, 332, 341, 346, 352, 357, 359, 361, 362, 368, 371, 372, 379, 381, 391, 399, 401, 408, 417, 419, 432, 433, 440, 447, 461, 462, 464, 465, 467, 469, 470, 504, 509, 510, 512, 521, 524, 544], "push": [2, 35, 37, 39, 40, 41, 42, 45, 55, 59, 60, 62, 63, 65, 66, 68, 69, 70, 71, 73, 74, 75, 76, 84, 148, 155, 158, 180, 192, 207, 209, 217, 223, 228, 242, 257, 265, 287, 307, 327, 330, 334, 335, 339, 343, 359, 367, 368, 373, 374, 379, 380, 381, 382, 387, 389, 390, 391, 397, 399, 400, 401, 402, 403, 404, 410, 423, 430, 432, 435, 436, 439, 454], "predic": [2, 35, 41, 42, 45, 54, 57, 60, 67, 69, 74, 100, 112, 144, 151, 155, 162, 180, 192, 194, 196, 197, 207, 209, 217, 218, 219, 220, 223, 224, 225, 227, 228, 232, 233, 241, 242, 243, 245, 248, 258, 259, 260, 262, 263, 264, 265, 267, 269, 272, 274, 276, 278, 280, 307, 308, 320, 324, 327, 329, 330, 333, 339, 346, 347, 352, 354, 356, 357, 358, 359, 360, 361, 363, 364, 365, 368, 369, 371, 372, 373, 374, 377, 379, 380, 383, 384, 386, 387, 388, 389, 390, 392, 396, 397, 398, 399, 400, 401, 402, 405, 406, 409, 414, 417, 419, 420, 423, 424, 425, 429, 430, 432, 434, 435, 436, 438, 442, 448, 450, 451, 457, 520], "scan": [2, 35, 39, 41, 45, 63, 69, 70, 78, 84, 94, 147, 151, 155, 160, 224, 242, 246, 255, 257, 258, 264, 269, 274, 276, 278, 282, 306, 327, 334, 349, 366, 368, 369, 374, 375, 380, 382, 385, 392, 395, 398, 427, 433, 434, 438, 455, 520], "all": [2, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 23, 24, 25, 27, 32, 33, 35, 37, 38, 39, 40, 41, 42, 43, 45, 47, 48, 51, 52, 54, 55, 56, 57, 59, 60, 61, 62, 63, 64, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 82, 83, 84, 85, 86, 88, 89, 90, 91, 92, 93, 94, 95, 96, 97, 99, 100, 107, 108, 109, 112, 113, 114, 117, 118, 120, 121, 123, 124, 126, 127, 128, 129, 131, 132, 135, 136, 137, 138, 139, 142, 143, 144, 146, 148, 151, 156, 167, 169, 177, 178, 180, 182, 183, 186, 191, 192, 196, 208, 217, 220, 223, 228, 233, 242, 245, 247, 253, 254, 258, 262, 264, 266, 269, 270, 276, 278, 280, 281, 282, 284, 287, 293, 297, 302, 303, 304, 307, 309, 315, 317, 320, 325, 326, 330, 332, 333, 342, 345, 348, 352, 357, 359, 363, 364, 371, 373, 374, 378, 381, 394, 395, 398, 403, 406, 409, 412, 424, 428, 441, 444, 447, 450, 459, 460, 461, 462, 464, 466, 467, 468, 469, 470, 471, 473, 474, 475, 476, 480, 482, 486, 488, 489, 490, 493, 494, 505, 507, 510, 511, 512, 517, 520, 522, 524, 532, 533, 535, 538, 541, 542], "ar": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 21, 22, 23, 24, 25, 27, 29, 30, 31, 32, 33, 35, 37, 38, 39, 40, 41, 42, 43, 44, 45, 47, 48, 49, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 82, 83, 84, 85, 86, 87, 88, 89, 90, 92, 93, 94, 95, 96, 97, 98, 99, 100, 103, 104, 105, 107, 108, 109, 112, 113, 116, 118, 119, 121, 123, 124, 126, 128, 129, 131, 135, 136, 137, 138, 139, 141, 142, 143, 144, 146, 147, 148, 149, 151, 152, 154, 155, 156, 162, 163, 165, 170, 172, 173, 177, 178, 179, 180, 183, 186, 191, 192, 194, 203, 207, 208, 209, 210, 211, 213, 217, 220, 221, 223, 225, 228, 229, 232, 233, 235, 236, 239, 242, 244, 245, 246, 247, 248, 249, 251, 256, 258, 259, 260, 263, 264, 265, 266, 268, 269, 270, 271, 272, 274, 275, 276, 277, 278, 279, 280, 282, 283, 287, 288, 291, 293, 295, 296, 297, 301, 302, 303, 304, 305, 307, 309, 313, 320, 321, 327, 330, 331, 333, 336, 342, 345, 347, 351, 353, 355, 356, 357, 359, 360, 361, 362, 364, 366, 368, 369, 371, 372, 373, 374, 375, 377, 378, 379, 380, 381, 382, 383, 384, 385, 388, 389, 390, 391, 392, 394, 395, 396, 397, 398, 399, 400, 402, 406, 408, 410, 415, 416, 420, 422, 424, 430, 432, 433, 434, 435, 436, 438, 439, 442, 443, 444, 447, 452, 453, 454, 459, 460, 461, 462, 463, 464, 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 480, 483, 486, 489, 491, 493, 497, 500, 503, 504, 505, 506, 508, 510, 511, 512, 513, 517, 518, 520, 523, 524, 532, 538, 542], "end": [2, 35, 36, 39, 40, 45, 52, 55, 57, 58, 59, 64, 65, 66, 71, 73, 74, 75, 76, 78, 83, 84, 94, 99, 100, 101, 103, 105, 109, 112, 124, 126, 141, 142, 151, 152, 196, 336, 366, 383, 397, 450, 465, 467, 470, 489, 512, 520, 523, 527], "up": [2, 5, 14, 15, 19, 22, 23, 32, 35, 37, 41, 45, 48, 51, 52, 54, 57, 58, 59, 64, 66, 68, 69, 71, 72, 73, 75, 76, 77, 84, 93, 94, 99, 118, 124, 136, 138, 144, 151, 210, 243, 255, 256, 258, 276, 297, 303, 317, 324, 328, 398, 465, 471, 475, 476, 510, 520], "throw": [2, 54, 63, 78, 84, 94, 96, 99, 112, 183, 317, 374, 380, 449, 456], "awai": [2, 35, 281], "most": [2, 5, 8, 14, 15, 18, 25, 26, 27, 32, 41, 45, 48, 51, 52, 60, 64, 71, 77, 78, 83, 86, 87, 88, 94, 104, 107, 112, 113, 124, 131, 132, 136, 137, 143, 146, 151, 204, 209, 213, 242, 256, 264, 282, 293, 302, 350, 373, 396, 474, 475, 489, 510, 512, 520, 523], "probe": [2, 9, 146, 157, 193, 369], "side": [2, 9, 10, 14, 33, 35, 40, 42, 51, 55, 62, 66, 68, 69, 71, 74, 75, 76, 88, 116, 146, 156, 163, 173, 183, 186, 193, 226, 227, 232, 237, 242, 256, 257, 264, 265, 269, 274, 277, 278, 280, 282, 287, 313, 334, 357, 362, 369, 379, 380, 381, 387, 392, 409, 422, 438, 480, 520], "row": [2, 4, 7, 16, 23, 25, 32, 35, 36, 37, 38, 39, 40, 41, 43, 44, 45, 53, 55, 56, 58, 60, 62, 65, 66, 67, 68, 69, 70, 71, 72, 74, 75, 76, 77, 78, 79, 80, 83, 84, 86, 88, 91, 99, 100, 105, 109, 112, 113, 117, 120, 124, 126, 131, 132, 135, 137, 146, 147, 148, 149, 151, 154, 156, 157, 162, 164, 166, 167, 173, 178, 179, 182, 197, 203, 208, 209, 210, 212, 213, 217, 220, 223, 227, 228, 232, 233, 236, 242, 244, 245, 248, 249, 250, 251, 254, 257, 261, 264, 265, 266, 267, 270, 271, 272, 273, 274, 275, 277, 278, 279, 282, 287, 288, 295, 309, 313, 314, 315, 316, 333, 341, 342, 343, 347, 350, 352, 354, 355, 356, 358, 359, 360, 362, 366, 367, 368, 372, 374, 375, 377, 379, 381, 382, 383, 384, 386, 387, 388, 389, 390, 391, 392, 394, 397, 400, 402, 406, 409, 415, 416, 419, 420, 424, 426, 430, 432, 433, 434, 438, 439, 440, 442, 444, 446, 448, 449, 450, 453, 455, 457, 462, 472, 475, 491, 493, 496, 497, 505, 506, 509, 511, 520, 530, 538, 541, 542, 544], "criteria": [2, 45, 51, 94, 159, 173, 196, 217, 219, 226, 233, 283, 355, 366, 462], "highli": [2, 41, 54, 64, 80, 206, 239, 254, 268, 269, 274, 275, 276, 293, 310, 313, 327, 420], "candid": 2, "valu": [2, 3, 4, 5, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 29, 32, 33, 35, 36, 38, 39, 40, 42, 43, 45, 47, 48, 49, 51, 55, 56, 57, 59, 60, 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 76, 78, 83, 84, 86, 87, 88, 94, 95, 96, 97, 99, 100, 102, 103, 104, 105, 106, 107, 108, 109, 112, 113, 116, 117, 118, 119, 120, 123, 124, 125, 126, 127, 129, 133, 136, 137, 142, 143, 144, 146, 147, 149, 154, 156, 159, 162, 167, 170, 173, 176, 177, 178, 180, 186, 191, 204, 209, 210, 217, 220, 223, 227, 229, 235, 236, 239, 242, 244, 245, 246, 247, 248, 249, 254, 255, 256, 257, 258, 259, 261, 262, 264, 266, 267, 268, 270, 272, 274, 276, 278, 280, 281, 282, 293, 296, 297, 301, 302, 303, 305, 312, 313, 317, 320, 321, 324, 326, 330, 332, 333, 334, 335, 336, 339, 340, 341, 345, 350, 351, 354, 355, 356, 357, 358, 359, 360, 361, 362, 364, 365, 366, 368, 369, 371, 372, 373, 374, 375, 377, 379, 380, 381, 382, 383, 384, 385, 386, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 400, 402, 403, 405, 407, 408, 409, 413, 418, 419, 421, 423, 424, 425, 426, 427, 428, 432, 433, 434, 436, 437, 438, 440, 441, 445, 446, 448, 449, 450, 452, 453, 454, 457, 462, 466, 467, 468, 469, 470, 474, 476, 477, 478, 480, 486, 489, 503, 504, 505, 509, 510, 511, 512, 513, 515, 520, 523, 538, 542], "process": [2, 3, 4, 5, 9, 10, 12, 13, 14, 15, 16, 18, 19, 21, 23, 25, 27, 32, 35, 36, 37, 39, 41, 42, 43, 45, 48, 51, 52, 53, 54, 57, 58, 59, 62, 64, 65, 66, 67, 68, 70, 71, 73, 74, 76, 77, 84, 86, 91, 96, 99, 112, 118, 119, 126, 133, 134, 136, 137, 139, 141, 146, 148, 151, 152, 154, 178, 184, 198, 201, 208, 210, 221, 222, 223, 239, 241, 244, 251, 255, 256, 257, 259, 266, 267, 276, 280, 284, 287, 296, 297, 307, 353, 355, 357, 359, 361, 371, 380, 390, 391, 395, 398, 402, 408, 423, 429, 432, 434, 445, 450, 462, 468, 469, 470, 475, 476, 486, 505, 510, 511, 514, 520], "right": [2, 10, 48, 94, 96, 101, 102, 105, 109, 112, 124, 142, 146, 155, 156, 163, 179, 217, 227, 232, 257, 260, 264, 265, 278, 313, 352, 355, 372, 381, 471, 472, 480, 510, 520], "In": [2, 5, 10, 14, 19, 23, 25, 27, 29, 31, 32, 35, 36, 37, 38, 39, 40, 41, 42, 44, 45, 48, 51, 52, 53, 54, 55, 57, 58, 59, 60, 62, 63, 64, 65, 66, 67, 68, 69, 71, 72, 73, 74, 75, 76, 78, 80, 84, 86, 88, 93, 94, 95, 96, 97, 99, 100, 104, 105, 107, 112, 119, 121, 126, 132, 137, 138, 144, 146, 148, 167, 170, 180, 263, 271, 282, 287, 293, 296, 302, 303, 317, 320, 321, 356, 371, 450, 459, 461, 462, 464, 465, 466, 468, 469, 470, 471, 472, 473, 475, 476, 491, 506, 510, 511, 512, 520, 524, 532], "case": [2, 5, 14, 19, 25, 27, 32, 33, 37, 38, 44, 45, 47, 48, 51, 52, 53, 57, 60, 64, 67, 72, 84, 86, 88, 94, 99, 100, 104, 112, 114, 119, 121, 124, 126, 128, 133, 136, 137, 142, 143, 146, 150, 156, 158, 165, 167, 175, 180, 183, 186, 197, 198, 217, 223, 235, 245, 248, 255, 261, 270, 271, 273, 274, 279, 280, 282, 287, 288, 290, 301, 303, 313, 327, 331, 336, 340, 341, 346, 351, 357, 361, 366, 371, 372, 380, 381, 383, 384, 387, 388, 389, 391, 396, 399, 400, 402, 418, 422, 426, 432, 433, 435, 436, 446, 450, 456, 461, 462, 464, 466, 467, 469, 470, 475, 476, 510, 511, 512, 520], "broadcast": [2, 10, 45, 63, 146, 262, 272, 277, 278, 304, 348, 356, 365, 369, 372, 375, 394, 430, 447, 505], "runtim": [2, 5, 32, 35, 41, 56, 66, 69, 71, 75, 88, 93, 125, 132, 135, 147, 155, 210, 217, 244, 271, 327, 345, 346, 354, 355, 357, 358, 368, 371, 403, 417, 432, 447, 462, 520], "gener": [2, 7, 8, 12, 15, 16, 18, 19, 20, 25, 32, 35, 36, 37, 49, 52, 56, 58, 65, 72, 77, 78, 79, 80, 83, 84, 88, 94, 97, 98, 100, 101, 112, 130, 132, 133, 136, 137, 139, 140, 147, 148, 151, 152, 282, 287, 288, 461, 465, 466, 467, 470, 472, 475, 477, 510, 512, 520, 523], "local": [2, 19, 25, 33, 34, 35, 36, 39, 45, 48, 51, 57, 58, 59, 60, 63, 65, 73, 83, 121, 124, 133, 135, 155, 208, 209, 210, 246, 249, 258, 262, 263, 270, 280, 306, 342, 362, 375, 389, 407, 425, 432, 449, 450, 462, 464, 467, 470, 475, 476, 525], "left": [2, 10, 44, 101, 102, 105, 109, 112, 124, 142, 146, 155, 179, 227, 237, 242, 248, 264, 268, 353, 381, 383, 480, 510, 520], "run": [2, 3, 4, 5, 6, 12, 13, 14, 19, 23, 24, 25, 27, 29, 30, 33, 36, 37, 39, 40, 41, 42, 43, 44, 45, 46, 48, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 64, 65, 66, 68, 69, 70, 71, 72, 74, 75, 76, 77, 83, 84, 88, 94, 95, 119, 122, 131, 132, 134, 138, 139, 151, 155, 156, 158, 170, 173, 177, 180, 183, 187, 190, 193, 198, 206, 207, 210, 217, 222, 223, 228, 239, 242, 245, 246, 247, 248, 249, 251, 254, 256, 257, 258, 262, 270, 273, 274, 276, 277, 278, 293, 297, 302, 305, 311, 317, 323, 327, 328, 333, 334, 335, 337, 345, 346, 353, 357, 359, 360, 366, 371, 386, 387, 391, 393, 394, 398, 399, 402, 403, 407, 415, 416, 418, 421, 424, 430, 432, 434, 438, 450, 455, 462, 467, 468, 469, 471, 472, 475, 482, 483, 486, 488, 489, 490, 512, 520, 530, 533], "same": [2, 8, 13, 14, 19, 23, 25, 27, 32, 33, 35, 37, 38, 39, 40, 41, 43, 44, 45, 47, 48, 51, 52, 54, 56, 57, 59, 60, 62, 64, 65, 66, 68, 69, 71, 75, 76, 78, 79, 80, 84, 85, 86, 88, 94, 99, 100, 101, 102, 104, 107, 108, 109, 112, 117, 118, 119, 120, 124, 131, 136, 137, 138, 146, 148, 151, 162, 173, 191, 196, 203, 209, 210, 213, 219, 221, 229, 248, 250, 253, 254, 255, 256, 261, 264, 267, 270, 272, 274, 275, 282, 293, 296, 317, 323, 331, 337, 342, 345, 372, 375, 381, 386, 388, 390, 396, 403, 415, 418, 419, 432, 438, 450, 461, 462, 464, 465, 466, 467, 469, 475, 476, 486, 489, 490, 507, 510, 512, 520, 523], "addition": [2, 7, 25, 27, 35, 43, 53, 59, 60, 82, 90, 92, 96, 112, 118, 121, 124, 154, 156, 157, 177, 255, 287, 296, 300, 302, 303, 305, 388, 403, 507, 517, 520], "commun": [2, 9, 15, 31, 32, 33, 37, 51, 57, 64, 70, 83, 132, 133, 136, 151, 152, 187, 228, 245, 249, 257, 260, 262, 268, 272, 282, 303, 339, 353, 357, 358, 359, 360, 364, 374, 375, 377, 384, 387, 392, 398, 400, 405, 410, 420, 458, 469, 470, 471], "over": [2, 13, 14, 25, 32, 33, 35, 37, 41, 45, 54, 57, 60, 64, 65, 71, 77, 78, 84, 86, 90, 94, 99, 123, 131, 132, 141, 144, 146, 150, 151, 152, 159, 164, 178, 180, 185, 209, 210, 223, 235, 239, 242, 245, 246, 250, 253, 255, 257, 259, 267, 269, 273, 282, 284, 286, 289, 293, 297, 302, 303, 306, 311, 317, 328, 330, 331, 334, 335, 337, 346, 348, 353, 354, 356, 359, 361, 362, 364, 367, 368, 374, 379, 383, 385, 386, 389, 420, 430, 432, 433, 435, 436, 453, 462, 468, 469, 470, 472, 473, 475, 480, 486, 491, 506, 510, 512, 520], "network": [2, 4, 9, 10, 14, 19, 27, 32, 35, 37, 39, 40, 41, 42, 45, 48, 53, 54, 55, 57, 60, 62, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 78, 111, 132, 137, 138, 140, 146, 147, 148, 151, 178, 185, 206, 223, 259, 260, 276, 277, 278, 282, 284, 329, 330, 331, 339, 346, 362, 374, 383, 386, 387, 407, 438, 464, 465, 466, 467, 468, 475, 505, 506], "so": [2, 3, 5, 7, 8, 13, 14, 15, 16, 19, 24, 25, 32, 35, 37, 39, 41, 42, 45, 47, 54, 56, 57, 63, 65, 66, 68, 71, 75, 76, 77, 78, 79, 80, 87, 88, 93, 94, 96, 104, 107, 112, 119, 129, 136, 137, 139, 146, 151, 152, 172, 192, 223, 225, 247, 264, 278, 282, 283, 287, 293, 294, 301, 302, 305, 316, 325, 356, 357, 388, 402, 408, 419, 462, 464, 466, 469, 470, 510, 512, 520, 523], "also": [2, 5, 7, 8, 14, 15, 19, 23, 27, 32, 33, 35, 37, 38, 39, 40, 41, 42, 43, 44, 45, 47, 48, 51, 54, 55, 57, 58, 59, 60, 62, 64, 65, 66, 68, 69, 70, 71, 73, 74, 75, 76, 82, 83, 84, 85, 86, 88, 89, 90, 91, 92, 94, 95, 96, 97, 99, 102, 104, 109, 112, 113, 117, 124, 126, 135, 136, 144, 147, 148, 149, 151, 152, 155, 157, 178, 179, 180, 183, 193, 203, 209, 242, 256, 280, 281, 282, 283, 293, 296, 300, 327, 333, 365, 386, 387, 435, 438, 460, 461, 462, 467, 468, 469, 473, 474, 475, 476, 510, 512, 520], "dure": [2, 5, 13, 14, 15, 17, 25, 32, 38, 40, 41, 45, 54, 55, 62, 63, 66, 68, 69, 71, 74, 75, 76, 77, 84, 94, 96, 97, 136, 138, 139, 147, 154, 158, 173, 180, 186, 207, 218, 219, 221, 234, 235, 240, 245, 247, 250, 258, 263, 264, 268, 271, 274, 276, 278, 279, 282, 287, 297, 310, 316, 329, 342, 346, 352, 356, 358, 359, 367, 368, 373, 380, 382, 383, 386, 387, 392, 393, 394, 396, 400, 405, 410, 413, 425, 426, 429, 433, 434, 438, 462, 467, 469, 470, 475, 486, 510, 520, 523], "enumer": [2, 14, 83, 94], "split": [2, 3, 5, 23, 27, 32, 35, 37, 39, 41, 42, 45, 53, 54, 56, 57, 60, 69, 70, 72, 78, 84, 85, 91, 94, 96, 114, 115, 121, 123, 124, 135, 137, 154, 155, 157, 160, 162, 180, 183, 206, 209, 221, 223, 229, 231, 232, 255, 257, 259, 267, 278, 280, 286, 287, 293, 297, 302, 306, 311, 327, 341, 342, 345, 355, 359, 360, 366, 368, 369, 375, 389, 392, 396, 397, 402, 405, 415, 417, 419, 432, 434, 438, 450, 472, 475, 505], "For": [2, 5, 7, 8, 9, 11, 13, 14, 15, 17, 21, 23, 27, 29, 32, 33, 35, 36, 37, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 51, 54, 55, 56, 57, 59, 60, 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 83, 84, 86, 88, 93, 94, 96, 97, 99, 101, 103, 104, 106, 108, 109, 110, 112, 115, 117, 118, 121, 126, 131, 132, 135, 136, 137, 138, 143, 144, 146, 147, 148, 149, 151, 164, 165, 167, 176, 177, 189, 191, 203, 204, 209, 217, 250, 254, 257, 261, 262, 268, 270, 271, 276, 282, 284, 287, 288, 293, 296, 297, 305, 307, 335, 336, 344, 350, 355, 362, 364, 367, 368, 388, 419, 438, 461, 462, 464, 465, 466, 468, 469, 470, 473, 474, 475, 476, 483, 499, 506, 507, 508, 510, 511, 512, 517, 518, 520, 522, 524, 526, 527, 532, 536, 537, 539], "exampl": [2, 3, 4, 5, 7, 8, 9, 11, 13, 14, 15, 17, 18, 29, 33, 35, 36, 37, 39, 40, 41, 42, 43, 44, 46, 47, 48, 49, 50, 51, 55, 56, 57, 58, 59, 61, 62, 64, 65, 66, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 81, 82, 84, 85, 89, 90, 91, 92, 93, 94, 95, 96, 97, 99, 103, 104, 106, 107, 108, 109, 110, 116, 121, 123, 124, 126, 131, 132, 133, 135, 136, 137, 138, 142, 143, 144, 146, 147, 148, 149, 151, 164, 165, 167, 177, 182, 191, 203, 204, 206, 209, 217, 220, 250, 257, 262, 263, 264, 268, 270, 271, 281, 282, 284, 288, 293, 296, 297, 305, 307, 336, 341, 355, 362, 367, 384, 385, 388, 419, 459, 461, 464, 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 520, 526, 527, 532, 536, 537, 539], "hive": [2, 5, 14, 21, 22, 24, 28, 30, 33, 34, 37, 41, 50, 53, 54, 64, 66, 67, 68, 71, 73, 74, 76, 84, 94, 133, 136, 137, 138, 148, 149, 151, 152, 160, 172, 212, 281, 283, 284, 285, 287, 288, 289, 294, 295, 296, 298, 299, 309, 310, 321, 406, 410, 441, 462, 480, 482, 488, 505, 506, 515, 520, 543], "connector": [2, 4, 5, 8, 13, 14, 19, 21, 22, 29, 50, 64, 67, 81, 83, 91, 93, 95, 96, 125, 126, 132, 133, 136, 137, 138, 143, 144, 148, 149, 155, 159, 161, 162, 166, 167, 168, 172, 173, 177, 180, 182, 184, 186, 187, 188, 194, 198, 206, 207, 208, 212, 213, 217, 219, 223, 227, 232, 233, 242, 245, 246, 247, 251, 262, 267, 281, 282, 283, 284, 285, 288, 293, 294, 298, 300, 302, 306, 307, 311, 313, 316, 317, 321, 325, 460, 462, 471, 473, 478, 480, 482, 483, 486, 487, 493, 494, 499, 507, 508, 511, 514, 517, 518, 520, 521, 522, 523, 533, 542], "us": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 15, 16, 17, 19, 20, 22, 23, 24, 25, 27, 30, 31, 32, 33, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 50, 51, 52, 53, 55, 56, 57, 59, 60, 61, 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 76, 78, 79, 80, 82, 83, 84, 85, 86, 87, 88, 89, 90, 92, 93, 94, 95, 96, 97, 98, 99, 100, 101, 103, 104, 105, 106, 107, 108, 109, 110, 112, 113, 117, 119, 120, 121, 122, 123, 124, 125, 126, 127, 128, 131, 132, 133, 135, 136, 138, 139, 142, 143, 144, 146, 147, 148, 149, 150, 151, 154, 155, 157, 162, 166, 167, 168, 170, 172, 173, 174, 175, 176, 177, 178, 179, 180, 182, 183, 184, 186, 187, 189, 190, 191, 192, 193, 196, 197, 198, 200, 203, 206, 207, 208, 209, 210, 212, 213, 217, 218, 219, 220, 221, 223, 224, 225, 226, 232, 233, 235, 239, 240, 242, 243, 244, 245, 246, 247, 248, 249, 251, 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, 266, 267, 268, 269, 270, 271, 272, 274, 275, 276, 277, 278, 279, 280, 281, 282, 283, 284, 285, 288, 293, 294, 296, 297, 298, 301, 302, 303, 304, 305, 306, 307, 311, 312, 313, 315, 316, 317, 318, 320, 324, 325, 327, 328, 329, 331, 332, 333, 334, 335, 338, 339, 341, 342, 343, 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, 406, 407, 408, 409, 413, 414, 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, 442, 445, 446, 447, 448, 449, 450, 454, 456, 459, 460, 462, 463, 464, 465, 466, 468, 470, 471, 472, 473, 474, 476, 477, 480, 482, 483, 486, 488, 489, 490, 491, 503, 504, 505, 506, 510, 511, 512, 514, 520, 521, 523, 525, 544], "skip": [2, 14, 32, 37, 40, 45, 51, 55, 62, 64, 66, 67, 68, 69, 71, 74, 75, 76, 96, 112, 124, 126, 142, 162, 166, 173, 207, 209, 210, 245, 248, 254, 259, 264, 270, 271, 327, 331, 332, 339, 346, 348, 356, 360, 366, 367, 368, 371, 386, 391, 408, 429, 431, 438, 450, 471, 512, 520], "load": [2, 5, 13, 14, 19, 23, 25, 33, 39, 40, 42, 44, 45, 48, 54, 55, 59, 62, 66, 68, 69, 71, 73, 74, 75, 83, 93, 132, 137, 138, 146, 148, 161, 180, 192, 197, 209, 221, 235, 239, 246, 251, 256, 260, 302, 303, 307, 313, 315, 318, 341, 350, 355, 356, 357, 359, 367, 375, 380, 387, 389, 407, 417, 426, 427, 434, 462, 467, 471, 474, 476, 509], "partit": [2, 5, 8, 9, 10, 19, 25, 40, 41, 48, 51, 53, 57, 58, 59, 68, 84, 86, 94, 96, 110, 126, 131, 133, 142, 146, 148, 155, 156, 157, 158, 162, 163, 164, 165, 177, 180, 186, 188, 191, 194, 197, 198, 200, 206, 208, 209, 217, 218, 221, 224, 226, 227, 231, 235, 242, 245, 247, 249, 255, 256, 257, 258, 259, 261, 262, 264, 265, 266, 267, 268, 269, 270, 272, 274, 276, 278, 280, 282, 283, 287, 289, 293, 297, 302, 303, 304, 305, 306, 307, 312, 317, 323, 327, 331, 335, 337, 339, 342, 345, 352, 353, 354, 356, 357, 358, 359, 360, 362, 364, 365, 367, 368, 369, 370, 373, 375, 377, 379, 380, 381, 382, 383, 384, 386, 387, 388, 389, 390, 391, 392, 393, 395, 396, 397, 399, 400, 402, 403, 404, 405, 408, 409, 412, 414, 415, 417, 418, 420, 421, 423, 424, 425, 426, 427, 430, 432, 433, 434, 435, 436, 438, 439, 440, 443, 445, 447, 448, 449, 450, 452, 453, 455, 456, 457, 482, 486, 505, 506, 512, 520], "match": [2, 11, 14, 16, 22, 23, 25, 32, 37, 39, 43, 45, 51, 52, 54, 56, 57, 58, 59, 61, 72, 73, 84, 87, 88, 97, 100, 104, 105, 112, 121, 124, 135, 136, 146, 171, 173, 177, 179, 191, 196, 197, 209, 210, 218, 221, 226, 227, 255, 256, 261, 262, 266, 267, 282, 287, 295, 297, 302, 313, 336, 340, 350, 352, 354, 356, 359, 361, 368, 369, 372, 379, 380, 383, 384, 385, 386, 387, 389, 396, 408, 415, 422, 425, 426, 432, 433, 435, 456, 462, 467, 468, 469, 470, 473, 476, 493, 509, 511, 520, 542], "known": [2, 10, 35, 37, 54, 57, 66, 75, 84, 96, 104, 109, 112, 124, 144, 147, 242, 245, 246, 257, 263, 276, 278, 354, 373, 381, 383, 389, 391, 424, 425, 467, 471, 510], "prune": [2, 45, 126, 197, 223, 258, 279, 307, 342, 362, 365, 367, 371, 373, 392, 405, 437], "after": [2, 4, 6, 10, 11, 15, 23, 29, 32, 33, 35, 41, 44, 45, 51, 52, 54, 63, 64, 65, 66, 67, 73, 76, 78, 82, 83, 84, 85, 89, 90, 92, 94, 95, 112, 118, 124, 131, 135, 136, 139, 141, 146, 163, 169, 170, 173, 186, 210, 218, 223, 232, 241, 243, 245, 249, 251, 253, 257, 261, 270, 275, 279, 293, 303, 307, 318, 327, 330, 333, 346, 352, 356, 357, 360, 368, 373, 380, 384, 392, 396, 399, 401, 402, 403, 409, 415, 419, 421, 424, 432, 435, 438, 446, 465, 468, 471, 475, 476, 486, 512, 520], "complet": [2, 3, 5, 6, 8, 13, 14, 25, 27, 32, 37, 41, 52, 53, 54, 62, 76, 83, 85, 94, 109, 112, 134, 155, 162, 172, 180, 182, 198, 209, 217, 231, 236, 243, 248, 249, 253, 259, 265, 266, 274, 278, 282, 288, 307, 317, 338, 343, 345, 352, 358, 369, 373, 374, 375, 387, 394, 399, 400, 402, 403, 420, 423, 424, 434, 462, 468, 475, 506, 510], "them": [2, 3, 13, 14, 23, 25, 29, 32, 33, 37, 38, 40, 41, 42, 45, 47, 52, 54, 55, 56, 57, 60, 62, 65, 66, 68, 69, 71, 74, 75, 76, 88, 93, 94, 97, 104, 109, 110, 112, 123, 135, 137, 138, 146, 151, 182, 211, 233, 236, 253, 254, 293, 303, 305, 330, 345, 391, 394, 399, 400, 420, 462, 465, 470, 475, 510, 512, 517, 520], "down": [2, 3, 4, 6, 14, 19, 32, 35, 37, 38, 39, 40, 41, 42, 44, 55, 60, 62, 65, 66, 68, 69, 70, 71, 73, 74, 75, 76, 83, 84, 118, 136, 139, 148, 155, 158, 180, 192, 207, 209, 217, 223, 235, 257, 265, 287, 292, 307, 327, 330, 358, 367, 373, 374, 380, 381, 382, 387, 389, 390, 391, 399, 402, 410, 423, 432, 435, 436, 439, 454], "It": [2, 5, 10, 13, 14, 18, 25, 27, 32, 35, 37, 38, 39, 40, 41, 42, 43, 44, 45, 47, 48, 51, 53, 54, 56, 57, 58, 59, 60, 62, 63, 65, 66, 68, 71, 73, 74, 76, 77, 78, 82, 83, 84, 85, 87, 88, 89, 90, 91, 92, 93, 95, 96, 99, 100, 104, 112, 119, 123, 124, 126, 131, 132, 136, 137, 139, 144, 146, 148, 151, 152, 155, 157, 168, 170, 182, 183, 184, 187, 233, 242, 258, 264, 271, 278, 282, 296, 303, 304, 306, 307, 360, 366, 368, 369, 374, 380, 384, 387, 388, 392, 396, 401, 408, 423, 460, 461, 462, 464, 470, 475, 476, 510, 512, 520, 525], "disabl": [2, 3, 5, 9, 10, 12, 14, 15, 24, 27, 32, 33, 37, 39, 40, 42, 43, 45, 48, 52, 53, 54, 55, 57, 60, 62, 63, 64, 66, 67, 68, 69, 70, 71, 74, 75, 76, 136, 155, 163, 167, 176, 177, 181, 186, 209, 210, 211, 236, 239, 242, 245, 246, 254, 255, 261, 263, 265, 266, 267, 271, 273, 278, 297, 303, 307, 317, 320, 323, 327, 328, 331, 337, 339, 342, 343, 346, 347, 351, 352, 355, 356, 357, 358, 360, 361, 362, 364, 369, 374, 375, 381, 383, 384, 388, 394, 396, 398, 399, 403, 405, 408, 409, 414, 416, 419, 429, 430, 432, 433, 435, 436, 438, 439, 447, 448, 449, 453, 454, 455, 469, 475, 522], "either": [2, 5, 12, 13, 19, 29, 32, 35, 40, 45, 47, 51, 54, 55, 57, 59, 60, 62, 64, 65, 66, 68, 69, 71, 73, 74, 75, 76, 78, 83, 84, 88, 96, 100, 103, 104, 109, 112, 116, 124, 135, 136, 139, 144, 186, 262, 264, 282, 389, 405, 462, 464, 465, 467, 469, 508, 510, 512, 518, 520, 523], "enable_coordinator_dynamic_filters_distribut": 2, "fals": [2, 3, 5, 8, 11, 14, 15, 18, 23, 32, 33, 35, 37, 39, 40, 41, 42, 45, 48, 51, 52, 54, 55, 57, 58, 59, 60, 62, 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 76, 84, 88, 94, 99, 100, 103, 104, 109, 111, 112, 116, 124, 136, 142, 144, 148, 155, 162, 163, 178, 196, 197, 211, 217, 246, 255, 261, 272, 296, 297, 301, 307, 311, 327, 354, 362, 363, 380, 381, 397, 418, 419, 421, 434, 453, 462, 469, 470, 476, 497, 505, 520, 523], "result": [2, 5, 8, 9, 13, 14, 15, 19, 22, 23, 25, 29, 31, 32, 35, 37, 39, 40, 41, 42, 43, 45, 48, 52, 54, 55, 57, 59, 60, 62, 64, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 83, 84, 87, 88, 94, 96, 97, 99, 100, 101, 102, 104, 105, 107, 108, 109, 112, 116, 117, 119, 123, 124, 126, 131, 138, 144, 146, 148, 151, 156, 162, 164, 166, 170, 173, 176, 177, 178, 179, 183, 190, 191, 194, 198, 199, 202, 205, 206, 208, 209, 210, 212, 216, 217, 218, 219, 226, 228, 229, 232, 244, 245, 248, 254, 258, 259, 265, 266, 267, 268, 270, 271, 273, 274, 278, 280, 282, 283, 287, 293, 297, 298, 301, 302, 304, 307, 313, 315, 316, 317, 323, 327, 331, 334, 336, 338, 339, 340, 341, 342, 346, 348, 350, 352, 353, 354, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 368, 369, 371, 372, 373, 374, 375, 377, 379, 380, 381, 382, 383, 386, 387, 388, 389, 390, 391, 392, 393, 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, 408, 409, 410, 411, 413, 414, 415, 416, 417, 418, 420, 423, 424, 427, 428, 429, 430, 432, 433, 434, 437, 438, 439, 440, 441, 445, 446, 447, 448, 449, 450, 451, 454, 457, 459, 461, 462, 467, 469, 470, 471, 486, 490, 510, 512, 520, 526, 527, 532, 536, 537, 538, 539], "includ": [2, 4, 8, 12, 14, 15, 17, 23, 30, 32, 39, 40, 41, 42, 43, 45, 46, 47, 51, 54, 55, 57, 62, 64, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 83, 84, 85, 87, 88, 93, 94, 99, 100, 104, 108, 112, 121, 124, 129, 132, 135, 136, 137, 138, 139, 140, 142, 143, 144, 148, 151, 152, 208, 217, 219, 228, 231, 240, 244, 251, 259, 260, 262, 264, 265, 270, 271, 273, 274, 278, 282, 287, 293, 296, 300, 301, 304, 317, 328, 329, 332, 341, 344, 346, 347, 359, 360, 361, 367, 374, 380, 382, 386, 389, 399, 403, 405, 409, 415, 423, 428, 430, 450, 461, 462, 465, 467, 468, 470, 471, 474, 475, 476, 486, 489, 497, 510, 512, 513, 520, 523, 538], "benefit": [2, 10, 41, 57, 84, 148, 266, 307, 382], "overal": [2, 5, 11, 14, 15, 45, 48, 59, 72, 146, 148, 193, 520], "reduc": [2, 3, 4, 5, 9, 14, 18, 19, 22, 23, 25, 35, 41, 45, 48, 59, 60, 65, 70, 73, 99, 100, 114, 115, 136, 148, 155, 157, 158, 178, 180, 192, 209, 218, 219, 221, 223, 224, 232, 235, 237, 239, 240, 242, 243, 244, 249, 251, 253, 254, 256, 258, 259, 260, 261, 262, 270, 271, 273, 274, 276, 280, 287, 293, 297, 307, 311, 320, 321, 326, 329, 331, 337, 341, 346, 349, 350, 352, 354, 355, 356, 357, 359, 360, 364, 367, 368, 369, 371, 374, 375, 377, 380, 383, 385, 387, 389, 390, 392, 393, 395, 397, 400, 405, 410, 412, 420, 422, 424, 426, 427, 432, 433, 434, 435, 438, 443, 450, 451, 454, 455, 476, 510, 520], "traffic": [2, 32, 48, 132, 148, 386, 466, 479, 488], "between": [2, 3, 9, 13, 14, 15, 29, 32, 33, 35, 37, 39, 40, 41, 42, 43, 54, 55, 56, 57, 59, 60, 62, 64, 65, 66, 68, 69, 71, 73, 74, 75, 76, 78, 84, 86, 88, 94, 99, 103, 106, 109, 112, 114, 118, 120, 123, 126, 127, 138, 142, 144, 147, 148, 151, 171, 176, 183, 184, 186, 189, 203, 207, 209, 210, 213, 229, 245, 251, 254, 256, 261, 266, 267, 268, 269, 272, 275, 278, 282, 285, 287, 296, 303, 324, 333, 337, 339, 341, 347, 356, 358, 359, 360, 364, 366, 371, 373, 379, 380, 382, 388, 389, 390, 391, 396, 430, 433, 437, 438, 442, 450, 462, 466, 467, 471, 475, 478, 480, 505, 510, 512, 520], "sourc": [2, 5, 23, 24, 27, 30, 31, 32, 33, 34, 35, 37, 39, 41, 43, 45, 54, 55, 60, 64, 65, 67, 69, 73, 75, 77, 79, 80, 83, 84, 86, 87, 88, 93, 94, 96, 98, 105, 124, 132, 136, 140, 143, 144, 148, 150, 152, 155, 160, 198, 201, 208, 209, 219, 229, 235, 236, 249, 255, 256, 261, 262, 267, 271, 280, 285, 304, 317, 334, 372, 379, 385, 389, 419, 430, 436, 443, 457, 466, 468, 505, 506, 511, 514, 520, 538], "remot": [2, 5, 32, 39, 40, 42, 45, 55, 57, 59, 62, 66, 68, 69, 71, 73, 74, 75, 76, 144, 148, 184, 223, 228, 260, 263, 268, 273, 279, 280, 303, 311, 317, 367, 374, 377, 379, 387, 390, 402, 434], "enable_dynamic_filt": [2, 348], "support": [2, 4, 5, 7, 8, 10, 11, 13, 15, 18, 23, 27, 29, 30, 32, 33, 47, 48, 49, 51, 52, 58, 64, 67, 81, 82, 83, 84, 85, 87, 90, 92, 96, 98, 99, 104, 106, 107, 108, 109, 112, 113, 119, 121, 126, 128, 129, 132, 133, 136, 137, 138, 139, 140, 146, 148, 149, 154, 155, 158, 161, 162, 165, 169, 172, 173, 176, 177, 178, 180, 183, 186, 187, 191, 192, 194, 196, 197, 198, 206, 207, 208, 209, 210, 211, 212, 213, 217, 218, 219, 220, 221, 226, 228, 230, 231, 232, 233, 234, 235, 236, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 253, 254, 255, 256, 257, 258, 260, 261, 262, 263, 264, 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, 276, 277, 278, 279, 280, 281, 282, 283, 284, 285, 287, 293, 296, 297, 300, 302, 303, 304, 305, 306, 309, 313, 315, 317, 320, 323, 327, 328, 329, 330, 331, 332, 333, 334, 335, 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, 377, 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, 412, 413, 414, 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, 442, 444, 445, 446, 447, 448, 449, 450, 451, 452, 453, 454, 455, 456, 457, 459, 461, 462, 464, 465, 466, 467, 468, 469, 470, 471, 474, 478, 480, 486, 487, 493, 494, 499, 507, 508, 510, 511, 512, 517, 518, 520, 522, 524, 533, 542], "specif": [2, 3, 4, 8, 11, 12, 15, 25, 26, 33, 37, 39, 41, 42, 43, 44, 45, 47, 48, 51, 53, 54, 59, 62, 65, 66, 67, 68, 69, 71, 72, 74, 78, 80, 82, 84, 85, 89, 90, 91, 92, 94, 95, 98, 99, 103, 104, 109, 112, 124, 131, 132, 136, 137, 143, 146, 148, 151, 167, 177, 194, 196, 207, 213, 217, 218, 227, 242, 250, 254, 261, 276, 278, 280, 282, 287, 288, 302, 303, 305, 315, 335, 339, 359, 368, 380, 386, 388, 396, 399, 402, 426, 450, 456, 460, 462, 464, 468, 473, 475, 482, 486, 510, 512, 514, 520, 521], "relev": [2, 5, 35, 54, 57, 60, 76, 88, 136, 148, 335, 510], "underli": [2, 18, 37, 39, 41, 42, 43, 45, 54, 62, 65, 66, 67, 68, 71, 74, 76, 77, 84, 88, 94, 98, 99, 110, 140, 143, 144, 148, 151, 155, 217, 225, 264, 305, 330, 365, 373, 380, 381, 382, 400, 410, 418, 433, 462, 491, 514, 520], "databas": [2, 4, 33, 35, 37, 39, 40, 41, 42, 43, 45, 51, 54, 55, 62, 64, 66, 68, 69, 75, 79, 80, 84, 104, 132, 136, 137, 148, 151, 152, 182, 209, 217, 228, 241, 244, 253, 303, 330, 335, 341, 361, 364, 365, 368, 369, 370, 372, 373, 374, 377, 379, 380, 381, 382, 383, 391, 396, 398, 400, 401, 402, 403, 404, 410, 432, 457, 488], "storag": [2, 13, 25, 41, 46, 51, 53, 54, 60, 64, 77, 78, 94, 120, 127, 132, 138, 148, 197, 217, 259, 296, 303, 305, 310, 329, 332, 337, 342, 356, 358, 364, 366, 368, 372, 375, 382, 383, 387, 388, 399, 406, 408, 409, 413, 415, 420, 428, 430, 432, 486], "system": [2, 3, 4, 5, 6, 7, 11, 12, 14, 18, 24, 25, 27, 32, 33, 34, 36, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 51, 52, 53, 54, 55, 56, 57, 59, 60, 61, 62, 65, 66, 68, 69, 73, 74, 75, 76, 81, 82, 89, 90, 92, 93, 94, 96, 98, 124, 126, 132, 133, 135, 137, 139, 143, 148, 151, 155, 156, 157, 167, 173, 177, 178, 187, 207, 208, 210, 232, 244, 245, 246, 249, 251, 257, 260, 261, 262, 264, 266, 270, 271, 272, 274, 278, 279, 282, 287, 302, 303, 307, 328, 329, 331, 332, 333, 341, 344, 345, 346, 347, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 366, 368, 371, 379, 381, 383, 384, 386, 387, 388, 389, 392, 393, 394, 395, 396, 397, 401, 402, 403, 404, 408, 420, 427, 432, 434, 435, 438, 441, 450, 456, 458, 459, 464, 465, 466, 467, 471, 474, 475, 476, 482, 483, 486, 487, 488, 489, 490, 494, 499, 508, 518, 520, 521, 522, 523, 524, 534, 535], "document": [2, 4, 5, 8, 11, 27, 33, 37, 39, 40, 41, 43, 45, 46, 47, 51, 52, 55, 57, 60, 62, 65, 66, 68, 71, 74, 75, 76, 83, 124, 126, 132, 136, 137, 138, 143, 144, 149, 151, 156, 254, 257, 276, 281, 282, 303, 317, 355, 359, 372, 391, 462, 468, 470, 471, 472, 475, 478, 483, 486, 487, 493, 499, 507, 508, 511, 517, 518, 522, 533, 542], "further": [2, 3, 4, 5, 23, 32, 43, 52, 58, 64, 69, 73, 74, 76, 98, 140, 151, 209, 282, 302, 459, 467, 469, 477, 510, 520], "detail": [2, 3, 5, 12, 19, 27, 32, 35, 40, 41, 42, 43, 44, 45, 46, 47, 48, 52, 53, 54, 55, 57, 60, 62, 64, 65, 66, 68, 69, 71, 73, 74, 75, 76, 83, 85, 96, 97, 98, 112, 115, 132, 140, 143, 162, 172, 176, 183, 191, 197, 213, 214, 217, 229, 242, 244, 246, 248, 259, 264, 269, 272, 279, 280, 284, 285, 297, 307, 329, 330, 331, 332, 334, 344, 357, 368, 386, 388, 394, 410, 442, 462, 468, 469, 470, 474, 477, 478, 483, 486, 487, 493, 499, 505, 506, 507, 508, 510, 512, 517, 518, 520, 522, 533, 542], "depend": [2, 5, 13, 15, 32, 33, 37, 41, 45, 47, 53, 54, 57, 59, 60, 65, 68, 72, 73, 84, 93, 94, 99, 107, 109, 112, 126, 132, 136, 137, 144, 149, 157, 186, 213, 217, 232, 268, 287, 292, 334, 368, 399, 424, 455, 466, 468, 470, 510, 511, 520], "factor": [2, 5, 19, 29, 45, 48, 60, 64, 78, 79, 80, 126, 136, 148, 278, 280, 374], "planner": [2, 84, 149, 157, 183, 201, 206, 290, 293, 317, 323, 433], "given": [2, 5, 6, 13, 16, 18, 19, 23, 27, 37, 45, 51, 54, 57, 59, 60, 65, 73, 76, 78, 84, 86, 87, 88, 94, 95, 96, 99, 100, 101, 109, 112, 117, 118, 119, 120, 127, 144, 151, 209, 270, 476, 478, 482, 490, 499, 508, 513, 518, 520, 522], "current": [2, 5, 7, 14, 15, 20, 23, 25, 27, 32, 35, 40, 41, 44, 45, 48, 51, 52, 53, 54, 56, 57, 59, 62, 64, 73, 77, 84, 99, 100, 107, 112, 122, 128, 131, 132, 136, 137, 149, 157, 162, 167, 170, 172, 177, 178, 180, 183, 223, 232, 249, 264, 266, 280, 282, 283, 296, 297, 302, 306, 332, 360, 364, 368, 369, 375, 380, 384, 389, 390, 423, 428, 435, 462, 470, 476, 478, 485, 487, 488, 494, 507, 508, 510, 512, 516, 518, 519, 520, 521, 522, 523, 524, 525, 532, 533, 535, 536, 537, 539, 540, 543], "inner": [2, 18, 142, 148, 156, 162, 182, 186, 232, 248, 264, 265, 268, 278, 283, 313, 346, 359, 373, 374, 520], "IS": [2, 94, 99, 100, 114, 117, 142, 167, 266, 267, 339, 352, 359, 372, 377, 379, 386, 389, 391, 397, 402, 417, 418, 448, 484, 542], "NOT": [2, 40, 48, 55, 62, 65, 66, 68, 71, 74, 75, 76, 94, 99, 100, 102, 114, 116, 117, 142, 165, 194, 196, 246, 266, 274, 287, 320, 324, 333, 339, 365, 367, 377, 379, 389, 391, 397, 402, 417, 418, 419, 420, 421, 448, 450, 457, 480, 486, 488, 489, 490, 511], "distinct": [2, 41, 45, 57, 58, 68, 70, 94, 99, 114, 119, 123, 142, 144, 149, 167, 178, 179, 180, 182, 191, 219, 220, 229, 232, 236, 242, 255, 257, 263, 266, 267, 268, 274, 275, 277, 278, 288, 293, 302, 307, 313, 334, 339, 345, 346, 352, 366, 368, 373, 375, 377, 379, 381, 388, 389, 391, 392, 393, 396, 400, 403, 410, 436, 437, 439, 450, 454, 520, 538], "semi": [2, 165, 171, 301, 330, 344, 374, 389], "IN": [2, 39, 54, 55, 62, 66, 68, 70, 71, 73, 74, 75, 76, 124, 142, 166, 175, 176, 178, 180, 194, 218, 219, 225, 227, 231, 237, 242, 243, 262, 278, 313, 334, 339, 354, 356, 359, 364, 369, 373, 377, 379, 380, 383, 384, 387, 388, 389, 392, 405, 409, 419, 435, 448, 450, 457, 487, 493, 499, 508, 518, 522], "orc": [2, 45, 47, 48, 54, 155, 156, 166, 167, 168, 178, 191, 193, 196, 198, 207, 209, 210, 215, 217, 229, 240, 242, 243, 245, 247, 248, 249, 250, 254, 256, 257, 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, 276, 277, 278, 279, 304, 308, 315, 324, 325, 326, 327, 329, 331, 332, 333, 335, 336, 341, 342, 350, 354, 360, 368, 379, 383, 386, 387, 390, 392, 395, 396, 397, 403, 409, 414, 415, 418, 432, 434, 446, 450, 456, 486, 489, 490, 530], "parquet": [2, 41, 45, 47, 48, 53, 54, 155, 158, 169, 173, 180, 191, 194, 196, 207, 208, 213, 219, 228, 230, 235, 240, 245, 247, 265, 268, 269, 271, 272, 273, 278, 280, 304, 324, 327, 329, 330, 331, 332, 341, 351, 352, 354, 355, 356, 358, 359, 360, 361, 363, 364, 365, 366, 368, 371, 372, 373, 379, 383, 387, 388, 389, 390, 391, 392, 395, 396, 397, 403, 413, 414, 415, 416, 418, 419, 421, 423, 424, 425, 426, 429, 432, 433, 434, 435, 436, 437, 440, 442, 444, 445, 446, 447, 449, 450, 451, 452, 454, 455], "reader": [2, 5, 14, 36, 37, 45, 47, 64, 148, 151, 166, 167, 168, 169, 193, 194, 196, 198, 210, 213, 215, 217, 219, 220, 228, 230, 233, 235, 240, 245, 247, 250, 257, 259, 265, 267, 268, 269, 271, 272, 274, 278, 280, 299, 306, 315, 320, 321, 329, 331, 333, 366, 392, 428, 432, 433, 435, 436, 438, 445, 447, 451, 454], "stripe": [2, 45, 67, 178, 210, 257, 262, 266, 267, 269, 270, 273, 274, 275, 277, 279, 324, 368, 432, 434, 446], "stage": [2, 5, 9, 13, 14, 27, 45, 51, 83, 84, 94, 146, 148, 157, 160, 191, 193, 198, 207, 208, 210, 211, 217, 221, 239, 248, 273, 274, 280, 322, 328, 333, 344, 345, 367, 368, 375, 393, 403, 406, 408, 415, 426, 428, 429, 432, 439, 505, 506], "size": [2, 7, 10, 13, 18, 19, 22, 25, 35, 39, 40, 41, 42, 43, 44, 45, 48, 51, 53, 54, 55, 62, 64, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 76, 84, 100, 101, 117, 124, 137, 147, 148, 149, 157, 180, 198, 200, 206, 213, 218, 221, 223, 234, 235, 248, 249, 250, 255, 256, 257, 260, 262, 263, 266, 267, 270, 271, 272, 273, 274, 275, 277, 278, 279, 280, 282, 297, 305, 321, 328, 329, 331, 335, 345, 346, 355, 356, 358, 360, 361, 366, 367, 372, 373, 379, 387, 388, 389, 390, 391, 398, 410, 413, 414, 430, 432, 434, 438, 442, 444, 453, 455, 472, 473, 480, 497, 506, 510, 520, 538], "build": [2, 9, 16, 25, 40, 42, 55, 58, 62, 66, 68, 69, 71, 74, 75, 76, 84, 88, 96, 146, 157, 217, 251, 274, 277, 287, 369, 448], "you": [2, 3, 4, 5, 6, 7, 10, 11, 12, 15, 16, 23, 25, 26, 27, 29, 32, 33, 35, 37, 38, 39, 40, 41, 42, 43, 44, 45, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 59, 60, 61, 62, 64, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 84, 86, 88, 93, 96, 97, 98, 99, 104, 112, 119, 121, 126, 134, 135, 136, 137, 138, 139, 143, 144, 146, 148, 151, 152, 155, 159, 167, 170, 172, 173, 177, 178, 188, 192, 194, 211, 212, 213, 217, 220, 251, 255, 260, 270, 275, 282, 283, 287, 293, 294, 296, 302, 303, 305, 307, 316, 317, 320, 325, 327, 334, 336, 342, 350, 352, 375, 397, 446, 459, 460, 461, 462, 464, 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 480, 486, 506, 510, 512, 520, 521, 526, 527, 532, 536, 537, 539], "take": [2, 5, 13, 15, 19, 23, 32, 33, 35, 36, 40, 41, 42, 44, 45, 48, 51, 52, 54, 55, 62, 64, 65, 66, 68, 69, 71, 72, 74, 75, 76, 88, 94, 96, 99, 100, 109, 112, 120, 144, 146, 148, 149, 151, 166, 170, 180, 183, 220, 239, 240, 245, 246, 247, 256, 264, 267, 268, 274, 275, 295, 297, 300, 357, 362, 364, 398, 438, 466, 475, 510], "closer": 2, "look": [2, 35, 59, 70, 77, 84, 86, 88, 93, 98, 99, 119, 137, 148, 359, 462, 464, 465], "explain": [2, 5, 14, 23, 24, 32, 41, 45, 52, 54, 108, 112, 133, 135, 143, 145, 148, 156, 162, 198, 223, 228, 229, 230, 245, 246, 247, 248, 261, 268, 274, 276, 277, 278, 280, 296, 302, 317, 329, 333, 334, 338, 341, 346, 352, 358, 359, 361, 371, 373, 375, 380, 382, 384, 390, 391, 392, 395, 396, 415, 416, 421, 424, 425, 426, 427, 428, 430, 432, 435, 436, 446, 450, 477, 510, 512], "plan": [2, 14, 27, 41, 45, 54, 70, 77, 84, 94, 96, 143, 146, 147, 148, 151, 155, 162, 163, 167, 172, 175, 178, 180, 186, 190, 191, 192, 193, 196, 208, 209, 217, 218, 221, 223, 225, 226, 229, 232, 234, 235, 236, 237, 239, 240, 241, 242, 244, 247, 248, 249, 253, 254, 255, 257, 258, 259, 261, 263, 264, 266, 268, 270, 271, 272, 273, 274, 275, 276, 278, 279, 289, 294, 297, 302, 303, 305, 309, 312, 316, 317, 324, 327, 329, 330, 331, 333, 338, 341, 344, 345, 346, 347, 352, 358, 359, 360, 361, 365, 366, 367, 369, 373, 375, 379, 380, 382, 383, 386, 387, 389, 390, 392, 394, 396, 398, 401, 403, 405, 408, 418, 419, 421, 424, 426, 432, 436, 438, 442, 445, 446, 450, 505, 506, 520], "analyz": [2, 23, 27, 32, 41, 45, 54, 62, 66, 71, 77, 84, 133, 143, 147, 186, 198, 223, 228, 230, 246, 248, 267, 268, 274, 277, 278, 280, 287, 301, 328, 332, 346, 350, 352, 353, 354, 357, 358, 371, 373, 375, 382, 384, 389, 390, 391, 392, 395, 396, 414, 425, 426, 427, 428, 430, 432, 435, 436, 447, 453, 465, 477, 505, 510], "ad": [2, 9, 11, 19, 22, 32, 33, 40, 42, 45, 48, 54, 55, 57, 59, 60, 62, 66, 68, 69, 71, 73, 74, 75, 76, 83, 84, 88, 94, 96, 100, 104, 112, 117, 126, 131, 136, 148, 160, 161, 172, 173, 180, 192, 198, 204, 209, 211, 231, 254, 256, 261, 265, 278, 282, 287, 293, 297, 301, 302, 303, 304, 307, 310, 315, 317, 333, 334, 357, 368, 369, 372, 373, 388, 390, 394, 401, 403, 409, 414, 420, 421, 422, 430, 432, 440, 445, 446, 449, 462, 464, 466, 468, 469, 470, 476, 489, 510, 520], "s": [2, 3, 5, 8, 13, 15, 23, 32, 33, 35, 37, 39, 40, 41, 42, 45, 46, 47, 49, 51, 52, 54, 55, 57, 59, 60, 62, 65, 66, 68, 69, 70, 71, 73, 75, 76, 83, 84, 88, 90, 91, 94, 95, 96, 97, 99, 100, 101, 102, 106, 107, 109, 112, 113, 117, 118, 120, 121, 131, 132, 135, 136, 137, 144, 146, 151, 154, 157, 162, 205, 218, 236, 242, 254, 257, 280, 281, 284, 296, 339, 356, 358, 365, 367, 379, 389, 392, 395, 409, 415, 423, 424, 431, 438, 446, 448, 449, 454, 456, 461, 462, 464, 465, 467, 468, 469, 470, 471, 472, 473, 475, 476, 486, 506, 508, 511, 518, 520, 532], "abov": [2, 5, 14, 18, 22, 23, 38, 40, 41, 42, 45, 51, 55, 57, 59, 62, 66, 68, 69, 70, 71, 73, 74, 75, 76, 78, 88, 94, 104, 107, 112, 123, 136, 144, 148, 161, 177, 226, 256, 293, 296, 353, 357, 360, 398, 449, 460, 462, 468, 472, 475, 505, 510, 520], "obtain": [2, 64, 76, 86, 113, 468, 470, 475, 520], "statement": [2, 5, 23, 24, 32, 35, 36, 37, 39, 40, 41, 42, 43, 44, 45, 47, 50, 53, 54, 55, 56, 57, 59, 60, 62, 63, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 83, 87, 88, 91, 94, 98, 99, 104, 113, 124, 133, 140, 141, 144, 146, 147, 148, 177, 183, 187, 198, 217, 218, 226, 244, 246, 247, 253, 254, 257, 262, 267, 270, 274, 275, 278, 279, 282, 288, 297, 302, 328, 329, 330, 334, 335, 341, 343, 347, 352, 355, 357, 358, 359, 360, 373, 377, 379, 380, 381, 383, 384, 387, 389, 390, 392, 393, 394, 396, 397, 398, 399, 400, 402, 406, 413, 415, 421, 422, 424, 428, 432, 433, 436, 443, 445, 447, 448, 449, 450, 451, 452, 453, 454, 456, 471, 478, 480, 486, 492, 496, 497, 499, 503, 504, 505, 506, 507, 508, 511, 513, 517, 518, 520, 522, 524, 528, 529, 530, 531], "show": [2, 23, 24, 32, 35, 36, 38, 40, 41, 45, 47, 54, 56, 57, 58, 59, 60, 61, 62, 64, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 79, 80, 84, 87, 98, 104, 112, 119, 123, 126, 133, 135, 137, 143, 148, 149, 156, 167, 183, 188, 208, 226, 232, 237, 239, 242, 249, 254, 255, 258, 259, 261, 262, 263, 264, 266, 267, 268, 274, 278, 279, 280, 281, 301, 306, 307, 317, 327, 329, 330, 331, 333, 339, 341, 343, 345, 346, 351, 352, 353, 356, 357, 358, 359, 360, 361, 371, 375, 379, 380, 383, 385, 386, 387, 390, 391, 395, 397, 398, 400, 402, 405, 406, 412, 420, 422, 425, 428, 430, 432, 436, 438, 448, 456, 460, 461, 462, 464, 465, 471, 475, 477, 486, 489, 491, 494, 495, 498, 505, 506, 507, 510, 514, 515, 517, 520, 523], "dynamicfilterassign": 2, "innerjoin": 2, "df_370": 2, "symbol": [2, 53, 104, 106, 144, 218, 262, 505], "see": [2, 3, 5, 6, 17, 19, 20, 23, 25, 26, 27, 32, 35, 37, 39, 40, 41, 42, 43, 44, 45, 46, 47, 52, 54, 56, 57, 58, 59, 60, 61, 62, 64, 65, 66, 68, 69, 71, 73, 74, 75, 76, 83, 84, 88, 93, 96, 97, 99, 102, 109, 112, 115, 117, 118, 124, 126, 131, 132, 135, 136, 137, 138, 144, 148, 151, 167, 172, 176, 183, 194, 213, 217, 236, 244, 258, 264, 266, 281, 282, 284, 285, 296, 297, 302, 305, 307, 328, 334, 336, 357, 460, 465, 468, 469, 470, 472, 475, 483, 493, 510, 511, 520, 542], "dynamicfilt": [2, 359, 366, 369], "part": [2, 5, 9, 15, 25, 32, 35, 41, 43, 45, 48, 51, 54, 57, 58, 59, 60, 65, 73, 78, 84, 88, 89, 99, 108, 112, 122, 132, 141, 143, 144, 147, 148, 217, 233, 285, 293, 296, 302, 329, 380, 392, 396, 432, 436, 445, 450, 456, 464, 468, 469, 471, 510, 512, 520, 521], "scanfilterproject": [2, 84, 147, 148, 506], "associ": [2, 23, 24, 35, 51, 52, 73, 84, 99, 112, 117, 147, 151, 155, 270, 279, 336, 347, 393, 405, 470, 510, 512, 520], "success": [2, 4, 6, 32, 85, 118, 138, 148, 178, 397, 459, 461, 465, 469, 472, 486, 512], "fragment": [2, 41, 45, 54, 84, 91, 94, 129, 148, 317, 320, 321, 346, 505, 506], "1": [2, 3, 5, 8, 9, 11, 13, 14, 15, 19, 23, 24, 25, 29, 32, 35, 36, 38, 39, 41, 43, 44, 45, 47, 48, 50, 51, 53, 54, 55, 56, 57, 59, 60, 62, 64, 65, 66, 67, 68, 69, 70, 71, 72, 74, 75, 76, 78, 79, 80, 88, 93, 94, 96, 99, 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 112, 113, 117, 118, 119, 120, 121, 123, 124, 126, 127, 128, 129, 131, 135, 137, 138, 144, 147, 148, 153, 158, 198, 212, 213, 217, 232, 233, 235, 240, 245, 250, 251, 253, 258, 264, 276, 278, 281, 288, 293, 303, 307, 314, 316, 321, 327, 331, 337, 350, 354, 367, 368, 380, 381, 384, 387, 388, 389, 396, 398, 405, 413, 432, 441, 462, 465, 472, 475, 476, 496, 497, 503, 504, 505, 506, 509, 510, 513, 520, 530, 544], "output": [2, 5, 9, 10, 14, 23, 24, 27, 29, 35, 41, 45, 53, 54, 57, 59, 73, 76, 78, 88, 94, 96, 99, 100, 112, 119, 133, 136, 137, 143, 147, 148, 151, 179, 180, 182, 186, 189, 196, 197, 198, 206, 218, 221, 223, 226, 227, 230, 232, 244, 245, 246, 248, 254, 255, 261, 267, 274, 278, 280, 287, 303, 307, 317, 320, 323, 325, 327, 329, 332, 333, 338, 341, 342, 346, 359, 364, 380, 383, 386, 389, 395, 398, 405, 418, 421, 424, 425, 426, 427, 428, 430, 432, 434, 435, 436, 446, 448, 453, 464, 465, 468, 477, 505, 506, 510, 512, 513, 520, 525], "count_3": 2, "aggreg": [2, 5, 12, 15, 40, 41, 54, 55, 62, 65, 66, 68, 70, 71, 76, 77, 84, 98, 110, 113, 117, 119, 123, 127, 133, 151, 152, 157, 162, 176, 178, 184, 187, 191, 194, 196, 198, 203, 208, 209, 212, 213, 217, 219, 220, 221, 223, 226, 227, 229, 232, 236, 239, 240, 242, 244, 246, 248, 255, 259, 260, 263, 264, 266, 267, 268, 270, 271, 274, 277, 279, 288, 291, 293, 296, 302, 307, 310, 313, 315, 317, 327, 328, 330, 333, 338, 346, 354, 357, 359, 361, 362, 364, 367, 368, 369, 370, 371, 372, 373, 374, 375, 379, 384, 385, 386, 388, 389, 391, 392, 393, 394, 395, 396, 400, 401, 402, 403, 406, 408, 410, 416, 418, 419, 420, 422, 425, 426, 430, 432, 434, 435, 436, 437, 438, 439, 441, 442, 447, 453, 457, 466, 486, 505, 506, 512, 520], "bigint": [2, 29, 35, 36, 37, 38, 39, 40, 41, 42, 43, 45, 47, 50, 54, 55, 57, 58, 59, 60, 62, 65, 66, 68, 69, 70, 71, 73, 75, 76, 78, 84, 96, 99, 100, 101, 102, 103, 104, 105, 106, 107, 109, 110, 112, 117, 118, 119, 120, 121, 123, 124, 126, 128, 129, 131, 147, 148, 171, 177, 189, 191, 194, 217, 227, 228, 234, 241, 245, 251, 264, 267, 277, 293, 296, 303, 339, 348, 355, 361, 380, 390, 392, 421, 433, 456, 480, 489, 496, 497, 505, 506, 510, 520, 527, 530], "hashvalu": [2, 148, 505, 506], "hashvalue_4": 2, "estim": [2, 5, 14, 70, 83, 99, 123, 144, 146, 147, 148, 196, 251, 266, 269, 270, 271, 273, 274, 276, 278, 280, 329, 332, 346, 360, 367, 381, 390, 400, 401, 425, 433, 441, 505, 506, 538], "0": [2, 3, 5, 8, 11, 13, 14, 16, 17, 18, 27, 29, 30, 32, 33, 35, 39, 40, 41, 42, 43, 45, 47, 53, 54, 55, 57, 58, 59, 60, 62, 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, 75, 76, 78, 79, 80, 84, 88, 93, 94, 96, 99, 100, 102, 103, 104, 105, 107, 108, 109, 111, 112, 113, 117, 118, 119, 120, 123, 124, 126, 127, 128, 129, 131, 133, 135, 136, 137, 144, 147, 148, 153, 328, 332, 334, 344, 345, 348, 374, 380, 381, 386, 390, 391, 392, 396, 397, 398, 399, 402, 406, 408, 414, 415, 417, 420, 421, 424, 427, 428, 430, 432, 435, 441, 449, 452, 453, 454, 458, 459, 467, 471, 472, 475, 496, 497, 505, 506, 510, 520], "0b": [2, 32, 135, 137, 144, 148, 455, 472, 475, 505, 506], "cpu": [2, 3, 4, 13, 14, 19, 23, 25, 45, 136, 147, 148, 157, 158, 160, 180, 186, 189, 197, 209, 211, 219, 221, 222, 223, 241, 242, 243, 246, 249, 250, 251, 253, 256, 257, 259, 275, 276, 278, 280, 293, 301, 303, 333, 335, 337, 345, 364, 365, 389, 391, 392, 400, 426, 428, 432, 442, 466, 475, 505, 506], "replic": [2, 9, 39, 45, 60, 99, 276, 296, 369, 374], "filterpred": [2, 147, 506], "true": [2, 3, 5, 8, 10, 11, 12, 13, 14, 15, 19, 20, 21, 22, 23, 24, 32, 33, 35, 37, 39, 40, 41, 42, 43, 45, 48, 51, 52, 53, 54, 55, 57, 59, 60, 62, 64, 66, 67, 68, 69, 70, 71, 73, 74, 75, 76, 84, 88, 94, 99, 100, 103, 104, 105, 109, 111, 112, 116, 117, 121, 124, 136, 142, 144, 152, 155, 178, 183, 191, 192, 193, 198, 225, 244, 246, 251, 261, 272, 283, 297, 302, 307, 310, 317, 336, 340, 350, 352, 357, 358, 362, 380, 389, 390, 392, 394, 429, 441, 446, 448, 449, 450, 462, 466, 468, 469, 470, 471, 475, 497, 505, 510, 520, 523, 532], "combine_hash": [2, 148, 505, 506], "coalesc": [2, 78, 99, 100, 113, 114, 115, 148, 249, 270, 328, 331, 335, 505, 506], "hash_cod": [2, 148, 505, 506], "regular": [2, 8, 19, 23, 45, 51, 58, 70, 84, 98, 133, 251, 262, 332, 336, 355, 416, 418, 459, 461, 462, 467, 468, 471, 475, 476, 505, 506, 510, 512, 520], "localexchang": [2, 148, 505, 506], "hash": [2, 5, 9, 10, 17, 54, 57, 73, 94, 110, 123, 144, 146, 148, 155, 157, 163, 165, 170, 176, 177, 209, 220, 230, 235, 247, 270, 272, 276, 287, 307, 310, 326, 334, 368, 398, 434, 472, 505, 506, 523], "remotesourc": [2, 148, 505, 506], "2": [2, 3, 5, 6, 8, 11, 13, 16, 17, 19, 23, 27, 29, 30, 32, 33, 35, 36, 39, 40, 41, 43, 45, 47, 48, 54, 55, 57, 59, 60, 62, 64, 65, 66, 68, 70, 71, 72, 73, 74, 75, 76, 78, 84, 94, 96, 97, 99, 100, 102, 104, 105, 106, 107, 109, 112, 113, 117, 118, 119, 121, 123, 128, 129, 131, 133, 136, 137, 138, 144, 148, 153, 157, 203, 204, 209, 210, 217, 233, 235, 251, 264, 269, 288, 293, 307, 327, 332, 341, 348, 357, 381, 385, 389, 391, 397, 405, 408, 415, 420, 424, 430, 432, 435, 441, 449, 453, 458, 459, 467, 471, 475, 491, 496, 497, 505, 506, 509, 510, 520, 544], "hashvalue_5": 2, "hashvalue_6": 2, "cast": [2, 43, 54, 58, 65, 68, 84, 100, 105, 106, 107, 110, 113, 114, 115, 116, 117, 120, 126, 127, 142, 144, 156, 162, 165, 171, 176, 178, 182, 183, 189, 197, 207, 208, 210, 217, 229, 236, 241, 243, 247, 248, 250, 251, 254, 255, 257, 262, 264, 269, 270, 274, 287, 293, 301, 305, 317, 342, 362, 371, 373, 384, 385, 387, 389, 391, 392, 393, 396, 398, 412, 414, 417, 421, 424, 426, 520, 542], "AS": [2, 15, 23, 24, 29, 35, 37, 38, 39, 41, 42, 45, 47, 50, 54, 55, 58, 60, 62, 63, 65, 66, 68, 69, 71, 75, 81, 94, 99, 100, 104, 105, 106, 107, 110, 112, 113, 116, 117, 119, 123, 126, 131, 133, 142, 143, 144, 148, 155, 164, 166, 177, 178, 191, 193, 196, 197, 208, 212, 238, 247, 248, 270, 283, 287, 288, 296, 302, 320, 335, 353, 357, 361, 371, 374, 379, 382, 386, 390, 398, 412, 413, 414, 418, 421, 423, 432, 434, 438, 441, 443, 449, 450, 454, 456, 472, 475, 477, 486, 489, 491, 497, 510, 511, 512, 520, 544], "char": [2, 40, 45, 55, 62, 66, 68, 69, 71, 75, 76, 84, 112, 124, 217, 223, 236, 242, 248, 249, 250, 271, 272, 273, 327, 330, 333, 335, 342, 357, 362, 367, 373, 380, 381, 386, 388, 389, 391, 399, 401, 419, 430, 432, 438, 450], "int": [2, 37, 39, 40, 54, 55, 57, 59, 60, 62, 70, 73, 78, 84, 96, 100, 103, 191, 240, 341, 362, 368, 375, 377, 432], "popul": [2, 23, 41, 48, 50, 54, 57, 69, 74, 76, 99, 486, 514, 538], "statist": [2, 10, 14, 15, 23, 27, 32, 36, 40, 42, 48, 55, 60, 64, 68, 69, 74, 75, 77, 83, 84, 94, 133, 145, 146, 147, 155, 158, 188, 217, 227, 228, 230, 232, 242, 243, 248, 249, 257, 261, 262, 263, 265, 267, 269, 272, 273, 274, 276, 277, 278, 280, 307, 320, 327, 328, 329, 337, 338, 339, 343, 352, 353, 354, 356, 357, 359, 360, 361, 362, 365, 367, 368, 369, 373, 374, 380, 381, 386, 388, 389, 391, 392, 396, 400, 403, 406, 417, 418, 419, 420, 421, 422, 428, 429, 430, 432, 433, 435, 436, 437, 438, 439, 441, 443, 446, 447, 450, 452, 455, 457, 482, 506], "about": [2, 3, 4, 27, 32, 33, 35, 36, 39, 40, 41, 42, 45, 48, 52, 53, 54, 55, 56, 60, 63, 64, 66, 68, 69, 71, 74, 75, 76, 77, 83, 84, 86, 88, 91, 94, 97, 122, 125, 126, 132, 135, 136, 138, 151, 204, 209, 242, 246, 249, 278, 303, 335, 345, 359, 362, 367, 382, 388, 399, 406, 414, 424, 441, 468, 486, 505, 510, 520], "queryinfo": 2, "json": [2, 3, 5, 6, 12, 23, 24, 27, 32, 37, 40, 42, 44, 45, 50, 51, 54, 55, 58, 60, 62, 66, 67, 68, 69, 70, 71, 74, 75, 76, 78, 83, 84, 94, 98, 113, 133, 165, 171, 176, 182, 189, 198, 213, 231, 233, 236, 241, 243, 247, 250, 251, 254, 257, 267, 274, 281, 282, 287, 293, 296, 305, 309, 310, 312, 313, 317, 325, 329, 331, 339, 341, 342, 348, 367, 368, 369, 372, 375, 384, 387, 388, 390, 410, 411, 412, 415, 419, 420, 421, 422, 424, 426, 427, 436, 438, 441, 446, 451, 452, 461, 462, 467, 470, 471, 476], "avail": [2, 4, 5, 7, 8, 9, 13, 14, 15, 19, 20, 25, 27, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 48, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63, 64, 65, 66, 68, 69, 70, 71, 73, 74, 75, 76, 77, 78, 79, 80, 84, 86, 93, 96, 112, 125, 126, 132, 136, 137, 138, 139, 140, 146, 151, 154, 173, 177, 209, 235, 243, 272, 276, 279, 282, 293, 296, 305, 328, 339, 344, 347, 359, 360, 368, 383, 392, 393, 399, 432, 436, 447, 459, 461, 462, 463, 467, 470, 471, 472, 473, 482, 483, 486, 488, 489, 490, 512, 526, 532, 538], "through": [2, 13, 23, 25, 32, 37, 48, 51, 54, 58, 59, 64, 69, 84, 126, 148, 242, 244, 302, 359, 368, 375, 380, 383, 386, 392, 393, 411, 416, 421, 425, 428, 434, 438, 461, 468, 469, 510], "querystat": [2, 251, 255, 275], "section": [2, 3, 8, 13, 27, 32, 33, 34, 35, 37, 39, 40, 41, 43, 45, 47, 48, 52, 54, 57, 60, 62, 64, 65, 66, 68, 69, 71, 73, 75, 76, 83, 84, 98, 107, 112, 128, 129, 134, 135, 136, 138, 140, 144, 148, 151, 152, 373, 462, 466, 468, 470, 475, 477, 486, 510, 512], "found": [2, 32, 33, 39, 48, 52, 54, 78, 86, 87, 93, 94, 99, 100, 105, 109, 112, 117, 121, 123, 124, 135, 156, 157, 287, 431, 435, 469, 505, 510, 512, 520, 538], "dynamicfiltersstat": 2, "structur": [2, 41, 43, 53, 54, 59, 64, 78, 99, 112, 132, 135, 140, 147, 148, 157, 167, 176, 180, 182, 183, 217, 253, 265, 266, 310, 327, 339, 341, 361, 387, 416, 423, 435, 440, 446, 448, 520], "dynamicfilterdomainstat": 2, "dynamicfilterid": 2, "simplifieddomain": 2, "sortedrangeset": 2, "type": [2, 5, 6, 7, 9, 11, 12, 14, 15, 16, 17, 18, 19, 21, 22, 23, 24, 32, 33, 36, 45, 47, 48, 51, 52, 53, 56, 58, 64, 81, 83, 85, 86, 87, 88, 93, 98, 99, 100, 104, 105, 106, 107, 108, 109, 119, 120, 123, 126, 127, 130, 132, 133, 135, 137, 140, 146, 148, 151, 156, 157, 162, 167, 173, 175, 176, 177, 178, 180, 182, 183, 184, 190, 191, 194, 201, 204, 206, 207, 208, 209, 210, 212, 213, 217, 222, 223, 225, 227, 228, 229, 230, 231, 232, 233, 234, 236, 240, 241, 242, 245, 247, 249, 251, 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, 278, 279, 280, 287, 288, 301, 302, 303, 307, 309, 310, 313, 314, 317, 323, 324, 326, 327, 328, 329, 330, 331, 333, 334, 336, 337, 339, 340, 341, 343, 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, 379, 380, 381, 382, 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 399, 401, 402, 404, 409, 412, 413, 414, 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, 430, 432, 433, 434, 435, 436, 437, 438, 439, 440, 442, 446, 447, 448, 449, 450, 452, 453, 454, 456, 457, 458, 461, 462, 464, 467, 468, 469, 470, 472, 473, 475, 476, 477, 480, 496, 497, 506, 510, 520, 523, 525, 527, 532, 538, 542], "rang": [2, 35, 39, 40, 41, 42, 45, 54, 55, 57, 59, 62, 66, 68, 69, 71, 72, 73, 74, 75, 76, 78, 84, 96, 99, 103, 105, 107, 108, 109, 112, 118, 120, 126, 131, 137, 144, 166, 178, 223, 242, 246, 250, 255, 267, 308, 327, 354, 361, 366, 373, 379, 380, 384, 392, 396, 423, 432, 433, 473, 505, 506, 510, 512, 520, 525], "3": [2, 9, 15, 17, 19, 23, 29, 32, 35, 36, 37, 39, 41, 48, 53, 54, 55, 56, 57, 60, 62, 64, 65, 66, 67, 68, 70, 71, 72, 74, 75, 76, 78, 84, 94, 96, 99, 100, 102, 104, 106, 107, 109, 112, 113, 117, 119, 121, 123, 126, 131, 136, 137, 144, 147, 153, 204, 207, 209, 232, 341, 345, 346, 348, 350, 354, 355, 358, 365, 366, 368, 371, 372, 373, 375, 380, 381, 386, 389, 391, 396, 398, 415, 417, 423, 428, 429, 432, 453, 470, 475, 486, 496, 503, 504, 506, 509, 510, 520, 536, 544], "2451546": 2, "2451905": 2, "collectiondur": 2, "34": [2, 54, 58, 99, 209, 330], "lazydynamicfilt": 2, "replicateddynamicfilt": 2, "totaldynamicfilt": 2, "dynamicfilterscomplet": 2, "verifi": [2, 11, 32, 33, 48, 52, 58, 95, 109, 132, 137, 148, 183, 189, 190, 281, 302, 303, 305, 357, 359, 375, 379, 461, 464, 467, 468, 469, 470, 471], "dynamicfiltersplitsprocess": 2, "record": [2, 12, 32, 36, 37, 41, 45, 54, 57, 59, 73, 78, 84, 148, 155, 191, 217, 257, 277, 279, 282, 295, 328, 331, 337, 353, 354, 363, 373, 386, 414], "operatortyp": [2, 88], "scanfilterandprojectoper": 2, "totaldriv": 2, "addinputcal": 2, "762": 2, "addinputwal": 2, "00n": [2, 506], "addinputcpu": 2, "physicalinputdatas": 2, "physicalinputposit": 2, "28800991": 2, "inputposit": [2, 7], "report": [2, 5, 13, 32, 33, 65, 68, 83, 86, 152, 186, 196, 208, 213, 219, 248, 256, 257, 280, 282, 287, 289, 328, 330, 331, 335, 337, 338, 339, 347, 348, 350, 360, 363, 366, 368, 373, 375, 380, 382, 386, 387, 389, 390, 392, 395, 403, 405, 406, 419, 432, 433, 434, 439, 449, 451, 452, 453, 465, 506], "11859": 2, "8": [2, 23, 24, 32, 33, 39, 40, 57, 58, 59, 66, 73, 78, 84, 101, 102, 104, 106, 107, 111, 112, 117, 119, 121, 124, 129, 136, 144, 153, 156, 209, 210, 233, 250, 261, 265, 268, 282, 303, 314, 342, 374, 391, 392, 397, 422, 425, 451, 472, 475, 497, 506, 520], "84m": 2, "19kb": 2, "78": [2, 112, 153, 506], "00m": [2, 506], "30": [2, 23, 36, 40, 42, 48, 55, 58, 60, 62, 64, 66, 68, 69, 71, 72, 74, 75, 76, 103, 107, 113, 117, 144, 153, 293, 328, 470, 506, 520], "00": [2, 32, 53, 54, 56, 60, 65, 72, 78, 107, 112, 135, 137, 144, 147, 162, 234, 262, 263, 293, 368, 472, 475, 506, 525], "schedul": [2, 5, 8, 15, 19, 41, 45, 56, 60, 133, 136, 137, 151, 155, 156, 173, 175, 177, 192, 197, 208, 209, 210, 217, 221, 229, 242, 246, 249, 254, 255, 257, 258, 262, 263, 267, 270, 276, 278, 280, 282, 287, 304, 305, 306, 307, 311, 313, 324, 342, 345, 346, 348, 362, 374, 380, 383, 395, 396, 397, 400, 402, 415, 425, 428, 432, 433, 447, 450, 506], "295": 2, "47": [2, 58], "05": [2, 41, 45, 53, 54, 107, 144], "296": [2, 331], "input": [2, 5, 7, 10, 16, 23, 24, 32, 57, 59, 73, 76, 88, 96, 99, 100, 101, 103, 104, 107, 109, 110, 112, 118, 120, 123, 124, 126, 127, 131, 133, 143, 144, 146, 148, 151, 158, 178, 191, 196, 201, 206, 212, 217, 218, 228, 230, 234, 244, 248, 251, 257, 263, 267, 274, 276, 278, 280, 282, 304, 307, 313, 327, 328, 329, 332, 354, 357, 358, 360, 361, 364, 368, 369, 374, 381, 382, 383, 386, 392, 396, 402, 403, 406, 422, 430, 432, 435, 477, 486, 505, 506, 510, 513, 520], "avg": [2, 14, 40, 48, 55, 62, 66, 68, 70, 71, 76, 99, 114, 115, 217, 240, 302, 364, 368, 369, 379, 384, 388, 392, 396, 416, 506, 510, 520], "120527": 2, "std": [2, 506], "dev": [2, 38, 58, 466, 506], "19": [2, 29, 41, 56, 68, 72, 84, 102, 104, 105, 107, 121, 136, 153, 391, 506, 520], "208": [2, 153], "17": [2, 58, 65, 102, 136, 144, 153, 392, 416, 417, 475], "03mb": 2, "1017": [2, 350], "64k": 2, "99m": 2, "4": [2, 5, 15, 18, 23, 35, 40, 41, 45, 53, 57, 60, 64, 65, 75, 78, 99, 100, 101, 102, 103, 104, 105, 106, 107, 109, 112, 113, 117, 123, 128, 130, 131, 135, 137, 144, 148, 153, 232, 233, 263, 321, 364, 392, 413, 417, 428, 432, 453, 462, 491, 497, 505, 506, 520], "02m": 2, "49": [2, 101, 113, 506], "18": [2, 42, 56, 58, 72, 107, 113, 136, 144, 153, 352, 359, 395, 407, 413, 473, 506], "85": [2, 109, 136, 153], "123": [2, 106, 107, 112, 144, 153, 478, 480, 483], "62": [2, 102, 153], "07mb": 2, "time": [2, 5, 8, 9, 14, 16, 18, 19, 20, 22, 23, 24, 25, 26, 27, 32, 33, 35, 36, 37, 39, 40, 41, 43, 44, 45, 47, 48, 51, 53, 56, 62, 64, 66, 67, 68, 69, 70, 71, 72, 75, 76, 77, 78, 83, 84, 98, 99, 100, 104, 112, 113, 114, 119, 133, 136, 140, 143, 146, 157, 158, 160, 162, 163, 170, 176, 186, 191, 192, 196, 197, 198, 208, 209, 217, 219, 220, 225, 229, 232, 233, 234, 241, 242, 244, 245, 246, 247, 249, 250, 251, 252, 253, 256, 257, 260, 262, 263, 264, 268, 270, 271, 273, 275, 276, 278, 279, 282, 283, 296, 297, 305, 307, 327, 331, 333, 334, 337, 339, 344, 345, 348, 350, 353, 356, 357, 358, 360, 361, 362, 364, 365, 366, 367, 368, 369, 371, 372, 373, 375, 377, 379, 380, 381, 382, 383, 385, 386, 387, 388, 389, 390, 391, 392, 395, 396, 398, 400, 402, 403, 406, 408, 409, 413, 414, 416, 419, 423, 424, 425, 426, 427, 428, 429, 430, 432, 433, 434, 435, 436, 438, 440, 442, 445, 449, 450, 451, 454, 464, 465, 467, 470, 471, 475, 477, 486, 491, 506, 510, 513, 520], "order": [2, 6, 10, 13, 14, 19, 23, 24, 29, 32, 33, 35, 36, 37, 39, 40, 41, 42, 43, 44, 45, 47, 48, 50, 51, 52, 54, 57, 58, 59, 60, 62, 64, 65, 66, 68, 70, 71, 73, 74, 75, 76, 78, 83, 84, 88, 93, 94, 96, 100, 101, 104, 105, 112, 113, 118, 123, 124, 126, 131, 137, 142, 144, 147, 148, 156, 162, 164, 175, 177, 183, 190, 191, 196, 217, 218, 223, 226, 227, 233, 237, 240, 242, 244, 246, 250, 255, 258, 261, 262, 264, 265, 266, 268, 271, 272, 278, 279, 283, 288, 297, 302, 307, 313, 315, 321, 325, 328, 331, 334, 335, 337, 339, 346, 347, 353, 358, 366, 372, 373, 375, 380, 381, 382, 383, 385, 386, 387, 389, 392, 393, 396, 400, 402, 408, 420, 429, 430, 432, 433, 436, 437, 439, 447, 450, 457, 459, 461, 462, 465, 467, 469, 470, 471, 475, 486, 489, 490, 491, 493, 494, 506, 507, 509, 511, 512, 517, 530, 533, 541], "work": [2, 5, 13, 15, 18, 25, 26, 32, 33, 35, 38, 41, 45, 48, 51, 52, 54, 55, 57, 58, 59, 60, 62, 63, 66, 68, 69, 71, 74, 75, 76, 84, 86, 87, 88, 104, 108, 124, 126, 136, 137, 138, 151, 152, 154, 157, 167, 172, 208, 248, 250, 258, 270, 271, 278, 279, 282, 284, 287, 293, 301, 303, 305, 325, 326, 334, 337, 339, 342, 346, 354, 362, 368, 384, 397, 399, 408, 419, 451, 467, 468, 469, 471, 475, 485, 519, 520, 532], "smaller": [2, 5, 10, 13, 14, 45, 64, 70, 84, 100, 104, 146, 148, 180, 355, 450], "chosen": [2, 56, 78, 104, 109, 146, 266, 510], "cost": [2, 10, 16, 18, 19, 25, 37, 41, 45, 48, 54, 62, 84, 110, 133, 145, 148, 245, 253, 271, 272, 273, 274, 276, 278, 280, 293, 329, 330, 332, 338, 346, 352, 356, 357, 358, 360, 368, 369, 382, 401, 425, 439, 472, 506], "base": [2, 3, 5, 8, 9, 10, 15, 19, 23, 27, 32, 33, 35, 37, 41, 45, 51, 53, 54, 55, 57, 58, 59, 62, 65, 68, 73, 80, 82, 84, 85, 87, 89, 90, 92, 93, 94, 95, 96, 99, 100, 107, 109, 112, 119, 121, 128, 133, 136, 138, 145, 147, 148, 149, 155, 191, 193, 213, 219, 228, 229, 232, 233, 244, 246, 248, 255, 266, 272, 273, 274, 276, 279, 283, 287, 296, 302, 324, 327, 331, 343, 344, 346, 356, 357, 358, 362, 363, 368, 369, 371, 372, 373, 374, 378, 379, 382, 386, 389, 392, 395, 397, 401, 408, 409, 410, 414, 417, 419, 428, 432, 433, 439, 444, 447, 456, 458, 460, 461, 466, 471, 476, 506, 510, 520, 525], "automat": [2, 5, 10, 14, 20, 23, 27, 32, 33, 41, 45, 51, 54, 57, 65, 66, 69, 70, 71, 76, 77, 83, 88, 106, 112, 126, 136, 139, 146, 157, 173, 183, 227, 239, 263, 272, 275, 278, 296, 302, 330, 334, 338, 346, 355, 356, 379, 381, 382, 387, 392, 393, 399, 407, 408, 422, 434, 453, 466, 467, 475, 476], "do": [2, 5, 8, 10, 13, 14, 15, 18, 23, 24, 25, 32, 33, 35, 40, 42, 44, 45, 47, 48, 51, 55, 57, 59, 62, 64, 65, 66, 68, 69, 71, 73, 74, 75, 76, 84, 93, 94, 96, 107, 109, 112, 124, 126, 129, 131, 136, 137, 138, 143, 147, 152, 157, 178, 183, 191, 210, 217, 218, 242, 244, 246, 250, 254, 257, 258, 264, 266, 270, 274, 278, 297, 301, 310, 328, 341, 346, 360, 364, 365, 368, 372, 373, 380, 382, 386, 388, 408, 413, 419, 430, 432, 457, 461, 462, 466, 467, 468, 469, 471, 475, 487, 491, 499, 508, 510, 518, 520, 522], "provid": [2, 3, 4, 6, 7, 11, 13, 14, 15, 24, 25, 27, 31, 32, 33, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 47, 48, 49, 50, 52, 53, 54, 55, 56, 57, 59, 60, 61, 62, 63, 64, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 81, 83, 84, 87, 88, 91, 93, 94, 96, 97, 99, 100, 101, 104, 106, 109, 112, 113, 119, 122, 124, 125, 126, 128, 132, 133, 135, 136, 140, 143, 144, 146, 148, 149, 151, 152, 154, 171, 176, 213, 217, 246, 247, 254, 260, 264, 270, 273, 279, 281, 293, 307, 328, 329, 332, 334, 342, 345, 346, 349, 359, 361, 366, 371, 374, 375, 377, 379, 383, 392, 423, 442, 458, 460, 461, 462, 464, 465, 468, 469, 470, 471, 475, 476, 482, 483, 491, 494, 516, 520], "therefor": [2, 5, 29, 39, 41, 45, 48, 54, 55, 57, 68, 71, 73, 74, 75, 76, 78, 112, 123, 132, 136, 143, 146, 148, 350, 464, 466, 505, 510, 520, 521], "recommend": [2, 5, 6, 13, 18, 25, 32, 33, 45, 47, 51, 54, 58, 66, 71, 76, 112, 136, 137, 138, 139, 247, 254, 271, 283, 296, 303, 334, 336, 401, 466, 475, 510], "keep": [2, 3, 6, 13, 15, 32, 35, 41, 43, 44, 45, 54, 68, 69, 84, 88, 112, 126, 135, 136, 151, 284, 388, 399, 475, 476, 489, 491, 510], "date": [2, 35, 37, 39, 40, 41, 45, 47, 50, 54, 55, 60, 62, 64, 65, 66, 68, 69, 71, 75, 76, 78, 84, 98, 99, 100, 104, 106, 110, 112, 114, 126, 133, 137, 140, 162, 163, 180, 194, 208, 209, 217, 247, 249, 262, 263, 271, 280, 282, 297, 304, 305, 323, 328, 331, 342, 354, 358, 367, 369, 381, 383, 386, 388, 393, 395, 396, 397, 398, 399, 404, 414, 420, 421, 423, 424, 429, 430, 432, 433, 450, 454, 456, 465, 489, 490, 506, 538, 542], "reli": [2, 35, 38, 41, 45, 54, 73, 88, 132, 148, 339, 346, 450, 464, 468], "cbo": [2, 84], "correctli": [2, 35, 45, 94, 155, 159, 203, 208, 209, 233, 246, 255, 258, 268, 271, 274, 282, 289, 293, 301, 302, 315, 331, 335, 342, 344, 350, 358, 368, 381, 386, 391, 432, 445, 447, 471], "choos": [2, 13, 41, 44, 47, 51, 54, 57, 78, 94, 146, 147, 151, 164, 227, 295, 302, 330, 397, 475, 476], "kei": [2, 3, 5, 10, 14, 18, 23, 24, 25, 32, 33, 35, 37, 40, 41, 42, 43, 44, 45, 49, 50, 51, 52, 54, 55, 58, 60, 62, 64, 65, 66, 68, 69, 71, 74, 76, 78, 83, 94, 99, 100, 101, 109, 117, 119, 124, 132, 138, 146, 155, 162, 177, 183, 184, 188, 191, 194, 210, 217, 226, 228, 229, 232, 233, 235, 245, 249, 267, 268, 269, 276, 278, 284, 287, 289, 291, 296, 297, 299, 304, 305, 307, 313, 315, 317, 326, 329, 331, 333, 337, 339, 342, 357, 358, 366, 368, 372, 377, 379, 383, 388, 389, 399, 402, 422, 432, 438, 445, 450, 462, 464, 466, 467, 468, 469, 470, 482, 505, 506, 520, 527], "column": [2, 3, 4, 14, 23, 32, 38, 39, 40, 42, 44, 52, 55, 56, 62, 64, 65, 66, 67, 68, 69, 70, 71, 74, 75, 76, 77, 78, 80, 83, 84, 86, 87, 88, 91, 94, 99, 104, 112, 113, 125, 126, 132, 133, 137, 142, 143, 144, 148, 149, 151, 155, 162, 166, 171, 173, 178, 179, 180, 182, 186, 188, 191, 194, 196, 198, 199, 203, 206, 207, 208, 209, 213, 217, 221, 223, 225, 226, 227, 228, 229, 234, 237, 239, 241, 242, 245, 246, 247, 248, 249, 250, 253, 254, 255, 256, 258, 259, 260, 261, 262, 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, 276, 278, 279, 280, 282, 288, 291, 295, 296, 297, 301, 302, 303, 304, 305, 308, 309, 313, 317, 320, 321, 323, 327, 328, 329, 330, 331, 333, 335, 336, 337, 338, 342, 343, 345, 346, 347, 350, 351, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, 368, 369, 370, 371, 372, 373, 374, 375, 377, 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, 396, 397, 399, 400, 401, 402, 403, 405, 406, 408, 409, 410, 412, 413, 414, 415, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, 432, 433, 434, 435, 436, 438, 439, 440, 441, 442, 444, 445, 446, 447, 448, 449, 450, 451, 452, 453, 454, 455, 456, 457, 471, 477, 480, 482, 484, 489, 490, 494, 495, 497, 507, 509, 510, 511, 512, 517, 538, 542, 544], "mai": [2, 5, 8, 9, 13, 14, 15, 19, 23, 25, 26, 32, 33, 35, 37, 39, 40, 42, 43, 44, 45, 47, 51, 52, 54, 55, 56, 57, 59, 60, 62, 63, 64, 65, 66, 68, 69, 70, 71, 73, 74, 75, 76, 78, 83, 84, 88, 94, 100, 104, 105, 109, 112, 116, 120, 121, 123, 124, 136, 138, 144, 146, 148, 151, 153, 154, 156, 157, 167, 173, 178, 182, 186, 192, 198, 212, 219, 220, 221, 242, 244, 248, 251, 252, 261, 267, 270, 276, 280, 282, 303, 304, 307, 324, 331, 347, 354, 357, 380, 381, 410, 450, 460, 462, 464, 465, 466, 467, 468, 469, 470, 475, 489, 491, 506, 520], "incur": [2, 37], "addit": [2, 5, 8, 9, 14, 19, 22, 23, 31, 35, 37, 38, 39, 40, 41, 42, 43, 44, 45, 51, 52, 54, 55, 56, 57, 58, 59, 60, 62, 63, 64, 65, 66, 68, 69, 71, 73, 74, 75, 76, 78, 83, 84, 88, 93, 99, 100, 112, 118, 135, 136, 144, 146, 147, 155, 176, 225, 243, 246, 250, 253, 256, 269, 271, 278, 282, 284, 293, 317, 321, 330, 366, 368, 373, 380, 388, 392, 404, 418, 461, 462, 464, 467, 469, 470, 471, 472, 473, 489, 506, 509, 510, 520, 538], "overhead": [2, 10, 19, 45, 84, 241, 242, 244, 246, 248, 249, 251, 253, 261, 262, 263, 271, 274, 275, 359, 374, 375, 377, 383, 389, 400, 410, 420, 451], "like": [2, 5, 13, 16, 22, 25, 27, 29, 31, 32, 35, 37, 38, 40, 41, 45, 47, 48, 54, 55, 57, 59, 60, 62, 65, 66, 68, 71, 72, 73, 74, 75, 76, 78, 83, 84, 87, 94, 95, 97, 99, 105, 106, 112, 119, 124, 132, 135, 136, 137, 139, 142, 143, 144, 152, 154, 157, 208, 213, 218, 232, 263, 264, 266, 287, 296, 301, 302, 303, 324, 351, 352, 360, 361, 367, 373, 375, 380, 386, 392, 400, 401, 402, 412, 421, 433, 436, 438, 443, 450, 454, 457, 462, 464, 465, 467, 470, 474, 489, 510, 520, 526, 527, 532, 536, 537, 539], "defin": [2, 4, 8, 13, 14, 23, 27, 33, 35, 37, 40, 41, 42, 43, 45, 47, 50, 52, 53, 54, 55, 56, 57, 58, 59, 60, 62, 63, 64, 65, 66, 68, 69, 71, 72, 73, 74, 75, 76, 77, 78, 82, 84, 85, 87, 88, 89, 90, 92, 93, 94, 95, 97, 103, 107, 108, 109, 131, 132, 137, 138, 144, 151, 187, 242, 262, 264, 266, 297, 328, 341, 368, 371, 373, 374, 380, 392, 394, 408, 433, 462, 469, 476, 483, 486, 491, 503, 504, 510, 512, 514, 520, 521, 523, 544], "task": [2, 6, 8, 9, 17, 25, 27, 37, 41, 42, 43, 45, 54, 62, 66, 68, 71, 74, 76, 96, 133, 136, 138, 154, 156, 160, 161, 170, 173, 175, 180, 182, 187, 193, 208, 209, 211, 212, 213, 216, 217, 220, 221, 228, 241, 246, 249, 256, 260, 263, 270, 276, 278, 280, 282, 287, 293, 296, 297, 302, 305, 307, 317, 321, 322, 329, 357, 371, 380, 382, 392, 395, 396, 399, 400, 401, 402, 403, 406, 408, 409, 413, 415, 419, 432, 434, 438, 449, 450, 455, 474, 483, 506], "larg": [2, 5, 7, 9, 13, 14, 15, 18, 19, 25, 40, 41, 42, 45, 48, 54, 55, 59, 62, 66, 68, 69, 71, 74, 75, 76, 86, 119, 132, 136, 138, 146, 148, 150, 152, 157, 175, 180, 183, 184, 185, 197, 204, 209, 217, 221, 227, 228, 229, 240, 246, 247, 250, 251, 252, 260, 269, 271, 276, 280, 286, 287, 288, 289, 291, 303, 305, 312, 317, 320, 334, 337, 356, 357, 358, 359, 361, 362, 367, 368, 369, 374, 379, 383, 384, 388, 390, 392, 401, 405, 409, 422, 423, 424, 427, 429, 432, 434, 438, 447, 449, 451, 453, 466, 467, 520], "enable_large_dynamic_filt": [2, 369], "driver": [2, 4, 23, 31, 32, 39, 40, 42, 47, 55, 64, 65, 66, 68, 69, 71, 74, 75, 76, 83, 87, 133, 135, 155, 156, 160, 162, 165, 173, 177, 191, 198, 200, 204, 206, 209, 242, 257, 268, 284, 287, 288, 292, 293, 302, 303, 307, 317, 321, 397, 435, 436, 438, 461, 467, 468, 469, 470, 471, 472, 473, 506], "similarli": [2, 23, 37, 84, 112, 143, 144, 208, 505, 510, 520], "small": [2, 5, 7, 9, 13, 14, 19, 22, 25, 41, 54, 64, 67, 72, 79, 80, 136, 137, 146, 148, 183, 248, 250, 256, 265, 293, 297, 302, 320, 321, 324, 348, 352, 369, 382, 391, 392, 409, 412, 413, 432, 434, 438, 439, 440, 447, 455, 467], "appli": [2, 5, 8, 10, 13, 14, 15, 17, 19, 23, 24, 39, 41, 45, 52, 64, 84, 94, 100, 104, 109, 112, 113, 117, 124, 126, 137, 144, 146, 148, 151, 162, 180, 197, 204, 209, 217, 258, 278, 302, 313, 334, 348, 349, 352, 353, 355, 357, 358, 359, 360, 368, 372, 377, 384, 386, 388, 389, 399, 406, 420, 452, 462, 464, 471, 478, 480, 510, 512, 520, 523, 542], "befor": [2, 5, 9, 10, 12, 15, 16, 19, 20, 22, 23, 25, 29, 32, 33, 35, 37, 38, 40, 41, 42, 45, 47, 48, 51, 54, 55, 57, 59, 60, 62, 66, 68, 69, 71, 72, 74, 75, 76, 94, 95, 99, 100, 131, 136, 137, 144, 155, 159, 166, 170, 183, 186, 188, 194, 198, 204, 208, 209, 212, 213, 217, 219, 220, 225, 242, 247, 248, 251, 255, 263, 267, 270, 271, 275, 282, 283, 287, 294, 296, 302, 316, 317, 322, 325, 327, 333, 334, 336, 339, 342, 350, 357, 359, 365, 366, 367, 368, 369, 371, 381, 383, 392, 413, 423, 424, 425, 450, 460, 465, 467, 471, 475, 476, 480, 510, 520, 533, 542], "switch": [2, 9, 10, 19, 110, 144, 266, 304, 389, 435, 466], "min": [2, 5, 7, 27, 40, 41, 45, 51, 54, 55, 57, 62, 64, 66, 68, 70, 71, 76, 78, 99, 104, 108, 114, 115, 170, 210, 221, 239, 256, 257, 259, 260, 262, 263, 265, 266, 267, 270, 307, 321, 325, 327, 364, 365, 368, 369, 373, 379, 381, 382, 384, 388, 403, 419, 430, 432, 434, 443, 455, 506], "ha": [2, 5, 6, 7, 9, 13, 14, 15, 19, 27, 29, 32, 33, 35, 37, 38, 41, 43, 44, 45, 48, 51, 52, 53, 54, 57, 58, 59, 60, 63, 65, 68, 69, 72, 76, 82, 83, 84, 85, 86, 88, 89, 90, 92, 94, 95, 96, 100, 105, 109, 112, 124, 132, 136, 137, 144, 146, 148, 151, 152, 155, 163, 169, 170, 173, 179, 182, 186, 192, 211, 225, 232, 235, 242, 246, 247, 250, 251, 252, 254, 255, 256, 260, 263, 264, 265, 267, 270, 273, 280, 282, 287, 293, 296, 297, 302, 303, 304, 307, 317, 318, 337, 342, 352, 357, 358, 365, 368, 371, 377, 379, 383, 387, 388, 389, 390, 391, 392, 397, 399, 402, 405, 408, 409, 410, 414, 420, 422, 424, 425, 428, 433, 434, 435, 438, 450, 456, 462, 464, 466, 470, 471, 474, 475, 487, 510, 512, 514, 520, 522, 525, 533], "much": [2, 10, 14, 25, 39, 48, 51, 72, 86, 88, 101, 144, 146, 155, 157, 167, 223, 302, 402, 466, 475], "lower": [2, 5, 10, 13, 14, 19, 25, 54, 57, 60, 66, 69, 72, 78, 88, 109, 114, 115, 118, 121, 124, 136, 144, 146, 155, 161, 248, 278, 380, 434, 476, 510], "granular": [2, 49, 59, 73, 280, 462, 520], "than": [2, 5, 9, 10, 11, 13, 14, 15, 17, 19, 22, 23, 24, 25, 35, 39, 41, 45, 48, 51, 54, 62, 64, 66, 68, 70, 71, 82, 83, 84, 85, 88, 89, 90, 92, 94, 95, 99, 100, 101, 102, 104, 109, 112, 113, 118, 120, 121, 124, 127, 131, 137, 144, 146, 151, 154, 162, 168, 169, 173, 176, 177, 180, 183, 186, 190, 194, 203, 204, 208, 209, 212, 213, 216, 217, 219, 233, 242, 247, 248, 249, 251, 254, 255, 256, 258, 262, 264, 266, 267, 270, 272, 274, 275, 277, 278, 279, 282, 297, 303, 304, 317, 327, 331, 332, 334, 335, 341, 342, 345, 348, 352, 355, 356, 359, 364, 365, 366, 368, 371, 372, 373, 375, 377, 379, 380, 381, 383, 386, 390, 391, 392, 393, 395, 396, 397, 398, 407, 416, 420, 421, 425, 432, 433, 434, 438, 440, 441, 444, 448, 450, 451, 453, 460, 462, 464, 465, 469, 470, 472, 476, 486, 491, 510, 511, 520, 538], "howev": [2, 5, 10, 14, 22, 25, 32, 33, 35, 38, 40, 41, 42, 48, 55, 58, 62, 64, 65, 66, 68, 69, 71, 74, 75, 76, 84, 97, 99, 112, 121, 143, 146, 147, 148, 157, 270, 357, 438, 460, 462, 464, 466, 469, 475, 510, 512, 520], "still": [2, 25, 35, 45, 47, 54, 57, 59, 64, 73, 132, 135, 167, 221, 242, 270, 285, 304, 328, 356, 361, 383, 389, 391, 392, 393, 397, 407, 420, 462], "benefici": [2, 45, 54, 148], "some": [2, 5, 8, 14, 15, 19, 21, 23, 31, 33, 35, 37, 38, 39, 40, 41, 42, 43, 44, 45, 54, 55, 57, 59, 60, 62, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 78, 79, 80, 88, 94, 97, 99, 105, 109, 112, 114, 124, 135, 136, 137, 139, 142, 143, 144, 148, 151, 155, 156, 167, 175, 178, 181, 182, 183, 191, 192, 193, 202, 208, 220, 228, 236, 241, 247, 249, 267, 274, 277, 279, 287, 290, 303, 307, 320, 321, 357, 364, 373, 387, 390, 392, 432, 462, 466, 468, 475, 478, 480, 483, 487, 493, 499, 506, 507, 508, 510, 512, 517, 518, 522, 533, 542], "especi": [2, 19, 25, 119, 136, 256, 344, 506], "best": [2, 5, 15, 35, 45, 47, 112, 136, 137, 144, 147, 227, 272, 283, 389, 471, 475], "correl": [2, 14, 33, 99, 186, 196, 217, 220, 229, 239, 240, 242, 243, 247, 248, 263, 267, 268, 269, 270, 272, 273, 274, 339, 346, 355, 359, 368, 373, 375, 380, 387, 391, 401, 405, 410, 419, 434, 457, 506, 520], "should": [2, 3, 5, 7, 13, 15, 16, 19, 24, 25, 26, 27, 32, 33, 35, 37, 40, 42, 43, 44, 45, 48, 49, 51, 55, 57, 58, 59, 62, 63, 65, 66, 68, 69, 71, 73, 74, 75, 76, 83, 84, 88, 91, 93, 94, 96, 99, 103, 112, 126, 136, 137, 151, 162, 174, 178, 211, 213, 254, 270, 282, 284, 293, 296, 301, 302, 303, 304, 317, 318, 328, 344, 347, 357, 360, 368, 376, 379, 384, 389, 391, 398, 450, 460, 462, 466, 468, 469, 470, 473, 475, 499, 507, 508, 512, 517, 518, 522], "monoton": 2, "increas": [2, 5, 9, 11, 13, 14, 15, 18, 19, 22, 25, 39, 40, 42, 44, 45, 48, 55, 62, 66, 68, 69, 71, 74, 75, 76, 144, 146, 148, 157, 161, 177, 180, 193, 208, 209, 241, 246, 250, 253, 260, 263, 267, 287, 302, 345, 371, 372, 392, 408, 427, 510], "an": [2, 3, 4, 5, 8, 9, 11, 12, 13, 14, 15, 16, 20, 23, 24, 25, 32, 33, 35, 37, 39, 40, 41, 42, 43, 44, 45, 47, 48, 50, 51, 52, 54, 55, 56, 57, 58, 59, 60, 62, 63, 64, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 82, 83, 84, 85, 86, 87, 88, 89, 90, 91, 92, 93, 94, 96, 99, 100, 103, 104, 105, 106, 107, 108, 109, 110, 112, 113, 116, 117, 118, 119, 120, 121, 123, 124, 127, 131, 132, 136, 137, 138, 140, 142, 143, 144, 146, 147, 148, 151, 152, 155, 157, 159, 167, 169, 171, 173, 176, 178, 179, 180, 182, 183, 186, 191, 192, 194, 195, 201, 203, 206, 208, 209, 213, 217, 225, 226, 227, 229, 232, 235, 240, 241, 243, 245, 246, 247, 248, 251, 254, 257, 258, 259, 260, 262, 263, 264, 267, 268, 271, 272, 273, 274, 275, 277, 278, 279, 281, 282, 284, 287, 288, 289, 293, 296, 297, 298, 301, 302, 303, 304, 305, 307, 310, 311, 313, 315, 317, 324, 328, 329, 330, 331, 332, 334, 336, 337, 342, 344, 346, 350, 352, 355, 356, 357, 358, 359, 360, 361, 364, 365, 366, 368, 375, 379, 380, 381, 383, 384, 385, 386, 387, 388, 390, 392, 397, 403, 408, 409, 414, 415, 416, 417, 418, 419, 420, 421, 422, 424, 427, 428, 430, 432, 433, 434, 437, 438, 442, 445, 446, 447, 448, 449, 450, 453, 456, 457, 460, 461, 462, 463, 464, 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 478, 479, 480, 481, 486, 489, 490, 491, 495, 498, 500, 501, 502, 505, 510, 511, 512, 513, 520, 523, 525, 544], "address": [2, 25, 32, 33, 41, 43, 44, 47, 54, 57, 58, 60, 70, 94, 98, 112, 133, 137, 140, 151, 247, 256, 344, 358, 392, 417, 454, 462, 464, 466, 468, 469, 511], "compos": [2, 45, 120, 123, 127, 144, 462, 472, 476, 520], "other": [2, 4, 6, 7, 9, 13, 14, 15, 18, 19, 23, 25, 30, 31, 32, 33, 35, 37, 38, 39, 40, 41, 42, 43, 44, 45, 47, 48, 51, 52, 54, 55, 57, 59, 60, 62, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 80, 82, 83, 84, 85, 86, 87, 88, 89, 90, 92, 93, 94, 95, 97, 98, 100, 104, 105, 109, 113, 119, 121, 123, 129, 132, 135, 136, 137, 138, 139, 140, 142, 143, 144, 146, 147, 148, 151, 152, 154, 169, 170, 183, 210, 212, 229, 240, 248, 258, 264, 265, 267, 270, 271, 282, 287, 293, 303, 323, 325, 333, 338, 346, 350, 368, 372, 381, 383, 392, 397, 402, 413, 426, 433, 460, 462, 465, 466, 467, 468, 469, 470, 471, 474, 475, 477, 486, 488, 507, 510, 511, 512, 517, 520, 523, 527, 538], "countri": [2, 43, 45, 54, 68, 282, 284], "state": [2, 3, 6, 27, 53, 54, 63, 83, 88, 99, 100, 136, 137, 151, 158, 160, 182, 232, 281, 328, 394, 395, 399, 413, 420, 470, 482, 511, 520], "zip": [2, 93, 100, 114, 115, 209, 263, 480], "address_id": 2, "ny": [2, 482], "10001": [2, 102], "1234": [2, 45, 57, 107], "usag": [2, 3, 4, 5, 9, 10, 13, 14, 15, 16, 18, 19, 23, 24, 25, 32, 40, 48, 55, 64, 67, 69, 96, 104, 112, 132, 136, 144, 146, 155, 157, 158, 182, 183, 189, 191, 209, 219, 221, 222, 223, 232, 237, 243, 244, 246, 248, 249, 250, 251, 254, 256, 259, 261, 263, 264, 266, 267, 269, 270, 271, 273, 280, 293, 296, 330, 333, 342, 345, 356, 357, 360, 362, 365, 367, 387, 390, 392, 395, 410, 417, 426, 427, 428, 433, 434, 435, 438, 443, 452, 454, 455, 468, 469, 474, 475, 520], "succe": [2, 54, 112, 287, 384, 469, 499, 507, 508, 510, 517, 518, 522, 524], "even": [2, 13, 14, 45, 48, 54, 57, 58, 59, 66, 68, 71, 76, 84, 96, 104, 112, 126, 136, 151, 223, 232, 249, 251, 254, 259, 293, 342, 403, 466, 475, 510, 520, 523, 544], "doubl": [2, 14, 18, 29, 35, 36, 37, 39, 40, 41, 42, 43, 45, 47, 50, 54, 55, 56, 57, 58, 59, 60, 62, 65, 66, 68, 69, 70, 71, 72, 73, 75, 76, 78, 83, 84, 88, 99, 100, 101, 103, 104, 106, 107, 109, 118, 119, 120, 123, 127, 131, 141, 142, 155, 173, 189, 191, 217, 231, 233, 245, 254, 255, 257, 263, 267, 295, 317, 323, 333, 337, 356, 361, 362, 364, 365, 366, 389, 395, 396, 405, 419, 420, 427, 432, 433, 454, 455, 489, 524, 530], "real": [2, 35, 39, 40, 41, 42, 43, 45, 54, 55, 57, 59, 60, 62, 66, 68, 69, 70, 71, 75, 76, 84, 86, 101, 112, 118, 120, 213, 217, 223, 228, 229, 241, 243, 245, 257, 265, 271, 333, 337, 342, 345, 347, 356, 358, 361, 364, 365, 366, 368, 389, 392, 396, 413, 419, 427, 433], "unorder": 2, "contain": [2, 5, 12, 13, 15, 19, 23, 27, 32, 33, 35, 37, 39, 40, 41, 42, 43, 44, 45, 47, 48, 49, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 65, 66, 68, 69, 71, 73, 74, 75, 76, 77, 78, 82, 83, 84, 86, 88, 89, 90, 91, 92, 93, 94, 99, 100, 103, 107, 109, 111, 112, 113, 114, 115, 117, 121, 123, 124, 126, 129, 132, 133, 134, 136, 137, 139, 151, 155, 156, 161, 162, 180, 183, 191, 194, 207, 210, 213, 217, 219, 220, 221, 223, 225, 226, 228, 229, 236, 237, 239, 240, 242, 244, 245, 246, 248, 249, 250, 251, 254, 255, 258, 261, 262, 264, 265, 266, 267, 268, 269, 270, 271, 274, 275, 276, 278, 279, 282, 293, 296, 297, 302, 305, 307, 313, 317, 320, 321, 323, 326, 327, 329, 330, 331, 332, 333, 334, 337, 338, 339, 342, 343, 344, 346, 347, 350, 352, 353, 355, 356, 357, 358, 359, 360, 361, 362, 364, 365, 366, 367, 368, 369, 370, 372, 373, 374, 375, 377, 380, 381, 382, 383, 384, 385, 386, 388, 389, 390, 391, 392, 393, 394, 396, 397, 398, 399, 400, 402, 403, 405, 406, 408, 409, 410, 414, 417, 418, 419, 420, 421, 423, 424, 426, 427, 428, 430, 432, 433, 435, 436, 438, 439, 440, 441, 443, 444, 446, 448, 449, 450, 453, 454, 457, 460, 461, 462, 463, 464, 465, 467, 468, 469, 470, 472, 474, 488, 490, 491, 500, 510, 512, 520], "scenario": [2, 41, 43, 45, 54, 99, 278, 303, 434, 455, 469, 470], "implicit": [2, 62, 112, 144, 156, 159, 165, 172, 231, 233, 237, 242, 248, 251, 255, 263, 271, 293, 307, 324, 417, 421, 438, 510, 512, 542], "integ": [2, 9, 10, 11, 12, 13, 14, 15, 16, 18, 19, 35, 37, 39, 40, 41, 43, 45, 47, 50, 53, 54, 57, 59, 60, 62, 66, 68, 69, 70, 71, 73, 75, 76, 78, 83, 84, 94, 96, 99, 100, 102, 106, 112, 118, 121, 124, 140, 148, 178, 186, 206, 212, 213, 217, 240, 245, 251, 261, 262, 264, 271, 296, 316, 327, 348, 356, 362, 364, 365, 368, 380, 433, 456, 510, 520, 538], "plugin": [3, 4, 7, 35, 41, 51, 57, 64, 65, 81, 82, 84, 85, 89, 90, 92, 95, 97, 119, 132, 135, 136, 139, 144, 178, 180, 213, 217, 220, 245, 254, 260, 270, 275, 285, 287, 293, 296, 315, 327, 330, 332, 342, 345, 350, 378, 382, 393, 413, 426, 462, 469], "encod": [3, 35, 37, 39, 44, 45, 59, 60, 69, 73, 78, 84, 93, 94, 112, 124, 128, 136, 156, 184, 185, 187, 215, 217, 219, 228, 239, 244, 245, 261, 267, 268, 269, 275, 276, 282, 296, 302, 305, 309, 327, 331, 332, 344, 351, 358, 365, 367, 368, 369, 371, 372, 375, 383, 384, 392, 395, 403, 419, 422, 432, 451, 464, 465, 467, 470, 472, 475], "format": [3, 5, 11, 33, 35, 37, 40, 41, 42, 43, 44, 45, 47, 48, 50, 53, 54, 55, 57, 58, 59, 60, 62, 64, 65, 66, 68, 69, 71, 73, 74, 75, 76, 78, 83, 86, 94, 101, 103, 107, 112, 114, 115, 119, 124, 128, 132, 136, 144, 147, 177, 193, 196, 197, 204, 208, 209, 219, 220, 231, 233, 248, 258, 259, 274, 276, 278, 282, 283, 293, 296, 303, 305, 307, 317, 332, 337, 341, 342, 343, 346, 348, 357, 358, 361, 364, 367, 368, 369, 379, 381, 384, 387, 392, 396, 404, 405, 407, 408, 410, 415, 419, 421, 422, 433, 435, 436, 438, 445, 447, 449, 451, 455, 461, 462, 464, 465, 467, 475, 476, 486, 489, 490, 530], "extern": [3, 4, 5, 7, 11, 33, 40, 41, 43, 45, 48, 51, 54, 62, 64, 66, 68, 71, 74, 75, 76, 78, 79, 80, 126, 136, 137, 138, 221, 223, 225, 236, 245, 246, 261, 303, 335, 342, 356, 357, 359, 361, 397, 418, 430, 438, 441, 452, 466, 469, 474, 475], "servic": [3, 5, 11, 32, 33, 37, 40, 41, 42, 44, 45, 49, 50, 53, 54, 55, 59, 62, 64, 65, 66, 68, 70, 71, 73, 74, 75, 76, 78, 84, 86, 87, 93, 132, 136, 137, 138, 196, 263, 287, 329, 335, 342, 348, 355, 356, 359, 368, 371, 378, 396, 426, 466, 467], "post": [3, 83, 178, 470, 520], "specifi": [3, 5, 7, 8, 11, 14, 19, 23, 24, 27, 32, 33, 35, 36, 37, 38, 40, 41, 42, 43, 44, 45, 49, 50, 51, 52, 54, 55, 56, 57, 59, 60, 62, 64, 65, 66, 67, 68, 69, 71, 73, 74, 75, 76, 77, 78, 83, 86, 87, 88, 94, 95, 96, 99, 101, 103, 104, 105, 106, 107, 109, 112, 118, 121, 124, 126, 128, 129, 131, 132, 135, 136, 137, 144, 146, 148, 154, 168, 170, 173, 177, 180, 186, 197, 204, 209, 212, 217, 227, 229, 235, 242, 251, 254, 256, 258, 259, 263, 267, 269, 270, 278, 279, 284, 293, 303, 310, 315, 333, 335, 336, 341, 346, 357, 359, 367, 368, 371, 375, 379, 383, 388, 389, 396, 398, 401, 404, 405, 408, 410, 413, 417, 418, 420, 423, 430, 432, 434, 436, 438, 444, 449, 452, 456, 460, 461, 462, 464, 465, 467, 468, 469, 471, 472, 475, 476, 478, 480, 486, 487, 489, 493, 494, 499, 505, 507, 508, 509, 510, 512, 517, 518, 520, 521, 522, 523, 524, 525, 526, 527, 528, 529, 530, 531, 532, 533, 536, 537, 539, 543], "uri": [3, 5, 37, 41, 45, 49, 50, 51, 53, 54, 64, 65, 70, 72, 86, 129, 136, 137, 138, 254, 281, 334, 373, 392, 466], "simpl": [3, 14, 39, 40, 41, 42, 45, 52, 54, 55, 57, 59, 60, 66, 68, 69, 71, 73, 74, 75, 76, 84, 86, 88, 94, 104, 105, 109, 138, 144, 151, 219, 266, 293, 334, 335, 337, 343, 357, 358, 401, 438, 462, 468, 469, 471, 473, 476, 486, 491, 520], "first": [3, 4, 23, 32, 33, 35, 41, 44, 45, 48, 54, 56, 57, 60, 64, 65, 66, 71, 73, 78, 84, 86, 88, 94, 100, 105, 107, 108, 109, 112, 113, 114, 115, 121, 124, 126, 129, 131, 136, 137, 146, 148, 151, 203, 209, 213, 253, 257, 268, 285, 287, 296, 301, 302, 337, 342, 356, 360, 364, 367, 392, 416, 459, 461, 462, 468, 469, 470, 476, 510, 511, 512], "step": [3, 4, 5, 6, 32, 33, 40, 44, 55, 57, 62, 66, 68, 69, 71, 74, 75, 76, 94, 100, 112, 126, 137, 146, 263, 307, 389, 461, 464, 465, 470, 471, 510, 512, 520], "better": [3, 19, 37, 40, 45, 54, 62, 66, 68, 70, 71, 73, 75, 76, 88, 101, 109, 136, 137, 213, 260, 271, 317, 372, 375, 400, 401], "understand": [3, 35, 51, 83, 86, 93, 137, 151, 152, 155, 475, 506, 520], "datalak": 3, "These": [3, 5, 6, 8, 16, 18, 23, 24, 27, 33, 35, 41, 43, 45, 52, 53, 54, 57, 58, 59, 64, 69, 83, 94, 95, 104, 107, 109, 124, 128, 132, 136, 137, 142, 146, 147, 148, 151, 152, 171, 173, 186, 217, 219, 220, 251, 256, 270, 275, 282, 288, 303, 327, 330, 334, 336, 342, 345, 346, 349, 352, 353, 355, 356, 357, 358, 360, 366, 436, 438, 462, 467, 483, 520, 523], "metric": [3, 4, 7, 19, 35, 72, 77, 156, 196, 241, 256, 288, 338, 360, 364, 373, 377, 389, 392, 396, 405, 425, 506], "what": [3, 4, 5, 14, 35, 41, 45, 88, 91, 93, 94, 132, 144, 151, 415, 510, 538], "being": [3, 4, 9, 13, 15, 19, 25, 27, 37, 41, 44, 45, 54, 57, 59, 73, 84, 101, 112, 113, 124, 136, 146, 148, 149, 151, 152, 166, 169, 170, 194, 196, 197, 213, 228, 233, 242, 247, 255, 257, 268, 289, 293, 324, 328, 353, 357, 375, 381, 395, 414, 433, 434, 446, 453, 456, 462, 464, 468, 474, 480, 509, 510, 520], "access": [3, 4, 5, 6, 7, 20, 27, 32, 33, 34, 35, 36, 37, 39, 40, 41, 42, 43, 44, 45, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 60, 62, 63, 65, 66, 67, 68, 69, 70, 72, 73, 74, 75, 76, 77, 78, 79, 80, 81, 82, 84, 89, 90, 92, 93, 94, 100, 112, 132, 133, 134, 136, 137, 138, 143, 144, 148, 151, 152, 155, 178, 183, 191, 217, 218, 223, 232, 239, 244, 246, 249, 250, 251, 259, 261, 264, 266, 267, 268, 270, 272, 274, 279, 281, 282, 284, 287, 307, 311, 317, 323, 329, 332, 333, 335, 337, 341, 344, 346, 347, 348, 351, 352, 354, 355, 356, 357, 358, 359, 360, 362, 363, 364, 366, 367, 368, 371, 372, 373, 374, 375, 379, 383, 384, 386, 388, 389, 392, 394, 395, 397, 399, 400, 402, 408, 410, 414, 420, 428, 430, 432, 433, 434, 441, 447, 454, 456, 459, 461, 463, 466, 467, 469, 470, 474, 475, 476, 486, 491, 494, 505, 510, 514, 521, 524], "resolut": [3, 4, 60, 89, 96, 206, 226, 254, 422], "metadata": [3, 4, 23, 24, 37, 38, 39, 43, 52, 57, 58, 59, 60, 62, 63, 65, 70, 72, 73, 79, 80, 84, 137, 155, 166, 167, 173, 177, 196, 208, 227, 229, 239, 242, 244, 245, 246, 262, 281, 282, 288, 293, 296, 303, 306, 313, 324, 332, 338, 346, 347, 352, 355, 358, 366, 367, 373, 375, 379, 380, 382, 383, 385, 386, 388, 389, 392, 393, 394, 396, 399, 401, 403, 404, 405, 408, 418, 419, 422, 424, 426, 427, 429, 430, 432, 434, 443, 446, 447, 450, 453, 460, 470, 482, 486, 488, 489, 490], "separ": [3, 4, 5, 11, 12, 13, 14, 18, 21, 25, 32, 39, 40, 41, 42, 43, 44, 45, 48, 51, 54, 56, 57, 59, 60, 62, 64, 66, 68, 69, 70, 71, 73, 74, 75, 76, 78, 83, 84, 85, 86, 93, 95, 99, 124, 129, 138, 139, 151, 156, 192, 253, 254, 296, 305, 307, 320, 345, 346, 347, 357, 359, 368, 390, 449, 459, 460, 463, 464, 465, 467, 469, 472, 473, 475, 483, 486, 505, 510, 520, 521, 523], "impact": [3, 4, 10, 13, 22, 40, 42, 44, 48, 52, 55, 62, 66, 68, 69, 70, 71, 74, 75, 76, 77, 94, 112, 136, 146, 148, 296, 301, 466, 520], "downtim": [3, 4], "non": [3, 4, 5, 8, 16, 19, 23, 32, 33, 35, 39, 41, 42, 45, 50, 51, 52, 54, 56, 57, 70, 78, 83, 84, 88, 94, 99, 100, 101, 104, 105, 109, 112, 121, 124, 138, 155, 162, 166, 173, 175, 180, 183, 186, 188, 194, 196, 201, 205, 208, 209, 210, 212, 217, 218, 219, 220, 221, 223, 228, 230, 233, 234, 236, 242, 245, 247, 248, 249, 250, 257, 258, 259, 261, 264, 270, 271, 275, 276, 279, 280, 281, 285, 297, 301, 302, 303, 317, 320, 332, 335, 342, 343, 344, 346, 352, 356, 357, 358, 360, 362, 364, 366, 367, 368, 369, 374, 375, 383, 384, 385, 386, 387, 389, 391, 402, 403, 405, 406, 408, 413, 416, 418, 420, 428, 430, 432, 433, 436, 438, 446, 448, 449, 450, 455, 475, 510, 512, 520, 534], "client": [3, 4, 5, 7, 8, 10, 23, 24, 27, 32, 33, 37, 39, 40, 42, 45, 47, 50, 51, 52, 55, 57, 60, 64, 66, 68, 69, 70, 71, 74, 75, 76, 81, 82, 133, 134, 135, 136, 137, 151, 157, 178, 183, 208, 209, 210, 227, 229, 245, 251, 254, 255, 257, 259, 260, 262, 263, 264, 267, 268, 271, 278, 279, 287, 293, 303, 327, 328, 329, 331, 334, 339, 341, 344, 347, 350, 357, 359, 360, 363, 365, 366, 368, 373, 374, 375, 377, 378, 379, 380, 383, 388, 391, 392, 404, 405, 410, 418, 422, 430, 432, 433, 457, 464, 465, 466, 468, 469, 470, 472, 473, 475, 516, 521, 524], "face": [3, 4, 26, 144], "chang": [3, 4, 5, 10, 23, 24, 32, 35, 40, 41, 42, 44, 45, 52, 54, 55, 57, 59, 62, 64, 66, 68, 69, 71, 73, 74, 75, 76, 84, 88, 94, 112, 124, 132, 136, 146, 155, 156, 157, 159, 167, 172, 173, 188, 189, 193, 194, 196, 198, 206, 209, 210, 212, 213, 217, 220, 226, 228, 233, 239, 242, 246, 248, 251, 253, 254, 255, 260, 261, 262, 263, 264, 266, 270, 271, 272, 273, 275, 277, 278, 280, 282, 283, 285, 287, 293, 294, 296, 302, 303, 305, 314, 316, 317, 320, 321, 325, 327, 334, 336, 341, 342, 343, 345, 349, 350, 352, 353, 355, 357, 358, 360, 361, 368, 371, 373, 374, 375, 377, 378, 379, 380, 382, 384, 386, 388, 389, 392, 395, 397, 402, 407, 409, 413, 415, 417, 418, 419, 424, 425, 426, 427, 428, 433, 434, 435, 438, 440, 448, 449, 450, 454, 455, 456, 457, 462, 466, 468, 469, 470, 471, 478, 479, 480, 481, 484, 510, 514, 520, 523, 524, 542], "accept": [3, 23, 27, 32, 71, 78, 84, 88, 91, 103, 132, 136, 169, 178, 257, 270, 315, 345, 346, 375, 377, 407, 440, 450, 467, 475, 510, 523], "bodi": [3, 32, 83, 113, 182, 229], "connect": [3, 4, 5, 7, 23, 31, 32, 35, 37, 39, 41, 42, 43, 44, 45, 47, 48, 51, 52, 55, 57, 58, 60, 62, 64, 68, 69, 70, 73, 78, 84, 86, 87, 126, 132, 135, 136, 137, 140, 143, 148, 151, 155, 157, 159, 162, 167, 173, 206, 230, 232, 233, 235, 240, 241, 242, 245, 254, 259, 275, 277, 279, 302, 303, 305, 327, 332, 345, 346, 350, 353, 355, 357, 360, 361, 365, 366, 367, 368, 373, 374, 379, 380, 381, 383, 384, 392, 397, 399, 401, 406, 408, 421, 426, 427, 429, 434, 436, 457, 461, 464, 466, 468, 469, 472, 473, 474, 475, 521, 523, 524], "ingest": [3, 45, 120], "file": [3, 4, 5, 6, 7, 11, 18, 21, 22, 24, 25, 27, 33, 34, 35, 36, 37, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 53, 55, 56, 59, 60, 62, 63, 64, 65, 66, 68, 69, 70, 71, 72, 74, 75, 76, 78, 79, 80, 82, 84, 85, 89, 90, 92, 93, 94, 95, 129, 132, 133, 135, 136, 137, 138, 139, 148, 151, 155, 156, 157, 177, 178, 183, 191, 193, 196, 207, 208, 209, 210, 217, 219, 223, 228, 229, 231, 232, 233, 236, 242, 243, 244, 245, 246, 248, 249, 251, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 265, 267, 269, 272, 273, 274, 275, 276, 277, 278, 279, 280, 281, 282, 285, 287, 296, 297, 301, 302, 307, 320, 321, 324, 325, 327, 328, 329, 331, 332, 334, 335, 336, 339, 341, 343, 344, 345, 346, 348, 349, 350, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, 369, 371, 372, 373, 375, 379, 381, 383, 384, 388, 389, 390, 391, 392, 394, 395, 396, 397, 398, 399, 401, 402, 403, 404, 407, 408, 409, 410, 412, 413, 415, 416, 417, 418, 419, 420, 422, 423, 424, 425, 426, 427, 428, 429, 430, 432, 433, 434, 435, 436, 437, 438, 440, 441, 442, 443, 444, 445, 446, 447, 449, 450, 451, 452, 453, 454, 455, 456, 458, 459, 460, 461, 466, 467, 470, 471, 473, 474, 480], "send": [3, 6, 31, 37, 57, 58, 132, 136, 138, 139, 254, 268, 276, 277, 377, 387, 390, 464, 467, 468, 469, 472, 475], "To": [3, 4, 5, 13, 23, 24, 25, 29, 32, 33, 35, 37, 39, 40, 41, 42, 43, 44, 45, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 78, 79, 80, 84, 88, 91, 94, 96, 104, 105, 112, 119, 126, 132, 135, 136, 137, 151, 155, 170, 177, 183, 191, 192, 226, 264, 272, 287, 293, 297, 301, 302, 307, 310, 317, 327, 357, 381, 389, 460, 462, 466, 468, 469, 470, 471, 472, 473, 474, 475, 482, 488, 489, 490, 520], "creat": [3, 4, 5, 11, 13, 14, 15, 19, 23, 24, 32, 33, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 59, 61, 62, 63, 64, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 81, 82, 83, 84, 85, 86, 87, 89, 90, 92, 93, 94, 95, 99, 103, 109, 110, 112, 117, 123, 132, 133, 135, 136, 139, 142, 143, 144, 146, 148, 151, 155, 165, 166, 177, 178, 186, 191, 193, 196, 197, 198, 208, 209, 212, 217, 218, 219, 221, 223, 225, 227, 229, 236, 238, 242, 243, 246, 247, 256, 257, 258, 261, 262, 264, 267, 269, 270, 274, 275, 278, 281, 283, 287, 288, 291, 293, 296, 297, 300, 303, 305, 313, 320, 327, 328, 329, 333, 334, 335, 339, 342, 344, 346, 351, 352, 353, 355, 356, 357, 358, 359, 360, 361, 366, 367, 368, 369, 371, 374, 379, 380, 381, 382, 383, 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, 397, 398, 401, 402, 403, 405, 410, 412, 413, 416, 418, 419, 420, 421, 422, 423, 425, 428, 430, 432, 433, 434, 435, 436, 438, 440, 441, 443, 448, 449, 450, 452, 453, 454, 456, 460, 462, 463, 464, 466, 467, 468, 469, 471, 473, 475, 477, 478, 479, 480, 481, 497, 498, 499, 500, 501, 502, 505, 508, 510, 514, 518, 522, 523, 544], "name": [3, 4, 5, 7, 11, 13, 19, 23, 24, 29, 30, 32, 33, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 47, 48, 49, 50, 51, 53, 54, 55, 56, 58, 59, 60, 61, 62, 63, 64, 66, 67, 68, 69, 70, 71, 72, 74, 75, 76, 77, 78, 79, 80, 82, 83, 84, 85, 86, 87, 88, 89, 90, 92, 93, 95, 96, 99, 104, 106, 107, 112, 121, 122, 124, 126, 129, 131, 132, 135, 136, 137, 138, 142, 144, 148, 151, 154, 155, 169, 173, 180, 182, 183, 191, 194, 196, 206, 207, 217, 221, 223, 224, 225, 232, 234, 242, 246, 250, 251, 254, 261, 262, 263, 264, 266, 267, 271, 272, 273, 274, 275, 281, 282, 287, 288, 293, 296, 302, 305, 313, 314, 329, 333, 334, 336, 337, 339, 340, 341, 345, 349, 350, 351, 352, 354, 356, 357, 359, 361, 362, 363, 364, 365, 366, 368, 371, 372, 373, 375, 377, 379, 380, 381, 382, 383, 384, 386, 388, 389, 390, 392, 393, 395, 396, 397, 398, 399, 401, 402, 404, 405, 407, 409, 413, 415, 416, 419, 420, 421, 422, 426, 427, 428, 432, 433, 435, 436, 438, 439, 440, 441, 442, 445, 446, 447, 449, 453, 455, 456, 460, 461, 462, 463, 464, 465, 466, 467, 468, 470, 472, 473, 475, 476, 478, 479, 480, 481, 483, 484, 486, 489, 490, 492, 496, 497, 503, 504, 505, 509, 510, 512, 513, 515, 521, 523, 527, 532, 533, 538, 542, 544], "content": [3, 4, 6, 12, 18, 23, 24, 25, 37, 38, 39, 41, 43, 44, 45, 48, 50, 54, 56, 57, 59, 60, 61, 62, 63, 64, 65, 66, 69, 70, 73, 74, 75, 78, 79, 80, 93, 136, 137, 155, 192, 244, 249, 281, 287, 326, 393, 432, 460, 462, 464, 468, 469, 471, 486], "log": [3, 6, 8, 18, 20, 25, 32, 40, 41, 42, 45, 52, 54, 55, 58, 61, 66, 68, 69, 71, 74, 75, 76, 83, 84, 85, 114, 115, 118, 133, 139, 194, 217, 244, 253, 254, 263, 265, 273, 280, 281, 287, 294, 296, 298, 323, 328, 341, 342, 344, 345, 368, 378, 384, 386, 395, 396, 399, 403, 419, 420, 452, 453, 454, 456, 466, 467, 468, 469, 470, 472, 475], "your": [3, 4, 5, 6, 10, 23, 25, 26, 32, 33, 35, 36, 37, 39, 40, 41, 42, 43, 45, 47, 48, 49, 50, 51, 52, 54, 55, 56, 57, 58, 62, 64, 65, 66, 68, 69, 71, 74, 75, 76, 78, 83, 84, 96, 134, 135, 136, 138, 146, 155, 159, 167, 170, 173, 178, 183, 188, 191, 194, 211, 212, 213, 217, 220, 251, 255, 260, 270, 275, 282, 283, 284, 287, 294, 296, 297, 301, 302, 307, 316, 317, 320, 325, 327, 334, 336, 342, 350, 460, 462, 464, 465, 466, 467, 468, 470, 471, 472, 473, 475, 476, 520], "And": [3, 4, 57, 84, 136, 512], "add": [3, 4, 23, 24, 32, 33, 35, 37, 39, 40, 41, 42, 43, 44, 45, 47, 52, 54, 55, 57, 59, 60, 64, 65, 66, 68, 71, 73, 74, 75, 76, 78, 83, 84, 88, 93, 94, 99, 107, 132, 137, 138, 144, 154, 155, 156, 157, 158, 159, 161, 162, 163, 165, 167, 169, 170, 171, 173, 177, 178, 180, 183, 184, 186, 187, 188, 189, 190, 191, 193, 194, 196, 197, 198, 200, 205, 206, 207, 208, 209, 210, 211, 212, 213, 217, 218, 219, 220, 221, 223, 224, 225, 226, 227, 228, 229, 230, 231, 232, 233, 234, 235, 236, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, 269, 270, 271, 272, 273, 274, 276, 277, 278, 279, 280, 281, 285, 287, 291, 293, 296, 297, 301, 302, 303, 304, 305, 306, 307, 309, 310, 313, 317, 320, 321, 323, 327, 328, 329, 330, 331, 332, 333, 334, 335, 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, 377, 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, 442, 444, 445, 446, 447, 448, 449, 450, 451, 452, 453, 454, 455, 456, 457, 460, 462, 466, 469, 472, 474, 480, 511, 520], "descript": [3, 4, 5, 19, 23, 32, 33, 35, 36, 37, 39, 40, 41, 42, 43, 44, 45, 47, 48, 50, 51, 52, 53, 54, 55, 60, 61, 62, 64, 65, 66, 67, 68, 69, 70, 71, 72, 74, 75, 76, 77, 78, 83, 84, 88, 96, 104, 106, 107, 112, 116, 118, 124, 128, 167, 384, 390, 408, 442, 460, 461, 463, 467, 468, 469, 470, 472, 473], "querycreatedev": [3, 328, 329, 414], "querycompletedev": [3, 250, 262, 263, 276, 328, 329, 333, 359, 382, 414, 421, 427], "splitcompletedev": [3, 275], "none": [3, 5, 9, 12, 14, 15, 20, 23, 32, 33, 35, 36, 37, 40, 41, 45, 54, 60, 64, 69, 70, 76, 100, 137, 146, 172, 207, 262, 272, 276, 377, 460, 462, 510, 520, 522], "list": [3, 5, 8, 11, 13, 14, 18, 23, 24, 27, 32, 33, 35, 39, 40, 41, 42, 43, 45, 49, 51, 53, 54, 55, 56, 57, 58, 59, 60, 62, 64, 65, 66, 68, 69, 70, 71, 73, 74, 75, 76, 77, 78, 80, 83, 84, 85, 91, 93, 94, 96, 98, 104, 105, 107, 112, 122, 123, 126, 132, 133, 136, 137, 142, 151, 169, 175, 176, 177, 178, 180, 194, 207, 213, 217, 220, 232, 242, 249, 258, 260, 262, 264, 267, 269, 278, 279, 280, 288, 293, 296, 302, 305, 327, 328, 332, 335, 341, 344, 345, 348, 354, 359, 365, 367, 368, 371, 379, 380, 383, 384, 385, 386, 388, 389, 390, 392, 399, 403, 405, 408, 415, 420, 423, 426, 428, 432, 433, 434, 436, 438, 444, 454, 456, 460, 462, 463, 464, 468, 472, 473, 475, 482, 488, 489, 490, 492, 496, 497, 509, 510, 520, 526, 527, 532, 533, 534, 535, 536, 537, 538, 539], "sent": [3, 13, 15, 40, 42, 55, 57, 66, 68, 69, 71, 74, 75, 76, 78, 83, 178, 242, 263, 334, 408], "along": [3, 25, 27, 39, 41, 45, 54, 59, 64, 77, 109, 142, 151, 170, 204, 209, 450, 451, 455, 461, 475, 496, 500, 506, 510, 527], "more": [3, 5, 8, 9, 13, 14, 15, 16, 19, 22, 23, 27, 32, 35, 38, 39, 40, 41, 42, 43, 44, 45, 47, 48, 51, 52, 54, 55, 57, 59, 60, 62, 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 76, 78, 83, 84, 85, 86, 88, 94, 96, 97, 99, 100, 102, 104, 109, 110, 112, 115, 125, 126, 132, 135, 136, 137, 138, 144, 146, 148, 150, 151, 156, 173, 176, 177, 180, 183, 186, 192, 194, 210, 213, 217, 228, 233, 240, 242, 244, 246, 251, 258, 260, 266, 271, 273, 284, 297, 303, 304, 307, 317, 335, 356, 364, 368, 369, 371, 373, 377, 379, 380, 390, 391, 392, 396, 399, 432, 433, 444, 448, 459, 462, 464, 465, 466, 468, 469, 470, 471, 473, 475, 476, 478, 486, 487, 493, 499, 506, 507, 508, 510, 511, 517, 518, 520, 522, 533, 542], "empti": [3, 14, 32, 38, 42, 45, 54, 57, 59, 60, 69, 70, 78, 83, 84, 94, 99, 100, 101, 109, 110, 112, 117, 120, 121, 124, 126, 162, 183, 203, 207, 209, 210, 226, 232, 233, 243, 249, 251, 257, 259, 263, 274, 276, 278, 297, 302, 317, 327, 329, 339, 345, 355, 356, 357, 361, 362, 373, 380, 381, 383, 387, 392, 418, 420, 421, 424, 430, 450, 472, 475, 488, 489, 490, 500, 520], "retri": [3, 9, 17, 37, 39, 41, 43, 45, 51, 54, 57, 64, 65, 66, 68, 70, 71, 74, 76, 78, 136, 138, 156, 157, 158, 175, 206, 209, 217, 235, 240, 242, 255, 257, 268, 276, 277, 279, 301, 303, 307, 317, 331, 355, 360, 367, 381, 384, 390, 393, 394, 396, 399, 400, 401, 403, 406, 408, 413, 414, 420, 432, 435, 449], "server": [3, 5, 6, 7, 11, 15, 19, 27, 32, 33, 34, 35, 36, 37, 39, 45, 51, 52, 54, 56, 57, 58, 59, 60, 64, 65, 69, 70, 72, 77, 78, 84, 132, 133, 134, 135, 136, 137, 138, 139, 151, 158, 163, 173, 177, 180, 183, 186, 195, 196, 204, 209, 223, 232, 233, 242, 244, 253, 258, 260, 262, 273, 276, 281, 285, 293, 297, 298, 329, 346, 348, 350, 352, 355, 356, 358, 361, 362, 363, 364, 368, 375, 384, 386, 416, 434, 446, 447, 449, 459, 461, 464, 465, 466, 467, 468, 471, 472, 473, 475, 476, 516, 524], "error": [3, 5, 8, 19, 25, 32, 35, 44, 45, 51, 52, 57, 58, 59, 63, 64, 66, 73, 83, 84, 94, 99, 100, 104, 105, 108, 112, 120, 131, 136, 155, 156, 161, 172, 175, 178, 180, 182, 185, 186, 194, 195, 196, 197, 207, 208, 210, 213, 214, 217, 218, 219, 221, 226, 228, 229, 233, 234, 235, 240, 242, 243, 245, 246, 248, 251, 254, 255, 257, 258, 259, 260, 266, 267, 268, 270, 273, 274, 276, 277, 278, 279, 280, 281, 287, 288, 290, 291, 293, 294, 296, 297, 301, 302, 303, 311, 313, 315, 317, 328, 330, 331, 335, 337, 338, 339, 343, 344, 345, 350, 353, 355, 358, 359, 360, 363, 365, 367, 368, 371, 374, 384, 386, 389, 393, 394, 396, 399, 400, 409, 410, 412, 413, 418, 420, 422, 426, 428, 432, 433, 434, 435, 449, 454, 455, 456, 468, 470, 471, 478, 480, 486, 488, 489, 490, 491, 498, 500, 501, 502, 505, 520], "A": [3, 5, 6, 7, 8, 9, 11, 13, 14, 19, 23, 25, 31, 32, 33, 35, 41, 44, 45, 47, 48, 50, 51, 52, 53, 54, 57, 59, 60, 64, 65, 68, 69, 70, 73, 78, 83, 84, 86, 94, 96, 97, 99, 101, 103, 104, 107, 112, 119, 120, 121, 124, 126, 127, 132, 134, 136, 137, 138, 144, 148, 151, 217, 226, 264, 278, 286, 287, 305, 357, 364, 373, 374, 460, 461, 462, 464, 465, 466, 467, 468, 469, 470, 472, 475, 478, 480, 487, 488, 489, 491, 510, 511, 512, 520, 523], "respons": [3, 5, 52, 82, 84, 85, 86, 89, 90, 92, 94, 95, 151, 155, 167, 173, 195, 263, 267, 268, 278, 279, 374, 384, 391, 446, 451, 468], "code": [3, 14, 32, 35, 41, 54, 78, 83, 84, 88, 97, 103, 119, 124, 132, 136, 144, 155, 156, 159, 161, 167, 173, 185, 188, 194, 208, 212, 213, 217, 218, 220, 234, 240, 251, 253, 255, 261, 270, 271, 274, 275, 281, 282, 283, 287, 293, 294, 296, 301, 302, 303, 316, 317, 325, 327, 334, 336, 342, 350, 420, 434, 467, 470, 475], "500": [3, 23, 38, 41, 51, 195, 210, 257, 302, 303, 334], "higher": [3, 5, 9, 13, 19, 23, 32, 33, 39, 40, 41, 42, 45, 53, 54, 57, 60, 62, 65, 66, 68, 70, 71, 73, 75, 76, 109, 113, 137, 138, 144, 151, 219, 246, 248, 255, 293, 366, 371, 373, 375, 377, 386, 434, 472, 510], "delai": [3, 5, 38, 39, 43, 64, 78, 170, 268, 279, 369], "durat": [3, 5, 11, 13, 17, 19, 20, 32, 35, 37, 40, 41, 42, 43, 44, 54, 55, 59, 62, 64, 66, 68, 69, 71, 72, 74, 75, 76, 78, 83, 84, 170, 228, 251, 260, 303, 337, 350, 353, 364, 374, 400, 403, 419, 469, 470, 473, 523], "attempt": [3, 5, 13, 41, 43, 47, 51, 55, 60, 63, 64, 68, 71, 72, 74, 75, 76, 78, 84, 89, 158, 217, 218, 221, 259, 276, 282, 317, 414, 423, 461, 512], "request": [3, 5, 6, 12, 13, 19, 25, 32, 33, 36, 39, 43, 48, 51, 58, 59, 61, 64, 65, 68, 70, 78, 84, 90, 94, 136, 157, 158, 175, 182, 183, 196, 217, 218, 251, 252, 253, 255, 263, 268, 274, 276, 277, 287, 303, 341, 345, 346, 359, 360, 365, 368, 373, 378, 379, 384, 390, 391, 396, 421, 425, 427, 428, 434, 455, 462, 465, 467, 469, 470, 475, 510], "1s": [3, 23, 43, 57, 64, 462], "backoff": [3, 39, 43, 51, 64, 78, 155, 235, 303, 384], "exponenti": [3, 16, 51, 78, 235, 301, 305], "formula": 3, "calcul": [3, 41, 84, 94, 100, 103, 108, 109, 110, 113, 131, 144, 147, 257, 270, 278, 291, 293, 332, 379, 450, 520], "attemptdelai": 3, "retrydelai": 3, "backoffbas": 3, "attemptcount": 3, "start": [3, 4, 7, 10, 23, 27, 32, 35, 36, 37, 39, 40, 41, 42, 45, 48, 51, 55, 57, 58, 59, 62, 66, 68, 69, 71, 73, 74, 75, 76, 77, 83, 93, 94, 100, 101, 104, 107, 109, 110, 112, 121, 124, 126, 131, 133, 135, 136, 137, 139, 142, 143, 144, 151, 178, 187, 223, 251, 253, 268, 287, 303, 330, 335, 339, 357, 361, 374, 378, 397, 414, 416, 445, 459, 466, 469, 470, 471, 472, 477, 485, 489, 505, 512, 519, 520], "leav": [3, 37, 45, 57, 59, 73, 136, 245, 402, 432, 478, 480], "constant": [3, 14, 29, 38, 55, 62, 66, 68, 71, 74, 75, 76, 84, 99, 104, 118, 120, 126, 144, 165, 167, 170, 173, 178, 180, 186, 194, 201, 206, 209, 232, 238, 246, 249, 250, 270, 274, 302, 313, 317, 339, 360, 367, 373, 375, 384, 387, 388, 389, 390, 426, 433, 512, 520], "upper": [3, 5, 54, 60, 72, 78, 99, 109, 112, 113, 114, 115, 118, 121, 124, 161, 186, 217, 288, 380, 388, 476], "bound": [3, 5, 19, 54, 60, 72, 78, 84, 99, 109, 112, 118, 161, 186, 203, 209, 257, 270, 278, 282, 291, 317, 347, 380, 420, 505, 510, 520], "1m": [3, 5, 13, 19, 36, 39, 40, 42, 43, 45, 55, 62, 66, 68, 69, 71, 74, 75, 76, 84, 106, 235, 257], "pass": [3, 4, 18, 25, 32, 33, 35, 37, 39, 40, 41, 42, 43, 45, 51, 54, 55, 62, 65, 66, 68, 69, 71, 74, 75, 76, 78, 82, 83, 84, 85, 89, 90, 91, 92, 94, 95, 103, 109, 113, 117, 119, 121, 137, 148, 151, 166, 246, 275, 362, 366, 368, 380, 391, 399, 405, 411, 416, 421, 425, 428, 434, 438, 450, 461, 467, 475, 480, 510, 525], "onto": [3, 25, 57, 89, 463], "mechan": [3, 5, 20, 25, 31, 32, 52, 57, 72, 96, 112, 132, 137, 242, 254, 333, 357, 396, 415, 469, 475, 510, 512, 514], "messag": [3, 5, 6, 12, 15, 32, 45, 54, 59, 60, 65, 66, 70, 73, 77, 78, 83, 84, 136, 170, 180, 186, 197, 208, 210, 217, 219, 221, 226, 229, 235, 242, 245, 248, 254, 259, 260, 268, 270, 273, 274, 279, 281, 297, 301, 304, 313, 317, 328, 331, 343, 355, 368, 369, 371, 379, 386, 394, 420, 441, 445, 448, 465, 468, 505], "pair": [3, 32, 33, 35, 41, 54, 57, 60, 73, 83, 94, 99, 109, 112, 117, 119, 121, 123, 124, 132, 465, 478, 480, 486], "comma": [3, 4, 5, 11, 13, 18, 32, 35, 39, 40, 42, 43, 45, 48, 54, 56, 57, 59, 60, 62, 64, 66, 68, 69, 70, 71, 73, 74, 75, 76, 78, 83, 85, 86, 144, 226, 254, 333, 345, 368, 388, 390, 459, 460, 463, 473, 475, 486], "If": [3, 4, 5, 6, 9, 13, 14, 15, 16, 18, 19, 20, 23, 25, 27, 32, 33, 35, 37, 39, 40, 41, 42, 43, 44, 45, 47, 48, 49, 51, 52, 53, 54, 55, 57, 59, 60, 61, 62, 64, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 78, 83, 84, 88, 94, 96, 98, 99, 100, 101, 103, 104, 105, 108, 109, 112, 116, 117, 124, 126, 131, 135, 136, 137, 138, 144, 146, 147, 148, 151, 152, 155, 159, 167, 170, 173, 178, 188, 192, 194, 212, 213, 217, 220, 244, 251, 255, 257, 270, 272, 275, 282, 293, 296, 303, 307, 317, 320, 327, 336, 342, 350, 352, 373, 460, 462, 464, 465, 466, 467, 468, 469, 470, 475, 476, 486, 487, 489, 493, 505, 508, 509, 510, 511, 512, 517, 518, 520, 525, 533, 543], "colon": [3, 35, 43, 51, 57, 59, 73, 255, 302, 463, 469, 472], "escap": [3, 12, 33, 45, 56, 104, 112, 121, 129, 142, 144, 226, 232, 233, 239, 263, 351, 382, 413, 425, 433, 454, 457], "backslash": [3, 56, 121, 233, 425, 454], "mind": [3, 6, 35, 44, 45], "static": [3, 5, 20, 45, 57, 84, 88, 96, 104, 270, 338, 474], "thei": [3, 5, 6, 14, 23, 24, 25, 27, 32, 33, 35, 37, 38, 39, 41, 44, 45, 52, 54, 57, 59, 60, 64, 73, 77, 78, 84, 85, 93, 94, 96, 98, 104, 109, 110, 112, 120, 124, 126, 127, 131, 136, 138, 143, 144, 146, 170, 173, 179, 193, 198, 225, 245, 247, 248, 249, 251, 257, 258, 261, 262, 263, 264, 272, 276, 280, 293, 297, 341, 345, 359, 375, 382, 384, 390, 410, 430, 452, 462, 473, 474, 475, 509, 510, 512, 520, 538], "carri": [3, 91, 96], "inform": [3, 4, 5, 6, 8, 12, 15, 19, 26, 27, 33, 35, 36, 37, 39, 40, 41, 42, 43, 45, 46, 47, 52, 53, 54, 55, 56, 57, 59, 60, 62, 64, 65, 66, 68, 69, 71, 73, 74, 75, 76, 77, 78, 83, 84, 86, 88, 91, 94, 96, 97, 99, 115, 120, 123, 126, 127, 132, 134, 135, 136, 138, 144, 147, 149, 151, 180, 183, 226, 228, 240, 246, 249, 257, 263, 266, 274, 317, 323, 329, 344, 345, 346, 347, 359, 362, 364, 367, 375, 382, 383, 386, 388, 392, 399, 403, 406, 414, 425, 441, 464, 465, 467, 469, 470, 471, 475, 505, 506, 510, 511, 520, 532], "taken": [3, 5, 29, 37, 41, 54, 56, 57, 72, 123, 201, 206, 389, 462], "itself": [3, 5, 29, 35, 56, 57, 59, 60, 64, 73, 83, 104, 136, 151, 244, 302, 360, 386, 464, 475], "histori": [4, 36, 56, 132, 155, 157, 200, 206, 209, 217, 257, 273, 279, 341, 348, 368, 391, 408, 414, 420, 434, 435, 445, 450, 454], "directli": [4, 17, 31, 32, 35, 37, 39, 40, 41, 42, 43, 44, 45, 48, 54, 55, 58, 60, 62, 65, 66, 68, 69, 70, 71, 74, 75, 76, 79, 80, 84, 88, 93, 99, 104, 119, 121, 135, 212, 213, 281, 293, 334, 350, 372, 382, 387, 448, 464, 466, 483, 491, 517, 521], "catalog": [4, 5, 8, 24, 29, 32, 33, 35, 36, 37, 38, 39, 40, 42, 43, 44, 45, 47, 48, 49, 50, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63, 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 76, 78, 79, 80, 83, 84, 93, 95, 96, 122, 126, 132, 133, 135, 138, 139, 142, 143, 148, 154, 155, 173, 183, 191, 192, 208, 211, 212, 232, 233, 235, 254, 259, 281, 282, 284, 293, 296, 297, 301, 302, 305, 307, 316, 327, 336, 343, 345, 346, 348, 350, 351, 355, 357, 359, 362, 371, 372, 373, 375, 378, 379, 380, 383, 388, 389, 390, 392, 393, 396, 397, 398, 399, 401, 403, 404, 406, 407, 409, 412, 420, 424, 426, 428, 430, 432, 433, 434, 435, 440, 445, 446, 447, 449, 450, 454, 460, 469, 471, 474, 477, 478, 482, 483, 486, 487, 488, 497, 499, 505, 508, 515, 518, 521, 522, 523, 533, 534, 535, 536, 543], "store": [4, 5, 9, 14, 23, 25, 29, 33, 35, 37, 39, 40, 41, 42, 43, 45, 46, 47, 51, 52, 53, 54, 55, 56, 57, 59, 61, 62, 63, 64, 65, 66, 68, 69, 70, 71, 73, 74, 75, 76, 84, 88, 94, 97, 110, 112, 120, 123, 127, 132, 136, 137, 139, 144, 151, 158, 228, 253, 257, 267, 296, 297, 302, 303, 341, 369, 379, 384, 389, 392, 401, 403, 414, 415, 418, 426, 432, 433, 438, 450, 461, 464, 465, 469, 472, 474, 482, 483, 486, 491, 514], "determin": [4, 9, 14, 32, 40, 41, 42, 44, 45, 54, 55, 57, 62, 66, 68, 71, 74, 75, 76, 78, 79, 80, 83, 84, 94, 95, 96, 104, 109, 112, 118, 121, 146, 148, 379, 382, 434, 439, 475, 496, 510, 512, 520], "jdbc": [4, 23, 31, 32, 40, 42, 54, 55, 62, 66, 68, 69, 71, 74, 75, 76, 81, 83, 84, 133, 135, 137, 155, 156, 159, 162, 165, 173, 177, 191, 198, 204, 209, 268, 282, 284, 288, 292, 293, 302, 303, 305, 317, 343, 345, 353, 380, 387, 390, 392, 429, 433, 434, 435, 436, 461, 467, 468, 469, 470, 471, 472, 473], "url": [4, 23, 32, 33, 40, 42, 43, 45, 47, 51, 55, 57, 59, 62, 64, 66, 68, 69, 70, 71, 74, 75, 76, 83, 84, 87, 98, 101, 133, 135, 137, 173, 241, 278, 284, 341, 359, 372, 378, 379, 383, 385, 388, 389, 392, 401, 422, 450, 456, 467, 469, 470, 472, 475], "ensur": [4, 6, 10, 13, 15, 23, 25, 40, 41, 42, 45, 48, 52, 53, 55, 57, 62, 65, 66, 68, 69, 71, 74, 75, 76, 84, 86, 94, 138, 144, 223, 242, 358, 360, 362, 394, 396, 398, 399, 408, 421, 422, 432, 438, 454, 459, 466, 468, 470, 473, 533], "port": [4, 7, 11, 12, 27, 32, 33, 35, 37, 39, 40, 41, 42, 43, 45, 47, 48, 51, 55, 57, 58, 60, 62, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 78, 129, 135, 136, 137, 209, 213, 232, 258, 304, 331, 392, 466, 468, 469, 470, 475], "3306": [4, 23, 62, 66, 75], "db": [4, 23, 65, 253, 255, 256, 262, 296, 344, 366, 472, 474], "net": [4, 5, 11, 33, 41, 45, 47, 50, 51, 53, 54, 62, 64, 66, 68, 71, 74, 75, 136, 137, 462, 467, 469, 472, 475], "domain": [4, 41, 43, 78, 84, 166, 272, 276, 327, 372, 374, 378, 426, 466, 469, 470, 475, 505], "paramet": [4, 25, 29, 32, 40, 41, 42, 43, 45, 51, 54, 55, 57, 59, 62, 64, 66, 68, 69, 71, 73, 74, 75, 76, 77, 86, 87, 88, 97, 103, 104, 112, 118, 119, 120, 126, 129, 137, 144, 147, 157, 180, 212, 218, 226, 254, 257, 270, 271, 274, 278, 293, 297, 315, 336, 347, 358, 359, 361, 366, 367, 372, 373, 379, 382, 397, 398, 401, 403, 408, 412, 414, 430, 438, 439, 450, 457, 461, 467, 468, 469, 470, 480, 486, 496, 503, 504, 513], "develop": [4, 32, 33, 58, 66, 85, 93, 126, 133, 151, 285, 297, 420, 467, 475], "guid": [4, 5, 37, 39, 45, 51, 52, 53, 66, 126, 133, 144, 151], "anoth": [4, 5, 22, 39, 40, 45, 48, 51, 54, 55, 57, 65, 66, 68, 71, 73, 74, 75, 76, 78, 99, 112, 119, 136, 138, 142, 151, 186, 232, 267, 334, 357, 374, 420, 461, 462, 470, 471, 491, 510, 520, 523], "alreadi": [4, 9, 13, 14, 15, 19, 23, 35, 41, 43, 44, 45, 47, 48, 50, 84, 94, 134, 135, 137, 198, 233, 246, 255, 268, 302, 338, 339, 359, 380, 386, 394, 408, 410, 418, 432, 433, 461, 462, 468, 475, 478, 480, 486, 488, 489, 490, 491, 510, 525], "new": [4, 5, 9, 11, 19, 20, 21, 23, 33, 35, 37, 38, 41, 43, 44, 45, 47, 52, 54, 57, 59, 60, 64, 65, 66, 68, 73, 76, 77, 84, 86, 87, 88, 93, 96, 97, 99, 100, 105, 121, 126, 132, 136, 137, 139, 155, 156, 157, 163, 167, 170, 173, 177, 180, 182, 194, 204, 209, 210, 217, 219, 220, 226, 233, 246, 247, 253, 256, 258, 261, 262, 263, 264, 267, 271, 272, 276, 278, 283, 287, 293, 296, 297, 300, 301, 302, 303, 304, 306, 334, 338, 339, 345, 355, 357, 359, 362, 363, 364, 366, 368, 369, 373, 378, 379, 381, 387, 388, 390, 391, 392, 394, 399, 401, 403, 410, 413, 414, 416, 418, 419, 420, 425, 428, 430, 432, 433, 436, 438, 446, 452, 455, 462, 464, 467, 471, 475, 486, 488, 489, 490, 491, 509, 511, 520, 523, 540, 544], "tabl": [4, 5, 10, 15, 17, 19, 23, 24, 25, 29, 32, 33, 36, 38, 46, 47, 48, 49, 50, 51, 52, 56, 67, 72, 78, 79, 80, 81, 83, 84, 86, 87, 94, 98, 104, 105, 107, 108, 110, 112, 114, 116, 125, 128, 132, 133, 135, 136, 137, 141, 142, 144, 145, 147, 148, 154, 155, 156, 157, 160, 162, 163, 165, 166, 167, 173, 178, 180, 183, 187, 191, 193, 196, 197, 198, 207, 208, 209, 211, 212, 217, 218, 219, 221, 223, 224, 225, 226, 227, 228, 229, 232, 236, 237, 238, 239, 240, 242, 243, 245, 246, 247, 248, 249, 250, 253, 254, 255, 256, 257, 258, 259, 260, 261, 263, 264, 265, 266, 267, 269, 270, 271, 272, 273, 274, 275, 276, 278, 280, 281, 286, 287, 291, 295, 296, 300, 302, 303, 304, 305, 306, 307, 311, 312, 313, 317, 320, 323, 327, 328, 329, 330, 331, 332, 333, 334, 335, 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, 350, 351, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, 377, 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, 405, 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, 442, 443, 444, 445, 446, 447, 448, 449, 450, 452, 453, 454, 455, 456, 457, 466, 467, 471, 476, 477, 478, 479, 482, 484, 486, 488, 491, 493, 494, 497, 505, 506, 507, 509, 510, 511, 514, 517, 520, 521, 527, 533, 538, 541, 542, 544], "trino_queri": 4, "ani": [4, 5, 8, 12, 14, 15, 17, 20, 23, 25, 27, 32, 33, 35, 37, 38, 39, 40, 41, 42, 43, 45, 48, 50, 51, 54, 55, 58, 59, 60, 62, 63, 64, 65, 66, 68, 69, 70, 71, 73, 74, 75, 76, 78, 80, 83, 84, 87, 88, 90, 91, 93, 94, 95, 96, 99, 100, 107, 109, 112, 114, 123, 124, 126, 127, 129, 131, 132, 135, 136, 137, 138, 139, 140, 142, 143, 144, 146, 147, 148, 151, 157, 167, 178, 186, 210, 220, 225, 228, 233, 245, 254, 258, 271, 272, 274, 275, 276, 278, 282, 284, 287, 293, 295, 302, 303, 310, 331, 333, 334, 335, 339, 346, 355, 356, 357, 364, 373, 379, 380, 389, 392, 393, 405, 424, 425, 436, 448, 460, 461, 462, 466, 469, 470, 471, 473, 474, 475, 476, 486, 491, 510, 511, 512, 520, 523, 533, 538, 542], "insert": [4, 15, 19, 23, 24, 35, 37, 38, 39, 41, 44, 45, 50, 52, 54, 63, 65, 81, 84, 94, 110, 124, 133, 137, 142, 143, 155, 177, 180, 186, 191, 196, 198, 210, 213, 223, 225, 235, 236, 242, 245, 257, 258, 269, 270, 302, 320, 339, 341, 345, 350, 351, 352, 353, 354, 355, 356, 357, 360, 362, 364, 366, 367, 368, 369, 372, 373, 374, 380, 381, 382, 383, 386, 387, 388, 389, 390, 391, 392, 396, 400, 403, 407, 408, 409, 414, 419, 420, 421, 422, 423, 426, 428, 429, 430, 432, 433, 434, 435, 436, 438, 441, 443, 445, 446, 448, 449, 450, 454, 460, 462, 477, 494, 505, 507, 511, 513, 517, 520, 544], "mani": [4, 5, 7, 9, 13, 19, 27, 29, 32, 39, 41, 43, 45, 51, 57, 60, 65, 66, 73, 75, 77, 78, 83, 84, 86, 105, 109, 112, 124, 132, 134, 136, 137, 143, 151, 152, 156, 178, 232, 246, 256, 259, 275, 276, 278, 280, 284, 287, 301, 355, 372, 377, 380, 381, 412, 413, 419, 432, 434, 454, 512, 520, 523], "identifi": [4, 36, 40, 41, 42, 54, 55, 57, 64, 66, 68, 69, 71, 74, 75, 76, 77, 78, 83, 93, 94, 96, 99, 112, 126, 129, 133, 136, 140, 144, 148, 244, 282, 296, 331, 332, 342, 357, 380, 384, 388, 404, 464, 467, 470, 471, 476, 510, 525, 538], "string": [4, 5, 9, 10, 11, 12, 13, 14, 15, 16, 18, 20, 23, 24, 33, 35, 37, 39, 40, 41, 42, 45, 47, 51, 54, 55, 57, 58, 59, 60, 62, 64, 65, 66, 68, 69, 70, 71, 73, 74, 75, 76, 78, 83, 84, 86, 88, 89, 90, 96, 98, 99, 100, 101, 103, 106, 107, 109, 112, 113, 118, 121, 122, 129, 132, 133, 140, 156, 161, 166, 197, 235, 236, 239, 248, 249, 250, 254, 257, 261, 262, 276, 287, 293, 295, 296, 297, 303, 304, 305, 317, 320, 324, 343, 344, 365, 369, 375, 377, 381, 384, 392, 396, 399, 403, 414, 423, 424, 432, 433, 434, 440, 446, 450, 453, 462, 464, 467, 469, 470, 474, 476, 486, 510, 525, 538], "user": [4, 5, 7, 15, 17, 23, 24, 29, 31, 32, 33, 35, 37, 39, 40, 41, 42, 43, 44, 45, 48, 51, 52, 54, 55, 57, 58, 60, 62, 64, 65, 66, 68, 69, 70, 71, 72, 74, 75, 76, 77, 83, 84, 89, 95, 96, 110, 122, 124, 132, 133, 136, 137, 138, 139, 140, 151, 152, 156, 176, 177, 197, 217, 228, 233, 254, 259, 261, 262, 264, 266, 270, 272, 275, 278, 282, 284, 285, 287, 293, 297, 305, 328, 329, 331, 334, 346, 349, 355, 357, 358, 359, 360, 363, 364, 366, 368, 369, 371, 373, 374, 379, 386, 392, 394, 396, 398, 399, 403, 408, 410, 432, 433, 434, 459, 460, 461, 462, 463, 467, 470, 472, 473, 478, 479, 480, 481, 484, 486, 487, 488, 491, 494, 499, 507, 508, 516, 517, 518, 520, 522, 523, 524, 532, 533, 534], "credenti": [4, 5, 32, 33, 37, 40, 42, 43, 49, 55, 59, 62, 64, 65, 66, 68, 71, 74, 75, 76, 83, 92, 155, 213, 261, 266, 329, 332, 334, 338, 343, 345, 346, 349, 357, 359, 366, 368, 379, 383, 384, 391, 392, 396, 399, 421, 432, 459, 462, 467, 469, 473], "By": [5, 9, 23, 25, 29, 32, 35, 37, 39, 40, 42, 45, 52, 53, 54, 55, 57, 60, 62, 66, 67, 68, 69, 71, 74, 75, 76, 80, 93, 94, 95, 112, 123, 131, 144, 146, 155, 191, 297, 302, 359, 462, 466, 468, 470, 510], "lack": [5, 27, 39, 45, 112, 268, 276, 296, 317, 520], "otherwis": [5, 6, 8, 14, 31, 32, 35, 54, 57, 59, 73, 77, 84, 94, 95, 99, 100, 105, 109, 112, 126, 254, 270, 293, 297, 473, 491, 493, 509, 510, 512, 517, 520, 523], "fail": [5, 7, 13, 15, 18, 19, 23, 25, 27, 32, 39, 40, 42, 45, 51, 54, 55, 57, 58, 59, 60, 62, 63, 66, 68, 69, 70, 71, 73, 74, 75, 76, 78, 83, 84, 94, 100, 105, 106, 112, 126, 136, 137, 138, 155, 162, 167, 170, 176, 177, 178, 182, 186, 187, 190, 194, 198, 201, 205, 206, 208, 211, 213, 214, 217, 218, 221, 223, 228, 229, 234, 235, 236, 240, 241, 243, 244, 246, 248, 250, 251, 255, 257, 258, 259, 260, 262, 264, 268, 270, 272, 276, 277, 279, 280, 281, 282, 287, 293, 297, 301, 303, 304, 310, 311, 315, 316, 317, 319, 327, 331, 333, 335, 337, 339, 343, 348, 354, 355, 356, 359, 361, 366, 368, 373, 375, 380, 384, 387, 390, 399, 400, 401, 406, 408, 410, 414, 418, 423, 424, 425, 427, 429, 433, 434, 449, 462, 468, 469, 470, 505, 510, 520], "must": [5, 6, 8, 13, 15, 17, 18, 19, 22, 23, 24, 27, 29, 32, 33, 35, 37, 39, 40, 41, 42, 44, 45, 46, 47, 48, 51, 52, 54, 55, 56, 57, 58, 59, 60, 62, 64, 65, 66, 68, 69, 70, 71, 73, 74, 75, 76, 78, 82, 83, 84, 85, 88, 89, 90, 91, 92, 93, 94, 95, 96, 99, 100, 101, 104, 109, 112, 118, 119, 120, 121, 123, 124, 126, 127, 128, 131, 135, 136, 138, 139, 142, 144, 146, 148, 149, 151, 177, 232, 248, 249, 256, 261, 270, 276, 284, 285, 296, 357, 427, 454, 460, 461, 462, 464, 465, 466, 467, 468, 469, 470, 472, 475, 500, 509, 510, 512, 514, 520, 521, 523, 524, 525], "again": [5, 6, 32, 35, 41, 48, 83, 470, 520], "manual": [5, 33, 41, 45, 57, 63, 65, 107, 128, 136, 139, 462, 466, 474], "longer": [5, 10, 15, 19, 25, 35, 39, 45, 54, 57, 59, 73, 88, 100, 146, 163, 182, 183, 208, 235, 248, 249, 256, 264, 276, 278, 282, 296, 297, 302, 303, 305, 328, 333, 335, 350, 351, 356, 357, 361, 367, 368, 369, 379, 383, 388, 392, 397, 399, 416, 420, 421, 438, 454, 470], "suscept": 5, "failur": [5, 10, 15, 23, 32, 45, 52, 66, 69, 71, 75, 77, 83, 85, 105, 112, 138, 156, 169, 173, 175, 177, 178, 180, 201, 206, 209, 210, 211, 212, 213, 217, 219, 220, 223, 225, 226, 229, 231, 232, 237, 239, 240, 243, 244, 245, 246, 247, 248, 249, 251, 252, 254, 255, 257, 258, 259, 260, 261, 262, 263, 264, 265, 267, 268, 270, 271, 273, 274, 275, 276, 277, 278, 279, 281, 287, 288, 291, 301, 317, 327, 329, 330, 331, 332, 333, 336, 337, 339, 340, 342, 343, 345, 346, 347, 350, 351, 352, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, 377, 379, 380, 381, 383, 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, 411, 412, 413, 414, 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, 430, 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, 442, 445, 446, 447, 448, 449, 450, 452, 453, 454, 457, 470, 510, 520], "mitig": [5, 253, 280, 303, 374], "compon": [5, 38, 45, 103, 123, 129, 131, 137, 144, 151, 204, 209, 280, 510, 512, 520], "event": [5, 15, 39, 41, 60, 81, 93, 133, 198, 217, 227, 231, 236, 240, 243, 253, 276, 282, 294, 296, 317, 344, 352, 354, 358, 359, 364, 367, 373, 374, 375, 390, 396, 400, 402, 427, 453, 510], "With": [5, 25, 32, 40, 42, 51, 55, 57, 59, 60, 62, 65, 66, 68, 69, 71, 73, 74, 75, 76, 89, 96, 112, 126, 146, 218, 253, 462, 474, 476, 510, 512], "intermedi": [5, 25, 84, 132, 151, 157, 178, 208, 240], "spool": [5, 17, 138, 401, 408, 409, 410, 428, 433], "re": [5, 14, 25, 29, 32, 35, 41, 45, 54, 67, 84, 138, 320, 352, 408, 415, 417, 425, 431, 432, 441, 475], "outag": [5, 138], "doe": [5, 6, 14, 15, 17, 18, 23, 25, 32, 33, 35, 37, 39, 40, 41, 42, 43, 44, 45, 48, 49, 50, 52, 54, 55, 57, 58, 59, 60, 62, 63, 64, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 78, 83, 84, 86, 87, 88, 91, 96, 99, 104, 107, 109, 112, 122, 124, 131, 132, 136, 143, 148, 149, 151, 178, 191, 197, 207, 221, 227, 243, 245, 254, 262, 273, 278, 281, 282, 284, 287, 296, 303, 339, 342, 350, 352, 355, 356, 357, 360, 375, 399, 403, 413, 414, 415, 423, 428, 430, 434, 452, 461, 462, 464, 465, 468, 469, 473, 475, 476, 478, 480, 486, 488, 489, 490, 498, 500, 501, 502, 505, 510, 512, 520, 523], "broken": [5, 73, 84, 112, 167, 174, 211, 217, 235, 293, 303, 331, 348, 360, 376], "spend": [5, 15, 99, 246], "becaus": [5, 13, 14, 15, 19, 35, 37, 39, 40, 41, 42, 43, 44, 45, 47, 51, 52, 54, 55, 56, 57, 58, 59, 60, 62, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 78, 84, 88, 93, 112, 126, 136, 146, 151, 242, 276, 293, 296, 304, 425, 431, 467, 470, 475, 505, 510, 512, 520], "its": [5, 8, 15, 23, 33, 37, 41, 42, 43, 51, 52, 54, 59, 62, 63, 65, 66, 68, 71, 74, 76, 80, 83, 84, 88, 89, 93, 96, 97, 99, 103, 110, 112, 123, 132, 136, 139, 169, 241, 313, 322, 331, 336, 347, 358, 418, 446, 447, 456, 461, 464, 465, 468, 469, 475, 476, 478, 480, 510, 520], "sql": [5, 7, 15, 32, 33, 34, 83, 88, 96, 97, 98, 100, 101, 104, 105, 107, 109, 112, 113, 122, 123, 124, 126, 128, 132, 133, 137, 138, 141, 142, 144, 146, 148, 149, 150, 151, 152, 157, 172, 186, 207, 213, 217, 223, 226, 232, 249, 257, 261, 265, 270, 271, 274, 278, 279, 283, 288, 289, 293, 317, 328, 331, 334, 341, 344, 346, 349, 351, 352, 353, 355, 356, 357, 358, 360, 361, 362, 368, 373, 374, 375, 384, 385, 386, 389, 393, 447, 456, 462, 486, 520, 525, 528, 529, 530, 531], "cannot": [5, 13, 14, 32, 35, 48, 50, 54, 55, 57, 59, 62, 65, 66, 68, 71, 74, 75, 76, 84, 96, 97, 105, 108, 112, 126, 143, 148, 161, 223, 280, 297, 302, 358, 371, 374, 380, 394, 399, 421, 428, 432, 445, 462, 476, 496, 510, 512], "pars": [5, 43, 45, 57, 58, 59, 73, 84, 106, 107, 112, 128, 151, 180, 195, 204, 208, 209, 211, 223, 243, 244, 251, 255, 263, 268, 270, 293, 302, 303, 310, 343, 346, 358, 362, 366, 368, 424, 433, 455], "how": [5, 13, 15, 20, 23, 27, 32, 35, 37, 40, 41, 42, 43, 44, 45, 46, 48, 51, 53, 54, 57, 58, 59, 60, 62, 64, 66, 68, 69, 71, 72, 74, 75, 76, 77, 84, 87, 96, 123, 124, 126, 137, 138, 146, 148, 149, 151, 157, 167, 193, 281, 287, 330, 341, 355, 357, 360, 368, 397, 434, 462, 463, 464, 472, 473, 475, 486, 505, 510, 520], "resili": [5, 133, 134, 260, 273, 368], "featur": [5, 24, 25, 35, 37, 38, 39, 40, 41, 42, 43, 44, 45, 48, 54, 55, 57, 60, 62, 63, 64, 65, 66, 68, 69, 71, 74, 75, 76, 84, 109, 112, 113, 114, 115, 132, 146, 152, 155, 157, 172, 178, 183, 219, 227, 233, 234, 256, 267, 270, 280, 285, 293, 296, 297, 301, 302, 304, 305, 310, 337, 344, 357, 364, 371, 375, 379, 380, 393, 403, 418, 436, 455, 461, 471, 520], "desir": [5, 23, 33, 38, 47, 57, 95, 112, 137, 397, 460, 462, 468, 510, 526, 527, 532, 536, 537, 539], "caus": [5, 9, 13, 14, 15, 18, 19, 23, 25, 27, 39, 40, 41, 42, 43, 45, 51, 54, 55, 66, 68, 69, 70, 71, 74, 75, 76, 84, 88, 112, 137, 138, 144, 157, 159, 160, 161, 163, 166, 167, 169, 172, 175, 176, 177, 178, 183, 186, 187, 193, 194, 197, 199, 201, 206, 208, 211, 213, 214, 218, 221, 222, 223, 228, 229, 232, 234, 235, 239, 240, 241, 242, 243, 244, 246, 247, 248, 249, 251, 252, 253, 254, 256, 257, 258, 260, 262, 265, 268, 276, 278, 280, 291, 293, 295, 304, 305, 306, 307, 309, 311, 315, 320, 321, 322, 323, 333, 335, 341, 349, 355, 361, 375, 389, 392, 397, 410, 418, 424, 428, 432, 433, 434, 478, 480, 486, 487, 488, 489, 490, 491, 498, 500, 501, 502, 508, 510, 518], "explicitli": [5, 37, 51, 52, 57, 60, 65, 106, 110, 196, 261, 343, 359, 372, 388, 397, 412, 415, 454, 462, 473, 505, 520], "vari": [5, 11, 45, 68, 80, 94, 103, 107, 132, 136, 143, 144, 148, 149, 466, 468, 478, 480, 514], "basi": [5, 8, 51, 83, 96, 109, 126, 192, 245, 293, 307, 354, 389, 462, 480, 520, 523], "bigqueri": [5, 34, 119, 133, 358], "delta": [5, 34, 45, 50, 54, 64, 67, 133, 138, 359, 371, 381, 395, 397, 399, 400, 407], "lake": [5, 34, 45, 47, 50, 54, 64, 67, 133, 138, 346, 359, 364, 371, 395, 397, 400, 407], "iceberg": [5, 30, 34, 41, 45, 67, 132, 133, 137, 138, 368, 378, 462, 486], "mongodb": [5, 34, 133, 207, 209], "oracl": [5, 23, 30, 34, 133, 152, 341, 361, 374, 377, 380, 384, 469], "postgresql": [5, 23, 34, 64, 104, 132, 133, 137, 138, 148, 152, 184, 334, 349, 352, 353, 355, 358, 360, 374, 375, 377, 462, 483, 521], "redshift": [5, 34, 133, 251, 334, 349, 352, 353, 355, 357, 358, 360, 373, 374, 375, 377, 384], "control": [5, 6, 13, 17, 18, 19, 20, 24, 27, 32, 35, 36, 38, 41, 45, 52, 54, 57, 59, 64, 66, 68, 69, 70, 71, 73, 75, 76, 81, 82, 89, 90, 92, 93, 119, 132, 133, 137, 139, 155, 157, 167, 178, 180, 190, 196, 200, 206, 218, 223, 232, 235, 239, 244, 246, 257, 261, 266, 267, 269, 270, 272, 278, 279, 282, 307, 328, 344, 346, 347, 351, 354, 355, 357, 358, 359, 361, 362, 363, 364, 368, 371, 372, 377, 383, 387, 388, 392, 395, 397, 402, 408, 421, 427, 428, 430, 432, 433, 441, 447, 455, 456, 463, 470, 494, 520, 524], "behavior": [5, 8, 9, 15, 24, 29, 32, 35, 45, 51, 55, 57, 59, 64, 66, 69, 71, 73, 75, 76, 87, 88, 95, 98, 138, 144, 146, 148, 156, 189, 206, 212, 226, 228, 246, 255, 257, 261, 263, 264, 266, 269, 270, 271, 272, 276, 287, 297, 327, 328, 333, 336, 339, 344, 352, 355, 356, 357, 359, 362, 366, 369, 373, 375, 381, 382, 384, 389, 392, 395, 399, 402, 414, 425, 426, 428, 432, 434, 438, 446, 450, 462, 471, 476, 489, 510, 520], "whole": [5, 8, 15, 25, 51, 67, 84, 414, 452, 462, 510, 523], "individu": [5, 13, 15, 54, 84, 86, 94, 109, 110, 123, 135, 147, 400, 401, 462, 464, 471], "dedupl": 5, "buffer": [5, 15, 19, 27, 32, 41, 45, 58, 138, 151, 173, 197, 198, 200, 206, 217, 228, 233, 239, 250, 255, 257, 259, 261, 262, 263, 269, 271, 274, 293, 319, 320, 321, 323, 372, 426, 427, 428], "fill": [5, 100, 269, 509, 513], "descriptor": [5, 56, 57, 126, 136, 207, 213, 358, 360, 505], "capac": [5, 48, 99], "been": [5, 13, 19, 23, 24, 27, 33, 41, 43, 45, 48, 51, 52, 53, 54, 59, 60, 65, 72, 82, 84, 85, 89, 90, 91, 92, 94, 95, 137, 152, 155, 156, 163, 170, 173, 192, 193, 232, 239, 242, 247, 252, 256, 257, 260, 261, 263, 267, 271, 293, 296, 297, 302, 303, 304, 307, 316, 330, 337, 357, 379, 388, 402, 414, 433, 438, 450, 451, 454, 466, 475, 522, 533, 534, 542], "exceed": [5, 14, 15, 18, 19, 25, 41, 65, 278, 333, 380, 381, 412, 427, 454], "unless": [5, 8, 14, 20, 44, 45, 52, 68, 72, 78, 242, 283, 305, 348, 510, 520, 523], "32mb": [5, 9, 17, 41], "compress": [5, 19, 32, 33, 41, 45, 54, 60, 69, 132, 136, 196, 197, 230, 260, 262, 269, 274, 275, 276, 278, 281, 293, 321, 327, 335, 340, 341, 348, 356, 359, 374, 377, 379, 389, 392, 399, 412, 418, 438, 449], "design": [5, 13, 38, 54, 132, 138, 150, 151, 152, 217, 420, 465, 467, 510], "whether": [5, 20, 32, 33, 37, 39, 41, 45, 51, 54, 57, 59, 64, 66, 67, 68, 69, 71, 73, 75, 76, 94, 95, 100, 104, 112, 124, 146, 180, 261, 293, 328, 347, 355, 362, 369, 392, 410, 462, 476, 510, 512, 520], "entir": [5, 17, 27, 39, 45, 51, 54, 57, 67, 68, 71, 84, 121, 157, 177, 204, 209, 245, 267, 276, 392, 506], "instruct": [5, 58, 83, 134, 135, 137, 176, 182, 470, 472], "occur": [5, 32, 40, 41, 45, 52, 55, 62, 66, 68, 69, 71, 74, 75, 76, 99, 112, 123, 136, 177, 182, 186, 229, 235, 245, 248, 255, 259, 260, 261, 262, 274, 276, 277, 303, 333, 357, 360, 390, 406, 510], "major": [5, 65, 136, 282, 288, 296, 302, 357, 390, 392, 471], "workload": [5, 24, 26, 48, 136, 151, 152, 255, 287, 396, 520], "consist": [5, 8, 24, 39, 47, 54, 57, 59, 60, 65, 73, 83, 94, 96, 126, 136, 151, 170, 187, 235, 242, 259, 280, 303, 337, 338, 373, 375, 408, 412, 510, 512, 520], "implement": [5, 14, 23, 25, 29, 32, 35, 45, 47, 48, 51, 54, 64, 71, 74, 76, 83, 84, 91, 93, 96, 97, 98, 99, 101, 110, 112, 119, 123, 126, 132, 143, 144, 148, 151, 154, 156, 167, 177, 178, 183, 192, 208, 209, 213, 245, 247, 255, 260, 264, 265, 270, 282, 284, 285, 287, 288, 301, 302, 303, 304, 306, 307, 315, 324, 333, 336, 339, 344, 357, 358, 361, 364, 365, 368, 373, 374, 388, 390, 396, 409, 423, 426, 427, 432, 450, 457, 460, 462, 468, 469, 471, 475, 506, 514, 520], "whose": [5, 14, 40, 41, 42, 45, 52, 54, 62, 66, 68, 71, 74, 76, 83, 94, 96, 109, 119, 221, 305, 311, 392, 497, 510], "return": [5, 9, 14, 25, 29, 32, 37, 38, 41, 42, 43, 45, 48, 54, 56, 57, 59, 62, 63, 65, 66, 68, 70, 71, 72, 73, 74, 76, 78, 82, 83, 84, 85, 86, 87, 88, 89, 90, 91, 92, 94, 95, 96, 97, 99, 100, 101, 102, 103, 104, 105, 106, 107, 109, 110, 111, 117, 118, 119, 120, 121, 122, 123, 124, 125, 126, 127, 128, 129, 130, 131, 136, 137, 139, 144, 148, 151, 156, 167, 173, 176, 177, 179, 183, 190, 191, 194, 195, 204, 209, 210, 213, 217, 218, 223, 226, 227, 228, 233, 234, 245, 247, 248, 257, 259, 261, 262, 264, 267, 274, 276, 278, 281, 287, 293, 301, 303, 305, 316, 327, 334, 345, 346, 350, 352, 356, 359, 362, 366, 368, 369, 371, 373, 375, 380, 382, 383, 384, 388, 389, 391, 395, 401, 402, 403, 405, 406, 413, 415, 419, 421, 430, 432, 434, 439, 441, 446, 448, 449, 456, 461, 462, 467, 473, 475, 491, 505, 510, 512, 520, 532, 538, 544], "veri": [5, 9, 14, 25, 45, 48, 56, 79, 80, 86, 87, 99, 110, 198, 227, 257, 287, 324, 356, 362, 405, 472], "modifi": [5, 15, 35, 37, 39, 40, 41, 42, 44, 45, 52, 54, 57, 60, 62, 65, 66, 68, 69, 71, 72, 75, 76, 88, 99, 112, 225, 282, 302, 331, 406, 414, 415, 462, 480, 510, 512, 523], "greater": [5, 14, 17, 25, 60, 62, 68, 84, 99, 100, 104, 109, 113, 118, 120, 127, 131, 190, 204, 209, 216, 217, 262, 368, 373, 375, 379, 386, 396, 425, 438, 441, 486, 520], "larger": [5, 9, 10, 16, 19, 22, 25, 41, 45, 48, 54, 64, 68, 99, 100, 124, 136, 138, 146, 151, 156, 256, 275, 277, 279, 282, 296, 302, 304, 321, 352, 355, 359, 365, 372, 391, 395, 407, 432, 453], "strongli": [5, 41, 137, 475], "beyond": [5, 15, 124, 126, 136, 510], "batch": [5, 13, 35, 39, 40, 41, 42, 53, 55, 62, 64, 66, 67, 68, 69, 71, 74, 75, 76, 78, 83, 138, 206, 221, 259, 287, 330, 334, 345, 373, 387, 402, 432, 453], "effici": [5, 14, 25, 37, 48, 99, 110, 123, 144, 151, 152, 160, 186, 192, 209, 250, 259, 270, 278, 301, 303, 304, 307, 317, 342, 368, 369, 371, 391, 392], "within": [5, 15, 22, 23, 25, 35, 45, 47, 51, 52, 54, 56, 57, 58, 59, 63, 65, 71, 73, 74, 76, 77, 79, 80, 86, 87, 96, 99, 100, 103, 104, 109, 112, 113, 121, 123, 126, 131, 135, 137, 144, 148, 151, 154, 180, 240, 250, 268, 270, 297, 357, 364, 369, 370, 375, 379, 380, 384, 399, 401, 403, 415, 419, 422, 432, 435, 436, 448, 470, 491, 510, 512, 520], "rather": [5, 9, 22, 45, 51, 88, 99, 121, 154, 162, 168, 183, 186, 203, 204, 208, 209, 212, 213, 217, 233, 249, 254, 255, 256, 262, 266, 267, 270, 272, 274, 275, 278, 297, 303, 317, 331, 332, 334, 341, 342, 348, 352, 368, 371, 375, 377, 379, 383, 440, 460, 469, 486, 491], "have": [5, 6, 7, 10, 11, 13, 14, 19, 22, 23, 24, 25, 27, 32, 33, 35, 36, 37, 39, 40, 41, 42, 43, 44, 45, 47, 48, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 62, 64, 65, 66, 68, 69, 71, 72, 73, 74, 75, 76, 78, 82, 83, 84, 86, 87, 88, 90, 91, 92, 94, 95, 96, 100, 104, 109, 112, 121, 124, 131, 132, 134, 135, 136, 137, 142, 143, 146, 148, 149, 151, 155, 156, 159, 167, 170, 173, 176, 177, 182, 188, 192, 193, 194, 198, 212, 213, 214, 217, 219, 220, 225, 239, 241, 242, 244, 247, 248, 249, 250, 251, 254, 255, 256, 257, 258, 259, 261, 262, 263, 264, 266, 267, 270, 271, 272, 273, 275, 282, 283, 284, 287, 288, 293, 294, 296, 297, 301, 302, 303, 307, 314, 316, 317, 325, 327, 330, 336, 337, 338, 341, 342, 346, 350, 351, 352, 355, 357, 360, 365, 368, 371, 373, 377, 379, 380, 381, 382, 384, 387, 388, 389, 392, 399, 402, 405, 414, 418, 419, 421, 430, 434, 436, 438, 450, 451, 454, 455, 461, 462, 464, 465, 466, 467, 468, 469, 470, 473, 475, 476, 483, 493, 494, 507, 510, 512, 517, 522, 523, 533, 534, 536, 542], "practic": [5, 25, 35, 45, 52, 68, 71, 137, 471, 475], "affect": [5, 6, 27, 32, 37, 45, 51, 54, 57, 96, 259, 265, 270, 271, 306, 330, 336, 339, 368, 450, 510, 520], "make": [5, 6, 10, 14, 15, 23, 32, 35, 39, 41, 45, 47, 48, 54, 57, 59, 65, 66, 71, 73, 74, 75, 76, 77, 78, 84, 88, 109, 112, 113, 135, 137, 146, 151, 152, 154, 155, 157, 162, 167, 183, 192, 204, 209, 217, 219, 220, 228, 230, 240, 242, 243, 245, 255, 256, 259, 264, 267, 269, 273, 276, 278, 279, 280, 282, 287, 293, 302, 303, 307, 310, 325, 326, 327, 330, 332, 334, 337, 339, 360, 362, 366, 373, 375, 384, 392, 407, 420, 462, 464, 465, 468, 469, 475], "low": [5, 13, 14, 19, 35, 41, 43, 48, 54, 78, 103, 105, 123, 144, 149, 157, 170, 198, 246, 253, 256, 297, 368, 397, 400, 408, 433, 493, 505, 506], "killer": [5, 170, 256, 273, 400, 408], "total": [5, 11, 13, 15, 23, 32, 35, 48, 54, 58, 60, 64, 78, 79, 99, 100, 109, 131, 135, 136, 137, 144, 149, 157, 167, 217, 256, 258, 270, 272, 332, 335, 346, 348, 377, 395, 396, 400, 405, 419, 472, 475, 520, 538], "reserv": [5, 15, 56, 157, 167, 219, 228, 230, 231, 232, 256, 257, 258, 270, 278, 329, 333, 351, 352, 356, 390, 395, 399, 400, 420, 422], "block": [5, 6, 15, 19, 27, 41, 46, 64, 65, 67, 69, 78, 84, 93, 94, 97, 110, 121, 141, 154, 155, 158, 159, 160, 172, 197, 219, 232, 242, 256, 257, 260, 264, 270, 276, 278, 285, 287, 302, 325, 329, 333, 334, 345, 350, 366, 367, 369, 380, 390, 397, 400, 420, 428, 432, 454, 457, 462, 466, 506, 520], "kill": [5, 7, 15, 17, 25, 27, 77, 136, 137, 170, 186, 187, 198, 249, 251, 256, 269, 274, 334, 358, 408, 462, 483], "out": [5, 7, 15, 19, 20, 22, 23, 25, 32, 41, 45, 53, 54, 57, 72, 73, 77, 88, 94, 105, 108, 109, 112, 123, 137, 138, 148, 161, 192, 209, 210, 223, 225, 244, 248, 249, 251, 252, 254, 256, 267, 268, 273, 274, 278, 288, 297, 308, 330, 333, 337, 371, 373, 392, 393, 395, 398, 413, 427, 455, 456, 470, 475, 491, 520], "0s": [5, 8, 40, 41, 42, 54, 55, 62, 64, 66, 68, 69, 71, 74, 75, 76], "immedi": [5, 10, 15, 32, 39, 44, 63, 133, 143, 293, 327, 333, 384, 445, 477, 492, 503, 510, 513, 520], "unblock": [5, 395], "adjust": [5, 9, 19, 40, 41, 42, 45, 55, 62, 65, 66, 67, 68, 69, 71, 74, 75, 76, 136, 193, 206, 228, 233, 368, 392, 520], "long": [5, 15, 20, 41, 42, 43, 44, 45, 52, 54, 57, 58, 59, 64, 70, 72, 73, 77, 78, 80, 84, 88, 94, 96, 97, 123, 144, 158, 198, 222, 223, 228, 229, 239, 246, 257, 258, 260, 264, 268, 287, 289, 303, 355, 360, 368, 391, 395, 396, 399, 408, 416, 418, 420, 421, 432, 473, 475, 486], "try": [5, 18, 25, 54, 59, 83, 84, 113, 114, 115, 137, 194, 201, 206, 213, 217, 221, 227, 229, 250, 354, 360, 377, 400, 433, 442, 466, 467, 468, 512], "reconnect": [5, 293], "lost": [5, 60, 63, 197, 523], "reschedul": [5, 138], "suit": [5, 48, 137, 138, 276, 297, 468, 475], "latenc": [5, 7, 9, 13, 25, 40, 42, 45, 55, 59, 62, 66, 68, 69, 71, 74, 75, 76, 138, 186, 263, 282, 287, 288, 329, 355, 367, 368, 369, 371, 374, 393, 397, 405, 413, 418, 424, 427, 433, 434, 438, 439], "short": [5, 35, 41, 43, 57, 59, 67, 73, 103, 138, 232, 233, 245, 246, 257, 361, 384, 405, 419, 427, 432, 446, 467], "high": [5, 9, 13, 15, 19, 35, 38, 45, 48, 54, 78, 97, 103, 105, 137, 138, 144, 149, 161, 173, 222, 223, 243, 248, 249, 256, 257, 260, 263, 271, 273, 293, 373, 395, 418, 434, 438, 466, 470, 475, 505], "volum": [5, 36, 48, 135, 138], "As": [5, 15, 32, 35, 39, 41, 42, 45, 48, 54, 58, 60, 62, 65, 66, 68, 71, 78, 84, 86, 88, 94, 99, 109, 119, 138, 144, 146, 148, 151, 271, 293, 296, 297, 302, 379, 401, 454, 510, 512], "dedic": [5, 43, 48, 96, 112, 126, 136, 138, 383, 467, 469], "handl": [5, 8, 10, 15, 19, 25, 35, 36, 40, 41, 42, 45, 54, 62, 68, 72, 74, 76, 84, 85, 86, 91, 93, 94, 96, 104, 105, 116, 121, 129, 138, 144, 152, 155, 157, 158, 162, 166, 167, 173, 180, 182, 183, 194, 196, 203, 204, 208, 209, 213, 217, 218, 229, 242, 246, 253, 255, 257, 268, 272, 274, 279, 285, 287, 288, 290, 291, 295, 297, 302, 304, 308, 310, 311, 325, 332, 334, 341, 344, 346, 354, 355, 356, 365, 366, 367, 368, 372, 373, 377, 380, 383, 384, 386, 397, 399, 400, 401, 402, 405, 420, 425, 462, 464, 467, 474], "deploy": [5, 41, 42, 48, 52, 57, 59, 65, 73, 132, 134, 136, 137, 138, 251, 252, 269, 466], "test": [5, 29, 32, 33, 38, 41, 45, 48, 51, 54, 57, 58, 64, 66, 71, 76, 79, 80, 84, 100, 104, 124, 136, 137, 151, 172, 178, 190, 192, 206, 277, 287, 297, 307, 320, 356, 462, 465, 467, 472, 475, 476, 483, 491, 517, 520], "troubleshoot": [5, 66, 71, 76], "threshold": [5, 14, 22, 25, 35, 41, 45, 54, 67, 166, 177, 218, 256, 267, 270, 279, 333, 361, 372, 374, 399, 423, 426, 455], "repeat": [5, 9, 48, 57, 100, 109, 114, 115, 137, 229, 244, 257, 303, 331, 360, 361, 385, 418, 486, 520, 540], "maximum": [5, 8, 9, 11, 12, 13, 14, 15, 16, 17, 19, 22, 23, 36, 37, 39, 40, 41, 42, 43, 44, 45, 51, 53, 54, 55, 56, 57, 59, 60, 62, 64, 65, 66, 67, 68, 69, 71, 74, 75, 76, 78, 99, 100, 110, 120, 136, 137, 144, 146, 164, 231, 234, 255, 257, 259, 262, 263, 267, 270, 272, 277, 278, 279, 282, 302, 307, 313, 321, 329, 361, 367, 383, 388, 389, 413, 421, 426, 427, 444, 464, 465, 470, 473], "declar": [5, 29, 45, 67, 69, 88, 112, 126, 167, 186, 245, 268, 270, 282, 341, 352, 356, 365, 368, 371, 373, 381, 395, 397, 462, 510, 520], "onli": [5, 10, 13, 14, 15, 19, 21, 22, 23, 27, 29, 32, 33, 35, 37, 39, 40, 41, 42, 43, 44, 45, 49, 51, 52, 53, 54, 55, 57, 59, 60, 62, 64, 66, 67, 68, 69, 71, 73, 74, 75, 76, 78, 83, 84, 86, 88, 93, 94, 96, 99, 104, 105, 109, 112, 117, 121, 124, 135, 136, 137, 138, 146, 147, 148, 154, 157, 162, 166, 172, 177, 183, 196, 206, 207, 217, 228, 232, 235, 245, 246, 247, 248, 250, 259, 260, 264, 267, 270, 271, 272, 282, 287, 293, 296, 302, 306, 313, 331, 336, 342, 343, 344, 345, 346, 347, 354, 357, 359, 360, 364, 368, 371, 373, 377, 381, 386, 389, 391, 392, 400, 403, 420, 421, 425, 427, 430, 432, 436, 439, 444, 448, 452, 461, 462, 465, 466, 467, 469, 470, 474, 475, 476, 482, 486, 493, 500, 510, 511, 512, 517, 520, 521, 540], "initi": [5, 15, 41, 45, 59, 65, 68, 77, 83, 84, 86, 99, 100, 110, 123, 136, 163, 180, 193, 206, 209, 217, 245, 247, 268, 282, 297, 307, 317, 330, 337, 361, 392, 432, 435, 454, 461, 466, 471, 510, 512, 514, 525], "minimum": [5, 8, 9, 11, 13, 14, 15, 16, 19, 22, 41, 43, 45, 51, 53, 54, 59, 64, 65, 69, 78, 99, 100, 109, 124, 136, 144, 164, 221, 256, 259, 266, 270, 279, 302, 314, 328, 329, 339, 367, 374, 386, 389, 396, 414, 417, 425, 427, 428, 432, 468, 472], "wait": [5, 9, 13, 15, 19, 27, 39, 40, 41, 42, 45, 48, 54, 55, 59, 62, 64, 66, 68, 69, 71, 73, 74, 75, 76, 157, 161, 247, 251, 279, 287, 329, 333, 339, 366, 368, 369, 374, 392, 403, 412, 438], "overridden": [5, 24, 32, 33, 41, 58, 137, 364], "retry_initial_delai": 5, "10": [5, 11, 13, 19, 23, 35, 41, 43, 44, 45, 51, 53, 54, 55, 56, 57, 58, 59, 60, 62, 64, 65, 68, 69, 72, 74, 76, 78, 84, 97, 100, 102, 103, 107, 108, 111, 112, 113, 117, 118, 124, 137, 144, 153, 157, 177, 209, 259, 262, 293, 302, 307, 328, 344, 367, 383, 393, 395, 399, 406, 417, 427, 466, 472, 480, 505, 506, 510, 520, 525], "ref": [5, 64, 109, 438], "prop": [5, 121], "subsequ": [5, 16, 48, 64, 83, 88, 94, 96, 100, 126, 142, 146, 278, 333, 350, 462, 511, 512], "retry_max_delai": 5, "retry_delay_scale_factor": 5, "import": [5, 8, 41, 57, 84, 137, 256, 287, 293, 464, 476], "amount": [5, 8, 9, 14, 15, 17, 19, 22, 23, 25, 41, 43, 44, 45, 48, 64, 72, 86, 132, 138, 146, 147, 148, 152, 157, 167, 178, 239, 248, 255, 256, 334, 349, 408, 430, 466, 468], "too": [5, 9, 13, 19, 43, 72, 77, 84, 136, 137, 163, 173, 185, 223, 240, 246, 278, 280, 291, 301, 303, 313, 317, 330, 381, 397, 402, 425, 466], "requir": [5, 8, 10, 13, 14, 15, 16, 18, 23, 24, 25, 27, 29, 43, 44, 48, 56, 59, 63, 64, 77, 79, 80, 83, 84, 86, 88, 91, 94, 95, 96, 99, 109, 112, 124, 131, 138, 139, 144, 146, 172, 178, 183, 207, 217, 218, 223, 232, 233, 247, 248, 249, 254, 255, 260, 262, 263, 268, 270, 271, 273, 274, 276, 278, 279, 281, 284, 293, 304, 305, 328, 335, 338, 339, 342, 346, 357, 358, 360, 365, 366, 367, 371, 374, 381, 383, 386, 388, 389, 396, 397, 400, 401, 403, 405, 414, 417, 420, 422, 425, 427, 428, 430, 432, 435, 436, 447, 450, 451, 457, 459, 461, 462, 464, 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 506, 510, 512, 520, 533], "one": [5, 8, 12, 13, 15, 19, 22, 23, 25, 27, 29, 32, 35, 37, 39, 40, 41, 42, 44, 45, 47, 49, 50, 51, 52, 54, 55, 56, 57, 59, 60, 62, 63, 64, 65, 66, 67, 68, 69, 71, 73, 74, 75, 76, 77, 78, 84, 86, 88, 89, 94, 96, 99, 100, 102, 103, 104, 105, 108, 109, 110, 112, 113, 116, 117, 120, 121, 124, 126, 131, 132, 134, 136, 137, 144, 146, 147, 148, 150, 151, 155, 157, 167, 180, 216, 217, 226, 250, 253, 258, 259, 260, 263, 264, 267, 269, 271, 274, 282, 287, 293, 313, 334, 348, 354, 356, 364, 365, 373, 381, 383, 392, 409, 413, 425, 428, 434, 459, 460, 461, 462, 463, 464, 465, 466, 469, 471, 472, 473, 475, 476, 489, 505, 506, 510, 511, 512, 520, 540, 544], "prevent": [5, 9, 13, 15, 19, 25, 41, 44, 52, 54, 63, 84, 112, 136, 146, 148, 166, 175, 183, 185, 189, 213, 217, 223, 228, 233, 236, 242, 243, 246, 247, 249, 255, 257, 264, 269, 270, 274, 275, 277, 279, 280, 281, 287, 334, 335, 337, 348, 350, 356, 357, 358, 361, 362, 368, 371, 372, 373, 380, 387, 388, 389, 394, 395, 396, 397, 398, 399, 400, 406, 407, 408, 415, 416, 419, 420, 421, 423, 427, 434, 435, 436, 449, 451, 462, 474], "issu": [5, 9, 14, 26, 35, 36, 37, 41, 45, 47, 54, 64, 70, 76, 132, 138, 159, 162, 167, 169, 177, 178, 179, 180, 182, 183, 186, 190, 192, 194, 196, 197, 198, 203, 206, 209, 212, 213, 217, 218, 220, 223, 226, 227, 228, 229, 231, 232, 233, 235, 236, 237, 238, 239, 241, 242, 243, 244, 247, 248, 249, 251, 252, 253, 255, 257, 260, 261, 267, 268, 272, 273, 274, 275, 279, 287, 288, 289, 294, 298, 301, 307, 311, 312, 313, 315, 316, 317, 324, 330, 335, 350, 353, 354, 355, 357, 368, 373, 388, 392, 395, 396, 400, 450, 467, 468, 470, 475, 505], "standard": [5, 11, 13, 30, 32, 41, 45, 47, 51, 53, 56, 57, 59, 60, 73, 79, 80, 93, 99, 100, 101, 104, 105, 107, 108, 112, 118, 122, 124, 126, 132, 136, 139, 140, 142, 144, 151, 152, 172, 188, 207, 213, 217, 226, 232, 234, 249, 254, 255, 257, 261, 265, 270, 271, 283, 293, 315, 317, 331, 343, 351, 353, 356, 357, 361, 362, 386, 396, 404, 408, 447, 464, 465, 468, 471, 476, 520], "interpret": [5, 32, 45, 57, 59, 71, 73, 97, 101, 112, 118, 124, 136, 204, 209, 260, 262, 297, 317, 373, 381, 438, 520], "weight": [5, 13, 41, 53, 54, 99, 120, 127, 262, 344], "account": [5, 25, 29, 36, 37, 41, 44, 47, 48, 50, 51, 52, 58, 59, 94, 109, 123, 160, 162, 181, 183, 186, 188, 201, 206, 219, 234, 240, 243, 245, 262, 269, 271, 273, 274, 277, 278, 282, 293, 300, 327, 333, 353, 356, 358, 359, 372, 392, 401, 432, 433, 434, 450, 452, 511, 520, 542], "produc": [5, 13, 14, 22, 32, 38, 41, 45, 54, 57, 58, 59, 73, 84, 94, 96, 97, 99, 101, 104, 105, 107, 109, 112, 131, 136, 146, 151, 152, 162, 163, 166, 170, 178, 182, 186, 194, 198, 205, 210, 212, 217, 218, 219, 220, 223, 227, 232, 235, 240, 242, 245, 254, 264, 267, 272, 273, 280, 287, 293, 307, 315, 316, 334, 350, 355, 361, 368, 374, 381, 383, 384, 389, 391, 401, 415, 420, 429, 433, 441, 509, 510, 512, 520], "denot": [5, 510], "lighter": 5, "heavier": 5, "accordingli": [5, 512], "fault_tolerant_execution_standard_split_s": 5, "64mb": [5, 41], "serv": [5, 45, 48, 78, 151, 266, 281, 462], "protect": [5, 25, 52, 72, 73, 464, 465, 471], "against": [5, 23, 24, 35, 39, 40, 42, 43, 45, 51, 55, 57, 58, 62, 66, 68, 69, 71, 72, 74, 75, 76, 84, 112, 132, 136, 137, 138, 143, 151, 245, 265, 270, 297, 302, 307, 317, 346, 354, 379, 386, 438, 459, 462, 468, 469, 470, 472, 476], "situat": [5, 37, 39, 41, 42, 43, 45, 52, 62, 65, 66, 68, 71, 74, 76, 157, 413, 450, 466], "incorrect": [5, 14, 65, 66, 68, 124, 156, 162, 166, 176, 177, 179, 183, 190, 194, 198, 199, 202, 205, 206, 207, 208, 209, 216, 217, 218, 219, 223, 228, 232, 237, 244, 245, 250, 254, 257, 258, 260, 263, 265, 266, 273, 274, 280, 289, 303, 307, 315, 316, 320, 323, 330, 333, 334, 336, 338, 339, 340, 341, 342, 346, 347, 348, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, 368, 369, 371, 372, 373, 375, 377, 379, 380, 381, 382, 383, 384, 386, 387, 388, 389, 390, 391, 392, 393, 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, 408, 410, 411, 413, 414, 415, 416, 417, 418, 419, 420, 423, 424, 427, 428, 429, 430, 432, 433, 434, 437, 438, 439, 440, 441, 445, 446, 447, 448, 449, 450, 451, 452, 453, 455, 457, 472, 510], "fault_tolerant_execution_max_task_split_count": 5, "256": [5, 45, 62, 270, 320, 470], "arbitrari": [5, 32, 39, 45, 57, 59, 73, 83, 84, 94, 99, 112, 114, 115, 126, 167, 173, 212, 239, 254, 274, 302, 315, 328, 331, 375, 385, 406, 444, 462, 464, 511, 520], "comput": [5, 10, 13, 14, 23, 35, 37, 41, 45, 88, 99, 101, 110, 112, 118, 131, 137, 144, 146, 147, 201, 206, 219, 220, 233, 236, 260, 271, 290, 293, 328, 338, 340, 364, 368, 369, 370, 373, 374, 379, 380, 381, 382, 384, 386, 391, 400, 401, 402, 403, 404, 425, 436, 439, 486, 510, 512, 520], "target": [5, 13, 19, 40, 41, 43, 45, 54, 55, 57, 62, 66, 68, 69, 70, 71, 74, 75, 76, 93, 119, 151, 162, 201, 208, 246, 255, 257, 258, 267, 352, 357, 377, 382, 383, 387, 389, 391, 398, 410, 413, 453, 462, 511], "growth": [5, 520], "period": [5, 6, 11, 15, 18, 23, 40, 41, 42, 51, 54, 55, 56, 62, 66, 68, 69, 71, 74, 75, 76, 209, 257, 260, 279, 345, 425, 427, 438, 462, 463, 464, 465, 467, 472, 486, 521], "writer": [5, 8, 21, 24, 41, 45, 54, 67, 133, 156, 194, 209, 217, 233, 245, 247, 250, 256, 257, 261, 262, 263, 264, 266, 267, 268, 269, 270, 273, 275, 276, 277, 278, 279, 317, 327, 331, 332, 350, 357, 364, 366, 379, 380, 389, 391, 407, 412, 413, 414, 415, 416, 418, 422, 423, 424, 428, 430, 434, 436, 438, 445, 446, 447, 449, 450, 451, 452, 454, 455], "64": [5, 19, 45, 56, 65, 101, 102, 136, 144, 153, 392, 403], "adapt": [5, 57, 84, 151, 233, 434, 468], "everi": [5, 23, 24, 25, 35, 40, 41, 44, 45, 48, 52, 54, 56, 59, 62, 66, 68, 71, 72, 74, 75, 79, 80, 84, 93, 99, 112, 114, 115, 121, 126, 136, 151, 154, 179, 217, 239, 242, 254, 284, 293, 302, 317, 462, 466, 470, 475, 491, 510, 512, 520], "old": [5, 12, 41, 44, 45, 54, 64, 84, 132, 154, 170, 189, 208, 213, 221, 245, 251, 261, 266, 276, 278, 301, 327, 332, 339, 342, 361, 368, 378, 389, 397, 399, 414, 415, 435, 450, 462], "multipli": [5, 246], "26": [5, 55, 57, 102, 107, 153, 262, 509], "512mb": 5, "50gb": 5, "write": [5, 8, 12, 19, 22, 25, 29, 35, 36, 37, 38, 39, 40, 41, 42, 45, 46, 51, 52, 53, 54, 55, 57, 60, 62, 63, 64, 66, 67, 68, 69, 71, 74, 75, 76, 84, 86, 87, 88, 94, 99, 124, 133, 136, 138, 140, 156, 162, 163, 180, 183, 186, 191, 196, 197, 207, 211, 212, 213, 218, 227, 232, 236, 240, 245, 246, 255, 256, 258, 261, 262, 263, 264, 266, 267, 269, 273, 274, 275, 276, 277, 278, 279, 281, 288, 293, 297, 301, 303, 305, 310, 327, 328, 331, 332, 334, 335, 336, 339, 341, 342, 343, 344, 345, 346, 348, 350, 355, 356, 357, 358, 360, 364, 365, 367, 368, 369, 370, 373, 375, 379, 380, 381, 386, 387, 388, 389, 390, 391, 392, 394, 396, 397, 398, 399, 400, 402, 406, 412, 413, 416, 418, 419, 420, 423, 426, 428, 432, 433, 434, 435, 436, 438, 443, 445, 446, 448, 449, 450, 452, 453, 454, 460, 462, 467, 514, 540], "4gb": 5, "soft": [5, 23, 45, 136], "due": [5, 9, 10, 13, 19, 22, 40, 42, 45, 55, 57, 58, 62, 66, 68, 69, 71, 74, 75, 76, 84, 112, 123, 138, 156, 157, 158, 178, 180, 197, 211, 217, 227, 231, 233, 235, 237, 242, 243, 245, 246, 248, 251, 252, 256, 257, 258, 260, 261, 264, 270, 274, 278, 283, 286, 287, 293, 317, 337, 345, 346, 360, 362, 368, 371, 379, 380, 381, 399, 401, 409, 411, 427, 433, 434, 435, 441, 450, 454, 520], "restart": [5, 20, 33, 35, 38, 40, 42, 55, 57, 58, 62, 63, 64, 66, 68, 69, 71, 73, 74, 75, 76, 93, 136, 137, 138, 139, 187, 462, 466, 471], "full": [5, 15, 27, 29, 32, 33, 35, 37, 39, 41, 42, 45, 48, 54, 62, 65, 66, 68, 70, 71, 74, 76, 88, 137, 142, 146, 151, 155, 156, 166, 173, 179, 186, 243, 260, 262, 278, 281, 282, 293, 323, 331, 335, 347, 352, 358, 381, 383, 411, 416, 421, 423, 432, 434, 448, 462, 468, 484, 520, 521], "bin": [5, 32, 36, 58, 118, 135, 136, 469], "pack": [5, 35, 57, 84], "fault_tolerant_execution_task_memori": 5, "5gb": [5, 14], "extra": [5, 25, 32, 33, 35, 39, 40, 42, 45, 51, 58, 60, 62, 66, 68, 71, 74, 76, 83, 226, 265, 274, 329, 332, 350, 357, 359, 366, 379, 399, 438, 466, 527], "abl": [5, 16, 18, 27, 40, 41, 42, 43, 45, 47, 51, 55, 57, 59, 62, 66, 68, 69, 71, 74, 75, 76, 84, 89, 137, 257, 293, 425, 466, 468, 469, 472, 524], "jvm": [5, 17, 18, 25, 30, 32, 33, 37, 41, 45, 51, 52, 55, 56, 66, 67, 132, 137, 139, 151, 218, 253, 262, 268, 274, 276, 278, 281, 303, 334, 336, 339, 356, 357, 366, 368, 451, 466, 468, 469, 475], "heap": [5, 7, 19, 41, 136, 137, 209, 210, 222, 223, 249, 320, 321], "15": [5, 11, 12, 47, 50, 55, 57, 58, 68, 70, 72, 107, 113, 117, 144, 153, 177, 367, 414, 473, 505, 506], "similar": [5, 13, 25, 29, 39, 40, 42, 45, 48, 54, 55, 62, 66, 68, 69, 71, 74, 75, 76, 84, 86, 88, 93, 96, 110, 112, 118, 119, 123, 124, 126, 131, 138, 144, 151, 227, 244, 250, 375, 381, 464, 467, 468, 472, 473, 512], "function": [5, 8, 13, 15, 18, 29, 32, 40, 45, 46, 54, 55, 57, 58, 64, 70, 75, 81, 84, 85, 86, 87, 93, 97, 105, 113, 122, 125, 132, 133, 136, 140, 142, 143, 144, 148, 154, 155, 158, 159, 162, 163, 164, 165, 167, 169, 170, 173, 180, 184, 186, 191, 194, 196, 197, 198, 203, 205, 208, 210, 212, 213, 218, 220, 221, 227, 228, 229, 232, 233, 236, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 253, 254, 255, 257, 258, 259, 261, 262, 263, 264, 265, 267, 268, 269, 270, 271, 273, 274, 276, 277, 278, 279, 280, 284, 288, 293, 294, 300, 301, 303, 304, 305, 306, 307, 308, 309, 313, 315, 324, 325, 327, 328, 329, 330, 331, 332, 333, 334, 336, 338, 339, 341, 342, 343, 344, 345, 346, 347, 352, 355, 356, 357, 358, 359, 360, 361, 362, 364, 366, 368, 369, 370, 371, 372, 373, 374, 375, 379, 380, 383, 384, 385, 386, 388, 392, 398, 399, 401, 402, 403, 405, 408, 409, 410, 411, 412, 414, 416, 418, 421, 422, 423, 425, 426, 427, 428, 429, 430, 432, 434, 436, 437, 438, 439, 440, 441, 442, 445, 446, 447, 448, 450, 453, 456, 457, 466, 470, 477, 491, 505, 506, 512, 520, 521, 525], "50": [5, 15, 23, 45, 56, 65, 70, 83, 99, 100, 255, 302, 506, 520], "instabl": [5, 15, 25, 41, 45, 229, 384], "poor": [5, 185], "fault_tolerant_execution_max_partition_count": 5, "fault_tolerant_execution_min_partition_count": 5, "fault_tolerant_execution_min_partition_count_for_writ": 5, "paus": [5, 18, 41, 45, 155, 241, 260, 273, 278, 360], "5": [5, 7, 8, 14, 16, 23, 29, 38, 39, 41, 51, 57, 60, 64, 66, 67, 68, 69, 78, 84, 97, 99, 100, 102, 103, 105, 106, 107, 108, 109, 112, 113, 119, 121, 123, 144, 148, 153, 164, 263, 302, 307, 327, 335, 351, 353, 354, 383, 387, 388, 449, 506, 510, 520], "locat": [5, 13, 14, 15, 32, 33, 37, 40, 41, 42, 45, 46, 47, 48, 49, 50, 51, 52, 54, 58, 61, 62, 64, 66, 68, 69, 70, 71, 73, 74, 76, 78, 109, 136, 138, 139, 177, 180, 196, 217, 232, 233, 273, 278, 310, 311, 328, 341, 342, 343, 353, 356, 357, 361, 366, 367, 378, 383, 388, 394, 395, 396, 405, 410, 412, 413, 415, 419, 422, 427, 432, 439, 446, 447, 450, 452, 456, 462, 464, 468, 469, 475, 476, 488, 521], "compat": [5, 32, 41, 45, 48, 51, 54, 57, 59, 64, 69, 73, 88, 107, 112, 124, 126, 128, 132, 144, 151, 162, 213, 217, 226, 239, 251, 261, 264, 268, 274, 275, 278, 332, 339, 350, 353, 354, 362, 364, 365, 368, 378, 392, 393, 398, 406, 426, 430, 432, 438, 462, 505], "solut": [5, 7, 49, 64, 475], "directori": [5, 12, 18, 23, 24, 32, 33, 35, 45, 47, 48, 51, 52, 54, 57, 59, 61, 64, 73, 83, 86, 87, 88, 93, 135, 136, 138, 151, 155, 207, 210, 217, 285, 328, 329, 335, 348, 357, 359, 365, 368, 373, 378, 379, 391, 399, 415, 439, 447, 449, 452, 459, 475, 476], "sink": [5, 84, 91, 372], "pool": [5, 7, 56, 58, 65, 68, 249, 258, 266, 277, 278, 329, 338, 351, 361, 362, 365, 366, 367, 368, 395, 399], "written": [5, 18, 21, 25, 38, 39, 41, 43, 44, 45, 47, 51, 54, 61, 70, 78, 94, 113, 136, 144, 151, 155, 159, 167, 173, 188, 194, 212, 213, 217, 220, 248, 251, 255, 256, 257, 258, 261, 267, 270, 272, 273, 274, 275, 278, 282, 283, 287, 294, 296, 302, 316, 317, 325, 327, 330, 331, 332, 336, 339, 342, 346, 350, 354, 366, 375, 381, 389, 391, 405, 414, 416, 422, 423, 433, 434, 435, 436, 438, 444, 451], "1gb": [5, 11, 12, 23, 24, 41, 45, 54, 398], "concurr": [5, 14, 22, 23, 25, 36, 37, 41, 64, 146, 154, 157, 163, 208, 212, 216, 217, 220, 270, 278, 307, 321, 354, 356, 371, 389, 391, 392, 399, 400, 402, 406, 415, 418, 422, 427, 434, 435, 466], "gc": [5, 26, 41, 50, 121, 136, 241, 242, 243, 244, 246, 248, 249, 251, 253, 260, 261, 262, 263, 309, 311, 329, 332, 345, 346, 360], "ignor": [5, 32, 38, 40, 41, 42, 43, 45, 51, 52, 54, 57, 62, 66, 68, 69, 71, 74, 75, 76, 78, 79, 80, 84, 93, 94, 96, 99, 100, 128, 131, 141, 144, 156, 162, 183, 196, 210, 212, 238, 245, 250, 251, 254, 255, 258, 259, 260, 265, 270, 277, 287, 297, 317, 320, 324, 328, 331, 337, 341, 346, 357, 359, 364, 366, 368, 373, 381, 390, 396, 410, 420, 429, 436, 437, 453, 469, 470, 473, 511], "secret": [5, 18, 25, 33, 35, 40, 42, 43, 47, 49, 51, 55, 58, 62, 64, 66, 68, 70, 71, 74, 75, 76, 84, 133, 137, 138, 139, 284, 355, 357, 368, 371, 398, 405, 458, 459, 461, 467, 468, 469, 470, 471, 472, 473], "iam": [5, 43, 51, 59, 64, 359, 387, 410], "role": [5, 33, 37, 43, 47, 51, 52, 59, 64, 83, 84, 133, 143, 151, 328, 337, 343, 357, 359, 362, 368, 387, 389, 390, 392, 398, 403, 410, 430, 447, 456, 462, 477, 479, 480, 481, 488, 494, 507, 517], "assum": [5, 13, 14, 25, 32, 43, 51, 54, 57, 58, 59, 64, 73, 76, 78, 88, 107, 108, 109, 118, 124, 156, 278, 317, 337, 387, 462, 470, 510], "id": [5, 15, 23, 27, 35, 37, 40, 41, 43, 45, 47, 51, 54, 55, 57, 58, 59, 64, 66, 67, 78, 83, 94, 99, 110, 112, 123, 126, 135, 136, 139, 148, 217, 262, 263, 268, 271, 277, 281, 284, 307, 358, 359, 360, 379, 381, 388, 392, 418, 419, 423, 432, 442, 445, 446, 447, 453, 470, 480, 483, 505, 525, 542, 544], "trust": [5, 6, 32, 33, 39, 40, 43, 51, 64, 65, 66, 71, 74, 75, 132, 359, 360, 384, 403, 408, 461, 464, 466, 470, 471, 473, 475], "region": [5, 13, 32, 43, 48, 51, 58, 64, 65, 99, 104, 112, 126, 138, 173, 183, 258, 414, 422, 426, 476, 520, 525], "bucket": [5, 41, 45, 49, 51, 54, 59, 60, 64, 69, 94, 99, 110, 118, 131, 138, 177, 198, 199, 206, 208, 211, 218, 221, 223, 229, 245, 255, 256, 258, 264, 267, 268, 269, 270, 271, 273, 274, 295, 327, 331, 332, 334, 339, 344, 345, 348, 356, 357, 358, 359, 360, 361, 364, 365, 366, 368, 370, 373, 375, 380, 381, 382, 387, 388, 389, 391, 398, 408, 409, 418, 420, 423, 438, 439, 447, 448], "endpoint": [5, 6, 19, 37, 43, 47, 49, 51, 59, 64, 70, 83, 211, 217, 245, 278, 346, 359, 365, 375, 383, 390, 401, 414, 426, 441, 462, 466, 467, 469, 470], "googleapi": 5, "com": [5, 13, 15, 30, 32, 49, 51, 52, 58, 59, 60, 64, 66, 78, 84, 93, 281, 327, 385, 397, 428, 464, 465, 468, 469, 470, 471, 472, 475, 476], "path": [5, 23, 24, 25, 32, 33, 36, 37, 39, 40, 41, 42, 43, 44, 45, 47, 48, 49, 50, 51, 52, 54, 55, 57, 59, 60, 62, 64, 66, 68, 69, 70, 71, 72, 74, 75, 76, 84, 96, 109, 129, 133, 135, 136, 139, 157, 171, 208, 213, 217, 233, 248, 251, 274, 282, 302, 328, 329, 339, 349, 359, 360, 388, 403, 404, 405, 412, 413, 417, 419, 428, 432, 438, 440, 441, 446, 450, 456, 459, 461, 463, 464, 466, 467, 468, 469, 472, 475, 476, 477, 510, 532], "style": [5, 29, 51, 103, 251, 428], "upload": [5, 51, 279, 375, 393], "multi": [5, 42, 45, 51, 55, 60, 109, 121, 227, 278, 347, 355, 357, 397, 430, 454], "5mb": [5, 446], "platform": [5, 31, 37, 46, 134, 135, 137], "Not": [5, 25, 35, 71, 73, 104, 282, 431, 465, 520], "togeth": [5, 10, 23, 39, 59, 84, 88, 94, 104, 109, 144, 146, 151, 334, 360, 380, 388, 397], "4mb": [5, 18, 65], "site": [5, 30, 45, 47, 49, 52, 64, 69, 461, 464, 465, 467, 475], "xml": [5, 45, 47, 49, 52, 64, 69, 93], "exist": [5, 13, 15, 23, 29, 32, 33, 35, 37, 41, 44, 45, 47, 48, 50, 54, 57, 59, 60, 65, 66, 68, 69, 73, 78, 79, 80, 83, 94, 99, 105, 111, 112, 119, 124, 135, 142, 155, 165, 167, 180, 182, 196, 198, 208, 212, 221, 226, 233, 246, 254, 262, 267, 268, 273, 278, 281, 287, 293, 296, 303, 304, 331, 342, 345, 346, 357, 359, 367, 368, 369, 372, 379, 386, 391, 392, 393, 394, 396, 397, 399, 403, 405, 406, 410, 413, 418, 419, 430, 432, 433, 434, 435, 436, 438, 452, 461, 462, 464, 468, 471, 478, 479, 480, 481, 486, 488, 489, 490, 491, 498, 500, 501, 502, 505, 521, 542], "i": [5, 14, 19, 23, 27, 35, 36, 45, 48, 57, 60, 65, 69, 84, 96, 100, 107, 112, 121, 124, 136, 138, 139, 223, 242, 282, 307, 334, 339, 520], "o": [5, 19, 27, 35, 36, 45, 48, 58, 65, 99, 137, 138, 148, 240, 245, 282, 476, 486, 520], "lifecycl": [5, 217], "rule": [5, 6, 27, 51, 57, 59, 65, 68, 73, 77, 84, 89, 104, 108, 169, 210, 226, 261, 266, 268, 269, 293, 333, 357, 358, 362, 363, 364, 371, 384, 392, 398, 408, 432, 439, 441, 447, 456, 460, 461, 467, 468, 469, 470, 471, 510, 520], "expir": [5, 27, 32, 35, 37, 45, 51, 54, 64, 70, 187, 210, 218, 253, 257, 260, 310, 313, 386, 400, 432, 434, 470, 473], "abandon": [5, 15, 253, 293, 296, 392], "object": [5, 7, 12, 19, 32, 35, 40, 41, 42, 45, 46, 50, 51, 53, 54, 55, 56, 57, 59, 62, 64, 65, 66, 68, 69, 71, 73, 74, 75, 76, 83, 84, 86, 88, 89, 91, 93, 94, 96, 97, 109, 112, 132, 137, 138, 142, 144, 155, 157, 158, 166, 167, 193, 224, 244, 261, 262, 263, 270, 274, 287, 320, 331, 345, 355, 360, 366, 367, 368, 375, 377, 384, 388, 408, 428, 462, 469, 484, 488, 500, 505], "crash": [5, 178, 180, 248, 249, 270, 434], "destin": [5, 15, 76, 413], "note": [5, 15, 25, 35, 37, 39, 40, 41, 42, 43, 44, 45, 52, 54, 55, 57, 60, 62, 65, 66, 68, 69, 71, 73, 74, 75, 76, 84, 86, 88, 94, 96, 104, 112, 124, 126, 133, 137, 139, 148, 161, 178, 254, 282, 305, 328, 367, 371, 388, 393, 462, 465, 466, 469, 486, 506, 520, 538], "west": [5, 138], "across": [5, 9, 10, 13, 14, 17, 21, 33, 43, 45, 55, 56, 57, 68, 71, 74, 75, 76, 78, 95, 131, 132, 136, 137, 146, 151, 242, 258, 282, 287, 302, 359, 372, 389, 462, 486, 505], "softwar": [5, 30, 41, 51, 56, 83, 132], "amazon": [5, 30, 41, 45, 48, 59, 74, 132, 137, 298, 303, 348, 417, 419], "awssdk": 5, "model": [5, 29, 54, 68, 109, 119, 132, 280, 296], "s3except": 5, "pleas": [5, 25, 33, 35, 37, 45, 65, 70, 73, 264, 296, 297, 305, 335, 371, 472], "rate": [5, 7, 38, 53, 77, 242, 341, 358, 367, 368, 383], "indic": [5, 23, 43, 45, 65, 67, 84, 88, 94, 96, 109, 126, 137, 147, 197, 217, 223, 242, 268, 462, 476, 497, 520], "intens": [5, 16, 25, 41, 466, 475, 514], "balanc": [5, 13, 39, 40, 41, 42, 55, 62, 66, 68, 69, 71, 74, 75, 76, 83, 132, 138, 209, 302, 396, 471, 520], "hierarch": [5, 169], "option": [5, 12, 13, 23, 24, 29, 35, 37, 39, 40, 41, 42, 43, 44, 45, 47, 49, 51, 52, 54, 55, 57, 59, 60, 62, 64, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 84, 86, 88, 94, 95, 96, 100, 104, 107, 112, 126, 136, 137, 138, 144, 155, 157, 161, 162, 163, 170, 173, 180, 196, 197, 208, 212, 217, 220, 226, 227, 228, 230, 232, 233, 235, 236, 239, 240, 242, 245, 246, 247, 257, 259, 261, 262, 263, 264, 265, 266, 268, 270, 272, 274, 275, 276, 278, 297, 301, 302, 305, 306, 307, 317, 328, 329, 330, 332, 333, 335, 336, 341, 342, 345, 346, 348, 357, 358, 362, 366, 379, 380, 384, 386, 391, 392, 396, 403, 408, 420, 433, 435, 441, 450, 451, 452, 453, 455, 457, 461, 462, 464, 466, 468, 469, 473, 475, 476, 478, 480, 482, 486, 487, 488, 489, 490, 491, 498, 499, 500, 501, 502, 505, 506, 507, 508, 510, 512, 517, 518, 520, 522, 525, 526, 527, 532, 536, 537, 539], "abf": [5, 357, 369], "container_nam": 5, "account_nam": [5, 47], "df": [5, 45, 47, 210, 374], "core": [5, 45, 47, 49, 52, 86, 93, 136, 143, 151, 209, 302, 392], "window": [5, 15, 18, 32, 33, 45, 47, 71, 72, 98, 99, 133, 155, 159, 162, 164, 167, 180, 196, 203, 209, 213, 218, 227, 232, 233, 239, 242, 247, 250, 255, 280, 294, 297, 300, 315, 324, 327, 330, 331, 333, 334, 346, 347, 362, 366, 373, 379, 380, 384, 386, 388, 392, 396, 410, 426, 430, 432, 434, 440, 448, 450, 457, 466, 506, 510], "hmac": [5, 467, 470], "deploi": [5, 58, 70, 132, 133, 134, 135, 137, 138, 139, 155, 159, 188, 194, 212, 213, 217, 220, 251, 255, 270, 275, 282, 283, 284, 287, 294, 302, 316, 317, 325, 327, 334, 336, 342, 350], "gcp": [5, 30], "migrat": [5, 10, 28, 45, 47, 133, 171, 176, 226, 254, 272, 282, 392, 438, 439, 441, 446, 448, 450], "gs": [5, 45, 50], "gcs_keyfil": 5, "hadoop": [5, 30, 45, 47, 49, 51, 54, 64, 67, 69, 109, 132, 152, 213, 219, 232, 278, 281, 287, 288, 343, 346, 360, 436, 449], "master": [5, 41, 54, 60, 78, 484], "9000": [5, 54], "usr": [5, 36, 45, 54, 57, 59, 73, 135, 139, 379, 475], "lib": [5, 35, 48, 135, 139, 341, 379], "tmp": [5, 45, 64, 136, 328, 469], "standalon": [5, 112, 287], "product": [5, 12, 30, 68, 136, 139, 247, 296, 391, 469, 475, 520], "share": [5, 9, 14, 23, 33, 44, 45, 60, 109, 136, 180, 213, 246, 263, 320, 357, 386, 401, 405, 459, 461, 464, 467, 468, 469, 470, 471, 472, 473, 476], "api": [6, 32, 33, 35, 51, 58, 64, 72, 76, 78, 81, 84, 87, 132, 133, 151, 155, 167, 173, 188, 191, 204, 209, 210, 219, 233, 279, 282, 285, 288, 302, 333, 345, 356, 358, 364, 379, 385, 389, 396, 432, 446, 468, 471, 473], "exclus": [6, 41, 57, 59, 71, 73, 112, 118, 475, 486], "termin": [6, 15, 32, 39, 83, 121, 136, 137, 173, 206, 260, 265, 270, 315, 435, 438, 471], "suffici": [6, 52, 88, 136, 278], "invok": [6, 32, 45, 85, 96, 99, 100, 121, 126, 131, 170, 249, 328, 362, 375, 398, 407, 415, 424, 445, 469, 491], "put": [6, 35, 43, 58, 84, 93, 152, 182], "curl": [6, 43, 58], "v": [6, 54, 99, 107, 117, 118, 123, 136, 464, 510], "x": [6, 10, 29, 32, 35, 45, 54, 57, 58, 59, 60, 64, 68, 69, 71, 73, 83, 84, 99, 100, 102, 103, 107, 108, 109, 110, 112, 113, 117, 118, 120, 121, 123, 127, 131, 132, 136, 144, 170, 180, 217, 227, 232, 250, 261, 264, 265, 267, 268, 274, 277, 281, 293, 327, 335, 344, 346, 351, 353, 354, 358, 360, 366, 378, 384, 385, 391, 430, 461, 469, 475, 478, 480, 520], "d": [6, 8, 32, 35, 99, 100, 107, 112, 117, 118, 121, 135, 139, 173, 261, 264, 277, 510, 520], "shutting_down": [6, 182], "h": [6, 8, 32, 80, 83, 107, 151, 287], "applic": [6, 7, 8, 12, 15, 23, 25, 31, 32, 33, 42, 43, 47, 53, 54, 96, 109, 132, 134, 135, 137, 139, 382, 413, 433, 462, 467, 470, 471, 510], "8081": [6, 57], "v1": [6, 19, 47, 54, 64, 72, 83, 112, 117, 123, 182, 198, 226, 245, 278, 281, 330, 346, 378, 399, 407, 462], "info": [6, 32, 33, 58, 65, 83, 136, 137, 161, 175, 182, 227, 259, 330, 367, 383, 389, 462], "invoc": [6, 32, 77, 84, 96, 233, 399], "level": [6, 15, 17, 25, 26, 32, 39, 40, 41, 42, 43, 45, 46, 49, 52, 55, 60, 62, 66, 68, 69, 71, 74, 75, 76, 93, 94, 96, 97, 109, 112, 132, 144, 146, 148, 151, 169, 173, 232, 246, 267, 270, 272, 273, 288, 302, 327, 357, 367, 380, 387, 396, 399, 460, 461, 468, 469, 470, 471, 475, 506, 520, 540, 544], "aspect": [6, 98, 112, 132, 136, 146, 462, 473, 477], "secur": [6, 27, 32, 33, 36, 37, 39, 42, 48, 62, 64, 68, 95, 132, 133, 136, 140, 172, 206, 207, 213, 244, 245, 274, 275, 339, 345, 375, 380, 393, 407, 426, 437, 462, 467, 468, 470, 472], "basic": [6, 32, 42, 48, 51, 57, 70, 72, 78, 84, 124, 156, 165, 267, 302, 357, 396, 409, 459, 469, 473, 510], "author": [6, 24, 33, 35, 39, 40, 45, 51, 71, 72, 74, 82, 83, 84, 90, 92, 95, 112, 126, 132, 133, 178, 207, 217, 254, 258, 331, 335, 343, 357, 358, 359, 371, 373, 374, 375, 379, 383, 389, 394, 398, 402, 430, 447, 453, 454, 457, 460, 461, 464, 465, 470, 475, 477, 479, 480, 481, 488, 533], "header": [6, 32, 43, 44, 45, 57, 72, 78, 81, 93, 133, 187, 220, 258, 263, 264, 267, 271, 274, 293, 341, 344, 346, 348, 356, 361, 367, 369, 378, 399, 438, 449, 469, 475, 476], "satisfi": [6, 112, 148, 462, 512, 520], "whatev": [6, 84, 94, 137], "tl": [6, 33, 37, 39, 40, 66, 70, 71, 74, 75, 76, 82, 90, 132, 133, 245, 254, 260, 276, 348, 360, 367, 383, 384, 387, 392, 399, 403, 408, 419, 421, 432, 458, 459, 461, 464, 467, 470, 471, 472, 473], "certif": [6, 11, 20, 27, 33, 37, 40, 43, 64, 66, 71, 74, 75, 76, 81, 93, 132, 133, 254, 260, 346, 360, 361, 368, 379, 384, 387, 392, 399, 403, 408, 458, 459, 464, 466, 467, 469, 471, 476], "ca": [6, 33, 132, 254, 461, 464, 475, 482], "sign": [6, 57, 59, 73, 102, 114, 115, 118, 121, 132, 144, 148, 178, 204, 208, 209, 217, 253, 254, 316, 333, 464, 467, 470, 476], "call": [6, 19, 25, 29, 40, 41, 42, 45, 51, 54, 55, 58, 59, 60, 62, 64, 66, 68, 69, 70, 71, 74, 75, 76, 77, 78, 84, 86, 87, 88, 91, 93, 94, 96, 109, 110, 112, 120, 123, 126, 127, 133, 136, 137, 143, 146, 148, 152, 159, 162, 187, 201, 206, 211, 246, 262, 263, 268, 279, 288, 293, 296, 333, 342, 345, 347, 353, 354, 355, 357, 368, 380, 382, 387, 389, 402, 413, 416, 427, 438, 459, 477, 505, 510, 512, 520, 525], "shut": [6, 136, 139, 358], "insecur": [6, 32, 346, 364, 469, 475, 476], "isn": [6, 84, 94, 137], "present": [6, 23, 33, 37, 40, 41, 42, 45, 57, 59, 62, 66, 68, 71, 73, 74, 76, 83, 84, 94, 117, 132, 157, 198, 228, 229, 242, 262, 276, 346, 357, 358, 406, 433, 442, 447, 450, 462, 509, 510, 520], "onc": [6, 14, 15, 23, 33, 35, 41, 43, 47, 50, 57, 65, 69, 84, 113, 134, 135, 137, 146, 173, 182, 194, 253, 254, 267, 272, 379, 390, 395, 432, 460, 462, 486, 520, 523], "go": [6, 37, 51, 84, 112, 126, 139, 304, 461, 468, 471], "sleep": [6, 58], "minut": [6, 8, 51, 54, 59, 64, 68, 83, 107, 114, 128, 144, 155, 297, 368, 470, 525], "awar": [6, 39, 121, 177, 255, 311, 317, 342, 348, 384, 385, 399, 400, 403, 407, 417, 466, 514], "stop": [6, 15, 32, 100, 126, 135, 136, 139, 209, 278, 287, 302, 399, 405, 421, 456, 466], "until": [6, 33, 37, 41, 45, 63, 83, 105, 141, 268, 278, 279, 369, 464, 469, 510, 512], "activ": [6, 7, 15, 19, 41, 48, 54, 64, 136, 151, 172, 217, 232, 247, 260, 263, 289, 333, 418, 419, 420, 438, 462, 466, 506], "expos": [7, 36, 37, 41, 45, 48, 53, 54, 55, 57, 59, 68, 70, 73, 78, 125, 143, 165, 188, 196, 208, 209, 242, 257, 259, 288, 292, 332, 341, 359, 367, 369, 371, 375, 379, 380, 386, 388, 391, 403, 417, 439, 442, 475], "differ": [7, 8, 9, 16, 23, 25, 29, 32, 33, 34, 35, 37, 39, 40, 41, 42, 43, 45, 47, 48, 51, 52, 54, 55, 56, 57, 58, 59, 60, 62, 63, 64, 65, 66, 68, 69, 71, 73, 74, 75, 76, 78, 84, 85, 87, 88, 93, 94, 96, 99, 108, 109, 112, 121, 124, 126, 131, 132, 134, 137, 143, 144, 146, 148, 151, 152, 170, 178, 201, 208, 218, 242, 250, 253, 254, 261, 264, 275, 293, 297, 303, 328, 331, 337, 339, 342, 350, 351, 355, 358, 366, 367, 373, 375, 380, 381, 383, 387, 388, 400, 404, 412, 420, 423, 429, 433, 437, 438, 447, 449, 459, 462, 464, 468, 470, 471, 475, 476, 480, 486, 510, 511, 512, 520, 523, 542], "java": [7, 23, 30, 32, 33, 35, 37, 39, 40, 42, 43, 45, 47, 51, 52, 55, 56, 57, 58, 59, 60, 62, 64, 66, 68, 71, 73, 74, 76, 83, 84, 88, 93, 97, 121, 132, 139, 167, 209, 210, 217, 232, 233, 264, 274, 321, 335, 339, 353, 355, 357, 360, 361, 367, 374, 375, 377, 378, 385, 386, 389, 392, 414, 416, 417, 457, 466, 475, 520], "extens": [7, 47, 56, 68, 104, 109, 132, 192, 293, 297, 303, 307, 465, 476, 510, 512], "rmi": 7, "registri": [7, 377, 380, 397], "rmiregistri": 7, "9080": 7, "rmiserv": 7, "9081": [7, 410, 412], "export": [7, 23, 32, 37, 137, 343, 344, 360, 373, 441, 474, 475], "dcom": [7, 469], "sun": [7, 107, 469], "jmxremot": 7, "jconsol": 7, "suppli": [7, 32, 45, 54, 57, 64, 77, 79, 80, 83, 94, 103, 104, 105, 109, 137, 227, 287, 470, 472, 505], "jdk": [7, 70, 136, 217, 314, 342, 357, 360, 374], "visualvm": 7, "tool": [7, 23, 31, 36, 40, 41, 42, 47, 48, 53, 55, 58, 66, 68, 69, 71, 74, 75, 76, 137, 140, 151, 152, 273, 279, 466, 470, 471, 475, 510, 521], "complex": [7, 14, 35, 43, 45, 47, 54, 59, 79, 80, 84, 88, 98, 113, 146, 148, 196, 197, 198, 210, 219, 236, 243, 245, 260, 268, 270, 274, 278, 287, 293, 316, 346, 347, 355, 357, 361, 362, 373, 374, 379, 384, 385, 387, 392, 393, 401, 405, 419, 426, 462, 468, 469, 471, 475, 476, 482, 510], "counterstat": 7, "collect": [7, 14, 26, 27, 36, 40, 41, 42, 45, 54, 55, 60, 62, 63, 66, 68, 69, 71, 74, 75, 76, 78, 83, 84, 91, 94, 96, 109, 112, 132, 136, 138, 156, 158, 196, 218, 267, 273, 277, 280, 293, 297, 307, 310, 328, 338, 343, 345, 348, 352, 353, 357, 359, 361, 362, 369, 373, 386, 387, 389, 392, 416, 425, 430, 432, 438, 439, 482, 521, 538], "relat": [7, 14, 23, 25, 32, 54, 77, 84, 96, 109, 132, 134, 137, 139, 143, 148, 151, 152, 155, 179, 258, 265, 279, 293, 303, 327, 329, 332, 347, 355, 418, 425, 432, 434, 436, 468, 469, 471, 520], "totalcount": 7, "oneminut": 7, "subset": [7, 24, 25, 32, 33, 35, 41, 45, 51, 54, 57, 100, 151, 228, 352, 444, 448, 510, 512, 526, 527, 532, 536, 537, 539], "describ": [7, 8, 13, 23, 24, 25, 32, 33, 34, 35, 37, 39, 40, 41, 42, 43, 44, 45, 48, 52, 54, 55, 57, 58, 59, 60, 62, 64, 65, 66, 68, 69, 70, 71, 73, 74, 75, 76, 78, 84, 96, 98, 107, 112, 119, 128, 133, 136, 138, 142, 143, 144, 146, 151, 155, 156, 217, 218, 221, 226, 281, 293, 297, 301, 336, 356, 364, 375, 425, 430, 438, 461, 464, 465, 466, 467, 469, 470, 475, 477, 505, 510, 512, 513], "below": [7, 14, 15, 23, 35, 41, 43, 45, 54, 57, 59, 60, 67, 68, 69, 73, 78, 83, 88, 97, 103, 108, 112, 136, 137, 144, 146, 258, 293, 400, 415, 455, 462, 467, 468, 469, 476, 506, 520, 524], "lang": [7, 32, 56, 58, 124], "heapmemoryusag": 7, "thread": [7, 14, 32, 45, 53, 58, 59, 64, 65, 136, 151, 154, 155, 157, 158, 173, 209, 217, 232, 236, 242, 249, 251, 278, 280, 291, 303, 321, 343, 360, 371, 386], "threadcount": 7, "failuredetector": 7, "heartbeatfailuredetector": 7, "activecount": [7, 19], "free": [7, 25, 94, 160, 258, 261, 318, 475], "clustermemorypool": 7, "freedistributedbyt": 7, "cumul": [7, 25, 131, 188, 280, 387, 400, 410, 452], "ran": [7, 41, 77], "were": [7, 14, 41, 45, 54, 57, 59, 73, 77, 78, 84, 94, 112, 157, 166, 173, 201, 204, 206, 209, 210, 213, 233, 249, 254, 258, 262, 263, 265, 268, 272, 275, 278, 282, 287, 289, 302, 303, 311, 317, 330, 339, 348, 350, 359, 361, 366, 367, 373, 375, 380, 381, 383, 390, 392, 397, 399, 420, 421, 441, 450, 469, 520], "clustermemorymanag": 7, "querieskilledduetooutofmemori": 7, "queu": [7, 13, 15, 23, 27, 39, 53, 77, 136, 163, 210, 217, 235, 245, 251, 269, 280, 287, 307, 341, 353, 364, 506], "querymanag": [7, 373], "runningqueri": 7, "startedqueri": 7, "fiveminut": 7, "last": [7, 23, 41, 45, 48, 51, 54, 58, 88, 94, 100, 107, 109, 112, 114, 115, 117, 124, 126, 131, 137, 148, 191, 217, 258, 350, 390, 486, 510, 512, 520], "failedqueri": 7, "intern": [7, 41, 52, 53, 54, 58, 87, 93, 94, 119, 123, 133, 136, 137, 144, 157, 167, 242, 245, 268, 270, 271, 273, 285, 329, 331, 339, 341, 353, 355, 357, 358, 359, 360, 364, 365, 371, 375, 383, 384, 387, 392, 398, 400, 405, 416, 428, 438, 458, 468, 471, 475, 506], "internalfailur": 7, "externalfailur": 7, "usererrorfailur": 7, "p50": [7, 506], "executiontim": [7, 354], "p90": [7, 506], "wallinputbytesr": 7, "byte": [7, 8, 35, 37, 41, 43, 55, 57, 58, 59, 65, 68, 70, 73, 77, 78, 83, 88, 94, 99, 101, 106, 129, 147, 183, 184, 196, 208, 239, 293, 295, 302, 313, 355, 366, 368, 374, 392, 402, 403, 405, 420, 449, 451, 454, 497, 506, 538], "sqltaskmanag": 7, "inputdatas": 7, "own": [7, 8, 27, 35, 37, 42, 43, 47, 51, 52, 62, 66, 68, 71, 74, 76, 83, 93, 112, 264, 334, 460, 462, 464, 467, 475, 483], "typic": [7, 12, 15, 27, 32, 33, 40, 41, 42, 45, 48, 53, 54, 55, 59, 62, 65, 66, 68, 71, 74, 75, 76, 94, 104, 124, 129, 132, 134, 136, 148, 192, 265, 293, 307, 354, 465, 466, 467, 468, 475, 476, 486, 510, 514], "correspond": [8, 14, 24, 27, 35, 37, 39, 40, 41, 42, 43, 44, 53, 54, 55, 57, 59, 60, 62, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 78, 79, 80, 86, 88, 89, 94, 99, 112, 117, 123, 124, 126, 147, 217, 254, 259, 261, 262, 274, 359, 375, 379, 392, 462, 470, 472, 510, 520, 523, 525], "alter": [8, 23, 24, 35, 52, 60, 62, 84, 95, 133, 142, 143, 173, 178, 217, 246, 265, 302, 320, 335, 354, 358, 367, 371, 373, 374, 389, 391, 396, 397, 402, 403, 407, 409, 422, 428, 432, 439, 447, 462, 477, 488, 489, 500, 501, 505], "page": [8, 15, 18, 25, 27, 35, 38, 41, 44, 63, 67, 70, 73, 76, 84, 91, 94, 96, 123, 144, 155, 158, 162, 183, 187, 189, 191, 193, 197, 198, 200, 206, 217, 219, 229, 242, 245, 248, 249, 253, 256, 259, 261, 263, 267, 269, 274, 278, 279, 280, 303, 307, 317, 329, 334, 344, 367, 373, 377, 386, 389, 390, 394, 403, 410, 442, 464, 465, 475], "exchang": [8, 14, 15, 64, 93, 133, 138, 200, 206, 210, 211, 228, 230, 235, 242, 249, 260, 261, 263, 268, 276, 280, 287, 330, 360, 408, 409, 410, 420, 428, 433, 438, 470, 505], "express": [8, 13, 19, 23, 24, 40, 45, 51, 55, 58, 62, 66, 68, 71, 74, 75, 76, 84, 94, 98, 99, 104, 106, 107, 109, 114, 116, 118, 119, 124, 126, 131, 133, 144, 148, 151, 165, 167, 173, 180, 191, 194, 196, 197, 201, 206, 217, 223, 225, 226, 227, 229, 231, 233, 236, 237, 239, 242, 244, 246, 248, 250, 251, 253, 254, 255, 258, 262, 270, 274, 279, 280, 282, 287, 293, 297, 305, 312, 313, 315, 317, 328, 331, 332, 334, 336, 337, 339, 343, 346, 350, 355, 357, 358, 360, 366, 367, 368, 369, 373, 374, 377, 379, 383, 384, 385, 386, 387, 388, 392, 397, 402, 403, 416, 418, 419, 421, 426, 429, 433, 436, 438, 439, 441, 443, 453, 454, 461, 462, 465, 467, 468, 471, 475, 476, 478, 480, 482, 483, 488, 489, 490, 497, 511, 512, 523, 525, 542, 544], "syntax": [8, 32, 37, 39, 40, 41, 42, 43, 44, 52, 54, 55, 57, 62, 65, 66, 68, 69, 71, 74, 75, 76, 79, 80, 96, 98, 104, 107, 108, 121, 124, 126, 129, 131, 133, 140, 144, 148, 161, 162, 169, 178, 208, 209, 270, 282, 283, 288, 297, 302, 307, 334, 337, 338, 342, 347, 359, 365, 367, 373, 374, 375, 382, 386, 391, 404, 414, 474, 505, 512, 520, 521, 524], "limit": [8, 10, 11, 12, 13, 14, 15, 17, 23, 24, 25, 30, 32, 37, 40, 41, 52, 53, 54, 55, 58, 59, 62, 64, 65, 66, 68, 70, 71, 72, 75, 76, 87, 119, 124, 136, 143, 152, 155, 157, 161, 166, 167, 169, 178, 183, 186, 190, 193, 197, 198, 213, 217, 218, 223, 224, 228, 229, 233, 234, 235, 239, 242, 244, 245, 248, 254, 255, 257, 258, 262, 264, 265, 266, 267, 268, 271, 273, 274, 276, 277, 278, 282, 284, 296, 297, 302, 307, 313, 315, 320, 321, 333, 334, 335, 337, 339, 340, 345, 346, 348, 356, 357, 366, 367, 368, 372, 373, 374, 375, 379, 380, 381, 382, 383, 385, 386, 387, 388, 389, 395, 403, 410, 412, 420, 426, 432, 433, 439, 444, 446, 451, 454, 460, 466, 471, 510, 512], "disallow": [8, 45, 57, 241, 269, 270, 296, 334, 344, 346, 359, 360, 382, 386, 394, 395, 396, 397, 398, 400, 402, 403, 413, 414, 420, 421, 432, 440, 441, 446, 447, 449, 450], "valid": [8, 32, 33, 35, 37, 42, 43, 45, 57, 59, 60, 62, 64, 66, 67, 68, 69, 71, 73, 74, 76, 84, 86, 90, 92, 96, 99, 109, 112, 124, 129, 132, 137, 140, 141, 142, 144, 146, 154, 156, 163, 167, 180, 191, 217, 223, 229, 232, 236, 240, 242, 245, 254, 257, 260, 261, 262, 263, 266, 278, 280, 301, 313, 346, 356, 367, 375, 377, 401, 403, 423, 449, 450, 459, 461, 467, 468, 469, 470, 471, 472, 473, 475, 486, 510], "two": [8, 19, 23, 27, 29, 35, 37, 39, 40, 41, 42, 45, 48, 51, 52, 53, 55, 56, 57, 59, 60, 62, 66, 68, 69, 70, 71, 73, 74, 75, 76, 84, 87, 88, 94, 96, 97, 99, 100, 101, 102, 104, 105, 107, 109, 112, 117, 123, 124, 126, 131, 132, 144, 146, 148, 151, 155, 173, 203, 209, 217, 233, 249, 264, 268, 284, 287, 297, 302, 303, 356, 371, 388, 395, 403, 437, 441, 461, 462, 466, 467, 469, 473, 475, 476, 486, 503, 504, 510, 512, 520, 523, 544], "measur": [8, 10, 41, 68, 79, 80, 101, 109, 123, 282, 512, 520], "unit": [8, 29, 36, 45, 79, 84, 106, 107, 109, 118, 124, 147, 156, 242, 293, 338, 396, 424, 520], "increment": [8, 41, 45, 100, 110, 251, 263, 511], "1024": [8, 106], "megabyt": [8, 41, 45, 54, 83, 106, 293, 320, 321, 480], "kilobyt": [8, 83, 106], "6gb": 8, "six": 8, "gigabyt": [8, 79, 83, 106], "6": [8, 9, 23, 37, 40, 41, 43, 54, 57, 60, 72, 75, 78, 84, 99, 100, 102, 104, 106, 107, 112, 113, 119, 123, 131, 136, 144, 153, 158, 208, 368, 371, 373, 379, 389, 391, 413, 415, 520], "6442450944": 8, "b": [8, 23, 29, 35, 44, 83, 99, 100, 103, 104, 105, 106, 107, 112, 113, 116, 117, 118, 121, 124, 148, 220, 226, 250, 265, 283, 288, 307, 375, 472, 510, 512, 520, 544], "kb": [8, 83, 106, 453], "mb": [8, 9, 45, 51, 67, 83, 106, 250], "gb": [8, 83, 106, 389, 413], "tb": [8, 48, 106], "terabyt": [8, 106, 152], "pb": [8, 106], "petabyt": [8, 106, 152], "numer": [8, 31, 35, 37, 45, 54, 57, 59, 70, 71, 73, 84, 99, 105, 106, 107, 112, 119, 127, 132, 144, 217, 245, 274, 287, 303, 307, 342, 359, 365, 368, 389, 392, 393, 396, 417, 434, 447, 486, 542], "decim": [8, 37, 39, 40, 41, 45, 54, 55, 60, 62, 65, 76, 84, 94, 97, 98, 107, 114, 118, 133, 206, 209, 217, 223, 231, 233, 235, 236, 245, 247, 249, 254, 255, 257, 261, 263, 271, 288, 336, 337, 342, 348, 352, 354, 355, 356, 358, 359, 361, 367, 368, 369, 372, 373, 379, 380, 388, 389, 390, 391, 392, 395, 396, 398, 407, 413, 422, 424, 426, 432, 434, 435, 440, 450, 452, 453, 455, 456, 520], "neg": [8, 13, 40, 42, 55, 62, 65, 66, 68, 69, 71, 74, 75, 76, 96, 100, 101, 102, 107, 112, 124, 131, 167, 180, 204, 209, 253, 261, 274, 339, 348, 361, 366, 368, 373, 398, 420, 510, 520], "7m": 8, "seven": 8, "ns": [8, 107], "nanosecond": [8, 45, 57, 107, 211, 369, 373, 375, 377, 379, 389], "microsecond": [8, 45, 57, 107, 368, 369, 372, 373, 379, 389], "ms": [8, 60, 84, 107], "millisecond": [8, 45, 57, 59, 65, 73, 78, 83, 97, 107, 114, 144, 155, 186, 204, 209, 247, 301, 303, 327, 332, 369, 373, 375, 384, 389, 450], "second": [8, 11, 23, 32, 36, 38, 39, 41, 45, 51, 53, 54, 57, 58, 60, 64, 68, 72, 73, 83, 84, 85, 100, 107, 109, 112, 114, 128, 148, 155, 157, 170, 186, 204, 207, 209, 210, 242, 247, 267, 278, 287, 297, 341, 350, 360, 368, 369, 375, 380, 381, 469, 473, 510, 520], "m": [8, 60, 83, 100, 107, 118, 121, 146, 293, 468, 510], "hour": [8, 24, 54, 59, 64, 68, 83, 107, 114, 128, 135, 144, 168, 253, 257, 293, 297, 301, 424, 470, 525], "dai": [8, 11, 29, 36, 41, 45, 54, 64, 71, 78, 84, 100, 107, 110, 114, 128, 131, 180, 204, 209, 247, 253, 263, 282, 293, 316, 356, 368, 464, 465], "treat": [8, 15, 25, 33, 35, 42, 57, 59, 68, 73, 83, 102, 104, 109, 112, 121, 124, 142, 155, 156, 183, 204, 209, 233, 255, 257, 261, 278, 293, 334, 462, 469], "zero": [8, 38, 39, 43, 54, 68, 78, 99, 104, 105, 108, 112, 120, 131, 144, 155, 162, 167, 173, 259, 262, 265, 278, 287, 288, 327, 331, 332, 349, 356, 359, 361, 374, 390, 438, 454, 462, 510, 520], "regardless": [8, 39, 52, 267, 277, 491, 520], "0m": [8, 11, 37, 355], "both": [8, 10, 13, 14, 23, 27, 32, 33, 35, 37, 39, 40, 41, 45, 47, 48, 51, 52, 54, 57, 60, 62, 63, 65, 66, 68, 69, 70, 71, 74, 75, 76, 84, 88, 96, 104, 105, 109, 112, 116, 124, 132, 136, 137, 144, 146, 151, 162, 170, 177, 232, 239, 256, 264, 282, 287, 293, 296, 297, 302, 304, 354, 357, 368, 377, 382, 424, 438, 462, 464, 465, 475, 476, 510, 512, 517, 520], "mean": [8, 13, 23, 32, 37, 41, 45, 52, 54, 57, 59, 60, 65, 76, 78, 83, 84, 88, 94, 96, 97, 99, 104, 109, 112, 118, 137, 144, 146, 148, 218, 276, 282, 351, 384, 462, 465, 471, 473, 474, 510, 520], "thing": [8, 17, 58, 86], "25d": 8, "fraction": [8, 68, 103, 106, 107, 108, 109, 144, 146, 149, 246, 274, 278, 344, 368, 375, 380, 381, 432], "5m": [8, 11, 13, 15, 35, 41, 44, 64, 73, 107, 470], "equal": [8, 13, 25, 38, 54, 57, 59, 71, 72, 73, 78, 84, 88, 94, 99, 100, 104, 105, 108, 109, 126, 127, 144, 196, 217, 245, 247, 259, 260, 266, 267, 274, 282, 329, 333, 346, 354, 368, 369, 375, 381, 382, 392, 393, 395, 397, 408, 409, 416, 429, 434, 436, 438, 454, 466, 470, 520], "half": [8, 60, 491], "90": [8, 109, 153, 328, 506], "1000": [8, 23, 38, 40, 41, 42, 43, 44, 45, 53, 55, 57, 59, 62, 64, 66, 68, 69, 71, 74, 75, 76, 102, 176, 472, 506], "well": [8, 16, 26, 35, 37, 41, 45, 48, 49, 51, 69, 73, 83, 87, 94, 96, 104, 109, 113, 126, 131, 136, 138, 144, 257, 276, 346, 354, 373, 464, 467, 494, 506, 507, 510, 517, 532], "7": [8, 23, 29, 35, 41, 54, 57, 60, 66, 75, 76, 78, 99, 100, 102, 106, 107, 110, 113, 136, 144, 153, 282, 321, 360, 374, 397, 454, 506, 520], "enforc": [8, 13, 23, 41, 52, 54, 60, 207, 261, 266, 276, 358, 365, 368, 372, 384, 389, 398, 422, 450, 457, 460, 462, 464, 465, 472, 512], "sequenc": [8, 15, 41, 57, 59, 69, 73, 94, 100, 112, 114, 115, 119, 124, 126, 131, 151, 208, 210, 232, 239, 262, 263, 436, 441, 449, 510, 512], "charact": [8, 15, 32, 39, 40, 42, 45, 51, 54, 55, 56, 57, 59, 60, 66, 69, 71, 73, 74, 75, 101, 103, 104, 106, 107, 112, 121, 122, 124, 128, 129, 136, 142, 144, 156, 173, 186, 223, 232, 233, 250, 251, 263, 302, 330, 333, 335, 343, 344, 352, 355, 357, 360, 364, 366, 380, 391, 392, 399, 422, 423, 425, 426, 427, 432, 437, 438, 440, 446, 447, 449, 450, 454, 455, 456, 473, 510, 536], "transfer": [9, 14, 19, 36, 48, 65, 146, 148, 151, 223, 284, 331, 430, 466], "help": [9, 14, 18, 19, 23, 26, 32, 35, 39, 60, 83, 96, 136, 138, 144, 217, 257, 280, 321, 347, 368, 467, 468, 510, 512], "resolv": [9, 32, 33, 57, 72, 84, 126, 219, 223, 313, 389, 439, 463, 543], "inter": [9, 15], "25": [9, 13, 14, 32, 58, 68, 70, 100, 102, 103, 105, 107, 113, 135, 137, 147, 148, 153, 303, 505, 520, 527], "fetch": [9, 39, 44, 58, 64, 65, 70, 83, 84, 112, 148, 151, 210, 217, 235, 241, 257, 268, 282, 317, 337, 342, 353, 354, 358, 367, 368, 382, 383, 384, 408, 432, 441, 447, 453], "excess": [9, 15, 19, 99, 155, 157, 158, 242, 244, 246, 250, 251, 253, 263, 269, 309, 311, 315, 320, 321, 333, 346, 356, 358, 365], "drop": [9, 19, 23, 24, 35, 37, 38, 44, 45, 52, 55, 57, 62, 64, 65, 66, 68, 69, 75, 84, 118, 133, 142, 143, 155, 167, 205, 207, 208, 217, 228, 245, 246, 250, 257, 258, 260, 261, 262, 272, 275, 296, 297, 300, 327, 333, 344, 346, 356, 358, 359, 362, 371, 380, 383, 384, 387, 388, 389, 392, 394, 396, 397, 398, 399, 400, 401, 402, 403, 405, 406, 409, 419, 421, 422, 426, 427, 432, 433, 434, 435, 436, 442, 448, 450, 451, 452, 453, 462, 477, 478, 479, 480, 486, 487, 488, 489, 491, 505, 508, 514, 518, 520, 522, 528], "context": [9, 19, 33, 40, 41, 42, 43, 45, 54, 55, 66, 68, 69, 71, 74, 75, 76, 83, 84, 112, 121, 124, 132, 137, 226, 262, 270, 391, 392, 462, 469, 510], "rel": [9, 12, 23, 24, 32, 72, 93, 101, 124, 147, 232, 246, 356, 459, 475, 476, 506, 525, 543], "heurist": [9, 163], "space": [9, 19, 27, 48, 52, 60, 68, 76, 78, 103, 109, 121, 129, 136, 144, 210, 223, 242, 248, 270, 271, 287, 304, 330, 366, 372, 388], "averag": [9, 22, 48, 88, 99, 100, 242, 256, 271, 506, 510], "32": [9, 13, 19, 42, 45, 58, 62, 66, 68, 71, 74, 75, 76, 101, 110, 136, 144, 178, 316], "20": [9, 40, 42, 43, 44, 48, 54, 55, 57, 58, 62, 64, 65, 66, 68, 69, 70, 71, 74, 75, 76, 100, 101, 102, 104, 107, 108, 113, 117, 144, 153, 368, 397, 398, 520], "2mb": [9, 15], "20mb": 9, "abort": [9, 173, 206, 217, 251, 258, 259, 270, 334, 373, 380, 400, 520], "detect": [9, 14, 41, 54, 57, 58, 65, 83, 84, 123, 159, 187, 209, 274, 282, 287, 293, 303, 324, 450, 505, 506, 510], "built": [9, 17, 24, 35, 43, 58, 84, 88, 93, 97, 98, 144, 151, 276, 460, 532], "hold": [9, 35, 41, 45, 54, 57, 59, 72, 73, 83, 88, 97, 136, 488], "throughput": [9, 19, 25, 35, 170, 193, 248, 374, 466], "thu": [9, 19, 25, 33, 37, 41, 45, 51, 52, 71, 74, 76, 84, 93, 95, 131, 136, 183, 226, 256, 295, 342, 471, 476, 483, 491, 520], "decreas": [9, 14, 16, 22, 25, 41, 45, 66, 71, 76, 241, 246, 256, 345, 368, 374, 510], "1mb": [9, 41, 48, 70, 106, 231], "16mb": [9, 19, 41, 51, 78], "place": [9, 23, 45, 48, 49, 51, 57, 58, 64, 73, 89, 93, 96, 100, 118, 139, 148, 325, 348, 395, 464, 465, 474, 476, 513], "skew": [9, 10, 25, 27, 41, 54, 99, 114, 115, 137, 163, 217, 235, 236, 261, 280, 432, 506], "fewer": [9, 41, 45, 54, 84, 109, 137], "pull": 9, "upstream": [9, 14, 415], "consum": [9, 23, 27, 38, 39, 52, 57, 58, 84, 97, 151, 239, 241, 278, 486], "200mb": [9, 372], "join_distribution_typ": [10, 146, 272, 330, 333], "slower": [10, 16, 18, 39, 48, 146, 278, 296], "sometim": [10, 109, 132, 247, 279, 282, 350, 356], "substanti": [10, 14, 255, 284, 285, 302, 357], "particular": [10, 14, 29, 51, 52, 72, 73, 79, 83, 84, 86, 93, 97, 106, 109, 136, 149, 151, 167, 263, 302, 303, 317, 510, 512, 520], "faster": [10, 13, 16, 37, 39, 42, 43, 48, 62, 65, 66, 68, 71, 74, 76, 101, 146, 156, 257, 284, 293, 302, 486], "wherea": [10, 29, 71, 100, 146], "decis": [10, 45, 54, 66, 71, 76, 79, 80, 99, 332], "mode": [10, 33, 41, 45, 48, 52, 57, 65, 66, 68, 69, 71, 75, 121, 173, 206, 229, 253, 259, 328, 330, 331, 334, 351, 354, 355, 361, 362, 365, 369, 396, 400, 401, 403, 404, 408, 413, 415, 423, 425, 442, 446, 448, 450, 453, 462, 491, 512, 540], "could": [10, 14, 19, 23, 32, 35, 41, 52, 54, 57, 59, 73, 84, 144, 163, 169, 171, 177, 178, 180, 183, 187, 190, 191, 194, 197, 204, 209, 211, 213, 214, 218, 221, 222, 223, 226, 228, 232, 234, 235, 240, 244, 245, 248, 256, 257, 264, 268, 270, 273, 279, 280, 282, 287, 295, 296, 297, 303, 304, 313, 314, 315, 316, 323, 330, 332, 334, 335, 345, 361, 373, 381, 392, 394, 400, 465], "boolean": [10, 11, 12, 13, 14, 15, 18, 19, 20, 21, 22, 29, 32, 35, 37, 39, 40, 41, 43, 45, 51, 54, 55, 57, 58, 59, 60, 62, 65, 66, 69, 71, 73, 75, 76, 78, 84, 88, 94, 99, 100, 103, 105, 109, 111, 112, 116, 117, 118, 121, 124, 140, 156, 189, 213, 217, 229, 329, 335, 337, 350, 369, 383, 391, 402, 432, 434, 456, 462, 476, 497, 505, 510, 512, 520], "redistribute_writ": [10, 163], "elimin": [10, 14, 146, 268, 272, 274, 343, 520], "351": [10, 153], "releas": [10, 63, 112, 133, 135, 136, 137, 469], "version": [10, 32, 33, 35, 39, 40, 41, 42, 44, 45, 53, 54, 55, 56, 57, 58, 60, 62, 64, 66, 68, 69, 71, 72, 74, 75, 93, 109, 114, 115, 125, 135, 136, 137, 139, 155, 156, 173, 180, 186, 204, 209, 217, 220, 221, 232, 233, 242, 244, 270, 274, 285, 287, 291, 301, 314, 317, 327, 328, 332, 334, 338, 339, 342, 344, 345, 348, 354, 357, 359, 368, 374, 379, 381, 386, 389, 392, 396, 397, 398, 399, 405, 408, 411, 412, 413, 414, 415, 416, 417, 419, 421, 423, 425, 427, 428, 432, 436, 440, 449, 451, 454, 462, 465, 466, 468, 469, 470, 475, 505, 506], "350": [10, 23, 32, 33, 153, 332], "presto": [10, 30, 43, 45, 60, 132, 151, 157, 163, 167, 172, 177, 178, 191, 206, 207, 208, 209, 213, 217, 218, 227, 233, 235, 239, 242, 245, 247, 253, 254, 260, 262, 268, 270, 281, 282, 283, 285, 287, 288, 293, 296, 302, 303, 304, 305, 307, 309, 317, 320, 321, 324, 327, 331, 332, 333, 334, 336, 339, 341, 344, 348, 353, 355, 356, 357, 358, 359, 360, 361, 362, 366, 368, 374, 377, 378, 385, 393, 426], "while": [10, 13, 22, 25, 32, 33, 35, 38, 41, 43, 45, 47, 51, 54, 59, 84, 89, 109, 110, 112, 136, 137, 139, 146, 151, 169, 178, 247, 254, 256, 260, 261, 262, 276, 277, 279, 287, 317, 323, 328, 345, 360, 361, 365, 391, 392, 418, 419, 420, 432, 434, 437, 469, 470, 475, 510, 512, 520], "newer": [10, 32, 33, 41, 57, 59, 73, 136, 257, 359, 391], "older": [10, 11, 32, 33, 39, 45, 54, 57, 59, 64, 73, 204, 209, 239, 327, 363, 367, 378, 381, 389, 392, 393, 395, 397, 398, 415, 421, 451, 466, 475], "prefer": [10, 19, 33, 59, 66, 70, 83, 105, 136, 266, 284, 357, 373, 381, 443, 471, 475, 510, 520], "approach": [10, 25, 48, 54, 59], "earlier": [10, 48, 58, 68, 136, 274, 318, 345, 348, 368, 380], "updat": [10, 13, 15, 23, 35, 38, 41, 42, 57, 58, 59, 60, 73, 83, 84, 94, 133, 137, 143, 149, 155, 156, 159, 167, 173, 188, 194, 204, 208, 209, 212, 213, 217, 220, 233, 244, 251, 254, 255, 257, 259, 260, 261, 267, 270, 271, 275, 276, 279, 282, 283, 287, 293, 294, 302, 307, 316, 317, 320, 321, 325, 327, 332, 334, 336, 342, 344, 345, 350, 354, 357, 358, 374, 379, 381, 382, 383, 386, 387, 388, 389, 390, 391, 392, 398, 399, 401, 403, 407, 408, 409, 412, 414, 417, 419, 421, 427, 432, 435, 441, 445, 448, 450, 453, 454, 455, 462, 472, 477, 505, 507, 511, 543], "afterward": [10, 25, 48], "remov": [10, 15, 32, 40, 41, 42, 44, 45, 47, 48, 52, 54, 55, 57, 59, 65, 66, 68, 69, 71, 73, 74, 75, 76, 83, 99, 100, 112, 121, 124, 135, 154, 155, 156, 157, 159, 170, 171, 177, 188, 191, 193, 194, 208, 212, 213, 217, 219, 221, 223, 226, 232, 235, 239, 242, 248, 251, 255, 257, 258, 260, 262, 263, 265, 266, 267, 268, 270, 271, 272, 273, 274, 275, 276, 277, 278, 279, 280, 293, 296, 302, 327, 328, 329, 330, 331, 332, 333, 334, 339, 344, 345, 354, 355, 356, 357, 358, 359, 366, 367, 368, 369, 371, 373, 378, 379, 383, 385, 387, 388, 391, 392, 393, 396, 397, 398, 399, 400, 402, 403, 407, 408, 409, 413, 415, 416, 417, 418, 419, 422, 424, 426, 429, 430, 432, 433, 434, 435, 436, 438, 443, 445, 446, 447, 448, 450, 451, 452, 453, 454, 455, 456, 462, 466, 484, 492, 517], "temporari": [10, 37, 40, 41, 45, 51, 54, 55, 62, 63, 66, 68, 69, 71, 74, 75, 76, 136, 157, 167, 208, 212, 267, 274, 296, 327, 328, 342, 360, 368, 375], "assist": [10, 464], "effort": [10, 15, 45, 389], "2048": [10, 267], "submit": [10, 13, 23, 24, 32, 40, 42, 52, 55, 66, 68, 69, 71, 74, 75, 76, 83, 151, 163, 178, 182, 255, 293, 307, 323, 344, 462, 467], "transport": [10, 76, 78, 88, 132, 185, 213, 228, 232, 293, 367], "hit": [10, 16, 17, 43, 44, 48, 180, 357, 367, 368, 383], "512": [10, 334, 466], "less": [10, 16, 17, 19, 32, 35, 51, 62, 99, 100, 101, 102, 104, 109, 118, 120, 124, 132, 247, 256, 258, 259, 264, 278, 302, 330, 368, 392, 413, 465, 466, 470, 514], "uncompress": [10, 22, 356, 417, 451], "prefix": [11, 32, 33, 35, 46, 50, 51, 57, 58, 60, 69, 80, 109, 112, 121, 124, 137, 144, 188, 217, 226, 305, 323, 344, 351, 378, 385, 428, 510, 512, 523], "oauth": [11, 27, 32, 33, 50, 133, 329, 369, 381, 397, 408, 415, 420, 424, 430, 432, 435, 449, 453, 456, 458, 459, 467, 471], "authent": [11, 33, 37, 39, 47, 50, 51, 57, 60, 64, 65, 70, 81, 93, 95, 133, 172, 176, 178, 196, 209, 225, 232, 245, 251, 254, 258, 260, 261, 263, 264, 266, 268, 271, 272, 275, 329, 334, 345, 346, 350, 354, 355, 356, 357, 358, 359, 360, 361, 364, 368, 369, 371, 372, 379, 380, 381, 382, 383, 384, 386, 388, 391, 392, 394, 396, 397, 398, 399, 403, 405, 408, 409, 424, 430, 434, 435, 446, 449, 453, 462, 466, 475, 476, 516, 533], "interact": [11, 24, 31, 32, 39, 64, 83, 132, 151, 152], "oauth2": [11, 20, 27, 64, 375, 379, 380, 383, 384, 388, 392, 394, 396, 397, 435, 449, 459, 467, 476], "jwk": [11, 372, 379, 384, 467, 470], "ten": [11, 177, 297], "jwt": [11, 20, 27, 33, 133, 264, 268, 353, 358, 361, 372, 379, 384, 397, 424, 449, 458, 459, 470, 471, 476], "5s": [11, 38, 463, 472], "establish": [11, 32, 39, 65, 66, 76, 469, 510, 520], "200": [11, 41, 45, 58, 83, 99, 113, 153, 334, 505], "regex": [11, 23, 24, 45, 209, 371, 436, 462, 468, 469, 470, 476], "dn": [11, 32, 33, 70, 356, 392, 439, 461, 464, 465, 468, 469], "subjectalternativenam": [11, 464], "san": [11, 464, 465, 469, 482, 509, 513], "field": [11, 12, 23, 39, 41, 43, 45, 53, 54, 58, 65, 72, 83, 84, 86, 94, 96, 100, 107, 112, 124, 126, 142, 144, 148, 180, 194, 197, 198, 204, 209, 210, 213, 225, 226, 227, 228, 233, 249, 250, 254, 261, 262, 264, 267, 268, 270, 272, 273, 274, 293, 314, 333, 341, 342, 343, 344, 346, 350, 351, 352, 355, 356, 359, 361, 362, 367, 372, 373, 375, 379, 380, 381, 383, 384, 388, 389, 391, 392, 394, 396, 402, 412, 426, 432, 433, 435, 436, 439, 442, 444, 447, 448, 449, 450, 456, 461, 462, 464, 465, 467, 470, 476, 486, 511, 520, 542], "keystor": [11, 32, 33, 37, 39, 40, 42, 43, 62, 66, 68, 70, 71, 74, 76, 132, 254, 268, 339, 349, 367, 419, 432, 461, 465, 466, 475], "accord": [11, 13, 23, 37, 39, 40, 42, 43, 54, 57, 58, 59, 62, 70, 72, 73, 101, 103, 112, 118, 124, 131, 139, 265, 480, 510, 520], "possibl": [11, 14, 25, 27, 39, 41, 45, 47, 49, 51, 54, 55, 57, 64, 66, 68, 71, 73, 76, 78, 83, 84, 94, 99, 106, 136, 141, 146, 148, 151, 155, 157, 178, 180, 187, 210, 239, 245, 256, 264, 273, 282, 324, 330, 345, 364, 368, 379, 382, 383, 389, 391, 396, 401, 402, 423, 433, 461, 462, 464, 471, 475, 510, 512, 520], "nativeprng": 11, "nativeprngblock": 11, "nativeprngnonblock": 11, "pkcs11": 11, "sha1prng": [11, 466], "truststor": [11, 32, 33, 37, 39, 43, 70, 76, 132, 217, 254, 268, 339, 384, 403, 419, 432, 466, 469, 475], "host": [11, 12, 13, 32, 33, 36, 39, 42, 43, 47, 48, 51, 52, 57, 60, 70, 72, 73, 76, 78, 129, 132, 136, 137, 157, 209, 213, 235, 240, 258, 274, 367, 421, 467, 468], "8080": [11, 32, 33, 129, 135, 136, 137, 385, 475], "gz": [11, 69, 132, 136, 475], "frequenc": [11, 40, 41, 42, 55, 62, 66, 68, 69, 71, 74, 75, 76, 77, 99], "retent": [11, 41, 54, 72, 158, 320], "delet": [11, 23, 24, 35, 38, 41, 45, 52, 55, 64, 65, 73, 83, 84, 94, 124, 133, 137, 139, 142, 143, 156, 161, 177, 180, 197, 209, 217, 236, 240, 270, 341, 344, 354, 356, 359, 368, 372, 377, 381, 383, 386, 387, 388, 390, 392, 399, 400, 401, 402, 403, 406, 408, 409, 410, 412, 414, 415, 418, 419, 420, 424, 427, 429, 430, 432, 434, 436, 438, 441, 445, 450, 452, 454, 455, 460, 462, 477, 494, 505, 507, 511, 517, 541], "var": [11, 12, 48, 54, 124, 136, 139, 296, 341], "10000": [11, 35, 40, 41, 42, 44, 54, 55, 59, 62, 64, 65, 66, 67, 68, 69, 71, 74, 75, 76, 121, 136], "machin": [12, 13, 32, 37, 43, 45, 56, 57, 60, 69, 78, 88, 98, 99, 132, 133, 134, 135, 136, 137, 151, 160, 264, 282, 293, 301, 389, 468, 474], "environ": [12, 23, 32, 40, 42, 45, 55, 59, 66, 68, 69, 71, 74, 75, 76, 84, 95, 122, 125, 139, 194, 242, 243, 253, 273, 359, 371, 380, 466, 474, 475], "central": [12, 33, 58], "variabl": [12, 23, 32, 38, 40, 42, 45, 55, 57, 59, 66, 68, 69, 71, 74, 75, 76, 84, 93, 137, 139, 144, 237, 242, 246, 254, 262, 273, 304, 313, 315, 332, 362, 364, 368, 369, 371, 380, 390, 401, 430, 435, 442, 474, 512, 538], "env": [12, 40, 42, 50, 55, 66, 68, 69, 71, 74, 75, 76, 84, 139, 380, 474], "hostnam": [12, 32, 33, 52, 57, 70, 73, 223, 249, 329, 356, 357, 448, 466, 468], "text": [12, 15, 27, 32, 37, 39, 44, 57, 58, 59, 62, 66, 70, 73, 75, 76, 77, 84, 109, 112, 123, 148, 151, 178, 217, 233, 257, 258, 276, 277, 278, 293, 303, 327, 341, 347, 348, 357, 368, 377, 386, 390, 391, 398, 399, 435, 436, 438, 444, 445, 449, 451, 454, 465, 472, 475, 505, 510], "line": [12, 13, 31, 33, 45, 52, 56, 57, 83, 86, 93, 103, 109, 121, 131, 133, 135, 136, 137, 139, 141, 180, 264, 270, 271, 277, 278, 281, 287, 302, 305, 341, 347, 348, 356, 367, 379, 386, 401, 450, 451, 463, 464, 467, 468, 469, 472, 474, 475, 476, 493, 505], "newlin": [12, 302, 425], "except": [12, 23, 29, 40, 42, 43, 45, 51, 52, 54, 55, 57, 62, 64, 66, 68, 69, 71, 74, 75, 76, 78, 84, 86, 94, 99, 100, 104, 108, 112, 113, 121, 131, 139, 142, 148, 161, 162, 172, 177, 178, 183, 194, 208, 213, 226, 231, 240, 273, 278, 284, 287, 303, 305, 307, 363, 373, 374, 381, 386, 395, 432, 434, 456, 460, 462, 468, 469, 476, 511], "stack": [12, 19, 32, 83, 183, 278, 280, 457, 468], "trace": [12, 32, 33, 35, 40, 42, 55, 66, 68, 69, 71, 74, 75, 76, 83, 183, 263, 328, 329, 359, 387, 441, 468], "index": [12, 39, 43, 45, 53, 54, 57, 59, 60, 66, 67, 69, 78, 96, 100, 109, 112, 114, 115, 119, 121, 123, 124, 128, 160, 162, 163, 180, 181, 217, 233, 241, 246, 253, 254, 278, 300, 325, 327, 355, 359, 367, 373, 377, 379, 386, 388, 391, 392, 393, 394, 396, 403, 425, 432, 433, 436, 442, 446, 506], "singular": [12, 29, 109], "search": [12, 16, 32, 35, 43, 84, 105, 112, 123, 124, 335, 422, 469, 510, 512], "launcher": [12, 58, 139], "script": [12, 32, 121, 136, 141, 170, 177, 282, 469], "altern": [12, 15, 23, 32, 41, 44, 45, 47, 50, 60, 64, 84, 88, 112, 132, 135, 136, 144, 152, 288, 297, 378, 383, 385, 448, 450, 464, 465, 475], "next": [12, 23, 25, 32, 35, 41, 57, 59, 60, 63, 78, 83, 114, 115, 141, 146, 176, 227, 246, 293, 464, 465, 471, 475, 510, 520], "sidecar": 12, "tcp": [12, 48, 51, 135, 137], "protocol": [12, 33, 39, 41, 45, 49, 51, 64, 65, 78, 129, 132, 251, 254, 267, 327, 338, 339, 378, 383, 399, 407, 428, 467, 468, 471, 475], "100mb": [12, 22, 41, 45, 48, 54, 146, 277, 356], "combin": [12, 40, 43, 45, 50, 54, 84, 88, 97, 99, 100, 104, 107, 110, 112, 114, 115, 124, 144, 147, 148, 151, 156, 194, 217, 231, 271, 282, 303, 338, 369, 470, 471, 472, 510, 512], "gzip": [12, 41, 45, 54, 132, 196, 197, 293, 335, 359, 379, 449], "rotat": [12, 51, 136, 395, 396], "flag": [12, 23, 43, 65, 84, 88, 94, 121, 136, 157, 167, 170, 188, 206, 210, 212, 221, 223, 231, 233, 251, 301, 304, 374, 397, 468, 470], "replac": [12, 15, 32, 33, 35, 37, 39, 40, 42, 43, 45, 51, 54, 55, 57, 59, 60, 62, 65, 66, 69, 70, 71, 72, 73, 74, 75, 76, 78, 88, 96, 100, 114, 115, 121, 124, 136, 152, 173, 193, 208, 213, 217, 226, 232, 248, 256, 261, 264, 266, 272, 293, 306, 316, 330, 342, 357, 368, 371, 397, 403, 419, 424, 432, 438, 450, 454, 462, 468, 469, 470, 474, 475, 476, 486, 491, 513], "unlimit": [12, 72, 339, 438], "critic": [13, 84, 136, 296], "100": [13, 15, 19, 23, 32, 38, 41, 45, 48, 54, 58, 65, 73, 83, 94, 99, 100, 102, 105, 107, 112, 113, 126, 135, 137, 153, 209, 224, 273, 278, 328, 408, 434, 466, 472, 475], "e": [13, 23, 27, 35, 41, 45, 51, 52, 54, 57, 58, 59, 60, 69, 70, 73, 76, 83, 96, 99, 100, 104, 107, 109, 112, 115, 117, 118, 121, 124, 136, 139, 209, 223, 231, 233, 234, 235, 242, 246, 247, 250, 255, 258, 262, 263, 264, 282, 287, 307, 313, 327, 328, 333, 334, 339, 347, 364, 365, 375, 379, 385, 387, 466, 506, 520, 542, 544], "g": [13, 27, 41, 45, 51, 52, 57, 59, 60, 69, 70, 73, 76, 83, 107, 109, 112, 121, 209, 231, 233, 234, 235, 246, 247, 250, 255, 258, 262, 263, 264, 282, 287, 307, 313, 327, 328, 333, 347, 364, 365, 375, 379, 385, 387, 466, 506, 544], "quickli": [13, 35, 123, 144, 506], "assign": [13, 14, 15, 21, 23, 41, 45, 51, 53, 54, 55, 57, 59, 60, 62, 66, 68, 71, 73, 74, 75, 76, 80, 84, 135, 136, 139, 144, 302, 310, 345, 397, 462, 465, 468, 471, 510, 517, 520, 542], "enough": [13, 25, 27, 39, 53, 270, 438, 466], "fulli": [13, 14, 24, 35, 41, 45, 51, 54, 59, 60, 62, 64, 66, 68, 69, 70, 71, 74, 75, 76, 96, 109, 135, 136, 151, 173, 246, 284, 293, 303, 307, 331, 373, 387, 438, 466, 483], "compens": 13, "wast": [13, 15, 16, 19, 225, 408], "ideal": [13, 57], "alwai": [13, 14, 19, 27, 38, 45, 48, 57, 59, 60, 66, 69, 71, 75, 102, 121, 124, 143, 144, 151, 155, 163, 166, 167, 170, 183, 209, 219, 239, 242, 249, 259, 262, 263, 278, 287, 303, 331, 334, 371, 380, 383, 403, 406, 421, 438, 462, 464, 470, 471, 475, 491, 520], "least": [13, 14, 15, 19, 23, 27, 39, 55, 57, 60, 65, 70, 94, 96, 99, 109, 113, 114, 115, 116, 207, 249, 268, 274, 278, 297, 315, 373, 408, 413, 462, 465, 476, 510, 520], "outstand": [13, 41, 45, 53, 255, 355], "guarante": [13, 16, 19, 25, 57, 84, 99, 104, 157, 217, 270, 434, 505, 510, 520], "remain": [13, 19, 23, 32, 45, 63, 82, 85, 89, 90, 92, 95, 110, 112, 136, 137, 144, 148, 271, 280, 364, 470, 510, 520], "starvat": [13, 136, 198, 246, 302, 329], "deadlock": [13, 19, 25, 156, 157, 158, 187, 208, 217, 228, 259, 324, 358, 368, 371, 379, 383, 397, 430], "usual": [13, 35, 65, 83, 88, 94, 136, 248, 516, 520, 542], "reason": [13, 14, 41, 45, 57, 63, 66, 68, 72, 83, 84, 109, 136, 138, 146, 278, 358], "drawback": [13, 14], "queue": [13, 15, 23, 53, 154, 155, 169, 177, 210, 244, 251, 263, 276, 307, 310, 323, 324, 345, 436], "upward": 13, "receiv": [13, 15, 23, 31, 32, 35, 57, 83, 84, 91, 136, 140, 151, 173, 182, 282, 302, 334, 457, 464, 465, 468], "uniform": [13, 53, 346, 362], "yet": [13, 14, 23, 40, 44, 45, 47, 52, 58, 59, 64, 144, 188, 254, 260, 267, 271, 285, 367, 373, 423, 428, 486, 510], "confirm": [13, 144, 464, 465, 466, 472], "preced": [13, 32, 33, 40, 41, 42, 43, 54, 55, 57, 64, 66, 68, 69, 71, 74, 75, 76, 84, 94, 104, 112, 131, 142, 144, 148, 182, 203, 209, 347, 373, 382, 459, 466, 476, 510, 512, 520], "might": [13, 14, 18, 40, 41, 42, 43, 44, 45, 51, 55, 57, 66, 68, 69, 71, 72, 74, 75, 76, 84, 94, 126, 137, 151, 218, 317, 468, 470, 510], "evalu": [13, 66, 71, 76, 84, 89, 99, 104, 105, 112, 116, 121, 124, 131, 186, 194, 210, 217, 226, 227, 250, 264, 283, 347, 387, 391, 397, 421, 432, 434, 438, 462, 512, 520, 542], "properli": [13, 35, 44, 63, 112, 137, 162, 167, 183, 187, 219, 225, 257, 282, 287, 313, 382, 406, 413, 446, 462], "maintain": [13, 14, 40, 42, 45, 54, 55, 57, 59, 62, 65, 66, 68, 69, 71, 73, 74, 75, 76, 132, 272, 428, 467], "tri": [13, 41, 45, 59, 65, 84, 94, 258, 469, 512], "distanc": [13, 109, 124], "rack": [13, 60], "three": [13, 23, 33, 39, 41, 43, 45, 55, 57, 60, 62, 66, 68, 70, 71, 74, 75, 76, 96, 102, 103, 107, 112, 126, 137, 147, 242, 434, 461, 470, 471, 496, 510, 520, 532, 544], "flat": [13, 346], "deriv": [13, 23, 55, 84, 94, 109, 229, 245, 334, 369, 382], "map": [13, 24, 27, 45, 47, 56, 82, 83, 85, 87, 89, 90, 92, 95, 96, 98, 100, 110, 112, 114, 118, 119, 123, 124, 133, 135, 136, 140, 151, 154, 156, 165, 173, 176, 182, 191, 207, 209, 210, 212, 215, 217, 219, 226, 228, 229, 230, 233, 240, 241, 244, 245, 246, 247, 248, 249, 251, 254, 257, 258, 260, 261, 264, 266, 267, 268, 274, 279, 280, 287, 288, 289, 296, 297, 299, 304, 307, 310, 313, 317, 323, 326, 327, 331, 333, 339, 345, 348, 350, 351, 353, 355, 356, 357, 359, 360, 362, 365, 367, 368, 369, 373, 375, 377, 379, 384, 386, 387, 389, 390, 391, 392, 393, 396, 397, 398, 399, 400, 417, 419, 420, 423, 425, 430, 432, 433, 441, 442, 444, 445, 446, 448, 449, 453, 454, 457, 458, 459, 461, 462, 463, 467, 469, 470, 471, 510, 520], "whitespac": [13, 124, 357, 359, 414, 420, 450], "begin": [13, 16, 35, 59, 109, 131, 141, 152, 465, 467, 470, 475, 526, 532, 537, 539], "lead": [13, 19, 25, 37, 38, 44, 45, 64, 69, 76, 84, 108, 114, 115, 124, 131, 144, 158, 167, 248, 257, 273, 300, 308, 330, 354, 361, 388, 427, 454, 520], "192": [13, 60, 64, 78, 153], "168": [13, 153, 329], "region1": 13, "rack1": 13, "machine1": 13, "machine2": 13, "hdfs01": 13, "region2": 13, "rack2": 13, "machine3": 13, "often": [13, 14, 16, 23, 40, 42, 43, 45, 48, 51, 55, 59, 62, 64, 66, 68, 71, 74, 75, 76, 104, 148, 152, 156, 285, 287, 463, 472, 474, 520], "reload": [13, 23, 33, 463, 472], "ipv4": [13, 40, 144, 396, 454], "ipv6": [13, 40, 144, 274, 396, 454], "mask": [13, 358, 359, 365, 372, 377, 408, 433], "128": [13, 55, 58, 67, 101, 111, 153, 380], "respect": [13, 23, 35, 37, 40, 42, 45, 55, 60, 62, 66, 68, 69, 71, 74, 75, 76, 123, 131, 136, 196, 197, 232, 259, 267, 270, 278, 288, 297, 303, 327, 339, 344, 346, 357, 358, 381, 386, 390, 396, 424, 426, 434, 445, 462, 512], "24": [13, 41, 55, 56, 58, 59, 76, 107, 153, 168, 204, 209, 293, 328, 506], "27": [13, 35, 58, 102, 117, 153, 204, 209, 328], "255": [13, 62, 66, 75, 103, 111, 186], "224": 13, "172": [13, 153], "160": [13, 153, 330], "dictionary_aggreg": [14, 187, 271], "optimize_hash_gener": [14, 345, 515, 523], "earli": [14, 94, 136, 146, 161, 217, 264, 265, 287, 327, 373, 375, 425], "later": [14, 24, 25, 29, 43, 45, 47, 56, 57, 64, 72, 84, 110, 123, 135, 245, 334, 352, 354, 367, 380, 382, 385, 388, 392, 441, 465, 513], "easier": [14, 89, 135, 171, 242, 282, 302, 463], "approx_distinct": [14, 70, 99, 110, 114, 115, 144, 191, 255, 270, 271, 272, 284, 289, 315, 317, 329, 373, 379, 388], "cardin": [14, 29, 35, 54, 99, 100, 109, 110, 114, 115, 117, 123, 246, 263, 271, 305, 307, 379, 433, 520], "speed": [14, 16, 19, 44, 48, 64, 76, 155, 297, 324], "main": [14, 27, 58, 64, 68, 136, 155, 296, 302, 510], "mark_distinct_strategi": 14, "markdistinct": [14, 94], "mix": [14, 33, 54, 112, 126, 144, 158, 220, 236, 242, 341, 347, 351, 446, 454, 456], "global": [14, 24, 32, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 51, 53, 54, 55, 56, 57, 59, 60, 62, 63, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 140, 144, 154, 157, 244, 274, 277, 329, 333, 374, 384, 434, 460, 471, 475], "direct": [14, 32, 35, 37, 39, 40, 41, 54, 60, 62, 65, 66, 68, 69, 71, 75, 76, 144, 170, 203, 209, 275, 276, 364, 447, 475, 520], "push_aggregation_through_join": [14, 242, 273], "particularli": [14, 54, 183, 271, 476, 510, 520], "scalar": [14, 112, 114, 126, 131, 186, 191, 197, 213, 220, 221, 229, 245, 270, 274, 282, 300, 356, 360, 383, 384, 410, 510, 532], "subqueri": [14, 29, 104, 113, 191, 196, 197, 217, 218, 220, 221, 225, 226, 227, 229, 237, 239, 240, 242, 243, 246, 248, 262, 267, 268, 269, 270, 273, 274, 313, 334, 339, 346, 355, 356, 360, 364, 366, 368, 369, 373, 375, 380, 381, 383, 384, 386, 387, 388, 389, 391, 394, 397, 408, 433, 453, 457, 462, 510], "get": [14, 15, 23, 25, 27, 32, 35, 37, 39, 54, 57, 59, 64, 65, 71, 73, 77, 83, 84, 94, 96, 104, 112, 123, 135, 136, 137, 148, 170, 173, 227, 243, 246, 303, 386, 395, 418, 459, 467, 468, 471, 472, 475, 510], "rewritten": [14, 155, 156, 288, 520], "item": [14, 72, 84, 99, 112, 120, 127, 137, 284, 287, 462, 493, 520], "i_current_pric": 14, "j": [14, 66, 99, 107], "i_categori": 14, "slow": [14, 19, 38, 48, 157, 246, 282, 397], "push_table_write_through_union": 14, "synchron": [14, 331, 334], "satur": [14, 18, 25], "heavili": [14, 39, 56, 192, 221, 303, 307], "eliminate_cross_join": [14, 146, 272, 356], "join_reordering_strategi": [14, 146, 272], "cross": [14, 29, 142, 146, 176, 177, 239, 245, 249, 268, 272, 275, 296, 302, 347, 371], "origin": [14, 27, 41, 43, 45, 57, 59, 73, 77, 83, 84, 94, 102, 112, 123, 139, 144, 212, 231, 274, 278, 293, 383, 392, 516, 524], "strive": 14, "stat": [14, 41, 45, 48, 54, 64, 78, 133, 143, 149, 155, 165, 178, 186, 188, 190, 210, 213, 217, 221, 228, 232, 242, 243, 249, 258, 259, 270, 276, 278, 280, 296, 297, 307, 323, 327, 328, 332, 337, 352, 353, 355, 359, 363, 364, 375, 379, 385, 387, 389, 391, 399, 425, 438, 439, 453, 477, 506], "9": [14, 18, 37, 39, 41, 55, 58, 64, 68, 78, 84, 102, 107, 112, 113, 117, 119, 144, 153, 264, 278, 331, 335, 339, 383, 505, 520], "factori": [14, 84, 86, 87, 217, 232, 424], "seriou": 14, "exactli": [14, 52, 60, 78, 84, 101, 108, 217, 415, 505, 509, 510, 512, 520], "downstream": [14, 415], "use_exact_partit": [14, 415], "minim": [14, 15, 32, 48, 50, 64, 76, 136, 146, 194, 248, 345, 408], "wide": [14, 43, 45, 54, 124, 136, 143, 397, 474], "reshuffl": [14, 358], "compar": [14, 15, 39, 40, 45, 48, 55, 62, 66, 75, 78, 97, 100, 104, 112, 123, 144, 147, 148, 154, 173, 175, 180, 236, 264, 270, 271, 279, 282, 310, 313, 315, 323, 326, 358, 373, 382, 399, 432, 436, 454, 486, 513], "colocated_join": [14, 256], "co": [14, 49, 106, 114, 115, 118, 209, 357, 469], "met": [14, 41, 112, 148], "scheme": [14, 40, 42, 49, 55, 57, 59, 62, 66, 68, 69, 70, 71, 73, 74, 75, 76, 136, 155, 409, 412, 447, 469], "75": [14, 43, 100, 112, 153, 329, 520], "strength": [14, 339], "assumpt": 14, "conserv": 14, "degre": [14, 109, 114, 115, 118, 154], "drive": [14, 18, 25, 48], "accur": [14, 41, 109, 183, 228, 392, 506], "presenc": [14, 104, 401], "previous": [14, 24, 32, 35, 41, 45, 54, 64, 83, 112, 155, 166, 168, 173, 182, 191, 204, 209, 210, 213, 225, 232, 233, 245, 248, 251, 253, 257, 258, 259, 261, 262, 263, 264, 267, 268, 270, 272, 273, 274, 275, 278, 280, 287, 293, 296, 302, 303, 330, 331, 332, 333, 334, 338, 341, 345, 346, 347, 348, 350, 354, 355, 356, 357, 359, 360, 361, 366, 373, 375, 379, 380, 381, 384, 388, 390, 391, 396, 401, 406, 410, 415, 416, 417, 420, 421, 427, 432, 441, 446, 449, 469], "1000000": [14, 60, 106, 126, 427], "lookup": [14, 32, 33, 35, 78, 180, 271, 279, 403], "0mb": 14, "min_input_size_per_task": 14, "min_hash_partition_count": [14, 15], "max_hash_partition_count": [14, 15], "10000000": 14, "min_input_rows_per_task": 14, "use_cost_based_partit": [14, 439], "repartit": [14, 181, 200, 206, 272, 389, 466, 505], "necessari": [14, 32, 40, 41, 42, 45, 47, 54, 55, 57, 58, 59, 62, 65, 66, 68, 69, 71, 74, 75, 76, 83, 96, 105, 106, 119, 139, 140, 146, 151, 293, 389, 433, 467, 475, 520], "contact": [15, 39, 223, 461, 464, 475], "cli": [15, 23, 33, 35, 40, 42, 52, 55, 58, 66, 68, 69, 71, 74, 75, 76, 83, 95, 135, 137, 155, 157, 158, 162, 180, 198, 254, 268, 293, 297, 305, 307, 320, 461, 462, 464, 467, 468, 471, 472, 473, 475], "cancel": [15, 161, 162, 173, 183, 210, 236, 244, 246, 247, 253, 270, 279, 296, 334, 345, 353, 367, 383, 384, 412, 424, 440, 448, 486], "phase": [15, 37, 77, 96, 197, 329, 359, 383, 392, 395, 396, 397, 433], "execution_polici": 15, "algorithm": [15, 16, 45, 69, 79, 80, 99, 109, 124, 146, 258, 276, 282, 301, 466, 469, 470, 475], "organ": [15, 96, 132, 151, 155, 475], "blockag": 15, "maxim": [15, 41, 45, 248, 371], "lowest": [15, 35, 57, 59, 146, 151, 510, 538], "wall": [15, 22, 132, 157, 192, 217, 307, 395, 396, 426, 506], "determine_partition_count_for_write_en": 15, "min_hash_partition_count_for_writr": 15, "max_writer_tasks_count": 15, "redistribut": [15, 146, 163, 193, 255, 274, 304, 357], "now": [15, 32, 38, 42, 45, 47, 50, 58, 72, 88, 107, 114, 115, 132, 137, 138, 148, 154, 157, 166, 167, 170, 173, 176, 177, 178, 182, 183, 186, 188, 193, 196, 198, 207, 213, 217, 219, 225, 227, 232, 233, 235, 239, 255, 257, 260, 263, 264, 265, 266, 270, 271, 274, 275, 278, 282, 284, 287, 288, 293, 294, 296, 297, 300, 301, 302, 303, 304, 305, 307, 309, 310, 313, 315, 316, 317, 323, 324, 325, 327, 328, 331, 334, 342, 346, 350, 351, 357, 361, 362, 368, 369, 374, 375, 379, 380, 381, 382, 385, 388, 389, 391, 392, 401, 419, 435, 436, 464, 465, 469, 475, 510, 524, 542], "signific": [15, 25, 45, 48, 57, 70, 84, 146, 148, 235, 282, 337, 520], "recov": [15, 25], "differnt": 15, "100d": 15, "query_max_execution_tim": [15, 24, 251], "analysi": [15, 36, 41, 85, 96, 152, 155, 186, 232, 263, 283, 310, 345, 353, 358, 377, 382, 436, 506, 520], "000": [15, 38, 41, 44, 45, 54, 56, 58, 60, 65, 70, 72, 100, 103, 107, 224, 259, 293], "query_text_too_larg": 15, "10m": [15, 19, 35, 59, 64, 72, 84], "query_max_planning_tim": 15, "easili": [15, 136, 217], "query_max_run_tim": [15, 170], "essenti": [15, 35, 88], "creation": [15, 38, 45, 65, 69, 85, 119, 157, 180, 188, 213, 224, 240, 261, 262, 264, 268, 273, 279, 287, 305, 315, 317, 320, 339, 353, 357, 359, 361, 367, 383, 395, 410, 438, 466, 486], "query_max_scan_physical_byt": [15, 366, 368], "reach": [15, 16, 23, 41, 44, 84, 198, 383, 397, 468], "150": [15, 23, 153, 408], "query_has_too_many_stag": 15, "introduc": [15, 25, 37, 64, 155, 156, 170, 183, 227, 242, 257, 259, 265, 274, 275, 279, 286, 287, 291, 295, 334, 347, 351, 358, 383, 386, 387, 394, 397, 408, 410, 411, 423, 451, 510], "unrel": [15, 421], "remote_task_error": [15, 194], "httpdestin": 15, "15m": [15, 37, 470], "query_remote_task_enable_adaptive_request_s": 15, "schema": [15, 23, 24, 29, 32, 33, 35, 36, 37, 38, 42, 43, 44, 50, 52, 55, 56, 58, 62, 63, 64, 66, 68, 69, 70, 75, 76, 77, 78, 83, 84, 86, 96, 112, 122, 125, 126, 132, 133, 136, 137, 142, 154, 156, 169, 173, 180, 208, 209, 217, 219, 221, 232, 242, 243, 247, 248, 258, 261, 263, 272, 274, 275, 278, 281, 284, 288, 293, 303, 316, 336, 337, 340, 341, 343, 344, 345, 346, 347, 348, 350, 352, 355, 356, 357, 358, 359, 360, 362, 365, 366, 367, 371, 373, 375, 377, 380, 381, 382, 383, 387, 388, 389, 391, 392, 394, 395, 396, 399, 401, 402, 403, 410, 413, 415, 418, 419, 420, 421, 423, 427, 428, 430, 432, 433, 434, 440, 444, 446, 450, 451, 452, 453, 456, 469, 471, 477, 478, 480, 482, 483, 490, 494, 497, 505, 507, 517, 521, 532, 533, 539, 543], "advanc": [15, 32, 36, 39, 60, 87, 138, 285, 293, 297, 408], "extrem": [15, 45, 229, 296, 337], "query_remote_task_guaranteed_splits_per_task": 15, "unabl": [15, 32], "8mb": [15, 321], "query_remote_task_max_request_s": 15, "made": [15, 40, 41, 42, 54, 55, 57, 59, 62, 66, 68, 69, 70, 71, 74, 75, 76, 83, 132, 144, 146, 262, 287, 293, 389, 462, 468, 469, 470, 510, 523, 524], "query_remote_task_request_size_headroom": 15, "alloc": [15, 17, 19, 22, 25, 41, 57, 70, 136, 157, 167, 246, 256, 259, 262, 266, 279, 304, 309, 311, 313, 320, 321, 325, 334, 381], "redirect": [15, 32, 358, 374, 380, 383, 385, 388, 393, 396, 397, 399, 400, 402, 403, 407, 413, 432, 433, 454, 468, 470], "placehold": [15, 45, 49, 52, 328], "query_id": [15, 40, 42, 55, 66, 68, 69, 71, 74, 75, 76, 77], "actual": [15, 24, 40, 41, 42, 45, 54, 55, 58, 62, 66, 68, 69, 70, 71, 74, 75, 76, 84, 86, 109, 136, 137, 147, 196, 198, 248, 259, 273, 282, 303, 352, 402, 403, 469, 474, 510, 520], "joni": [16, 19, 332, 336], "certain": [16, 19, 24, 40, 42, 45, 55, 66, 68, 69, 71, 74, 75, 76, 105, 108, 112, 144, 158, 166, 179, 180, 194, 217, 218, 222, 223, 227, 229, 231, 232, 233, 234, 236, 240, 242, 245, 246, 247, 250, 255, 270, 271, 278, 279, 284, 287, 294, 297, 302, 303, 317, 329, 332, 341, 342, 345, 346, 347, 349, 352, 354, 355, 357, 358, 359, 361, 368, 371, 373, 374, 375, 379, 381, 384, 385, 386, 388, 389, 390, 392, 393, 394, 398, 399, 401, 410, 416, 417, 419, 420, 424, 429, 430, 433, 437, 441, 454, 455, 464, 510, 512, 520], "linear": [16, 99, 112, 113, 223, 305], "2147483647": [16, 19, 55, 76, 100, 190, 380, 381, 391], "fast": [16, 23, 27, 48, 70, 156, 282, 371, 374], "potenti": [16, 27, 40, 42, 45, 55, 62, 66, 68, 69, 71, 74, 75, 76, 156, 158, 198, 200, 206, 208, 210, 213, 217, 218, 220, 224, 228, 240, 246, 250, 255, 273, 274, 278, 334, 335, 354, 364, 367, 371, 377, 380, 386, 390, 399, 407, 408, 409, 411, 413, 416, 417, 418, 419, 420, 421, 424, 425, 426, 428, 430, 432, 433, 434, 437, 438, 440, 451, 454, 455, 462, 486, 520], "determinist": [16, 79, 80, 88, 100, 194, 242, 320, 396, 438, 520, 532], "finit": [16, 118], "automaton": 16, "fall": [16, 25, 103, 104, 297, 317, 449], "back": [16, 25, 35, 37, 41, 57, 58, 72, 77, 78, 83, 84, 93, 94, 132, 171, 209, 225, 245, 247, 282, 297, 301, 317, 372, 389, 390, 398, 399, 449, 450, 459, 464, 470, 478, 480, 510, 516], "nfa": 16, "footprint": [16, 18, 25, 123, 144, 454], "futur": [16, 47, 64, 94, 112, 137, 154, 155, 157, 170, 171, 212, 217, 221, 240, 242, 251, 260, 293, 301, 302, 303, 305, 354, 356, 357, 359, 378, 388, 389, 419, 435, 455, 486, 491, 494, 507, 510], "outlier": 16, "want": [16, 29, 33, 35, 41, 47, 66, 68, 71, 74, 76, 84, 88, 99, 104, 119, 135, 137, 283, 352, 460, 462, 465, 468, 469, 474, 476, 506, 512], "correct": [16, 40, 41, 42, 45, 52, 55, 57, 62, 65, 66, 68, 70, 71, 73, 75, 76, 106, 112, 124, 137, 156, 173, 178, 186, 191, 198, 203, 209, 217, 220, 226, 227, 229, 233, 234, 237, 239, 242, 247, 255, 257, 261, 267, 274, 275, 281, 282, 297, 301, 317, 335, 350, 354, 355, 366, 375, 381, 392, 398, 419, 422, 432, 464, 468, 475, 505, 510], "1_000_000_000d": 17, "attribut": [17, 40, 42, 55, 57, 58, 59, 62, 66, 68, 69, 71, 73, 74, 75, 76, 257, 313, 393, 462, 469, 527], "sum": [17, 40, 55, 58, 62, 66, 68, 70, 71, 76, 78, 88, 99, 100, 109, 110, 113, 114, 115, 131, 148, 220, 240, 268, 271, 283, 364, 368, 369, 379, 384, 388, 416, 438, 439, 486, 490, 491, 512, 520], "polici": [17, 23, 37, 39, 41, 43, 45, 51, 54, 64, 65, 66, 68, 71, 74, 76, 136, 137, 138, 209, 254, 256, 257, 262, 263, 276, 303, 317, 324, 331, 346, 359, 384, 393, 395, 396, 397, 400, 401, 403, 406, 407, 408, 413, 414, 417, 430, 433, 449, 462], "20gb": 17, "revoc": [17, 25, 258, 332, 333, 364], "asid": 17, "track": [17, 54, 83, 84, 88, 151, 157, 178, 188, 191, 210, 241, 246, 255, 257, 258, 261, 269, 274, 275, 279, 288, 313, 345, 415, 418, 420, 429, 434, 475, 489, 491, 510], "spill_en": 18, "offload": [18, 25], "outer": [18, 45, 70, 142, 156, 179, 183, 186, 205, 209, 212, 217, 219, 228, 242, 260, 267, 269, 270, 275, 278, 331, 334, 335, 346, 347, 355, 359, 364, 375, 383, 385, 388, 416, 433, 453, 520], "No": [18, 23, 32, 35, 37, 39, 40, 41, 42, 43, 44, 52, 54, 55, 57, 59, 60, 62, 65, 66, 68, 69, 70, 71, 72, 73, 75, 233, 271, 274, 423, 470, 474], "simultan": [18, 43, 51, 55, 62, 66, 68, 71, 74, 75, 76, 96], "instal": [18, 27, 36, 40, 45, 57, 66, 71, 74, 75, 82, 85, 89, 90, 92, 93, 95, 132, 133, 135, 137, 138, 151, 264, 268, 334, 342, 356, 360, 378, 389, 459, 464, 471, 475, 476], "importantli": [18, 112], "overutil": 18, "lengthi": 18, "ratio": [18, 35, 48, 264, 368, 382], "elig": [18, 23, 240, 263, 274, 364, 368, 369, 377, 384], "devic": [18, 25, 38, 48, 132, 466], "raid": [18, 25], "100gb": 18, "instanc": [18, 23, 33, 35, 51, 55, 59, 64, 65, 66, 68, 71, 72, 74, 76, 78, 82, 84, 85, 89, 90, 92, 93, 94, 95, 119, 121, 124, 136, 138, 151, 155, 201, 208, 217, 245, 303, 346, 359, 375, 470, 506], "randomli": [18, 20, 25, 78, 130, 162, 211], "decrypt": [18, 47, 51], "restrict": [19, 27, 136, 259, 277, 302, 327, 377, 387, 391, 392, 462, 469, 471, 475, 491, 500, 520], "power": [19, 45, 112, 114, 115, 118, 169, 217, 264, 510, 520], "physic": [19, 22, 57, 59, 73, 137, 151, 155, 180, 256, 329, 331, 357, 358, 366, 368, 374, 387, 392, 396, 406, 435, 486, 506, 514, 520], "task_concurr": [19, 208], "few": [19, 27, 37, 52, 80, 93, 113, 121, 136, 157, 247, 282, 293, 330, 339, 474], "demand": [19, 23, 137], "clean": [19, 41, 210, 243, 317], "idl": [19, 25, 77, 78, 157, 187, 206, 341, 343, 364, 367, 378, 398], "hundr": [19, 320, 321], "thousand": [19, 297, 309], "frequent": [19, 35, 41, 97, 99, 193], "asynchttpexecutionmbean": 19, "timeoutexecutor": 19, "poolsiz": 19, "3s": 19, "stale": [19, 23, 77, 257, 392], "suboptim": 19, "likelihood": [19, 243], "kept": [19, 38, 65, 474], "flush": [19, 40, 42, 45, 55, 66, 68, 69, 71, 74, 75, 76, 394, 396, 402, 425, 438], "runningsplit": 19, "executor": [19, 37, 287], "taskexecutor": [19, 326], "leaf": [19, 193, 280], "underutil": 19, "task_min_writer_count": [19, 455], "advantag": [19, 37, 40, 42, 45, 55, 62, 65, 66, 68, 69, 71, 72, 74, 75, 76, 144, 146, 149, 151, 166, 239, 240, 245, 255, 267, 295, 364], "bottleneck": [19, 22, 27, 466], "becom": [19, 23, 25, 45, 59, 63, 65, 89, 112, 144, 158, 178, 362, 466, 476, 510], "overload": [19, 351, 384, 423, 427], "engin": [19, 37, 39, 40, 41, 42, 43, 45, 62, 66, 68, 71, 74, 76, 78, 83, 84, 88, 91, 94, 96, 123, 126, 132, 135, 137, 140, 143, 150, 151, 206, 242, 284, 287, 293, 302, 305, 382, 383, 393, 420, 432, 433, 512], "task_max_writer_count": [19, 455], "third": [19, 23, 112, 146, 151, 292, 475, 520, 536], "parti": [19, 151, 292], "librari": [19, 49, 68, 88, 93, 132, 139, 336, 344, 468], "print": [19, 32, 107, 136, 139, 147, 162, 167, 173, 259, 280, 293, 338, 375, 384], "maxactivesplit": 19, "3m": 19, "length": [19, 38, 54, 57, 59, 60, 62, 68, 69, 73, 78, 99, 100, 101, 109, 112, 114, 115, 124, 144, 183, 184, 191, 231, 244, 257, 269, 272, 277, 278, 313, 331, 335, 349, 373, 383, 395, 396, 399, 416, 420, 421, 432, 438, 470], "2m": [19, 32, 70, 473], "check": [19, 33, 37, 40, 42, 43, 45, 52, 55, 59, 62, 66, 68, 69, 71, 74, 75, 76, 84, 94, 95, 96, 112, 124, 137, 148, 156, 172, 186, 191, 198, 209, 232, 237, 239, 242, 244, 246, 250, 257, 262, 264, 268, 270, 274, 277, 280, 320, 334, 339, 344, 346, 351, 358, 371, 374, 386, 392, 398, 399, 423, 432, 433, 434, 439, 442, 448, 452, 454, 456, 462, 465, 469, 470, 473], "form": [20, 27, 58, 60, 72, 78, 83, 84, 96, 97, 99, 104, 105, 109, 112, 124, 144, 176, 180, 242, 248, 275, 277, 287, 297, 331, 358, 366, 462, 476, 510, 512, 520], "fix": [20, 22, 54, 55, 57, 59, 65, 69, 71, 72, 73, 107, 112, 140, 154, 155, 156, 157, 158, 159, 160, 161, 162, 163, 164, 166, 167, 168, 169, 171, 172, 173, 175, 176, 177, 178, 179, 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, 190, 191, 192, 193, 194, 195, 196, 197, 198, 199, 200, 201, 202, 203, 204, 206, 207, 208, 210, 211, 212, 213, 214, 216, 217, 218, 219, 220, 221, 222, 223, 225, 226, 227, 228, 229, 230, 231, 232, 233, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, 276, 277, 278, 279, 280, 284, 285, 286, 288, 289, 291, 292, 294, 295, 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, 319, 320, 321, 322, 323, 324, 325, 326, 327, 328, 329, 330, 331, 332, 333, 334, 335, 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, 377, 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, 442, 445, 446, 447, 448, 449, 450, 451, 452, 453, 454, 455, 456, 457, 462, 469, 505, 520, 538], "kerbero": [20, 27, 33, 45, 48, 64, 133, 196, 209, 225, 232, 245, 251, 254, 263, 268, 271, 272, 275, 329, 335, 336, 339, 350, 353, 357, 361, 371, 391, 392, 399, 421, 434, 438, 450, 458, 459, 462, 471, 476], "cooki": [20, 83, 379], "random": [20, 41, 54, 88, 100, 114, 115, 186, 339, 361, 388, 419, 466, 468, 470, 520], "turn": [20, 23, 31, 58, 67, 71, 84, 86, 100, 151, 165, 184, 187, 220, 227, 242, 257, 258, 263, 272, 293, 307, 310, 342, 345, 393, 470], "invalid": [20, 33, 57, 59, 73, 84, 105, 109, 112, 124, 142, 155, 156, 166, 178, 191, 196, 210, 213, 229, 233, 242, 248, 257, 259, 261, 268, 272, 278, 279, 287, 323, 337, 360, 368, 373, 375, 388, 389, 403, 405, 422, 424, 438, 466, 469, 505], "1d": [20, 72], "forc": [20, 25, 32, 40, 41, 42, 45, 51, 54, 62, 66, 68, 69, 70, 71, 74, 75, 76, 210, 218, 253, 254, 297, 306, 344, 345, 368, 390, 417, 418, 426, 525], "usernam": [20, 23, 27, 33, 35, 37, 39, 41, 51, 52, 64, 70, 72, 73, 77, 92, 95, 232, 259, 261, 266, 320, 334, 368, 408, 440, 462, 469, 472, 473, 474, 475, 476, 524], "use_preferred_write_partit": [21, 357], "instead": [21, 23, 32, 35, 37, 40, 41, 45, 48, 49, 51, 54, 57, 59, 62, 63, 64, 66, 68, 69, 71, 74, 75, 76, 84, 88, 105, 107, 112, 121, 132, 135, 144, 146, 148, 155, 162, 170, 173, 186, 198, 204, 205, 209, 217, 223, 227, 228, 233, 247, 253, 257, 260, 263, 270, 276, 293, 301, 302, 303, 305, 309, 321, 325, 327, 331, 333, 334, 339, 346, 379, 380, 381, 384, 389, 392, 395, 396, 397, 402, 419, 425, 432, 434, 444, 445, 450, 454, 456, 461, 462, 465, 468, 475, 491, 505, 518, 524], "ramp": [22, 256], "scale_writ": [22, 24, 256], "task_scale_writers_en": 22, "writer_scaling_min_data_process": 22, "divid": [23, 25, 72, 96, 112, 118, 131, 287, 342, 520], "among": [23, 58, 78, 94, 135], "sub": [23, 24, 100, 112, 186, 201, 206, 262, 263, 419, 467, 470, 510], "belong": [23, 35, 45, 51, 54, 60, 76, 123, 126, 144, 359, 462, 469], "ancestor": [23, 54], "pluggabl": [23, 24, 35, 260, 361], "absolut": [23, 24, 32, 41, 45, 57, 112, 118, 246, 270, 459, 461, 476], "localhost": [23, 32, 33, 35, 43, 58, 60, 64, 72, 129, 135, 137, 464], "resource_group": 23, "password": [23, 33, 35, 37, 39, 40, 42, 51, 52, 55, 62, 64, 66, 68, 70, 71, 72, 74, 75, 76, 81, 93, 133, 137, 232, 251, 260, 268, 280, 320, 334, 343, 344, 349, 354, 355, 356, 357, 358, 360, 364, 366, 368, 379, 382, 386, 399, 408, 419, 432, 434, 458, 461, 464, 465, 467, 468, 471, 473, 474, 475, 476], "resource_groups_global_properti": 23, "descend": [23, 99, 100, 510], "prioriti": [23, 246, 255, 270, 493], "reflect": [23, 44, 45, 57, 59, 73, 183, 233], "incom": [23, 476], "refresh": [23, 40, 42, 43, 47, 51, 54, 55, 57, 62, 64, 66, 68, 69, 71, 74, 75, 76, 84, 133, 143, 155, 161, 167, 175, 235, 239, 242, 257, 278, 279, 328, 343, 346, 358, 375, 383, 384, 386, 395, 396, 408, 415, 420, 424, 430, 432, 463, 472, 477, 478, 486, 498, 505, 528], "interv": [23, 36, 41, 43, 45, 57, 64, 72, 78, 84, 99, 100, 110, 118, 155, 193, 204, 210, 217, 239, 240, 242, 247, 255, 257, 279, 282, 293, 296, 346, 358, 366, 384, 408, 432, 486, 525], "continu": [23, 25, 32, 35, 52, 54, 78, 84, 119, 136, 138, 241, 249, 251, 254, 256, 259, 278, 287, 342, 397], "1h": [23, 24, 64, 72, 469, 470], "exact": [23, 44, 45, 68, 88, 112, 144, 157, 229, 245, 255, 256, 262, 389, 462, 510, 520], "exact_match_source_selector": 23, "templat": [23, 45, 137, 383], "maxqueu": 23, "reject": [23, 32, 169, 210, 247, 358, 388, 391, 420, 475, 494], "softconcurrencylimit": 23, "peer": [23, 39, 131, 379, 390, 520], "inelig": 23, "hardconcurrencylimit": [23, 251], "softmemorylimit": 23, "percentag": [23, 27, 36, 48, 67, 99, 131, 136, 178, 266, 278, 362, 368, 423, 450, 520], "softcpulimit": 23, "cpuquotaperiod": 23, "penalti": 23, "hardcpulimit": 23, "schedulingpolici": 23, "fair": [23, 25, 246, 345], "weighted_fair": [23, 254, 257, 263], "schedulingweight": 23, "stochast": 23, "proport": [23, 249], "query_prior": 23, "strictli": [23, 112], "jmxexport": 23, "subgroup": 23, "method": [23, 27, 32, 33, 35, 48, 50, 57, 70, 78, 82, 84, 85, 86, 87, 88, 89, 90, 91, 92, 93, 94, 97, 112, 132, 154, 177, 183, 185, 188, 213, 217, 232, 240, 245, 251, 258, 262, 264, 270, 272, 287, 293, 317, 330, 333, 334, 336, 339, 341, 342, 343, 344, 345, 355, 359, 362, 366, 367, 369, 373, 374, 375, 378, 379, 380, 382, 385, 387, 388, 397, 398, 402, 408, 413, 418, 422, 424, 430, 432, 433, 438, 440, 448, 450, 451, 453, 459, 461, 466, 467, 471, 476, 520], "pipelin": [23, 24, 73, 152, 337, 339, 344, 354], "adhoc": 23, "approxim": [23, 59, 112, 120, 123, 127, 144, 219, 245, 260, 284, 291, 307, 311, 386, 398, 538], "70": [23, 107, 136, 153, 298, 328], "come": [23, 72, 94, 99, 117, 170, 225, 293], "timefram": 23, "evenli": [23, 131, 287], "usergroup": 23, "querytyp": [23, 24, 264], "materi": [23, 30, 37, 51, 64, 84, 133, 217, 269, 337, 350, 352, 354, 361, 373, 381, 382, 383, 384, 385, 386, 387, 388, 389, 391, 394, 395, 396, 400, 404, 406, 408, 412, 415, 420, 424, 432, 434, 438, 450, 452, 462, 477], "view": [23, 24, 27, 29, 32, 38, 40, 41, 53, 62, 64, 66, 69, 71, 74, 75, 76, 84, 133, 155, 170, 173, 184, 191, 206, 208, 213, 217, 223, 229, 242, 250, 257, 262, 264, 269, 270, 272, 279, 327, 328, 329, 331, 332, 333, 341, 342, 350, 351, 354, 357, 358, 359, 360, 365, 368, 371, 372, 373, 374, 375, 377, 379, 380, 382, 383, 384, 385, 386, 387, 388, 389, 391, 394, 395, 396, 399, 400, 403, 404, 405, 406, 407, 408, 412, 413, 415, 417, 419, 420, 423, 424, 425, 426, 427, 429, 432, 434, 437, 438, 444, 447, 450, 452, 461, 462, 465, 468, 469, 477, 484, 488, 510], "data_definit": [23, 24], "prepar": [23, 24, 83, 87, 126, 133, 142, 143, 217, 218, 226, 250, 270, 275, 329, 341, 347, 358, 359, 379, 384, 392, 399, 426, 475, 477, 496, 497, 503, 504, 505], "privileg": [23, 24, 52, 68, 270, 335, 358, 360, 364, 383, 386, 392, 393, 447, 462, 494, 499, 507, 517], "transact": [23, 24, 29, 41, 45, 53, 64, 83, 91, 94, 96, 133, 140, 152, 186, 187, 209, 217, 218, 239, 253, 254, 276, 278, 346, 354, 355, 367, 371, 379, 383, 386, 387, 389, 390, 391, 392, 396, 397, 398, 403, 406, 408, 409, 412, 415, 430, 432, 439, 444, 452, 453, 454, 477, 485, 519], "clienttag": [23, 24, 33, 259, 357], "tag": [23, 24, 26, 32, 33, 54, 64, 83, 135, 137, 255, 259, 357, 362, 454], "sequenti": [23, 131, 510], "quota": [23, 211], "app": [23, 47, 84, 86, 137], "program": [23, 32, 58, 83, 242, 268], "shown": [23, 32, 41, 45, 54, 57, 59, 60, 103, 104, 112, 116, 282, 465, 505, 510, 520, 524], "sever": [23, 37, 41, 54, 56, 60, 79, 80, 95, 123, 132, 135, 143, 146, 148, 151, 204, 209, 263, 293, 300, 457, 461, 467, 471], "administr": [23, 24, 41, 45, 54, 60, 82, 85, 90, 92, 95, 133, 136, 151, 152, 462, 464, 467, 474, 475, 483], "construct": [23, 24, 29, 45, 94, 100, 109, 117, 119, 144, 209, 226, 263, 271, 278, 293, 380, 436, 520], "tree": [23, 60, 86, 87, 93, 99, 112, 147, 151, 520], "pipeline_": 23, "expand": [23, 232, 264, 271, 280, 293, 330, 334, 438, 520], "custom": [23, 32, 35, 40, 42, 43, 45, 54, 55, 57, 58, 59, 60, 62, 64, 66, 68, 69, 71, 73, 74, 75, 76, 78, 82, 85, 89, 90, 92, 94, 95, 96, 99, 112, 126, 135, 137, 144, 148, 213, 217, 234, 245, 275, 282, 284, 293, 303, 305, 330, 341, 346, 394, 395, 398, 439, 456, 460, 464, 471, 475, 482, 486, 510, 511, 520, 532, 542], "There": [23, 25, 27, 33, 35, 41, 43, 45, 53, 57, 58, 59, 65, 69, 71, 93, 96, 99, 112, 124, 126, 135, 136, 146, 151, 156, 157, 176, 183, 242, 256, 287, 297, 301, 303, 305, 307, 352, 461, 462, 467, 471, 473, 475, 476, 510, 512, 520, 523], "four": [23, 57, 59, 65, 73, 107, 136, 470, 497, 510], "bob": [23, 51, 117, 462, 476, 487, 494, 507, 517], "admin": [23, 52, 60, 64, 447, 462, 464, 468, 469, 473, 476, 487, 499, 508, 518], "definit": [23, 35, 37, 43, 45, 51, 54, 58, 60, 64, 67, 77, 78, 84, 94, 132, 144, 151, 152, 155, 191, 236, 342, 351, 366, 368, 379, 380, 384, 386, 420, 429, 438, 479, 480, 481, 486, 489, 512, 520], "ddl": [23, 35, 41, 45, 55, 371, 398], "class": [23, 35, 39, 45, 51, 64, 83, 84, 86, 87, 88, 93, 94, 96, 121, 136, 188, 204, 209, 213, 217, 232, 234, 253, 257, 262, 270, 283, 287, 296, 305, 327, 330, 334, 342, 344, 345, 358, 368, 375, 378, 385, 397, 409, 413, 418, 435], "fourth": [23, 112, 520], "under": [23, 24, 30, 41, 45, 52, 54, 57, 59, 61, 73, 132, 161, 246, 248, 252, 260, 274, 297, 305, 361, 420, 466], "fifth": 23, "bi": [23, 140], "toolnam": 23, "superset": [23, 78], "hipri": 23, "powerfulbi": 23, "kayla": 23, "rout": [23, 24, 70, 94, 274], "catch": [23, 287, 396], "fifo": 23, "45": [23, 102, 107, 506], "fewest": 23, "slot": 23, "behav": [23, 38, 45, 54, 60, 84, 112], "rootgroup": 23, "80": [23, 48, 103, 136, 153, 329], "INTO": [23, 29, 35, 38, 41, 57, 60, 63, 65, 68, 94, 110, 142, 462, 505, 509, 511, 513, 520], "cpu_quota_period": 23, "enviro": 23, "test_environ": 23, "sure": [23, 35, 39, 41, 45, 47, 57, 65, 66, 71, 73, 74, 75, 76, 78, 137, 204, 209, 302, 464, 465, 469, 475], "parent": [23, 37, 54, 57, 146, 296, 359], "child": [23, 112], "relationship": 23, "root": [23, 33, 35, 42, 48, 50, 59, 62, 66, 68, 69, 71, 74, 75, 76, 118, 132, 137, 151, 426, 447, 469, 475, 476], "null": [23, 29, 32, 35, 38, 40, 41, 42, 45, 55, 57, 59, 60, 62, 66, 68, 69, 70, 71, 73, 74, 75, 76, 78, 83, 84, 88, 94, 96, 99, 100, 105, 106, 109, 113, 114, 117, 120, 123, 124, 126, 131, 142, 144, 148, 149, 154, 156, 167, 171, 172, 180, 183, 186, 191, 192, 194, 198, 207, 210, 212, 213, 217, 218, 220, 223, 227, 228, 229, 242, 245, 257, 261, 263, 264, 266, 267, 268, 269, 270, 271, 274, 276, 278, 287, 293, 297, 299, 301, 304, 305, 307, 308, 310, 316, 320, 326, 327, 328, 331, 333, 339, 341, 342, 346, 359, 360, 365, 368, 369, 372, 373, 386, 389, 394, 397, 400, 402, 403, 405, 409, 417, 418, 419, 420, 421, 430, 432, 433, 434, 435, 438, 441, 448, 450, 480, 484, 489, 509, 510, 512, 520, 538, 542], "soft_memory_limit": 23, "hard_concurrency_limit": 23, "max_queu": 23, "scheduling_polici": 23, "jmx_export": 23, "resource_group_id": 23, "scheduling_weight": 23, "nest": [23, 40, 43, 45, 54, 112, 119, 148, 156, 178, 182, 217, 226, 227, 247, 274, 310, 337, 338, 344, 357, 358, 359, 360, 361, 366, 367, 368, 379, 380, 383, 389, 391, 392, 397, 408, 432, 435, 436, 439, 440, 449, 450, 451, 462, 520], "user_regex": 23, "user_group_regex": 23, "source_regex": 23, "query_typ": 23, "disambigu": [23, 40, 42, 55, 62, 66, 68, 69, 71, 74, 75, 76], "self_id": 23, "parent_id": [23, 54], "concat": [23, 100, 101, 114, 115, 117, 124, 167, 186, 187, 233, 239], "name_with_par": 23, "client_tag": 23, "characterist": [24, 96, 140, 276, 465], "point": [24, 29, 37, 39, 41, 44, 45, 47, 48, 53, 54, 57, 59, 69, 70, 73, 78, 86, 93, 99, 109, 123, 124, 132, 136, 137, 140, 156, 161, 198, 223, 254, 261, 266, 277, 278, 296, 303, 329, 339, 365, 375, 389, 461, 462, 466, 510, 538], "meet": [24, 52, 54, 76, 276], "contribut": [24, 30, 132, 151], "overrid": [24, 32, 33, 39, 45, 51, 84, 86, 88, 93, 96, 126, 136, 137, 254, 272, 273, 305, 344, 345, 360, 385, 387, 462, 470, 475, 520], "encount": [24, 70, 105, 112, 180, 279, 434, 449, 469], "qualifi": [24, 35, 41, 43, 45, 51, 54, 57, 59, 60, 64, 73, 96, 126, 136, 151, 173, 206, 220, 242, 246, 263, 282, 284, 331, 373, 381, 466, 483, 521], "sessionproperti": [24, 33, 357], "entri": [24, 41, 45, 51, 54, 56, 65, 84, 93, 112, 117, 167, 218, 257, 264, 267, 276, 435, 444, 459, 461, 464, 468, 520], "matter": [24, 44, 468], "tighter": 24, "constraint": [24, 37, 60, 71, 78, 84, 112, 142, 272, 274, 278, 282, 334, 379, 399, 405, 420, 421, 423, 433, 434, 439, 442, 448, 452], "etl": [24, 137, 140], "8h": 24, "writer_min_s": [24, 256], "insert_existing_partitions_behavior": [24, 29, 45, 269, 400], "overwrit": [24, 45, 54, 269, 341, 352, 368, 369, 390, 391, 400, 428], "goal": [25, 86], "os": [25, 33, 45], "swap": [25, 54, 136, 291, 354], "query_max_memori": [25, 161], "query_max_memory_per_nod": [25, 345], "lot": [25, 45, 146, 232, 287, 303, 401], "stai": [25, 146], "overcom": 25, "ineffici": [25, 45], "concept": [25, 84, 95, 133, 143, 150, 296], "wa": [25, 35, 36, 39, 41, 44, 45, 54, 56, 57, 58, 59, 63, 73, 77, 83, 84, 88, 94, 112, 132, 148, 152, 155, 156, 196, 204, 209, 210, 217, 218, 226, 238, 242, 255, 258, 260, 261, 262, 263, 265, 267, 268, 269, 270, 274, 278, 279, 282, 287, 291, 293, 295, 296, 302, 303, 307, 309, 324, 328, 331, 332, 334, 345, 346, 347, 351, 356, 357, 359, 360, 366, 368, 369, 373, 375, 379, 380, 381, 384, 388, 391, 396, 400, 410, 415, 416, 417, 420, 424, 426, 427, 438, 462, 465, 469, 470], "toward": [25, 68, 94], "revok": [25, 52, 133, 143, 209, 217, 244, 368, 373, 389, 392, 402, 477, 487, 494, 499, 505, 507, 508, 522, 533], "runner": [25, 520], "On": [25, 37, 44, 45, 48, 54, 60, 72, 88, 94, 136, 157, 167, 475], "hand": [25, 35, 48, 88, 227, 242, 313, 520], "magnitud": [25, 156], "chunk": [25, 48, 72, 84, 86], "retriev": [25, 35, 36, 37, 41, 45, 46, 47, 48, 51, 53, 54, 57, 59, 62, 65, 66, 71, 76, 93, 94, 112, 117, 120, 127, 132, 144, 151, 239, 269, 305, 342, 344, 380, 420, 446, 464, 470, 474, 486, 510, 512, 514, 520], "expens": [25, 27, 35, 45, 54, 84, 109, 144, 251, 263, 320, 321], "term": [25, 57, 107, 132, 151, 233, 275, 313, 434, 467, 475, 512], "io": [25, 32, 33, 35, 41, 51, 57, 58, 83, 84, 93, 136, 137, 250, 274, 280, 327, 329, 334, 342, 345, 346, 359, 375, 378, 380, 385, 395, 413, 420, 424, 447, 467, 469, 470], "throttl": [25, 53, 175, 341], "spiller": 25, "independ": [25, 32, 54, 78, 84, 99, 126, 173, 204, 209, 464, 468, 475, 510, 520], "jbod": 25, "decompress": [25, 132, 243, 278, 320, 324], "peak": [25, 191, 231, 256, 257, 258, 259, 261, 264, 267, 272, 329, 330, 332, 333, 338, 340, 364, 392, 432], "those": [25, 33, 37, 51, 54, 56, 57, 65, 83, 84, 94, 100, 117, 138, 183, 259, 327, 390, 507, 512, 517, 520, 538], "influenc": 25, "finish": [25, 27, 32, 37, 83, 91, 94, 135, 137, 160, 161, 169, 220, 232, 259, 264, 267, 287, 322, 346, 380, 420, 472, 475], "largest": [25, 41, 45, 99, 104, 108, 146, 149, 170, 256, 339], "problem": [26, 32, 45, 49, 54, 70, 83, 119, 204, 209, 287, 332, 347, 468, 505], "diagnos": [26, 347], "garbag": [26, 136, 209, 273, 297, 303, 315], "xlog": 26, "safepoint": 26, "tid": 26, "login": [27, 32, 266, 469, 470, 472, 473], "permiss": [27, 37, 39, 41, 44, 51, 52, 83, 95, 173, 209, 217, 228, 237, 264, 268, 270, 274, 317, 357, 359, 371, 374, 391, 393, 442, 462, 470, 487, 491, 518], "ldap": [27, 32, 33, 133, 232, 260, 328, 355, 356, 359, 360, 374, 386, 399, 403, 408, 458, 459, 462, 471, 472, 473, 476], "displai": [27, 32, 41, 53, 54, 141, 147, 149, 158, 173, 176, 183, 189, 213, 239, 242, 249, 256, 259, 269, 289, 297, 329, 332, 333, 335, 345, 351, 357, 358, 362, 368, 390, 391, 395, 396, 403, 432, 446, 450, 472, 486, 520, 532], "uniqu": [27, 41, 49, 54, 60, 65, 78, 94, 99, 100, 110, 124, 131, 136, 144, 263, 293, 388, 419, 446, 520], "top": [27, 32, 44, 55, 62, 66, 68, 70, 71, 74, 75, 76, 93, 94, 99, 136, 155, 164, 219, 279, 288, 302, 327, 380, 416, 432, 462, 472, 476, 520, 544], "recent": [27, 77, 267, 293, 373, 433], "await": [27, 287, 369], "commit": [27, 41, 45, 53, 64, 84, 91, 94, 133, 143, 178, 209, 219, 253, 257, 317, 355, 391, 397, 403, 406, 424, 425, 429, 477, 505, 519, 540], "autocommit": [27, 355], "persist": [27, 112], "investig": 27, "insuffici": [27, 248, 270, 421], "goe": [27, 32, 281, 461, 468, 510], "computation": 27, "simpli": [27, 35, 39, 45, 48, 54, 57, 65, 66, 71, 73, 74, 75, 76, 78, 84, 88, 104, 151, 287, 473, 491], "click": [27, 40, 44, 54, 62, 66, 68, 69, 71, 74, 75, 76, 136, 249], "link": [27, 47, 54, 59, 73, 83, 268, 374], "summari": [27, 54, 84, 99, 144, 189, 232, 248, 384, 386, 490, 538], "graphic": 27, "represent": [27, 57, 78, 84, 87, 99, 101, 102, 108, 109, 110, 112, 118, 124, 128, 144, 151, 167, 316, 325, 342, 372, 375, 392, 393, 395, 418], "variou": [27, 30, 33, 35, 52, 73, 83, 86, 88, 93, 125, 137, 139, 186, 212, 219, 243, 246, 343, 351, 360, 378], "button": [27, 44, 217, 269, 274, 280, 373, 387], "visual": [27, 158, 190, 197, 291, 307, 341], "timelin": [27, 85, 158, 197, 331, 346, 420, 438], "spent": [27, 36, 219, 327, 333, 345, 419, 506], "ag": [27, 35, 40, 54, 55, 62, 69, 76, 112, 137, 210, 284], "legal": [28, 29, 43, 133], "notic": [28, 43, 133, 137, 146, 462], "semant": [29, 45, 84, 126, 170, 180, 226, 233, 248, 261, 263, 266, 268, 271, 276, 333, 339, 345, 353, 361, 368, 379, 407, 450, 464, 505, 512, 520], "languag": [29, 45, 69, 78, 83, 94, 97, 124, 132, 133, 144, 156, 161, 234, 293, 441, 520], "hiveql": [29, 45], "loos": [29, 99], "unlik": [29, 53, 88, 119, 137, 512], "my_arrai": [29, 100], "last_el": 29, "element": [29, 45, 65, 78, 84, 94, 99, 100, 109, 110, 112, 113, 119, 123, 124, 126, 144, 156, 169, 176, 186, 217, 227, 246, 259, 282, 302, 304, 317, 375, 379, 380, 435, 446, 510, 520, 521], "IF": [29, 65, 100, 113, 155, 165, 170, 196, 246, 270, 367, 406, 418, 462, 478, 480, 486, 488, 489, 490, 498, 500, 501, 502], "first_el": [29, 100], "delimit": [29, 35, 57, 100, 124, 142, 258, 331, 345, 348], "backquot": 29, "7day_act": 29, "foo": [29, 33, 35, 45, 70, 100, 112, 124, 144, 271, 478, 480, 497, 508, 518], "c": [29, 35, 58, 99, 100, 106, 107, 112, 117, 124, 136, 137, 148, 162, 173, 183, 226, 472, 476, 486, 510, 512, 520, 544], "varchar": [29, 35, 36, 38, 40, 41, 44, 45, 47, 50, 53, 54, 55, 57, 58, 59, 60, 65, 69, 70, 72, 73, 75, 78, 84, 88, 94, 97, 99, 100, 101, 103, 104, 106, 107, 109, 112, 113, 117, 118, 121, 123, 124, 125, 126, 128, 129, 148, 162, 165, 171, 176, 177, 180, 189, 191, 207, 208, 209, 213, 239, 246, 248, 250, 251, 257, 260, 263, 269, 270, 271, 272, 273, 287, 304, 309, 313, 320, 321, 333, 335, 342, 345, 351, 354, 355, 357, 360, 362, 367, 368, 369, 373, 380, 381, 383, 384, 386, 388, 389, 390, 391, 392, 393, 396, 400, 401, 412, 414, 417, 419, 420, 421, 424, 425, 430, 432, 450, 454, 456, 480, 489, 496, 497, 506, 510, 527, 530, 532], "divis": [29, 105, 112, 118, 288, 391], "float": [29, 37, 39, 40, 41, 42, 43, 54, 57, 59, 60, 62, 66, 68, 69, 70, 73, 75, 76, 99, 109, 140, 210, 212, 213, 217, 257, 266, 296, 365, 368, 389, 413, 419, 420, 432, 433, 538], "inlin": [29, 40, 42, 62, 66, 68, 71, 74, 76, 227, 288, 339, 520, 544], "claus": [29, 35, 37, 39, 41, 42, 43, 45, 54, 60, 62, 66, 68, 69, 71, 72, 74, 75, 76, 84, 94, 96, 99, 104, 105, 112, 126, 131, 148, 151, 159, 162, 166, 177, 179, 182, 196, 208, 212, 226, 227, 229, 237, 239, 240, 242, 244, 246, 249, 250, 255, 258, 264, 265, 267, 268, 270, 278, 282, 287, 288, 293, 297, 301, 313, 317, 324, 327, 334, 338, 339, 347, 350, 351, 353, 357, 359, 366, 367, 368, 371, 372, 373, 374, 375, 379, 380, 381, 382, 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, 397, 401, 406, 412, 413, 417, 418, 420, 421, 423, 430, 432, 433, 434, 436, 437, 438, 450, 453, 454, 455, 478, 480, 482, 486, 487, 488, 489, 490, 491, 493, 498, 499, 500, 501, 502, 503, 504, 507, 508, 510, 511, 517, 518, 522, 526, 527, 532, 536, 537, 539, 544], "substr": [29, 68, 101, 114, 115, 121, 124, 128, 196, 248, 250, 352, 362, 462], "explod": [29, 45, 306, 385], "student": [29, 520], "score": [29, 43, 118, 520], "keyword": [29, 32, 43, 99, 104, 112, 124, 126, 133, 140, 219, 228, 230, 243, 327, 352, 356, 359, 384, 390, 397, 420, 422, 505, 510, 512, 520], "08": [29, 41, 45, 53, 54, 65, 107, 144, 153, 293, 525], "14": [29, 54, 55, 58, 105, 113, 121, 137, 153, 293, 383, 395, 441, 520, 525], "equival": [29, 32, 33, 37, 41, 45, 48, 54, 57, 59, 66, 67, 68, 69, 71, 73, 76, 84, 99, 104, 105, 106, 107, 109, 125, 144, 248, 265, 272, 503, 504, 510, 520], "enddat": 29, "startdat": 29, "date_diff": [29, 107, 114, 115, 155], "revers": [29, 32, 33, 100, 101, 114, 115, 124, 213, 239, 328, 351, 368], "prepend": [29, 136, 148], "hdf": [29, 41, 46, 48, 51, 54, 132, 152, 201, 208, 209, 210, 228, 235, 266, 271, 272, 275, 278, 282, 288, 321, 347, 357, 361, 362, 368, 369, 374, 378, 433, 520], "encrypt": [29, 45, 52, 54, 66, 76, 183, 217, 232, 244, 272, 340, 343, 369, 372, 379, 408, 422, 430, 465, 466, 467, 470, 475], "unpartit": [29, 180, 209, 283, 341, 345, 352, 359, 361, 389, 433], "open": [30, 32, 37, 44, 51, 54, 55, 56, 77, 109, 132, 136, 235, 268, 331, 332, 334, 357, 380, 397, 412, 465, 467], "apach": [30, 35, 37, 39, 42, 45, 46, 51, 54, 55, 57, 60, 64, 67, 69, 70, 78, 132, 192, 231, 281, 303, 307, 413, 415, 432, 434, 436, 447, 472], "github": [30, 78, 84, 137], "trinodb": [30, 84, 135, 137, 378], "project": [30, 41, 44, 45, 54, 84, 94, 109, 135, 147, 158, 180, 184, 239, 267, 297, 359, 360, 365, 389, 391, 399, 418, 425, 432, 433, 448, 468, 505, 506], "volunt": 30, "foundat": [30, 475], "logo": 30, "regist": [30, 45, 82, 85, 89, 90, 92, 95, 126, 136, 330, 380, 381, 430, 432, 435, 439, 449, 466], "entiti": [30, 40, 42, 54, 55, 62, 66, 68, 69, 71, 74, 75, 76, 406], "owner": [30, 44, 47, 77, 173, 213, 262, 264, 270, 344, 358, 359, 373, 374, 383, 392, 462, 479, 480, 481, 488, 491], "american": 30, "nation": [30, 32, 38, 39, 40, 41, 54, 55, 58, 62, 63, 66, 68, 71, 73, 74, 75, 76, 104, 112, 124, 126, 135, 137, 147, 148, 486, 490, 496, 497, 503, 504, 505, 507, 509, 513, 517, 520, 521, 527], "institut": 30, "ansi": [30, 103, 140, 151, 283, 293], "kafka": [30, 34, 73, 133, 206, 436], "aw": [30, 41, 45, 49, 51, 54, 138, 196, 217, 263, 284, 288, 337, 338, 343, 346, 348, 352, 359, 379, 380, 396, 397, 398, 410, 417, 418, 422, 426, 432, 437, 450, 454], "s3": [30, 41, 45, 46, 48, 54, 59, 64, 132, 136, 138, 155, 156, 157, 158, 173, 183, 193, 207, 208, 209, 210, 213, 217, 228, 232, 244, 251, 258, 279, 281, 298, 301, 302, 303, 304, 307, 317, 327, 329, 331, 337, 342, 343, 344, 345, 346, 353, 355, 356, 358, 359, 360, 361, 365, 367, 368, 369, 374, 375, 379, 380, 384, 389, 390, 392, 393, 396, 398, 399, 400, 401, 409, 410, 413, 415, 417, 419, 420, 422, 424, 426, 428, 430, 432, 434, 437, 445, 447, 448, 450, 454, 456], "glue": [30, 41, 45, 54, 263, 328, 337, 338, 346, 347, 348, 352, 353, 358, 359, 367, 368, 369, 373, 380, 381, 386, 388, 389, 396, 397, 399, 401, 403, 404, 405, 406, 417, 418, 422, 424, 425, 426, 428, 429, 432, 434, 435, 438, 440, 450, 454], "emr": [30, 51], "docker": [30, 133, 134, 136, 137, 139], "inc": 30, "googl": [30, 34, 37, 41, 45, 48, 57, 84, 132, 133, 137, 329, 332, 345, 364, 375, 406, 409, 413, 428], "youtub": 30, "linux": [30, 36, 139, 273, 357, 466, 475], "mark": [30, 57, 121, 217, 263, 264, 419, 436, 450, 512, 513], "microsoft": [30, 76], "azur": [30, 41, 45, 48, 76, 132, 137, 346, 357, 361, 364, 365, 366, 375, 383, 408, 420], "ak": 30, "openjdk": [30, 136, 264, 378], "kubernet": [30, 48, 70, 132, 133, 134, 135, 138, 139, 474], "command": [31, 33, 35, 41, 43, 45, 52, 54, 55, 62, 66, 68, 71, 73, 74, 75, 76, 133, 135, 137, 149, 155, 157, 167, 173, 183, 242, 281, 287, 305, 379, 384, 386, 388, 401, 408, 419, 450, 451, 457, 462, 464, 465, 466, 467, 469, 472, 474, 475, 480, 533], "interfac": [31, 51, 52, 78, 84, 87, 88, 90, 91, 93, 94, 96, 97, 119, 126, 132, 133, 135, 137, 149, 154, 155, 172, 176, 183, 187, 188, 210, 213, 232, 255, 262, 267, 281, 287, 293, 294, 304, 306, 317, 409, 433, 448, 462], "python": [31, 139, 471], "shell": [32, 35, 65, 136, 137, 282, 374], "self": [32, 57, 109], "jar": [32, 33, 35, 49, 57, 58, 64, 93, 287, 292, 362, 469], "act": [32, 41, 45, 54, 151, 486], "unix": [32, 38, 97, 107], "virtual": [32, 51, 56, 84, 132, 136, 474], "rest": [32, 33, 37, 51, 54, 70, 81, 101, 124, 133, 136, 151, 155, 204, 209, 210, 245, 432, 445, 462, 464], "ident": [32, 33, 35, 45, 52, 54, 77, 79, 80, 83, 89, 97, 99, 100, 123, 218, 225, 242, 263, 347, 358, 379, 403, 414, 450, 466, 520], "regularli": [32, 33, 41, 54], "download": [32, 33, 44, 48, 49, 58, 135, 136, 139, 447, 475], "430": [32, 33, 135, 136, 137, 139, 153, 333, 516, 524], "renam": [32, 40, 45, 52, 54, 57, 59, 60, 65, 73, 154, 155, 156, 157, 173, 196, 208, 210, 213, 217, 221, 228, 239, 240, 250, 251, 258, 262, 275, 276, 293, 302, 325, 333, 335, 344, 345, 346, 348, 351, 354, 358, 359, 371, 374, 377, 378, 381, 382, 386, 388, 391, 396, 397, 400, 403, 407, 409, 410, 415, 419, 420, 421, 422, 424, 427, 432, 434, 435, 447, 449, 450, 462, 478, 479, 480, 481], "chmod": [32, 58], "preceed": 32, "prompt": [32, 257, 464, 465, 472], "clear": [32, 83, 137, 148, 167, 183, 279, 318, 384], "exit": [32, 173, 182, 186, 247, 258, 259, 334, 434], "quit": [32, 84, 123, 135, 137, 148], "graphviz": [32, 505], "logic": [32, 36, 45, 96, 98, 102, 104, 126, 133, 146, 147, 151, 259, 282, 301, 302, 375, 389, 415, 440, 491, 512, 520], "tpch": [32, 34, 38, 39, 41, 47, 58, 63, 65, 68, 71, 74, 76, 99, 105, 133, 135, 137, 147, 148, 151, 209, 297, 389, 497, 505, 521, 530, 536, 539], "tini": [32, 38, 41, 58, 62, 63, 79, 80, 99, 137, 178, 248, 267, 270, 273, 274, 334, 366, 505, 539], "_col0": [32, 58, 68, 135, 137, 288, 520], "20220324_213359_00007_w6hbk": 32, "13": [32, 41, 54, 58, 60, 99, 112, 121, 137, 153, 398, 453, 506, 520], "done": [32, 35, 37, 57, 84, 94, 135, 137, 144, 182, 261, 375, 377, 472, 475], "92": [32, 142, 153, 320], "lineitem": [32, 58, 80, 493, 505, 520], "partsupp": [32, 58], "supplier": [32, 58], "timeout": [32, 36, 39, 40, 41, 42, 43, 44, 45, 51, 54, 55, 60, 62, 64, 66, 68, 69, 70, 71, 72, 74, 75, 76, 78, 84, 157, 173, 183, 206, 210, 218, 233, 235, 240, 244, 247, 259, 260, 268, 279, 303, 341, 346, 367, 368, 369, 374, 389, 392, 393, 398, 399, 403, 407, 410, 418, 425, 432, 433, 469, 470], "autocomplet": [32, 157], "edit": [32, 124, 347, 401], "bind": [32, 87, 88, 281, 328, 347, 356, 374, 386, 469, 520], "vi": 32, "emac": [32, 401], "editor": [32, 33, 44], "proxi": [32, 33, 37, 41, 47, 51, 64, 70, 78, 245, 344, 347, 358, 383, 392, 400, 427, 446, 447, 467, 471, 475], "trino_histori": 32, "align": [32, 182, 257, 264, 282, 342, 392, 435], "pager": [32, 173, 186, 247, 258, 259, 435], "carefulli": [32, 58, 137, 371], "progress": [32, 155, 157, 191, 209, 242, 249, 279, 305, 307, 330, 375, 405, 441], "trino_password": 32, "listen": [32, 81, 93, 133, 227, 331, 354, 359, 364, 367, 375, 380, 390, 396, 402, 427, 453], "session_property_nam": 32, "sock": [32, 33, 47, 64, 78, 167, 245, 320, 347], "timezon": [32, 33, 36, 40, 45, 54, 57, 59, 66, 72, 73, 83, 97, 180, 220, 244, 257, 297, 358, 361, 386, 419, 430, 432, 446, 457], "zone": [32, 33, 36, 37, 39, 40, 41, 45, 54, 57, 59, 65, 66, 67, 68, 71, 73, 76, 83, 84, 104, 112, 114, 115, 133, 143, 160, 162, 176, 191, 198, 209, 220, 234, 253, 257, 262, 263, 268, 270, 271, 296, 297, 331, 334, 339, 343, 344, 345, 356, 358, 361, 362, 364, 365, 366, 368, 369, 371, 372, 373, 375, 377, 379, 382, 386, 389, 390, 391, 396, 409, 413, 414, 423, 424, 429, 430, 432, 434, 438, 440, 446, 449, 450, 451, 454, 477], "workstat": [32, 135], "find": [32, 47, 54, 65, 72, 73, 77, 82, 85, 89, 90, 92, 94, 95, 104, 105, 109, 113, 121, 134, 136, 137, 146, 148, 217, 446, 469, 473, 510, 512, 526, 527, 532, 536, 537, 539], "ssl": [32, 33, 40, 51, 64, 65, 70, 71, 74, 132, 233, 245, 258, 373, 403, 408, 432, 448, 471, 475], "sslverif": [32, 33, 373], "clientinfo": [32, 33, 359], "recogn": [32, 33, 41, 53, 119, 121, 132, 225, 398, 475], "jk": [32, 33, 39, 40, 42, 43, 57, 62, 65, 66, 68, 70, 71, 74, 76, 132, 133, 458, 465, 468, 469, 475], "cryptographi": [32, 47, 132], "architectur": [32, 143, 389, 471], "jca": 32, "maco": [32, 33, 274], "exampleusernam": 32, "quot": [32, 45, 56, 60, 65, 112, 136, 142, 223, 233, 268, 282, 302, 329, 332, 357, 371, 384, 388, 422, 426, 450, 472, 524], "special": [32, 35, 42, 51, 57, 59, 60, 65, 73, 94, 100, 107, 112, 124, 131, 136, 154, 270, 302, 333, 347, 360, 392, 399, 422, 423, 425, 426, 427, 432, 510], "longsecurepassword123": 32, "browser": [32, 33, 83, 239, 464, 465, 470, 472, 475], "appear": [32, 57, 100, 112, 124, 173, 183, 186, 244, 250, 287, 293, 346, 347, 356, 366, 368, 373, 392, 424, 430, 468, 496, 520], "token": [32, 33, 40, 42, 47, 50, 55, 58, 64, 66, 68, 69, 71, 74, 75, 76, 78, 83, 123, 142, 263, 264, 328, 329, 348, 359, 360, 361, 372, 379, 381, 388, 391, 396, 415, 417, 420, 424, 426, 430, 432, 437, 447, 453, 467, 471, 472, 476], "successfulli": [32, 59, 83, 112, 137, 148, 162, 209, 251, 259, 468, 469, 471, 475, 510], "proce": [32, 134, 464, 465, 520], "argument": [32, 44, 45, 51, 52, 54, 88, 100, 103, 104, 105, 106, 107, 108, 109, 113, 118, 124, 167, 170, 203, 209, 210, 211, 217, 220, 226, 227, 229, 238, 242, 246, 247, 249, 254, 263, 265, 270, 274, 282, 284, 293, 301, 303, 305, 315, 328, 357, 360, 361, 362, 366, 380, 401, 414, 416, 426, 436, 439, 442, 448, 450, 454, 469, 483, 505, 506, 512, 520, 532], "pem": [32, 33, 39, 40, 42, 43, 62, 65, 66, 68, 71, 74, 76, 132, 133, 268, 339, 458, 467, 469, 475], "simplest": [32, 99, 469, 475, 520], "wai": [32, 35, 37, 38, 39, 40, 41, 42, 44, 45, 47, 51, 54, 57, 59, 60, 62, 65, 66, 68, 69, 71, 74, 75, 76, 84, 87, 94, 96, 104, 112, 119, 131, 132, 135, 137, 144, 146, 151, 178, 227, 271, 280, 296, 334, 371, 396, 397, 403, 460, 462, 464, 467, 469, 471, 476, 510, 520], "wrapper": [32, 88, 112, 469], "bash": [32, 469], "krb5": [32, 52, 60, 196, 263, 329, 468, 476], "conf": [32, 45, 51, 52, 60, 136, 468], "princip": [32, 33, 52, 60, 64, 82, 90, 92, 95, 275, 329, 335, 357, 362, 378, 384, 449, 467, 470, 508, 516, 518, 532], "someus": 32, "keytab": [32, 33, 60, 64, 196, 378, 391, 392], "home": 32, "cach": [32, 33, 35, 37, 40, 42, 44, 45, 46, 52, 53, 55, 57, 62, 64, 66, 68, 69, 71, 72, 74, 75, 76, 78, 155, 167, 180, 196, 217, 218, 223, 228, 232, 235, 239, 242, 243, 257, 281, 296, 297, 303, 335, 343, 346, 347, 348, 355, 358, 359, 360, 361, 362, 364, 365, 367, 368, 372, 375, 383, 386, 388, 391, 392, 394, 396, 399, 401, 408, 409, 416, 418, 419, 421, 425, 430, 432, 433, 438, 443, 444, 469, 472, 473], "canonic": [32, 33, 64], "dsun": [32, 468], "dtrino": 32, "debugkerbero": 32, "djava": [32, 52, 136, 466, 468], "gssloginconfig": 32, "configfil": 32, "configpars": 32, "logincontext": 32, "trino_pag": [32, 435], "pspg": 32, "scroll": [32, 43], "arrow": [32, 432], "r": [32, 58, 100, 107, 131, 148, 520], "press": [32, 137, 173, 387], "enter": [32, 63, 135, 137, 177, 472, 475], "trino_history_fil": [32, 435], "ctrl": [32, 136, 137, 162, 173, 183], "mac": 32, "trino_config": 32, "xdg_config_hom": 32, "8443": [32, 33, 40, 469, 470, 472, 475], "europ": [32, 36, 41, 54, 104, 262, 293, 520], "warsaw": 32, "csv": [32, 45, 67, 86, 281, 332, 343, 348, 364, 366, 368, 369, 381, 386, 422, 436, 454], "f": [32, 100, 107, 113, 117, 137, 154, 156, 250], "safe": [32, 41, 54, 84, 101, 113, 129, 394, 467], "nationkei": [32, 38, 41, 54, 58, 147, 486, 497, 503, 504, 509, 513, 520, 527], "regionkei": [32, 41, 65, 104, 126, 148, 496, 497, 503, 504, 505, 509, 513, 520, 527], "sf1": [32, 47, 79, 80, 105, 135, 147, 148, 248, 497, 506, 530], "algeria": [32, 43, 520], "argentina": [32, 520], "brazil": 32, "vertic": [32, 332, 435], "incorrectli": [32, 57, 121, 203, 204, 209, 213, 220, 226, 242, 248, 262, 264, 273, 287, 316, 330, 332, 339, 346, 368, 379, 381, 420, 426, 432, 436, 437, 450, 453, 466], "statu": [32, 45, 54, 77, 83, 126, 135, 136, 137, 139, 142, 162, 173, 217, 242, 249, 259, 280, 330, 335, 357, 399, 444, 462, 542], "unaffect": 32, "20200707_170726_00030_2iup9": 32, "uppercas": [32, 65, 124, 126, 129, 196, 225, 368, 426, 432, 437, 438, 439, 440, 446, 447, 456, 476], "csv_header": 32, "csv_unquot": 32, "csv_header_unquot": 32, "tsv": 32, "tab": [32, 44, 45, 180, 387, 432], "tsv_header": 32, "emit": [32, 476], "ascii": [32, 39, 129, 250, 303, 324, 343, 360, 366], "orient": [32, 304, 317, 438, 451], "width": [32, 57, 59, 72, 73, 103, 118, 173, 401, 538], "markdown": 32, "suppress": [32, 112, 478, 480, 488, 489, 490, 498, 500, 501, 502], "someth": [32, 84, 465], "wrong": [32, 180, 204, 209, 212, 226, 248, 258, 265, 282, 295, 314, 331, 332, 345, 354, 379, 386], "20200804_201646_00003_f5f6c": 32, "22": [32, 41, 43, 58, 65, 97, 107, 112, 117, 144, 147, 153, 506, 520], "20200804_201629_00002_f5f6c": 32, "spi": [32, 81, 84, 87, 132, 133, 151, 161, 172, 209, 282, 283, 285], "trinoexcept": [32, 84, 96, 378], "semanticexcept": 32, "48": [32, 43], "43": [32, 41, 65, 104, 119, 347], "834": 32, "desktop": 33, "grant": [33, 51, 52, 84, 133, 143, 217, 239, 244, 264, 339, 343, 351, 362, 368, 373, 384, 386, 389, 390, 392, 400, 402, 462, 477, 487, 494, 499, 505, 517, 518, 522], "classpath": [33, 35, 51, 93], "maven": [33, 58, 217, 287], "groupid": [33, 93], "artifactid": [33, 93], "we": [33, 35, 45, 47, 48, 59, 84, 88, 99, 112, 136, 137, 138, 148, 170, 172, 177, 192, 247, 282, 283, 287, 293, 296, 297, 301, 302, 305, 307, 320, 321, 334, 364, 401, 431, 469, 475], "latest": [33, 54, 55, 59, 135, 137], "repositori": [33, 84, 137], "navig": [33, 512], "xxx": [33, 35, 462], "ll": [33, 35, 151], "Then": [33, 35, 41, 84, 282, 471], "commonli": [33, 123, 132, 267, 470, 520], "gui": 33, "upon": [33, 63, 113, 137, 151, 395, 396, 512], "readi": [33, 50, 93, 135, 137, 296], "sale": [33, 37, 39, 40, 43, 45, 51, 55, 57, 65, 66, 68, 71, 74, 75, 76, 482, 488, 500], "classnam": 33, "trinodriv": [33, 378], "wherev": [33, 474], "drivermanag": 33, "setproperti": 33, "getconnect": 33, "sessionus": [33, 379], "imperson": [33, 35, 48, 64, 225, 266, 347, 357, 362, 363, 364, 379, 386, 398, 420, 432, 460, 524], "socksproxi": 33, "1080": 33, "httpproxi": 33, "8888": 33, "abc": [33, 51, 100, 112, 251], "xyz": [33, 51], "tracetoken": [33, 359], "applicationnam": [33, 271], "applicationnameprefix": [33, 271], "append": [33, 41, 44, 45, 53, 54, 57, 100, 144, 464, 470, 510], "neither": [33, 57, 462, 510, 512, 520], "nor": [33, 297, 462, 510, 512], "accesstoken": [33, 467], "verif": [33, 360, 373, 399, 423, 448, 465], "mismatch": [33, 112, 209, 261, 269, 272, 280, 327, 337, 384, 505], "sslkeystorepath": [33, 254], "sslkeystorepassword": [33, 254, 461], "sslkeystoretyp": [33, 461], "ssltruststorepath": 33, "ssltruststorepassword": 33, "ssltruststoretyp": 33, "sslusesystemtruststor": [33, 403], "keychainstor": 33, "hostnameincertif": [33, 76, 448], "kerberosremoteservicenam": 33, "kerberosprincip": 33, "kerberosusecanonicalhostnam": 33, "canon": [33, 64, 124, 144, 155, 317, 357, 380], "ip": [33, 36, 43, 45, 48, 70, 98, 133, 137, 144, 247, 344, 359, 392, 466, 469, 473], "kerberosserviceprincipalpattern": 33, "kerberosconfigpath": 33, "kerberoskeytabpath": 33, "kerberoscredentialcachepath": 33, "kerberosdeleg": 33, "extracredenti": [33, 40, 42, 62, 66, 68, 71, 74, 76, 357], "bar": [33, 35, 70, 100, 103, 112, 114, 115, 144, 242, 362, 478, 480, 508, 518], "catalog1": 33, "rolea": 33, "catalog2": 33, "roleb": 33, "externalauthent": 33, "idp": [33, 470], "externalauthenticationtokencach": 33, "classload": [33, 332], "disablecompress": [33, 374], "assumeliteralunderscoreinmetadatacallsfornonconformingcli": [33, 413, 419], "databasemetadata": [33, 155, 173, 177, 213, 232, 233, 235, 287, 344, 350, 361, 365, 375, 382, 385, 389, 396, 399, 401, 402, 413, 432], "underscor": [33, 104, 121, 136, 142, 144, 182, 287, 302, 323, 356, 447], "workaround": [33, 45, 54, 136, 382], "accumulo": [34, 133, 217, 523], "atop": [34, 133], "black": [34, 103, 133, 162], "hole": [34, 109, 133, 162], "cassandra": [34, 84, 133, 152, 206, 282, 285, 304, 315], "clickhous": [34, 133, 380], "druid": [34, 133, 364, 380, 384], "elasticsearch": [34, 133, 329, 378], "sheet": [34, 133, 345], "hudi": [34, 64, 133, 371, 425], "ignit": [34, 133], "kinesi": [34, 133, 347, 356], "kudu": [34, 133, 274], "mariadb": [34, 133, 401, 406], "phoenix": [34, 133, 339, 357, 358, 375, 384], "pinot": [34, 133, 361], "prometheu": [34, 133, 361], "redi": [34, 133, 173, 206, 209], "singlestor": [34, 133, 353, 361], "thrift": [34, 41, 45, 50, 53, 54, 133, 137, 232, 245, 260, 327, 331, 342, 346, 347, 360, 361, 367, 420, 427, 436, 453], "tpcd": [34, 133, 247], "thoroughli": 35, "capabl": [35, 40, 42, 45, 55, 62, 66, 68, 69, 71, 74, 75, 76, 79, 80, 98, 119, 126, 148, 232, 457], "pushdown": [35, 41, 42, 54, 57, 67, 69, 74, 87, 96, 133, 145, 192, 232, 233, 241, 245, 248, 262, 264, 269, 272, 274, 278, 280, 308, 320, 327, 329, 333, 334, 341, 354, 360, 362, 364, 367, 368, 369, 371, 373, 377, 379, 380, 381, 382, 384, 386, 388, 389, 392, 396, 399, 402, 414, 419, 420, 424, 429, 432, 434, 437, 438, 441, 450], "ext": 35, "tabletserv": 35, "scp": 35, "trino_hom": 35, "tabletserver_address": 35, "accumulo_hom": 35, "pick": [35, 93, 99, 146, 303], "zookeep": [35, 58, 69, 426], "2181": [35, 58, 69], "mount": [35, 37, 38, 44, 48, 58, 62, 66, 72, 74, 75, 135, 136, 281], "appropri": [35, 36, 37, 40, 41, 43, 45, 47, 51, 54, 55, 57, 59, 60, 62, 65, 66, 71, 72, 73, 74, 75, 76, 78, 94, 255, 338, 384, 461, 465, 469, 476], "setup": [35, 36, 37, 40, 43, 45, 46, 48, 55, 62, 66, 71, 72, 74, 75, 76, 139, 360, 466, 467, 473, 474], "ye": [35, 36, 43, 69, 70, 464], "znode": [35, 69], "100000": [35, 57, 427], "effect": [35, 45, 51, 55, 69, 71, 84, 88, 112, 144, 167, 239, 246, 273, 392, 512, 520, 525], "far": [35, 72, 84], "concern": [35, 471], "previou": [35, 37, 41, 78, 88, 109, 151, 155, 159, 167, 173, 180, 188, 194, 212, 213, 217, 220, 251, 255, 256, 259, 270, 275, 282, 302, 316, 317, 320, 325, 327, 336, 339, 342, 350, 356, 359, 362, 366, 373, 375, 395, 396, 423, 432, 438, 465, 510, 512, 520, 523], "cell": [35, 69, 73], "overwritten": [35, 57], "datatyp": 35, "row_id": 35, "WITH": [35, 36, 37, 38, 39, 40, 41, 45, 47, 50, 52, 54, 55, 57, 59, 60, 65, 66, 68, 69, 71, 72, 73, 76, 84, 99, 104, 112, 123, 142, 154, 162, 177, 178, 208, 257, 271, 293, 297, 339, 342, 347, 362, 364, 365, 366, 368, 369, 380, 394, 430, 432, 450, 480, 482, 486, 487, 488, 489, 490, 507, 508, 510, 512, 530], "example_schema": [35, 40, 41, 42, 45, 53, 54, 55, 60, 66, 68, 69, 71, 74, 75, 76], "scientist": [35, 69, 76], "recordkei": [35, 69, 76], "birthdai": [35, 40, 55, 69, 76], "comment": [35, 37, 38, 39, 40, 41, 42, 45, 47, 50, 55, 57, 58, 59, 60, 63, 65, 66, 68, 69, 71, 74, 75, 76, 77, 78, 84, 112, 126, 133, 140, 143, 147, 188, 194, 226, 227, 238, 265, 297, 301, 334, 338, 353, 356, 357, 365, 371, 372, 373, 386, 390, 396, 399, 401, 402, 403, 408, 410, 412, 415, 417, 418, 422, 423, 425, 428, 429, 439, 440, 442, 444, 447, 448, 449, 450, 454, 462, 477, 480, 486, 489, 490, 491, 497, 505, 509, 527], "auto": [35, 45, 60, 65, 155, 209, 253, 293, 387, 391, 397, 403, 407, 408, 435], "famili": [35, 40, 69], "column_map": 35, "tripl": 35, "conveni": [35, 86, 96, 293, 475], "mutat": 35, "row1": 35, "hopper": 35, "109": [35, 153, 330], "1906": [35, 353], "12": [35, 41, 53, 54, 57, 58, 68, 99, 102, 107, 112, 121, 128, 132, 137, 144, 153, 209, 253, 293, 397, 421, 441, 464, 465, 472, 475, 506, 520], "09": [35, 43, 45, 54, 72, 107, 137, 293], "row2": 35, "alan": 35, "ture": 35, "103": [35, 112, 153, 167], "1912": 35, "06": [35, 60, 72, 107, 135, 137, 144, 270, 506, 520], "23": [35, 41, 54, 58, 65, 107, 109, 112, 121, 128, 144, 153, 293, 506], "programmat": [35, 59, 254], "think": [35, 151, 167, 431], "5321": 35, "just": [35, 45, 64, 80, 83, 84, 86, 87, 88, 99, 137, 139, 151, 187, 468, 475, 476], "tbl": 35, "littl": [35, 110, 163, 475], "younger": 35, "u": [35, 45, 100, 107, 121, 123, 124, 144, 161, 510, 512, 520], "p": [35, 37, 40, 41, 54, 60, 62, 65, 66, 68, 69, 71, 75, 76, 84, 107, 118, 121, 135, 173, 254, 368, 373, 375, 379, 380, 382, 412, 413, 419, 449, 454, 539], "row3": 35, "tim": 35, "berner": 35, "lee": 35, "60": [35, 56, 99, 153, 210, 520], "1984": 35, "07": [35, 37, 41, 45, 54, 55, 57, 58, 60, 65, 106, 107, 153, 520], "batchscann": 35, "tablet": [35, 60], "narrow": [35, 173, 468], "But": [35, 84, 104, 170], "drastic": [35, 76], "heavi": 35, "lift": 35, "index_column": 35, "wish": [35, 51], "here": [35, 45, 57, 78, 84, 94, 104, 124, 136, 139, 483, 538], "lexicod": 35, "scientists_idx": 35, "scientists_idx_metr": 35, "21011": 35, "metadata_d": 35, "23034": 35, "metadata_ag": 35, "metadata_nam": 35, "bundl": [35, 132, 180, 285], "index_rows_per_split": 35, "tend": 35, "bit": [35, 56, 66, 71, 75, 76, 88, 101, 102, 110, 136, 144, 178, 256, 316, 462, 520], "difficult": [35, 57, 255, 475], "gonna": 35, "bad": [35, 65, 146, 244], "external_t": 35, "md": [35, 78], "locality_group": 35, "2015": [35, 65, 177, 353], "03": [35, 41, 45, 54, 57, 107, 144, 262, 293], "external_table_idx": 35, "external_table_idx_metr": 35, "col_nam": 35, "col_famili": 35, "col_qualifi": 35, "pipe": [35, 242, 343], "group1": 35, "fama": 35, "famb": 35, "famc": 35, "group2": 35, "famd": 35, "fame": 35, "famf": 35, "accumulorowseri": 35, "getdefault": [35, 475], "scan_auth": 35, "auth": [35, 43, 51, 57, 72, 254, 469, 470, 472], "scanner": [35, 392], "column_filter_optimizations_en": 35, "optimize_locality_en": [35, 523], "optimize_split_ranges_en": 35, "optimize_index_en": 35, "secondari": [35, 47, 65, 177], "index_threshold": 35, "index_lowest_cardinality_threshold": 35, "01": [35, 41, 45, 54, 55, 56, 57, 58, 60, 72, 78, 79, 80, 107, 112, 128, 144, 148, 209, 217, 234, 263, 267, 270, 293, 365, 367, 368, 371, 482, 505, 506, 520], "intersect": [35, 84, 100, 109, 123, 142, 208, 374, 380, 381, 386], "index_metrics_en": 35, "scan_usernam": 35, "trump": 35, "index_short_circuit_cardinality_fetch": 35, "circuit": [35, 232, 233], "index_cardinality_cache_polling_dur": 35, "poll": [35, 303, 379], "todai": 35, "framework": [35, 45, 88, 109, 302, 304, 364], "abil": [35, 45, 54, 56, 110, 123, 155, 180, 187, 227, 243, 256, 266, 357, 364, 415, 462], "deseri": [35, 110, 112, 123, 182, 194, 230, 278, 375, 454], "abstract": [35, 132, 155, 283], "leverag": [35, 46, 70, 71, 361, 369, 371, 391], "convers": [35, 45, 57, 59, 73, 84, 98, 112, 133, 287, 302, 358, 400], "forth": [35, 42], "arrai": [35, 37, 40, 41, 45, 54, 55, 57, 59, 60, 65, 69, 73, 78, 84, 98, 99, 104, 109, 113, 117, 118, 120, 121, 123, 124, 127, 133, 155, 156, 165, 166, 173, 175, 177, 178, 182, 183, 184, 186, 187, 191, 194, 212, 213, 217, 218, 220, 230, 233, 234, 236, 239, 244, 246, 247, 253, 255, 257, 261, 264, 266, 267, 269, 272, 274, 277, 278, 279, 282, 287, 304, 310, 311, 313, 315, 317, 328, 333, 336, 337, 341, 348, 350, 356, 361, 368, 369, 375, 379, 380, 381, 382, 384, 385, 387, 389, 390, 392, 394, 399, 400, 402, 426, 438, 446, 448, 453, 475, 482, 486, 510, 520, 530, 532], "proper": [35, 96, 287, 301, 345, 368, 381, 388, 422], "lexigraph": 35, "timestamp": [35, 36, 37, 39, 40, 41, 42, 43, 45, 53, 54, 56, 58, 60, 62, 64, 67, 68, 70, 71, 72, 73, 75, 76, 78, 84, 88, 100, 104, 107, 112, 128, 158, 162, 173, 176, 194, 208, 209, 211, 217, 228, 247, 257, 262, 267, 268, 270, 271, 282, 288, 293, 296, 297, 305, 307, 323, 331, 332, 334, 339, 342, 345, 347, 354, 358, 359, 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, 375, 377, 379, 380, 381, 382, 383, 384, 386, 389, 391, 392, 396, 397, 399, 400, 404, 408, 409, 412, 413, 414, 415, 419, 420, 421, 423, 424, 425, 426, 429, 430, 432, 434, 437, 438, 440, 448, 449, 450, 451, 454, 456], "extend": [35, 54, 87, 88, 96, 141, 152, 155, 156, 167, 248, 250, 262, 280, 283, 294, 305, 369, 371, 375, 377, 380, 389, 393, 425, 454, 470, 471], "x08": 35, "x80": 35, "x00": 35, "x00m": 35, "x7f": 35, "xff": 35, "xa6": 35, "x06": 35, "x00g": 35, "xad": 35, "xed": 35, "stringy_scientist": 35, "custom_scientist": 35, "my": [35, 41, 45, 54, 93, 137, 474], "packag": [35, 58, 84, 93, 105, 132, 133, 134, 135, 136, 137, 163, 180, 217, 268, 287, 297, 298, 327, 378, 447, 475, 510], "myseri": 35, "easiest": [35, 464, 475], "mess": 35, "namespac": [35, 47, 57, 60, 137, 293, 305], "accumulot": 35, "resid": [35, 37, 57, 64, 462], "manipul": [35, 41, 45, 54, 113, 371, 483], "zookeepermetadatamanag": 35, "simplifi": [35, 49, 80, 109, 264, 272, 280, 302, 305, 396, 408, 455, 520], "zkcli": 35, "sh": [35, 58, 139, 380], "zk": 35, "had": [35, 84, 163, 166, 204, 209, 217, 253, 255, 274, 296, 388, 389, 433], "tinyint": [35, 37, 39, 40, 41, 43, 45, 55, 57, 59, 60, 62, 66, 68, 69, 71, 73, 75, 76, 84, 94, 109, 112, 209, 211, 213, 217, 223, 243, 271, 279, 329, 331, 336, 347, 348, 380, 388, 400, 415, 432, 450, 456], "smallint": [35, 37, 39, 40, 41, 43, 45, 55, 57, 59, 60, 62, 66, 68, 69, 71, 73, 75, 76, 84, 112, 209, 213, 217, 223, 243, 271, 279, 329, 331, 347, 348, 355, 380, 388, 400, 450, 456], "n": [35, 41, 55, 58, 62, 66, 68, 69, 71, 75, 76, 99, 100, 113, 118, 119, 121, 124, 131, 137, 144, 155, 164, 170, 207, 208, 209, 219, 236, 240, 245, 259, 264, 271, 279, 287, 302, 337, 373, 392, 393, 394, 396, 412, 430, 432, 454, 468, 510, 520], "varbinari": [35, 37, 39, 40, 41, 43, 45, 54, 55, 57, 59, 60, 62, 65, 66, 68, 69, 70, 71, 73, 75, 76, 84, 99, 101, 109, 110, 112, 120, 123, 124, 127, 173, 180, 191, 227, 228, 239, 245, 250, 251, 257, 260, 263, 269, 270, 273, 279, 296, 302, 305, 320, 321, 347, 358, 368, 369, 373, 375, 386, 389, 399, 401], "127": [35, 45, 55, 76, 129, 137, 153], "past": [36, 54, 144, 419, 510], "allow_al": [36, 41, 54], "sampl": [36, 40, 45, 56, 58, 65, 84, 88, 99, 113, 137, 209, 219, 266, 272, 296, 328, 368, 469, 520], "vienna": [36, 41, 54], "0100": 36, "utc": [36, 37, 45, 54, 57, 59, 65, 66, 67, 73, 78, 107, 144, 209, 262, 268, 368, 391, 525], "offer": [36, 41, 45, 54, 123, 126, 460], "host_ip": 36, "start_tim": [36, 369], "end_tim": [36, 369], "device_nam": 36, "hard": [36, 136, 255], "utilization_perc": 36, "busi": [36, 45], "io_tim": 36, "TO": [36, 52, 65, 84, 100, 137, 204, 209, 335, 366, 391, 402, 407, 422, 462, 478, 479, 480, 481, 494, 507, 508, 510, 525], "read_request": 36, "sectors_read": 36, "sector": 36, "write_request": 36, "sectors_written": 36, "power_on_tim": 36, "boot": [36, 139], "grpc": [37, 414, 419, 427, 441], "cloud": [37, 41, 45, 48, 59, 132, 137, 329, 332, 364, 375, 392, 406, 409, 413, 428, 475], "intermediari": 37, "flow": [37, 64, 383, 386, 394], "avro": [37, 54, 219, 243, 274, 275, 278, 341, 357, 366, 368, 374, 377, 381, 389, 391, 398, 399, 412, 419, 428, 447, 449, 453, 456], "wire": [37, 52, 272], "interest": [37, 72, 83, 84, 85, 86, 94, 126, 144, 510], "columnar": [37, 78, 184, 201, 208, 210, 239, 296], "datastor": 37, "shard": [37, 59, 154, 310, 368], "rebal": 37, "nearli": [37, 123], "blog": [37, 520], "articl": [37, 467], "dataflow": 37, "443": [37, 209, 258, 392], "authenti": 37, "dataproc": 37, "gce": 37, "outsid": [37, 51, 103, 109, 120, 136, 137, 167, 178, 183, 267, 366, 468, 475, 510, 512, 520], "google_application_credenti": 37, "base64": [37, 44, 60, 302, 432, 466, 467, 470], "analyt": [37, 54, 152], "400mb": 37, "roughli": [37, 136], "experiment": [37, 45, 71, 76, 157, 163, 167, 170, 172, 193, 208, 219, 223, 228, 233, 242, 245, 247, 248, 263, 267, 273, 278, 279, 293, 296, 304, 306, 307, 310, 331, 337, 339, 344, 348, 350, 351, 354, 357, 359, 367, 380, 389, 391, 392, 396, 399, 408, 413, 414, 420, 424, 432, 441, 450, 520], "caveat": 37, "nio": [37, 55, 136, 334], "unnam": [37, 55, 144, 375], "preliminari": 37, "bill": 37, "dataset": [37, 54, 79, 80, 99, 148, 287], "24h": [37, 470], "bignumer": [37, 392, 396], "unsupport": [37, 40, 42, 43, 54, 62, 66, 68, 69, 71, 74, 75, 76, 84, 236, 248, 269, 328, 344, 352, 354, 355, 368, 389, 391, 414, 420, 430, 449], "ttl": [37, 40, 41, 42, 44, 48, 51, 55, 62, 64, 66, 68, 69, 71, 72, 74, 75, 76, 218, 239, 242, 355, 361, 362, 386, 396, 408, 419, 430, 432, 434, 438, 469, 470], "reus": [37, 144, 266, 429], "retryabl": [37, 78, 268], "insensit": [37, 57, 72, 84, 99, 121, 126, 128, 183, 198, 217, 271, 307, 336, 340, 361, 372, 379, 380, 383, 384, 389, 396, 422, 426, 432, 433, 456], "rpc": [37, 447], "mutual": [37, 486], "int64": [37, 40, 57, 60, 65, 245, 247, 331, 372, 389, 392], "byteint": 37, "alias": [37, 40, 62, 144, 208, 247, 274, 275, 355, 359, 380, 400, 497, 520, 544], "float64": [37, 40, 389], "precis": [37, 39, 45, 55, 57, 66, 68, 69, 71, 75, 76, 83, 84, 101, 107, 108, 120, 140, 173, 211, 217, 251, 261, 335, 352, 355, 358, 359, 362, 364, 365, 368, 369, 371, 372, 373, 375, 377, 379, 386, 388, 389, 390, 391, 392, 395, 407, 413, 430, 434, 437, 450, 456, 538], "38": [37, 65, 66, 68, 69, 71, 75, 108, 144, 257, 355, 392, 398], "77": [37, 153, 328], "datetim": [37, 40, 66, 75, 112, 128, 351, 365, 380, 381, 389, 391, 423, 430], "geographi": [37, 109, 390], "wkt": [37, 109, 257], "example_view": 37, "view_definit": [37, 379], "hidden": [37, 41, 43, 45, 53, 54, 57, 58, 59, 65, 73, 78, 155, 183, 223, 248, 250, 264, 275, 287, 297, 320, 331, 332, 336, 343, 346, 350, 360, 361, 365, 366, 379, 380, 382, 389, 405, 419, 426, 439, 440], "partition_d": 37, "_partitiond": 37, "pseudo": [37, 118, 130], "partition_tim": 37, "_partitiontim": 37, "inspect": [37, 45, 48, 54, 143, 287, 371, 396, 434, 438, 450, 468, 469, 470], "page_view": [37, 45, 54], "2022": [37, 54, 72, 107, 137], "04": [37, 58, 106, 107, 144, 153, 293, 520], "__null__": 37, "__unpartitioned__": 37, "truncat": [37, 39, 40, 54, 57, 62, 66, 68, 69, 71, 74, 75, 76, 84, 99, 101, 103, 114, 115, 118, 124, 133, 143, 144, 164, 197, 217, 264, 277, 377, 381, 392, 401, 416, 420, 421, 427, 453, 477, 505], "nativ": [37, 39, 41, 42, 43, 62, 65, 66, 68, 71, 74, 76, 88, 109, 119, 136, 197, 246, 250, 263, 296, 334, 389, 436, 438, 445, 447, 449, 454], "passthrough": [37, 42, 43, 62, 66, 68, 71, 74, 76, 384, 402, 430], "concaten": [37, 40, 66, 99, 101, 112, 114, 124, 233, 245, 313, 449, 465, 473], "employe": [37, 62, 66, 462, 542], "manager_id": [37, 66, 542], "string_agg": 37, "employee_id": [37, 66, 542], "compani": [37, 42, 66, 148], "BY": [37, 39, 41, 42, 43, 45, 52, 53, 54, 58, 62, 66, 68, 70, 71, 74, 76, 84, 88, 99, 110, 123, 126, 131, 142, 148, 162, 164, 176, 179, 180, 182, 183, 184, 186, 190, 191, 196, 198, 206, 210, 217, 220, 226, 227, 232, 233, 235, 237, 240, 241, 242, 244, 246, 249, 250, 251, 255, 261, 264, 266, 267, 268, 271, 275, 279, 282, 283, 284, 287, 288, 291, 293, 297, 302, 307, 312, 313, 315, 317, 321, 331, 333, 334, 337, 338, 339, 346, 353, 358, 366, 367, 372, 373, 375, 380, 381, 382, 383, 384, 385, 386, 387, 389, 391, 392, 393, 401, 406, 420, 432, 433, 437, 439, 450, 452, 453, 454, 455, 457, 486, 490, 491, 505, 506, 508, 510, 512, 518], "preserv": [37, 39, 40, 42, 43, 62, 66, 68, 70, 71, 73, 74, 75, 76, 112, 121, 136, 243, 273, 278, 329, 331, 357, 368], "primarili": [38, 72, 137, 253, 254, 520], "manner": [38, 56, 96, 121, 129, 137, 217, 271, 385, 391, 512], "discard": [38, 63, 198, 520], "shouldn": 38, "blackhol": [38, 180, 399], "split_count": 38, "pages_per_split": 38, "rows_per_pag": 38, "field_length": [38, 180], "16": [38, 41, 51, 112, 113, 144, 153, 183, 184, 220, 506], "page_processing_delai": [38, 217], "dummi": [38, 41, 57, 59, 73], "9042": 39, "host1": [39, 40, 55, 57, 69, 70], "host2": [39, 57, 69, 70], "discoveri": [39, 69, 136, 151, 155, 281, 286, 341, 392, 408, 466], "topolog": [39, 255, 346, 374], "dc": [39, 100, 407, 469], "datacenter1": 39, "discov": [39, 57, 65, 217, 259, 338, 347], "At": [39, 40, 42, 55, 59, 62, 64, 65, 66, 68, 69, 71, 74, 75, 76, 88, 271, 468, 469, 470, 510], "ONE": [39, 57, 70, 510, 512], "each_quorum": 39, "quorum": [39, 58], "local_quorum": 39, "local_on": 39, "serial": [39, 45, 57, 88, 93, 112, 119, 246, 264, 284, 368, 384, 390, 427, 432, 445], "local_seri": 39, "v3": [39, 112, 117], "v4": [39, 51], "v5": [39, 407], "v6": [39, 407], "size_estim": 39, "partion": 39, "answer": [39, 144, 284, 347], "linger": 39, "close": [39, 41, 65, 109, 141, 159, 251, 254, 264, 275, 287, 304, 334, 368, 384, 390, 392, 402, 434], "unsent": 39, "socket": [39, 51, 58, 132, 156, 167, 303, 360, 388, 390], "acknowledg": [39, 57, 65, 261, 268], "replica": [39, 60, 137, 417], "downgrading_consist": 39, "fallthrough": 39, "datacent": 39, "failov": 39, "defaultloadbalancingpolici": 39, "allnodesfailedexcept": 39, "specul": [39, 231, 389], "500m": [39, 43], "example_keyspac": 39, "keyspac": 39, "cqlsh": 39, "cql": [39, 300], "simplestrategi": 39, "replication_factor": 39, "user_id": [39, 41, 45, 60, 110, 146, 480], "fname": 39, "lname": 39, "utf": [39, 57, 59, 73, 78, 84, 124, 128, 129, 136, 156, 210, 261, 265, 342, 451], "varint": 39, "blob": [39, 45, 47, 48, 59, 62, 66, 68, 78, 132, 173, 297, 342, 408, 420], "tupl": [39, 54, 390], "anonym": [39, 50, 113, 264, 267, 328, 333, 421, 439, 520, 544], "udt": 39, "inet": [39, 417], "ipaddress": [39, 40, 43, 84, 111, 247, 267, 269, 271, 345, 396, 417], "uuid": [39, 40, 51, 54, 55, 71, 84, 98, 112, 114, 133, 140, 162, 281, 339, 340, 343, 345, 384, 389, 391, 392, 397, 414, 418, 432, 449], "timeuuid": 39, "_cassandra": 39, "21": [40, 41, 58, 104, 107, 135, 137, 153, 204, 209, 396, 425, 506], "altin": [40, 398], "8123": 40, "exampleus": [40, 55], "examplepassword": [40, 55], "sensit": [40, 42, 52, 62, 66, 68, 71, 74, 76, 84, 104, 112, 121, 124, 126, 142, 197, 223, 244, 313, 380, 435, 436, 450, 468], "save": [40, 45, 47, 55, 60, 83, 119, 148, 293, 335, 368, 414, 475, 513], "miss": [40, 42, 45, 55, 62, 66, 68, 69, 71, 74, 75, 76, 100, 112, 117, 148, 162, 198, 268, 269, 272, 274, 276, 296, 306, 333, 345, 346, 355, 367, 369, 371, 383, 392, 398, 403, 405, 433, 442, 443, 452, 453], "datasourc": [40, 42, 55, 66, 68, 69, 71, 74, 75, 76, 287, 331, 387], "who": [40, 42, 52, 55, 66, 68, 69, 71, 74, 75, 76, 77, 95, 112, 262, 462, 467, 468, 469, 475, 542], "trace_token": [40, 42, 55, 66, 68, 69, 71, 74, 75, 76], "example_t": [40, 41, 42, 45, 54, 55, 60, 66, 68, 69, 71, 74, 75, 76], "jane": [40, 42, 55, 66, 68, 69, 71, 74, 75, 76], "20230622_180528_00000_bkizg": [40, 42, 55, 66, 68, 69, 71, 74, 75, 76], "compromis": [40, 41, 42, 55, 62, 66, 68, 69, 71, 74, 75, 76], "simpler": [40, 42, 55, 62, 66, 68, 69, 71, 74, 75, 76, 84, 137], "domain_compaction_threshold": [40, 42, 55, 62, 66, 68, 69, 71, 74, 75, 76, 374], "flush_metadata_cach": [40, 41, 42, 45, 55, 66, 68, 69, 71, 74, 75, 76, 394, 396, 425, 426, 438], "lowercas": [40, 42, 45, 55, 62, 66, 68, 69, 71, 74, 75, 76, 88, 124, 128, 136, 362, 368, 383, 402, 405, 428, 436, 476], "ambigu": [40, 42, 55, 62, 66, 68, 69, 71, 74, 75, 76, 211, 380, 520], "remoteschema": [40, 42, 55, 62, 66, 68, 69, 71, 74, 75, 76], "casesensitivenam": [40, 42, 55, 62, 66, 68, 69, 71, 74, 75, 76], "case_insensitive_1": [40, 42, 55, 62, 66, 68, 69, 71, 74, 75, 76], "case_insensitive_2": [40, 42, 55, 62, 66, 68, 69, 71, 74, 75, 76], "remotet": [40, 42, 55, 62, 66, 68, 69, 71, 74, 75, 76], "tablex": [40, 42, 55, 62, 66, 68, 69, 71, 74, 75, 76], "table_1": [40, 42, 55, 62, 66, 68, 69, 71, 74, 75, 76], "table_2": [40, 42, 55, 62, 66, 68, 69, 71, 74, 75, 76], "forward": [40, 42, 55, 59, 62, 66, 68, 69, 71, 74, 75, 76, 82, 90, 92, 137, 267, 274, 276, 293, 344, 346, 361, 386, 467, 469, 475, 510], "non_transactional_insert": [40, 55, 62, 66, 68, 69, 71, 74, 75, 76, 387], "corrupt": [40, 41, 55, 58, 62, 63, 66, 68, 69, 71, 74, 75, 76, 105, 197, 228, 242, 248, 257, 267, 277, 278, 325, 327, 354, 373, 387, 396, 397, 398, 407, 424, 425, 427, 429, 442], "rare": [40, 45, 55, 62, 66, 68, 69, 71, 74, 75, 76, 188, 193, 194, 217, 235, 245, 248, 261, 263, 277, 297, 316, 332, 336, 338, 346, 350, 359, 360, 363, 450], "rollback": [40, 55, 62, 66, 68, 69, 71, 74, 75, 76, 133, 143, 155, 355, 477, 485, 505, 540], "trino_ck": 40, "logdat": [40, 55], "mergetre": [40, 382], "order_bi": 40, "partition_bi": 40, "toyyyymm": 40, "primary_kei": [40, 60], "sample_bi": [40, 436], "tech": 40, "doc": [40, 43, 44, 55, 57, 78], "en": [40, 58, 83], "replicatedmergetre": 40, "int8": [40, 60, 148], "bool": [40, 57, 60, 62, 78], "int1": 40, "int16": [40, 60], "int2": 40, "int32": [40, 57, 60, 65, 245, 247, 440], "int4": [40, 148], "alia": [40, 71, 79, 80, 99, 107, 118, 124, 128, 144, 169, 217, 232, 240, 317, 373, 380, 386, 419, 438, 464, 495, 497, 520], "uint8": [40, 401], "uint16": [40, 401], "uint32": [40, 57, 401], "uint64": [40, 57, 401], "float32": 40, "fixedstr": [40, 396], "enum8": 40, "enum16": 40, "convert_to_varchar": [40, 42, 62, 66, 68, 69, 71, 74, 75, 76, 354, 355, 368], "convert": [40, 42, 45, 51, 57, 58, 59, 62, 64, 66, 68, 69, 71, 73, 74, 75, 76, 84, 88, 99, 106, 107, 109, 110, 112, 118, 124, 129, 151, 162, 167, 247, 275, 276, 287, 303, 309, 350, 354, 362, 367, 434, 438, 473, 475, 520], "unbound": [40, 42, 60, 62, 66, 68, 69, 71, 74, 75, 76, 78, 203, 209, 274, 510, 512, 520], "unsupported_type_handl": [40, 42, 62, 66, 68, 69, 71, 74, 75, 76, 354, 355], "risk": [40, 62, 66, 68, 70, 71, 73, 75, 76, 395, 467], "priorit": [40, 62, 66, 68, 70, 71, 73, 75, 76, 246], "textual": [40, 55, 57, 59, 62, 66, 71, 75, 112, 151, 278], "canada": [40, 55, 62, 66, 68, 71, 73, 75], "databrick": [41, 417, 432, 441, 447, 453], "lt": [41, 417, 441, 453], "11": [41, 54, 58, 70, 71, 79, 80, 99, 102, 107, 111, 112, 117, 136, 153, 253, 353, 357, 360, 374, 381, 386, 414, 451, 506, 520], "metastor": [41, 45, 46, 47, 49, 50, 53, 54, 94, 137, 167, 173, 178, 186, 206, 208, 217, 218, 223, 225, 228, 229, 231, 233, 239, 240, 254, 255, 257, 259, 260, 262, 263, 275, 277, 278, 279, 288, 296, 297, 317, 328, 331, 337, 338, 339, 341, 342, 343, 344, 346, 347, 348, 350, 352, 353, 354, 355, 357, 358, 359, 360, 361, 366, 367, 368, 369, 373, 375, 380, 381, 386, 387, 389, 396, 397, 398, 399, 401, 404, 405, 406, 409, 417, 418, 420, 421, 422, 424, 425, 426, 427, 428, 429, 430, 432, 434, 436, 438, 439, 445, 450, 453], "hm": [41, 45, 49, 53, 54, 64], "9083": [41, 45, 50, 53, 54, 64, 137], "delta_lak": [41, 50, 435], "visibl": [41, 76, 77, 124, 156, 232, 336, 344, 373, 375, 510], "consult": [41, 54, 464, 465], "live": [41, 48, 57, 64, 69, 73, 190, 235, 239, 257, 274, 296, 303, 327, 330, 361, 396, 419], "30m": 41, "codec": [41, 45, 54, 196, 281, 341, 412], "snappi": [41, 45, 54, 60, 69, 281, 389], "zstd": [41, 45, 54, 278, 341, 392, 418], "compression_codec": [41, 412], "hide": [41, 53, 58, 64, 65, 354, 356, 358, 371, 394, 397, 427, 471], "warn": [41, 52, 57, 83, 88, 136, 137, 209, 279, 333, 347, 432], "checkpoint": [41, 325, 403, 422, 423, 449, 520], "nth": [41, 121, 352], "checkpoint_interv": 41, "dynamic_filtering_wait_timeout": [41, 45, 374], "statistics_en": [41, 45, 54, 242], "extended_statistics_en": [41, 54], "extended_statistics_collect_on_writ": 41, "fallback": [41, 51, 54, 64, 282, 357, 375, 410, 436], "target_max_file_s": [41, 389, 398, 413, 451], "register_t": [41, 54, 430, 432, 434, 449], "vacuum_min_retent": 41, "parquet_max_read_block_s": 41, "parquet_writer_block_s": [41, 278], "128mb": [41, 53, 63, 70], "parquet_writer_page_s": [41, 278], "parquet_writer_batch_s": 41, "projection_pushdown_en": [41, 360], "binari": [41, 43, 45, 48, 54, 55, 60, 65, 66, 69, 78, 97, 98, 102, 109, 121, 124, 132, 133, 144, 186, 276, 283, 289, 302, 307, 317, 368, 393, 395, 439, 447, 475], "timestampntz": 41, "timestamp_ntz": [41, 450], "struct": [41, 45, 54, 57, 78, 155, 225, 272, 274, 287, 309, 324, 379, 385, 390, 402, 433], "read_onli": [41, 54], "permit": [41, 52, 54, 144, 460], "snippet": [41, 54], "example_procedur": [41, 45, 54], "examplecatalog": [41, 54], "caller": [41, 54, 83, 167], "schema_nam": [41, 45, 54, 73, 77, 126, 488, 494, 500, 507, 517, 529], "testdb": [41, 54], "table_nam": [41, 45, 54, 62, 65, 66, 71, 73, 76, 259, 260, 329, 346, 482, 489, 490, 493, 494, 495, 501, 507, 509, 517, 520, 530, 533, 541, 542], "customer_ord": [41, 54], "table_loc": [41, 45, 54], "unauthor": [41, 52, 54, 355, 365, 432], "unregister_t": [41, 54, 434], "snapshot": [41, 53, 56, 76, 151, 280, 373, 379, 381, 383, 388, 406, 408, 413, 414, 416, 419, 420, 423, 438], "minu": [41, 112], "exampleschemanam": 41, "exampletablenam": 41, "7d": [41, 48, 54, 112], "safeti": [41, 167, 334, 344], "retain": [41, 45, 99, 320, 367, 419, 444, 517], "adl": [41, 346, 357, 365, 383], "gen2": [41, 357], "collis": [41, 230], "modif": [41, 45, 54, 57, 84, 94, 397, 435], "subdirectori": [41, 45, 54, 93, 139, 297], "omit": [41, 45, 54, 57, 59, 73, 99, 112, 124, 148, 365, 510, 520], "deprec": [41, 112, 154, 155, 176, 212, 217, 221, 226, 228, 232, 251, 254, 261, 264, 265, 266, 270, 272, 296, 328, 329, 330, 331, 333, 339, 341, 344, 347, 351, 353, 354, 357, 358, 359, 365, 367, 368, 373, 378, 380, 384, 385, 387, 389, 390, 392, 397, 398, 401, 402, 405, 408, 413, 414, 416, 418, 419, 420, 423, 425, 426, 429, 430, 432, 435, 436, 438, 440, 443, 444, 447, 450, 453, 455, 462], "temporarili": [41, 233, 357, 366, 424, 469, 523], "legaci": [41, 45, 52, 67, 132, 188, 207, 212, 217, 219, 220, 226, 228, 232, 246, 256, 261, 264, 265, 266, 268, 270, 271, 272, 276, 278, 280, 302, 328, 332, 333, 344, 346, 361, 365, 366, 368, 373, 375, 378, 384, 385, 389, 390, 392, 395, 397, 407, 408, 409, 414, 424, 425, 426, 432, 433, 434, 436, 438, 446, 447, 449, 450, 451, 454, 455, 462, 464, 475], "legacy_create_table_with_existing_location_en": [41, 432], "new_tabl": 41, "rewrit": [41, 45, 54, 192, 272, 278, 279, 282, 307, 374], "compact": [41, 45, 54, 78, 166, 194, 253, 350, 368, 372, 374, 390, 392, 399, 423, 426, 434], "file_size_threshold": [41, 45, 54, 391, 450, 480], "test_tabl": [41, 45, 53, 54, 480], "10mb": [41, 45, 54, 480], "test_partitioned_t": [41, 45, 54], "partition_kei": [41, 45, 54], "partitioned_bi": [41, 45, 177, 530], "change_data_feed_en": [41, 435], "feed": [41, 274, 428, 434, 435, 440], "column_mapping_mod": 41, "example_partitioned_t": 41, "changelog": [41, 54], "user_nam": [41, 58], "operation_paramet": 41, "cluster_id": 41, "read_vers": 41, "isolation_level": 41, "is_blind_append": 41, "2023": [41, 45], "40": [41, 54, 103, 109, 218, 505, 506, 520], "54": [41, 58, 72, 153, 328], "684": 41, "queryid": 41, "20230119_064054_00008_4vq5t": 41, "406": [41, 153], "writeserializ": 41, "41": [41, 104, 506], "373": [41, 153, 332], "20230119_064041_00007_4vq5t": 41, "497": 41, "20230119_064010_00005_4vq5t": 41, "becam": [41, 54, 275], "isol": [41, 76, 93, 183, 379, 383, 406, 540], "minreadervers": 41, "minwritervers": 41, "columnmap": [41, 84, 87, 413, 415, 423], "file_modified_tim": [41, 45, 54, 346, 361, 382, 419, 420], "file_s": [41, 45, 346], "cdf": [41, 118], "test_schema": 41, "tablenam": [41, 45, 57, 58, 59, 73, 78, 142], "since_vers": 41, "_change_typ": [41, 440], "give": [41, 45, 109, 144, 276, 382, 506], "update_preimag": 41, "update_postimag": 41, "_commit_vers": [41, 440], "_commit_timestamp": [41, 440], "repres": [41, 43, 45, 54, 55, 57, 59, 65, 72, 73, 78, 83, 84, 89, 91, 94, 96, 100, 108, 109, 112, 118, 119, 120, 122, 127, 136, 144, 147, 155, 247, 248, 249, 293, 339, 358, 368, 375, 377, 399, 423, 470, 476, 505, 513, 520, 538], "happen": [41, 45, 57, 59, 70, 73, 86, 88, 95, 166, 193, 261, 264, 267, 388, 425, 432, 512], "page_url": [41, 45], "url1": 41, "domain1": 41, "url2": 41, "domain2": 41, "url3": 41, "url4": 41, "400": [41, 153], "url5": 41, "url6": 41, "domain3": 41, "domain4": 41, "asc": [41, 148, 250, 520], "10t21": 41, "0000": [41, 58], "10t22": 41, "10t20": 41, "ndv": 41, "fly": [41, 79, 80], "broader": 41, "reorder": [41, 54, 146, 227, 272, 280, 355, 356, 358], "table_schema": [41, 71, 76, 346], "recalcul": [41, 447], "scratch": 41, "full_refresh": 41, "gain": [41, 48, 57, 148, 355], "files_modified_aft": 41, "2021": [41, 54, 55, 65], "321": [41, 107, 144, 153, 350], "z": [41, 60, 99, 100, 112, 117, 118, 121, 124, 144, 155, 468, 520], "stamp": 41, "broaden": 41, "reanalyz": 41, "drop_extended_stat": [41, 402], "grow": [41, 395, 396], "provis": [41, 51, 387, 474], "routin": [41, 76], "good": [41, 65, 84, 93, 136, 137], "deltalak": 41, "transactionlog": 41, "transactionlogaccess": 41, "bean": [41, 56, 58], "datafilemetadatacachestat": 41, "hitrat": 41, "97": [41, 153, 328], "missrat": 41, "requestcount": 41, "3232": 41, "metadatacachestat": 41, "98": [41, 153], "02": [41, 45, 54, 57, 60, 107, 144, 209, 262, 293, 482, 505, 520], "6783": [41, 380], "object_nam": [41, 56], "healthi": [41, 135, 356], "transpar": [41, 45, 46, 48, 54], "accustom": [41, 45, 54], "varieti": [41, 45, 46, 54, 151, 471], "concret": [41, 45, 54, 60, 88, 148, 421], "let": [41, 45, 54, 59, 68, 70, 84, 112, 366], "columnnam": [41, 45, 54, 505], "tablescan": [41, 45, 54, 84, 148, 505], "another_catalog": [41, 45, 54], "expert": [41, 45], "degrad": [41, 45, 66, 71, 76, 253, 266, 356, 466], "suggest": [41, 66, 71, 76, 112, 192, 408, 469], "trivial": [41, 248, 257, 258, 275, 303], "backup": [41, 310, 329], "filesystem": [41, 47, 49, 51, 132, 136, 138, 139, 217, 251, 288, 301, 393, 474], "max_valu": [41, 53], "max_initial_split_s": [41, 206], "max_split_s": [41, 206, 462], "query_partition_filter_requir": [41, 45, 54, 358, 389, 450, 457], "broker": [42, 57, 58, 70, 328, 421, 427], "8082": 42, "avatica": 42, "v2": [42, 51, 54, 58, 112, 117, 123, 332, 345, 403, 407, 410, 420], "_time": 42, "string_to_mv": 42, "mv_length": 42, "num_report": 42, "direct_report": 42, "opensearch": 43, "9200": 43, "aliv": [43, 388], "init": [43, 70, 139, 177], "backpressur": [43, 278], "overwhelm": 43, "struggl": 43, "es": [43, 58, 425], "publish": [43, 94, 135, 137, 287, 378], "annot": [43, 88, 93, 213, 270, 389, 420, 440], "_meta": [43, 393], "array_string_field": 43, "lean": 43, "oh": 43, "long_field": 43, "314159265359": 43, "id_field": 43, "564e6982": 43, "88ee": 43, "4498": 43, "aa98": 43, "df9e3f6b6109": 43, "timestamp_field": 43, "1987": 43, "17t06": 43, "000z": 43, "object_field": 43, "array_int_field": 43, "86": [43, 107, 153], "309": [43, 153], "int_field": 43, "_map": 43, "isarrai": 43, "asrawjson": 43, "occurr": [43, 100, 121, 141, 240, 352, 446, 510], "multidimension": 43, "besto": 43, "dense_vector": 43, "constantli": [43, 510], "emerg": [43, 461], "beneath": 43, "_id": [43, 65, 347], "_score": [43, 347], "_sourc": [43, 347], "tweet": 43, "NO": [43, 178, 490], "chain": [43, 59, 64, 132, 209, 379, 464, 465, 520], "arn": [43, 51, 64], "elast": [43, 137, 392], "dsl": 43, "payload": [43, 227], "spreadsheet": 44, "gsheet": [44, 432], "exampleid": 44, "menu": 44, "panel": 44, "anyth": [44, 136, 406], "email": [44, 132, 468, 470], "propag": [44, 57, 269, 284, 287, 373], "503": [44, 83], "unavail": [44, 66, 71, 76, 538], "information_schema": [44, 66, 79, 80, 188, 239, 256, 258, 259, 260, 262, 264, 275, 297, 306, 311, 313, 329, 337, 339, 341, 346, 347, 348, 354, 356, 357, 358, 359, 362, 365, 371, 380, 384, 385, 386, 387, 388, 397, 400, 402, 405, 406, 413, 419, 422, 445, 450, 462, 543], "googlesheetidher": 44, "tabnam": 44, "a1": [44, 117], "b4": [44, 101, 117], "warehous": [45, 52, 54, 64, 307], "mapreduc": [45, 152, 287], "tez": [45, 339], "ibm": [45, 428], "rctext": [45, 67, 301, 364, 369, 449], "rcfile": [45, 67, 217, 220, 233, 242, 245, 246, 250, 256, 282, 283, 293, 299, 303, 306, 308, 332, 345, 368, 392, 395, 453], "columnarserd": [45, 67, 293], "rcbinari": [45, 67, 301, 303, 368, 369], "lazybinarycolumnarserd": [45, 67, 282, 283], "sequencefil": [45, 301, 357, 364, 368, 379, 392, 453], "org": [45, 51, 55, 57, 58, 59, 64, 67, 69, 73, 231, 470, 473, 475], "hcatalog": [45, 67, 231], "jsonserd": [45, 67, 231], "serde2": [45, 67], "opencsvserd": [45, 67], "textfil": [45, 301, 346, 364, 450], "feder": [45, 132], "namenod": [45, 186], "referenc": [45, 51, 54, 56, 112, 121, 151, 154, 197, 213, 228, 270, 275, 325, 331, 350, 359, 365, 367, 382, 383, 403, 432, 433, 459, 470, 486, 491, 510, 512, 520], "copi": [45, 48, 53, 54, 57, 76, 124, 139, 217, 263, 303, 358, 475, 489], "dir": [45, 64, 93, 136], "nobodi": 45, "hadoop_user_nam": 45, "hdfs_user": 45, "dhadoop_user_nam": 45, "whenev": [45, 47, 66, 71, 76, 88, 93, 144, 284, 510], "throughout": [45, 96, 151], "recurs": [45, 112, 142, 155, 297, 367, 368, 389, 415, 441, 449, 534], "mapr": 45, "absent": [45, 57, 59, 73, 83, 112, 278, 357, 435], "lz4": [45, 54, 60, 69, 209, 260, 281, 341], "datanod": 45, "colloc": 45, "immut": [45, 180], "eager": [45, 66, 71, 76, 253, 427, 434], "eagerli": [45, 261], "fruit": [45, 288], "appl": [45, 121, 288, 483], "orang": [45, 121], "veget": 45, "round": [45, 66, 68, 69, 71, 75, 114, 115, 118, 144, 211, 217, 261, 264, 277, 348, 365, 371, 390, 427, 432, 505], "never": [45, 112, 144, 218, 251, 315, 394, 420, 475], "translat": [45, 49, 70, 72, 84, 94, 114, 115, 124, 132, 140, 144, 151, 265, 364, 365, 372, 375, 380, 388, 391, 407, 419, 424, 425], "hive_views_legacy_transl": [45, 407], "fs": [45, 47, 49, 359, 369, 391, 475], "octal": [45, 144, 447], "newli": [45, 47, 51, 57, 59, 73, 293, 361, 438, 488, 489, 490], "0777": [45, 359], "smart": 45, "purg": [45, 236, 407], "auto_purg": [45, 407], "athena": [45, 418], "advic": 45, "regard": [45, 362, 379, 400], "view_tim": 45, "ds": [45, 79, 247, 282, 335], "bucketed_bi": [45, 208], "bucket_count": 45, "2016": [45, 56, 94, 142, 209], "request_log": 45, "request_tim": 45, "user_ag": 45, "external_loc": [45, 47, 221, 329, 367], "create_empty_partit": [45, 274], "partition_column": [45, 426], "partition_valu": [45, 362, 426], "drop_stat": [45, 356, 362], "sync_partition_metadata": [45, 331, 344, 362, 404, 409], "case_sensit": [45, 362], "msck": 45, "repair": 45, "col_x": 45, "somevalu": 45, "conform": [45, 57, 59, 73, 112, 129, 257, 261, 271, 280, 317, 366], "convent": [45, 60, 96, 137, 339, 448], "register_partit": [45, 357, 366], "unregister_partit": [45, 357], "unregist": 45, "acid": [45, 352, 358, 364, 368, 372, 379, 381, 383, 386, 387, 388, 390, 392, 396, 401, 432], "uncommon": 45, "widen": [45, 542], "liter": [45, 60, 78, 97, 104, 107, 121, 144, 163, 167, 186, 192, 198, 204, 209, 213, 239, 250, 255, 262, 263, 279, 287, 358, 375, 377, 399, 428, 432, 447, 450, 455, 513, 525, 544], "infer": [45, 148, 262, 278, 436], "avro_schema_url": [45, 275, 390], "avro_data": [45, 57, 59, 73], "avsc": [45, 57, 59, 73], "s3n": [45, 51, 409], "schema_bucket": 45, "avro_schema_liter": [45, 428], "backward": [45, 57, 59, 73, 76, 155, 159, 167, 172, 173, 188, 194, 212, 213, 217, 220, 226, 251, 255, 268, 270, 274, 275, 282, 283, 287, 293, 294, 302, 305, 316, 317, 325, 327, 336, 342, 350, 368, 393, 402, 438, 462, 505, 510], "coercion": [45, 57, 59, 73, 148, 156, 159, 165, 172, 173, 192, 198, 201, 206, 231, 233, 237, 242, 248, 251, 255, 260, 267, 271, 273, 293, 324, 333, 337, 346, 433, 438, 450, 453], "thrown": [45, 57, 59, 73], "incompat": [45, 57, 59, 73, 76, 155, 156, 159, 167, 172, 173, 188, 194, 212, 213, 217, 220, 251, 255, 270, 275, 282, 283, 287, 293, 294, 296, 302, 305, 316, 317, 325, 327, 334, 336, 342, 350, 388, 391, 402, 413, 415, 438, 447, 454], "non_transactional_optimize_en": 45, "catalog_nam": [45, 48, 77, 126], "outcom": [45, 104, 259, 277, 520], "duplic": [45, 57, 100, 109, 112, 123, 124, 162, 194, 209, 242, 267, 268, 297, 339, 367, 368, 375, 389, 394, 400, 405, 421, 450, 455, 520], "cleanup": [45, 208, 260, 274, 334, 360], "csv_escap": 45, "trash": 45, "bucketing_vers": [45, 358], "csv_quot": 45, "csv_separ": 45, "unicod": [45, 121, 144, 239, 303, 330, 450], "invis": 45, "0009": 45, "null_format": [45, 364, 448], "orc_bloom_filter_column": [45, 54], "bloom": [45, 54, 67, 217, 265, 341, 342, 396, 409, 414, 433, 442, 446], "orc_bloom_filter_fpp": [45, 54], "posit": [45, 54, 57, 59, 65, 67, 68, 73, 84, 96, 100, 101, 102, 114, 115, 118, 124, 144, 155, 162, 180, 191, 204, 209, 240, 247, 257, 269, 331, 341, 354, 387, 434, 483, 496, 510, 512, 520], "probabl": [45, 54, 95, 118, 468, 520], "skip_footer_line_count": [45, 348], "footer": [45, 67, 264, 270, 271, 331, 341, 348, 350, 356, 446], "skip_header_line_count": [45, 348, 386], "sorted_bi": [45, 54, 267, 436], "textfile_field_separ": [45, 346], "textfile_field_separator_escap": [45, 346], "partition_projection_en": 45, "partition_projection_ignor": [45, 64], "around": [45, 88, 109, 248, 271, 368], "partition_projection_location_templ": [45, 437], "s3a": [45, 51, 54, 155, 409], "partition_nam": 45, "extra_properti": [45, 444], "raw": [45, 58, 68, 78, 254, 280, 361, 365, 367], "stats_generated_via_stats_task": 45, "presto_query_id": [45, 217], "presto_vers": 45, "12730": 45, "20230705_152456_00001_nfugi": 45, "423": [45, 153], "pol": 45, "usa": 45, "partition_projection_typ": 45, "enum": [45, 66, 71, 78, 333, 419], "inject": [45, 84, 213, 287, 296, 324, 436], "partition_projection_valu": 45, "partition_projection_rang": 45, "partition_projection_interv": 45, "2001": [45, 60, 68, 107, 111, 112, 144, 293], "3dai": 45, "partition_projection_digit": 45, "digit": [45, 68, 107, 108, 118, 124, 128, 144, 210, 380], "partition_projection_format": 45, "yyyi": [45, 57, 128, 421], "mm": [45, 57, 109, 128, 144, 278, 421], "dd": [45, 57, 128, 421], "yy": [45, 128], "hh": [45, 57, 128, 144], "ss": [45, 57, 128], "datetimeformatt": 45, "partition_projection_interval_unit": 45, "signatur": [45, 51, 123, 132, 144, 264, 342, 380, 398, 450, 467], "union": [45, 54, 99, 100, 104, 109, 110, 117, 123, 142, 155, 165, 172, 173, 183, 196, 209, 210, 225, 235, 239, 240, 264, 278, 280, 287, 302, 305, 313, 316, 317, 324, 330, 346, 362, 380, 381, 386, 406, 413, 426, 437, 444, 512], "current_us": [45, 114, 115, 122, 142, 266, 380, 462, 487, 491, 508, 518], "json_tupl": [45, 382], "experi": [45, 138, 296, 337, 394], "incomplet": [45, 287, 369], "current_d": [45, 65, 107, 110, 114, 115, 142, 191, 234, 263], "current_timestamp": [45, 107, 114, 115, 142, 445], "honor": [45, 209, 360, 379], "udf": [45, 285], "numfil": 45, "numrow": [45, 274], "rawdatas": [45, 267], "totals": [45, 267], "p1_value1": 45, "p1_value2": 45, "p2_value1": 45, "p2_value2": 45, "detriment": 45, "unnecessari": [45, 54, 66, 68, 69, 71, 75, 146, 148, 219, 239, 242, 255, 261, 262, 269, 270, 272, 274, 330, 331, 347, 359, 383, 396, 397, 420, 462, 475], "One": [45, 47, 51, 52, 86, 87, 112, 123], "col_1": [45, 54], "col_2": [45, 54], "greatli": 45, "boost": [45, 58, 84], "sy": [45, 306, 311, 358, 365, 383], "21002": 45, "22167": 45, "gather": [45, 96, 138, 147, 249, 505], "orchestr": [46, 94, 135, 137, 151], "dispar": 46, "learn": [47, 88, 98, 99, 132, 133, 148, 149, 301], "offici": [47, 401], "contributor": [47, 81, 132, 151], "advis": 47, "clientcredenti": 47, "registr": [47, 136], "involv": [47, 48, 77, 96, 99, 104, 112, 132, 151, 157, 170, 178, 180, 183, 209, 218, 226, 232, 234, 239, 240, 242, 255, 257, 263, 267, 274, 275, 287, 313, 316, 317, 327, 328, 329, 330, 331, 334, 335, 336, 337, 339, 340, 341, 346, 347, 352, 353, 354, 358, 359, 360, 361, 364, 366, 368, 369, 373, 374, 375, 379, 381, 384, 385, 386, 389, 392, 395, 400, 401, 402, 403, 404, 405, 407, 410, 416, 417, 420, 421, 423, 424, 425, 426, 429, 430, 432, 433, 434, 435, 438, 439, 440, 441, 446, 448, 450, 451, 453, 454, 455, 457, 469, 512], "path_to_hadoop_cor": 47, "ask": [47, 51, 86, 104, 144, 242, 461], "abfsexampl": 47, "az": 47, "abfswales1": 47, "connectionstr": 47, "defaultendpointsprotocol": 47, "endpointsuffix": 47, "accountnam": 47, "accountkei": 47, "examplekei": 47, "jcek": 47, "distinctli": 47, "file_system": 47, "file_nam": 47, "data_lake_storage_gen1_nam": 47, "azuredatalakestor": 47, "orderkei": [47, 50, 65, 105, 131, 148, 486, 489, 491, 493, 505, 520, 530], "custkei": [47, 50, 99, 112, 148, 164, 302, 486, 505, 510], "orderstatu": [47, 50, 65, 126, 164, 302, 486, 489, 491, 510, 530], "totalpric": [47, 50, 65, 99, 105, 112, 131, 283, 486, 489, 490, 491, 510, 520, 530], "orderd": [47, 50, 65, 126, 131, 164, 283, 302, 486, 489, 490, 491, 506, 510, 520, 530], "orderprior": [47, 50], "clerk": [47, 50, 126, 131, 506, 520], "shipprior": [47, 50], "79": [47, 50, 153], "everyth": [47, 123, 124, 137, 151, 173, 500], "abfs_export": 47, "my_tabl": 47, "orders_abf": 47, "10gb": 48, "center": 48, "mention": [48, 112, 151, 510, 511, 520], "consider": [48, 66, 76, 99, 389, 466], "public": [48, 55, 84, 86, 88, 96, 132, 148, 270, 287, 347, 358, 465, 467, 470, 473, 479, 480, 488, 494, 507, 517], "async": [48, 58], "asynchron": [48, 64, 220, 303, 361, 434], "doesn": [48, 51, 68, 77, 84, 94, 104, 108, 112, 196, 403, 419, 424, 430, 435, 520], "pai": [48, 345], "warm": 48, "background": [48, 53, 135, 167, 235, 334, 375, 520], "bypass": [48, 64, 473], "opt": [48, 333, 466], "bookkeep": 48, "8898": 48, "8899": [48, 388], "toggl": [48, 155, 173, 180, 279, 398], "tip": [48, 137], "cache1": 48, "cache2": 48, "crucial": 48, "attach": [48, 136], "ssd": 48, "achiev": [48, 57, 65, 99, 104, 136, 462, 468], "ram": [48, 63, 136], "encapsul": [48, 123, 144], "month": [48, 54, 84, 100, 107, 114, 128, 204, 209, 296, 356, 366], "1tb": 48, "decid": [48, 84, 267, 342], "cft": 48, "rubix": 48, "cache_hit": 48, "is_nan": [48, 114, 115, 118], "ibmco": 49, "service1": 49, "key1": [49, 112, 307], "endpoint1": 49, "service2": 49, "key2": [49, 112, 307], "endpoint2": 49, "orders_tini": 49, "stocat": 49, "impl": [49, 64, 366], "objectstorefilesystem": 49, "cosapicli": 49, "eu": 49, "de": [49, 58, 197, 445], "appdomain": 49, "gcp_credentials_file_path": 50, "storage_catalog": [50, 77], "sales_data_in_gc": 50, "example_loc": 50, "folder": [50, 57, 58, 59, 73, 135, 367, 380], "accomplish": 51, "amazonaw": [51, 59, 64, 281], "intelligent_ti": [51, 358], "signer": [51, 217, 461], "s3signertyp": 51, "tmpdir": [51, 136], "pin": [51, 64, 183], "ec2": [51, 59, 64, 155, 173, 183], "sse": [51, 183], "km": [51, 109, 217, 219, 232, 343, 379, 422], "sdk": [51, 288], "encryptionmaterialsprovid": [51, 183], "acl": [51, 52, 279], "privat": [51, 52, 57, 64, 84, 132, 137, 464, 465, 467, 475], "authenticated_read": 51, "aws_exec_read": 51, "bucket_owner_full_control": 51, "bucket_owner_read": 51, "log_delivery_writ": 51, "public_read": 51, "public_read_writ": 51, "glacier": [51, 331, 368], "multipart": 51, "preemptiv": 51, "st": [51, 64, 355, 396, 401, 426], "facil": 51, "govern": [51, 124, 146, 462], "cleaner": 51, "awscredentialsprovid": 51, "constructor": [51, 212, 240, 264, 270, 347, 351, 358, 402, 424], "stssessioncredentialsprovid": 51, "stsassumerolesessioncredentialsprovid": 51, "flexibl": [51, 59, 84, 276, 359], "alic": [51, 462, 468, 471, 476, 479, 480, 481, 488, 494, 507, 517], "financ": [51, 462, 476, 494, 507, 517, 543], "accesskei": 51, "secretkei": [51, 59], "iamrol": 51, "although": [51, 112, 137, 192, 282, 303, 307, 475, 476], "anywai": 51, "rolesessionnam": 51, "portion": [51, 59, 109, 282, 510, 512, 538], "allowediamrol": 51, "kmskeyid": 51, "allowedkmskeyid": 51, "cotain": 51, "useclusterdefault": 51, "deni": [51, 95, 133, 143, 270, 371, 393, 403, 432, 434, 462, 468, 469, 470, 471, 476, 477, 505, 507, 517], "123456789101": 51, "test_path": 51, "charli": 51, "test_default": 51, "test1": 51, "test2": 51, "test3": 51, "akiaxxxaccess": 51, "ixbxxxxsecret": 51, "kmskey_10": 51, "test_us": 51, "finance_us": 51, "pointer": [51, 172, 462], "rfc": [51, 57, 58, 59, 73, 101, 112, 129, 144, 361, 462], "6901": [51, 462], "insid": [51, 54, 56, 78, 94, 109, 112, 126, 135, 136, 148, 176, 347, 354, 462, 464, 466, 475, 510], "endpont": 51, "clientconfigur": 51, "amazons3cli": 51, "reusag": 51, "ae": [51, 136, 408], "infrastructur": [51, 132, 137, 301, 447, 461, 475], "care": [51, 293, 462], "kerber": 52, "ticket": [52, 392, 434, 438, 468], "kdc": [52, 468], "precaut": 52, "ssh": 52, "ownership": [52, 132, 369, 371, 462], "sasl": 52, "_host": 52, "substitut": [52, 124, 371, 462, 476], "silent": [52, 63, 112, 183, 205, 217, 253, 254, 267, 317, 359, 390, 395, 471], "readabl": [52, 65, 83, 107, 144, 166, 276, 278, 373], "unsecur": 52, "discuss": [52, 64, 468, 475, 483], "superus": [52, 266], "someon": 52, "sync": 53, "meta": [53, 93, 143], "loader": [53, 58, 93, 217, 392], "breach": 53, "enqueu": [53, 263], "stock_ticks_cow": 53, "quickstart": 53, "ts": [53, 54], "goog": 53, "_col1": [53, 148, 288, 497, 505, 520], "2018": [53, 60, 270], "31": [53, 58, 107, 112, 121, 128, 144, 153, 183, 184, 397, 441], "59": [53, 54, 58, 107, 128, 153], "dt": 53, "99": [53, 112, 153, 328], "laid": [53, 520], "matrix": 53, "instant": [53, 66, 72, 144, 293, 454], "action": [53, 65, 84, 94, 136, 139, 143, 356, 358], "8667764846443717831": [53, 54], "7860805980949777961": [53, 54], "huge": 54, "spec": [54, 391, 392], "atom": [54, 217], "scalabl": [54, 132, 368, 372], "nessi": [54, 446], "hive_metastor": [54, 64], "topic": [54, 57, 59, 64, 73, 132, 133, 136, 143, 303, 365, 377, 441, 464, 465, 475], "storage_schema": [54, 77], "timestamptz": [54, 71], "l": [54, 58, 107, 121, 137, 146, 148], "k": [54, 99, 100, 107, 117, 168, 468], "explicit": [54, 58, 64, 124, 156, 287, 293, 368, 399, 417, 421], "example_s3_schema": 54, "minio": [54, 132], "example_s3a_schema": 54, "example_hdfs_schema": 54, "c1": [54, 271], "c2": [54, 271], "c3": 54, "tiny_n": 54, "flavor": 54, "yearly_click": 54, "year": [54, 60, 62, 68, 84, 100, 107, 114, 128, 204, 208, 209, 210, 296, 366, 438, 448], "20000": 54, "581fad8517934af6be1857a903559d44": 54, "figur": 54, "metadata_file_nam": 54, "00003": 54, "409702ba": 54, "4735": 54, "4645": 54, "8f14": 54, "09537cc0b2c8": 54, "move": [54, 93, 170, 209, 327, 338, 425, 447, 475], "recursive_directori": 54, "retention_threshold": 54, "00d": 54, "shorter": [54, 57, 100, 246, 377, 389, 468, 470], "orphan": 54, "format_vers": [54, 405, 412], "Or": [54, 60], "my_new_partition_column": 54, "fpp": 54, "made_current_at": 54, "snapshot_id": 54, "is_current_ancestor": 54, "committed_at": 54, "manifest_list": 54, "snap": 54, "100cf97e": 54, "6d56": 54, "446e": 54, "8961": 54, "afdaded63bc4": 54, "faa19903": 54, "1455": [54, 346], "4bb8": 54, "855a": 54, "61a1bbafbaa7": 54, "442": 54, "overview": [54, 81, 97, 133, 210, 217, 377, 389, 458], "partition_spec_id": 54, "added_snapshot_id": 54, "added_data_files_count": 54, "added_rows_count": [54, 417], "existing_data_files_count": 54, "existing_rows_count": [54, 417], "deleted_data_files_count": 54, "deleted_rows_count": [54, 417], "m0": 54, "6277": 54, "contains_nul": 54, "contains_nan": 54, "lower_bound": 54, "upper_bound": 54, "record_count": 54, "file_count": [54, 392], "total_s": [54, 392], "884": 54, "null_count": 54, "nan_count": [54, 403], "file_path": 54, "file_format": 54, "file_size_in_byt": [54, 392], "column_s": [54, 213], "value_count": 54, "null_value_count": 54, "nan_value_count": 54, "key_metadata": [54, 438], "split_offset": [54, 438], "equality_id": [54, 438], "af9872b2": 54, "40f3": 54, "428f": 54, "9c87": 54, "186d2750d84e": 54, "44": 54, "position_delet": 54, "equality_delet": 54, "comparison": [54, 64, 84, 98, 114, 116, 124, 133, 144, 155, 175, 209, 220, 227, 229, 233, 242, 245, 271, 293, 297, 339, 360, 361, 364, 365, 383, 384, 388, 390, 401, 407, 437, 520], "branch": [54, 64, 434], "max_reference_age_in_m": 54, "min_snapshots_to_keep": 54, "max_snapshot_age_in_m": 54, "example_tag": 54, "10000000000": 54, "example_branch": 54, "20000000000": 54, "30000": [54, 70], "file_01": 54, "456": [54, 57, 112, 144, 350, 478, 480], "wioth": 54, "januari": [54, 107, 316], "1970": [54, 78, 107, 210, 217, 263, 267, 316, 365, 368, 369, 371, 381, 413], "nbucket": 54, "inclus": [54, 57, 59, 62, 73, 78, 84, 109, 112, 126, 210], "nchar": [54, 68, 76], "order_d": [54, 177, 490, 512], "account_numb": 54, "order_id": [54, 177], "sorted_writing_en": [54, 267], "desc": [54, 74, 76, 99, 131, 250, 505, 520], "histor": [54, 84, 361], "FOR": [54, 142, 249, 327, 454, 517, 518, 538], "OF": [54, 412, 413, 414, 421, 423, 454], "8954597067493422955": 54, "week": [54, 71, 107, 114, 144], "ago": [54, 135], "29": [54, 58, 107, 119, 153, 328], "803": 54, "rollback_to_snapshot": [54, 392, 415], "layout": [54, 94, 148, 162, 272, 278, 296, 317, 333, 341, 505, 506], "_date": [54, 486], "event_d": 54, "outdat": [54, 418, 420], "latter": [55, 270, 520], "10800": 55, "thin": [55, 68], "advers": [55, 71], "person": [55, 469], "chose": [55, 302], "dummy_id": 55, "9223372036854775808": [55, 339], "9223372036854775807": [55, 112], "2147483648": [55, 100], "32768": 55, "32767": 55, "hello": [55, 104, 106, 144], "1972": 55, "col1": [55, 60, 62, 66, 68, 70, 71, 74, 75, 76, 259, 520], "col3": [55, 62, 66, 68, 70, 71, 74, 75, 76], "arithmet": [55, 62, 66, 68, 71, 74, 75, 76, 99, 100, 102, 144, 206, 293, 395, 402, 429, 439], "col2": [55, 60, 62, 66, 68, 70, 71, 74, 75, 76, 259], "schema_on": [55, 68, 71, 74, 75, 76], "table_on": [55, 68, 71, 74, 75, 76], "table_two": [55, 68, 71, 74, 75, 76], "schema_two": [55, 68, 71, 74, 75, 76], "debug": [56, 85, 136, 137, 173, 263, 267, 273, 279, 321, 327, 329, 342, 344, 345, 346, 470, 475], "instrument": [56, 288], "dump": [56, 136, 226, 307, 328], "nodeschedul": 56, "86400": [56, 59, 107], "mbean": [56, 209, 263, 344], "memorypool": 56, "vmname": 56, "vmversion": 56, "ddc4df17": 56, "0b8e": 56, "4843": 56, "bb14": 56, "1b8af1a7451a": 56, "hotspot": 56, "tm": 56, "vm": [56, 218], "b09": 56, "openfiledescriptorcount": 56, "maxfiledescriptorcount": 56, "operatingsystem": 56, "329": [56, 153], "10240": 56, "wildcard": [56, 104, 263, 348, 409], "freebyt": 56, "214748364": 56, "1073741825": 56, "858993459": 56, "uptim": [56, 242, 462], "28": [56, 58, 65, 119, 153, 506, 520], "11420": 56, "21422": 56, "31412": 56, "arriv": [57, 58, 59], "disappear": 57, "segment": [57, 59, 64, 70, 302, 368, 486, 520], "strang": 57, "tutori": [57, 88, 303], "9092": [57, 58], "agre": 57, "licens": [57, 132], "agreement": 57, "table1": [57, 73], "table2": [57, 73], "verbos": [57, 136, 246, 276, 307, 358, 392, 396, 425, 426, 427, 428, 432, 506], "pollut": 57, "_": [57, 104, 117, 129, 136, 287, 361, 367, 428, 454, 476], "createtim": 57, "plaintext": 57, "64kb": 57, "_timestamp": [57, 58, 371, 420], "logappendtim": 57, "timestamp_upper_bound_force_push_down_en": 57, "pkcs12": 57, "internal_": 57, "_partition_id": [57, 58, 371], "internal_partition_id": 57, "_partition_offset": [57, 58, 59, 371], "offset": [57, 58, 107, 131, 144, 180, 234, 257, 308, 338, 356, 361, 366, 367, 373, 386, 525], "_segment_start": [57, 59, 355], "_segment_end": [57, 59, 355], "highest": [57, 59, 510, 520, 538], "_segment_count": [57, 59, 355], "uncompact": [57, 59], "_message_corrupt": [57, 58, 445], "_messag": [57, 58, 59], "_message_length": [57, 58, 59], "_header": [57, 369], "_key_corrupt": [57, 58, 73, 445], "_kei": [57, 58, 59, 73], "_key_length": [57, 58, 73], "subsect": [57, 64], "unqualifi": [57, 73, 122], "cumbersom": 57, "schemanam": [57, 58, 59, 73, 78, 142, 453], "topicnam": [57, 58], "dataformat": [57, 58, 59, 73], "formathint": [57, 59, 73], "dataschema": [57, 59, 73], "hint": [57, 59, 83, 339], "subjectid": 57, "cachingschemaregistri": 57, "strategi": [57, 66, 71, 76, 78, 84, 146, 148, 272, 355, 356, 436, 450], "marker": [57, 96, 367, 380], "empty_field_mark": 57, "empty_field_strategi": [57, 450], "topicnamestrategi": 57, "beforehand": 57, "proto3": 57, "test_oneof_column": 57, "string_column": 57, "integer_column": [57, 148], "long_column": 57, "double_column": 57, "float_column": 57, "boolean_column": 57, "five": [57, 60, 112, 302, 470, 510, 520], "deliveri": 57, "big": [57, 59, 73, 101, 132, 475], "endian": [57, 59, 73, 101], "eight": [57, 58, 59, 73], "ieee": [57, 59, 73, 101, 144], "754": [57, 59, 73, 101, 144, 357], "know": [57, 84, 97, 112, 132, 152, 287], "straightforward": [57, 59, 73, 84, 112], "example_table_nam": 57, "example_schema_nam": 57, "example_topic_nam": 57, "field1": [57, 264, 362, 375, 520], "field2": [57, 520], "field3": 57, "field4": 57, "gap": [57, 131, 263, 386], "overlap": [57, 78, 84, 112, 380, 510], "example_raw_t": 57, "123456789": [57, 107], "123456": [57, 106, 144], "abcdef": 57, "ahead": [57, 107], "undefin": [57, 59, 63, 73, 104], "pad": [57, 101, 103, 124, 197, 330, 401, 520], "example_csv_t": 57, "4627": [57, 59, 73, 379], "slash": [57, 59, 73, 334, 405, 419, 422, 427, 447], "formatt": [57, 58], "tempor": [57, 71, 271, 293, 369, 421, 430, 438], "iso8601": [57, 59, 73, 220], "rfc2822": [57, 58, 59, 73, 369], "joda": [57, 58, 59, 73], "epoch": [57, 58, 59, 73, 97, 369, 375, 384], "sss": 57, "example_json_t": 57, "2020": [57, 107, 144, 505], "schemaless": [57, 59, 73], "dirit": 57, "avro_message_schema": 57, "example_avro_messag": 57, "example_avro_t": 57, "dynamicmessag": 57, "proto": [57, 267, 274, 346, 469, 475], "sint32": 57, "fixed32": 57, "sfixed32": 57, "sint64": 57, "fixed64": 57, "sfixed64": 57, "predefin": [57, 90], "message_schema": 57, "example_protobuf_t": 57, "till": 57, "impli": [57, 59, 73], "parselong": [57, 59, 73], "parsedoubl": [57, 59, 73], "_default": [57, 59, 73], "transform": [57, 59, 73, 94, 100, 113, 114, 115, 117, 123, 124, 137, 151, 212, 230, 245, 354, 369, 381, 384, 387], "iso": [57, 58, 59, 73, 94, 107, 408, 433], "8601": [57, 58, 59, 73, 107, 408, 433], "2822": [57, 58, 59, 73], "www": [57, 59, 73], "apidoc": [57, 59, 73], "datetimeformat": [57, 59, 73, 107], "html": [57, 59, 64, 69, 73, 198, 470], "embed": [57, 59, 65, 73, 112, 281], "mymessag": 57, "stringcolumn": 57, "integercolumn": 57, "longcolumn": 57, "any_messag": 57, "493857959588286460": 57, "numbercolumn": 57, "familiar": [58, 151, 297], "focus": 58, "integr": [58, 78, 139, 140, 360, 365, 380, 471], "extract": [58, 65, 82, 90, 93, 94, 112, 114, 137, 142, 208, 234, 297, 359, 368, 372, 373, 396, 469], "2013": 58, "37": [58, 99, 101, 368, 506], "495": [58, 334], "quorumpeerconfig": 58, "028": 58, "verifiableproperti": 58, "051": 58, "1048576": [58, 99, 106], "repo1": 58, "maven2": 58, "softwareforg": 58, "kafka_tpch_0811": 58, "755": [58, 339], "preload": 58, "2014": [58, 293], "28t17": 58, "594": 58, "0700": 58, "airlift": [58, 78, 83, 84, 395], "stderr": [58, 136], "623": 58, "loadcommand": 58, "981": [58, 343], "982": [58, 343], "612": 58, "defaulteventhandl": 58, "collat": [58, 71, 76, 380, 392, 430, 435, 436], "781": [58, 396], "797": [58, 354], "932": 58, "068": 58, "319": [58, 153, 332], "333": [58, 153], "466": 58, "597": [58, 335], "706": 58, "180": [58, 109, 153, 506], "1500": [58, 346], "251": [58, 332], "905": 58, "15000": 58, "919": [58, 341], "8000": [58, 76], "877": [58, 341], "60175": 58, "unstructur": 58, "besid": [58, 60, 64, 113, 144, 512], "ones": [58, 80, 460], "rownumb": [58, 520], "customerkei": 58, "000000001": 58, "ivhziaperb": 58, "ot": 58, "phone": 58, "989": 58, "741": [58, 343], "2988": 58, "accountbal": 58, "711": [58, 337], "56": [58, 153, 209], "marketseg": 58, "platelet": 58, "iron": 58, "epitaph": 58, "nag": 58, "000000003": 58, "mg9kdtd2wbhm": 58, "719": 58, "748": [58, 338], "3364": 58, "7498": 58, "automobil": [58, 520], "deposit": 58, "eat": 58, "slyli": 58, "fox": [58, 123], "blithel": 58, "000000005": 58, "kvpyuhcplrb84wgaigv6sypzq7tj": 58, "750": 58, "942": 58, "6364": 58, "794": 58, "household": [58, 520], "unwind": 58, "cajol": 58, "accor": 58, "000000007": 58, "tcge5gazngvepxu5krrvxbfkasdtea": 58, "190": [58, 153, 506], "9759": 58, "9561": 58, "95": [58, 153, 339, 506], "ainst": 58, "theodolit": 58, "pinto": 58, "000000009": 58, "xkiaftjuscuxfelenqefumtrj": 58, "338": [58, 153], "906": [58, 343], "3675": 58, "8324": 58, "furnitur": [58, 520], "wake": 58, "thinli": 58, "excus": 58, "pend": [58, 209, 217, 221, 278, 302, 427, 432], "json_extract_scalar": [58, 112, 114, 115, 233, 302], "6681865": 58, "shape": [58, 109, 158, 166, 249, 302, 342, 510, 520], "decod": [58, 101, 103, 109, 124, 191, 215, 217, 220, 243, 254, 282, 289, 299, 324, 333, 365, 369, 374, 384, 392, 395, 413, 467], "slice": [58, 84, 87, 88, 91, 93, 94, 100, 114, 115, 163, 197, 253, 317, 395], "kafka_kei": 58, "row_numb": [58, 114, 115, 131, 155, 162, 164, 166, 257, 258, 266, 280, 297, 302, 307, 315, 361, 392, 440], "customer_kei": 58, "nation_kei": 58, "account_bal": 58, "market_seg": 58, "000000002": 58, "xstf4": 58, "ncwdvawne6tegvwfmrchlxak": 58, "768": 58, "687": [58, 337], "3665": 58, "121": [58, 153], "65": [58, 112, 119, 153], "blith": 58, "boldli": 58, "caref": 58, "000000004": 58, "xxvsjslagtn": 58, "5944": 58, "2866": 58, "83": [58, 153], "machineri": [58, 94, 520], "idea": [58, 77], "accou": 58, "000000006": 58, "skzz0csnmd7mp4xd0yrbvx": 58, "lreykuwah": 58, "yvn": 58, "114": [58, 153], "968": 58, "4951": 58, "7638": [58, 383], "57": [58, 153, 286, 291], "tion": 58, "bold": 58, "furiou": 58, "000000008": 58, "i0b10bb0aymmc": 58, "0prrybcp1ygj8xcbpmwhl5": 58, "147": [58, 70, 153, 204, 209, 392], "574": 58, "9335": 58, "6819": 58, "74": [58, 109, 112, 153], "kindl": 58, "court": 58, "haggl": 58, "alon": 58, "000000010": 58, "6lreav6kr6plvcgl2arl": 58, "q3rqzlzct1": 58, "346": [58, 153, 332], "9870": 58, "2753": 58, "fur": 58, "simul": 58, "twistr": 58, "twistr_kafka_0811": 58, "twitter_fe": 58, "screen_nam": 58, "created_at": 58, "favorite_count": 58, "retweet_count": 58, "favorit": [58, 140], "id_str": 58, "in_reply_to_screen_nam": 58, "place_nam": 58, "full_nam": 58, "dness": 58, "pwd": [58, 135, 474], "4467": 58, "4517": 58, "4572": 58, "494227746231685121": 58, "burncaniff": 58, "494227746214535169": 58, "gu8tn": 58, "ja": 58, "494227746219126785": 58, "pequitamedicen": 58, "494227746201931777": 58, "josni": 58, "ht": 58, "494227746219110401": 58, "cafe510": 58, "494227746210332673": 58, "da_juanand_onli": 58, "494227746193956865": 58, "smile_kidrauhl6": 58, "pt": 58, "494227750426017793": 58, "cashforevercd": 58, "494227750396653569": 58, "filmarsivimiz": 58, "tr": 58, "494227750388256769": 58, "jmola": 58, "raw_dat": 58, "tue": [58, 137], "jul": [58, 153], "33": [58, 99, 117, 256], "35": [58, 104, 107, 137, 328, 506], "miillisecond": 58, "hood": 59, "seri": [59, 151], "getrecord": 59, "xxxxxx": 59, "blank": 59, "caught": [59, 112], "trim": [59, 114, 115, 124, 142, 162, 210, 248, 401, 404], "horizon": 59, "aws_access_key_id": 59, "aws_secret_access_kei": 59, "accesskeyid": 59, "profil": 59, "deliv": [59, 84, 151], "east": [59, 64, 258, 520], "milli": 59, "behind": [59, 132, 344, 358, 432, 475], "observ": [59, 252], "though": [59, 112], "That": [59, 83, 84, 94, 109, 509, 520], "why": [59, 109, 151, 475, 520], "1000m": 59, "at_timestamp": 59, "trim_horizon": 59, "oldest": [59, 263, 302], "_shard_id": 59, "_shard_sequence_id": 59, "_message_valid": [59, 445], "_message_timestamp": 59, "_partition_kei": 59, "streamnam": 59, "drill": 59, "javascript": [59, 73], "notaion": [59, 73], "7051": 60, "db8": [60, 144], "createt": [60, 251], "deletet": 60, "clientprincipalnam": 60, "spn": 60, "first_nam": 60, "last_nam": 60, "partition_by_hash_column": 60, "partition_by_hash_bucket": 60, "nullabl": [60, 84, 100, 188, 217, 272, 275, 420, 432, 450], "donald": 60, "duck": 60, "mickei": 60, "mous": 60, "part1": 60, "part2": 60, "unixtime_micro": 60, "\u00b5s": 60, "upsert": 60, "cours": [60, 144], "user_ev": 60, "event_nam": 60, "plain": [60, 70, 84, 276, 465], "number_of_replica": [60, 327], "odd": 60, "kind": [60, 87, 112, 152, 393, 456], "bitshuffl": 60, "runlength": 60, "dictionari": [60, 184, 185, 187, 210, 215, 217, 219, 228, 239, 244, 255, 256, 257, 261, 269, 274, 275, 276, 327, 332, 350, 351, 358, 383, 389, 399, 403], "group_varint": 60, "zlib": 60, "extrainfo": 60, "add_range_partit": 60, "drop_range_partit": 60, "partition_by_second_hash_column": 60, "partition_by_second_hash_bucket": 60, "partition_by_range_column": 60, "themselv": [60, 151], "event_tim": 60, "01t00": [60, 78], "partition_design": 60, "serialno": 60, "2017": [60, 262], "range_partition_as_json_str": 60, "value_col1": 60, "01t12": 60, "localfil": 61, "glob": 61, "unsign": [62, 66, 75, 76, 365, 475], "tinytext": [62, 66, 75], "65535": [62, 66, 75], "mediumtext": [62, 66, 75], "16777215": [62, 66, 75], "longtext": [62, 66, 75], "tinyblob": [62, 66], "mediumblob": [62, 66], "longblob": [62, 66, 75], "explicit_defaults_for_timestamp": 62, "65536": 62, "timestampdiff": 62, "curdat": 62, "date_of_birth": 62, "stddev": [62, 66, 68, 71, 76, 99, 114, 115, 288, 370], "stddev_pop": [62, 66, 68, 71, 76, 99, 114, 115, 370, 379], "stddev_samp": [62, 66, 68, 71, 76, 99, 114, 115, 370, 379], "varianc": [62, 66, 68, 71, 76, 99, 114, 115, 288, 370], "var_pop": [62, 66, 68, 71, 76, 99, 114, 115, 370, 379], "var_samp": [62, 66, 68, 71, 76, 99, 114, 115, 370, 379], "lazi": [63, 123, 155, 192, 307, 380], "loss": [63, 273, 274, 381, 390, 391, 394, 396, 406, 408, 434, 450], "recreat": [63, 208, 217, 402], "inaccess": [63, 394, 395, 456, 471], "mediat": 64, "ecosystem": [64, 132], "depict": 64, "inconsist": [64, 136, 208, 313, 346, 382, 520], "fresh": [64, 77, 161, 384, 438], "spark": [64, 381, 398, 436, 447], "deleg": [64, 86, 148, 361, 386, 450], "txn": 64, "lock": [64, 76, 155, 176, 192, 249, 326, 389, 393, 396, 412, 427], "acquir": [64, 343], "govcloud": 64, "gov": 64, "vpc": 64, "gatewai": [64, 83], "catalogid": [64, 404], "archiv": [64, 132, 139, 429, 500], "8181": 64, "my_bucket": 64, "warehouse_loc": 64, "bearer": 64, "abcdef123456": 64, "break": [64, 151, 213, 226, 271, 293, 476, 510], "5432": [64, 71, 137], "19120": 64, "reimplement": 64, "hortonwork": 64, "supportkb": 64, "247055": 64, "errorjavalangunsupportedoperationexcept": 64, "serdestorageschemaread": 64, "27017": 65, "mongod": 65, "mongo": 65, "inconnect": 65, "srv": 65, "startup": [65, 84, 136, 137, 160, 163, 192, 197, 223, 272, 281, 333, 434, 447, 470], "guess": [65, 278], "_schema": [65, 359, 386], "mongocli": 65, "exhaust": [65, 401], "indefinit": [65, 84, 259, 265, 278, 395, 396], "120000": 65, "sole": 65, "primary_pref": 65, "secondary_pref": 65, "nearest": [65, 68, 109, 118], "journal": 65, "unacknowledg": 65, "member": [65, 83, 152, 441, 462, 463], "refus": 65, "batchsiz": 65, "getcollect": 65, "deleted_table_nam": 65, "55": [65, 101, 107, 144, 153, 328, 520], "b1": 65, "51": [65, 506], "63": [65, 101, 119, 144, 153, 217, 256, 506], "d6": 65, "8c": 65, "61": [65, 119, 153, 328], "a9": 65, "ce": 65, "67": [65, 153, 328], "cf": [65, 100, 117, 532], "55b151633864d6438c61a9c": 65, "render": [65, 103, 114, 115, 144, 156, 206, 211, 217, 228, 235, 255, 257, 258, 313, 327, 330, 338, 346, 361, 364, 365, 375, 380, 462], "coupl": 65, "objectid_timestamp": [65, 114, 359], "507f191e810c19729de860ea": 65, "2012": [65, 76, 107], "46": [65, 506], "timestamp_objectid": [65, 114, 388], "36": [65, 107, 113, 328, 506], "0e": 65, "c8": 65, "gt": 65, "610ec8280000000000000000": 65, "decimal128": [65, 257, 452, 454], "dbref": [65, 387, 447], "sslmode": 66, "cj": 66, "cjexcept": 66, "unknown": [66, 104, 112, 117, 191, 245, 251, 263, 278, 317, 360, 496, 505], "unspecifi": [66, 69, 71, 75, 100, 392], "decimal_map": [66, 69, 71, 75], "allow_overflow": [66, 69, 71, 75], "decimal_rounding_mod": [66, 69, 71, 75], "ceil": [66, 68, 69, 71, 75, 114, 115, 118, 217], "floor": [66, 68, 69, 71, 75, 99, 114, 115, 118, 217], "half_up": [66, 68, 69, 71, 75], "half_down": [66, 68, 69, 71, 75], "half_even": [66, 68, 69, 71, 75], "roundingmod": [66, 69, 71, 75], "group_concat": 66, "accuraci": [66, 99, 120, 144, 180, 183, 186, 278, 371, 433], "histogram": [66, 99, 114, 115, 118, 165, 177, 259, 260, 307, 373], "column_name1": 66, "column_name2": 66, "intellig": [66, 71, 76, 358], "join_pushdown_en": [66, 68, 71, 75, 76], "serializ": [67, 384, 540], "serd": [67, 282, 283, 293], "did": [67, 191, 331, 357, 368, 406], "ordin": [67, 154, 155, 191, 261, 264, 274, 333, 385, 520], "orc_use_column_nam": [67, 354], "parquet_use_column_nam": [67, 379], "parquet_optimized_writer_validation_percentag": [67, 423], "off": [67, 71, 78, 136, 180, 183, 184, 187, 227, 263, 267, 301, 307, 310, 470], "parquet_use_bloom_filt": [67, 433], "parquet_use_column_index": 67, "parquet_max_read_block_row_count": [67, 432], "8192": 67, "parquet_small_file_threshold": [67, 455], "3mb": [67, 106, 455], "1521": [68, 349], "sid": 68, "orcl": 68, "remark": [68, 373], "inact": [68, 368], "20m": 68, "binary_float": 68, "binary_doubl": 68, "varchar2": 68, "nvarchar2": 68, "clob": 68, "nclob": 68, "nine": 68, "1582": [68, 367, 393, 395], "4000": [68, 76, 335], "trail": [68, 76, 108, 121, 124, 144, 223, 242, 248, 251, 261, 271, 334, 357, 359, 388, 422, 446, 447, 510], "val": 68, "col": [68, 259, 337, 387], "bb": 68, "number_default_scal": 68, "number_rounding_mod": 68, "assert": [68, 156], "henc": [68, 507], "infin": [68, 114, 115, 118, 311, 317, 389, 486], "neighbor": 68, "equidist": 68, "sales_view": 68, "simple_model": 68, "bounc": 68, "2002": 68, "box": [68, 109, 257, 413, 472, 475], "covar_samp": [68, 71, 99, 114, 115, 158, 274, 379], "covar_pop": [68, 71, 99, 114, 115, 158, 379], "trino_test": 68, "host3": 69, "phoenix5": [69, 380], "zk_quorum": 69, "zk_port": 69, "zk_hbase_path": 69, "properi": 69, "max_scans_per_split": 69, "update_statist": 69, "guidepost": 69, "5000": 69, "unsigned_tinyint": 69, "unsigned_smallint": 69, "unsigned_int": 69, "unsigned_long": 69, "unsigned_float": 69, "unsigned_doubl": 69, "unsigned_d": 69, "salt_bucket": 69, "split_on": 69, "presplit": 69, "salt": [69, 472], "disable_w": 69, "wal": 69, "immutable_row": 69, "default_column_famili": 69, "suffix": [69, 107, 126, 217], "_seq": 69, "min_vers": 69, "lzo": [69, 278, 348, 359, 379], "data_block_encod": 69, "fast_diff": 69, "diff": 69, "row_index_v1": 69, "forev": [69, 163, 197, 280], "bloomfilt": 69, "rowcol": 69, "8098": [70, 435], "fdqn": 70, "expiri": [70, 381], "483": [70, 204, 209], "647": [70, 204, 209], "pre": [70, 131, 158, 178, 189, 279, 350, 413, 459], "forbid": [70, 372], "retriabl": 70, "8090": 70, "inbound": 70, "flight_statu": 70, "pinot_t": 70, "baz": [70, 100, 112, 508, 518], "25000": 70, "pql": 70, "col4": 70, "suitabl": [70, 76, 296, 520], "int_arrai": 70, "long_arrai": 70, "float_arrai": 70, "double_arrai": 70, "string_arrai": 70, "variat": 70, "aggregation_pushdown_en": [70, 388], "distint": 70, "count_distinct_pushdown_en": 70, "postgresq": 71, "pg_namespac": 71, "pg_catalog": [71, 347], "42": [71, 99, 100, 102, 104, 107, 109, 112, 126, 144, 263, 348, 520], "bytea": [71, 257], "monei": [71, 357], "jsonb": [71, 331], "hstore": [71, 344, 351], "dimens": [71, 109], "array_map": [71, 348], "as_arrai": 71, "as_json": [71, 348], "frame": [71, 78, 99, 131, 203, 209, 218, 233, 315, 347, 373, 380, 457, 512, 520], "array_agg": [71, 99, 109, 114, 115, 155, 186, 209, 210, 212, 263, 328, 375, 385, 510], "exclud": [71, 78, 80, 112, 126, 131, 219, 242, 276, 365, 383, 412, 475, 476, 489, 510, 520], "time_data": 71, "corr": [71, 99, 114, 115, 158, 247, 274, 379], "regr_intercept": [71, 99, 114, 115, 158, 274, 379], "regr_slop": [71, 99, 114, 115, 158, 274, 379], "inequ": [71, 73, 242, 257, 269, 313, 358, 364, 371, 373, 387, 388, 392, 403], "enable_string_pushdown_with_col": [71, 392], "9090": 72, "21d": 72, "1568229904": 72, "awhil": 72, "perhap": 72, "issuer": [72, 388, 467, 470], "meant": [72, 475], "label": [72, 94, 119, 296, 417, 445, 510, 512], "job": [72, 94, 152, 445], "__name__": 72, "481": 72, "446": 72, "zset": 73, "6379": 73, "schema1": 73, "mget": 73, "hgetal": 73, "_valu": 73, "_value_length": 73, "_value_corrupt": [73, 445], "redis_kei": 73, "poland": [73, 509], "5439": 74, "formerli": [75, 471], "memsql": [75, 358, 360, 375, 384, 435], "usessl": 75, "1433": 76, "sqlserver": 76, "databasenam": [76, 447], "layer": [76, 94, 132, 214, 303, 466, 473, 475], "trustservercertif": 76, "truststorepassword": 76, "nvarchar": [76, 450], "ntext": [76, 390], "1073741823": 76, "datetime2": [76, 379], "smalldatetim": 76, "datetimeoffset": [76, 390], "53": [76, 107, 328, 506], "percent": [76, 136], "storedprocedur": 76, "implic": 76, "employee_sp": 76, "example_statistics_nam": 76, "column_nam": [76, 480, 489, 538], "bulk_copy_for_writ": 76, "bulk_copy_for_write_lock_destination_t": 76, "data_compress": [76, 377], "kill_queri": [77, 345], "20151207_215727_00146_tx3nr": 77, "storage_t": 77, "creator": [77, 384, 491], "average_tim": 77, "backend": [78, 396], "talk": [78, 209, 301], "idempot": [78, 240], "freeli": 78, "trino_thrift": [78, 435], "unfram": 78, "fb_compact": 78, "7777": 78, "7779": 78, "trinothriftbound": 78, "trinothriftserviceexcept": 78, "trinothriftnullableschemanam": 78, "trinothriftschematablenam": 78, "trinothrifttablemetadata": 78, "schematablenam": [78, 246, 450], "trinothriftcolumnmetadata": 78, "php": 78, "indexablekei": 78, "trinothriftnullablecolumnset": 78, "trinothrifttupledomain": 78, "trinothriftdomain": 78, "trinothriftallornonevalueset": 78, "infinit": [78, 80, 118, 250, 258, 299, 304, 324, 356, 398, 402, 510], "trinothriftequatablevalueset": 78, "trinothriftblock": 78, "trinothriftinteg": 78, "i32": 78, "trinothriftbigint": 78, "i64": 78, "trinothriftdoubl": 78, "trinothriftvarchar": 78, "trinothriftboolean": 78, "trinothriftd": 78, "trinothrifttimestamp": 78, "trinothriftjson": 78, "hyperloglog": [78, 84, 98, 99, 123, 133, 140, 183, 385, 386], "href": 78, "hll": [78, 110], "trinothrifthyperloglog": 78, "trinothriftbigintarrai": 78, "trinothriftrangevalueset": 78, "trinothriftrang": 78, "trinothriftid": 78, "trinothriftsplitbatch": 78, "trinothriftsplit": 78, "nexttoken": 78, "splitid": 78, "trinothrifthostaddress": 78, "trinothriftpageresult": 78, "columnblock": 78, "rowcount": [78, 96, 267], "trinothriftnullabletablemetadata": 78, "tablemetadata": 78, "trinothriftvalueset": 78, "allornonevalueset": 78, "equatablevalueset": 78, "rangevalueset": 78, "integerdata": 78, "bigintdata": 78, "doubledata": 78, "varchardata": 78, "booleandata": 78, "datedata": 78, "timestampdata": 78, "jsondata": 78, "hyperloglogdata": 78, "bigintarraydata": 78, "trinothriftmark": 78, "trinothriftnullabletoken": 78, "valueset": 78, "nullallow": 78, "trinolistschemanam": 78, "ex1": 78, "param": [78, 112, 119], "schemanameornul": 78, "trinolistt": 78, "trinogettablemetadata": 78, "desiredcolumn": 78, "outputconstraint": 78, "maxsplitcount": 78, "trinogetsplit": 78, "indexcolumnnam": 78, "outputcolumnnam": 78, "trinogetindexsplit": 78, "maxbyt": 78, "trinogetrow": 78, "tpc": [79, 80, 151, 247, 287, 335], "benchmark": [79, 80, 151, 247, 282], "sf10": 79, "sf100": [79, 80], "sf1000": [79, 80], "sf10000": [79, 80], "sf100000": [79, 80], "sf300": [79, 80], "sf3000": [79, 80], "sf30000": [79, 80], "300": [79, 80, 112, 113, 153], "l_": 80, "intend": [81, 94, 96, 170, 285, 426, 462, 467], "x509": [82, 465], "certificateauthenticatorfactori": 82, "certificateauthent": 82, "wrap": [82, 85, 89, 90, 92, 95, 112, 462], "getnam": [82, 85, 86, 89, 90, 92, 95], "property1": [82, 85, 89, 90, 92, 95], "value1": [82, 85, 88, 89, 90, 92, 95, 104, 105], "property2": [82, 85, 89, 90, 92, 95], "value2": [82, 85, 88, 89, 90, 92, 95, 104, 105], "nexturi": 83, "502": 83, "504": 83, "intermitt": [83, 235, 242, 362], "front": [83, 258, 475], "queryerror": 83, "succeed": [83, 424, 425], "echo": [83, 137, 475], "unsuccessfulli": 83, "loop": [83, 84, 247, 250, 258, 299, 304, 324, 398, 402, 510], "human": [83, 107], "consumpt": [83, 259, 364, 392, 422], "tell": [83, 137, 475], "modul": [83, 84, 87, 88, 297], "updatetyp": 83, "errorcod": [83, 155], "failureinfo": 83, "errorloc": 83, "trinowarn": [83, 378], "statementstat": [83, 210], "Of": 83, "rootstag": 83, "stagestat": [83, 275], "english": [83, 107, 124], "accumul": [83, 88, 94, 396], "assig": 83, "choic": [83, 94, 137, 520], "execution_tim": 83, "cpu_tim": 83, "peak_memori": 83, "peak_task_memori": 83, "timeunit": [83, 84], "datas": 83, "meaning": [83, 97], "reset": [83, 133, 143, 180, 183, 218, 254, 317, 399, 432, 453, 457, 477, 505, 523, 524, 537], "dealloc": [83, 133, 142, 143, 275, 477, 503, 504, 505, 513], "getconnectorfactori": [84, 86, 93, 213], "mistak": [84, 152], "robust": [84, 469], "configdescript": 84, "configsecuritysensit": 84, "maxdur": 84, "mindur": 84, "javax": [84, 254, 296, 469, 475], "notnul": 84, "exampleconfig": 84, "succinctdur": 84, "getsecret": 84, "setsecret": 84, "gettimeout": 84, "settimeout": 84, "spell": 84, "stuck": [84, 395, 418], "guic": [84, 86], "binder": 84, "configbind": 84, "examplemodul": [84, 86], "void": [84, 88, 94, 287], "bindconfig": 84, "connectornam": 84, "connectorcontext": [84, 217], "requirenonnul": 84, "bootstrap": [84, 86, 217], "injector": [84, 86], "donotinitializelog": [84, 86], "setrequiredconfigurationproperti": [84, 86], "getinst": [84, 86], "exampleconnector": 84, "ex": 84, "fine": [84, 266, 396, 471], "grain": [84, 266, 462, 471], "git": 84, "listschemanam": 84, "listtabl": [84, 330], "streamtablecolumn": [84, 385], "gettablehandl": [84, 94, 450], "gettablemetadata": 84, "getcolumnhandl": 84, "getcolumnmetadata": 84, "creativ": 84, "ir": 84, "applylimit": [84, 333, 336, 367], "applytopn": 84, "applyfilt": [84, 334, 336, 399, 405], "applyproject": [84, 343], "applysampl": [84, 339], "applyaggreg": [84, 362, 380], "applyjoin": [84, 379, 380, 423], "applytablefunct": [84, 96], "applytablescanredirect": 84, "connectortablehandl": [84, 94, 96, 293], "limitguarante": 84, "limitapplicationresult": 84, "topnguarante": 84, "topnapplicationresult": 84, "topn": [84, 148, 367, 380, 381], "invalid_order_bi": 84, "impos": [84, 94, 112], "tupledomain": [84, 182, 384, 387, 392, 399], "held": 84, "strict": [84, 94, 112], "constraintapplicationresult": 84, "connectorsess": [84, 87, 94, 96, 167, 249, 293, 305, 368, 398, 450, 453], "tablehandl": [84, 94, 293], "exampletablehandl": 84, "columnhandl": [84, 94, 96, 155, 293], "olddomain": 84, "getconstraint": 84, "newdomain": 84, "getsummari": 84, "noth": [84, 187, 278], "notifi": 84, "remainingfilt": 84, "mongometadata": 84, "bigquerymetadata": 84, "kafkametadata": 84, "jdbctablehandl": 84, "isnon": 84, "newconstraintexpress": 84, "immutablelist": [84, 86], "remainingexpress": 84, "els": [84, 88, 94, 100, 105, 142, 187], "getdomain": 84, "orelsethrow": 84, "jdbccolumnhandl": 84, "keyset": 84, "toimmutablelist": 84, "jdbcclient": [84, 345], "tocolumnmap": 84, "getjdbctypehandl": 84, "hashmap": 84, "domainpushdownresult": 84, "pushdownresult": 84, "getpredicatepushdowncontrol": 84, "getpusheddown": 84, "getremainingfilt": 84, "withcolumndomain": 84, "getrelationhandl": 84, "illustr": [84, 124, 144], "compliant": [84, 109, 112, 140, 361, 391], "predicatepushdowncontrol": 84, "connectorexpress": [84, 423], "iscomplexexpressionpushdown": 84, "newexpress": 84, "arraylist": 84, "conjunct": [84, 105, 109, 112, 205, 278, 368, 470, 475, 520], "extractconjunct": 84, "getexpress": [84, 399], "understood": 84, "convertpred": 84, "getassign": 84, "ispres": 84, "immutableset": [84, 88], "builder": [84, 88, 96, 347, 351, 358], "addal": 84, "getconstraintexpress": 84, "aslist": [84, 475], "unconvert": 84, "emploi": [84, 113], "Such": [84, 112, 265, 375, 506], "recordset": [84, 233], "recordcursor": [84, 251], "getrecordset": [84, 416], "inmemoryrecordset": 84, "advancenextposit": 84, "cursor": [84, 86, 193, 390], "int128": [84, 395], "longtimewithtimezon": [84, 377], "longtimestamp": 84, "longtimestampwithtimezon": 84, "p4hyperloglog": [84, 110, 183], "setdigest": [84, 123, 140], "qdigest": [84, 99, 120, 279, 427], "tdigest": [84, 99, 127, 385], "gettyp": [84, 97, 213], "getboolean": 84, "getlong": [84, 88, 373], "getdoubl": [84, 88, 373], "getslic": 84, "getobject": [84, 375], "datetimeencod": 84, "packdatetimewithzon": 84, "utf8slic": [84, 88], "valueof": 84, "encodearrai": 84, "blockbuild": [84, 88, 264, 334, 397, 448, 454], "createblockbuild": 84, "appendnul": [84, 88], "writestr": 84, "encodemap": 84, "maptyp": 84, "typemanag": [84, 213], "typesignatur": 84, "gettypesignatur": 84, "beginblockentri": 84, "entryset": 84, "getkei": 84, "getvalu": [84, 96], "tostr": [84, 361], "closeentri": 84, "connectorpagesourc": [84, 233, 251, 304, 306, 387, 392], "recordpagesourceprovid": 84, "recordpagesourc": [84, 233], "connectorpagesink": [84, 91, 94, 220, 317], "iter": [84, 86, 94, 146, 157, 233, 362, 392, 472], "completablefutur": [84, 94], "appendpag": 84, "channel": [84, 263], "getchannelcount": 84, "getblock": [84, 263], "getpositioncount": 84, "isnul": [84, 88], "tostringutf8": [84, 88], "todo": 84, "not_block": 84, "eventlistenerfactori": [85, 93, 332], "eventlisten": [85, 227, 341, 359, 361, 386, 426, 428], "devot": 86, "exampleconnectorfactori": 86, "guava": [86, 88], "example_http": 86, "meat": 86, "jsonmodul": 86, "catalognam": [86, 275, 523], "requiredconfig": 86, "connectormetadata": [86, 87, 91, 217, 232, 251, 272, 283, 293, 302, 330, 333, 334, 336, 339, 341, 342, 343, 362, 367, 379, 380, 385, 386, 396, 399, 405, 408, 413, 422, 423, 425, 433, 440, 450, 451, 453], "examplecli": [86, 87], "examplerecordcursor": 86, "jdbcplugin": 87, "connectorrecordsetprovid": [87, 416], "predicact": 87, "exampleclientmodul": 87, "basejdbccli": 87, "jdbctypehandl": 87, "writemap": 87, "getfunct": [88, 213], "examplefunctionsplugin": 88, "examplenullfunct": 88, "isnullfunct": 88, "isequalornullfunct": 88, "examplestringfunct": 88, "exampleaveragefunct": 88, "codebas": 88, "ml": [88, 119], "teradata": [88, 98, 119, 133, 162, 180], "is_nul": [88, 402], "scalarfunct": [88, 213, 270], "sqltype": 88, "standardtyp": 88, "sqlnullabl": [88, 217], "shuffl": [88, 100, 114, 115, 217, 239, 417], "unexpect": [88, 170, 194, 293], "primit": [88, 442], "biginttyp": [88, 97], "our": [88, 282, 356], "typeparamet": 88, "isnullslic": 88, "isnulllong": 88, "isnulldoubl": 88, "operatordepend": 88, "is_equal_or_nul": 88, "isequalornullslic": 88, "returntyp": 88, "argumenttyp": 88, "methodhandl": [88, 284], "invokeexact": 88, "tolowercas": 88, "accumulatorst": 88, "getter": 88, "setter": 88, "accumulatorstatefactori": 88, "accumulatorstateseri": 88, "accumulatorstatemetadata": 88, "avg_doubl": 88, "aggregationfunct": [88, 213], "averageaggreg": 88, "inputfunct": [88, 99, 100], "longanddoublest": 88, "setlong": 88, "setdoubl": 88, "combinefunct": [88, 99], "otherst": 88, "outputfunct": [88, 100], "writedoubl": 88, "seen": [88, 281, 373, 469], "depth": [88, 112, 520], "groupedaccumulatorst": 88, "exampledeprecatedfunct": 88, "bad_funct": 88, "good_funct": 88, "groupprovid": 89, "groupproviderfactori": 89, "getgroup": 89, "connectorident": [89, 358], "headerauthenticatorfactori": 90, "headerauthent": 90, "createauthenticatedprincip": [90, 92], "gethead": 90, "begininsert": 91, "finishinsert": 91, "connectorpagesinkprovid": [91, 430], "connectorinserttablehandl": 91, "pagesinkprovid": 91, "connectoroutputtablehandl": 91, "begincreatet": 91, "passwordauthenticatorfactori": 92, "passwordauthent": 92, "serviceload": 93, "inf": 93, "exampleplugin": 93, "pom": 93, "connectorfactori": [93, 275, 397], "parametrictyp": [93, 97], "systemaccesscontrol": [93, 95, 232, 270, 275, 346, 357, 358, 361, 375, 423], "scope": [93, 217, 237, 379, 383, 470, 474], "assembl": 93, "jackson": 93, "travers": [93, 112], "rowid": [94, 409], "paradigm": 94, "dml": [94, 398], "iec": 94, "9075": 94, "target_t": [94, 511], "source_t": [94, 511], "purchas": [94, 511, 542], "monthly_accounts_upd": [94, 511], "berkelei": 94, "THEN": [94, 99, 100, 105, 142, 511], "joe": [94, 520], "shmoe": 94, "unmatch": [94, 358], "classifi": [94, 114, 115, 119, 152, 287, 296, 512], "getrowchangeparadigm": 94, "change_only_updated_column": 94, "processor": [94, 96, 337, 339, 354, 397, 408], "changeonlyupdatedcolumnsmergeprocessor": 94, "delete_row_and_insert_row": 94, "deleteandinsertmergeprocessor": 94, "searchedcaseexpress": 94, "case_numb": 94, "ultim": [94, 146], "storemergedrow": 94, "getmergerowidcolumnhandl": 94, "indetermin": [94, 242, 339], "ascend": [94, 99, 118], "getinsertlayout": 94, "getupdatelayout": 94, "beginmerg": [94, 432], "createmergesink": 94, "finishmerg": 94, "transformpag": 94, "rowblock": [94, 249, 263, 264, 276], "is_distinct": 94, "plu": [94, 112, 131, 132, 144, 155, 464, 489, 510, 538], "assignuniqueid": 94, "merge_target_row_multiple_match": [94, 511], "rais": [94, 100, 108, 118, 186, 232, 267, 274, 333, 339, 386, 423, 486, 491, 511], "connectortransactionhandl": [94, 96, 275], "transactionhandl": 94, "mergehandl": 94, "not_support": 94, "connectortablelayout": [94, 194, 276, 397, 430], "systemaccesscontrolfactori": 95, "connectoraccesscontrol": [95, 172, 217, 232, 270, 275, 358, 375, 383, 427], "connectortablefunct": [96, 409], "subclass": [96, 213], "abstractconnectortablefunct": [96, 409], "gettablefunct": 96, "myfunct": 96, "super": [96, 108, 474], "my_funct": [96, 126], "scalarargumentspecif": 96, "column_count": [96, 126], "defaultvalu": 96, "row_count": [96, 126, 392, 538], "generic_t": 96, "suppos": [96, 512], "oppos": [96, 112, 196, 487, 499, 508, 518, 522], "describedt": 96, "only_pass_through": 96, "descriptorargumentspecif": 96, "tableargumentspecif": 96, "rowsemant": 96, "prunewhenempti": 96, "passthroughcolumn": 96, "tablefunctionanalysi": 96, "columncount": 96, "scalarargu": 96, "invalid_function_argu": 96, "col_a": 96, "col_b": 96, "col_c": 96, "sublist": [96, 520], "tolist": 96, "returnedtyp": 96, "myhandl": 96, "compris": [96, 151], "connectortablefunctionhandl": 96, "opaqu": [96, 342, 388, 470], "tablescannod": 96, "easi": [96, 151, 287], "tablefunctiondataprocessor": 96, "tablefunctionsplitprocessor": 96, "connectorsplitsourc": [96, 212, 293, 294], "subtask": 96, "ship": [97, 105, 178, 493, 510, 520, 542], "varchartyp": 97, "getparametrictyp": 97, "javadoc": 97, "getjavatyp": 97, "64bit": 97, "complement": [97, 99, 101, 102, 116, 144, 395], "timestampwithtimezonetyp": [97, 426], "8byte": 97, "parametr": [97, 381], "bitwis": [98, 133], "color": [98, 114, 133, 217, 270, 293, 305], "geospati": [98, 133, 251, 255, 257, 262, 264, 265, 267, 271, 274, 329, 352, 375, 379, 453], "lambda": [98, 114, 121, 133, 225, 226, 229, 232, 233, 236, 237, 239, 242, 244, 250, 254, 258, 261, 264, 279, 360, 369, 373, 441, 453], "math": [98, 133, 210, 212], "quantil": [98, 99, 127, 133, 140, 279], "digest": [98, 99, 133, 140, 279, 360, 363, 371], "count_if": [99, 114, 115], "max_bi": [99, 114, 115, 157, 173, 176, 183, 229, 232, 236, 239, 245, 259, 261, 274, 302, 313, 325, 360, 373, 394, 396, 510], "min_bi": [99, 114, 115, 157, 173, 176, 183, 229, 232, 236, 239, 245, 259, 261, 274, 313, 360, 373, 394, 396, 510], "aggregate_funct": 99, "imagin": 99, "iri": [99, 119], "flower": 99, "speci": [99, 119], "setosa": 99, "virginica": [99, 119], "versicolor": 99, "lose": [99, 120, 144, 520], "petal_length_cm": 99, "any_valu": [99, 114, 115, 447], "inlud": 99, "price": [99, 112, 113, 131, 283, 486, 489, 490, 491, 510], "bool_and": [99, 114, 115, 317], "bool_or": [99, 114, 115, 317], "checksum": [99, 114, 115, 124, 170, 172, 220, 327, 375], "geometric_mean": [99, 114, 115, 173], "geometr": [99, 109], "listagg": [99, 114, 142, 389], "synopsi": 99, "overflow": [99, 206, 217, 262, 287, 293, 327, 396], "overflow_behaviour": 99, "sort_item": 99, "csv_valu": 99, "behaviour": [99, 387], "filler": 99, "smallest": [99, 104, 146, 149, 170], "bitwise_and_agg": [99, 102, 114, 115, 221], "bitwise_or_agg": [99, 102, 114, 115, 221], "OR": [99, 102, 104, 114, 116, 142, 166, 227, 267, 274, 287, 358, 359, 389, 401, 402, 429, 486, 491, 510, 520], "map_agg": [99, 114, 115, 117, 155, 157, 186, 219, 234, 237, 317], "map_union": [99, 114, 115, 209, 219, 237], "petal_data": 99, "petal_data_union": 99, "multimap_agg": [99, 114, 115, 117, 170, 219, 234, 403], "multimap": [99, 117], "deviat": [99, 118], "0040625": [99, 255], "26000": [99, 255], "approx_most_frequ": [99, 114, 115, 366], "sacrif": [99, 387], "permut": [99, 100, 114, 115], "propos": 99, "paper": [99, 151], "metwallei": 99, "agrawl": 99, "abbadi": 99, "approx_percentil": [99, 114, 115, 144, 178, 180, 191, 231, 247, 344, 371], "percentil": [99, 120, 127, 144, 191], "w": [99, 107, 113, 120, 121, 127, 217, 512, 520], "weigh": 99, "thought": [99, 151], "approx_set": [99, 110, 114, 115], "numeric_histogram": [99, 114, 115, 307], "variant": [99, 183, 186, 220, 261, 262, 263, 274, 330, 352, 357, 359, 361, 362, 367, 520], "yael": 99, "ben": 99, "haim": 99, "elad": 99, "tom": 99, "tov": 99, "research": [99, 151], "2010": 99, "pp": 99, "849": 99, "872": [99, 340], "qdigest_agg": [99, 114, 115, 120], "tdigest_agg": [99, 114, 115, 127], "coeffici": [99, 123, 144, 278], "covari": 99, "kurtosi": [99, 114, 115, 236], "unbias": 99, "x_i": 99, "regress": [99, 114, 115, 175, 183, 184, 188, 197, 198, 208, 212, 213, 217, 218, 221, 222, 223, 227, 234, 235, 236, 240, 241, 243, 246, 255, 256, 257, 259, 265, 274, 276, 280, 284, 287, 289, 291, 293, 295, 304, 309, 310, 313, 328, 335, 341, 342, 348, 349, 352, 353, 387, 389, 396, 397, 398, 401, 410, 411, 423, 435, 454], "intercept": 99, "slope": [99, 510], "fisher": 99, "moment": 99, "reduce_agg": [99, 113, 114, 115, 327, 328, 362], "inputvalu": 99, "initialst": [99, 100], "all_match": [100, 114, 115, 344], "any_match": [100, 113, 114, 115, 344], "array_distinct": [100, 114, 115, 154, 158, 194, 217, 327, 339, 402], "array_intersect": [100, 114, 115, 155, 208, 239, 327, 339], "array_union": [100, 114, 115, 213, 403], "array_except": [100, 114, 115, 233, 403], "array_histogram": [100, 446], "array_join": [100, 114, 115, 123, 156, 229, 261], "null_replac": 100, "array_max": [100, 114, 115, 163, 255, 373], "array_min": [100, 114, 115, 163, 255, 373], "array_posit": [100, 114, 115, 155], "array_remov": [100, 114, 115, 157], "array_sort": [100, 114, 115, 209, 217, 239, 264, 279, 307, 402], "bc": 100, "ab": [100, 114, 115, 118, 121, 186, 217], "abcd": [100, 124], "arrays_overlap": [100, 114, 115, 233, 239], "array1": 100, "array2": 100, "arrayn": 100, "contains_sequ": [100, 114, 374], "seq": 100, "consecut": [100, 213, 267, 510], "element_at": [100, 114, 115, 117, 167, 209, 223, 249], "flatten": [100, 114, 115, 208, 520], "ngram": [100, 114, 115, 123, 279], "gram": [100, 279], "adjac": [100, 109], "none_match": [100, 114, 115, 344], "trim_arrai": [100, 114, 115, 399], "x0": 100, "abc0": 100, "z0": 100, "wise": 100, "th": [100, 124], "uneven": [100, 357], "1b": [100, 106], "3b": 100, "zip_with": [100, 114, 115, 232, 269], "binary1": 101, "binaryn": 101, "binary2": 101, "lpad": [101, 114, 115, 124, 197, 269], "padbinari": 101, "rpad": [101, 114, 115, 124, 197, 269], "4648": 101, "from_base64": [101, 114, 115], "to_base64": [101, 114, 115], "from_base64url": [101, 114, 115], "alphabet": 101, "to_base64url": [101, 114, 115], "from_base32": [101, 114, 115, 405], "base32": 101, "to_base32": [101, 114, 115, 405], "from_hex": [101, 114, 115, 158], "to_hex": [101, 114, 115], "from_big_endian_32": [101, 114, 115, 269], "to_big_endian_32": [101, 114, 115, 269], "from_big_endian_64": [101, 114, 115, 228], "to_big_endian_64": [101, 114, 115, 227], "from_ieee754_32": [101, 114, 115, 265], "to_ieee754_32": [101, 114, 115, 243], "from_ieee754_64": [101, 114, 115, 265], "to_ieee754_64": [101, 114, 115, 243], "crc32": [101, 114, 115, 244], "crc": 101, "xxhash64": [101, 114, 115, 227, 228], "qualiti": 101, "md5": [101, 114, 115, 163], "sha1": [101, 114, 115, 169], "sha256": [101, 114, 115, 169], "sha512": [101, 114, 115, 169], "spooky_hash_v2_32": [101, 114, 115, 273], "spookyhashv2": 101, "spooky_hash_v2_64": [101, 114, 115, 273], "murmur3": [101, 114, 115, 371], "murmurhash3": 101, "aaaaaa": 101, "ba": 101, "58": [101, 153, 329, 520], "69": [101, 107, 119, 153], "2f": [101, 106], "2c": [101, 112], "a0": 101, "e3": 101, "96": [101, 107, 137, 153, 506], "ef": 101, "hmac_md5": [101, 114, 115, 264], "hmac_sha1": [101, 114, 115, 264], "hmac_sha256": [101, 114, 115, 264], "hmac_sha512": [101, 114, 115, 264], "bit_count": [102, 114, 115, 191, 256], "bitwise_and": [102, 114, 115, 191], "10011": 102, "11001": [102, 399], "bitwise_not": [102, 114, 115, 191], "bitwise_or": [102, 114, 115, 191], "11011": [102, 401], "bitwise_xor": [102, 114, 115, 191], "xor": 102, "01010": 102, "bitwise_left_shift": [102, 114, 115, 367], "shift": [102, 356], "001": [102, 107], "00100": 102, "0101": 102, "010100": 102, "bitwise_right_shift": [102, 114, 115, 367], "1001": [102, 344], "bitwise_right_shift_arithmet": [102, 114, 115, 367], "chart": [103, 137, 138, 139], "low_color": 103, "red": [103, 305], "high_color": 103, "green": 103, "drawn": 103, "horizont": 103, "0xff0000": 103, "0x00ff00": 103, "yellow": 103, "0xffff00": 103, "remaind": [103, 118, 131], "rgb": [103, 114, 115], "css": 103, "blue": 103, "magenta": 103, "cyan": 103, "white": [103, 366], "interpol": [103, 109, 355, 368, 467], "popular": [104, 132], "paul": 104, "john": [104, 520, 524, 542], "ringo": 104, "signifi": 104, "truth": [104, 116], "demonstr": [104, 116, 144, 148, 520], "valuen": 104, "world": [104, 106, 109], "biggest": 104, "interchang": 104, "contin": 104, "america": [104, 107, 144, 209, 263, 293, 469, 520, 525], "asia": [104, 112, 160, 345, 419, 469, 520], "africa": [104, 112, 520], "australia": 104, "antarctica": 104, "negat": [104, 108, 112, 173], "a__a": 104, "south_america": 104, "south": 104, "_america": 104, "starts_with": [104, 114, 115, 124, 360, 367], "aaa": 105, "bbb": 105, "ccc": 105, "true_valu": 105, "false_valu": 105, "150000": [105, 148], "origin_st": [105, 520], "origin_zip": [105, 520], "total_cost": 105, "california": [105, 520], "94131": [105, 520], "p332a": 105, "72": [105, 112, 119, 153, 328], "94025": 105, "155": [105, 153, 258, 329], "jersei": [105, 520], "08544": 105, "225": [105, 148, 505, 520], "490": 105, "per_packag": 105, "implicitli": [106, 110, 232, 297, 510], "vice": 106, "versa": 106, "try_cast": [106, 113, 114, 115, 217, 301, 321, 323], "arg": [106, 124], "5f": 106, "pi": [106, 114, 115, 118], "14159": 106, "03d": 106, "008": 106, "1234567": 106, "89": [106, 153, 330], "234": [106, 112], "567": 106, "7s": 106, "ta": 106, "te": [106, 126], "ty": 106, "2006": 106, "tuesdai": 106, "juli": 106, "format_numb": [106, 114, 384], "123k": 106, "parse_data_s": [106, 114], "eb": 106, "exabyt": 106, "zb": 106, "zettabyt": 106, "yb": 106, "yottabyt": 106, "1kb": 106, "2411724": 106, "typeof": [106, 114, 115, 144, 217], "expr": 106, "cat": [106, 465, 520], "AT": [107, 114, 115, 198, 209, 220, 230, 293, 368], "los_angel": [107, 209, 293, 525], "current_tim": [107, 114, 115, 142, 234, 361], "subsecond": 107, "759993": 107, "current_timezon": [107, 114, 115, 309, 525], "iana": [107, 144, 468], "last_day_of_month": [107, 114, 115, 345, 356, 358], "from_iso8601_timestamp": [107, 114, 115, 162], "vancouv": 107, "11t11": 107, "055": 107, "from_iso8601_timestamp_nano": [107, 114, 369], "000000000": 107, "from_iso8601_d": [107, 114, 115, 162], "calendar": [107, 144, 357, 368], "w10": 107, "at_timezon": [107, 114, 347], "new_york": [107, 144], "with_timezon": [107, 114, 115, 347], "from_unixtim": [107, 114, 115, 220, 382], "unixtim": 107, "from_unixtime_nano": [107, 114, 115, 371, 382], "000000100": 107, "000001234": 107, "499": 107, "1969": [107, 355], "999998766": 107, "localtim": [107, 114, 115, 142, 361], "localtimestamp": [107, 114, 115, 142, 144], "383628": 107, "to_iso8601": [107, 114, 115, 162, 414], "to_millisecond": [107, 114, 115, 247], "to_unixtim": [107, 114, 115, 382], "parenthesi": [107, 121, 122, 358], "date_trunc": [107, 114, 115, 293, 423, 424], "quarter": [107, 114, 491], "date_add": [107, 114, 115, 155, 293], "subtract": [107, 118, 247, 366, 368, 432], "timestamp1": 107, "timestamp2": 107, "86400123": 107, "parse_dur": [107, 114, 115, 241], "8m": 107, "043": 107, "81": [107, 153, 331], "human_readable_second": [107, 114, 115, 368], "3762": 107, "56363463": 107, "93": [107, 153], "date_pars": [107, 114, 115, 154, 168, 210, 211, 217, 293], "str_to_dat": 107, "abbrevi": [107, 132], "weekdai": 107, "sat": 107, "jan": [107, 153], "dec": [107, 153], "0th": 107, "1st": 107, "2nd": 107, "3rd": 107, "000000": 107, "999000": 107, "999999999": 107, "366": [107, 153, 332], "decemb": 107, "am": [107, 293], "pm": 107, "sundai": 107, "mondai": 107, "saturdai": 107, "day_of_week": [107, 114], "2069": [107, 210], "date_format": [107, 114, 115, 154, 168, 217, 293, 343], "jodatim": 107, "format_datetim": [107, 114, 115], "parse_datetim": [107, 114], "day_of_month": [107, 114], "dow": [107, 114], "day_of_year": [107, 114], "doi": [107, 114], "year_of_week": [107, 114, 317], "yow": [107, 114], "timezone_hour": [107, 114, 301, 368, 373], "timezone_minut": [107, 114, 368, 373], "week_of_year": [107, 114], "xxxxxxx": 108, "yyyyyyi": 108, "12345": 108, "0000012345": 108, "1234500000": 108, "mathemat": [108, 109, 114, 115], "xp": 108, "xs": 108, "yp": 108, "ys": 108, "bs": 108, "coerc": [108, 232, 297, 448, 450, 520], "operand": [108, 112, 351], "st_": 109, "consortium": 109, "ogc": 109, "opengi": 109, "geometri": [109, 256, 257, 264, 267, 269, 270, 276, 277, 278, 357], "sens": [109, 151, 152], "area": [109, 512], "polygon": [109, 264, 270], "boundari": [109, 121, 336, 366, 368], "wkb": 109, "spatial": [109, 262, 264, 268, 278, 280, 336, 361], "linestr": [109, 277, 332], "multipoint": 109, "multilinestr": 109, "multipolygon": 109, "geometrycollect": 109, "st_geometryfromtext": [109, 114, 115], "st_geomfrombinari": [109, 114, 115, 276, 277], "sphericalgeographi": [109, 329, 336, 346, 375], "geograph": 109, "geodet": 109, "lat": 109, "lon": 109, "spheric": 109, "angular": 109, "plane": [109, 137], "shortest": 109, "straight": 109, "cartesian": [109, 520], "vector": [109, 118, 192, 276, 307, 441, 454], "sphere": 109, "great": 109, "circl": 109, "arc": [109, 118], "complic": 109, "spheroid": 109, "st_distanc": [109, 114, 115, 263, 278], "st_length": [109, 114, 115, 346], "meter": 109, "st_area": [109, 114, 115, 269, 336], "squar": [109, 113, 118, 302], "to_spherical_geographi": [109, 114, 115], "st_point": [109, 114, 115, 332, 357], "71": [109, 153, 506], "0882": 109, "3607": 109, "1197": 109, "6976": [109, 380], "4577": 109, "euclidean": 109, "312822": 109, "179": [109, 153, 506], "st_asbinari": [109, 114, 115, 276], "st_astext": [109, 114, 115], "st_linefromtext": [109, 114, 115], "st_polygon": [109, 114, 115], "geometry_from_hadoop_shap": [109, 114, 115, 347], "st_linestr": [109, 114, 115, 277, 280], "vertex": 109, "st_multipoint": [109, 114, 115, 278], "earth": 109, "radiu": [109, 274], "2d": 109, "longitud": 109, "latitud": 109, "to_geometri": [109, 114, 115], "st_contain": [109, 114, 115, 258], "lie": [109, 118], "exterior": 109, "interior": [109, 112], "li": 109, "st_cross": [109, 114, 115, 264], "st_disjoint": [109, 114, 115, 264], "st_equal": [109, 114, 115], "st_intersect": [109, 114, 115, 263, 264], "disjoint": 109, "st_overlap": [109, 114, 115, 264], "st_relat": [109, 114, 115], "st_touch": [109, 114, 115, 264], "st_within": [109, 114, 115, 264, 280], "geometry_nearest_point": [109, 114, 115, 386], "arbitrarili": 109, "geometry_union": [109, 114, 115, 278], "geometry_union_agg": [109, 114, 115, 274], "st_boundari": [109, 114, 115], "closur": 109, "combinatori": 109, "st_buffer": [109, 114, 115, 255], "st_differ": [109, 114, 115], "st_envelop": [109, 114, 115, 261, 264], "rectangular": 109, "st_envelopeaspt": [109, 114, 269], "corner": [109, 327, 472], "st_exterior": [109, 114, 115, 270], "ring": 109, "st_symdiffer": [109, 114, 115], "symmetr": 109, "st_union": [109, 114, 115, 271], "st_centroid": [109, 114, 115], "centroid": 109, "st_convexhul": [109, 114, 115, 267, 277], "convex": 109, "enclos": [109, 112, 237, 351, 512, 520], "st_coorddim": [109, 114, 115], "st_dimens": [109, 114, 115], "inher": [109, 520], "dimension": [109, 119], "st_geometryn": [109, 114, 115, 267], "st_numgeometri": [109, 114, 115, 267], "st_interiorringn": [109, 114, 115, 271], "st_numinterior": [109, 114, 115], "st_geometrytyp": [109, 114, 115, 265], "st_isclos": [109, 114, 115], "coincid": [109, 455], "st_isempti": [109, 114, 115], "st_issimpl": [109, 114, 115, 262], "anomal": 109, "tangenc": 109, "st_isr": [109, 114, 115], "st_isvalid": [109, 114, 115, 263, 267], "geometry_invalid_reason": [109, 114, 115, 263, 267], "st_pointn": [109, 114, 115, 271], "st_numpoint": [109, 114, 115, 267], "st_xmax": [109, 114, 115, 262], "maxima": 109, "st_ymax": [109, 114, 115, 262], "st_xmin": [109, 114, 115, 262], "minima": 109, "st_ymin": [109, 114, 115, 262], "st_startpoint": [109, 114, 115], "shortcut": 109, "simplify_geometri": [109, 114, 115, 263], "dougla": 109, "peucker": 109, "Will": 109, "st_endpoint": [109, 114, 115], "st_x": [109, 114, 115], "st_y": [109, 114, 115], "st_interior": [109, 114, 115, 271], "st_geometri": [109, 114, 115, 271], "line_interpolate_point": [109, 114, 115, 351], "line_locate_point": [109, 114, 115, 264, 267], "closest": 109, "great_circle_dist": [109, 114, 115, 263], "latitude1": 109, "longitude1": 109, "latitude2": 109, "longitude2": 109, "surfac": 109, "kilomet": 109, "to_geojson_geometri": [109, 114, 115, 380], "geojson": 109, "from_geojson_geometri": [109, 114, 115, 380], "strip": 109, "featurecollect": 109, "convex_hull_agg": [109, 114, 115, 274, 277], "bing_til": [109, 114, 115], "zoom_level": 109, "bingtil": [109, 255], "xy": 109, "zoom": 109, "quadkei": 109, "bing_tile_at": [109, 114, 115, 254], "05112878": 109, "bing_tiles_around": [109, 114, 115, 267, 274], "surround": [109, 112, 121, 520], "radius_in_km": 109, "cover": [109, 112, 151, 391], "bing_tile_coordin": [109, 114, 115, 265], "bing_tile_polygon": [109, 114, 115, 254, 264], "bing_tile_quadkei": [109, 114, 115], "bing_tile_zoom_level": [109, 114, 115], "geometry_to_bing_til": [109, 114, 115, 257, 267], "to_encoded_polylin": [109, 114, 115, 352], "from_encoded_polylin": [109, 114, 115, 352], "sketch": [110, 120, 123, 127, 144], "spars": [110, 118, 119, 144], "dens": [110, 144], "contigu": 110, "lifetim": [110, 438, 470], "entireti": [110, 455], "daili": [110, 144], "weekli": 110, "monthli": 110, "revenu": 110, "visit_summari": 110, "visit_d": 110, "user_visit": 110, "weekly_unique_us": 110, "summar": [110, 287, 462, 486, 490, 491], "empty_approx_set": [110, 114, 115], "cidr": [111, 392, 454], "0db8": 111, "ff00": 111, "0042": 111, "8329": [111, 386], "8328": 111, "explor": [112, 135, 137, 512], "deeper": 112, "moreov": 112, "unnest": [112, 123, 142, 154, 180, 258, 272, 273, 306, 307, 310, 313, 317, 325, 333, 343, 346, 350, 353, 361, 380, 417], "diverg": 112, "2e3": 112, "nan": [112, 114, 115, 118, 155, 255, 266, 311, 317, 345, 356, 364, 365, 366, 373, 433, 505], "subscript": [112, 144, 156, 210, 228, 229, 233, 240, 245, 333, 341, 382], "innermost": 112, "path1": 112, "path2": 112, "parenthes": [112, 146, 147, 388, 510, 520], "faulti": 112, "preorder": 112, "visit": [112, 137], "unwrap": [112, 305, 391], "path3": 112, "path4": 112, "singleton": 112, "improp": [112, 227, 237, 373, 425, 432], "syntact": [112, 275, 505, 510, 512], "examin": [112, 151, 510], "predicate1": 112, "predicate2": 112, "disjunct": 112, "pairwis": 112, "23e4": 112, "1e0": 112, "6e0": 112, "like_regex": 112, "unchang": [112, 478, 480], "prior": [112, 467, 468], "json_input": 112, "utf8": 112, "utf16": 112, "utf32": 112, "json_path": 112, "json_argu": 112, "variable_nam": 112, "malform": [112, 258, 361], "tax": [112, 510], "o_pric": 112, "o_tax": 112, "json_desc": 112, "o_desc": 112, "binary_record": 112, "o_rec": 112, "unquot": [112, 126], "keynam": 112, "101": [112, 153], "nice": 112, "children": [112, 505], "102": [112, 153], "problemat": 112, "children_above_ten": 112, "child_3_above_ten": 112, "uncondit": 112, "cust_id": 112, "last_child": 112, "quoted_com": 112, "unquoted_com": 112, "err": 112, "array_el": [112, 169], "12e": 112, "5b0035005d00": 112, "12151fd2": [112, 144], "7586": [112, 144], "11e9": [112, 144], "8f9e": [112, 144], "2a86e4085a59": [112, 144], "altogeth": 112, "treatment": 112, "5b": 112, "5d": 112, "key_valu": 112, "7b": 112, "3a": 112, "shall": 112, "succes": [112, 148], "json_pars": [112, 114, 115, 170, 171, 189, 248, 251, 341], "k1": [112, 117], "k2": [112, 117], "k3": [112, 117], "imposs": 112, "is_json_scalar": [112, 114, 115, 261], "json_array_contain": [112, 114, 115], "json_array_get": [112, 114, 115, 217, 310], "json_array_length": [112, 114, 115], "json_extract": [112, 114, 115, 196, 233, 302], "jsonpath": [112, 137], "book": [112, 151, 302], "rich": 112, "json_format": [112, 114, 115, 170, 171, 189], "invers": [112, 118, 129, 247], "7159": 112, "gone": [112, 302], "not_json": 112, "json_siz": [112, 114, 115, 300, 301], "regexp_lik": [113, 114, 115, 121], "squared_numb": 113, "price_tag": 113, "showcas": [113, 123, 137], "ax": [113, 264], "xvalu": 113, "linear_function_valu": 113, "capit": [113, 275, 331], "word": [113, 121, 124, 329, 336, 475, 510, 512], "regexp_replac": [113, 114, 115, 121, 261], "overli": 113, "sum_valu": 113, "aco": [114, 115, 118], "asin": [114, 115, 118], "atan": [114, 115, 118], "atan2": [114, 118], "beta_cdf": [114, 115, 118, 280], "cbrt": [114, 115, 118], "char2hexint": [114, 115, 128, 162], "chr": [114, 115, 124, 303], "codepoint": [114, 115, 124, 236], "concat_w": [114, 115, 124, 372, 448, 525], "cosh": [114, 115, 118], "cosine_similar": [114, 115, 118, 205, 212], "cume_dist": [114, 115, 131], "current_group": [114, 115, 122, 386], "dense_rank": [114, 115, 131, 392], "evaluate_classifier_predict": [114, 296], "exclude_column": [114, 126, 439], "exp": [114, 115, 118], "first_valu": [114, 115, 131, 158, 167, 297], "from_bas": [114, 115, 118, 154], "from_utf8": [114, 115, 124, 144, 156], "greatest": [114, 115, 297, 315, 373], "hamming_dist": [114, 115, 124, 249], "hash_count": [114, 115, 123, 386], "intersection_cardin": [114, 115, 123], "inverse_beta_cdf": [114, 115, 118, 280], "inverse_normal_cdf": [114, 115, 118, 251], "is_finit": [114, 115, 118], "is_infinit": [114, 118], "jaccard_index": [114, 115, 123], "json_arrai": [114, 115, 142, 412, 439], "json_exist": [114, 115, 142, 410], "json_object": [114, 115, 142, 412, 439], "json_queri": [114, 115, 142, 410, 411], "json_valu": [114, 115, 142, 410], "lag": [114, 115, 131, 158, 167, 250, 300, 308], "last_valu": [114, 115, 131, 158, 167, 297], "learn_classifi": [114, 115, 119, 296], "learn_libsvm_classifi": [114, 115, 119, 297], "learn_libsvm_regressor": [114, 115, 119, 297], "learn_regressor": [114, 115, 119], "levenshtein_dist": [114, 115, 124, 232, 234], "ln": [114, 115, 118, 265], "log10": [114, 115, 118], "log2": [114, 115, 118], "ltrim": [114, 115, 124, 401], "luhn_check": [114, 115, 124, 367], "make_set_digest": [114, 115, 123, 386], "map_concat": [114, 115, 117, 184, 207, 226, 233, 239, 403], "map_entri": [114, 115, 117, 248], "map_filt": [114, 115, 117, 233], "map_from_entri": [114, 115, 117, 248, 267, 403, 520], "map_kei": [114, 115, 117, 307], "map_valu": [114, 115, 117, 307], "map_zip_with": [114, 115, 117, 258], "match_numb": [114, 115, 512], "merge_set_digest": [114, 115, 123], "mod": [114, 115, 118, 124, 217], "multimap_from_entri": [114, 115, 117, 267, 403], "normal_cdf": [114, 115, 118, 258], "nth_valu": [114, 115, 131, 158, 167, 297, 308], "ntile": [114, 115, 131, 313], "nullif": [114, 115, 260, 270, 301, 402, 429], "objectid": [114, 241, 330, 342, 359, 432], "percent_rank": [114, 115, 131], "pow": [114, 115, 118, 169], "prev": [114, 115, 510, 512], "radian": [114, 115, 118, 154], "rand": [114, 115, 118, 288, 466], "rank": [114, 115, 379, 392, 440, 520], "regexp_count": [114, 115, 121, 355], "regexp_extract": [114, 115, 121], "regexp_extract_al": [114, 115, 121, 317], "regexp_posit": [114, 115, 121, 355], "regexp_split": [114, 115, 121, 155], "rtrim": [114, 115, 124, 210, 248], "sin": [114, 115, 118], "sinh": [114, 115, 118, 438], "soundex": [114, 115, 124, 383], "spatial_partit": 114, "split_part": [114, 115, 124], "split_to_map": [114, 115, 124, 209, 249], "split_to_multimap": [114, 115, 124, 271], "sqrt": [114, 115, 118], "strpo": [114, 115, 124, 128, 352], "tan": [114, 115, 118], "tanh": [114, 115, 118], "to_bas": [114, 115, 118, 154], "to_char": [114, 115, 128, 162], "to_dat": [114, 115, 128, 162], "to_timestamp": [114, 115, 128, 162], "to_utf8": [114, 115, 124, 156], "transform_kei": [114, 115, 117, 232, 246], "transform_valu": [114, 115, 117, 232, 241, 246, 354], "url_decod": [114, 115, 129, 167, 213], "url_encod": [114, 115, 129, 167], "url_extract_frag": [114, 115, 129], "url_extract_host": [114, 115, 129], "url_extract_paramet": [114, 115, 129, 344], "url_extract_path": [114, 115, 129], "url_extract_protocol": [114, 115, 129], "url_extract_port": [114, 115, 129], "url_extract_queri": [114, 115, 129], "value_at_quantil": [114, 115, 120, 127], "values_at_quantil": [114, 115, 120, 127], "width_bucket": [114, 115, 118, 197], "wilson_interval_low": [114, 115, 118, 267], "wilson_interval_upp": [114, 115, 118, 267], "word_stem": [114, 115, 124, 259, 354], "current_catalog": [115, 122, 142, 384], "current_schema": [115, 122, 142, 384], "name_to_age_map": 117, "bob_ag": 117, "map1": 117, "map2": 117, "mapn": 117, "c9": 117, "b2": 117, "one_1": 117, "two_1": 117, "modulu": 118, "cube": [118, 142, 148, 196, 239, 441], "angl": 118, "euler": 118, "natur": [118, 132, 142, 269, 468], "logarithm": 118, "signum": 118, "bound1": 118, "bound2": 118, "equi": [118, 159, 182, 205, 209, 212, 248], "cosin": 118, "sine": 118, "tangent": 118, "hyperbol": 118, "radix": 118, "trial": 118, "wilson": 118, "bernoulli": [118, 520], "confid": 118, "beta": [118, 302], "sd": 118, "train": [119, 296], "svm": 119, "regressor": [119, 296], "supervis": [119, 136], "solv": [119, 468], "predict": 119, "unseen": 119, "aster": 119, "sepal_length": 119, "sepal_width": 119, "petal_length": 119, "petal_width": 119, "3c": [119, 121], "6c": 119, "73": [119, 153, 328, 506], "66": [119, 153, 295], "76": [119, 153, 305], "68": [119, 153, 338], "3e": 119, "predicted_label": 119, "libsvm": [119, 297], "predicted_target": 119, "407376822560477": 119, "quantile_at_valu": [120, 438], "notabl": 121, "surrog": 121, "ud800": 121, "udc00": 121, "q": 121, "z123": 121, "oldital": 121, "old_ital": 121, "Is": 121, "sc": 121, "hiragana": 121, "blk": 121, "mongolian": 121, "categori": [121, 143, 278, 461, 476], "general_categori": 121, "noncharactercodepoint": 121, "1a": 121, "2b": 121, "14m": 121, "anchor": [121, 512], "pear": 121, "dollar": 121, "3ca": 121, "3cb": 121, "york": [121, 520], "deal": [123, 373], "minhash": [123, 144], "jaccard": [123, 144], "mine": 123, "naiv": 123, "shingl": 123, "text_input": 123, "quick": 123, "brown": 123, "jump": [123, 510], "dog": [123, 144, 520], "text_ngram": 123, "minhash_digest": 123, "setdigest_side_by_sid": 123, "id1": 123, "digest1": 123, "id2": 123, "digest2": 123, "m1": 123, "m2": 123, "argu": 123, "somewhat": 123, "murmur3hash128": 123, "19144387141682250": 123, "2447670524089286488": 123, "graphem": [124, 156], "perceiv": [124, 156, 248], "lithuanian": 124, "turkish": 124, "azeri": 124, "string1": 124, "stringn": 124, "string2": 124, "string0": 124, "ham": 124, "levenshtein": 124, "padstr": 124, "luhn": 124, "modulo": 124, "credit": 124, "card": 124, "identif": [124, 469], "distinguish": [124, 254, 268, 282, 461, 469, 512], "mistyp": 124, "79927398713": 124, "79927398714": 124, "phonet": 124, "sound": 124, "spoken": 124, "chyna": 124, "china": [124, 183], "entrydelimit": 124, "keyvaluedelimit": 124, "zbcd": 124, "abcda": 124, "zbcdz": 124, "palho\u00e7a": 124, "\u00e7": 124, "palhoca": 124, "01f600": [124, 144], "cd": 124, "bcd": 124, "zy": 124, "ac": 124, "zbd": 124, "aac": 124, "zq": 124, "er": 124, "stem": 124, "nfc": [124, 244], "nfd": [124, 244], "decomposit": 124, "composit": [124, 441], "nfkd": [124, 244], "nfkc": [124, 244], "fffd": 124, "node_vers": 125, "polymorph": 126, "enhanc": [126, 277, 278, 430, 465], "guidanc": 126, "sequential_numb": 126, "2000000": [126, 136, 334], "reachabl": [126, 467], "analog": [126, 510, 520], "position": 126, "stmt": 126, "hexadecim": [128, 129, 144, 447], "16be": 128, "punctuat": 128, "hh24": 128, "mi": 128, "2396": 129, "1866": [129, 352], "req_path": 129, "ftp": 129, "alphanumer": [129, 136, 142, 302, 446, 449, 455], "xx": [129, 136, 147, 462], "unescap": [129, 361], "rnk": [131, 520], "roll": [131, 209, 293, 450], "rolling_sum": 131, "tie": [131, 303], "default_valu": 131, "cert": [132, 464, 465, 475], "509": [132, 334, 461], "Its": [132, 281], "lightweight": 132, "heterogen": [132, 150], "technic": [132, 254], "earliest": 132, "cryptograph": [132, 467], "lb": 132, "hardwar": 132, "sit": 132, "edg": [132, 254], "behalf": [132, 254, 462, 467, 470], "globe": 132, "vendor": [132, 276, 278, 464, 475], "pkc": [132, 268, 464, 465, 475], "p12": [132, 464, 475], "pfx": 132, "prestosql": [132, 327, 342, 345, 375, 385], "supersed": 132, "tarbal": [132, 136, 139, 297, 298, 361], "tar": [132, 136, 139, 475], "successor": 132, "helm": [133, 134, 135, 138, 139], "rpm": [133, 134, 136, 163, 177, 348, 361], "salesforc": [133, 368, 458, 459, 471, 476], "match_recogn": [133, 143, 383, 388, 432, 433, 477, 512, 520], "glossari": [133, 475], "appendix": 133, "imag": [135, 136, 137], "hub": 135, "websit": 135, "ps": 135, "955c3b3d3d0a": 135, "390": [135, 153, 334], "39": 135, "health": [135, 260, 399], "exec": 135, "20181105_001601_00002_e6r6i": [135, 137], "rm": 135, "adequ": 136, "ulimit": 136, "nofil": 136, "nproc": 136, "131072": 136, "128000": 136, "eclips": 136, "temurin": 136, "adoptium": 136, "charset": [136, 451], "dfile": [136, 451], "unpack": 136, "upgrad": [136, 158, 178, 281, 293, 320, 364, 396, 425, 428, 432, 466, 475], "environment": [136, 435], "ffffffff": 136, "ffff": 136, "ffffffffffff": 136, "reboot": [136, 225], "launch": 136, "xmx16g": 136, "initialrampercentag": 136, "maxrampercentag": 136, "g1heapregions": 136, "32m": 136, "explicitgcinvokesconcurr": 136, "exitonoutofmemoryerror": 136, "heapdumponoutofmemoryerror": 136, "omitstacktraceinfastthrow": 136, "reservedcodecaches": 136, "512m": 136, "permethodrecompilationcutoff": 136, "perbytecoderecompilationcutoff": 136, "djdk": [136, 334], "allowattachself": 136, "maxcachedbuffers": [136, 334], "unlockdiagnosticvmopt": 136, "useaesctrintrins": 136, "8293861": 136, "g1usepreventivegc": 136, "gclocker": 136, "8192647": 136, "gclockerretryallocationcount": 136, "xmx": 136, "64gb": 136, "xmx54g": 136, "32gb": 136, "outofmemoryerror": [136, 218, 240, 368], "forcibl": 136, "noexec": 136, "arm64": [136, 357, 386, 389, 397, 408], "8271567": 136, "oom": [136, 243, 276, 320, 321], "explan": 136, "heartbeat": [136, 323], "comprehens": 136, "logger": 136, "hierarchi": [136, 139, 151], "dot": [136, 287, 305, 369, 390], "daemon": [136, 139, 303], "foreground": 136, "sigterm": [136, 139], "signal": [136, 139, 510], "hung": 136, "sigkil": 136, "pid": [136, 139], "stdout": [136, 343, 468], "daemontool": 136, "fh": 136, "diagnost": 136, "laptop": 137, "fastest": 137, "prototyp": 137, "premis": 137, "kubectl": 137, "adher": 137, "haven": [137, 542], "repo": 137, "Be": [137, 466], "sep": [137, 153], "revis": 137, "pod_nam": 137, "pod": 137, "bfb74c98d": 137, "rnrxd": 137, "161m": 137, "76f6bf54d6": 137, "hvl8n": 137, "tcqgb": 137, "clusterip": 137, "replicaset": 137, "tunnel": 137, "consol": [137, 343, 380, 468], "mimic": 137, "tradit": [137, 152, 476], "maxheaps": 137, "8g": 137, "8gb": 137, "guidelin": [137, 460], "additionalcatalog": 137, "lakehous": 137, "rdbm": 137, "obviou": 137, "kube": 137, "foll": 137, "uninstal": [137, 328, 331, 373], "interrupt": [138, 155, 158, 173, 186, 407, 470], "piec": 138, "exchangemanag": 138, "additionalexchangemanagerproperti": 138, "additionalconfigproperti": 138, "resum": [138, 510], "discourag": 139, "favor": [139, 163, 251, 320, 321, 353, 388, 414, 425, 436, 440, 447], "nodep": 139, "systemctl": 139, "rc": 139, "rpmsave": 139, "complianc": [140, 173], "dash": 141, "span": [141, 144, 293, 464, 465], "current_path": 142, "current_rol": [142, 487, 508, 518], "json_tabl": 142, "rollup": [142, 148, 196, 239, 329, 441], "uescap": [142, 144], "letter": [142, 217, 288, 331], "example_catalog": 142, "a_schema": 142, "identifierwith": 142, "123schemanam": 142, "colum": 142, "categor": [143, 194, 213, 233, 246, 278, 294, 296], "0x": 144, "0x0a": 144, "0x11": 144, "0o": 144, "0o40": 144, "0o11": 144, "0b1001": 144, "0b101010": 144, "123_456": 144, "789_123": 144, "789123": 144, "0x80": 144, "0x7f": 144, "0x8000": 144, "0x7fff": 144, "0x80000000": 144, "0x7fffffff": 144, "0x8000000000000000": 144, "0x7fffffffffffffff": 144, "scientif": 144, "notat": 144, "03e1": 144, "789e4": 144, "inexact": 144, "3e0": 144, "000123": 144, "1234567890": 144, "winter": 144, "2603": 144, "snowman": 144, "grin": 144, "emoji": 144, "eh": 144, "65683f": 144, "picosecond": [144, 375], "383345": 144, "383345000000": 144, "gmt": [144, 234, 253], "ut": 144, "usabl": 144, "2e0": 144, "pure": [144, 146], "4291": 144, "5952": 144, "univers": [144, 510, 512], "4122": 144, "cheapli": 144, "tunabl": 144, "99th": 144, "particip": 146, "disadvantag": 146, "join_max_broadcast_table_s": [146, 278, 332], "misestim": 146, "formal": 146, "medium": 146, "large_t": 146, "medium_t": 146, "small_tabl": 146, "artifact": [146, 287], "69kb": 147, "6148": 147, "1734": 147, "remoteexchang": [147, 505], "94kb": 147, "2207": 147, "4414": [147, 368], "reduct": [148, 282], "neccessari": 148, "regionkey_0": 148, "_generated_1": 148, "groupingset": 148, "gro": 148, "count_0": [148, 505, 506], "hashvalue_1": [148, 505, 506], "hashvalue_2": [148, 505, 506], "sf0": [148, 505], "225b": [148, 505], "postgr": [148, 451, 462], "orderkey_0": 148, "custkey_1": 148, "custkey_2": 148, "unsort": 148, "ungrouped_execut": 148, "sortord": 148, "topnparti": 148, "58mb": [148, 506], "58m": [148, 506], "absenc": [148, 520], "solid": 151, "collabor": 151, "spread": 151, "brain": 151, "alongsid": 151, "advertis": [151, 233], "fundament": [151, 217], "mandatori": [151, 407, 512, 520], "test_data": 151, "unord": [151, 520], "curiou": 151, "realiz": 151, "interconnect": 151, "instanti": [151, 350], "encompass": 151, "concert": 151, "billion": [151, 206, 441], "resembl": 151, "hors": 151, "deconstruct": 151, "perspect": 152, "prospect": 152, "onlin": [152, 467], "oltp": [152, 441], "wareh": 152, "vast": 152, "pig": [152, 520], "olap": 152, "oct": 153, "429": [153, 333], "428": [153, 333], "427": 153, "426": 153, "425": 153, "aug": 153, "424": 153, "422": 153, "421": [153, 333], "420": [153, 333], "jun": 153, "419": [153, 334], "418": [153, 333], "417": 153, "416": 153, "415": 153, "apr": 153, "414": [153, 333], "413": [153, 333], "412": 153, "411": 153, "mar": 153, "410": 153, "409": 153, "408": [153, 333], "feb": 153, "407": [153, 333], "405": 153, "404": 153, "403": [153, 226], "nov": 153, "402": 153, "401": [153, 388], "399": 153, "398": [153, 332, 464, 465], "397": 153, "396": 153, "395": 153, "394": [153, 423], "393": 153, "392": 153, "391": 153, "389": 153, "388": 153, "387": 153, "386": 153, "385": 153, "384": 153, "383": [153, 336, 411], "382": 153, "381": 153, "380": 153, "379": 153, "378": 153, "377": 153, "376": [153, 332], "375": 153, "374": [153, 506], "372": 153, "371": 153, "370": 153, "369": 153, "368": 153, "367": 153, "365": [153, 282], "364": 153, "363": 153, "362": 153, "361": 153, "360": [153, 397], "359": 153, "358": 153, "357": 153, "356": 153, "355": 153, "354": 153, "353": [153, 332], "352": [153, 337], "349": 153, "348": 153, "347": 153, "345": 153, "344": 153, "343": [153, 335], "342": 153, "341": [153, 369], "340": 153, "339": 153, "337": 153, "336": 153, "335": 153, "334": [153, 331], "332": 153, "331": 153, "330": [153, 333], "328": [153, 333], "327": [153, 506], "326": 153, "325": 153, "324": 153, "323": 153, "322": 153, "320": 153, "318": 153, "317": [153, 336], "316": [153, 332], "315": [153, 332], "314": 153, "313": 153, "312": 153, "311": 153, "310": 153, "308": 153, "307": [153, 375], "306": [153, 331], "305": [153, 331], "304": 153, "303": 153, "302": 153, "301": 153, "215": [153, 330], "214": [153, 280], "213": [153, 279, 330], "212": [153, 331], "211": 153, "210": [153, 276, 506], "209": [153, 275], "207": [153, 330], "206": 153, "205": 153, "204": 153, "203": [153, 330], "202": 153, "201": 153, "199": [153, 265], "198": [153, 505], "197": 153, "196": 153, "195": [153, 330], "194": [153, 505], "193": 153, "191": [153, 257, 260, 368], "189": 153, "188": 153, "187": 153, "186": [153, 345], "185": 153, "184": 153, "183": 153, "182": [153, 330], "181": [153, 258], "178": 153, "177": [153, 330], "176": 153, "175": 153, "174": 153, "173": 153, "171": [153, 345], "170": 153, "169": [153, 330], "167": 153, "166": [153, 329], "165": 153, "164": [153, 329], "163": [153, 329], "162": 153, "161": [153, 329], "159": 153, "158": [153, 506], "157": [153, 329, 506], "156": [153, 242], "154": 153, "153": [153, 329], "152": [153, 527], "151": 153, "149": 153, "148": 153, "146": [153, 329], "145": [153, 198, 505], "144": 153, "143": 153, "142": [153, 329], "141": [153, 331], "140": 153, "139": 153, "138": 153, "137": 153, "136": 153, "135": 153, "134": [153, 189, 329], "133": [153, 186, 329], "132": 153, "131": [153, 329], "130": [153, 183], "129": 153, "126": [153, 432], "125": [153, 520], "124": [153, 329], "122": [153, 189], "120": [153, 329], "119": 153, "118": [153, 329], "117": [153, 329], "116": [153, 176], "115": [153, 170], "113": 153, "112": 153, "111": 153, "110": 153, "108": 153, "107": 153, "106": 153, "105": [153, 329], "104": 153, "94": 153, "91": 153, "88": [153, 468, 506], "87": 153, "84": 153, "82": [153, 329], "getsystemt": 154, "leak": [154, 156, 160, 161, 167, 180, 181, 188, 193, 197, 200, 206, 210, 213, 217, 225, 229, 230, 239, 244, 246, 250, 251, 255, 261, 273, 279, 287, 293, 294, 310, 317, 318, 319, 322, 323, 338, 340, 341, 365, 374, 379, 381, 392, 396, 401, 416, 424, 429], "teardown": [154, 197, 287, 327], "errornam": 155, "errortyp": 155, "getidentifierquotestr": 155, "resultset": [155, 165, 177, 254, 334, 347, 366, 368, 373, 375, 403], "unari": 155, "hash_partition_count": 155, "reactj": 155, "splittabl": [155, 285], "prestos3filesystem": [155, 156, 158, 378], "seek": [155, 156, 512], "callback": [155, 249, 470], "connectorpartit": [155, 293], "connectorcolumnhandl": [155, 293], "twice": [156, 464], "quadrat": [156, 520], "minor": [156, 183, 232, 289, 290, 297, 301, 368, 392], "task_writer_count": [156, 396, 425, 455], "invalidrang": 156, "task_default_concurr": 157, "task_join_concurr": [157, 208], "task_hash_build_concurr": [157, 208], "task_aggregation_concurr": [157, 208], "eventu": [157, 266, 321], "npe": [157, 310, 403], "250": [157, 331], "statementcli": [158, 173], "hang": [158, 163, 183, 233, 254, 257, 259, 262, 265, 278, 281, 286, 293, 303, 304, 305, 360, 367, 380, 394, 396, 433, 449, 454], "unreach": 158, "perman": [159, 355, 361], "ordinalposit": 159, "columnmetadata": [159, 194, 351, 402], "BYs": 160, "chita": 160, "srednekolymsk": 160, "pacif": [160, 379, 396], "bougainvil": 160, "race": [160, 169, 177, 178, 187, 194, 197, 201, 208, 211, 214, 235, 260, 287, 304, 305, 361], "sooner": 160, "prematur": 160, "reenabl": 161, "classif": [161, 290, 291], "10ffff": 161, "setreadonli": 162, "bug": [163, 164, 169, 171, 176, 180, 189, 192, 193, 196, 197, 199, 201, 206, 209, 211, 229, 233, 239, 240, 241, 242, 243, 245, 254, 257, 263, 275, 280, 285, 286, 288, 291, 297, 299, 300, 304, 306, 314, 317, 323, 325, 346, 392, 394, 399, 426], "soon": [163, 287, 313, 364, 420], "experimental_big_queri": [163, 307], "rn": [164, 302], "distributed_join": [165, 272, 323, 333], "announc": 167, "sessionpropertymetadata": 167, "hivesessionproperti": 167, "getproperti": [167, 305], "midnight": 168, "reintroduc": 170, "bys": 171, "sqltask": [172, 305, 323], "ve": [172, 282, 307], "lightli": 172, "restor": [173, 212, 226, 228, 261, 264, 265, 266, 276, 281, 339, 355, 356, 366, 373, 384, 389, 395, 396, 414, 432, 438, 450], "notif": [173, 232], "nosuchelementexcept": 176, "icon": 176, "getcolumn": [177, 213, 361, 375, 399, 402, 432], "detector": [177, 273, 281], "table_properti": [177, 489, 490], "order_region": 177, "order_info": 177, "apac": 177, "redund": [178, 191, 194, 223, 251, 268, 290, 339, 347, 353, 355, 356, 358, 398, 438, 450, 520], "task_intermediate_aggreg": [178, 208], "verifier_queri": [178, 320], "test_postqueri": 178, "test_prequeri": 178, "control_postqueri": 178, "control_prequeri": 178, "occasion": [180, 277, 427], "task_share_index_load": 180, "tableau": [180, 212], "equat": 180, "exchangecli": 180, "maxresponses": 180, "slightli": [180, 470], "pagetoolargeexcept": 180, "disrupt": 181, "trigger": [182, 218, 329, 332, 381, 382, 460, 462, 486], "useless": [183, 375], "trade": [183, 510], "getupdatecount": 183, "getlargeupdatecount": 183, "screen": [183, 384], "retention_dai": [183, 191], "columnar_processing_dictionari": [184, 186, 187, 210], "unresolv": 186, "resource_overcommit": [186, 198, 462], "reclaim": [186, 296], "sporad": [186, 365, 397], "nullpointerexcept": [186, 228, 292, 324, 383, 432], "procedur": [187, 274, 328, 331, 333, 344, 345, 356, 357, 358, 359, 362, 366, 374, 391, 392, 394, 396, 402, 403, 404, 405, 409, 410, 415, 425, 426, 428, 430, 432, 434, 438, 439, 441, 446, 448, 449, 450, 483], "sticki": 187, "is_partition_kei": 188, "dictionaryblock": [188, 424], "tablelayout": 188, "safari": 189, "clock": [192, 293, 307], "latent": [192, 307], "parquet_optimized_reader_en": [192, 278], "parquet_predicate_pushdown_en": [192, 278], "initial_splits_per_nod": 193, "split_concurrency_adjustment_interv": 193, "dwrf": [193, 210, 219, 248, 249, 254, 276, 304, 332], "failure_host": 194, "failure_task": 194, "querycompletionev": [194, 380], "unnecessarili": [194, 271], "bogu": 194, "getdiscretepred": 194, "optimize_metadata_queri": [196, 379], "infouri": 198, "compil": [201, 206, 219, 232, 260, 287, 293, 297, 302, 303, 317, 321, 337, 368, 380], "symlink": [201, 207, 208, 357, 375, 398, 454], "prestointerv": [204, 209], "misinterpret": [204, 209], "raptor": [206, 297, 302, 378, 399], "reliabl": [206, 253, 268, 288, 339, 428, 457], "congest": [206, 395], "hex": [206, 302, 307, 375, 472], "4096": [207, 364, 473], "1680": [208, 348], "coloc": [208, 209, 256, 278], "intra": 208, "noclassdeffounderror": [208, 209, 219, 245, 413], "subnetutil": 208, "unrecover": [208, 293], "clustered_bi": 208, "g1": [209, 263, 320, 321], "collector": [209, 272], "kmsclientprovid": 209, "intuit": 210, "realtim": [210, 242], "columnar_dictionari": 210, "columnar_process": 210, "processing_optim": [210, 239], "ipc": 210, "ping": 210, "repeatedli": 210, "agg": [212, 328], "uncorrel": [212, 218, 373], "shadow": [212, 213, 278], "getdatasourcenam": 212, "datasourcenam": 212, "fixedsplitsourc": 212, "decimal_digit": [213, 375], "num_prec_radix": 213, "char_octet_length": 213, "allview": 213, "getallview": 213, "thrifthivemetastor": 213, "connectortablemetadata": 213, "getservic": [213, 287], "functionfactori": 213, "windowfunct": 213, "rankingwindowfunct": 213, "valuewindowfunct": 213, "spuriou": [217, 268, 274, 304, 386, 396, 399], "torn": [217, 287], "getresourcegroupconfigurationmanagerfactori": 217, "ish": 217, "orc_bloom_filters_en": 217, "rcfile_optimized_reader_en": 217, "intention": 217, "agent": [217, 339], "schemaexist": 217, "iscoordin": 217, "getvers": 217, "setoptionalconfig": 217, "serverinfo": 217, "nodemanag": [217, 287, 328], "connectorfactorycontext": 217, "introduct": [218, 471], "evolut": [219, 274, 275, 379, 389, 392, 397, 413], "optimize_mixed_distinct_aggreg": [220, 242, 327], "quantifi": [220, 227, 364, 383, 384, 520], "selector": [223, 255, 256, 262, 332, 398], "flood": 225, "bucket_execution_en": [225, 462], "eas": [226, 287, 321, 471], "legacy_order_bi": [226, 244], "respond": [226, 461, 475], "forbidden": [226, 510, 520], "reorder_join": [227, 272, 368], "former": [228, 270, 368, 398, 407], "alreadyexistsexcept": 231, "splitstatist": [231, 275], "monopol": 232, "elaps": [232, 246, 251, 348, 361, 486], "longest": 232, "8u92": [232, 274], "viewfilesystem": 232, "cdh4": [232, 281], "hadoop1": [232, 281], "cdh": [232, 321, 353, 354], "cdh5": 232, "hadoop2": [232, 284, 386, 435], "defunct": 232, "partition": 232, "beginqueri": 232, "cleanupqueri": 232, "revert": [233, 253, 301, 466, 478, 480], "iterative_optimizer_timeout": 233, "rcfile_optimized_writer_en": 233, "connectorindex": [233, 325], "resultsetmetadata": [234, 335, 375], "getcolumnclassnam": 234, "interruptedexcept": 235, "gettypeinfo": 235, "mislead": 235, "fire": [236, 267, 358], "underestim": [239, 428, 430], "table_privileg": [239, 339, 346], "es6": 239, "checkcanshowt": 239, "checkcanshowtablesmetadata": 239, "enable_intermediate_aggreg": 240, "un": [240, 297, 302, 357], "recoveri": 240, "overestim": 241, "mishandl": 242, "fast_inequality_join": 242, "semijoin": 242, "push_partial_aggregation_through_join": [242, 359], "violat": 242, "hud": 242, "sparklin": 242, "preprocess": 242, "presto_preprocessor": 242, "8262": 244, "manifest": [244, 392, 417, 422, 434, 455, 486], "redact": [244, 368, 384, 419], "2017b": 244, "unencrypt": 244, "setquerytimeout": 244, "grante": [244, 494, 507, 517], "revoke": 244, "resourcegroupst": [245, 346], "resourcegroupinfo": 245, "ever": 246, "quanta": 246, "disproportion": [246, 256], "broke": 246, "upcom": [247, 364], "orc_optimized_writer_en": [247, 278], "operator_memory_limit_before_spil": 248, "unspil": [248, 264], "aggregation_operator_unspill_memory_limit": 248, "222": 248, "arrayblock": [248, 249, 263, 276, 334], "interleavedblock": [248, 249], "mapblock": [248, 263, 276], "singlemapblock": 248, "nullscount": 248, "nullsfract": 248, "mandat": 249, "singlerowblock": 249, "stabil": [251, 252, 274, 297, 331, 432], "counter": [251, 255, 289, 341], "decrement": 251, "bing": [251, 257, 258], "tile": [251, 254, 257, 258], "maxrun": 251, "getqueuedtimemilli": 251, "getelapsedtimemilli": 251, "ignoreexist": 251, "gettotalbyt": 251, "stress": 252, "invert": 253, "spike": 253, "differenti": [253, 510], "dereferenc": 254, "gracefulli": [254, 358, 470], "x500": 254, "x500princip": 254, "leg": 254, "spnego": 254, "setcatalog": 254, "setschema": 254, "slicearrayblock": [254, 257], "sessionpropertyconfigurationmanag": 254, "imbal": 255, "dictionary_processing_join": [255, 257], "parse_decimal_literals_as_doubl": [255, 263], "getpeakmemorybyt": 255, "recordsink": [255, 317], "pagesink": 255, "concurrent_lifespans_per_task": [256, 262, 270, 415], "tradeoff": 256, "elev": 257, "irrelev": 257, "bottom": [257, 462, 476, 510], "border": 257, "100000000000000000000000000000000": 257, "100000000000000005366162204393472": 257, "object_not_read": 257, "viewf": 257, "getposit": 257, "constrain": [258, 329, 450], "set_digest": 258, "__internal_partitions__": 258, "concurrentmodificationexcept": 258, "region_nam": 258, "nonexist": 258, "setnetworktimeout": 259, "getupdatetyp": 259, "prestostat": [259, 270], "offlin": [259, 260, 264, 297, 470], "hive_storage_format": 259, "respect_table_format": 259, "lazili": 259, "slide": [260, 512, 520], "zstandard": [260, 399], "accident": [260, 462], "legacy_join_us": 261, "cet": 262, "brussel": 262, "parameter": [262, 389], "unset": [262, 428], "2gb": [262, 355, 359, 396], "orc_optimized_writer_max_stripe_s": 262, "stipe": 262, "orc_string_statistics_limit": 262, "resourcegroupmanag": 262, "getselector": 262, "resourcegroupselector": 262, "selectioncontext": 262, "selectioncriteria": 262, "connectorviewdefinit": [262, 342, 456], "bahia_bandera": 263, "use_mark_distinct": 263, "helper": [263, 438], "appendcolumn": 263, "yield": [264, 520], "legacy_round_n_bigint": 264, "field0": [264, 362, 375], "legacy_row_field_ordinal_access": 264, "rectangl": 264, "incompatibleclasschangeerror": 264, "subject": [264, 296, 380, 392, 461, 465, 467], "presto_offlin": [264, 297], "resourcegroupid": 264, "sessionconfigurationcontext": 264, "writepositionto": [264, 397], "writeobject": [264, 397], "appendstructur": [264, 397], "unus": [266, 275, 297, 298, 343, 358, 367, 432, 446], "10592": 267, "10587": [267, 397], "max_grouping_set": 267, "coercibl": 267, "500kb": 267, "filecount": 267, "orc_tiny_stripe_threshold": 267, "piggyback": 267, "pagesort": 267, "pageindex": 267, "10696": [268, 397], "pushback": 268, "challeng": [268, 468, 470], "uuidgen": 268, "lazyblock": [268, 350, 380], "9601": 268, "addexchang": 269, "predicatepushdown": 269, "erron": 270, "lifespan": [270, 278, 415, 470], "query_max_total_memori": 270, "checkcanselectfromcolumn": 270, "checkcancreateviewwithselectfromcolumn": 270, "partialcancel": 270, "longnvarchar": 270, "longvarbinari": 270, "sessionpropertyconfigurationmanagerfactori": 270, "checkcanselectfromt": 270, "checkcanselectfromview": 270, "checkcancreateviewwithselectfromt": 270, "checkcancreateviewwithselectfromview": 270, "pressur": [271, 309, 311, 329, 375], "aka": 271, "stabl": [271, 342], "vs": [272, 381], "10724": 273, "query_max_stage_count": 273, "ppc64le": [273, 420], "presto_history_fil": 273, "11296": [274, 400], "11168": 274, "defici": 274, "8u151": 274, "11385": 275, "11120": 275, "realm": [275, 468], "preparedstat": [275, 357, 358, 374, 375, 377, 388, 402], "getusertimemilli": 275, "checkcansetus": [275, 357], "connectorid": 275, "queryinputmetadata": 275, "queryoutputmetadata": 275, "checkcansetcatalogsessionproperti": 275, "getusertim": 275, "cipher": [276, 468, 475], "weak": 276, "secreci": [276, 475], "rsa": [276, 465], "consequ": 276, "verbose_stat": 276, "timer": 276, "11196": [276, 400], "autom": 276, "transit": [276, 293, 399, 414, 471], "11322": 276, "clarifi": [276, 410, 475, 486], "unenforcedconstraint": 276, "gettablelayout": [276, 396], "11462": 277, "11526": 277, "saniti": 277, "ignore_corrupted_statist": 277, "enable_stats_calcul": 278, "max_drivers_per_task": 278, "11667": 278, "targetresults": 278, "11072": [278, 399], "dialog": 278, "peform": 278, "lzop": [278, 348], "impala": [278, 434], "prealloc": 279, "getmoreresult": 279, "11757": [280, 414], "dynamic_schedule_for_grouped_execut": [280, 415], "discret": 280, "parse_presto_data_s": 280, "default_filter_factor_en": 280, "dropdown": 280, "micro": 282, "2x": 282, "roadmap": 282, "lai": [282, 301], "ground": 282, "hbase": 282, "data_1_year": 282, "data_1_week": 282, "grammar": [282, 305, 393], "unfortun": 282, "safer": 282, "orders_by_d": [283, 490, 491, 498, 501, 502], "readonlyconnectormetadata": 283, "citi": [284, 482, 509, 513], "akiaiosfodnn7exampl": 284, "wjalrxutnfemi": 284, "k7mdeng": 284, "bpxrficyexamplekei": 284, "inputformat": 284, "recordread": 284, "weird": 284, "readm": 285, "stopgap": 285, "semaphor": 285, "hivesplitsourc": 286, "gettabl": [287, 330, 350, 385, 399], "getschema": [287, 330], "getcatalog": 287, "anymor": 287, "synthet": [287, 297, 413, 414], "encourag": [287, 364], "reproduc": 287, "setnodemanag": 287, "unbalanc": 287, "anywher": [288, 520, 544], "ap": 288, "bear": 288, "anim": [288, 520], "banana": 288, "tear": 292, "timestamp_column": 293, "daylight": [293, 368, 414], "lo": 293, "angel": 293, "march": [293, 357], "9th": 293, "berlin": 293, "prestoconnect": [293, 305], "settimezoneid": 293, "spanish": 293, "enero": 293, "japanes": 293, "1\u6708": 293, "setlocal": 293, "inadvert": 293, "connectortyp": 293, "partitionresult": 293, "connectorpartitionresult": 293, "connectorsplit": 293, "splitsourc": 293, "canhandl": 293, "di": 293, "closeabl": 294, "filenam": [296, 361], "intertwin": 296, "refactor": [296, 302], "h2": 296, "proof": [296, 467], "training_data": 296, "validation_data": 296, "testraptordistributedqueri": 297, "bytecod": [297, 303, 317], "slowli": [297, 317], "dramat": 297, "groundwork": 301, "equijoin": 301, "approx_avg": 302, "bracket": 302, "recordcursorprovid": 302, "focu": [302, 468], "consolid": 302, "connectoroutputhandleresolv": 302, "connectorhandleresolv": 302, "connectorrecordsinkprovid": 302, "25m": 303, "byqueri": 303, "accessor": [304, 373], "wors": 304, "caution": 304, "abortedexcept": 304, "ariti": 304, "hashaggreg": 304, "setsessionproperti": 305, "preregistr": 305, "storage_format": 305, "__hive_default_partition__": [305, 307], "force_local_schedul": [306, 462], "optimized_reader_en": [306, 307, 515], "securityexcept": 315, "placement": [317, 476], "unimpl": 317, "sqlexcept": 317, "unsupportedoperationexcept": [317, 396], "unparenthes": 317, "getcolumntyp": [317, 375], "lazyblockload": [318, 350], "cm": [320, 321], "test_usernam": 320, "test_password": 320, "control_usernam": 320, "control_password": 320, "iop": 321, "substag": 322, "statementresourc": 323, "variablewidthblockencod": 325, "unreferenc": 325, "precomput": 326, "facebook": [327, 385], "prestodriv": [327, 385], "node_id": 327, "11930": 327, "parquet_fail_with_corrupted_statist": [327, 354], "facilit": 327, "connectorbucketnodemap": 327, "temporary_staging_directory_en": [328, 445], "temporary_staging_directory_path": [328, 445], "supplementari": 330, "getschemanam": 330, "gettablenam": 330, "schematableprefix": 330, "listview": 330, "288": 331, "271": 331, "226": [331, 505], "249": 331, "239": 331, "228": 331, "267": 331, "283": 331, "289": 331, "286": 331, "delta_binary_pack": 331, "240": 331, "223": 331, "229": 331, "252": 331, "299": 332, "247": 332, "273": 332, "analyze_properti": [332, 482], "rle_dictionari": 332, "281": 332, "437": 333, "468": 333, "433": 333, "too_many_stag": 333, "432": 333, "431": 333, "452": 333, "465": 333, "454": 333, "useslegacytablelayout": 333, "461": 333, "543": 334, "467": 334, "551": 334, "steal": 334, "561": 334, "542": 334, "508": 334, "580": 334, "482": 334, "longdecimaltyp": 334, "ipaddresstyp": 334, "int128arrayblock": 334, "fixedwithblock": 334, "492": 334, "fixedwidthblock": 334, "541": 334, "610": 335, "622": 335, "507": 335, "615": 335, "619": 335, "555": 335, "586": 335, "589": 335, "latin": 335, "573": 335, "guard": [335, 379], "438": 335, "matcher": [336, 468], "661": 336, "652": 336, "gss": [336, 468], "645": 336, "642": 336, "664": 336, "268": 336, "614": 336, "592": 336, "636": [336, 469], "537": 337, "696": 337, "665": 337, "late": [337, 352, 354, 361, 381, 383], "work_processor_pipelin": [337, 354], "602": 337, "666": 337, "692": 337, "241": 337, "568": 337, "698": 337, "typo": 337, "550": 337, "733": 338, "725": 338, "764": 338, "773": 338, "732": 338, "714": 338, "657": 338, "596": 338, "728": 338, "table_com": [338, 386, 450, 489, 490], "531": 338, "795": 339, "559": 339, "805": 339, "838": 339, "743": 339, "730": 339, "731": 339, "skip_redundant_sort": [339, 450], "818": 339, "767": 339, "441": 339, "786": 339, "672": 339, "790": 339, "788": 339, "8u161": [339, 342], "crypto": 339, "779": 339, "785": 339, "791": 339, "822": 339, "create_empty_bucket_fil": 339, "824": 339, "640": 339, "tablesampl": [339, 359, 368, 385, 397], "753": 339, "843": 340, "864": 340, "778": 340, "854": 340, "939": 341, "944": 341, "947": 341, "904": 341, "888": 341, "860": 341, "878": 341, "894": 341, "derbi": 341, "833": 341, "serdeproperti": 341, "898": 341, "921": 341, "910": 341, "891": 341, "textfile_skip_header_line_count": [341, 348], "textfile_skip_footer_line_count": [341, 348], "845": 341, "534": 341, "924": 341, "_int4": 341, "659": 341, "887": 341, "918": 341, "908": 341, "makecompatiblepartit": 341, "689": 341, "applydelet": 341, "executedelet": 341, "893": 341, "958": 342, "ti": [342, 520, 523], "832": 342, "680": 342, "548": 342, "983": 342, "871": 342, "984": 342, "emrf": 342, "935": 342, "914": 342, "967": 342, "thrifthivemetastorecli": 342, "946": 342, "933": 342, "976": 342, "getview": 342, "bulk": [342, 407, 430], "1056": 343, "901": 343, "1042": 343, "1016": 343, "972": 343, "920": 343, "997": 343, "collect_column_statistics_on_writ": 343, "1061": 343, "1011": 343, "1005": 343, "676": 343, "1151": 344, "1045": 344, "1168": 344, "workprocessor": 344, "1075": 344, "1155": 344, "1119": 344, "1164": 344, "tooltip": 344, "1113": 344, "1115": 344, "1139": 344, "remoteclientaddress": 344, "querycontext": 344, "dispatch": [344, 361], "1033": 344, "geturl": 344, "1211": 344, "1199": 344, "1152": 344, "1093": 344, "1086": 344, "1094": 344, "1101": 344, "1015": 344, "connectorpagesourceprovid": [344, 359, 385], "1095": 344, "1180": 345, "1071": 345, "999": 345, "1306": 345, "1270": 345, "1128": 345, "distributed_planning_time_m": 345, "1084": 345, "qostanai": 345, "1221": 345, "query_max_total_memory_per_nod": 345, "1212": 345, "1030": 345, "planning_time_m": 345, "analysis_time_m": 345, "misnom": 345, "1295": 345, "1079": 345, "1298": 345, "1336": 345, "1321": 345, "1232": 345, "1200": 345, "1241": 345, "1127": 345, "1347": 345, "1124": 345, "1274": 345, "1023": 345, "1262": 345, "1328": 345, "isload": 345, "1216": 345, "connectorsecuritycontext": 345, "systemsecuritycontext": 345, "1251": [345, 520], "1510": 346, "1407": 346, "1345": 346, "1401": 346, "1368": 346, "1534": 346, "1453": 346, "1329": 346, "1308": 346, "1390": 346, "806": 346, "1415": 346, "1431": 346, "1524": 346, "1244": 346, "1522": 346, "10491": 346, "1551": 346, "1153": 346, "1442": 346, "1371": 346, "1484": 346, "1192": 346, "1443": 346, "1343": 346, "1499": 346, "hadoop_default": 346, "1397": [346, 520], "1363": 346, "1473": 346, "1439": 346, "1428": 346, "1346": 346, "1218": 346, "1191": 347, "1589": 347, "1515": 347, "1516": 347, "464": 347, "476": 347, "1593": 347, "1612": 347, "1640": 347, "highlight": [347, 512], "1380": 347, "1625": [347, 520], "1469": 347, "1598": 347, "1527": 347, "1639": 347, "1662": 347, "1624": 347, "1543": 348, "1686": 348, "1654": 348, "1714": 348, "1698": 348, "1090": 348, "1701": 348, "538": 348, "1668": 348, "682": 348, "1693": 348, "1572": 348, "querystatist": [348, 354, 442], "getwalltim": 348, "1719": 348, "1673": 349, "1767": 349, "1796": 350, "1732": 350, "1785": 350, "1488": 350, "unread": 350, "1725": 350, "1783": 350, "1793": 350, "1794": 350, "760": 350, "978": 350, "1820": 350, "1740": 350, "1761": 350, "1744": 350, "1825": 351, "1691": 351, "1888": 351, "1867": 351, "1875": 351, "1916": 351, "1517": 351, "1842": 351, "1846": 351, "1894": 351, "1844": 351, "1914": 351, "1891": 351, "1952": 352, "1985": 352, "1963": 352, "1688": 352, "1692": 352, "1256": 352, "1811": 352, "1827": 352, "1980": 352, "1958": 352, "1955": 352, "1907": 352, "1243": [352, 520], "1949": 352, "1925": 352, "1930": 352, "2097": 353, "2047": 353, "2055": 353, "2044": 353, "2057": 353, "2032": 353, "1664": 353, "973": 353, "1465": 353, "1922": 353, "1929": 353, "2024": 353, "destroi": 353, "1840": 353, "2144": 354, "2145": 354, "2315": 354, "2267": 354, "1060": 354, "error_typ": 354, "error_cod": 354, "2249": 354, "late_materi": 354, "2275": 354, "2178": 354, "2099": 354, "556": 354, "1921": 354, "576": 354, "2129": 354, "2154": 354, "2189": 354, "2248": 354, "2131": 354, "2252": 354, "2253": 354, "2156": 354, "1182": 354, "gtid": [354, 369], "2251": 354, "2310": 354, "2118": 354, "2247": 354, "2313": 355, "2320": 355, "2363": 355, "2061": 355, "2136": 355, "2370": 355, "2356": 355, "referr": [355, 469], "2354": 355, "2339": 355, "setautocommit": 355, "2338": 355, "2184": 355, "2395": 355, "1746": 355, "2331": 355, "2303": 355, "2217": 355, "2324": 355, "2427": 355, "1874": 355, "2290": 355, "2088": 355, "2452": 356, "2582": 356, "2571": 356, "2130": 356, "2444": 356, "2488": 356, "2456": 356, "2527": 356, "2528": 356, "extra_info": 356, "2306": 356, "1917": 356, "appar": 356, "2399": 356, "2471": 356, "2496": 356, "2451": 356, "2463": 356, "1812": 356, "2277": 356, "2538": 356, "2441": 356, "2561": 356, "2549": 356, "2629": 357, "2560": 357, "2659": 357, "2685": 357, "2575": 357, "2675": 357, "2593": 357, "2722": 357, "2788": 357, "2557": 357, "2535": 357, "required_workers_count": 357, "required_workers_max_wait_tim": 357, "2484": 357, "physical_input_byt": 357, "2803": 357, "2764": 357, "2768": 357, "relax": 357, "dpresto": 357, "java8": 357, "2751": 357, "aarch64": 357, "2809": 357, "2215": 357, "2202": 357, "2653": 357, "2805": 357, "2633": 357, "settimestamp": 357, "2732": 357, "2780": 357, "2468": 357, "2455": 357, "2720": 357, "2734": 357, "2358": 357, "1748": 357, "2657": 357, "ignore_absent_partit": 357, "2555": 357, "2669": 357, "wasb": 357, "gen1": 357, "2494": 357, "2715": 357, "2692": 357, "2762": 357, "2781": 357, "2601": 357, "1185": 357, "2706": 357, "checkcanimpersonateus": 357, "2648": 358, "2861": 358, "2842": 358, "2612": 358, "2851": 358, "deepli": 358, "2968": 358, "2888": 358, "2853": 358, "2859": 358, "2862": 358, "2793": 358, "2725": 358, "3023": 358, "2673": 358, "2532": 358, "2719": 358, "2709": 358, "2712": 358, "2213": 358, "2925": 358, "getmetadata": [358, 398], "2770": 358, "clipboard": 358, "2865": 358, "2872": 358, "2810": 358, "2908": 358, "2784": 358, "2873": 358, "2755": 358, "2858": 358, "2961": 358, "3008": 358, "2905": 358, "2984": 358, "3024": 358, "1870": 358, "2068": 358, "1591": 358, "2790": 358, "2823": 358, "2334": 358, "tier": 358, "3032": 358, "1781": 358, "2956": 358, "2877": 358, "getrowfilt": 358, "getcolumnmask": 358, "1480": 358, "2924": 358, "3149": 359, "3191": 359, "3250": 359, "2981": 359, "3231": 359, "3215": 359, "2743": 359, "3128": 359, "3066": 359, "3205": 359, "3217": 359, "3206": 359, "2665": 359, "3290": 359, "3188": 359, "3131": 359, "3347": 359, "3381": 359, "3265": 359, "2730": 359, "3175": 359, "cryptic": 359, "3366": 359, "3126": 359, "partition_use_column_nam": [359, 379], "2933": 359, "3239": 359, "2679": 359, "3209": 359, "alluxio": 359, "2116": 359, "3144": 359, "3351": 359, "3170": 359, "2963": 359, "3274": 359, "3208": 359, "3089": 359, "3234": 359, "3284": 359, "3183": 359, "3246": 359, "3166": 359, "createpagesourc": [359, 385], "3255": 359, "3218": 360, "3408": 360, "3380": 360, "3424": 360, "3625": 360, "3407": 360, "3413": 360, "3432": 360, "cost_estimation_worker_count": 360, "2705": 360, "3438": 360, "2997": 360, "3099": 360, "3392": 360, "2799": 360, "3595": 360, "3523": 360, "8u252": 360, "3444": 360, "3376": 360, "3499": 360, "3478": 360, "3447": 360, "3517": 360, "3390": 360, "3455": 360, "3557": 360, "3571": 360, "warmup": 360, "3591": 360, "3429": 360, "asleep": 360, "3490": 360, "3440": 360, "bson": 360, "2935": 360, "3355": 360, "3479": 360, "3745": 361, "3577": 361, "3587": 361, "3829": 361, "dst": [361, 386], "3846": 361, "3850": 361, "3694": 361, "3614": 361, "3618": 361, "3632": 361, "2672": 361, "1848": 361, "3724": 361, "1959": 361, "2028": 361, "2321": 361, "7239": [361, 381], "3714": 361, "3804": 361, "3803": 361, "3311": 361, "3433": 361, "1323": 361, "1620": 361, "3718": 361, "3512": 361, "3735": 361, "3702": 361, "3755": 361, "3819": 361, "3749": 361, "3729": 361, "3810": 361, "3772": 361, "evict": [361, 368], "3624": 361, "3581": 361, "3611": 361, "deplet": 361, "3820": 361, "3799": 361, "3771": 361, "3743": 361, "3453": 361, "3629": 361, "3883": 362, "3848": 362, "3956": 362, "3599": 362, "3456": 362, "3922": 362, "3783": 362, "3949": 362, "role_authorization_descriptor": [362, 422], "3535": 362, "3766": 362, "3920": 362, "3673": 362, "2469": 362, "3431": 362, "3874": 362, "3913": 362, "3840": 362, "uniontyp": [362, 432, 433], "3483": 362, "3937": 362, "1072": 362, "3953": 362, "3697": 362, "4036": 363, "4050": 363, "3965": 363, "3951": 363, "4040": 363, "3574": 363, "vulner": 364, "malici": [364, 475], "attack": [364, 379], "escal": 364, "everyon": [364, 479, 480, 488, 494, 507, 517], "4120": 364, "4129": 364, "4023": 364, "3533": 364, "3990": 364, "3947": 364, "4151": 364, "4128": 364, "3522": 364, "4080": 364, "4192": 364, "4199": 364, "4082": 364, "4165": 364, "separatorchar": 364, "quotechar": 364, "escapechar": 364, "3891": 364, "1375": 364, "4104": 364, "4156": 364, "4056": 364, "4143": 364, "parquet_optimized_writer_en": [364, 424, 450], "3400": 364, "aliyunoss": 364, "4213": 364, "21164": 364, "4049": 364, "3881": 364, "4251": 365, "4322": 365, "4305": 365, "4370": 365, "4119": 365, "4272": 365, "4304": 365, "4228": 365, "4262": 365, "2516": 365, "omit_datetime_type_precis": 365, "4349": 365, "4377": 365, "4144": 365, "3928": 365, "getclientinfoproperti": 365, "4318": 365, "4341": 365, "4200": 365, "4266": 365, "2296": 365, "3266": 365, "3987": 365, "4280": 365, "4240": 365, "3582": 365, "4183": 365, "4187": 365, "3784": 365, "3770": 365, "3834": 365, "3425": 366, "4075": 366, "4308": 366, "4380": 366, "4127": 366, "4423": 366, "2960": 366, "4233": 366, "4408": 366, "4475": 366, "4425": 366, "nnnn": 366, "4490": 366, "9999": 366, "4364": 366, "4411": 366, "getdat": 366, "gettim": 366, "gettimestamp": 366, "usesessiontimezon": [366, 373], "4017": 366, "4312": 366, "4443": 366, "4485": 366, "4457": 366, "4378": 366, "4267": 366, "4418": 366, "4287": 366, "grouped_execut": [366, 415], "3715": 366, "cosmo": 366, "4415": 366, "4430": 366, "3251": 366, "connectorsplitmanag": [366, 367, 387], "getsplit": [366, 367, 387], "4224": 366, "4529": 367, "4601": 367, "4250": 367, "740": 367, "4011": 367, "4651": 367, "4511": 367, "4388": 367, "4290": 367, "4123": 367, "4669": 367, "4730": 367, "4609": 367, "4635": 367, "unprivileg": 367, "4472": 367, "4563": 367, "1900": [367, 450], "4535": 367, "4069": 367, "2293": 367, "4516": 367, "4611": 367, "4432": 367, "4458": 367, "4513": 367, "4705": 367, "4552": 367, "3967": 367, "4417": 367, "4477": 367, "4664": 367, "4483": 367, "4617": 367, "4560": 367, "4362": 367, "4592": 367, "4249": 367, "4508": 367, "4381": 368, "4905": 368, "4344": 368, "4741": 368, "4867": 368, "4877": 368, "optimizer_rule_stat": [368, 389], "4659": 368, "4440": 368, "4294": 368, "4616": 368, "4924": 368, "polit": 368, "5027": 368, "legacy_timestamp": 368, "4799": 368, "4866": 368, "4778": 368, "4872": 368, "4824": 368, "4946": 368, "4847": 368, "4787": 368, "4994": 368, "5009": 368, "5074": 368, "4890": 368, "4372": 368, "4854": 368, "4804": 368, "4736": 368, "4962": 368, "getstat": 368, "4957": 368, "hourli": 368, "4968": 368, "2269": 368, "1147": 368, "deeparch": 368, "5002": 368, "3434": 368, "4768": 368, "4931": 368, "4759": 368, "4938": 368, "underw": [368, 397], "4622": 368, "4623": 368, "3580": 368, "4753": 368, "4683": 368, "4752": 368, "4138": 368, "4779": 368, "internation": 368, "4775": 368, "momentari": 368, "4947": 368, "4955": 368, "4918": 368, "4670": 368, "4725": 368, "4801": 368, "4723": 368, "4802": 368, "4111": 368, "4981": 368, "prolept": 368, "gregorian": 368, "islegacytimestamp": 368, "3414": 368, "5048": 369, "4886": 369, "5017": 369, "4988": 369, "5141": 369, "5237": 369, "5262": 369, "5087": 369, "5125": 369, "5269": 369, "2536": 369, "5052": 369, "4953": 369, "4999": 369, "dynamic_filtering_probe_blocking_timeout": 369, "4991": 369, "inherit": 369, "5187": 369, "5060": 369, "5142": 369, "5273": 369, "4462": 369, "4743": 369, "nonsens": 369, "4995": 369, "5124": 369, "5105": 369, "4846": 369, "4139": 369, "isawait": 369, "isblock": 369, "5043": 369, "4685": 369, "yearli": 370, "5298": 370, "5295": 370, "5299": 370, "5054": 371, "5046": 371, "5158": 371, "5081": 371, "5340": 371, "5341": 371, "4345": 371, "5375": 371, "5464": 371, "5276": 371, "5289": 371, "5310": 371, "5371": 371, "4760": 371, "5462": 371, "5039": 371, "5028": 371, "5416": 371, "5472": 371, "5430": 371, "5260": 371, "5364": 371, "4805": 371, "3601": 371, "5307": 371, "5342": 371, "5327": 371, "4680": 372, "5539": 372, "5555": 372, "5330": 372, "5551": 372, "5529": 372, "5461": 372, "5557": 372, "5181": 372, "5419": 372, "5460": 372, "5620": 372, "4739": 372, "5402": 372, "5365": 372, "5483": 372, "4661": 372, "5621": 372, "5443": 372, "5512": 372, "5615": 372, "5495": 372, "semicolon": 372, "5261": 372, "5397": 372, "5596": 372, "609": 373, "5713": 373, "5668": 373, "4396": 373, "5717": 373, "5683": 373, "2674": 373, "5582": 373, "5851": 373, "5787": 373, "arrayagg": 373, "multimapagg": 373, "4581": 373, "5696": 373, "5736": 373, "5685": 373, "5660": 373, "5640": 373, "5647": 373, "5846": 373, "5815": 373, "5702": 373, "5392": 373, "5759": 373, "unambigu": 373, "5509": 373, "5610": 373, "4521": 373, "getrow": 373, "5769": 373, "leftov": 373, "5782": 373, "5618": 373, "5047": 373, "5807": 373, "5283": 373, "5693": 373, "5634": 373, "3077": 373, "5858": 373, "misbehav": 373, "5795": 373, "5785": 373, "5484": 373, "5698": 373, "4832": 373, "5751": 373, "3691": 373, "5672": 373, "5399": 373, "5720": 373, "5543": 373, "5522": 373, "connectornodepartitioningprovid": [373, 432], "5690": 373, "5643": 373, "5789": 374, "2152": 374, "5593": 374, "subnet": 374, "4862": 374, "5905": 374, "5949": 374, "5818": 374, "rewrite_filtering_semi_join_to_inner_join": 374, "5954": 374, "5731": 374, "3398": 374, "8206955": 374, "5957": 374, "5874": 374, "5945": 374, "addbatch": 374, "clearbatch": 374, "executebatch": 374, "5507": 374, "5881": 374, "cento": [374, 378, 397], "5920": 374, "1829": 374, "1489": 374, "3638": 374, "5562": 374, "5594": 374, "5870": 374, "6057": 374, "4112": 374, "5792": 374, "5872": 374, "5904": 375, "3109": 375, "6126": 375, "6072": 375, "6176": 375, "6115": 375, "6141": 375, "6145": 375, "6318": 375, "6103": 375, "6294": 375, "5355": 375, "6114": 375, "zoneddatetim": 375, "locald": 375, "setobject": [375, 377], "6301": 375, "6300": 375, "4588": 375, "getstr": 375, "2de82bf8": 375, "6247": 375, "6307": 375, "data_typ": [375, 480, 489], "999999999500": 375, "6147": 375, "6204": 375, "getarrai": 375, "6048": 375, "miscategor": 375, "6251": 375, "6240": 375, "fake": 375, "4587": 375, "6119": 375, "6238": 375, "3869": 375, "3712": 375, "6201": 375, "6101": 375, "6156": 375, "validate_bucket": 375, "6012": 375, "legacy_hive_view_transl": [375, 407], "6195": 375, "6006": 375, "6158": 375, "6213": 375, "4932": 375, "6186": 375, "6074": 375, "ineffect": 375, "6081": 375, "6167": 375, "filtercolumn": 375, "6084": 375, "6453": 377, "6283": 377, "6349": 377, "6404": 377, "6017": 377, "6380": 377, "6216": 377, "offsettim": 377, "6352": 377, "5635": 377, "4059": 377, "6368": 377, "6137": 377, "4693": 377, "6075": 377, "getpicosecond": 377, "6354": 377, "azul": 378, "zulu": 378, "trinoconnect": 378, "trinos3filesystem": [378, 413], "prestoexcept": 378, "prestoprincip": 378, "trinoprincip": 378, "prestowarn": 378, "651": 379, "5861": 379, "6582": 379, "tzdata": 379, "2020d": 379, "6660": 379, "plan_with_table_node_partit": 379, "6811": 379, "6333": 379, "6951": [379, 380], "5981": 379, "6544": 379, "6472": 379, "6350": 379, "spatial_partitioning_table_nam": 379, "6587": 379, "6835": 379, "6600": 379, "6625": 379, "6428": 379, "6580": 379, "audienc": [379, 388, 467, 470], "aud": [379, 467], "6501": 379, "replai": 379, "nonc": 379, "openid": [379, 408], "6576": 379, "6549": 379, "6497": 379, "6567": 379, "3687": 379, "6706": 379, "6479": 379, "5195": 379, "6452": 379, "6503": 379, "6802": 379, "6760": 379, "6849": 379, "6650": 379, "5199": 379, "use_file_size_from_metadata": 379, "6369": 379, "6520": 379, "6442": 379, "6667": 379, "6673": 379, "6731": 379, "6465": 379, "6274": 379, "6654": 379, "6464": 379, "6752": 379, "4500": 380, "6326": 380, "6925": 380, "6355": 380, "6897": 380, "7015": 380, "6839": 380, "6896": 380, "6913": 380, "6972": 380, "7014": 380, "6958": 380, "6703": 380, "6838": 380, "6659": 380, "6635": 380, "3767": 380, "6748": 380, "6720": 380, "6178": 380, "6924": 380, "6866": 380, "6853": 380, "6992": 380, "6370": 380, "7180": 380, "6902": 380, "6907": 380, "7048": 380, "6852": 380, "6746": 380, "6671": 380, "6865": 380, "7169": 380, "6847": 380, "3645": 380, "6893": 380, "6637": 380, "6753": 380, "7069": 380, "6874": 380, "getfullyloadedblock": 380, "connectorawarenodemanag": 380, "getworkernod": 380, "7007": 380, "6189": 380, "6679": 380, "joinstatist": 380, "7000": 380, "7090": 381, "uncondition": 381, "6920": 381, "7345": 381, "subsum": 381, "7302": 381, "7342": 381, "6784": 381, "7172": 381, "7054": 381, "7012": 381, "6848": 381, "7178": 381, "7059": 381, "7259": 381, "7233": 381, "7049": 381, "7179": 381, "7326": 381, "6909": 381, "7170": 381, "7314": 381, "7236": 381, "7122": 381, "7324": 381, "7330": 382, "7425": 382, "7465": 382, "7257": 382, "7427": 382, "7346": 382, "7331": 382, "7450": 382, "7460": 382, "7151": 382, "assumeliteralnamesinmetadatacallsfornonconformingcli": [382, 419], "7438": 382, "7135": 382, "7242": 382, "7271": 382, "7511": 382, "6975": 382, "7490": 382, "connectormaterializedviewdefinit": [382, 383, 384, 456], "storaget": 382, "7319": 382, "6111": 383, "4022": 383, "materialized_view_properti": [383, 486], "7615": 383, "7213": 383, "7678": 383, "7028": 383, "7263": 383, "7556": 383, "7644": 383, "7695": 383, "7351": 383, "7455": 383, "7629": 383, "7792": 383, "7707": 383, "7645": 383, "7706": 383, "7309": 383, "7543": 383, "7628": 383, "7433": 383, "7754": 383, "7350": 383, "7621": 383, "_orc_acid_vers": 383, "7579": 383, "7622": 383, "7509": 383, "1592": 383, "7336": 383, "7711": 383, "7790": 383, "6929": 383, "7039": 383, "7491": 383, "6337": 383, "7306": 383, "discontinu": 383, "life": [383, 397], "7676": 383, "read_committed_snapshot": 383, "7548": 383, "1583": 383, "7634": 383, "7489": 383, "getmaterializedviewpropertymanag": 383, "checkcancreatematerializedview": [383, 398], "checkcandropmaterializedview": 383, "checkcanrefreshmaterializedview": 383, "7773": 384, "7863": 384, "7251": 384, "7824": 384, "1878": 384, "3536": 384, "7874": 384, "7840": 384, "7947": 384, "7797": 384, "7941": 384, "8017": 384, "7618": 384, "7936": 384, "sni": 384, "8007": 384, "7526": 384, "7819": 384, "7632": 384, "7830": 384, "7784": 384, "7434": 384, "7097": 384, "7838": 384, "7423": 384, "1609": 384, "7738": 384, "7862": 384, "7853": 384, "7161": 384, "posgresql": 384, "7841": 384, "7762": 384, "7980": 384, "8026": 385, "8094": 385, "6634": 385, "8149": 385, "7723": 385, "8023": 385, "8042": 385, "8120": 385, "8151": 385, "8095": 385, "8077": 385, "6066": 385, "redirectt": 385, "7606": 385, "listtablecolumn": 385, "8103": 386, "recognit": [386, 392, 440, 520], "8141": 386, "8112": 386, "8280": 386, "8446": 386, "8295": 386, "8279": 386, "8113": 386, "8327": 386, "8184": 386, "8311": 386, "8248": 386, "8247": 386, "7027": 386, "8134": 386, "4196": 386, "8397": 386, "8236": 386, "8183": 386, "3605": 386, "8158": 386, "8166": 386, "7635": 386, "8390": 386, "8373": 386, "8259": 386, "8208": 386, "8320": 386, "8153": 386, "8264": 386, "6910": 386, "7413": 386, "5449": 386, "listmaterializedview": 386, "getmaterializedview": 386, "7960": 386, "driven": 386, "8355": 386, "8405": 386, "8371": 386, "8554": 387, "8615": 387, "8660": 387, "8535": 387, "8598": 387, "8600": 387, "8624": 387, "8460": 387, "5183": 387, "8500": 387, "8639": 387, "7954": 387, "8529": 387, "write_batch_s": 387, "8434": 387, "8496": 387, "8714": 387, "8268": 387, "8452": 387, "8540": [387, 520], "8450": 387, "7502": 387, "8616": 387, "8647": 387, "5201": 387, "8676": 387, "8689": 387, "6287": 387, "8559": 387, "8352": 387, "3134": 387, "8171": 387, "8408": 387, "8569": 387, "getcompletedposit": 387, "8524": 387, "8056": 387, "8736": 388, "materialized_view": [388, 389, 392, 486], "8796": 388, "8683": 388, "8878": 388, "moder": [388, 487], "8833": 388, "8845": 388, "8984": 388, "8973": 388, "8869": 388, "oidc": [388, 467, 470], "8641": 388, "getparametermetadata": 388, "2978": 388, "8978": 388, "8999": 388, "3763": 388, "8652": 388, "7320": 388, "7308": 388, "8805": 388, "5977": 388, "8789": 388, "from_utc_timestamp": 388, "8502": 388, "pmod": 388, "8935": 388, "8954": 388, "8839": 388, "7349": 388, "8730": 388, "8573": 388, "interfer": [388, 419], "6063": 388, "8868": 388, "8824": 388, "8832": 388, "4140": 388, "8617": 388, "8307": 388, "4835": 389, "8974": 389, "8967": 389, "9194": 389, "9102": 389, "connectormetr": 389, "9156": 389, "9050": 389, "9265": 389, "9109": 389, "9066": 389, "9036": 389, "9150": 389, "9153": 389, "8700": 389, "6200": 389, "9250": 389, "9171": 389, "9087": 389, "8762": 389, "9214": 389, "noarch": 389, "9187": 389, "9164": 389, "9052": 389, "9005": 389, "9060": 389, "9086": 389, "9122": 389, "9096": 389, "9139": 389, "9188": 389, "6210": 389, "8986": 389, "9100": 389, "7991": 389, "query_partition_filter_required_schema": 389, "9106": 389, "9148": 389, "9161": 389, "9070": 389, "arrayindexoutofboundsexcept": [389, 391], "5518": 389, "9114": 389, "9297": 389, "6663": 389, "9124": 389, "8822": 389, "9180": 389, "9133": 389, "7990": 389, "issinglestatementwritesonli": 389, "8872": 389, "8821": 390, "9510": 390, "9402": 390, "9462": 390, "9270": 390, "9275": 390, "9327": 390, "9453": 390, "8985": 390, "9319": 390, "9396": 390, "9371": 390, "9496": 390, "9460": 390, "9391": 390, "8570": 390, "scaled_float": 390, "9358": 390, "9370": 390, "9234": 390, "9338": 390, "9097": 390, "9354": 390, "9469": 390, "9295": 390, "9516": 390, "9123": 390, "9326": 390, "9339": 390, "2084": 390, "9373": 390, "9329": 390, "9387": 390, "9492": 391, "9514": 391, "9640": 391, "9610": 391, "9744": 391, "8650": 391, "unwrap_cast": 391, "9550": 391, "9805": 391, "9673": 391, "8286": 391, "9696": 391, "9710": 391, "8892": 391, "4826": 391, "5231": 391, "9552": 391, "9539": 391, "9530": 391, "9665": 391, "9559": 391, "9793": 391, "9633": 391, "9515": 391, "6377": 391, "9557": 391, "9243": 391, "9587": 391, "9569": 391, "4115": 391, "9704": 391, "9583": 391, "9703": 391, "9757": 391, "9714": 391, "9725": 391, "9549": 391, "9644": 391, "8921": 392, "8738": 392, "travel": [392, 433], "8773": 392, "9723": 392, "9654": 392, "9858": 392, "10133": 392, "8856": 392, "9961": 392, "9564": 392, "9307": 392, "9851": 392, "10088": 392, "9888": 392, "9913": 392, "9874": 392, "10056": 392, "552": [392, 393, 396], "9822": 392, "submiss": 392, "9962": 392, "9615": 392, "9920": 392, "10069": 392, "9811": 392, "9821": 392, "8987": 392, "8798": 392, "9882": 392, "9954": 392, "9998": 392, "9892": 392, "9939": 392, "9929": 392, "9715": 392, "9738": 392, "9414": 392, "9866": 392, "9869": 392, "9979": 392, "10048": 392, "9914": 392, "10070": 392, "10094": 392, "10095": 392, "10120": 392, "9971": 392, "10106": 392, "9890": 392, "10076": 392, "9918": 392, "9959": 392, "10058": 392, "9974": 392, "conflict": [392, 406, 450], "9519": 392, "8729": 392, "9309": 392, "bigger": 392, "9826": 392, "derefer": [392, 399], "8129": 392, "9810": 392, "9264": 392, "9921": 392, "7250": 392, "seed": [392, 432, 466], "9819": 392, "9795": 392, "9151": 392, "9098": 392, "9746": 392, "7608": 392, "retry_polici": [393, 396], "9361": 393, "10206": 393, "10205": 393, "10246": 393, "10239": 393, "10225": 393, "contend": 393, "10249": 393, "10183": 393, "8383": 393, "10173": 393, "10180": 393, "9755": 393, "10054": 393, "10143": 393, "10101": 393, "lineag": 394, "10272": 394, "10186": 394, "10064": 394, "10274": 394, "10347": 394, "10321": 394, "10344": 394, "10141": 394, "9991": 394, "10351": 394, "land": 394, "logout": [394, 470], "10299": 394, "10251": 394, "10146": 394, "10252": 394, "9767": 394, "classnotfoundexcept": [394, 395, 413], "10341": 394, "query_max_total_memory_per_task": 395, "10308": 395, "10350": 395, "10455": 395, "10051": 395, "10475": 395, "10360": 395, "10472": 395, "10394": 395, "10447": 395, "9949": 395, "9732": 395, "10493": 395, "10459": 395, "10332": 395, "10380": 395, "10539": 395, "10400": 395, "carrier": 395, "10408": 395, "kanton": 396, "10679": 396, "succinct": 396, "10576": 396, "10566": 396, "9812": 396, "10591": 396, "10574": 396, "9613": 396, "9818": 396, "10660": 396, "10656": 396, "10732": 396, "10599": 396, "10738": 396, "claim": [396, 470], "10262": 396, "10584": 396, "10632": 396, "10697": 396, "10564": 396, "10703": 396, "10558": 396, "10641": 396, "7098": 396, "map_string_as_varchar": 396, "10601": 396, "10650": 396, "10544": 396, "10512": 396, "3939": 396, "10385": 396, "10735": 396, "10401": 396, "8693": 396, "10441": 396, "4108": 396, "10722": 396, "10646": 396, "10485": 396, "9149": 396, "10710": 396, "10480": 396, "10497": 396, "9938": 396, "9340": 396, "9716": 396, "10570": 396, "10626": 396, "9531": 396, "8939": 396, "10331": 397, "10469": 397, "10322": 397, "10868": 397, "10839": 397, "10861": 397, "10764": 397, "10929": 397, "10811": 397, "10177": 397, "handler": 397, "10248": 397, "8819": 397, "redhat": 397, "ubi8": 397, "ubi": 397, "10866": 397, "10799": 397, "10541": 397, "588": 397, "10575": 397, "10820": 397, "10864": 397, "10770": 397, "9749": 397, "1899": 397, "10749": 397, "connetormetadata": 397, "settableproperti": 397, "connectornewtablelayout": 397, "gethandleresolv": 397, "10858": 397, "10872": 397, "appendto": 397, "10602": 397, "10996": 398, "10914": 398, "10923": 398, "10532": 398, "10985": 398, "10954": 398, "10946": 398, "10939": 398, "10975": 398, "11027": 398, "10714": 398, "10815": 398, "10910": 398, "11035": 398, "10622": 398, "10957": 398, "10940": 398, "10263": 398, "9482": 398, "checkcancreatet": 398, "11238": 399, "11098": 399, "11170": 399, "6677": 399, "11030": 399, "11080": 399, "11171": 399, "11198": 399, "10925": 399, "10413": 399, "11063": 399, "11060": 399, "11068": 399, "11055": 399, "storagedescriptor": 399, "11092": 399, "11101": 399, "11090": 399, "10621": 399, "11089": 399, "11143": 399, "evolv": 399, "11091": 399, "11104": 399, "10953": 399, "bindata": 399, "11122": 399, "10451": 399, "10904": 399, "10059": 399, "10898": 399, "7994": [399, 400], "10897": 400, "11146": 400, "11259": 400, "11304": 400, "11226": 400, "11270": 400, "11129": 400, "11353": 400, "10754": 400, "11272": 400, "11308": 400, "11255": 400, "11333": 400, "hive_exceeded_partition_limit": 400, "10215": 400, "11159": 400, "11062": 400, "10976": 400, "11318": 400, "11351": 400, "11045": 400, "11209": 400, "10734": 400, "11485": 401, "11050": 401, "11392": 401, "10432": 401, "11324": 401, "11315": 401, "11234": 401, "rtim": 401, "11440": 401, "getimportedkei": 401, "8708": 401, "getpropertyinfo": 401, "10624": 401, "vim": 401, "3377": 401, "11425": 401, "11083": 401, "11490": 401, "10169": 401, "11418": 401, "11443": 401, "10845": 401, "11409": 401, "11424": 401, "11457": 401, "11487": 401, "11086": 401, "11449": 401, "10669": 401, "11459": 401, "qualified_nam": 402, "unqualified_nam": 402, "11282": 402, "11302": 402, "11469": 402, "8777": 402, "11604": 402, "11581": 402, "11613": 402, "11620": 402, "11644": 402, "vacuum": [402, 432, 449], "11633": 402, "11562": 402, "11559": 402, "11512": 402, "11560": 402, "11576": 402, "11264": 402, "emul": 402, "11536": 402, "11679": 402, "11211": 402, "9137": 402, "10148": 402, "9781": 402, "11514": 402, "11532": 402, "11635": 402, "11510": 402, "11615": 402, "11623": 402, "11621": 402, "11277": 403, "11486": 403, "11518": 403, "560": 403, "11776": 403, "11758": 403, "11801": 403, "11070": 403, "11694": 403, "11653": 403, "10482": 403, "getasciistream": 403, "getbinarystream": 403, "11753": 403, "11350": 403, "11808": 403, "11606": 403, "11591": 403, "11600": 403, "11721": 403, "11792": 403, "11652": 403, "11675": 403, "11650": 403, "11499": 403, "11642": 403, "11356": 403, "11617": 403, "10709": 403, "11832": 403, "11638": 403, "11657": 403, "11750": 403, "11236": 404, "11870": 404, "11873": 404, "11869": 404, "11864": 404, "11520": 404, "11780": 404, "8743": 404, "11637": 404, "11439": 405, "11902": 405, "11918": 405, "11956": 405, "12047": 405, "11983": 405, "11944": 405, "11894": 405, "11768": 405, "11858": 405, "11946": 405, "11924": 405, "12056": 405, "8769": 405, "11880": 405, "expire_snapshot": [405, 419, 430], "10810": 405, "delete_orphan_fil": [405, 409], "11964": 405, "11976": 405, "11980": 405, "6789": 405, "hybrid": 405, "10125": 405, "gettablestatist": [405, 413], "11877": 405, "10046": 406, "11935": 406, "12095": 406, "11555": 406, "12099": 406, "12135": 406, "12144": 406, "12122": 406, "11886": 406, "12164": 406, "11713": 406, "12419": 406, "12055": 406, "11475": 406, "12108": 406, "11662": 406, "10614": 407, "12236": 407, "7729": 407, "11550": 407, "11749": 407, "9227": 407, "12238": 407, "12161": 407, "10462": 407, "11423": 407, "12205": 407, "12176": 407, "12211": 408, "1839": [408, 436], "12292": 408, "fault_tolerant_execution_partition_count": 408, "12263": 408, "12393": 408, "12199": 408, "9788": 408, "12187": 408, "12262": 408, "12203": 408, "activedirectori": 408, "12321": 408, "12290": 408, "12251": 408, "11671": 408, "12352": 408, "12210": 408, "12380": 408, "11216": 408, "12229": 408, "10717": 408, "12250": 408, "11885": 408, "12026": 408, "12225": 408, "12240": 408, "12279": 408, "12342": 408, "12360": 409, "12257": 409, "exceeded_task_descriptor_storage_capac": 409, "12478": 409, "12408": 409, "4124": 409, "12007": 409, "12371": 409, "12343": 409, "12513": 409, "originaltransact": 409, "currenttransact": 409, "12401": 409, "12525": 409, "12174": 409, "11732": 409, "remove_orphan_fil": [409, 428], "12468": 409, "12197": 409, "12411": 409, "12400": 409, "12417": 409, "12302": 409, "12531": 409, "12444": 410, "11289": 410, "12336": 410, "12564": 410, "12596": 410, "12452": 410, "12574": 410, "12598": 410, "12300": 410, "12351": 410, "12330": 410, "12544": 410, "12667": 411, "12681": 411, "9163": 411, "12682": 412, "3171": 412, "12721": 412, "appendonli": 412, "12635": 412, "12455": 412, "12715": 412, "11798": 412, "12639": 412, "12579": 412, "12671": 412, "10258": 412, "12504": 412, "12200": 412, "dbgen_vers": 412, "12673": 412, "12646": 413, "12615": 413, "12672": 413, "12475": 413, "12755": 413, "12621": 413, "12820": 413, "12658": 413, "12520": 413, "12804": 413, "12573": 413, "12674": 413, "12676": 413, "12319": 413, "12790": 413, "12542": 413, "12852": 413, "12784": 413, "12772": 413, "getstatisticscollectionmetadata": [413, 425], "connectoranalyzemetadata": 413, "gettablehandleforstatisticscollect": 413, "12388": 413, "12489": 413, "12910": 414, "12841": 414, "11619": 414, "12847": 414, "12814": 414, "12757": 414, "12883": 414, "timestamp_col": 414, "12362": 414, "12795": 414, "7905": 414, "12834": 414, "12860": 414, "allow_legacy_snapshot_syntax": 414, "10768": 414, "row_uuid": 414, "12915": 414, "9296": 414, "12218": 414, "12898": 414, "12694": 415, "12495": 415, "12926": 415, "12916": 415, "12664": 415, "12971": 415, "12675": 415, "11400": 415, "11263": 415, "12443": 415, "12857": 415, "11769": 415, "8919": 415, "12591": 415, "12617": 415, "12887": 415, "12874": 415, "12762": 416, "12618": 416, "13109": 416, "12951": 416, "13106": 416, "13064": 416, "13030": 416, "13086": 416, "raw_queri": [416, 425, 438], "12324": 416, "12892": 416, "13108": 416, "13084": 416, "13014": 417, "8349": 417, "10506": 417, "force_spilling_join": 417, "13123": 417, "13145": 417, "851": 417, "12223": 417, "13169": 417, "13081": 417, "13087": 417, "13124": 417, "13147": 417, "12633": 417, "10809": 417, "12510": 417, "12548": 418, "13213": 418, "13047": 418, "12392": 418, "13265": 418, "13217": 418, "13164": 418, "13105": 418, "13203": 418, "13208": 418, "13242": 418, "13181": 418, "12526": 418, "11305": 418, "9775": 418, "12895": 418, "13122": 418, "13035": 418, "13125": 418, "9935": 419, "9447": 419, "13315": 419, "12761": 419, "13132": 419, "12980": 419, "13316": 419, "13368": 419, "13418": 419, "invari": [419, 438, 448], "13353": 419, "13389": 419, "8404": 419, "13412": 419, "jst": 419, "tokyo": 419, "13179": 419, "13354": 419, "13082": 419, "12125": 419, "12785": 419, "13092": 419, "13399": 419, "12941": 419, "12786": 419, "13012": 419, "quotat": 419, "13074": 419, "13303": 419, "column_statist": 419, "13323": 419, "13422": 419, "13136": 419, "7933": 420, "13698": 420, "13616": 420, "13462": 420, "13655": 420, "13627": 420, "13168": 420, "13522": 420, "13542": 420, "13334": 420, "13436": 420, "13505": 420, "13521": 420, "13553": 420, "13663": 420, "13477": 420, "13606": 420, "13488": 420, "13502": 420, "13450": 420, "13504": 420, "13395": 420, "12226": 420, "13673": 420, "13576": 420, "12843": 420, "13582": 420, "13574": 420, "13167": 420, "13073": 420, "13427": 420, "13428": 420, "13307": 420, "13302": 420, "methodhandleutil": 420, "13245": 420, "12968": 421, "13432": 421, "13479": 421, "13435": 421, "12422": 421, "13565": 421, "13736": 421, "12502": [421, 437], "6868": 421, "6869": 421, "13785": 421, "9424": 421, "13810": 421, "12925": 421, "12530": 421, "13482": 421, "13787": 421, "13613": 421, "13410": 421, "13429": 421, "13742": 421, "13855": 422, "12512": 422, "13573": 422, "13934": 422, "13986": 422, "11341": 422, "13483": 422, "13839": 422, "12985": 422, "13111": 422, "writestatsasjson": 422, "writestatsasstruct": 422, "12031": 422, "13754": 422, "13715": 422, "13759": 422, "13936": 422, "13536": 422, "8740": 422, "like_pattern_function_nam": 422, "like_function_nam": 422, "standardfunct": 422, "13965": 422, "listallrolegr": 422, "12798": 423, "13944": [423, 427], "13882": 423, "14029": 423, "11116": 423, "14058": 423, "14068": 423, "13629": 423, "13714": 423, "undo": 423, "14108": 423, "14094": 423, "13246": 423, "14011": 423, "14064": 423, "13960": 423, "checkcangrantexecutefunctionprivileg": [423, 427], "13943": 423, "14164": 424, "14168": 424, "10398": 424, "recipi": [424, 467], "13442": 424, "13575": 424, "14176": 424, "experimental_parquet_optimized_writer_en": 424, "14137": 424, "14040": 424, "14206": 424, "2817": 424, "14077": 424, "14161": 424, "14198": 424, "13875": 424, "14145": 424, "14174": [424, 425], "14092": 424, "runlengthencodedblock": 424, "10228": 425, "14135": 425, "14268": 425, "14256": 425, "14306": 425, "14114": 425, "14095": 425, "14254": 425, "14112": 425, "14247": 425, "14015": 425, "14028": 425, "14219": 425, "10199": 425, "14233": 425, "14370": 426, "13352": 426, "14388": 426, "14196": 426, "14472": 426, "14486": 426, "14326": 426, "14418": 426, "14473": 426, "14350": 426, "14260": 426, "14412": 426, "12386": 426, "14398": 426, "14408": 426, "14212": 426, "pariti": 426, "13566": 426, "13423": 426, "14341": 426, "14443": 426, "table_typ": 426, "14384": 426, "14286": 426, "14400": 426, "timetyp": 426, "timestamptyp": 426, "timestamp_with_time_zon": 426, "14414": 426, "14396": 427, "14395": 427, "14604": 427, "acquisit": 427, "14580": 427, "14613": 427, "14616": 427, "14494": 427, "14571": 427, "14225": 427, "13545": 427, "14510": 427, "14504": 427, "inabl": 427, "14386": 427, "14299": 427, "13015": 427, "14090": 427, "14272": 427, "14655": 428, "13713": 428, "14596": 428, "14640": 428, "14553": 428, "verifyexcept": 428, "nomoresplit": 428, "14668": 428, "14572": 428, "14562": 428, "14573": 428, "12264": 428, "14611": 428, "14577": 428, "14625": 428, "14426": 428, "13691": 428, "14583": 428, "14224": 428, "14535": 428, "14445": 428, "14071": 428, "14730": 428, "14638": 428, "14794": 429, "10705": 429, "14653": 429, "14777": 429, "14823": 429, "13413": 429, "14693": 429, "14795": 429, "14570": 429, "14907": 430, "14941": 430, "14567": 430, "14950": 430, "14948": 430, "14886": 430, "14317": 430, "14738": 430, "14954": 430, "14949": 430, "14729": 430, "14760": 430, "13541": 430, "15005": 430, "8817": 430, "14434": 430, "13552": 430, "14864": 430, "14900": 430, "14856": 430, "14956": 430, "14959": 430, "funni": 431, "15317": 432, "15286": 432, "11608": 432, "14514": 432, "5798": 432, "14718": 432, "14874": 432, "14735": 432, "15292": 432, "15369": 432, "15168": 432, "14459": 432, "15138": 432, "15203": 432, "15103": 432, "15334": 432, "15343": 432, "15364": 432, "15314": 432, "14962": 432, "14008": 432, "improperli": 432, "14000": 432, "disclos": 432, "14208": 432, "15336": 432, "15339": 432, "15214": 432, "14972": 432, "14083": 432, "11609": 432, "15474": 432, "15072": 432, "14407": 432, "14423": 432, "14667": 432, "15132": 432, "15257": 432, "15268": 432, "13974": 432, "15204": 432, "15374": 432, "13568": 432, "5729": 432, "13017": 432, "15183": 432, "15476": 432, "15322": 432, "15477": 432, "15042": 432, "15278": 432, "15241": 432, "15066": 432, "14673": 432, "15174": [432, 433], "14746": 432, "7310": 432, "15470": 432, "13294": 432, "15408": 432, "13131": 432, "14971": 432, "protobuf": [432, 442, 448], "14734": 432, "15062": 432, "15240": 432, "15329": 432, "15226": 432, "15263": 432, "1398": 432, "15294": 432, "15426": 432, "15332": 432, "14841": 432, "15365": 432, "getbucketnodemap": 432, "14067": 432, "begindelet": [432, 433], "beginupd": [432, 433], "updatablepagesourc": [432, 433], "13926": 432, "15160": 433, "14964": 433, "15428": 433, "15711": 433, "15548": 433, "15569": 433, "15657": 433, "15629": 433, "inaccur": [433, 435, 443], "15642": 433, "15608": 433, "15461": 433, "15583": 433, "15675": 433, "15659": 433, "15680": 433, "15620": 433, "14981": 433, "15771": 433, "15708": 433, "15498": 433, "15554": 433, "15269": 433, "15552": 433, "15496": 433, "15706": 433, "14750": 433, "13243": 433, "14428": 433, "15017": 433, "15515": [433, 434], "9968": 433, "15607": 433, "15650": 433, "15723": 433, "15716": 433, "getdeleterowidcolumnhandl": 433, "finishdelet": 433, "getupdaterowidcolumnhandl": 433, "finishupd": 433, "15161": 433, "15989": 434, "headroom": 434, "15721": 434, "15489": 434, "15983": 434, "16110": 434, "15744": 434, "15858": 434, "15579": 434, "15994": 434, "16113": 434, "15861": 434, "15917": 434, "reusabl": 434, "16121": 434, "16101": 434, "15973": 434, "15784": 434, "15396": 434, "enablechangedatafe": 434, "15453": 434, "15683": 434, "15713": 434, "15850": 434, "15897": 434, "15923": 434, "15954": 434, "15878": 434, "15436": 434, "15942": 434, "15909": 434, "15811": 434, "16005": 434, "16111": 434, "15853": 434, "16040": 434, "merge_manifests_on_writ": 434, "14822": 434, "15981": 434, "15969": 434, "15779": 434, "15893": 434, "14372": 434, "16062": 434, "zerodatetimebehavior": 434, "converttonul": 434, "16027": 434, "16190": 435, "16140": 435, "15669": 435, "12208": 435, "16151": 435, "16145": 435, "13931": 435, "16129": 435, "16026": 435, "parquet_optimized_nested_reader_en": 435, "16177": 435, "13199": 435, "reader_vers": 435, "writer_vers": 435, "15932": 435, "16127": 435, "16152": 435, "16166": 435, "15991": 435, "16196": 435, "16180": 435, "15714": 435, "15975": 436, "16329": 436, "16201": 436, "15927": 436, "15999": 436, "15375": 436, "sortabl": 436, "16242": 436, "16327": 436, "16075": 436, "13063": 436, "ru": 436, "yandex": 436, "clickhousedriv": 436, "16188": 436, "16261": 436, "16310": 436, "16264": 436, "defaultcodec": 436, "16250": 436, "csv_native_reader_en": 436, "csv_native_writer_en": 436, "15918": 436, "json_native_reader_en": 436, "json_native_writer_en": 436, "text_file_native_reader_en": 436, "text_file_native_writer_en": 436, "sequence_file_native_reader_en": 436, "sequence_file_native_writer_en": 436, "regex_native_reader_en": 436, "regex_case_insensit": 436, "16271": 436, "16229": 436, "16120": 436, "15995": 436, "14891": 436, "14836": 436, "16185": 436, "16019": 437, "16406": 437, "16444": 437, "airbyt": 437, "16362": 437, "5947": 437, "6295": 437, "16349": 437, "16244": 437, "16342": 437, "16442": 438, "16494": 438, "16736": 438, "legacy_materialized_view_grace_period": 438, "15842": 438, "15478": 438, "16238": 438, "14078": 438, "16624": 438, "16681": 438, "16668": 438, "16669": 438, "16328": 438, "keyvalu": 438, "16482": 438, "16512": 438, "16541": 438, "15979": 438, "15546": 438, "16136": 438, "16631": 438, "16466": 438, "16109": 438, "16650": 438, "16388": 438, "16225": 438, "13050": 438, "openx": 438, "openx_json_native_reader_en": 438, "openx_json_native_writer_en": 438, "16073": 438, "16402": 438, "16381": 438, "16203": 438, "16595": 438, "16625": 438, "16677": 438, "16492": 438, "16680": 438, "16149": 438, "13196": 438, "15649": 438, "16637": 438, "15441": 438, "16473": 438, "16622": 438, "16396": 438, "16428": 438, "16720": 438, "isod": 438, "16626": 438, "16644": 438, "16696": 438, "16489": 439, "16523": 439, "16525": 439, "2578": 439, "16584": 439, "16781": 439, "16012": 439, "16103": 439, "16768": 439, "16647": 439, "16452": 439, "16721": 439, "16788": 439, "16796": 439, "16773": 439, "16779": 439, "16755": 439, "16135": 439, "16748": 440, "16753": 440, "16960": 440, "16826": 440, "16913": 440, "16938": 440, "16386": 440, "16116": 440, "16898": 440, "16661": 440, "getschemaproperti": 440, "getschemaown": 440, "catalogschemanam": 440, "16862": 440, "16854": 441, "16716": 441, "13072": 441, "16981": 441, "opentelemetri": 441, "16950": 441, "15555": 441, "16515": 441, "16989": 441, "16742": 441, "17105": 441, "12638": [441, 448], "16905": 441, "16884": 441, "16704": 441, "tombston": 441, "16962": 441, "16848": 441, "12655": 441, "17061": 441, "16860": 441, "11361": 442, "17279": 442, "15318": 442, "17139": 442, "17120": 442, "15163": 442, "17192": 442, "12318": 442, "oneof": 442, "confluent": 442, "16836": 442, "planningcputim": 442, "16802": 443, "16167": 443, "17270": 443, "17265": 444, "17089": 444, "17373": 444, "17404": 444, "17085": 444, "954": 444, "17364": 444, "17285": 444, "17341": 445, "17455": 445, "16187": 445, "16600": 445, "table_chang": 445, "16205": 445, "14493": 445, "17127": 445, "17393": 445, "17390": 445, "16546": 445, "17347": 445, "17470": 445, "17479": 445, "17200": 445, "14725": 446, "17458": 446, "17557": 446, "17334": 446, "17508": 446, "16064": 446, "17236": 446, "17612": 446, "17665": 446, "17590": 446, "17552": 446, "shorthand": 446, "12303": 446, "17530": 446, "17621": 446, "11701": 446, "17587": 446, "16745": 446, "nulladaptationpolici": 446, "scalarfunctionadapt": 446, "17706": 446, "17777": 447, "17776": 447, "17831": 447, "16691": 447, "17783": 447, "17773": 447, "15968": 447, "17921": 447, "17978": 447, "7197": 447, "avro_native_reader_en": 447, "17221": 447, "17677": 447, "17848": 447, "17901": 447, "17871": 447, "17299": 447, "17874": 447, "collectionnam": 447, "17883": 447, "17774": 447, "17195": 448, "18092": 448, "18061": 448, "254": 448, "17816": 448, "17900": 448, "17939": 448, "17527": 448, "15792": 448, "3866": 448, "18076": 448, "17563": 448, "18103": 448, "17394": 448, "17710": 448, "16900": 448, "block_and_position_not_nul": 448, "18035": 448, "block_build": 448, "18094": 448, "read_valu": 448, "17342": 448, "writeentri": 448, "16248": 449, "18005": 449, "18175": 449, "17640": 449, "18210": 449, "17793": 449, "15821": 449, "17405": 449, "18095": 449, "1240": 449, "avro_native_writer_en": 449, "18064": 449, "18167": 449, "18215": 449, "18223": 449, "18255": 449, "1828": 449, "16363": 449, "18170": 449, "17387": 449, "18136": 449, "17945": 449, "18209": 449, "16757": 450, "16959": 450, "18016": 450, "5061": 450, "18212": 450, "18491": 450, "18501": 450, "18159": 450, "18488": 450, "18540": 450, "18383": 450, "18328": 450, "17853": 450, "12587": 450, "cascad": [450, 451, 452, 453, 500], "18305": 450, "18516": 450, "17294": 450, "17502": 450, "16822": 450, "18345": 450, "18427": 450, "18564": 450, "18200": 450, "18388": 450, "18206": 450, "18014": 450, "18420": 450, "18004": 450, "18003": 450, "17775": 450, "s3select": [450, 454], "17946": 450, "18535": 450, "18315": 450, "18517": 450, "18586": 450, "18205": 450, "15606": 450, "18330": 450, "18533": 450, "add_dummi": 450, "18485": 450, "18121": 450, "18382": 450, "18140": 450, "18441": 450, "17934": 450, "18596": 450, "supportsreportingwrittenbyt": 450, "18617": 450, "18542": 451, "18657": 451, "18651": 451, "18639": 451, "18703": 451, "18709": 451, "18701": 451, "18689": 451, "18588": 451, "18668": 451, "18663": 451, "18696": 451, "getnewtablewriterscalingopt": 451, "getinsertwriterscalingopt": 451, "18561": 451, "18106": 452, "18714": 452, "18628": 452, "15411": 452, "16743": 452, "18756": 452, "18332": 452, "18320": 452, "17920": 452, "18683": 452, "18629": 452, "18722": 452, "18777": 452, "16067": 453, "13994": 453, "18868": 453, "18815": 453, "18860": 453, "18865": 453, "18863": 453, "18836": 453, "18786": 453, "18333": 453, "18888": 453, "18661": 453, "18111": 453, "18798": 453, "18653": 453, "18837": 453, "dropschema": 453, "18839": 453, "18497": 454, "19059": 454, "scann": 454, "19009": 454, "18771": 454, "19146": 454, "18514": 454, "16903": 454, "15873": 454, "16927": 454, "19032": 454, "19122": 454, "18845": 454, "18966": 454, "19076": 454, "18959": 454, "18930": 454, "_native_reader_en": 454, "_native_writer_en": 454, "18241": 454, "s3_select_pushdown_en": 454, "optimize_symlink_list": 454, "opencsv": 454, "18918": 454, "18976": 454, "18811": 454, "18213": 454, "14214": 454, "19111": 454, "19090": 454, "17114": 454, "16445": 454, "19068": 454, "18470": 454, "18924": 454, "18738": 454, "19187": 455, "task_partitioned_writer_count": 455, "task_scale_writers_max_writer_count": 455, "19135": 455, "19166": 455, "19119": 455, "19127": 455, "19128": 455, "18922": 455, "19052": 455, "19078": 455, "19243": 456, "builtin": [456, 462], "19160": 456, "13060": 456, "19296": 456, "19201": 456, "19295": 456, "19249": 456, "13981": 456, "19218": 456, "viewexpress": 456, "19302": 457, "19399": 457, "19002": 457, "19217": 457, "19102": 457, "17263": 457, "19303": 457, "19464": 457, "19408": 457, "19404": 457, "19191": 457, "sqlmap": 457, "sqlrow": 457, "18948": 457, "ldap1": 459, "ldap2": 459, "xfcc": 459, "azuread": 459, "prohibit": 460, "pki": 461, "onsit": 461, "middl": [461, 520], "authenticaton": 461, "disc": 462, "coars": 462, "grant_select": 462, "grant_execut": 462, "table_funct": 462, "human_resourc": 462, "guest": 462, "filter_environ": 462, "mask_environ": 462, "banned_us": 462, "hr": 462, "ssn": 462, "system_us": 462, "credit_card": 462, "20200824_183358_00000_c62aw": 462, "system_session_properti": 462, "catalog_session_properti": 462, "queryown": 462, "begun": 462, "contractor": 462, "dave": [462, 520], "original_us": 462, "original_rol": 462, "new_us": 462, "team_backend": 462, "team_backend_sandbox": 462, "team_": 462, "1_sandbox": 462, "principal_to_us": 462, "criterion": 462, "conclus": 462, "system_inform": 462, "original_group": 462, "new_rol": 462, "session_properti": 462, "membership": 463, "txt": 463, "group_nam": 463, "user_1": 463, "user_2": 463, "user_3": 463, "keytool": [464, 475], "yourkeystor": 464, "privatekeyentri": 464, "trustedcertentri": 464, "modern": [464, 465, 475], "dnsname": 464, "circumst": 464, "trustcacert": 464, "localkei": 464, "rememb": [464, 476], "recognzi": 464, "cacert": 464, "mnt": 464, "localcacert": 464, "privaci": 465, "mail": 465, "transmit": 465, "openssl": [465, 466, 475], "certifi": 465, "clustercoord": [465, 475], "miieowibaakcaqeawjl8cledfahhze3qoof1vwt4vuk9vyo38y1y9sgbfb02b2jw": 465, "ec": 465, "dsa": 465, "ellipt": 465, "curv": 465, "miidujccaqicaqewdqyjkozihvcnaqefbqawgaixczajbgnvbaytalvtmrywfayd": 465, "miidwjccaqoccqcxyqwz9gk50janbgkqhkig9w0baqsfadcbojelmakga1uebhmc": 465, "presum": 465, "noout": 465, "ok": 465, "neglig": 466, "slowdown": 466, "urandom": 466, "entropi": 466, "bandwidth": 466, "infiniband": 466, "shaprng": 466, "egd": 466, "workflow": 467, "stand": 467, "counterpart": 467, "iss": [467, 470], "firewal": 467, "keyid": 467, "prove": 467, "rs256": 467, "odbc": [468, 469, 472, 473], "admin_serv": 468, "749": 468, "domain_realm": 468, "kadmin": 468, "addprinc": 468, "randkei": 468, "ktadd": 468, "norandkei": 468, "7778": 468, "keystore_password": [468, 469], "fqdn": 468, "telnet": 468, "kinit": 468, "klist": 468, "kt": 468, "dlog": 468, "jre": 468, "clue": 468, "frontend": [469, 472, 473], "ldap_serv": 469, "corp": 469, "uk": [469, 476], "domain_name_of_the_serv": 469, "uid": 469, "distinguished_name_of_the_us": 469, "ou": [469, 476], "cn": [469, 476], "city_": 469, "state_": 469, "password1234": 469, "objectclass": 469, "objectclass_of_us": 469, "samaccountnam": [469, 470], "memberof": 469, "dn_of_the_authorized_group": 469, "authorizedgroup": 469, "inetorgperson": 469, "overlai": 469, "normal_group": 469, "another_group": 469, "trino_truststor": 469, "djavax": 469, "other_cli_argu": 469, "regener": 469, "8u181": 469, "communicationexcept": 469, "ldapserv": 469, "sslhandshakeexcept": 469, "jndi": 469, "disableendpointidentif": 469, "approv": 470, "authorization_endpoint": 470, "token_endpoint": 470, "jwks_uri": 470, "userinfo_endpoint": 470, "userinfo": 470, "access_token_issu": 470, "end_session_endpoint": 470, "client_id": 470, "client_secret": 470, "ietf": 470, "rfc6749": 470, "sha": 470, "forg": 470, "upn": 470, "twenti": 470, "offline_access": 470, "trino_coordin": 470, "mainli": 470, "oauth2webuiauthenticationfilt": 470, "laps": 470, "reauthent": 470, "team": 471, "proceed": [471, 475], "bcrypt": 472, "pbkdf2": 472, "2y": 472, "bqtb8hscp5dfcpmho5peyugxhz5ki": 472, "qf3wrpd7snm8swua3vlgqsa": 472, "5b4240333032306164": 472, "f38d165fce8ce42f59d366139ef5d9e1ca1247f0e06e503ee1a611dd9ec40876bb5edb8409f5abe5504aab6628e70cfb3d3a18e99d70357d295002c3d0a308a0": 472, "htpasswd": 472, "touch": 472, "sucess": 472, "rock": [472, 475], "20220919_113804_00017_54qfi": [472, 475], "salesfor": 473, "whitelist": 473, "passwordtoken": 473, "ansibl": 474, "db_password": 474, "technolog": 475, "unfamiliar": 475, "defer": 475, "sslserversocketfactori": 475, "getsupportedciphersuit": 475, "foreach": 475, "println": 475, "jshell": 475, "tls_rsa_with_aes_128_cbc_sha": 475, "tls_rsa_with_aes_128_cbc_sha256": 475, "sunjc": 475, "implementat": 475, "unaccept": 475, "negoti": 475, "commerci": 475, "registrar": 475, "verisign": 475, "godaddi": 475, "letsencrypt": 475, "sslforfre": 475, "corpor": 475, "IT": 475, "depart": [475, 482], "preconfigur": 475, "worth": 475, "der": 475, "tamper": 475, "actor": 475, "keymanag": 475, "certtool": 475, "anyon": 475, "expedi": 475, "signoff": 475, "csr": 475, "smith": 476, "acm": 476, "reqular": 476, "bob_uk": 476, "new_nam": [478, 479, 480, 481], "property_nam": [478, 480, 482, 488, 489, 490, 523], "ommit": [478, 480], "peopl": [478, 480, 481], "old_nam": 480, "new_typ": 480, "collaps": 480, "1992": [482, 520], "francisco": [482, 509, 513], "product_id": 482, "procedure_nam": 483, "callabl": 483, "view_nam": [486, 491, 498, 502, 514, 528, 531], "cancelled_ord": 486, "meaningless": 486, "order_totals_by_d": 486, "orders_nation_mkgseg": 486, "market": 486, "mktsegment": [486, 520], "role_nam": [487, 499], "schema_properti": 488, "existing_table_nam": 489, "column_properti": 489, "mayb": 489, "cent": 489, "bigger_ord": 489, "another_orderkei": 489, "another_orderd": 489, "column_alia": [490, 520], "orders_column_alias": 490, "total_pric": [490, 512], "empty_n": 490, "view_com": 491, "test_with_com": 491, "statement_nam": [492, 496, 497, 503, 504, 513], "my_queri": 492, "air": 493, "shipmod": 493, "my_select1": [496, 497, 503, 513], "my_select2": [496, 497, 503, 513], "my_count": 497, "my_creat": 497, "possess": [499, 507, 508, 517, 518], "parameter1": [503, 504], "parameter2": [503, 504], "round_robin": 505, "robin": 505, "fashion": 505, "count_8": 505, "hashvalue_9": 505, "hashvalue_10": 505, "outputrowcount": 505, "outputsizeinbyt": 505, "cpucost": 505, "memorycost": 505, "networkcost": 505, "isreplicatenullsandani": 505, "hashcolumn": [505, 506], "tpch_sf1_orc_part": 505, "sourcefragmentid": [505, 506], "227": 505, "selet": 505, "20220929_234840_00001_vjwxj": 505, "20220929_235059_00003_vjwxj": 505, "test_lineitem": 505, "shipdat": 505, "quantiti": [505, 510], "inputtablecolumninfo": 505, "schemat": 505, "test_ord": 505, "columnconstraint": 505, "nullsallow": 505, "maxmemori": 505, "outputt": 505, "knowledg": 506, "anomali": 506, "1995": [506, 520], "17u": 506, "96m": 506, "03m": 506, "72m": 506, "11kb": 506, "32kb": 506, "793": 506, "60m": 506, "92m": 506, "1500000": 506, "17mb": 506, "818058": 506, "48mb": 506, "76m": 506, "816424": 506, "298": 506, "98mb": 506, "51mb": 506, "62mb": 506, "p01": 506, "p05": 506, "p10": 506, "p25": 506, "p75": 506, "p95": 506, "p99": 506, "unbounded_preced": 506, "current_row": 506, "grantor": 508, "qux": [508, 518], "new_ord": 509, "jose": 509, "oakland": 509, "measure_definit": [510, 512], "rows_per_match": 510, "skip_to": [510, 512], "row_pattern": [510, 512], "subset_definit": [510, 512], "variable_definit": [510, 512], "subclaus": 510, "trend": 510, "financi": 510, "unusu": 510, "incid": 510, "fraud": 510, "starting_pric": 510, "bottom_pric": 510, "top_pric": 510, "measure_express": 510, "measure_nam": 510, "pattern_vari": 510, "leftmost": 510, "lexicograph": 510, "repetit": 510, "greedi": 510, "reluct": 510, "upfront": 510, "brows": 510, "mach": 510, "pattern_variable_": 510, "target_alia": 511, "source_alia": 511, "search_condit": 511, "when_claus": 511, "column_list": 511, "condition": 511, "centrevil": 511, "existing_window_nam": 512, "window_fram": 512, "frame_ext": 512, "frame_end": 512, "extent": 512, "cust_kei": 512, "detach": 512, "compli": 512, "some_measur": 512, "question": [513, 520], "my_insert": 513, "with_queri": 520, "select_express": 520, "from_item": 520, "grouping_el": 520, "window_definition_list": 520, "join_typ": 520, "join_condit": 520, "join_column": 520, "pattern_recognition_specif": 520, "table_function_invoc": 520, "t1": 520, "t2": 520, "cycl": 520, "max_recursion_depth": 520, "row_express": 520, "preexist": 520, "alias1": 520, "alias2": 520, "29968": 520, "30142": 520, "30189": 520, "29949": 520, "29752": 520, "destination_st": 520, "destination_zip": 520, "package_weight": 520, "8648": 520, "7081": 520, "connecticut": 520, "6708": 520, "90210": 520, "6927": 520, "1337": 520, "colorado": 520, "80302": 520, "10002": 520, "1562": 520, "subtot": 520, "_col2": 520, "_col3": 520, "coln": 520, "rightmost": 520, "_col4": 520, "011": 520, "acctbal": 520, "totalb": 520, "5700000": 520, "1272": 520, "5856939": 520, "1253": 520, "5794887": 520, "1248": 520, "5784628": 520, "5757371": 520, "1231": 520, "5753216": 520, "5719140": 520, "1247": 520, "5701952": 520, "window_nam": 520, "window_specif": 520, "tightli": 520, "some_t": 520, "another_t": 520, "kingdom": 520, "vietnam": 520, "1994": 520, "1993": 520, "1997": 520, "ethiopia": 520, "morocco": 520, "kenya": 520, "mozambiqu": 520, "1974": 520, "first_appeared_year": 520, "mari": 520, "bird": 520, "cow": 520, "roger": 520, "levi": 520, "marathon": 520, "explit": 520, "disconnect": 523, "acc01": 523, "kevin": 524, "cf_getgroup": 532, "cf_getprincip": 532, "cf_getus": 532, "__3": 536, "data_s": 538, "distinct_values_count": 538, "nulls_fract": 538, "low_valu": 538, "high_valu": 538, "uncommit": 540, "overdu": 542, "ship_dat": 542, "account_manag": 542, "henri": 542, "assign_d": 542, "new_hir": 542, "column_express": 544}, "objects": {"": [[109, 0, 1, "", "ST_Area"], [109, 0, 1, "", "ST_AsBinary"], [109, 0, 1, "", "ST_AsText"], [109, 0, 1, "", "ST_Boundary"], [109, 0, 1, "", "ST_Buffer"], [109, 0, 1, "", "ST_Centroid"], [109, 0, 1, "", "ST_ConvexHull"], [109, 0, 1, "", "ST_CoordDim"], [109, 0, 1, "", "ST_Dimension"], [109, 0, 1, "", "ST_EndPoint"], [109, 0, 1, "", "ST_Envelope"], [109, 0, 1, "", "ST_EnvelopeAsPts"], [109, 0, 1, "", "ST_ExteriorRing"], [109, 0, 1, "", "ST_GeomFromBinary"], [109, 0, 1, "", "ST_Geometries"], [109, 0, 1, "", "ST_GeometryFromText"], [109, 0, 1, "", "ST_GeometryN"], [109, 0, 1, "", "ST_GeometryType"], [109, 0, 1, "", "ST_InteriorRingN"], [109, 0, 1, "", "ST_InteriorRings"], [109, 0, 1, "", "ST_IsClosed"], [109, 0, 1, "", "ST_IsEmpty"], [109, 0, 1, "", "ST_IsRing"], [109, 0, 1, "", "ST_IsSimple"], [109, 0, 1, "", "ST_IsValid"], [109, 0, 1, "", "ST_Length"], [109, 0, 1, "", "ST_LineFromText"], [109, 0, 1, "", "ST_LineString"], [109, 0, 1, "", "ST_MultiPoint"], [109, 0, 1, "", "ST_NumGeometries"], [109, 0, 1, "", "ST_NumInteriorRing"], [109, 0, 1, "", "ST_NumPoints"], [109, 0, 1, "", "ST_PointN"], [109, 0, 1, "", "ST_Points"], [109, 0, 1, "", "ST_Polygon"], [109, 0, 1, "", "ST_StartPoint"], [109, 0, 1, "", "ST_X"], [109, 0, 1, "", "ST_XMax"], [109, 0, 1, "", "ST_XMin"], [109, 0, 1, "", "ST_Y"], [109, 0, 1, "", "ST_YMax"], [109, 0, 1, "", "ST_YMin"], [118, 0, 1, "", "abs"], [118, 0, 1, "", "acos"], [100, 0, 1, "", "all_match"], [100, 0, 1, "", "any_match"], [99, 0, 1, "", "any_value"], [99, 0, 1, "", "approx_distinct"], [99, 0, 1, "", "approx_most_frequent"], [99, 0, 1, "", "approx_percentile"], [110, 0, 1, "", "approx_set"], [99, 0, 1, "", "arbitrary"], [99, 0, 1, "", "array_agg"], [100, 0, 1, "", "array_distinct"], [100, 0, 1, "", "array_except"], [100, 0, 1, "", "array_histogram"], [100, 0, 1, "", "array_intersect"], [100, 0, 1, "", "array_join"], [100, 0, 1, "", "array_max"], [100, 0, 1, "", "array_min"], [100, 0, 1, "", "array_position"], [100, 0, 1, "", "array_remove"], [100, 0, 1, "", "array_sort"], [100, 0, 1, "", "array_union"], [100, 0, 1, "", "arrays_overlap"], [118, 0, 1, "", "asin"], [107, 0, 1, "", "at_timezone"], [118, 0, 1, "", "atan"], [118, 0, 1, "", "atan2"], [99, 0, 1, "", "avg"], [103, 0, 1, "", "bar"], [118, 0, 1, "", "beta_cdf"], [109, 0, 1, "", "bing_tile"], [109, 0, 1, "", "bing_tile_at"], [109, 0, 1, "", "bing_tile_coordinates"], [109, 0, 1, "", "bing_tile_polygon"], [109, 0, 1, "", "bing_tile_quadkey"], [109, 0, 1, "", "bing_tile_zoom_level"], [109, 0, 1, "", "bing_tiles_around"], [102, 0, 1, "", "bit_count"], [102, 0, 1, "", "bitwise_and"], [99, 0, 1, "", "bitwise_and_agg"], [102, 0, 1, "", "bitwise_left_shift"], [102, 0, 1, "", "bitwise_not"], [102, 0, 1, "", "bitwise_or"], [99, 0, 1, "", "bitwise_or_agg"], [102, 0, 1, "", "bitwise_right_shift"], [102, 0, 1, "", "bitwise_right_shift_arithmetic"], [102, 0, 1, "", "bitwise_xor"], [99, 0, 1, "", "bool_and"], [99, 0, 1, "", "bool_or"], [100, 0, 1, "", "cardinality"], [106, 0, 1, "", "cast"], [118, 0, 1, "", "cbrt"], [118, 0, 1, "", "ceil"], [118, 0, 1, "", "ceiling"], [128, 0, 1, "", "char2hexint"], [99, 0, 1, "", "checksum"], [124, 0, 1, "", "chr"], [119, 0, 1, "", "classify"], [105, 0, 1, "id1", "coalesce"], [124, 0, 1, "", "codepoint"], [103, 0, 1, "", "color"], [100, 0, 1, "", "combinations"], [124, 0, 1, "", "concat"], [124, 0, 1, "", "concat_ws"], [100, 0, 1, "", "contains"], [100, 0, 1, "", "contains_sequence"], [109, 0, 1, "", "convex_hull_agg"], [99, 0, 1, "", "corr"], [118, 0, 1, "", "cos"], [118, 0, 1, "", "cosh"], [118, 0, 1, "", "cosine_similarity"], [99, 0, 1, "", "count"], [99, 0, 1, "", "count_if"], [99, 0, 1, "", "covar_pop"], [99, 0, 1, "", "covar_samp"], [101, 0, 1, "", "crc32"], [131, 0, 1, "", "cume_dist"], [122, 1, 1, "", "current_catalog"], [107, 1, 1, "", "current_date"], [122, 0, 1, "", "current_groups"], [122, 1, 1, "", "current_schema"], [107, 1, 1, "", "current_time"], [107, 1, 1, "", "current_timestamp"], [107, 0, 1, "", "current_timezone"], [122, 1, 1, "", "current_user"], [107, 0, 1, "", "date"], [107, 0, 1, "", "date_add"], [107, 0, 1, "", "date_diff"], [107, 0, 1, "", "date_format"], [107, 0, 1, "", "date_parse"], [107, 0, 1, "", "date_trunc"], [107, 0, 1, "", "day"], [107, 0, 1, "", "day_of_month"], [107, 0, 1, "", "day_of_week"], [107, 0, 1, "", "day_of_year"], [118, 0, 1, "", "degrees"], [131, 0, 1, "", "dense_rank"], [107, 0, 1, "", "dow"], [107, 0, 1, "", "doy"], [118, 0, 1, "", "e"], [100, 0, 1, "", "element_at"], [110, 0, 1, "", "empty_approx_set"], [99, 0, 1, "", "every"], [126, 0, 1, "", "exclude_columns"], [118, 0, 1, "", "exp"], [107, 0, 1, "", "extract"], [119, 0, 1, "", "features"], [100, 0, 1, "", "filter"], [131, 0, 1, "", "first_value"], [100, 0, 1, "", "flatten"], [118, 0, 1, "", "floor"], [106, 0, 1, "", "format"], [107, 0, 1, "", "format_datetime"], [106, 0, 1, "", "format_number"], [118, 0, 1, "", "from_base"], [101, 0, 1, "", "from_base32"], [101, 0, 1, "", "from_base64"], [101, 0, 1, "", "from_base64url"], [101, 0, 1, "", "from_big_endian_32"], [101, 0, 1, "", "from_big_endian_64"], [109, 0, 1, "", "from_encoded_polyline"], [109, 0, 1, "", "from_geojson_geometry"], [101, 0, 1, "", "from_hex"], [101, 0, 1, "", "from_ieee754_32"], [101, 0, 1, "", "from_ieee754_64"], [107, 0, 1, "", "from_iso8601_date"], [107, 0, 1, "", "from_iso8601_timestamp"], [107, 0, 1, "", "from_iso8601_timestamp_nanos"], [107, 0, 1, "", "from_unixtime"], [107, 0, 1, "", "from_unixtime_nanos"], [124, 0, 1, "", "from_utf8"], [99, 0, 1, "", "geometric_mean"], [109, 0, 1, "", "geometry_from_hadoop_shape"], [109, 0, 1, "", "geometry_invalid_reason"], [109, 0, 1, "", "geometry_to_bing_tiles"], [109, 0, 1, "", "geometry_union"], [109, 0, 1, "", "geometry_union_agg"], [109, 0, 1, "", "great_circle_distance"], [104, 0, 1, "", "greatest"], [124, 0, 1, "", "hamming_distance"], [123, 0, 1, "", "hash_counts"], [99, 0, 1, "", "histogram"], [101, 0, 1, "", "hmac_md5"], [101, 0, 1, "", "hmac_sha1"], [101, 0, 1, "", "hmac_sha256"], [101, 0, 1, "", "hmac_sha512"], [107, 0, 1, "", "hour"], [107, 0, 1, "", "human_readable_seconds"], [105, 0, 1, "id0", "if"], [128, 0, 1, "", "index"], [118, 0, 1, "", "infinity"], [123, 0, 1, "", "intersection_cardinality"], [118, 0, 1, "", "inverse_beta_cdf"], [118, 0, 1, "", "inverse_normal_cdf"], [118, 0, 1, "", "is_finite"], [118, 0, 1, "", "is_infinite"], [112, 0, 1, "", "is_json_scalar"], [118, 0, 1, "", "is_nan"], [123, 0, 1, "", "jaccard_index"], [112, 0, 1, "", "json_array_contains"], [112, 0, 1, "", "json_array_get"], [112, 0, 1, "", "json_array_length"], [112, 0, 1, "", "json_extract"], [112, 0, 1, "", "json_extract_scalar"], [112, 0, 1, "", "json_format"], [112, 0, 1, "", "json_parse"], [112, 0, 1, "", "json_size"], [99, 0, 1, "", "kurtosis"], [131, 0, 1, "", "lag"], [107, 0, 1, "", "last_day_of_month"], [131, 0, 1, "", "last_value"], [131, 0, 1, "", "lead"], [119, 0, 1, "", "learn_classifier"], [119, 0, 1, "", "learn_libsvm_classifier"], [119, 0, 1, "", "learn_libsvm_regressor"], [119, 0, 1, "", "learn_regressor"], [104, 0, 1, "", "least"], [124, 0, 1, "", "length"], [124, 0, 1, "", "levenshtein_distance"], [109, 0, 1, "", "line_interpolate_point"], [109, 0, 1, "", "line_interpolate_points"], [109, 0, 1, "", "line_locate_point"], [99, 0, 1, "", "listagg"], [118, 0, 1, "", "ln"], [107, 1, 1, "", "localtime"], [107, 1, 1, "", "localtimestamp"], [118, 0, 1, "", "log"], [118, 0, 1, "", "log10"], [118, 0, 1, "", "log2"], [124, 0, 1, "", "lower"], [124, 0, 1, "", "lpad"], [124, 0, 1, "", "ltrim"], [124, 0, 1, "", "luhn_check"], [123, 0, 1, "", "make_set_digest"], [117, 0, 1, "", "map"], [99, 0, 1, "", "map_agg"], [117, 0, 1, "", "map_concat"], [117, 0, 1, "", "map_entries"], [117, 0, 1, "", "map_filter"], [117, 0, 1, "", "map_from_entries"], [117, 0, 1, "", "map_keys"], [99, 0, 1, "", "map_union"], [117, 0, 1, "", "map_values"], [117, 0, 1, "", "map_zip_with"], [99, 0, 1, "", "max"], [99, 0, 1, "", "max_by"], [101, 0, 1, "", "md5"], [110, 0, 1, "", "merge"], [123, 0, 1, "", "merge_set_digest"], [107, 0, 1, "", "millisecond"], [99, 0, 1, "", "min"], [99, 0, 1, "", "min_by"], [107, 0, 1, "", "minute"], [118, 0, 1, "", "mod"], [107, 0, 1, "", "month"], [99, 0, 1, "", "multimap_agg"], [117, 0, 1, "", "multimap_from_entries"], [101, 0, 1, "", "murmur3"], [118, 0, 1, "", "nan"], [100, 0, 1, "", "ngrams"], [100, 0, 1, "", "none_match"], [118, 0, 1, "", "normal_cdf"], [124, 0, 1, "", "normalize"], [107, 0, 1, "", "now"], [131, 0, 1, "", "nth_value"], [131, 0, 1, "", "ntile"], [105, 0, 1, "id2", "nullif"], [99, 0, 1, "", "numeric_histogram"], [65, 0, 1, "", "objectid_timestamp"], [106, 0, 1, "", "parse_data_size"], [107, 0, 1, "", "parse_datetime"], [107, 0, 1, "", "parse_duration"], [131, 0, 1, "", "percent_rank"], [118, 0, 1, "", "pi"], [124, 0, 1, "", "position"], [118, 0, 1, "", "pow"], [118, 0, 1, "", "power"], [120, 0, 1, "", "qdigest_agg"], [120, 0, 1, "", "quantile_at_value"], [107, 0, 1, "", "quarter"], [118, 0, 1, "", "radians"], [118, 0, 1, "", "rand"], [118, 0, 1, "", "random"], [131, 0, 1, "", "rank"], [100, 0, 1, "", "reduce"], [99, 0, 1, "", "reduce_agg"], [121, 0, 1, "", "regexp_count"], [121, 0, 1, "", "regexp_extract"], [121, 0, 1, "", "regexp_extract_all"], [121, 0, 1, "", "regexp_like"], [121, 0, 1, "", "regexp_position"], [121, 0, 1, "", "regexp_replace"], [121, 0, 1, "", "regexp_split"], [99, 0, 1, "", "regr_intercept"], [99, 0, 1, "", "regr_slope"], [119, 0, 1, "", "regress"], [103, 0, 1, "", "render"], [100, 0, 1, "", "repeat"], [124, 0, 1, "", "replace"], [124, 0, 1, "", "reverse"], [103, 0, 1, "", "rgb"], [118, 0, 1, "", "round"], [131, 0, 1, "", "row_number"], [124, 0, 1, "", "rpad"], [124, 0, 1, "", "rtrim"], [107, 0, 1, "", "second"], [100, 0, 1, "", "sequence"], [101, 0, 1, "", "sha1"], [101, 0, 1, "", "sha256"], [101, 0, 1, "", "sha512"], [100, 0, 1, "", "shuffle"], [118, 0, 1, "", "sign"], [109, 0, 1, "", "simplify_geometry"], [118, 0, 1, "", "sin"], [118, 0, 1, "", "sinh"], [99, 0, 1, "", "skewness"], [100, 0, 1, "", "slice"], [124, 0, 1, "", "soundex"], [124, 0, 1, "", "split"], [124, 0, 1, "", "split_part"], [124, 0, 1, "", "split_to_map"], [124, 0, 1, "", "split_to_multimap"], [101, 0, 1, "", "spooky_hash_v2_32"], [101, 0, 1, "", "spooky_hash_v2_64"], [118, 0, 1, "", "sqrt"], [124, 0, 1, "", "starts_with"], [99, 0, 1, "", "stddev"], [99, 0, 1, "", "stddev_pop"], [99, 0, 1, "", "stddev_samp"], [124, 0, 1, "", "strpos"], [124, 0, 1, "", "substr"], [124, 0, 1, "", "substring"], [99, 0, 1, "", "sum"], [118, 0, 1, "", "tan"], [118, 0, 1, "", "tanh"], [127, 0, 1, "", "tdigest_agg"], [65, 0, 1, "", "timestamp_objectid"], [107, 0, 1, "", "timezone_hour"], [107, 0, 1, "", "timezone_minute"], [118, 0, 1, "", "to_base"], [101, 0, 1, "", "to_base32"], [101, 0, 1, "", "to_base64"], [101, 0, 1, "", "to_base64url"], [101, 0, 1, "", "to_big_endian_32"], [101, 0, 1, "", "to_big_endian_64"], [128, 0, 1, "", "to_char"], [128, 0, 1, "", "to_date"], [109, 0, 1, "", "to_encoded_polyline"], [109, 0, 1, "", "to_geojson_geometry"], [109, 0, 1, "", "to_geometry"], [101, 0, 1, "", "to_hex"], [101, 0, 1, "", "to_ieee754_32"], [101, 0, 1, "", "to_ieee754_64"], [107, 0, 1, "", "to_iso8601"], [107, 0, 1, "", "to_milliseconds"], [109, 0, 1, "", "to_spherical_geography"], [128, 0, 1, "", "to_timestamp"], [107, 0, 1, "", "to_unixtime"], [124, 0, 1, "", "to_utf8"], [100, 0, 1, "", "transform"], [117, 0, 1, "", "transform_keys"], [117, 0, 1, "", "transform_values"], [124, 0, 1, "", "translate"], [124, 0, 1, "", "trim"], [100, 0, 1, "", "trim_array"], [118, 0, 1, "", "truncate"], [105, 0, 1, "id3", "try"], [106, 0, 1, "", "try_cast"], [106, 0, 1, "", "typeof"], [124, 0, 1, "", "upper"], [129, 0, 1, "", "url_decode"], [129, 0, 1, "", "url_encode"], [129, 0, 1, "", "url_extract_fragment"], [129, 0, 1, "", "url_extract_host"], [129, 0, 1, "", "url_extract_parameter"], [129, 0, 1, "", "url_extract_path"], [129, 0, 1, "", "url_extract_port"], [129, 0, 1, "", "url_extract_protocol"], [129, 0, 1, "", "url_extract_query"], [130, 0, 1, "", "uuid"], [120, 0, 1, "", "value_at_quantile"], [120, 0, 1, "", "values_at_quantiles"], [99, 0, 1, "", "var_pop"], [99, 0, 1, "", "var_samp"], [99, 0, 1, "", "variance"], [125, 0, 1, "", "version"], [107, 0, 1, "", "week"], [107, 0, 1, "", "week_of_year"], [118, 0, 1, "", "width_bucket"], [118, 0, 1, "", "wilson_interval_lower"], [118, 0, 1, "", "wilson_interval_upper"], [107, 0, 1, "", "with_timezone"], [124, 0, 1, "", "word_stem"], [101, 0, 1, "", "xxhash64"], [107, 0, 1, "", "year"], [107, 0, 1, "", "year_of_week"], [107, 0, 1, "", "yow"], [100, 0, 1, "", "zip"], [100, 0, 1, "", "zip_with"]], "runtime": [[77, 0, 1, "", "kill_query"]]}, "objtypes": {"0": "py:function", "1": "py:data"}, "objnames": {"0": ["py", "function", "Python function"], "1": ["py", "data", "Python data"]}, "titleterms": {"administr": 0, "event": [0, 3, 4, 85, 269], "listen": [0, 3, 4, 85, 269], "distribut": [1, 10, 118, 146, 282, 304, 505], "sort": [1, 54], "dynam": [2, 29, 45, 63, 70, 193], "filter": [2, 14, 45, 63, 65, 99, 112, 287, 462], "analysi": 2, "confirm": 2, "collect": [2, 65], "threshold": [2, 10, 18, 19, 40, 42, 55, 62, 66, 68, 69, 71, 74, 75, 76], "dimens": 2, "tabl": [2, 14, 35, 37, 39, 40, 41, 42, 43, 44, 45, 53, 54, 55, 57, 58, 59, 60, 61, 62, 63, 64, 65, 66, 68, 69, 70, 71, 73, 74, 75, 76, 77, 91, 96, 126, 143, 146, 149, 151, 177, 282, 283, 288, 297, 462, 480, 489, 490, 501, 530, 539], "layout": 2, "limit": [2, 5, 16, 18, 39, 44, 45, 48, 57, 60, 63, 84, 112, 113, 148, 287, 475, 487, 493, 494, 499, 507, 508, 511, 517, 518, 520, 522, 525, 533, 542], "http": [3, 11, 12, 19, 32, 83, 86, 475], "rational": [3, 4], "requir": [3, 4, 32, 33, 35, 36, 37, 39, 40, 41, 42, 45, 50, 52, 53, 54, 55, 57, 60, 62, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 78, 136, 137], "configur": [3, 4, 5, 27, 32, 33, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 59, 60, 61, 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 82, 84, 85, 89, 90, 92, 95, 135, 136, 137, 138, 296, 461, 462, 463, 466, 467, 468, 469, 470, 471, 472, 473, 475], "properti": [3, 4, 5, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 26, 35, 36, 37, 39, 40, 41, 42, 43, 44, 45, 47, 50, 51, 52, 53, 54, 55, 57, 59, 60, 61, 62, 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 76, 78, 136, 167, 177, 305, 461, 462, 467, 468, 469, 470, 478, 480, 523], "custom": [3, 51, 93], "header": [3, 10, 83, 90, 459], "mysql": [4, 66, 107, 173, 182, 208, 209, 213, 217, 230, 241, 254, 256, 257, 260, 265, 303, 327, 333, 335, 336, 339, 343, 345, 347, 354, 365, 368, 369, 372, 373, 379, 380, 381, 383, 386, 387, 388, 390, 392, 393, 396, 397, 399, 402, 403, 405, 408, 409, 410, 411, 418, 419, 420, 422, 423, 428, 429, 432, 434, 438, 439, 450, 454], "fault": [5, 37, 41, 45, 54, 65, 66, 68, 71, 74, 76], "toler": [5, 37, 41, 45, 54, 65, 66, 68, 71, 74, 76], "execut": [5, 15, 37, 41, 45, 54, 63, 65, 66, 68, 71, 74, 76, 96, 135, 137, 151, 469, 480, 503, 504], "retri": [5, 15, 16], "polici": [5, 13, 15], "queri": [5, 7, 14, 15, 17, 18, 27, 29, 37, 39, 40, 42, 43, 44, 47, 54, 55, 56, 58, 60, 62, 65, 66, 68, 69, 70, 71, 74, 75, 76, 77, 83, 135, 137, 138, 145, 151, 163, 307, 462], "task": [5, 7, 13, 14, 15, 19, 22, 77, 151, 157], "advanc": [5, 47, 466], "size": [5, 8, 9, 11, 12, 14, 15, 17, 57, 59, 65, 78, 106, 112, 146], "node": [5, 7, 13, 14, 17, 18, 57, 73, 77, 136, 468, 469], "alloc": 5, "other": [5, 27, 112, 209, 334, 349, 352, 353, 355, 357, 358, 360, 373, 374, 375, 377, 380, 384], "tune": [5, 26, 41, 45, 51, 466], "exchang": [5, 9, 17, 151], "manag": [5, 15, 17, 23, 24, 25, 35, 41, 45, 53, 54, 60, 86, 143, 157, 167, 170, 458, 471], "aw": [5, 43, 59, 64], "s3": [5, 49, 51, 284, 288], "azur": [5, 47], "blob": 5, "storag": [5, 37, 45, 47, 48, 49, 50, 67, 301], "googl": [5, 44, 50, 354, 432, 437, 448], "cloud": [5, 49, 50], "hdf": [5, 45, 52], "local": [5, 61, 137, 213, 268, 293, 368, 378], "filesystem": 5, "grace": [6, 182], "shutdown": [6, 182], "behavior": [6, 60, 112], "monitor": [7, 41], "jmx": [7, 56, 209, 217, 226, 257, 263, 327, 348, 368, 378], "jvm": [7, 26, 136], "trino": [7, 26, 35, 37, 39, 40, 41, 42, 43, 44, 46, 54, 58, 60, 62, 65, 66, 68, 69, 70, 71, 72, 75, 76, 78, 83, 133, 135, 136, 137, 139, 144, 151, 152, 468, 469, 470, 475], "cluster": [7, 39, 45, 57, 65, 74, 137, 151, 157, 167, 170, 458, 471], "connector": [7, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 84, 86, 87, 94, 146, 151, 154, 209, 226, 243, 250, 254, 255, 256, 258, 264, 268, 269, 270, 271, 272, 273, 274, 275, 276, 277, 278, 279, 287, 296, 297, 303, 305, 327, 328, 329, 330, 331, 332, 333, 334, 335, 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, 349, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, 377, 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, 442, 443, 444, 445, 446, 447, 448, 449, 450, 451, 452, 453, 454, 455, 456, 457], "refer": [8, 33, 47, 510], "valu": [8, 41, 52, 54, 58, 131, 288, 544], "type": [8, 10, 13, 20, 27, 29, 35, 37, 39, 40, 41, 42, 43, 44, 54, 55, 57, 59, 60, 62, 63, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 84, 96, 97, 112, 144, 293, 296, 304, 305, 316, 325, 459, 505], "boolean": [8, 144], "data": [8, 9, 22, 29, 35, 40, 41, 42, 45, 47, 51, 54, 58, 60, 62, 66, 68, 71, 74, 76, 106, 110, 120, 123, 127, 143, 144, 151, 471], "doubl": [8, 112, 144], "durat": [8, 15, 107], "integ": [8, 29, 101, 144], "string": [8, 29, 115, 124, 128, 144], "client": [9, 11, 15, 31, 78, 83, 390, 461, 467, 471], "thread": [9, 18, 19, 78], "concurr": [9, 19, 193], "request": [9, 11, 15, 83], "multipli": 9, "integr": 9, "verif": [9, 11, 468, 469], "max": [9, 11, 12, 13, 14, 15, 17, 18, 19, 59, 65, 73, 78], "buffer": [9, 17, 57], "respons": [9, 19, 78, 83], "sink": 9, "broadcast": 9, "gener": [10, 11, 14, 40, 41, 42, 45, 50, 53, 54, 55, 62, 64, 66, 68, 69, 70, 71, 74, 75, 76, 99, 154, 155, 156, 157, 158, 159, 160, 161, 162, 163, 164, 165, 166, 167, 168, 169, 170, 171, 172, 173, 175, 176, 177, 178, 179, 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, 190, 191, 192, 193, 194, 195, 196, 197, 198, 200, 201, 202, 203, 204, 205, 206, 207, 208, 209, 210, 211, 212, 213, 214, 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, 226, 227, 228, 229, 230, 231, 232, 233, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, 276, 277, 278, 279, 280, 293, 296, 297, 300, 301, 302, 303, 304, 305, 306, 307, 308, 310, 313, 314, 315, 316, 317, 318, 319, 320, 321, 322, 323, 324, 325, 326, 327, 328, 329, 330, 331, 332, 333, 334, 335, 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, 371, 372, 373, 374, 375, 377, 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, 442, 443, 444, 445, 446, 447, 448, 449, 450, 451, 452, 453, 454, 455, 456, 457], "join": [10, 14, 25, 66, 68, 71, 75, 76, 146, 148, 283, 304, 520], "redistribut": [10, 94], "write": [10, 14, 15, 21, 44, 47, 65, 143], "protocol": [10, 13, 57], "v1": 10, "altern": [10, 49, 469, 510], "name": [10, 57, 65, 73, 98, 327, 378, 458, 469, 471, 520], "prepar": [10, 492, 513], "statement": [10, 143, 151, 477], "compress": [10, 11, 12, 18, 25, 76], "length": [10, 13, 15, 296], "min": [10, 13, 14, 15, 19, 22, 65], "gain": [10, 282], "connect": [11, 33, 40, 65, 66, 71, 74, 75, 76, 470], "timeout": [11, 15, 19, 20, 65], "tl": [11, 32, 43, 65, 466, 468, 469, 475], "secur": [11, 40, 41, 43, 45, 51, 52, 54, 57, 66, 71, 74, 75, 76, 143, 232, 254, 258, 260, 261, 264, 266, 268, 270, 272, 279, 328, 329, 334, 335, 336, 344, 346, 348, 351, 353, 354, 355, 356, 357, 358, 359, 360, 362, 363, 364, 367, 368, 371, 372, 374, 379, 381, 382, 383, 384, 386, 387, 388, 389, 392, 394, 396, 397, 398, 399, 403, 405, 408, 415, 420, 424, 430, 432, 435, 441, 447, 449, 453, 454, 456, 457, 458, 466, 469, 471, 473, 475, 491], "exclud": 11, "cipher": 11, "includ": [11, 13], "hostnam": 11, "kei": [11, 39, 47, 57, 59, 73, 112, 465, 475], "store": 11, "password": [11, 27, 32, 43, 57, 65, 73, 92, 459, 469, 472], "path": [11, 12, 18, 65, 112, 521], "random": [11, 118], "algorithm": [11, 57], "trust": 11, "proxi": 11, "sock": 11, "log": [11, 12, 57, 136], "enabl": [11, 12, 14, 15, 18, 19, 20, 22, 57, 65, 466], "flush": [11, 41], "interv": [11, 19, 29, 59, 107, 144, 209], "histori": [11, 12, 15, 27, 32, 41, 54], "queue": 11, "annot": 12, "file": [12, 13, 23, 32, 52, 54, 57, 58, 61, 67, 73, 213, 268, 368, 378, 462, 463, 464, 465, 468, 469, 472, 475, 476], "format": [12, 32, 67, 106, 301, 463, 472, 505], "total": [12, 17], "server": [12, 40, 55, 66, 68, 71, 73, 75, 76, 254, 256, 264, 268, 278, 327, 328, 331, 333, 334, 335, 336, 339, 341, 342, 343, 345, 349, 353, 357, 359, 360, 369, 370, 371, 372, 373, 374, 377, 378, 379, 380, 381, 382, 383, 387, 388, 389, 390, 391, 392, 396, 397, 398, 399, 400, 404, 405, 406, 407, 408, 410, 411, 418, 419, 420, 421, 422, 426, 427, 428, 429, 430, 435, 436, 438, 439, 454, 469, 470], "schedul": [13, 23, 183, 296, 302], "coordin": [13, 151, 296, 468, 469, 475], "split": [13, 15, 19, 86, 151, 193], "per": [13, 14, 15, 17, 18, 19, 65, 73, 98, 510], "pend": 13, "adjust": 13, "unacknowledg": 13, "candid": 13, "network": [13, 144, 183], "topolog": [13, 183], "segment": 13, "base": [13, 14, 52, 66, 71, 76, 118, 146, 328, 330, 332, 333, 341, 364, 365, 399, 462, 469], "refresh": [13, 59, 78, 462, 470, 514], "period": 13, "subnet": 13, "ip": [13, 111], "address": [13, 78, 111, 144], "cidr": 13, "prefix": [13, 73], "optim": [14, 41, 45, 54, 145, 146, 287, 293, 302, 307], "dictionari": 14, "aggreg": [14, 18, 19, 25, 88, 99, 109, 115, 131, 148, 282, 284, 510], "hash": [14, 15, 60, 101, 282], "metadata": [14, 35, 40, 41, 42, 44, 45, 53, 54, 55, 64, 66, 68, 69, 71, 74, 75, 76, 77, 78, 86, 93, 307], "mark": 14, "distinct": [14, 104, 282, 284, 287], "strategi": 14, "push": [14, 43, 57], "through": [14, 96], "outer": 14, "union": [14, 510, 520], "reorder": 14, "duplic": [14, 94], "insensit": [14, 40, 42, 55, 62, 65, 66, 68, 69, 71, 74, 75, 76], "us": [14, 18, 21, 29, 49, 54, 58, 77, 137, 152, 287, 461, 467, 469, 475, 543], "exact": 14, "partit": [14, 15, 21, 37, 39, 45, 54, 60, 64, 510], "scan": [14, 15, 73], "bucket": [14, 282], "ratio": 14, "coloc": 14, "conjunct": 14, "independ": 14, "factor": 14, "multi": 14, "claus": [14, 512, 520], "non": [14, 40, 55, 62, 66, 68, 69, 71, 74, 75, 76, 282], "estimat": 14, "predic": [14, 40, 43, 55, 62, 66, 68, 71, 73, 75, 76, 84, 104, 148, 282, 287], "approxim": [14, 99], "build": [14, 93], "row": [14, 54, 57, 59, 73, 94, 96, 104, 115, 144, 302, 325, 510, 512], "count": [14, 15, 19, 73, 510], "input": [14, 496, 512], "cost": [14, 66, 71, 76, 146, 147], "determin": 15, "writer": [15, 19, 22], "low": 15, "memori": [15, 17, 18, 19, 25, 41, 63, 170, 243, 250, 258, 272, 278, 320, 321, 359, 385, 410, 417, 422, 429, 434, 451], "killer": 15, "delai": [15, 45, 63], "time": [15, 17, 29, 54, 57, 58, 59, 65, 73, 107, 115, 144, 293, 525], "plan": 15, "run": [15, 32, 135, 136, 137, 510], "physic": [15, 510], "byte": 15, "stage": [15, 151], "expir": 15, "ag": 15, "remot": 15, "adapt": 15, "guarante": 15, "error": [15, 469], "headroom": [15, 17], "info": [15, 19], "url": [15, 65, 115, 129], "templat": 15, "regular": [16, 115, 121, 156], "express": [16, 29, 105, 112, 113, 115, 121, 156, 510, 520], "function": [16, 25, 37, 39, 41, 42, 43, 44, 62, 65, 66, 68, 71, 74, 76, 88, 96, 98, 99, 100, 101, 102, 103, 104, 106, 107, 108, 109, 110, 111, 112, 114, 115, 117, 118, 119, 120, 121, 123, 124, 126, 127, 128, 129, 130, 131, 156, 209, 217, 282, 287, 296, 297, 302, 317, 462, 510, 532], "regex": 16, "librari": [16, 328, 330, 332, 333, 341, 364, 365, 399], "re2j": 16, "dfa": 16, "state": 16, "resourc": [17, 23, 157, 167, 217, 251, 253, 254, 255, 256, 262, 263, 269, 276, 332, 467, 468], "cpu": [17, 282], "heap": [17, 72], "dedupl": 17, "spill": [18, 25], "spiller": 18, "space": [18, 25], "oper": [18, 25, 29, 98, 100, 101, 104, 107, 108, 109, 112, 114, 116, 117, 118, 124, 136, 143, 151, 520], "unspil": 18, "encrypt": [18, 25, 51, 471], "updat": [19, 45, 54, 55, 62, 66, 68, 71, 74, 75, 76, 542], "driver": [19, 33, 151, 183, 213, 217, 232, 233, 234, 235, 244, 245, 246, 249, 251, 253, 254, 255, 258, 259, 264, 270, 271, 274, 275, 279, 327, 334, 335, 340, 344, 347, 350, 355, 357, 358, 359, 360, 361, 365, 366, 367, 368, 372, 373, 374, 375, 377, 378, 379, 382, 383, 384, 385, 388, 389, 391, 396, 399, 401, 402, 403, 413, 419, 424, 432, 439, 448, 450, 451, 457], "partial": 19, "worker": [19, 151], "scale": [19, 22], "interrupt": 19, "stuck": 19, "warn": 19, "detect": [19, 94], "web": [20, 27, 155, 217, 229, 232, 235, 236, 239, 242, 248, 249, 257, 259, 269, 272, 274, 278, 279, 280, 327, 329, 330, 331, 332, 334, 341, 344, 345, 346, 352, 357, 358, 361, 362, 368, 373, 375, 377, 384, 386, 389, 391, 394, 400, 403, 406, 408, 410, 432, 442, 446, 456], "ui": [20, 27, 155, 217, 229, 232, 235, 236, 239, 242, 248, 249, 257, 259, 269, 272, 274, 278, 279, 280, 327, 329, 330, 331, 332, 334, 341, 344, 345, 346, 352, 357, 358, 361, 362, 368, 373, 375, 377, 384, 386, 389, 391, 394, 400, 403, 406, 408, 410, 432, 442, 446, 456], "authent": [20, 27, 32, 40, 42, 43, 52, 62, 66, 68, 71, 72, 74, 76, 82, 90, 92, 458, 459, 461, 467, 468, 469, 470, 471, 472, 473], "share": [20, 466], "secret": [20, 59, 466, 474], "session": [20, 24, 35, 41, 115, 122, 167, 305, 462, 515, 516, 523, 524, 537], "user": [20, 27, 73, 458, 468, 469, 471, 476], "prefer": [21, 65], "process": [22, 83, 94, 138, 512], "group": [23, 89, 217, 251, 253, 254, 255, 256, 262, 263, 269, 276, 332, 463, 469, 510, 520], "databas": [23, 65, 71, 73, 74, 76], "weight": 23, "exampl": [23, 24, 32, 38, 45, 52, 53, 54, 60, 63, 68, 86, 87, 88, 105, 112, 113, 141, 462, 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, 511, 513, 515, 517, 518, 519, 521, 523, 524, 525, 530, 533, 540, 541, 542, 543, 544], "selector": 23, "rule": [23, 24, 112, 462, 476], "global": [23, 143], "provid": [23, 51, 86, 89, 463], "match": [24, 40, 42, 55, 62, 65, 66, 68, 69, 71, 74, 75, 76, 94, 510, 512], "disk": [25, 36], "overview": [25, 27, 52, 83, 93, 94, 150, 151, 471], "support": [25, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 53, 54, 55, 56, 57, 59, 60, 62, 63, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 91, 94, 143, 144, 156, 163, 288, 307, 475], "order": [25, 99, 146, 510, 520], "window": [25, 115, 131, 287, 512, 520], "config": [26, 136, 468, 469], "set": [26, 46, 54, 65, 86, 96, 115, 123, 478, 480, 520, 521, 522, 523, 524, 525], "fix": [27, 144, 209, 282, 287, 293], "interfac": [27, 32], "appendix": 28, "migrat": [29, 54], "from": [29, 35, 37, 58, 59, 104, 112, 282], "hive": [29, 45, 46, 47, 48, 49, 51, 52, 155, 156, 157, 158, 166, 167, 168, 169, 173, 177, 178, 180, 183, 186, 188, 191, 192, 193, 194, 196, 197, 198, 199, 201, 206, 207, 208, 209, 210, 211, 213, 215, 217, 218, 219, 220, 221, 223, 224, 225, 226, 227, 228, 229, 230, 231, 232, 233, 235, 236, 239, 240, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, 276, 277, 278, 279, 280, 282, 293, 297, 301, 302, 303, 304, 305, 306, 307, 308, 317, 323, 324, 325, 327, 328, 329, 330, 331, 332, 333, 334, 335, 336, 337, 338, 339, 341, 342, 343, 344, 345, 346, 347, 348, 350, 351, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, 378, 379, 380, 381, 382, 383, 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, 407, 408, 409, 412, 413, 414, 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, 432, 433, 434, 435, 436, 437, 438, 439, 440, 442, 443, 444, 445, 446, 447, 448, 449, 450, 451, 452, 453, 454, 455, 456], "subscript": [29, 100, 117], "access": [29, 47, 59, 64, 71, 95, 96, 458, 460, 462, 468, 471], "index": [29, 35, 73, 304], "an": 29, "arrai": [29, 43, 71, 100, 112, 115, 144, 305, 325], "instead": 29, "udf": 29, "avoid": 29, "out": 29, "bound": [29, 57], "ansi": 29, "sql": [29, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 52, 53, 54, 55, 56, 57, 59, 60, 62, 63, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 84, 94, 140, 143, 254, 256, 327, 333, 335, 336, 343, 345, 359, 369, 370, 371, 372, 377, 379, 380, 381, 382, 383, 387, 388, 390, 391, 392, 396, 397, 398, 399, 400, 404, 405, 406, 407, 408, 410, 411, 418, 419, 420, 421, 422, 426, 427, 428, 429, 430, 435, 436, 438, 439, 454, 477], "syntax": [29, 112, 477, 510], "identifi": [29, 142], "quot": 29, "start": [29, 510, 540], "number": [29, 68, 302], "standard": [29, 52, 94, 475], "concaten": [29, 100, 510], "cast": [29, 112, 170], "target": [29, 94], "when": [29, 96], "divid": 29, "WITH": [29, 144, 520], "complex": [29, 520], "unnest": [29, 520], "expand": 29, "map": [29, 35, 37, 39, 40, 41, 42, 43, 44, 51, 54, 55, 57, 58, 59, 60, 62, 63, 65, 66, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 84, 99, 115, 117, 144, 305, 325, 468, 476], "date": [29, 43, 57, 59, 73, 107, 115, 128, 144, 293, 316], "caution": 29, "datediff": 29, "overwrit": 29, "insert": [29, 40, 55, 57, 60, 62, 66, 68, 69, 71, 74, 75, 76, 91, 509], "legal": 30, "notic": 30, "licens": 30, "code": [30, 86, 87, 93], "govern": 30, "trademark": 30, "command": [32, 136, 139], "line": 32, "instal": [32, 33, 35, 58, 134, 136, 139], "cli": [32, 173, 183, 186, 206, 217, 234, 242, 245, 247, 253, 255, 256, 257, 258, 259, 260, 262, 263, 267, 270, 273, 274, 282, 329, 330, 332, 334, 335, 337, 340, 341, 342, 343, 346, 347, 350, 357, 358, 361, 374, 375, 378, 379, 381, 384, 386, 387, 392, 397, 401, 403, 405, 408, 414, 435, 450, 469], "usernam": [32, 45], "extern": [32, 35], "sso": 32, "certif": [32, 82, 461, 465, 475], "option": [32, 282], "jwt": [32, 467], "kerbero": [32, 52, 60, 468], "addit": [32, 468], "debug": [32, 468, 469], "inform": [32, 122, 125, 462, 468], "pagin": 32, "auto": 32, "suggest": [32, 471], "batch": [32, 59, 65], "mode": [32, 112], "output": [32, 497], "troubleshoot": [32, 468, 469, 470], "jdbc": [33, 64, 87, 183, 213, 217, 232, 233, 234, 235, 244, 245, 246, 249, 251, 253, 254, 255, 258, 259, 260, 264, 270, 271, 274, 275, 279, 287, 327, 328, 330, 332, 333, 334, 335, 340, 341, 344, 347, 350, 355, 357, 358, 359, 360, 361, 364, 365, 366, 367, 368, 372, 373, 374, 375, 377, 378, 379, 382, 383, 384, 385, 388, 389, 391, 396, 399, 401, 402, 403, 413, 419, 424, 432, 439, 448, 450, 451, 457], "regist": [33, 41, 54], "paramet": [33, 48], "accumulo": [35, 228, 241, 246, 255, 362, 399, 403, 410, 413, 425, 426], "iter": [35, 59], "depend": 35, "variabl": [35, 112, 296, 510], "usag": [35, 36, 41, 44, 45, 50, 53, 54, 57, 320, 321, 510], "column": [35, 36, 37, 41, 43, 45, 53, 54, 57, 58, 59, 60, 72, 73, 96, 462, 520, 527], "load": [35, 58, 76, 475], "ad": [35, 137], "serial": [35, 37, 110, 123], "convert": 35, "intern": [35, 57, 59, 73, 466], "atop": [36, 369], "reboot": 36, "bigqueri": [37, 359, 360, 368, 370, 373, 377, 379, 380, 383, 384, 386, 388, 389, 390, 392, 396, 400, 408, 409, 410, 418, 421, 423, 425, 426, 427, 430, 432, 433, 435, 436, 438, 445, 446, 447, 449], "api": [37, 44, 83, 94], "multipl": [37, 39, 40, 45, 55, 57, 65, 66, 68, 71, 73, 74, 75, 76, 78, 85, 459, 520], "gcp": 37, "project": [37, 64, 148], "arrow": 37, "read": [37, 65, 143, 460], "view": [37, 45, 54, 77, 143, 297, 478, 481, 486, 491, 498, 502, 514, 528, 531], "system": [37, 48, 71, 77, 78, 95, 115, 125, 136, 154, 293, 327, 338, 460, 462, 468], "special": [37, 43], "varchar": [37, 39, 42, 43, 62, 66, 68, 71, 74, 76, 144, 170], "faq": 37, "what": [37, 152], "price": 37, "black": [38, 207, 338], "hole": [38, 207, 338], "cassandra": [39, 162, 173, 207, 209, 223, 228, 231, 232, 233, 234, 235, 242, 245, 259, 269, 288, 293, 300, 303, 307, 327, 331, 336, 338, 348, 357, 360, 368, 373, 377, 383, 388, 389, 390, 391, 392, 397, 399, 401, 402, 407, 410, 417, 428, 433, 434, 436], "drop": [39, 41, 54, 60, 63, 498, 499, 500, 501, 502], "delet": [39, 54, 60, 62, 66, 68, 69, 71, 74, 75, 76, 493], "clickhous": [40, 381, 382, 384, 387, 388, 392, 396, 397, 398, 399, 401, 403, 408, 410, 419, 420, 422, 423, 425, 426, 429, 430, 436, 439, 450], "sourc": [40, 42, 62, 66, 68, 71, 74, 76, 151, 282, 471], "append": [40, 42, 55, 66, 68, 69, 71, 74, 75, 76], "domain": [40, 42, 55, 62, 66, 68, 69, 71, 74, 75, 76], "compact": [40, 42, 55, 62, 66, 68, 69, 71, 74, 75, 76], "procedur": [40, 41, 42, 45, 54, 55, 60, 66, 68, 69, 71, 74, 75, 76, 77], "case": [40, 42, 55, 62, 65, 66, 68, 69, 71, 74, 75, 76, 105, 152], "transact": [40, 55, 62, 66, 68, 69, 71, 74, 75, 76, 77, 143, 540], "alter": [40, 41, 45, 54, 55, 65, 68, 71, 74, 75, 76, 478, 479, 480, 481], "schema": [40, 41, 45, 47, 53, 54, 57, 59, 60, 61, 65, 71, 73, 74, 79, 80, 143, 151, 282, 287, 462, 479, 488, 500, 529, 536], "perform": [40, 41, 45, 54, 62, 66, 68, 71, 73, 75, 76, 466], "pushdown": [40, 55, 62, 66, 68, 70, 71, 73, 75, 76, 84, 148, 282], "delta": [41, 401, 402, 403, 405, 406, 409, 410, 412, 413, 414, 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, 442, 444, 445, 446, 447, 448, 449, 450, 451, 452, 453, 454, 455, 456], "lake": [41, 401, 402, 403, 405, 406, 409, 410, 412, 413, 414, 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, 442, 444, 445, 446, 447, 448, 449, 450, 451, 452, 453, 454, 455, 456], "catalog": [41, 64, 77, 136, 137, 151, 287, 462, 526], "author": [41, 43, 52, 54, 172, 462, 469, 471, 516, 524], "check": [41, 54], "unregist": [41, 54], "cach": [41, 48, 73], "vacuum": 41, "renam": [41, 55, 68, 71, 74, 75, 76], "TO": [41, 55, 68, 71, 74, 75, 76, 144], "table_chang": 41, "statist": [41, 45, 54, 62, 66, 71, 76, 99, 118, 149, 538], "fine": 41, "disabl": 41, "extend": 41, "redirect": [41, 45, 54], "druid": [42, 366, 388, 392, 396, 397, 399, 408, 411, 418, 419, 420, 421, 429, 438, 447], "elasticsearch": [43, 335, 341, 344, 347, 348, 351, 354, 355, 356, 357, 359, 360, 361, 364, 365, 367, 369, 373, 377, 381, 384, 386, 387, 388, 390, 391, 392, 393, 400, 408, 415, 416, 421, 425, 438, 454], "raw": [43, 57, 59, 73], "json": [43, 57, 59, 73, 112, 115, 144, 170, 302, 505], "transform": [43, 54], "full": 43, "text": 43, "down": [43, 57], "raw_queri": 43, "sheet": [44, 354, 432, 437, 448], "credenti": [44, 47, 51], "id": [44, 473], "rang": [44, 60, 104, 282, 287], "permiss": 45, "basic": [45, 53, 54, 58], "evolut": [45, 54, 57, 59, 73], "avro": [45, 57, 59, 64, 73], "materi": [45, 54, 77, 143, 478, 486, 498, 514, 528], "avail": [45, 72, 143, 149], "3": [45, 58, 201, 216, 334, 337, 378, 392, 397, 411, 419, 436, 443], "relat": 45, "alluxio": 46, "up": [46, 135, 137], "adl": 47, "gen2": 47, "abf": 47, "servic": [47, 52, 139, 468, 469], "princip": [47, 462, 468], "oauth": [47, 470], "gen1": 47, "legaci": [47, 327], "wasb": 47, "uri": 47, "scheme": 47, "prerequisit": 47, "creat": [47, 58, 60, 91, 137, 472, 486, 487, 488, 489, 490, 491, 528, 529, 530, 531], "benefit": 48, "architectur": [48, 138, 151], "recommend": 48, "object": [48, 49, 67], "metric": 48, "ibm": 49, "compat": 49, "amazon": [51, 284, 288], "keytab": [52, 468], "imperson": [52, 462], "hadoop": [52, 284], "metastor": [52, 64], "thrift": [52, 64, 78, 254, 255, 256, 264, 268, 273, 276, 277, 278, 372, 378, 435], "default": [52, 57, 59, 60, 73], "none": 52, "without": [52, 60], "hudi": [53, 426, 427, 429, 432, 433, 434, 436, 437, 438, 440, 444, 445, 446, 447, 450, 451, 454, 455, 456], "timelin": 53, "iceberg": [54, 64, 369, 373, 375, 379, 380, 381, 383, 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, 412, 413, 414, 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, 442, 444, 445, 446, 447, 448, 449, 450, 451, 452, 453, 454, 455, 456, 457], "level": [54, 57, 136, 462], "expire_snapshot": 54, "remove_orphan_fil": 54, "drop_extended_stat": 54, "snapshot": 54, "manifest": 54, "ref": 54, "comment": [54, 141, 484], "travel": 54, "roll": 54, "back": 54, "previou": 54, "NOT": [54, 104], "null": [54, 104, 112, 116], "constraint": [54, 462], "ignit": [55, 435, 437, 438, 439, 454, 457], "primary_kei": 55, "kafka": [57, 58, 208, 209, 254, 260, 274, 303, 304, 328, 329, 355, 365, 366, 367, 368, 369, 371, 373, 375, 377, 380, 383, 404, 413, 420, 428, 432, 441, 442, 445, 447, 448, 450], "timestamp": [57, 59, 65, 66, 144], "upper": 57, "forc": 57, "hide": [57, 59, 73], "ssl": [57, 466, 469], "keystor": [57, 65, 464, 468, 469], "locat": [57, 59], "truststor": [57, 65, 464], "endpoint": 57, "identif": 57, "registri": 57, "descript": [57, 58, 59, 73, 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, 513, 514, 515, 516, 517, 518, 519, 520, 521, 522, 523, 524, 525, 526, 527, 528, 529, 530, 531, 532, 533, 534, 535, 536, 537, 538, 539, 540, 541, 542, 543, 544], "supplier": 57, "dir": [57, 73], "definit": [57, 59, 65, 73, 510], "messag": [57, 58], "confluent": 57, "subject": [57, 469], "protobuf": 57, "specif": [57, 64, 76], "handl": [57, 66, 69, 71, 75, 112], "oneof": 57, "encod": [57, 97, 101, 109, 129], "csv": [57, 59, 73], "decod": [57, 59, 73], "field": [57, 59, 73], "ani": [57, 104], "tutori": 58, "introduct": [58, 458], "step": [58, 138], "1": [58, 199, 214, 222, 344, 351, 385, 386, 391, 410], "apach": 58, "2": [58, 200, 215, 282, 284, 399, 429, 470], "make": 58, "topic": [58, 98, 115], "known": 58, "4": [58, 202, 455], "5": [58, 203, 380, 439, 446, 453], "add": [58, 288, 464, 475], "6": [58, 204, 329, 390, 407, 426, 448], "all": [58, 104], "onto": 58, "7": [58, 205, 332, 341, 403, 416, 422], "live": 58, "setup": 58, "twitter": 58, "feed": 58, "tweet": 58, "epilogu": 58, "stamp": 58, "kinesi": [59, 368, 379, 384, 445], "region": 59, "fetch": [59, 73, 520], "attempt": 59, "sleep": 59, "offset": [59, 520], "second": [59, 144], "kudu": [60, 327, 344, 354, 359, 366, 368, 373, 374, 384, 392, 398, 399, 402, 403, 410, 414, 420, 434, 441, 449, 450, 452], "behaviour": 60, "emul": 60, "chang": 60, "design": 60, "range_partit": 60, "http_request_log": 61, "mariadb": [62, 408, 410, 411, 412, 418, 419, 420, 422, 423, 429, 438, 439, 450, 454, 457], "glue": 64, "rest": [64, 83], "nessi": 64, "athena": 64, "mongodb": [65, 228, 233, 240, 241, 257, 258, 260, 342, 344, 354, 358, 359, 360, 361, 366, 367, 372, 374, 383, 384, 386, 387, 388, 392, 399, 401, 402, 407, 416, 420, 422, 428, 429, 430, 432, 433, 434, 438, 447, 448, 450, 452, 454, 456, 457], "host": 65, "wait": 65, "idl": 65, "socket": 65, "concern": 65, "replica": 65, "cursor": 65, "objectid": 65, "decim": [66, 68, 69, 71, 75, 108, 144], "orc": [67, 307, 320, 321], "parquet": [67, 192], "oracl": [68, 365, 366, 368, 373, 383, 387, 388, 392, 395, 396, 397, 399, 400, 403, 408, 409, 410, 411, 418, 419, 420, 421, 422, 423, 429, 438, 439, 440, 445, 450, 454, 457], "numer": [68, 76], "datetim": 68, "charact": [68, 76], "synonym": 68, "phoenix": [69, 340, 353, 355, 362, 366, 367, 368, 371, 380, 382, 383, 387, 388, 390, 392, 393, 395, 396, 397, 399, 400, 410, 413, 416, 419, 420, 421, 422, 423, 426, 429, 440], "rowkei": 69, "hbase": 69, "pinot": [70, 368, 384, 388, 389, 390, 392, 396, 402, 405, 406, 414, 419, 420, 421, 425, 427, 428, 441, 455], "grpc": 70, "postgresql": [71, 173, 208, 209, 213, 217, 230, 246, 254, 256, 257, 303, 327, 331, 333, 335, 336, 337, 339, 341, 343, 344, 345, 347, 348, 350, 351, 354, 356, 357, 364, 367, 368, 369, 371, 372, 373, 379, 380, 381, 383, 387, 388, 392, 393, 395, 396, 397, 399, 400, 401, 402, 405, 408, 409, 410, 411, 416, 418, 419, 420, 421, 422, 423, 428, 429, 432, 433, 438, 439, 441, 451, 454], "prometheu": [72, 368, 393, 409, 417, 422], "Not": 72, "exhaust": 72, "your": [72, 137], "bearer": 72, "token": [72, 470, 473], "redi": [73, 208, 220, 260, 280, 329, 368, 406, 408, 414, 445], "delimit": 73, "ttl": 73, "redshift": [74, 254, 256, 327, 333, 335, 336, 343, 345, 372, 379, 380, 381, 383, 387, 388, 392, 396, 397, 399, 408, 410, 411, 418, 419, 420, 422, 423, 429, 432, 438, 439, 441, 448, 449, 454], "singlestor": [75, 395, 396, 397, 399, 401, 408, 410, 411, 419, 420, 422, 429, 435, 438, 450, 454], "bulk": 76, "schema_properti": 77, "table_properti": 77, "materialized_view": 77, "materialized_view_properti": 77, "table_com": 77, "runtim": [77, 136], "optimizer_rule_stat": 77, "trinothriftservic": 78, "implement": [78, 82, 85, 86, 87, 88, 89, 90, 92, 94, 95, 146], "tpcd": [79, 248, 249], "tpch": [80, 248, 287], "develop": 81, "guid": 81, "method": [83, 96], "import": 83, "queryresult": 83, "attribut": 83, "diagnost": 83, "protocolhead": 83, "connectorfactori": [84, 86], "connectormetadata": [84, 94], "top": [84, 148], "n": [84, 114, 148], "connectorsplitmanag": 84, "connectorrecordsetprovid": 84, "carrier": 84, "connectorpagesourceprovid": 84, "connectorpagesinkprovid": [84, 94], "plugin": [86, 87, 88, 93, 300, 468], "exampleconnector": 86, "examplemetadata": 86, "examplesplitmanag": 86, "record": 86, "examplerecordsetprovid": 86, "jdbcclient": 87, "tocolumnmap": 87, "towritemap": 87, "scalar": [88, 96, 520], "parametr": [88, 304], "anoth": 88, "deprec": 88, "AS": [91, 490], "spi": [93, 155, 159, 167, 173, 188, 194, 212, 213, 217, 219, 220, 227, 231, 232, 233, 239, 243, 244, 245, 246, 248, 249, 251, 253, 254, 255, 257, 262, 263, 264, 266, 267, 270, 272, 274, 275, 276, 287, 293, 294, 302, 317, 325, 327, 328, 329, 330, 333, 334, 336, 339, 341, 342, 343, 344, 345, 347, 348, 350, 351, 354, 357, 358, 359, 361, 362, 366, 367, 368, 369, 373, 374, 375, 377, 378, 379, 380, 382, 383, 384, 385, 386, 387, 389, 391, 392, 394, 395, 396, 397, 398, 399, 400, 402, 403, 405, 408, 409, 410, 413, 414, 416, 418, 420, 421, 422, 423, 424, 425, 426, 427, 428, 430, 432, 433, 439, 440, 442, 446, 447, 448, 450, 451, 453, 454, 456, 457], "via": 93, "maven": 93, "deploi": [93, 136], "merg": [94, 511], "rowchangeparadigm": 94, "rowchangeprocessor": 94, "connectormergetablehandl": 94, "connectormergesink": 94, "control": [95, 96, 458, 460, 462, 468, 471], "declar": 96, "The": 96, "constructor": [96, 109, 288], "argument": [96, 112, 126, 510], "descriptor": 96, "semant": [96, 112, 510], "prune": 96, "keep": 96, "empti": [96, 510, 512], "pass": [96, 112, 126], "analyz": [96, 482, 506], "nativ": 97, "contain": [97, 135], "signatur": 97, "dure": 99, "bitwis": [99, 102, 115], "lambda": [99, 113, 115], "binari": [101, 108, 112, 115, 282, 296], "base64": 101, "hex": 101, "float": [101, 118, 144], "point": [101, 118, 144], "hmac": 101, "color": [103, 115], "comparison": [104, 108, 112, 115], "between": [104, 170], "IS": 104, "greatest": 104, "least": 104, "quantifi": [104, 510], "some": 104, "pattern": [104, 115, 476, 510, 512], "like": 104, "IN": [104, 520], "condit": [105, 115], "IF": 105, "coalesc": 105, "nullif": 105, "try": 105, "convers": [106, 107, 115, 118], "miscellan": [106, 284, 288], "zone": [107, 144, 293, 525], "truncat": [107, 541], "java": [107, 136, 464, 468, 469], "extract": [107, 129], "conveni": 107, "liter": [108, 112], "arithmet": [108, 112], "unari": [108, 112], "geospati": [109, 115, 278, 280], "relationship": 109, "test": 109, "accessor": [109, 112], "bing": 109, "tile": 109, "polylin": 109, "hyperloglog": [110, 115, 144], "structur": [110, 120, 123, 127, 139, 144, 512], "languag": [112, 140, 209, 217, 317], "member": 112, "wildcard": 112, "descend": 112, "ceil": 112, "floor": 112, "ab": 112, "keyvalu": 112, "lax": 112, "json_exist": 112, "json_queri": 112, "json_valu": 112, "json_arrai": 112, "return": 112, "json_object": 112, "convent": [112, 126], "uniqu": 112, "list": [114, 115], "A": 114, "b": 114, "c": 114, "d": 114, "e": 114, "f": 114, "g": 114, "h": [114, 343, 389, 412], "i": 114, "j": 114, "k": 114, "l": 114, "m": 114, "o": 114, "p": [114, 144], "q": 114, "r": 114, "s": 114, "t": [114, 115, 127, 144], "u": 114, "v": 114, "w": 114, "x": [114, 284], "y": 114, "z": 114, "machin": [115, 119, 296, 297], "learn": [115, 119, 296, 297], "math": 115, "quantil": [115, 120, 144], "digest": [115, 120, 123, 127, 144], "recognit": [115, 510, 512], "teradata": [115, 128], "uuid": [115, 130, 144], "logic": [116, 505, 510], "effect": 116, "mathemat": 118, "trigonometr": 118, "cumul": 118, "featur": [119, 209, 217, 317], "vector": 119, "classif": 119, "regress": 119, "unicod": [124, 156], "built": 126, "invoc": 126, "resolut": 126, "rank": 131, "glossari": 132, "document": 133, "docker": [135, 374, 378, 379, 386, 397, 399, 408, 420, 421], "clean": [135, 137], "linux": 136, "environ": [136, 462, 468, 469], "python": [136, 390], "launcher": 136, "kubernet": 137, "helm": 137, "own": 137, "yaml": 137, "kind": 137, "improv": [138, 170, 287, 297], "resili": 138, "best": 138, "practic": 138, "consider": 138, "next": 138, "rpm": [139, 264, 268, 278, 328, 331, 334, 336, 341, 342, 349, 353, 360, 373, 378, 380, 389, 397], "packag": [139, 397], "script": 139, "directori": [139, 469], "uninstal": 139, "synopsi": [141, 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, 511, 513, 514, 515, 516, 517, 518, 519, 520, 521, 522, 523, 524, 525, 526, 527, 528, 529, 530, 531, 532, 533, 534, 535, 536, 537, 538, 539, 540, 541, 542, 543, 544], "see": [141, 478, 479, 480, 481, 484, 485, 486, 487, 488, 489, 490, 491, 492, 494, 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, 506, 507, 508, 509, 513, 514, 515, 516, 517, 518, 519, 521, 522, 523, 524, 525, 528, 529, 530, 531, 533, 537, 540, 544], "also": [141, 478, 479, 480, 481, 484, 485, 486, 487, 488, 489, 490, 491, 492, 494, 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, 506, 507, 508, 509, 513, 514, 515, 516, 517, 518, 519, 521, 522, 523, 524, 525, 528, 529, 530, 531, 533, 537, 540, 544], "keyword": [142, 282], "reserv": [142, 282], "tinyint": 144, "smallint": 144, "int": 144, "bigint": 144, "real": 144, "precis": 144, "char": 144, "varbinari": 144, "year": 144, "month": 144, "dai": 144, "ipaddress": 144, "p4hyperloglog": 144, "setdigest": 144, "qdigest": 144, "tdigest": 144, "enumer": 146, "select": [146, 520], "cap": 146, "replic": 146, "syntact": 146, "explain": [147, 505, 506], "derefer": 148, "concept": [151, 473], "model": 151, "releas": [153, 154, 155, 156, 157, 158, 159, 160, 161, 162, 163, 164, 165, 166, 167, 168, 169, 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, 190, 191, 192, 193, 194, 195, 196, 197, 198, 199, 200, 201, 202, 203, 204, 205, 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, 226, 227, 228, 229, 230, 231, 232, 233, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, 276, 277, 278, 279, 280, 281, 282, 283, 284, 285, 286, 287, 288, 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, 321, 322, 323, 324, 325, 326, 327, 328, 329, 330, 331, 332, 333, 334, 335, 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, 431, 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, 442, 443, 444, 445, 446, 447, 448, 449, 450, 451, 452, 453, 454, 455, 456, 457], "note": 153, "2023": [153, 433, 434, 435, 436, 437, 438, 439, 440, 441, 442, 443, 444, 445, 446, 447, 448, 449, 450, 451, 452, 453, 454, 455, 456, 457], "2022": [153, 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, 428, 429, 430, 432], "2021": [153, 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, 394], "2020": [153, 355, 356, 357, 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, 376, 377], "2019": [153, 327, 328, 329, 330, 331, 332, 333, 334, 335, 336, 337, 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, 354], "befor": 153, "0": [154, 155, 156, 157, 158, 159, 160, 161, 162, 163, 164, 165, 166, 167, 168, 169, 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, 190, 191, 192, 193, 194, 195, 196, 197, 198, 199, 200, 201, 202, 203, 204, 205, 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, 226, 227, 228, 229, 230, 231, 232, 233, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, 276, 277, 278, 279, 280, 281, 282, 283, 284, 285, 286, 287, 288, 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, 321, 322, 323, 324, 325, 326, 470], "100": 154, "101": 155, "102": 156, "103": 157, "parallel": 157, "104": 158, "105": 159, "106": 160, "107": 161, "108": 162, "109": 163, "remov": 163, "big": [163, 307], "110": 164, "111": 165, "112": 166, "113": 167, "114": 168, "115": 169, "116": 170, "117": 171, "118": 172, "119": 173, "verifi": [173, 178, 206, 209, 212, 213, 235, 277, 278, 279, 297, 320, 327, 462, 466, 472, 475], "120": 174, "121": 175, "122": 176, "123": 177, "124": 178, "125": 179, "126": 180, "127": 181, "128": 182, "129": 183, "awar": 183, "130": 184, "131": 185, "132": 186, "133": 187, "134": 188, "135": 189, "136": 190, "137": 191, "138": 192, "new": [192, 307], "reader": [192, 307], "139": 193, "140": 194, "141": 195, "142": 196, "143": 197, "144": [198, 199, 200, 201, 202, 203, 204, 205], "145": 206, "146": 207, "147": 208, "148": 209, "149": 210, "150": 211, "151": 212, "152": [213, 214, 215, 216], "153": 217, "pluggabl": 217, "blackhol": [217, 258, 337, 429, 450, 452], "154": 218, "155": 219, "156": 220, "157": [221, 222], "158": 223, "159": 224, "160": 225, "161": 226, "162": 227, "163": 228, "164": 229, "165": 230, "166": 231, "167": 232, "168": 233, "169": 234, "170": 235, "171": 236, "172": 237, "173": 238, "174": 239, "175": 240, "176": 241, "177": 242, "178": 243, "179": 244, "180": 245, "181": 246, "182": 247, "183": 248, "184": 249, "185": 250, "186": 251, "187": 252, "188": 253, "189": 254, "shang": 254, "190": 255, "191": 256, "192": 257, "193": 258, "194": 259, "195": 260, "196": 261, "197": 262, "198": 263, "199": 264, "200": 265, "201": 266, "202": 267, "203": 268, "204": 269, "205": 270, "206": 271, "207": 272, "208": 273, "209": 274, "210": 275, "211": 276, "212": 277, "213": 278, "214": 279, "215": 280, "54": 281, "55": 282, "rc": 282, "4x": 282, "effici": 282, "simpl": 282, "json_array_get": 282, "show": [282, 526, 527, 528, 529, 530, 531, 532, 533, 534, 535, 536, 537, 538, 539], "56": 283, "creation": 283, "cross": [283, 520], "57": 284, "58": 285, "59": 286, "60": 287, "compound": 287, "bug": [287, 293], "61": 288, "62": 289, "63": 290, "64": 291, "65": 292, "66": 293, "67": 294, "68": 295, "69": 296, "prevent": 296, "work": 296, "datasourc": 296, "raptor": [296, 310, 327, 368, 410, 420], "70": 297, "dual": 297, "presto": 297, "71": 298, "72": 299, "73": 300, "74": 301, "bytecod": 301, "compil": 301, "75": 302, "76": 303, "77": 304, "stream": 304, "78": 305, "79": 306, "80": 307, "onli": [307, 460], "81": 308, "82": 309, "83": 310, "84": 311, "85": 312, "86": 313, "87": 314, "88": 315, "89": 316, "90": 317, "91": 318, "92": 319, "93": 320, "94": 321, "95": 322, "96": 323, "97": 324, "98": 325, "99": 326, "300": 327, "22": [327, 346, 394, 414, 418, 447], "jan": [327, 328, 355, 356, 378, 395, 396, 433], "mbean": [327, 378], "301": 328, "31": [328, 340], "302": 329, "feb": [329, 330, 331, 357, 379, 397, 398, 434, 435], "303": 330, "13": [330, 404, 417, 427, 449], "mongo": 330, "304": 331, "27": [331, 353, 388], "305": 332, "mar": [332, 333, 358, 380, 381, 399, 400, 401, 402, 436, 437, 438], "306": 333, "16": [333, 349, 358, 363, 398, 408, 434], "307": 334, "apr": [334, 335, 336, 359, 382, 383, 403, 404, 405, 406, 439, 440, 441, 442], "308": 335, "11": [335, 395, 456], "309": 336, "25": [336, 364, 370, 374, 409, 433], "310": 337, "mai": [337, 338, 339, 340, 360, 361, 384, 407, 408, 409, 443, 444, 445], "311": 338, "14": [338, 342, 352, 362, 375, 393], "312": 339, "29": [339, 361, 415, 421, 438], "313": 340, "314": 341, "jun": [341, 342, 362, 363, 364, 385, 410, 411, 412, 413, 414, 415, 446, 447], "315": 342, "316": 343, "8": [343, 367, 368, 382, 412, 437], "jul": [343, 365, 366, 386, 387, 416, 417, 418, 448, 449], "tpc": [343, 389, 412], "ds": 343, "317": 344, "aug": [344, 345, 367, 388, 419, 420, 421, 450, 451, 452], "318": 345, "26": [345, 428, 454], "319": 346, "sep": [346, 368, 369, 370, 389, 422, 423, 424, 425, 453, 454], "320": 347, "10": [347, 355, 373, 444, 450], "oct": [347, 348, 349, 350, 371, 372, 390, 426, 427, 428, 455, 456, 457], "321": 348, "15": [348, 413, 423, 430], "322": 349, "323": 350, "23": [350, 356, 372, 435], "324": 351, "nov": [351, 352, 353, 373, 374, 391, 429, 430], "325": 352, "326": 353, "327": 354, "20": [354, 389, 457], "dec": [354, 375, 376, 377, 392, 393, 394, 432], "328": 355, "329": 356, "330": 357, "18": 357, "331": 358, "332": 359, "08": 359, "333": 360, "04": 360, "334": 361, "memsql": [361, 372, 380, 381, 383, 387, 388, 391, 392, 393, 395, 396, 397, 399, 401, 408, 410, 411, 419, 420, 422, 429], "335": 362, "336": 363, "337": 364, "338": 365, "07": 365, "339": 366, "21": [366, 384, 405, 424], "340": 367, "341": 368, "342": 369, "24": [369, 396, 452], "343": 370, "344": 371, "9": [371, 379, 400], "345": 372, "346": 373, "347": 374, "imag": [374, 378, 379, 386, 397, 399, 408, 420, 421], "348": 375, "349": 376, "28": [376, 377, 402, 406, 425, 432, 442], "350": 377, "351": 378, "352": 379, "353": 380, "354": 381, "19": [381, 441], "355": 382, "356": 383, "30": [383, 387], "357": 384, "358": 385, "359": 386, "360": 387, "361": 388, "362": 389, "363": 390, "364": 391, "365": 392, "366": 393, "367": 394, "368": 395, "369": 396, "370": 397, "371": 398, "372": 399, "373": 400, "374": 401, "17": [401, 420, 445, 451], "375": 402, "376": 403, "377": 404, "378": 405, "379": 406, "380": 407, "381": 408, "382": 409, "383": 410, "384": 411, "385": 412, "386": 413, "387": 414, "388": 415, "389": 416, "390": 417, "391": 418, "392": 419, "393": 420, "394": 421, "395": 422, "396": 423, "397": 424, "398": 425, "399": 426, "400": 427, "401": 428, "402": 429, "403": 430, "404": 431, "405": 432, "406": 433, "407": 434, "408": 435, "409": 436, "410": 437, "411": 438, "412": 439, "413": 440, "12": 440, "414": 441, "415": 442, "416": 443, "417": 444, "418": 445, "419": 446, "420": 447, "421": 448, "422": 449, "423": 450, "424": 451, "425": 452, "426": 453, "427": 454, "428": 455, "429": 456, "430": 457, "insid": [458, 471], "visibl": 462, "mask": 462, "jk": 464, "inspect": [464, 465, 475], "valid": [464, 465, 505], "extra": 464, "pem": [464, 465], "section": 465, "commun": 466, "mit": 468, "ldap": 469, "activ": 469, "openldap": 469, "membership": 469, "issu": 469, "common": 469, "cert": 469, "certificateexcept": 469, "No": 469, "present": 469, "jdk": 469, "upgrad": 469, "openid": 470, "discoveri": 470, "oauth2": 470, "flow": 470, "aspect": 471, "workflow": 471, "salesforc": 473, "com": 473, "organ": 473, "approach": 475, "balanc": 475, "termin": 475, "directli": 475, "receiv": 475, "invalid": 475, "place": 475, "self": 475, "sign": 475, "call": 483, "commit": 485, "role": [487, 499, 508, 518, 522, 534, 535], "dealloc": 492, "deni": 494, "describ": [495, 496, 497], "immedi": 504, "io": 505, "grant": [507, 508, 533, 534], "match_recogn": 510, "measur": 510, "after": 510, "skip": 510, "permut": 510, "anchor": 510, "end": 510, "exclus": 510, "classifi": 510, "match_numb": 510, "navig": 510, "nest": 510, "final": 510, "evalu": 510, "unmatch": [510, 512], "reset": [515, 516], "revok": [517, 518], "rollback": 519, "recurs": 520, "BY": 520, "cube": 520, "rollup": 520, "combin": 520, "have": 520, "intersect": 520, "except": 520, "first": 520, "tablesampl": 520, "later": 520, "qualifi": 520, "subqueri": 520, "exist": 520, "stat": 538}, "envversion": {"sphinx.domains.c": 2, "sphinx.domains.changeset": 1, "sphinx.domains.citation": 1, "sphinx.domains.cpp": 6, "sphinx.domains.index": 1, "sphinx.domains.javascript": 2, "sphinx.domains.math": 2, "sphinx.domains.python": 3, "sphinx.domains.rst": 2, "sphinx.domains.std": 2, "sphinx": 56}}) \ No newline at end of file diff --git a/430/security.html b/430/security.html new file mode 100644 index 000000000..d9f92b426 --- /dev/null +++ b/430/security.html @@ -0,0 +1,622 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Security — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Security#

+
+

Introduction#

+ +
+
+

Cluster access security#

+ +
+
+

Authentication#

+ +
+
+

User name management#

+ +
+
+

Access control#

+ +
+
+

Security inside the cluster#

+ +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/security/authentication-types.html b/430/security/authentication-types.html new file mode 100644 index 000000000..24faa7625 --- /dev/null +++ b/430/security/authentication-types.html @@ -0,0 +1,615 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Authentication types — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Authentication types#

+

Trino supports multiple authentication types to ensure all users of the system +are authenticated. Different authenticators allow user management in one or more +systems. Using TLS and a configured shared secret are required for all authentications types.

+

You can configure one or more authentication types with the +http-server.authentication.type property. The following authentication types +and authenticators are available:

+ +

Get started with a basic password authentication configuration backed by a +password file:

+
http-server.authentication.type=PASSWORD
+
+
+
+

Multiple authentication types#

+

You can use multiple authentication types, separated with commas in the +configuration:

+
http-server.authentication.type=PASSWORD,CERTIFICATE
+
+
+

Authentication is performed in order of the entries, and first successful +authentication results in access, using the mapped user +from that authentication method.

+
+
+

Multiple password authenticators#

+

You can use multiple password authenticator types by referencing multiple +configuration files:

+
http-server.authentication.type=PASSWORD
+password-authenticator.config-files=etc/ldap1.properties,etc/ldap2.properties,etc/password.properties
+
+
+

In the preceding example, the configuration files ldap1.properties and +ldap2.properties are regular LDAP authenticator configuration files. The password.properties is a password file authenticator configuration file.

+

Relative paths to the installation directory or absolute paths can be used.

+

User authentication credentials are first validated against the LDAP server from +ldap1, then the separate server from ldap2, and finally the password +file. First successful authentication results in access, and no further +authenticators are called.

+
+
+

Multiple header authenticators#

+

You can use multiple header authenticator types by referencing multiple +configuration files:

+
http-server.authentication.type=HEADER
+header-authenticator.config-files=etc/xfcc.properties,etc/azureAD.properties
+
+
+

Relative paths to the installation directory or absolute paths can be used.

+

The pre-configured headers are first validated against the xfcc authenticator, +then the azureAD authenticator. First successful authentication results in access, +and no further authenticators are called.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/security/built-in-system-access-control.html b/430/security/built-in-system-access-control.html new file mode 100644 index 000000000..d13703403 --- /dev/null +++ b/430/security/built-in-system-access-control.html @@ -0,0 +1,604 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + System access control — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

System access control#

+

A system access control enforces authorization at a global level, +before any connector level authorization. You can use one of the built-in +implementations in Trino, or provide your own by following the guidelines in +System access control.

+

To use a system access control, add an etc/access-control.properties file +with the following content and the desired system access control name on all +cluster nodes:

+
access-control.name=allow-all
+
+
+

Multiple system access control implementations may be configured at once +using the access-control.config-files configuration property. It should +contain a comma separated list of the access control property files to use +(rather than the default etc/access-control.properties).

+

Trino offers the following built-in system access control implementations:

+ ++++ + + + + + + + + + + + + + + + + + + + +

Name

Description

default

All operations are permitted, except for user impersonation and triggering +Graceful shutdown.

+

This is the default access control if none are configured.

+

allow-all

All operations are permitted.

read-only

Operations that read data or metadata are permitted, but none of the +operations that write data or metadata are allowed.

file

Authorization rules are specified in a config file. See +File-based access control.

+

If you want to limit access on a system level in any other way than the ones +listed above, you must implement a custom System access control.

+

Access control must be configured on the coordinator. Authorization for +operations on specific worker nodes, such a triggering +Graceful shutdown, must also be configured on all workers.

+
+

Read only system access control#

+

This access control allows any operation that reads data or +metadata, such as SELECT or SHOW. Setting system level or catalog level +session properties is also permitted. However, any operation that writes data or +metadata, such as CREATE, INSERT or DELETE, is prohibited. +To use this access control, add an etc/access-control.properties +file with the following contents:

+
access-control.name=read-only
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/security/certificate.html b/430/security/certificate.html new file mode 100644 index 000000000..c679ffa40 --- /dev/null +++ b/430/security/certificate.html @@ -0,0 +1,650 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Certificate authentication — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Certificate authentication#

+

You can configure Trino to support client-provided certificates validated by the +Trino server on initial connection.

+
+

Important

+

This authentication method is only provided to support sites that have an +absolute requirement for client authentication and already have client +certificates for each client. Sites in this category have an existing PKI +infrastructure, possibly including an onsite Certificate Authority (CA).

+

This feature is not appropriate for sites that need to generate a set of +client certificates in order to use this authentication type. Consider +instead using another authentication type.

+
+

Using TLS and a configured shared secret is required for certificate authentication.

+
+

Using certificate authentication#

+

All clients connecting with TLS/HTTPS go through the following initial steps:

+
    +
  1. The client attempts to contact the coordinator.

  2. +
  3. The coordinator returns its certificate to the client.

  4. +
  5. The client validates the server’s certificate using the client’s trust store.

  6. +
+

A cluster with certificate authentication enabled goes through the following +additional steps:

+
    +
  1. The coordinator asks the client for its certificate.

  2. +
  3. The client responds with its certificate.

  4. +
  5. The coordinator verifies the client’s certificate, using the coordinator’s +trust store.

  6. +
+

Several rules emerge from these steps:

+
    +
  • Trust stores used by clients must include the certificate of the signer of +the coordinator’s certificate.

  • +
  • Trust stores used by coordinators must include the certificate of the signer +of client certificates.

  • +
  • The trust stores used by the coordinator and clients do not need to be the +same.

  • +
  • The certificate that verifies the coordinator does not need to be the same as +the certificate verifying clients.

  • +
+

Trino validates certificates based on the distinguished name (DN) from the +X.509 Subject field. You can use user mapping to map the subject DN to a Trino user name.

+

There are three levels of client certificate support possible. From the point of +view of the server:

+
    +
  • The server does not require a certificate from clients.

  • +
  • The server asks for a certificate from clients, but allows connection without one.

  • +
  • The server must have a certificate from clients to allow connection.

  • +
+

Trino’s client certificate support is the middle type. It asks for a certificate +but allows connection if another authentication method passes.

+
+
+

Certificate authentication configuration#

+

Enable certificate authentication by setting the Certificate authentication type in etc/config.properties:

+
http-server.authentication.type=CERTIFICATE
+
+
+

You can specify certificate authentication along with another authenticaton +method, such as PASSWORD. In this case, authentication is performed in the +order of entries, and the first successful authentication results in access. +For example, the following setting shows the use of two authentication types:

+
http-server.authentication.type=CERTIFICATE,PASSWORD
+
+
+

The following configuration properties are also available:

+ + ++++ + + + + + + + + + + + + + +
Configuration properties#

Property name

Description

http-server.authentication.certificate.user-mapping.pattern

A regular expression pattern to map all user +names for this authentication type to the format +expected by Trino.

http-server.authentication.certificate.user-mapping.file

The path to a JSON file that contains a set of user mapping +rules for this authentication type.

+
+
+

Use certificate authentication with clients#

+

When using the Trino CLI, specify the +--keystore-path and --keystore-password options as described +in Certificate authentication.

+

When using the Trino JDBC driver to connect to a +cluster with certificate authentication enabled, use the SSLKeyStoreType and +SSLKeyStorePassword parameters to specify +the path to the client’s certificate and its password, if any.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/security/file-system-access-control.html b/430/security/file-system-access-control.html new file mode 100644 index 000000000..5ba7ef35a --- /dev/null +++ b/430/security/file-system-access-control.html @@ -0,0 +1,1778 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + File-based access control — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

File-based access control#

+

To secure access to data in your cluster, you can implement file-based access +control where access to data and operations is defined by rules declared in +manually-configured JSON files.

+

There are two types of file-based access control:

+
    +
  • System-level access control uses the access control plugin with a single +JSON file that specifies authorization rules for the whole cluster.

  • +
  • Catalog-level access control uses individual JSON files for each catalog +for granular control over the data in that catalog, including column-level +authorization.

  • +
+
+

System-level access control files#

+

The access control plugin allows you to specify authorization rules for the +cluster in a single JSON file.

+
+

Configuration#

+

To use the access control plugin, add an etc/access-control.properties file +containing two required properties: access-control.name, which must be set +to file, and security.config-file, which must be set to the location +of the config file. The configuration file location can either point to the local +disc or to a http endpoint. For example, if a config file named rules.json resides +in etc, add an etc/access-control.properties with the following +contents:

+
access-control.name=file
+security.config-file=etc/rules.json
+
+
+

If the config should be loaded via the http endpoint http://trino-test/config and +is wrapped into a JSON object and available via the data key etc/access-control.properties +should look like this:

+
access-control.name=file
+security.config-file=http://trino-test/config
+security.json-pointer=/data
+
+
+

The config file is specified in JSON format. It contains rules that define which +users have access to which resources. The rules are read from top to bottom and +the first matching rule is applied. If no rule matches, access is denied. A JSON +pointer (RFC 6901) can be specified using the security.json-pointer property +to specify a nested object inside the JSON content containing the rules. Per default, +the file is assumed to contain a single object defining the rules rendering +the specification of security.json-pointer unnecessary in that case.

+
+
+

Refresh#

+

By default, when a change is made to the JSON rules file, Trino must be +restarted to load the changes. There is an optional property to refresh the +properties without requiring a Trino restart. The refresh period is specified in +the etc/access-control.properties:

+
security.refresh-period=1s
+
+
+
+
+

Catalog, schema, and table access#

+

Access to catalogs, schemas, tables, and views is controlled by the catalog, +schema, and table rules. The catalog rules are coarse-grained rules used to +restrict all access or write access to catalogs. They do not explicitly grant +any specific schema or table permissions. The table and schema rules are used to +specify who can create, drop, alter, select, insert, delete, etc. for schemas +and tables.

+
+

Note

+

These rules do not apply to system-defined tables in the +information_schema schema.

+
+

For each rule set, permission is based on the first matching rule read from top +to bottom. If no rule matches, access is denied. If no rules are provided at +all, then access is granted.

+

The following table summarizes the permissions required for each SQL command:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

SQL command

Catalog

Schema

Table

Note

SHOW CATALOGS

Always allowed

SHOW SCHEMAS

read-only

any*

any*

Allowed if catalog is visible

SHOW TABLES

read-only

any*

any*

Allowed if schema visible

CREATE SCHEMA

read-only

owner

DROP SCHEMA

all

owner

SHOW CREATE SCHEMA

all

owner

ALTER SCHEMA … RENAME TO

all

owner*

Ownership is required on both old and new schemas

ALTER SCHEMA … SET AUTHORIZATION

all

owner

CREATE TABLE

all

owner

DROP TABLE

all

owner

ALTER TABLE … RENAME TO

all

owner*

Ownership is required on both old and new tables

ALTER TABLE … SET PROPERTIES

all

owner

CREATE VIEW

all

owner

DROP VIEW

all

owner

ALTER VIEW … RENAME TO

all

owner*

Ownership is required on both old and new views

REFRESH MATERIALIZED VIEW

all

update

COMMENT ON TABLE

all

owner

COMMENT ON COLUMN

all

owner

ALTER TABLE … ADD COLUMN

all

owner

ALTER TABLE … DROP COLUMN

all

owner

ALTER TABLE … RENAME COLUMN

all

owner

SHOW COLUMNS

all

any

SELECT FROM table

read-only

select

SELECT FROM view

read-only

select, grant_select

INSERT INTO

all

insert

DELETE FROM

all

delete

UPDATE

all

update

+

Permissions required for executing functions:

+ ++++++ + + + + + + + + + + + + + + + + + + + +

SQL command

Catalog

Function permission

Note

SELECT function()

execute, grant_execute*

grant_execute is required when the function is used in a SECURITY DEFINER +view.

SELECT FROM TABLE(table_function())

all

execute, grant_execute*

grant_execute is required when the function is used in a SECURITY DEFINER +view.

+
+

Visibility#

+

For a catalog, schema, or table to be visible in a SHOW command, the user +must have at least one permission on the item or any nested item. The nested +items do not need to already exist as any potential permission makes the item +visible. Specifically:

+
    +
  • catalog: Visible if user is the owner of any nested schema, has +permissions on any nested table or function, or has permissions to +set session properties in the catalog.

  • +
  • schema: Visible if the user is the owner of the schema, or has permissions +on any nested table or function.

  • +
  • table: Visible if the user has any permissions on the table.

  • +
+
+
+

Catalog rules#

+

Each catalog rule is composed of the following fields:

+
    +
  • user (optional): regex to match against user name. Defaults to .*.

  • +
  • role (optional): regex to match against role names. Defaults to .*.

  • +
  • group (optional): regex to match against group names. Defaults to .*.

  • +
  • catalog (optional): regex to match against catalog name. Defaults to +.*.

  • +
  • allow (required): string indicating whether a user has access to the +catalog. This value can be all, read-only or none, and defaults to +none. Setting this value to read-only has the same behavior as the +read-only system access control plugin.

  • +
+

In order for a rule to apply the user name must match the regular expression +specified in user attribute.

+

For role names, a rule can be applied if at least one of the currently enabled +roles matches the role regular expression.

+

For group names, a rule can be applied if at least one group name of this user +matches the group regular expression.

+

The all value for allow means these rules do not restrict access in any +way, but the schema and table rules can restrict access.

+
+

Note

+

By default, all users have access to the system catalog. You can +override this behavior by adding a rule.

+

Boolean true and false are also supported as legacy values for +allow, to support backwards compatibility. true maps to all, +and false maps to none.

+
+

For example, if you want to allow only the role admin to access the +mysql and the system catalog, allow users from the finance and +human_resources groups access to postgres catalog, allow all users to +access the hive catalog, and deny all other access, you can use the +following rules:

+
{
+  "catalogs": [
+    {
+      "role": "admin",
+      "catalog": "(mysql|system)",
+      "allow": "all"
+    },
+    {
+      "group": "finance|human_resources",
+      "catalog": "postgres",
+      "allow": true
+    },
+    {
+      "catalog": "hive",
+      "allow": "all"
+    },
+    {
+      "user": "alice",
+      "catalog": "postgresql",
+      "allow": "read-only"
+    },
+    {
+      "catalog": "system",
+      "allow": "none"
+    }
+  ]
+}
+
+
+

For group-based rules to match, users need to be assigned to groups by a +Group provider.

+
+
+

Schema rules#

+

Each schema rule is composed of the following fields:

+
    +
  • user (optional): regex to match against user name. Defaults to .*.

  • +
  • role (optional): regex to match against role names. Defaults to .*.

  • +
  • group (optional): regex to match against group names. Defaults to .*.

  • +
  • catalog (optional): regex to match against catalog name. Defaults to +.*.

  • +
  • schema (optional): regex to match against schema name. Defaults to +.*.

  • +
  • owner (required): boolean indicating whether the user is to be considered +an owner of the schema. Defaults to false.

  • +
+

For example, to provide ownership of all schemas to role admin, treat all +users as owners of the default.default schema and prevent user guest +from ownership of any schema, you can use the following rules:

+
{
+  "schemas": [
+    {
+      "role": "admin",
+      "schema": ".*",
+      "owner": true
+    },
+    {
+      "user": "guest",
+      "owner": false
+    },
+    {
+      "catalog": "default",
+      "schema": "default",
+      "owner": true
+    }
+  ]
+}
+
+
+
+
+

Table rules#

+

Each table rule is composed of the following fields:

+
    +
  • user (optional): regex to match against user name. Defaults to .*.

  • +
  • role (optional): regex to match against role names. Defaults to .*.

  • +
  • group (optional): regex to match against group names. Defaults to .*.

  • +
  • catalog (optional): regex to match against catalog name. Defaults to +.*.

  • +
  • schema (optional): regex to match against schema name. Defaults to .*.

  • +
  • table (optional): regex to match against table names. Defaults to .*.

  • +
  • privileges (required): zero or more of SELECT, INSERT, +DELETE, UPDATE, OWNERSHIP, GRANT_SELECT

  • +
  • columns (optional): list of column constraints.

  • +
  • filter (optional): boolean filter expression for the table.

  • +
  • filter_environment (optional): environment use during filter evaluation.

  • +
+
+
+

Column constraint#

+

These constraints can be used to restrict access to column data.

+
    +
  • name: name of the column.

  • +
  • allow (optional): if false, column can not be accessed.

  • +
  • mask (optional): mask expression applied to column.

  • +
  • mask_environment (optional): environment use during mask evaluation.

  • +
+
+
+

Filter and mask environment#

+
    +
  • user (optional): username for checking permission of subqueries in mask.

  • +
+
+

Note

+

These rules do not apply to information_schema.

+

mask can contain conditional expressions such as IF or CASE, which achieves conditional masking.

+
+

The example below defines the following table access policy:

+
    +
  • Role admin has all privileges across all tables and schemas

  • +
  • User banned_user has no privileges

  • +
  • All users have SELECT privileges on default.hr.employees, but the +table is filtered to only the row for the current user.

  • +
  • All users have SELECT privileges on all tables in the default.default +schema, except for the address column which is blocked, and ssn which +is masked.

  • +
+
{
+  "tables": [
+    {
+      "role": "admin",
+      "privileges": ["SELECT", "INSERT", "DELETE", "UPDATE", "OWNERSHIP"]
+    },
+    {
+      "user": "banned_user",
+      "privileges": []
+    },
+    {
+      "catalog": "default",
+      "schema": "hr",
+      "table": "employee",
+      "privileges": ["SELECT"],
+      "filter": "user = current_user",
+      "filter_environment": {
+        "user": "system_user"
+      }
+    },
+    {
+      "catalog": "default",
+      "schema": "default",
+      "table": ".*",
+      "privileges": ["SELECT"],
+      "columns" : [
+         {
+            "name": "address",
+            "allow": false
+         },
+         {
+            "name": "SSN",
+            "mask": "'XXX-XX-' + substring(credit_card, -4)",
+            "mask_environment": {
+              "user": "system_user"
+            }
+         }
+      ]
+    }
+  ]
+}
+
+
+
+
+

Function rules#

+

These rules control the user’s ability to execute functions.

+
+

Note

+

Users always have access to functions in the system.builtin schema, and +you cannot override this behavior by adding a rule.

+
+

Each function rule is composed of the following fields:

+
    +
  • user (optional): regular expression to match against user name. +Defaults to .*.

  • +
  • role (optional): regular expression to match against role names. +Defaults to .*.

  • +
  • group (optional): regular expression to match against group names. +Defaults to .*.

  • +
  • catalog (optional): regular expression to match against catalog name. +Defaults to .*.

  • +
  • schema (optional): regular expression to match against schema name. +Defaults to .*.

  • +
  • function (optional): regular expression to match against function names. +Defaults to .*.

  • +
  • privileges (required): zero or more of EXECUTE, GRANT_EXECUTE.

  • +
+

To explicitly allow the system builtin functions in queries (and SECURITY +DEFINER views), you can use the following rule:

+
{
+  "functions": [
+    {
+      "catalog": "system",
+      "schema": "builtin",
+      "privileges": [
+        "EXECUTE",
+        "GRANT_EXECUTE"
+      ]
+    }
+  ]
+}
+
+
+

Care should be taken when granting permission to the system schema of any +catalog, as this is the schema Trino uses for table function such as query. +These table functions can be used to access or modify the underlying data of +the catalog.

+

The following example allows the admin user to execute query table +function from any catalog:

+
{
+  "functions": [
+    {
+      "catalog": "system",
+      "schema": "builtin",
+      "privileges": [
+        "EXECUTE",
+        "GRANT_EXECUTE"
+      ]
+    },
+    {
+      "user": "admin",
+      "schema": "system",
+      "function": "query",
+      "privileges": [
+        "EXECUTE"
+      ]
+    }
+  ]
+}
+
+
+
+
+

Verify configuration#

+

To verify the system-access control file is configured properly, set the +rules to completely block access to all users of the system:

+
{
+  "catalogs": [
+    {
+      "catalog": "system",
+      "allow": "none"
+    }
+  ]
+}
+
+
+

Restart your cluster to activate the rules for your cluster. With the +Trino CLI run a query to test authorization:

+
trino> SELECT * FROM system.runtime.nodes;
+Query 20200824_183358_00000_c62aw failed: Access Denied: Cannot access catalog system
+
+
+

Remove these rules and restart the Trino cluster.

+
+
+
+

Session property rules#

+

These rules control the ability of a user to set system and catalog session +properties. The user is granted or denied access, based on the first matching +rule, read from top to bottom. If no rules are specified, all users are allowed +set any session property. If no rule matches, setting the session property is +denied. System session property rules are composed of the following fields:

+
    +
  • user (optional): regex to match against user name. Defaults to .*.

  • +
  • role (optional): regex to match against role names. Defaults to .*.

  • +
  • group (optional): regex to match against group names. Defaults to .*.

  • +
  • property (optional): regex to match against the property name. Defaults to +.*.

  • +
  • allow (required): boolean indicating if the setting the session +property should be allowed.

  • +
+

The catalog session property rules have the additional field:

+
    +
  • catalog (optional): regex to match against catalog name. Defaults to +.*.

  • +
+

The example below defines the following table access policy:

+
    +
  • Role admin can set all session property

  • +
  • User banned_user can not set any session properties

  • +
  • All users can set the resource_overcommit system session property, and the +bucket_execution_enabled session property in the hive catalog.

  • +
+
{
+    "system_session_properties": [
+        {
+            "role": "admin",
+            "allow": true
+        },
+        {
+            "user": "banned_user",
+            "allow": false
+        },
+        {
+            "property": "resource_overcommit",
+            "allow": true
+        }
+    ],
+    "catalog_session_properties": [
+        {
+            "role": "admin",
+            "allow": true
+        },
+        {
+            "user": "banned_user",
+            "allow": false
+        },
+        {
+            "catalog": "hive",
+            "property": "bucket_execution_enabled",
+            "allow": true
+        }
+    ]
+}
+
+
+
+
+

Query rules#

+

These rules control the ability of a user to execute, view, or kill a query. The +user is granted or denied access, based on the first matching rule read from top +to bottom. If no rules are specified, all users are allowed to execute queries, +and to view or kill queries owned by any user. If no rule matches, query +management is denied. Each rule is composed of the following fields:

+
    +
  • user (optional): regex to match against user name. Defaults to .*.

  • +
  • role (optional): regex to match against role names. Defaults to .*.

  • +
  • group (optional): regex to match against group names. Defaults to .*.

  • +
  • queryOwner (optional): regex to match against the query owner name. +Defaults to .*.

  • +
  • allow (required): set of query permissions granted to user. Values: +execute, view, kill

  • +
+
+

Note

+

Users always have permission to view or kill their own queries.

+

A rule that includes queryOwner may not include the execute access mode. +Queries are only owned by a user once their execution has begun.

+
+

For example, if you want to allow the role admin full query access, allow +the user alice to execute and kill queries, allow members of the group +contractors to view queries owned by users alice or dave, allow any +user to execute queries, and deny all other access, you can use the following +rules:

+
{
+  "queries": [
+    {
+      "role": "admin",
+      "allow": ["execute", "kill", "view"]
+    },
+    {
+      "user": "alice",
+      "allow": ["execute", "kill"]
+    },
+    {
+      "group": "contractors",
+      "queryOwner": "alice|dave",
+      "allow": ["view"]
+    },
+    {
+      "allow": ["execute"]
+    }
+  ]
+}
+
+
+
+
+

Impersonation rules#

+

These rules control the ability of a user to impersonate another user. In +some environments it is desirable for an administrator (or managed system) to +run queries on behalf of other users. In these cases, the administrator +authenticates using their credentials, and then submits a query as a different +user. When the user context is changed, Trino verifies that the administrator +is authorized to run queries as the target user.

+

When these rules are present, the authorization is based on the first matching +rule, processed from top to bottom. If no rules match, the authorization is +denied. If impersonation rules are not present but the legacy principal rules +are specified, it is assumed impersonation access control is being handled by +the principal rules, so impersonation is allowed. If neither impersonation nor +principal rules are defined, impersonation is not allowed.

+

Each impersonation rule is composed of the following fields:

+
    +
  • original_user (optional): regex to match against the user requesting the +impersonation. Defaults to .*.

  • +
  • original_role (optional): regex to match against role names of the +requesting impersonation. Defaults to .*.

  • +
  • new_user (required): regex to match against the user to impersonate. Can +contain references to subsequences captured during the match against +original_user, and each reference is replaced by the result of evaluating +the corresponding group respectively.

  • +
  • allow (optional): boolean indicating if the authentication should be +allowed. Defaults to true.

  • +
+

The impersonation rules are a bit different than the other rules: The attribute +new_user is required to not accidentally prevent more access than intended. +Doing so it was possible to make the attribute allow optional.

+

The following example allows the admin role, to impersonate any user, except +for bob. It also allows any user to impersonate the test user. It also +allows a user in the form team_backend to impersonate the +team_backend_sandbox user, but not arbitrary users:

+
{
+    "impersonation": [
+        {
+            "original_role": "admin",
+            "new_user": "bob",
+            "allow": false
+        },
+        {
+            "original_role": "admin",
+            "new_user": ".*"
+        },
+        {
+            "original_user": ".*",
+            "new_user": "test"
+        },
+        {
+            "original_user": "team_(.*)",
+            "new_user": "team_$1_sandbox",
+            "allow": true
+        }
+    ]
+}
+
+
+
+
+

Principal rules#

+
+

Warning

+

Principal rules are deprecated. Instead, use User mapping +which specifies how a complex authentication user name is mapped to a simple +user name for Trino, and impersonation rules defined above.

+
+

These rules serve to enforce a specific matching between a principal and a +specified user name. The principal is granted authorization as a user, based +on the first matching rule read from top to bottom. If no rules are specified, +no checks are performed. If no rule matches, user authorization is denied. +Each rule is composed of the following fields:

+
    +
  • principal (required): regex to match and group against principal.

  • +
  • user (optional): regex to match against user name. If matched, it +grants or denies the authorization based on the value of allow.

  • +
  • principal_to_user (optional): replacement string to substitute against +principal. If the result of the substitution is same as the user name, it +grants or denies the authorization based on the value of allow.

  • +
  • allow (required): boolean indicating whether a principal can be authorized +as a user.

  • +
+
+

Note

+

You would at least specify one criterion in a principal rule. If you specify +both criteria in a principal rule, it returns the desired conclusion when +either of criteria is satisfied.

+
+

The following implements an exact matching of the full principal name for LDAP +and Kerberos authentication:

+
{
+  "principals": [
+    {
+      "principal": "(.*)",
+      "principal_to_user": "$1",
+      "allow": true
+    },
+    {
+      "principal": "([^/]+)(/.*)?@.*",
+      "principal_to_user": "$1",
+      "allow": true
+    }
+  ]
+}
+
+
+

If you want to allow users to use the exact same name as their Kerberos +principal name, and allow alice and bob to use a group principal named +as group@example.net, you can use the following rules.

+
{
+  "principals": [
+    {
+      "principal": "([^/]+)/?.*@example.net",
+      "principal_to_user": "$1",
+      "allow": true
+    },
+    {
+      "principal": "group@example.net",
+      "user": "alice|bob",
+      "allow": true
+    }
+  ]
+}
+
+
+
+
+

System information rules#

+

These rules specify which users can access the system information management +interface. System information access includes the following aspects:

+ +

The user is granted or denied access based on the first matching +rule read from top to bottom. If no rules are specified, all access to system +information is denied. If no rule matches, system access is denied. Each rule is +composed of the following fields:

+
    +
  • role (optional): regex to match against role. If matched, it +grants or denies the authorization based on the value of allow.

  • +
  • user (optional): regex to match against user name. If matched, it +grants or denies the authorization based on the value of allow.

  • +
  • allow (required): set of access permissions granted to user. Values: +read, write

  • +
+

The following configuration provides and example:

+
{
+  "system_information": [
+    {
+      "role": "admin",
+      "allow": ["read", "write"]
+    },
+    {
+      "user": "alice",
+      "allow": ["read"]
+    }
+  ]
+}
+
+
+
    +
  • All users with the admin role have read and write access to system +information. This includes the ability to trigger +Graceful shutdown.

  • +
  • The user alice can read system information.

  • +
  • All other users and roles are denied access to system information.

  • +
+

A fixed user can be set for management interfaces using the management.user +configuration property. When this is configured, system information rules must +still be set to authorize this user to read or write to management information. +The fixed management user only applies to HTTP by default. To enable the fixed +user over HTTPS, set the management.user.https-enabled configuration +property.

+
+
+

Authorization rules#

+

These rules control the ability of how owner of schema, table or view can +be altered. These rules are applicable to commands like:

+
+

ALTER SCHEMA name SET AUTHORIZATION ( user | USER user | ROLE role ) +ALTER TABLE name SET AUTHORIZATION ( user | USER user | ROLE role ) +ALTER VIEW name SET AUTHORIZATION ( user | USER user | ROLE role )

+
+

When these rules are present, the authorization is based on the first matching +rule, processed from top to bottom. If no rules match, the authorization is +denied.

+

Notice that in order to execute ALTER command on schema, table or view user requires OWNERSHIP +privilege.

+

Each authorization rule is composed of the following fields:

+
    +
  • original_user (optional): regex to match against the user requesting the +authorization. Defaults to .*.

  • +
  • original_group (optional): regex to match against group names of the +requesting authorization. Defaults to .*.

  • +
  • original_role (optional): regex to match against role names of the +requesting authorization. Defaults to .*.

  • +
  • new_user (optional): regex to match against the new owner user of the schema, table or view. +By default it does not match.

  • +
  • new_role (optional): regex to match against the new owner role of the schema, table or view. +By default it does not match.

  • +
  • allow (optional): boolean indicating if the authentication should be +allowed. Defaults to true.

  • +
+

Notice that new_user and new_role are optional, however it is required to provide at least one of them.

+

The following example allows the admin role, to change owner of any schema, table or view +to any user, except to``bob``.

+
{
+  "authorization": [
+    {
+      "original_role": "admin",
+      "new_user": "bob",
+      "allow": false
+    },
+    {
+      "original_role": "admin",
+      "new_user": ".*",
+      "new_role": ".*"
+    }
+  ],
+  "schemas": [
+    {
+      "role": "admin",
+      "owner": true
+    }
+  ],
+  "tables": [
+    {
+      "role": "admin",
+      "privileges": ["OWNERSHIP"]
+    }
+  ]
+}
+
+
+
+
+
+

Catalog-level access control files#

+

You can create JSON files for individual catalogs that define authorization +rules specific to that catalog. To enable catalog-level access control files, +add a connector-specific catalog configuration property that sets the +authorization type to FILE and the security.config-file catalog +configuration property that specifies the JSON rules file.

+

For example, the following Iceberg catalog configuration properties use the +rules.json file for catalog-level access control:

+
iceberg.security=FILE
+security.config-file=etc/catalog/rules.json
+
+
+

Catalog-level access control files are supported on a per-connector basis, refer +to the connector documentation for more information.

+
+

Note

+

These rules do not apply to system-defined tables in the +information_schema schema.

+
+
+

Configure a catalog rules file#

+

The configuration file is specified in JSON format. This file is composed of +the following sections, each of which is a list of rules that are processed in +order from top to bottom:

+
    +
  1. schemas

  2. +
  3. tables

  4. +
  5. session_properties

  6. +
+

The user is granted the privileges from the first matching rule. All regexes +default to .* if not specified.

+
+

Schema rules#

+

These rules govern who is considered an owner of a schema.

+
    +
  • user (optional): regex to match against user name.

  • +
  • group (optional): regex to match against every user group the user belongs +to.

  • +
  • schema (optional): regex to match against schema name.

  • +
  • owner (required): boolean indicating ownership.

  • +
+
+
+

Table rules#

+

These rules govern the privileges granted on specific tables.

+
    +
  • user (optional): regex to match against user name.

  • +
  • group (optional): regex to match against every user group the user belongs +to.

  • +
  • schema (optional): regex to match against schema name.

  • +
  • table (optional): regex to match against table name.

  • +
  • privileges (required): zero or more of SELECT, INSERT, +DELETE, UPDATE, OWNERSHIP, GRANT_SELECT.

  • +
  • columns (optional): list of column constraints.

  • +
  • filter (optional): boolean filter expression for the table.

  • +
  • filter_environment (optional): environment used during filter evaluation.

  • +
+
+
Column constraints#
+

These constraints can be used to restrict access to column data.

+
    +
  • name: name of the column.

  • +
  • allow (optional): if false, column can not be accessed.

  • +
  • mask (optional): mask expression applied to column.

  • +
  • mask_environment (optional): environment use during mask evaluation.

  • +
+
+
+
Filter environment and mask environment#
+

These rules apply to filter_environment and mask_environment.

+
    +
  • user (optional): username for checking permission of subqueries in a mask.

  • +
+
+

Note

+

mask can contain conditional expressions such as IF or CASE, which achieves conditional masking.

+
+
+
+
+

Function rules#

+

Each function rule is composed of the following fields:

+
    +
  • user (optional): regular expression to match against user name. +Defaults to .*.

  • +
  • group (optional): regular expression to match against group names. +Defaults to .*.

  • +
  • schema (optional): regular expression to match against schema name. +Defaults to .*.

  • +
  • function (optional): regular expression to match against function names. +Defaults to .*.

  • +
  • privileges (required): zero or more of EXECUTE, GRANT_EXECUTE.

  • +
+
+
+

Session property rules#

+

These rules govern who may set session properties.

+
    +
  • user (optional): regex to match against user name.

  • +
  • group (optional): regex to match against every user group the user belongs +to.

  • +
  • property (optional): regex to match against session property name.

  • +
  • allow (required): boolean indicating whether this session property may be +set.

  • +
+
+
+
+

Example#

+
{
+  "schemas": [
+    {
+      "user": "admin",
+      "schema": ".*",
+      "owner": true
+    },
+    {
+      "group": "finance|human_resources",
+      "schema": "employees",
+      "owner": true
+    },
+    {
+      "user": "guest",
+      "owner": false
+    },
+    {
+      "schema": "default",
+      "owner": true
+    }
+  ],
+  "tables": [
+    {
+      "user": "admin",
+      "privileges": ["SELECT", "INSERT", "DELETE", "UPDATE", "OWNERSHIP"]
+    },
+    {
+      "user": "banned_user",
+      "privileges": []
+    },
+    {
+      "schema": "hr",
+      "table": "employee",
+      "privileges": ["SELECT"],
+      "filter": "user = current_user"
+    },
+    {
+      "schema": "default",
+      "table": ".*",
+      "privileges": ["SELECT"],
+      "columns" : [
+         {
+            "name": "address",
+            "allow": false
+         },
+         {
+            "name": "ssn",
+            "mask": "'XXX-XX-' + substring(credit_card, -4)",
+            "mask_environment": {
+              "user": "admin"
+            }
+         }
+      ]
+    }
+  ],
+  "session_properties": [
+    {
+      "property": "force_local_scheduling",
+      "allow": true
+    },
+    {
+      "user": "admin",
+      "property": "max_split_size",
+      "allow": true
+    }
+  ]
+}
+
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/security/group-file.html b/430/security/group-file.html new file mode 100644 index 000000000..dd3682099 --- /dev/null +++ b/430/security/group-file.html @@ -0,0 +1,593 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + File group provider — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

File group provider#

+

Trino can map user names onto groups for easier access control and +resource group management. Group file resolves group membership using +a file on the coordinator.

+
+

Group file configuration#

+

Enable group file by creating an etc/group-provider.properties +file on the coordinator:

+
group-provider.name=file
+file.group-file=/path/to/group.txt
+
+
+

The following configuration properties are available:

+ + + + + + + + + + + + + + +

Property

Description

file.group-file

Path of the group file.

file.refresh-period

How often to reload the group file. Defaults to 5s.

+
+
+

Group files#

+
+

File format#

+

The group file contains a list of groups and members, one per line, +separated by a colon. Users are separated by a comma.

+
group_name:user_1,user_2,user_3
+
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/security/inspect-jks.html b/430/security/inspect-jks.html new file mode 100644 index 000000000..54a85ca93 --- /dev/null +++ b/430/security/inspect-jks.html @@ -0,0 +1,649 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + JKS files — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

JKS files#

+

This topic describes how to validate a Java keystore (JKS) +file used to configure TLS and HTTPS.

+

The Java KeyStore (JKS) system is provided as part of your Java installation. +Private keys and certificates for your server are stored in a keystore file. +The JKS system supports both PKCS #12 .p12 files as well as legacy +keystore .jks files.

+

The keystore file itself is always password-protected. The keystore file can +have more than one key in the the same file, each addressed by its alias +name.

+

If you receive a keystore file from your site’s network admin group, verify that +it shows the correct information for your Trino cluster, as described next.

+
+

Inspect and validate keystore#

+

Inspect the keystore file to make sure it contains the correct information for +your Trino server. Use the keytool command, which is installed as part of +your Java installation, to retrieve information from your keystore file:

+
keytool -list -v -keystore yourKeystore.jks
+
+
+

Keystores always require a password. If not provided on the keytool command +line, keytool prompts for the password.

+

Independent of the keystore’s password, it is possible that an individual key +has its own password. It is easiest to make sure these passwords are the same. +If the JKS key inside the keystore has a different password, you are prompted +twice.

+

In the output of the keytool -list command, look for:

+
    +
  • The keystore may contain either a private key (Entry type: PrivateKeyEntry) or certificate (Entry type: trustedCertEntry) or both.

  • +
  • Modern browsers now enforce 398 days as the maximum validity period for a +certificate. Look for the Valid from ... until entry, and make sure the +time span does not exceed 398 days.

  • +
  • Modern browsers and clients require the SubjectAlternativeName (SAN) +field. Make sure this shows the DNS name of your server, such as +DNS:cluster.example.com. Certificates without SANs are not +supported.

    +

    Example:

    +
  • +
+
SubjectAlternativeName [
+    DNSName:  cluster.example.com
+]
+
+
+

If your keystore shows valid information for your cluster, proceed to configure +the Trino server, as described in Place the certificate file and +Configure the coordinator.

+

The rest of this page describes additional steps that may apply in certain +circumstances.

+
+
+

Extra: add PEM to keystore#

+

Your site may have standardized on using JKS semantics for all servers. If a +vendor sends you a PEM-encoded certificate file for your Trino server, you can +import it into a keystore with a command like the following. Consult keytool +references for different options.

+
keytool -trustcacerts -import -alias cluster -file localhost.pem -keystore localkeys.jks
+
+
+

If the specified keystore file exists, keytool prompts for its password. If +you are creating a new keystore, keytool prompts for a new password, then +prompts you to confirm the same password. keytool shows you the +contents of the key being added, similar to the keytool -list format, then +prompts:

+
Trust this certificate? [no]:
+
+
+

Type yes to add the PEM certificate to the keystore.

+

The alias name is an arbitrary string used as a handle for the certificate +you are adding. A keystore can contain multiple keys and certs, so keytool +uses the alias to address individual entries.

+
+
+

Extra: Java truststores#

+
+

Note

+

Remember that there may be no need to identify a local truststore when +directly using a signed PEM-encoded certificate, independent of a keystore. +PEM certs can contain the server’s private key and the certificate chain all +the way back to a recognzied CA.

+
+

Truststore files contain a list of Certificate Authorities +trusted by Java to validate the private keys of servers, plus a list of the +certificates of trusted TLS servers. The standard Java-provided truststore file, +cacerts, is part of your Java installation in a standard location.

+

Keystores normally rely on the default location of the system truststore, which +therefore does not need to be configured.

+

However, there are cases in which you need to use an alternate truststore. For +example, if your site relies on the JKS system, your network managers may have +appended site-specific, local CAs to the standard list, to validate locally +signed keys.

+

If your server must use a custom truststore, identify its location in the +server’s config properties file. For example:

+
http-server.https.truststore.path=/mnt/shared/certs/localcacerts
+http-server.https.truststore.key=<truststore-password>
+
+
+

If connecting clients such as browsers or the Trino CLI must be separately +configured, contact your site’s network administrators for assistance.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/security/inspect-pem.html b/430/security/inspect-pem.html new file mode 100644 index 000000000..0d67fe324 --- /dev/null +++ b/430/security/inspect-pem.html @@ -0,0 +1,647 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + PEM files — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

PEM files#

+

PEM (Privacy Enhanced Mail) is a standard for public key and certificate +information, and an encoding standard used to transmit keys and certificates.

+

Trino supports PEM files. If you want to use other supported formats, see:

+ +

A single PEM file can contain either certificate or key pair information, or +both in the same file. Certified keys can contain a chain of certificates from +successive certificate authorities.

+

Follow the steps in this topic to inspect and validate key and certificate in +PEM files. See Inspect and validate keystore to validate JKS keystores.

+
+

Inspect PEM file#

+

The file name extensions shown on this page are examples only; there is no +extension naming standard.

+

You may receive a single file that includes a private key and its certificate, +or separate files. If you received separate files, concatenate them into one, +typically in order from key to certificate. For example:

+
cat clustercoord.key clustercoord.cert > clustercoord.pem
+
+
+

Next, use the cat command to view this plain text file. For example:

+
cat clustercoord.pem | less
+
+
+

Make sure the PEM file shows at least one KEY and one CERTIFICATE +section. A key section looks something like the following:

+
-----BEGIN PRIVATE KEY-----
+MIIEowIBAAKCAQEAwJL8CLeDFAHhZe3QOOF1vWt4Vuk9vyO38Y1y9SgBfB02b2jW
+....
+-----END PRIVATE KEY-----
+
+
+

If your key section reports BEGIN ENCRYPTED PRIVATE KEY instead, this means +the key is encrypted and you must use the password to open or inspect the key. +You may have specified the password when requesting the key, or the password +could be assigned by your site’s network managers. Note that password protected +PEM files are not supported by Trino.

+

If your key section reports BEGIN EC PRIVATE KEY or BEGIN DSA PRIVATE KEY, this designates a key using Elliptical Curve or DSA alternatives to RSA.

+

The certificate section looks like the following example:

+
-----BEGIN CERTIFICATE-----
+MIIDujCCAqICAQEwDQYJKoZIhvcNAQEFBQAwgaIxCzAJBgNVBAYTAlVTMRYwFAYD
+....
+-----END CERTIFICATE-----
+-----BEGIN CERTIFICATE-----
+MIIDwjCCAqoCCQCxyqwZ9GK50jANBgkqhkiG9w0BAQsFADCBojELMAkGA1UEBhMC
+....
+-----END CERTIFICATE-----
+
+
+

The file can show a single certificate section, or more than one to express a +chain of authorities, each certifying the previous.

+
+
+

Validate PEM key section#

+

This page presumes your system provides the openssl command from OpenSSL 1.1 +or later.

+

Test an RSA private key’s validity with the following command:

+
openssl rsa -in clustercoord.pem -check -noout
+
+
+

Look for the following confirmation message:

+
RSA key ok
+
+
+
+

Note

+

Consult openssl references for the appropriate versions of the +verification commands for EC or DSA keys.

+
+
+
+

Validate PEM certificate section#

+

Analyze the certificate section of your PEM file with the following openssl +command:

+
openssl x509 -in clustercoord.pem -text -noout
+
+
+

If your certificate was generated with a password, openssl prompts for it. +Note that password protected PEM files are not supported by Trino.

+

In the output of the openssl command, look for the following +characteristics:

+
    +
  • Modern browsers now enforce 398 days as the maximum validity period for a +certificate. Look for Not Before and Not After dates in the +Validity section of the output, and make sure the time span does not +exceed 398 days.

  • +
  • Modern browsers and clients require the Subject Alternative Name (SAN) +field. Make sure this shows the DNS name of your server, such as +DNS:clustercoord.example.com. Certificates without SANs are not +supported.

  • +
+

If your PEM file shows valid information for your cluster, proceed to configure +the server, as described in Place the certificate file and Configure the coordinator.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/security/internal-communication.html b/430/security/internal-communication.html new file mode 100644 index 000000000..054429a46 --- /dev/null +++ b/430/security/internal-communication.html @@ -0,0 +1,693 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Secure internal communication — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Secure internal communication#

+

The Trino cluster can be configured to use secured communication with internal +authentication of the nodes in the cluster, and to optionally use added security +with TLS.

+
+

Configure shared secret#

+

Configure a shared secret to authenticate all communication between nodes of the +cluster. Use this configuration under the following conditions:

+ +

Set the shared secret to the same value in config.properties on all nodes of the cluster:

+
internal-communication.shared-secret=<secret>
+
+
+

A large random key is recommended, and can be generated with the following Linux +command:

+
openssl rand 512 | base64
+
+
+
+

Verify configuration#

+

To verify shared secret configuration:

+
    +
  1. Start your Trino cluster with two or more nodes configured with a shared +secret.

  2. +
  3. Connect to the Web UI.

  4. +
  5. Confirm the number of ACTIVE WORKERS equals the number of nodes +configured with your shared secret.

  6. +
  7. Change the value of the shared secret on one worker, and restart the worker.

  8. +
  9. Log in to the Web UI and confirm the number of ACTIVE WORKERS is one +less. The worker with the invalid secret is not authenticated, and therefore +not registered with the coordinator.

  10. +
  11. Stop your Trino cluster, revert the value change on the worker, and restart +your cluster.

  12. +
  13. Confirm the number of ACTIVE WORKERS equals the number of nodes +configured with your shared secret.

  14. +
+
+
+
+

Configure internal TLS#

+

You can optionally add an extra layer of security by configuring the cluster to +encrypt communication between nodes with TLS.

+

You can configure the coordinator and all workers to encrypt all communication +with each other using TLS. Every node in the cluster must be configured. Nodes +that have not been configured, or are configured incorrectly, are not able to +communicate with other nodes in the cluster.

+

In typical deployments, you should enable TLS directly on the coordinator for fully encrypted access to the cluster by client +tools.

+

Enable TLS for internal communication with the following +configuration identical on all cluster nodes.

+
    +
  1. Configure a shared secret for internal communication as described in +the preceding section.

  2. +
  3. Enable automatic certificate creation and trust setup in +etc/config.properties:

    +
    internal-communication.https.required=true
    +
    +
    +
  4. +
  5. Change the URI for the discovery service to use HTTPS and point to the IP +address of the coordinator in etc/config.properties:

    +
    discovery.uri=https://<coordinator ip address>:<https port>
    +
    +
    +

    Note that using hostnames or fully qualified domain names for the URI is +not supported. The automatic certificate creation for internal TLS only +supports IP addresses.

    +
  6. +
  7. Enable the HTTPS endpoint on all workers.

    +
    http-server.https.enabled=true
    +http-server.https.port=<https port>
    +
    +
    +
  8. +
  9. Restart all nodes.

  10. +
+

Certificates are automatically created and used to ensure all communication +inside the cluster is secured with TLS.

+
+

Warning

+

Older versions of Trino required you to manually manage all the certificates +on the nodes. If you upgrade from this setup, you must remove the following +configuration properties:

+
    +
  • internal-communication.https.keystore.path

  • +
  • internal-communication.https.truststore.path

  • +
  • node.internal-address-source

  • +
+
+
+

Performance with SSL/TLS enabled#

+

Enabling encryption impacts performance. The performance degradation can vary +based on the environment, queries, and concurrency.

+

For queries that do not require transferring too much data between the Trino +nodes e.g. SELECT count(*) FROM table, the performance impact is negligible.

+

However, for CPU intensive queries which require a considerable amount of data +to be transferred between the nodes (for example, distributed joins, aggregations and +window functions, which require repartitioning), the performance impact can be +considerable. The slowdown may vary from 10% to even 100%+, depending on the network +traffic and the CPU utilization.

+
+
+

Advanced performance tuning#

+

In some cases, changing the source of random numbers improves performance +significantly.

+

By default, TLS encryption uses the /dev/urandom system device as a source of entropy. +This device has limited throughput, so on environments with high network bandwidth +(e.g. InfiniBand), it may become a bottleneck. In such situations, it is recommended to try +to switch the random number generator algorithm to SHA1PRNG, by setting it via +http-server.https.secure-random-algorithm property in config.properties on the coordinator +and all of the workers:

+
http-server.https.secure-random-algorithm=SHA1PRNG
+
+
+

Be aware that this algorithm takes the initial seed from +the blocking /dev/random device. For environments that do not have enough entropy to seed +the SHAPRNG algorithm, the source can be changed to /dev/urandom +by adding the java.security.egd property to jvm.config:

+
-Djava.security.egd=file:/dev/urandom
+
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/security/jwt.html b/430/security/jwt.html new file mode 100644 index 000000000..9fa213c90 --- /dev/null +++ b/430/security/jwt.html @@ -0,0 +1,697 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + JWT authentication — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

JWT authentication#

+

Trino can be configured to authenticate client access using JSON web tokens. A JWT is a small, web-safe +JSON file that contains cryptographic information similar to a certificate, +including:

+
    +
  • Subject

  • +
  • Valid time period

  • +
  • Signature

  • +
+

A JWT is designed to be passed between servers as proof of prior authentication +in a workflow like the following:

+
    +
  1. An end user logs into a client application and requests access to a server.

  2. +
  3. The server sends the user’s credentials to a separate authentication service +that:

    +
      +
    • validates the user

    • +
    • generates a JWT as proof of validation

    • +
    • returns the JWT to the requesting server

    • +
    +
  4. +
  5. The same JWT can then be forwarded to other services to maintain the user’s +validation without further credentials.

  6. +
+
+

Important

+

If you are trying to configure OAuth2 or OIDC, there is a dedicated system +for that in Trino, as described in OAuth 2.0 authentication. When using +OAuth2 authentication, you do not need to configure JWT authentication, +because JWTs are handled automatically by the OAuth2 code.

+

A typical use for JWT authentication is to support administrators at large +sites who are writing their own single sign-on or proxy system to stand +between users and the Trino coordinator, where their new system submits +queries on behalf of users.

+
+

Using TLS and a configured shared secret is required for JWT authentication.

+
+

Using JWT authentication#

+

Trino supports Base64 encoded JWTs, but not encrypted JWTs.

+

There are two ways to get the encryption key necessary to validate the JWT +signature:

+
    +
  • Load the key from a JSON web key set (JWKS) endpoint service (the +typical case)

  • +
  • Load the key from the local file system on the Trino coordinator

  • +
+

A JWKS endpoint is a read-only service that contains public key information in +JWK format. These public +keys are the counterpart of the private keys that sign JSON web tokens.

+
+
+

JWT authentication configuration#

+

Enable JWT authentication by setting the JWT authentication type in etc/config.properties, and +specifying a URL or path to a key file:

+
http-server.authentication.type=JWT
+http-server.authentication.jwt.key-file=https://cluster.example.net/.well-known/jwks.json
+
+
+

JWT authentication is typically used in addition to other authentication +methods:

+
http-server.authentication.type=PASSWORD,JWT
+http-server.authentication.jwt.key-file=https://cluster.example.net/.well-known/jwks.json
+
+
+

The following configuration properties are available:

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + + + + +
Configuration properties for JWT authentication#

Property

Description

http-server.authentication.jwt.key-file

Required. Specifies either the URL to a JWKS service or the path to a PEM or +HMAC file, as described below this table.

http-server.authentication.jwt.required-issuer

Specifies a string that must match the value of the JWT’s issuer (iss) +field in order to consider this JWT valid. The iss field in the JWT +identifies the principal that issued the JWT.

http-server.authentication.jwt.required-audience

Specifies a string that must match the value of the JWT’s Audience (aud) +field in order to consider this JWT valid. The aud field in the JWT +identifies the recipients that the JWT is intended for.

http-server.authentication.jwt.principal-field

String to identify the field in the JWT that identifies the subject of the +JWT. The default value is sub. This field is used to create the Trino +principal.

http-server.authentication.jwt.user-mapping.pattern

A regular expression pattern to map all user names +for this authentication system to the format expected by the Trino server.

http-server.authentication.jwt.user-mapping.file

The path to a JSON file that contains a set of user mapping +rules for this authentication system.

+

Use the http-server.authentication.jwt.key-file property to specify +either:

+
    +
  • The URL to a JWKS endpoint service, where the URL begins with https://. +The JWKS service must be reachable from the coordinator. If the coordinator +is running in a secured or firewalled network, the administrator may have +to open access to the JWKS server host.

    +
    +

    Caution

    +

    The Trino server also accepts JWKS URLs that begin with http://, but +using this protocol results in a severe security risk. Only use this +protocol for short-term testing during development of your cluster.

    +
    +
  • +
  • The path to a local file in PEM or HMAC format that contains a single key. +If the file path contains $KEYID, then Trino interpolates the keyid +from the JWT into the file path before loading this key. This enables support +for setups with multiple keys.

  • +
+
+
+

Using JWTs with clients#

+

When using the Trino CLI, specify a JWT as described +in JWT authentication.

+

When using the Trino JDBC driver, specify a JWT with the accessToken +parameter.

+
+
+

Resources#

+

The following resources may prove useful in your work with JWTs and JWKs.

+ +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/security/kerberos.html b/430/security/kerberos.html new file mode 100644 index 000000000..c3035dcab --- /dev/null +++ b/430/security/kerberos.html @@ -0,0 +1,839 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Kerberos authentication — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Kerberos authentication#

+

Trino can be configured to enable Kerberos authentication over HTTPS for +clients, such as the Trino CLI, or the JDBC and ODBC +drivers.

+

To enable Kerberos authentication for Trino, Kerberos-related configuration +changes are made on the Trino coordinator.

+

Using TLS and a configured shared secret is required for Kerberos authentication.

+
+

Environment configuration#

+
+

Kerberos services#

+

You will need a Kerberos KDC running on a +node that the Trino coordinator can reach over the network. The KDC is +responsible for authenticating principals and issuing session keys that can be +used with Kerberos-enabled services. KDCs typically run on port 88, which is +the IANA-assigned port for Kerberos.

+
+
+

MIT Kerberos configuration#

+

Kerberos needs to be configured on the Trino coordinator. At a minimum, there needs +to be a kdc entry in the [realms] section of the /etc/krb5.conf +file. You may also want to include an admin_server entry and ensure that +the Trino coordinator can reach the Kerberos admin server on port 749.

+
[realms]
+  TRINO.EXAMPLE.COM = {
+    kdc = kdc.example.com
+    admin_server = kdc.example.com
+  }
+
+[domain_realm]
+  .trino.example.com = TRINO.EXAMPLE.COM
+  trino.example.com = TRINO.EXAMPLE.COM
+
+
+

The complete documentation +for krb5.conf is hosted by the MIT Kerberos Project. If you are using a +different implementation of the Kerberos protocol, you will need to adapt the +configuration to your environment.

+
+
+

Kerberos principals and keytab files#

+

The Trino coordinator needs a Kerberos principal, as do users who are going to +connect to the Trino coordinator. You need to create these users in Kerberos +using kadmin.

+

In addition, the Trino coordinator needs a keytab file. After you +create the principal, you can create the keytab file using kadmin

+
kadmin
+> addprinc -randkey trino@EXAMPLE.COM
+> addprinc -randkey trino/trino-coordinator.example.com@EXAMPLE.COM
+> ktadd -k /etc/trino/trino.keytab trino@EXAMPLE.COM
+> ktadd -k /etc/trino/trino.keytab trino/trino-coordinator.example.com@EXAMPLE.COM
+
+
+
+

Note

+

Running ktadd randomizes the principal’s keys. If you have just +created the principal, this does not matter. If the principal already exists, +and if existing users or services rely on being able to authenticate using a +password or a keytab, use the -norandkey option to ktadd.

+
+
+
+

Configuration for TLS#

+

When using Kerberos authentication, access to the Trino coordinator must be +through TLS and HTTPS.

+
+
+
+

System access control plugin#

+

A Trino coordinator with Kerberos enabled probably needs a +System access control plugin to achieve the desired level of +security.

+
+
+

Trino coordinator node configuration#

+

You must make the above changes to the environment prior to configuring the +Trino coordinator to use Kerberos authentication and HTTPS. After making the +following environment changes, you can make the changes to the Trino +configuration files.

+ +
+

config.properties#

+

Kerberos authentication is configured in the coordinator node’s +config.properties file. The entries that need to be added are listed +below.

+
http-server.authentication.type=KERBEROS
+
+http-server.authentication.krb5.service-name=trino
+http-server.authentication.krb5.principal-hostname=trino.example.com
+http-server.authentication.krb5.keytab=/etc/trino/trino.keytab
+http.authentication.krb5.config=/etc/krb5.conf
+
+http-server.https.enabled=true
+http-server.https.port=7778
+
+http-server.https.keystore.path=/etc/trino/keystore.jks
+http-server.https.keystore.key=keystore_password
+
+node.internal-address-source=FQDN
+
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property

Description

http-server.authentication.type

Authentication type for the Trino coordinator. Must be set to KERBEROS.

http-server.authentication.krb5.service-name

The Kerberos service name for the Trino coordinator. Must match the Kerberos principal.

http-server.authentication.krb5.principal-hostname

The Kerberos hostname for the Trino coordinator. Must match the Kerberos principal. This parameter is optional. If included, Trino uses this value in the host part of the Kerberos principal instead of the machine’s hostname.

http-server.authentication.krb5.keytab

The location of the keytab that can be used to authenticate the Kerberos principal.

http.authentication.krb5.config

The location of the Kerberos configuration file.

http-server.https.enabled

Enables HTTPS access for the Trino coordinator. Should be set to true.

http-server.https.port

HTTPS server port.

http-server.https.keystore.path

The location of the Java Keystore file that is used to secure TLS.

http-server.https.keystore.key

The password for the keystore. This must match the password you specified when creating the keystore.

http-server.authentication.krb5.user-mapping.pattern

Regex to match against user. If matched, user will be replaced with first regex group. If not matched, authentication is denied. Default is (.*).

http-server.authentication.krb5.user-mapping.file

File containing rules for mapping user. See User mapping for more information.

node.internal-address-source

Kerberos is typically sensitive to DNS names. Setting this property to use FQDN ensures correct operation and usage of valid DNS host names.

+

See Standards supported for a discussion of the +supported TLS versions and cipher suites.

+
+
+

access-control.properties#

+

At a minimum, an access-control.properties file must contain an +access-control.name property. All other configuration is specific for the +implementation being configured. See System access control for +details.

+
+
+
+

User mapping#

+

After authenticating with Kerberos, the Trino server receives the user’s +principal which is typically similar to an email address. For example, when +alice logs in Trino might receive alice@example.com. By default, Trino +uses the full Kerberos principal name, but this can be mapped to a shorter +name using a user-mapping pattern. For simple mapping rules, the +http-server.authentication.krb5.user-mapping.pattern configuration property +can be set to a Java regular expression, and Trino uses the value of the +first matcher group. If the regular expression does not match, the +authentication is denied. For more complex user-mapping rules, see +User mapping.

+
+
+

Troubleshooting#

+

Getting Kerberos authentication working can be challenging. You can +independently verify some of the configuration outside of Trino to help narrow +your focus when trying to solve a problem.

+
+

Kerberos verification#

+

Ensure that you can connect to the KDC from the Trino coordinator using +telnet:

+
$ telnet kdc.example.com 88
+
+
+

Verify that the keytab file can be used to successfully obtain a ticket using +kinit and +klist

+
$ kinit -kt /etc/trino/trino.keytab trino@EXAMPLE.COM
+$ klist
+
+
+
+
+

Java keystore file verification#

+

Verify the password for a keystore file and view its contents using +Inspect and validate keystore.

+
+
+

Additional Kerberos debugging information#

+

You can enable additional Kerberos debugging information for the Trino +coordinator process by adding the following lines to the Trino jvm.config +file:

+
-Dsun.security.krb5.debug=true
+-Dlog.enable-console=true
+
+
+

-Dsun.security.krb5.debug=true enables Kerberos debugging output from the +JRE Kerberos libraries. The debugging output goes to stdout, which Trino +redirects to the logging system. -Dlog.enable-console=true enables output +to stdout to appear in the logs.

+

The amount and usefulness of the information the Kerberos debugging output +sends to the logs varies depending on where the authentication is failing. +Exception messages and stack traces can provide useful clues about the +nature of the problem.

+

See Troubleshooting Security +in the Java documentation for more details about the -Djava.security.debug +flag, and Troubleshooting for +more details about the Java GSS-API and Kerberos issues.

+
+
+

Additional resources#

+

Common Kerberos Error Messages (A-M)

+

Common Kerberos Error Messages (N-Z)

+

MIT Kerberos Documentation: Troubleshooting

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/security/ldap.html b/430/security/ldap.html new file mode 100644 index 000000000..a0c7f437d --- /dev/null +++ b/430/security/ldap.html @@ -0,0 +1,979 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + LDAP authentication — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

LDAP authentication#

+

Trino can be configured to enable frontend LDAP authentication over +HTTPS for clients, such as the Trino CLI, or the JDBC and ODBC +drivers. At present, only simple LDAP authentication mechanism involving +username and password is supported. The Trino client sends a username +and password to the coordinator, and the coordinator validates these +credentials using an external LDAP service.

+

To enable LDAP authentication for Trino, LDAP-related configuration changes are +made on the Trino coordinator.

+

Using TLS and a configured shared secret is required for LDAP authentication.

+
+

Trino server configuration#

+
+

Trino coordinator node configuration#

+

Access to the Trino coordinator should be through HTTPS, configured as described +on TLS and HTTPS.

+

You also need to make changes to the Trino configuration files. +LDAP authentication is configured on the coordinator in two parts. +The first part is to enable HTTPS support and password authentication +in the coordinator’s config.properties file. The second part is +to configure LDAP as the password authenticator plugin.

+
+

Server config properties#

+

The following is an example of the required properties that need to be added +to the coordinator’s config.properties file:

+
http-server.authentication.type=PASSWORD
+
+http-server.https.enabled=true
+http-server.https.port=8443
+
+http-server.https.keystore.path=/etc/trino/keystore.jks
+http-server.https.keystore.key=keystore_password
+
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property

Description

http-server.authentication.type

Enable the password authentication type for the Trino coordinator. Must be set to PASSWORD.

http-server.https.enabled

Enables HTTPS access for the Trino coordinator. Should be set to true. Default value is false.

http-server.https.port

HTTPS server port.

http-server.https.keystore.path

The location of the PEM or Java keystore file is used to enable TLS.

http-server.https.keystore.key

The password for the PEM or Java keystore. This must match the password you specified when creating the PEM or keystore.

http-server.process-forwarded

Enable treating forwarded HTTPS requests over HTTP as secure. Requires the X-Forwarded-Proto header to be set to https on forwarded requests. Default value is false.

http-server.authentication.password.user-mapping.pattern

Regex to match against user. If matched, user will be replaced with first regex group. If not matched, authentication is denied. Default is (.*).

http-server.authentication.password.user-mapping.file

File containing rules for mapping user. See User mapping for more information.

+
+
+

Password authenticator configuration#

+

Password authentication must be configured to use LDAP. Create an +etc/password-authenticator.properties file on the coordinator. Example:

+
password-authenticator.name=ldap
+ldap.url=ldaps://ldap-server:636
+ldap.ssl.truststore.path=/path/to/ldap_server.pem
+ldap.user-bind-pattern=<Refer below for usage>
+
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Property

Description

ldap.url

The URL to the LDAP server. The URL scheme must be ldap:// or ldaps://. Connecting to the LDAP server without TLS enabled requires ldap.allow-insecure=true.

ldap.allow-insecure

Allow using an LDAP connection that is not secured with TLS.

ldap.ssl.keystore.path

The path to the PEM or JKS keystore file.

ldap.ssl.keystore.password

Password for the key store.

ldap.ssl.truststore.path

The path to the PEM or JKS truststore file.

ldap.ssl.truststore.password

Password for the truststore.

ldap.user-bind-pattern

This property can be used to specify the LDAP user bind string for password authentication. This property must contain the pattern ${USER}, which is replaced by the actual username during the password authentication.The property can contain multiple patterns separated by a colon. Each pattern will be checked in order until a login succeeds or all logins fail. Example: ${USER}@corp.example.com:${USER}@corp.example.co.uk

ldap.ignore-referrals

Ignore referrals to other LDAP servers while performing search queries. Defaults to false.

ldap.cache-ttl

LDAP cache duration. Defaults to 1h.

ldap.timeout.connect

Timeout for establishing an LDAP connection.

ldap.timeout.read

Timeout for reading data from an LDAP connection.

+

Based on the LDAP server implementation type, the property +ldap.user-bind-pattern can be used as described below.

+
+
Active Directory#
+
ldap.user-bind-pattern=${USER}@<domain_name_of_the_server>
+
+
+

Example:

+
ldap.user-bind-pattern=${USER}@corp.example.com
+
+
+
+
+
OpenLDAP#
+
ldap.user-bind-pattern=uid=${USER},<distinguished_name_of_the_user>
+
+
+

Example:

+
ldap.user-bind-pattern=uid=${USER},OU=America,DC=corp,DC=example,DC=com
+
+
+
+
+
+

Authorization based on LDAP group membership#

+

You can further restrict the set of users allowed to connect to the Trino +coordinator, based on their group membership, by setting the optional +ldap.group-auth-pattern and ldap.user-base-dn properties, in addition +to the basic LDAP authentication properties.

+ + + + + + + + + + + + + + +

Property

Description

ldap.user-base-dn

The base LDAP distinguished name for the user who tries to connect to the server. Example: OU=America,DC=corp,DC=example,DC=com

ldap.group-auth-pattern

This property is used to specify the LDAP query for the LDAP group membership authorization. This query is executed against the LDAP server and if successful, the user is authorized. This property must contain a pattern ${USER}, which is replaced by the actual username in the group authorization search query. See samples below.

+

Based on the LDAP server implementation type, the property +ldap.group-auth-pattern can be used as described below.

+
+
+

Authorization using Trino LDAP service user#

+

Trino server can use dedicated LDAP service user for doing user group membership queries. +In such case Trino will first issue a group membership query for a Trino user that needs +to be authenticated. A user distinguished name will be extracted from a group membership +query result. Trino will then validate user password by creating LDAP context with +user distinguished name and user password. In order to use this mechanism ldap.bind-dn, +ldap.bind-password and ldap.group-auth-pattern properties need to be defined.

+ + + + + + + + + + + + + + + + + +

Property

Description

ldap.bind-dn

Bind distinguished name used by Trino when issuing group membership queries. Example: CN=admin,OU=CITY_OU,OU=STATE_OU,DC=domain

ldap.bind-password

Bind password used by Trino when issuing group membership queries. Example: password1234

ldap.group-auth-pattern

This property is used to specify the LDAP query for the LDAP group membership authorization. This query will be executed against the LDAP server and if successful, a user distinguished name will be extracted from a query result. Trino will then validate user password by creating LDAP context with user distinguished name and user password.

+
+
Active Directory#
+
ldap.group-auth-pattern=(&(objectClass=<objectclass_of_user>)(sAMAccountName=${USER})(memberof=<dn_of_the_authorized_group>))
+
+
+

Example:

+
ldap.group-auth-pattern=(&(objectClass=person)(sAMAccountName=${USER})(memberof=CN=AuthorizedGroup,OU=Asia,DC=corp,DC=example,DC=com))
+
+
+
+
+
OpenLDAP#
+
ldap.group-auth-pattern=(&(objectClass=<objectclass_of_user>)(uid=${USER})(memberof=<dn_of_the_authorized_group>))
+
+
+

Example:

+
ldap.group-auth-pattern=(&(objectClass=inetOrgPerson)(uid=${USER})(memberof=CN=AuthorizedGroup,OU=Asia,DC=corp,DC=example,DC=com))
+
+
+

For OpenLDAP, for this query to work, make sure you enable the +memberOf overlay.

+

You can use this property for scenarios where you want to authorize a user +based on complex group authorization search queries. For example, if you want to +authorize a user belonging to any one of multiple groups (in OpenLDAP), this +property may be set as follows:

+
ldap.group-auth-pattern=(&(|(memberOf=CN=normal_group,DC=corp,DC=com)(memberOf=CN=another_group,DC=com))(objectClass=inetOrgPerson)(uid=${USER}))
+
+
+
+
+
+
+
+

Trino CLI#

+
+

Environment configuration#

+
+

TLS configuration#

+

When using LDAP authentication, access to the Trino coordinator must be through +TLS/HTTPS.

+
+
+
+

Trino CLI execution#

+

In addition to the options that are required when connecting to a Trino +coordinator that does not require LDAP authentication, invoking the CLI +with LDAP support enabled requires a number of additional command line +options. You can either use --keystore-* or --truststore-* properties +to secure TLS connection. The simplest way to invoke the CLI is with a +wrapper script.

+
#!/bin/bash
+
+./trino \
+--server https://trino-coordinator.example.com:8443 \
+--keystore-path /tmp/trino.jks \
+--keystore-password password \
+--truststore-path /tmp/trino_truststore.jks \
+--truststore-password password \
+--catalog <catalog> \
+--schema <schema> \
+--user <LDAP user> \
+--password
+
+
+

Find details on the options used in TLS/HTTPS and +Username and password authentication.

+
+
+
+

Troubleshooting#

+
+

Java keystore file verification#

+

Verify the password for a keystore file and view its contents using +Inspect and validate keystore.

+
+
+

Debug Trino to LDAP server issues#

+

If you need to debug issues with Trino communicating with the LDAP server, +you can change the log level for the LDAP authenticator:

+
io.trino.plugin.password=DEBUG
+
+
+
+
+

TLS debugging for Trino CLI#

+

If you encounter any TLS related errors when running the Trino CLI, you can run +the CLI using the -Djavax.net.debug=ssl parameter for debugging. Use the +Trino CLI executable JAR to enable this. For example:

+
java -Djavax.net.debug=ssl \
+-jar \
+trino-cli-<version>-executable.jar \
+--server https://coordinator:8443 \
+<other_cli_arguments>
+
+
+
+

Common TLS/SSL errors#

+
+
java.security.cert.CertificateException: No subject alternative names present#
+

This error is seen when the Trino coordinator’s certificate is invalid, and does not have the IP you provide +in the --server argument of the CLI. You have to regenerate the coordinator’s TLS certificate +with the appropriate SAN added.

+

Adding a SAN to this certificate is required in cases where https:// uses IP address in the URL, rather +than the domain contained in the coordinator’s certificate, and the certificate does not contain the +SAN parameter with the matching IP address as an alternative attribute.

+
+
+
+

Authentication or TLS errors with JDK upgrade#

+

Starting with the JDK 8u181 release, to improve the robustness of LDAPS +(secure LDAP over TLS) connections, endpoint identification algorithms were +enabled by default. See release notes +from Oracle. +The same LDAP server certificate on the Trino coordinator, running on JDK +version >= 8u181, that was previously able to successfully connect to an +LDAPS server, may now fail with the following error:

+
javax.naming.CommunicationException: simple bind failed: ldapserver:636
+[Root exception is javax.net.ssl.SSLHandshakeException: java.security.cert.CertificateException: No subject alternative DNS name matching ldapserver found.]
+
+
+

If you want to temporarily disable endpoint identification, you can add the +property -Dcom.sun.jndi.ldap.object.disableEndpointIdentification=true +to Trino’s jvm.config file. However, in a production environment, we +suggest fixing the issue by regenerating the LDAP server certificate so that +the certificate SAN or certificate subject +name matches the LDAP server.

+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/security/oauth2.html b/430/security/oauth2.html new file mode 100644 index 000000000..ffc462732 --- /dev/null +++ b/430/security/oauth2.html @@ -0,0 +1,841 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + OAuth 2.0 authentication — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+ +
+ +
+
+ +
+

OAuth 2.0 authentication#

+

Trino can be configured to enable OAuth 2.0 authentication over HTTPS for the +Web UI and the JDBC driver. Trino uses the Authorization Code flow which exchanges an +Authorization Code for a token. At a high level, the flow includes the following +steps:

+
    +
  1. the Trino coordinator redirects a user’s browser to the Authorization Server

  2. +
  3. the user authenticates with the Authorization Server, and it approves the Trino’s permissions request

  4. +
  5. the user’s browser is redirected back to the Trino coordinator with an authorization code

  6. +
  7. the Trino coordinator exchanges the authorization code for a token

  8. +
+

To enable OAuth 2.0 authentication for Trino, configuration changes are made on +the Trino coordinator. No changes are required to the worker configuration; +only the communication from the clients to the coordinator is authenticated.

+

Set the callback/redirect URL to https://<trino-coordinator-domain-name>/oauth2/callback, +when configuring an OAuth 2.0 authorization server like an OpenID Connect (OIDC) +provider.

+

If Web UI is enabled, set the post-logout callback URL to +https://<trino-coordinator-domain-name>/ui/logout/logout.html when configuring +an OAuth 2.0 authentication server like an OpenID Connect (OIDC) provider.

+

Using TLS and a configured shared secret is required for OAuth 2.0 authentication.

+
+

OpenID Connect Discovery#

+

Trino supports reading Authorization Server configuration from OIDC provider +configuration metadata document. +During startup of the coordinator Trino retrieves the document and uses provided +values to set corresponding OAuth2 authentication configuration properties:

+
    +
  • authorization_endpoint -> http-server.authentication.oauth2.auth-url

  • +
  • token_endpoint -> http-server.authentication.oauth2.token-url

  • +
  • jwks_uri -> http-server.authentication.oauth2.jwks-url

  • +
  • userinfo_endpoint -> http-server.authentication.oauth2.userinfo-url

  • +
  • access_token_issuer -> http-server.authentication.oauth2.access-token-issuer

  • +
  • end_session_endpoint -> http-server.authentication.oauth2.end-session-url

  • +
+
+

Warning

+

If the authorization server is issuing JSON Web Tokens (JWTs) and the +metadata document contains userinfo_endpoint, Trino uses this endpoint to +check the validity of OAuth2 access tokens. Since JWTs can be inspected +locally, using them against userinfo_endpoint may result in authentication +failure. In this case, set the +http-server.authentication.oauth2.oidc.use-userinfo-endpoint configuration +property to false +(http-server.authentication.oauth2.oidc.use-userinfo-endpoint=false). This +instructs Trino to ignore userinfo_endpoint and inspect tokens locally.

+
+

This functionality is enabled by default but can be turned off with: +http-server.authentication.oauth2.oidc.discovery=false.

+
+
+

Trino server configuration#

+

Using the OAuth2 authentication requires the Trino coordinator to be secured +with TLS.

+

The following is an example of the required properties that need to be added +to the coordinator’s config.properties file:

+
http-server.authentication.type=oauth2
+
+http-server.https.port=8443
+http-server.https.enabled=true
+
+http-server.authentication.oauth2.issuer=https://authorization-server.com
+http-server.authentication.oauth2.client-id=CLIENT_ID
+http-server.authentication.oauth2.client-secret=CLIENT_SECRET
+
+
+

To enable OAuth 2.0 authentication for the Web UI, the following +property must be be added:

+
web-ui.authentication.type=oauth2
+
+
+

The following configuration properties are available:

+ + ++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
OAuth2 configuration properties#

Property

Description

http-server.authentication.type

The type of authentication to use. Must be set to oauth2 to enable OAuth2 +authentication for the Trino coordinator.

http-server.authentication.oauth2.issuer

The issuer URL of the IdP. All issued tokens must have this in the iss +field.

http-server.authentication.oauth2.access-token-issuer

The issuer URL of the IdP for access tokens, if different. All issued access +tokens must have this in the iss field. Providing this value while OIDC +discovery is enabled overrides the value from the OpenID provider metadata +document. Defaults to the value of +http-server.authentication.oauth2.issuer.

http-server.authentication.oauth2.auth-url

The authorization URL. The URL a user’s browser will be redirected to in +order to begin the OAuth 2.0 authorization process. Providing this value +while OIDC discovery is enabled overrides the value from the OpenID provider +metadata document.

http-server.authentication.oauth2.token-url

The URL of the endpoint on the authorization server which Trino uses to +obtain an access token. Providing this value while OIDC discovery is enabled +overrides the value from the OpenID provider metadata document.

http-server.authentication.oauth2.jwks-url

The URL of the JSON Web Key Set (JWKS) endpoint on the authorization server. +It provides Trino the set of keys containing the public key to verify any +JSON Web Token (JWT) from the authorization server. Providing this value +while OIDC discovery is enabled overrides the value from the OpenID provider +metadata document.

http-server.authentication.oauth2.userinfo-url

The URL of the IdPs /userinfo endpoint. If supplied then this URL is used +to validate the OAuth access token and retrieve any associated claims. This +is required if the IdP issues opaque tokens. Providing this value while OIDC +discovery is enabled overrides the value from the OpenID provider metadata +document.

http-server.authentication.oauth2.client-id

The public identifier of the Trino client.

http-server.authentication.oauth2.client-secret

The secret used to authorize Trino client with the authorization server.

http-server.authentication.oauth2.additional-audiences

Additional audiences to trust in addition to the client ID which is +always a trusted audience.

http-server.authentication.oauth2.scopes

Scopes requested by the server during the authorization challenge. See: +https://tools.ietf.org/html/rfc6749#section-3.3

http-server.authentication.oauth2.challenge-timeout

Maximum duration of the authorization challenge. +Default is 15m.

http-server.authentication.oauth2.state-key

A secret key used by the SHA-256 HMAC +algorithm to sign the state parameter in order to ensure that the +authorization request was not forged. Default is a random string generated +during the coordinator start.

http-server.authentication.oauth2.user-mapping.pattern

Regex to match against user. If matched, the user name is replaced with +first regex group. If not matched, authentication is denied. Default is +(.*) which allows any user name.

http-server.authentication.oauth2.user-mapping.file

File containing rules for mapping user. See User mapping for +more information.

http-server.authentication.oauth2.principal-field

The field of the access token used for the Trino user principal. Defaults to +sub. Other commonly used fields include sAMAccountName, name, +upn, and email.

http-server.authentication.oauth2.oidc.discovery

Enable reading the OIDC provider metadata. +Default is true.

http-server.authentication.oauth2.oidc.discovery.timeout

The timeout when reading OpenID provider metadata. Default is 30s.

http-server.authentication.oauth2.oidc.use-userinfo-endpoint

Use the value of userinfo_endpoint in the provider +metadata. +When a userinfo_endpoint value is supplied this URL is used to validate +the OAuth 2.0 access token, and retrieve any associated claims. This flag +allows ignoring the value provided in the metadata document. Default is +true.

http-server.authentication.oauth2.end-session-url

The URL of the endpoint on the authentication server to which the user’s +browser is redirected to so that End-User is logged out from the +authentication server when logging out from Trino.

+
+

Refresh tokens#

+

Refresh tokens allow you to securely control the length of user sessions +within applications. The refresh token has a longer lifespan (TTL) and is used +to refresh the access token that has a shorter lifespan. When refresh tokens +are used in conjunction with access tokens, users can remain logged in for an +extended duration without interruption by another login request.

+

In a refresh token flow, there are three tokens with different expiration times:

+
    +
  • access token

  • +
  • refresh token

  • +
  • Trino-encrypted token that is a combination of the access and refresh tokens. +The encrypted token manages the session lifetime with the timeout value that +is set with the +http-server.authentication.oauth2.refresh-tokens.issued-token.timeout +property.

  • +
+

In the following scenario, the lifespan of the tokens issued by an IdP are:

+
    +
  • access token 5m

  • +
  • refresh token 24h

  • +
+

Because the access token lifespan is only five minutes, Trino uses the longer +lifespan refresh token to request another access token every five minutes on +behalf of a user. In this case, the maximum +http-server.authentication.oauth2.refresh-tokens.issued-token.timeout is +twenty-four hours.

+

To use refresh token flows, the following property must be +enabled in the coordinator configuration.

+
http-server.authentication.oauth2.refresh-tokens=true
+
+
+

Additional scopes for offline access might be required, depending on +IdP configuration.

+
http-server.authentication.oauth2.scopes=openid,offline_access [or offline]
+
+
+

The following configuration properties are available:

+ + ++++ + + + + + + + + + + + + + + + + + + + +
OAuth2 configuration properties for refresh flow#

Property

Description

http-server.authentication.oauth2.refresh-tokens.issued-token.timeout

Expiration time for an issued token, which is the Trino-encrypted token that +contains an access token and a refresh token. The timeout value must be less +than or equal to the duration of the refresh token +expiration issued by the IdP. Defaults to 1h. The timeout value is the +maximum session time for an OAuth2-authenticated client with refresh tokens +enabled. For more details, see Troubleshooting.

http-server.authentication.oauth2.refresh-tokens.issued-token.issuer

Issuer representing the coordinator instance, that is referenced in the +issued token, defaults to Trino_coordinator. The current Trino version is +appended to the value. This is mainly used for debugging purposes.

http-server.authentication.oauth2.refresh-tokens.issued-token.audience

Audience representing this coordinator instance, that is used in the +issued token. Defaults to Trino_coordinator.

http-server.authentication.oauth2.refresh-tokens.secret-key

Base64-encoded secret key used to encrypt the generated token. By default +it’s generated during startup.

+
+
+
+

Troubleshooting#

+

To debug issues, change the log level for the OAuth 2.0 +authenticator:

+
io.trino.server.security.oauth2=DEBUG
+
+
+

To debug issues with OAuth 2.0 authentication use with the web UI, set the +following configuration property:

+
io.trino.server.ui.OAuth2WebUiAuthenticationFilter=DEBUG
+
+
+

This assumes the OAuth 2.0 authentication for the Web UI is enabled as described +in Trino server configuration.

+

The logged debug error for a lapsed refresh token is Tokens refresh challenge has failed.

+
+

Warning

+

If a refresh token lapses, the user session is interrupted and the user must +reauthenticate by logging in again. Ensure you set the +http-server.authentication.oauth2.refresh-tokens.issued-token.timeout +value to less than or equal to the duration of the refresh token expiration +issued by your IdP. Optimally, the timeout should be slightly less than the +refresh token lifespan of your IdP to ensure that sessions end gracefully.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/security/overview.html b/430/security/overview.html new file mode 100644 index 000000000..2c6223554 --- /dev/null +++ b/430/security/overview.html @@ -0,0 +1,719 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Security overview — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Security overview#

+

After the initial installation of your cluster, security +is the next major concern for successfully operating Trino. This overview +provides an introduction to different aspects of configuring security for your +Trino cluster.

+
+

Aspects of configuring security#

+

The default installation of Trino has no security features enabled. Security +can be enabled for different parts of the Trino architecture:

+ +
+
+

Suggested configuration workflow#

+

To configure security for a new Trino cluster, follow this best practice +order of steps. Do not skip or combine steps.

+
    +
  1. Enable TLS/HTTPS

    +
      +
    • Work with your security team.

    • +
    • Use a load balancer or proxy to terminate +HTTPS, if possible.

    • +
    • Use a globally trusted TLS certificate.

    • +
    +

    Verify this step is working correctly.

    +
  2. +
  3. Configure a a shared secret

    +

    Verify this step is working correctly.

    +
  4. +
  5. Enable authentication

    +
      +
    • Start with password file authentication to get up +and running.

    • +
    • Then configure your preferred authentication provider, such as LDAP.

    • +
    • Avoid the complexity of Kerberos for client authentication, if possible.

    • +
    +

    Verify this step is working correctly.

    +
  6. +
  7. Enable authorization and access control

    +
      +
    • Start with file-based rules.

    • +
    • Then configure another access control method as required.

    • +
    +

    Verify this step is working correctly.

    +
  8. +
+

Configure one step at a time. Always restart the Trino server after each +change, and verify the results before proceeding.

+
+
+

Securing client access to the cluster#

+

Trino clients include the Trino CLI, +the Web UI, the JDBC driver, Python, Go, or other clients, and any applications using these tools.

+

All access to the Trino cluster is managed by the coordinator. Thus, securing +access to the cluster means securing access to the coordinator.

+

There are three aspects to consider:

+ +
+

Encryption#

+

The Trino server uses the standard HTTPS protocol and TLS encryption, formerly known as SSL.

+
+
+

Authentication#

+

Trino supports several authentication providers. When setting up a new cluster, +start with simple password file authentication before configuring another +provider.

+ +
+

User name management#

+

Trino provides ways to map the user and group names from authentication +providers to Trino user names.

+
    +
  • User mapping applies to all authentication systems, +and allows for regular expression rules to be specified that map complex user +names from other systems (alice@example.com) to simple user names +(alice).

  • +
  • File group provider provides a way to assign a set +of user names to a group name to ease access control.

  • +
+
+
+
+

Authorization and access control#

+

Trino’s default method of access control +allows all operations for all authenticated users.

+

To implement access control, use:

+
    +
  • File-based system access control, where +you configure JSON files that specify fine-grained user access restrictions at +the catalog, schema, or table level.

  • +
+

In addition, Trino provides an API that +allows you to create a custom access control method, or to extend an existing +one.

+

Access control can limit access to columns of a table. The default behavior +of a query to all columns with a SELECT * statement is to show an error +denying access to any inaccessible columns.

+

You can change this behavior to silently hide inaccessible columns with the +global property hide-inaccessible-columns configured in +Config properties:

+
hide-inaccessible-columns = true
+
+
+
+
+
+

Securing inside the cluster#

+

You can secure the internal communication +between coordinator and workers inside the clusters.

+

Secrets in properties files, such as passwords in catalog files, can be secured +with secrets management.

+
+
+

Securing cluster access to data sources#

+

Communication between the Trino cluster and data sources is configured for each +catalog. Each catalog uses a connector, which supports a variety of +security-related configurations.

+

More information is available with the documentation for individual +connectors.

+

Secrets management can be used for the catalog properties files +content.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/security/password-file.html b/430/security/password-file.html new file mode 100644 index 000000000..09b919498 --- /dev/null +++ b/430/security/password-file.html @@ -0,0 +1,671 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Password file authentication — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Password file authentication#

+

Trino can be configured to enable frontend password authentication over +HTTPS for clients, such as the CLI, or the JDBC and ODBC drivers. The +username and password are validated against usernames and passwords stored +in a file.

+

Password file authentication is very similar to LDAP authentication. Please see +the LDAP documentation for generic instructions on configuring the server +and clients to use TLS and authenticate with a username and password.

+

Using TLS and a configured shared secret is required for password file +authentication.

+
+

Password authenticator configuration#

+

To enable password file authentication, set the password authentication type in etc/config.properties:

+
http-server.authentication.type=PASSWORD
+
+
+

In addition, create a etc/password-authenticator.properties file on the +coordinator with the file authenticator name:

+
password-authenticator.name=file
+file.password-file=/path/to/password.db
+
+
+

The following configuration properties are available:

+ + + + + + + + + + + + + + + + + +

Property

Description

file.password-file

Path of the password file.

file.refresh-period

How often to reload the password file. Defaults to 5s.

file.auth-token-cache.max-size

Max number of cached authenticated passwords. Defaults to 1000.

+
+
+

Password files#

+
+

File format#

+

The password file contains a list of usernames and passwords, one per line, +separated by a colon. Passwords must be securely hashed using bcrypt or PBKDF2.

+

bcrypt passwords start with $2y$ and must use a minimum cost of 8:

+
test:$2y$10$BqTb8hScP5DfcpmHo5PeyugxHz5Ky/qf3wrpD7SNm8sWuA3VlGqsa
+
+
+

PBKDF2 passwords are composed of the iteration count, followed by the +hex encoded salt and hash:

+
test:1000:5b4240333032306164:f38d165fce8ce42f59d366139ef5d9e1ca1247f0e06e503ee1a611dd9ec40876bb5edb8409f5abe5504aab6628e70cfb3d3a18e99d70357d295002c3d0a308a0
+
+
+
+
+

Creating a password file#

+

Password files utilizing the bcrypt format can be created using the +htpasswd +utility from the Apache HTTP Server. +The cost must be specified, as Trino enforces a higher minimum cost +than the default.

+

Create an empty password file to get started:

+
touch password.db
+
+
+

Add or update the password for the user test:

+
htpasswd -B -C 10 password.db test
+
+
+
+
+

Verify configuration#

+

To verify password file authentication, log in to the Web UI, and connect with the Trino CLI to +the cluster:

+
    +
  • Connect to the Web UI from your browser using a URL that uses HTTPS, such as +https://trino.example.com:8443. Enter a username in the Username text +box and the corresponding password in the Password text box, and log in to +the UI. Confirm that you are not able to log in using an incorrect username +and password combination. A successful login displays the username in the +top right corner of the UI.

  • +
  • Connect with the Trino CLI using a URL that uses HTTPS, such as +https://trino.example.net:8443 with the addition of the --user and +--password properties:

  • +
+
./trino --server https://trino.example.com:8443 --user test --password
+
+
+

The above command quotes you for a password. Supply the password set for the +user entered for the --user property to use the trino> prompt. Sucessful +authentication allows you to run queries from the CLI.

+

To test the connection, send a query:

+
trino> SELECT 'rocks' AS trino;
+
+trino
+-------
+rocks
+(1 row)
+
+Query 20220919_113804_00017_54qfi, FINISHED, 1 node
+Splits: 1 total, 1 done (100.00%)
+0.12 [0 rows, 0B] [0 rows/s, 0B/s]
+
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/security/salesforce.html b/430/security/salesforce.html new file mode 100644 index 000000000..4b0f19153 --- /dev/null +++ b/430/security/salesforce.html @@ -0,0 +1,625 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Salesforce authentication — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Salesforce authentication#

+

Trino can be configured to enable frontend password authentication over +HTTPS for clients, such as the CLI, or the JDBC and ODBC drivers. The +username and password (or password and security token concatenation) +are validated by having the Trino coordinator perform a login to Salesforce.

+

This allows you to enable users to authenticate to Trino via their Salesforce +basic credentials. This can also be used to secure the Web UI.

+
+

Note

+

This is not a Salesforce connector, and does not allow users to query +Salesforce data. Salesforce authentication is simply a means by which users +can authenticate to Trino, similar to LDAP authentication or Password file authentication.

+
+

Using TLS and a configured shared secret is required for Salesforce authentication.

+
+

Salesforce authenticator configuration#

+

To enable Salesfore authentication, set the password authentication type in etc/config.properties:

+
http-server.authentication.type=PASSWORD
+
+
+

In addition, create a etc/password-authenticator.properties file on the +coordinator with the salesforce authenticator name:

+
password-authenticator.name=salesforce
+salesforce.allowed-organizations=<allowed-org-ids or all>
+
+
+

The following configuration properties are available:

+ + + + + + + + + + + + + + + + + +

Property

Description

salesforce.allowed-organizations

Comma separated list of 18 character Salesforce.com Organization IDs for a second, simple layer of security. This option can be explicitly ignored using all, which bypasses any check of the authenticated user’s Salesforce.com Organization ID.

salesforce.cache-size

Maximum number of cached authenticated users. Defaults to 4096.

salesforce.cache-expire-duration

How long a cached authentication should be considered valid. Defaults to 2m.

+
+
+

Salesforce concepts#

+

There are two Salesforce specific aspects to this authenticator. They are the use of the +Salesforce security token, and configuration of one or more Salesforce.com Organization IDs.

+
+

Security token#

+

Credentials are a user’s Salesforce username and password if Trino is connecting from a whitelisted +IP, or username and password/security token +concatenation otherwise. For example, if Trino is not whitelisted, and your password is password +and security token is token, use passwordtoken to authenticate.

+

You can configure a public IP for Trino as a trusted IP by whitelisting an IP range.

+
+
+

Salesforce.com organization IDs#

+

You can configure one or more Salesforce Organization IDs for additional security. When the user authenticates, +the Salesforce API returns the 18 character Salesforce.com Organization ID for the user. The Trino Salesforce +authenticator ensures that the ID matches one of the IDs configured in salesforce.allowed-organizations.

+

Optionally, you can configure all to explicitly ignore this layer of security.

+

Admins can find their Salesforce.com Organization ID using the Salesforce Setup UI. This will be the 15 character +ID, which can be converted to the 18 character ID.

+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/security/secrets.html b/430/security/secrets.html new file mode 100644 index 000000000..50a7b806d --- /dev/null +++ b/430/security/secrets.html @@ -0,0 +1,564 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Secrets — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

Secrets#

+

Trino manages configuration details in static properties files. This +configuration needs to include values such as usernames, passwords and other +strings, that are often required to be kept secret. Only a few select +administrators or the provisioning system has access to the actual value.

+

The secrets support in Trino allows you to use environment variables as values +for any configuration property. All properties files used by Trino, including +config.properties and catalog properties files, are supported. When loading +the properties, Trino replaces the reference to the environment variable with +the value of the environment variable.

+

Environment variables are the most widely-supported means of setting and +retrieving values. Environment variables can be set in the scope of the task +being performed, preventing external access. Most provisioning and configuration +management systems include support for setting environment variables. This +includes systems such as Ansible, often used for virtual machines, and +Kubernetes for container usage. You can also manually set an environment +variable on the command line.

+
export DB_PASSWORD=my-super-secret-pwd
+
+
+

To use this variable in the properties file, you reference it with the syntax +${ENV:VARIABLE}. For example, if you want to use the password in a catalog +properties file like etc/catalog/db.properties, add the following line:

+
connection-password=${ENV:DB_PASSWORD}
+
+
+

With this setup in place, the secret is managed by the provisioning system +or by the administrators handling the machines. No secret is stored in the Trino +configuration files on the filesystem or wherever they are managed.

+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/security/tls.html b/430/security/tls.html new file mode 100644 index 000000000..ab452b7b8 --- /dev/null +++ b/430/security/tls.html @@ -0,0 +1,838 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + TLS and HTTPS — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

TLS and HTTPS#

+

Trino runs with no security by default. This allows you to connect to the server +using URLs that specify the HTTP protocol when using the Trino CLI, the Web UI, or other +clients.

+

This topic describes how to configure your Trino server to use TLS to require clients to use the HTTPS connection protocol. +All authentication technologies supported by Trino require configuring TLS as +the foundational layer.

+
+

Important

+

This page discusses only how to prepare the Trino server for secure client +connections from outside of the Trino cluster to its coordinator.

+
+

See the Glossary to clarify unfamiliar terms.

+
+

Supported standards#

+

When configured to use TLS, the Trino server responds to client connections +using TLS 1.2 and TLS 1.3 certificates. The server rejects TLS 1.1, TLS 1.0, and +all SSL format certificates.

+

The Trino server does not specify a set of supported ciphers, instead deferring +to the defaults set by the JVM version in use. The documentation for Java 17 +lists its supported cipher suites.

+

Run the following two-line code on the same JVM from the same vendor as +configured on the coordinator to determine that JVM’s default cipher list.

+
echo "java.util.Arrays.asList(((javax.net.ssl.SSLServerSocketFactory) \
+javax.net.ssl.SSLServerSocketFactory.getDefault()).getSupportedCipherSuites()).forEach(System.out::println)" | jshell -
+
+
+

The default Trino server specifies a set of regular expressions that exclude +older cipher suites that do not support forward secrecy (FS).

+

Use the http-server.https.included-cipher property to specify a +comma-separated list of ciphers in preferred use order. If one of your preferred +selections is a non-FS cipher, you must also set the +http-server.https.excluded-cipher property to an empty list to override the +default exclusions. For example:

+
http-server.https.included-cipher=TLS_RSA_WITH_AES_128_CBC_SHA,TLS_RSA_WITH_AES_128_CBC_SHA256
+http-server.https.excluded-cipher=
+
+
+

Specifying a different cipher suite is a complex issue that should only be +considered in conjunction with your organization’s security managers. Using a +different suite may require downloading and installing a different SunJCE +implementation package. Some locales may have export restrictions on cipher +suites. See the discussion in Java documentation that begins with Customizing +the Encryption Algorithm Providers.

+
+

Note

+

If you manage the coordinator’s direct TLS implementatation, monitor the CPU +usage on the Trino coordinator after enabling HTTPS. Java prefers the more +CPU-intensive cipher suites, if you allow it to choose from a big list of +ciphers. If the CPU usage is unacceptably high after enabling HTTPS, you can +configure Java to use specific cipher suites as described in this section.

+

However, best practice is to instead use an external load balancer, as +discussed next.

+
+
+
+

Approaches#

+

To configure Trino with TLS support, consider two alternative paths:

+
    +
  • Use the load balancer or proxy at your site +or cloud environment to terminate TLS/HTTPS. This approach is the simplest and +strongly preferred solution.

  • +
  • Secure the Trino server directly. This +requires you to obtain a valid certificate, and add it to the Trino +coordinator’s configuration.

  • +
+
+
+

Use a load balancer to terminate TLS/HTTPS#

+

Your site or cloud environment may already have a load balancer +or proxy server configured and running with a valid, globally trusted TLS +certificate. In this case, you can work with your network administrators to set +up your Trino server behind the load balancer. The load balancer or proxy server +accepts TLS connections and forwards them to the Trino coordinator, which +typically runs with default HTTP configuration on the default port, 8080.

+

When a load balancer accepts a TLS encrypted connection, it adds a +forwarded +HTTP header to the request, such as X-Forwarded-Proto: https.

+

This tells the Trino coordinator to process the connection as if a TLS +connection has already been successfully negotiated for it. This is why you do +not need to configure http-server.https.enabled=true for a coordinator +behind a load balancer.

+

However, to enable processing of such forwarded headers, the server’s +config properties file must include the following:

+
http-server.process-forwarded=true
+
+
+

This completes any necessary configuration for using HTTPS with a load balancer. +Client tools can access Trino with the URL exposed by the load balancer.

+
+
+

Secure Trino directly#

+

Instead of the preferred mechanism of using an external load balancer, you can secure the Trino coordinator itself. This +requires you to obtain and install a TLS certificate, and +configure Trino to use it for client connections.

+
+

Add a TLS certificate#

+

Obtain a TLS certificate file for use with your Trino server. Consider the +following types of certificates:

+
    +
  • Globally trusted certificates — A certificate that is automatically +trusted by all browsers and clients. This is the easiest type to use because +you do not need to configure clients. Obtain a certificate of this type from:

    +
      +
    • A commercial certificate vendor

    • +
    • Your cloud infrastructure provider

    • +
    • A domain name registrar, such as Verisign or GoDaddy

    • +
    • A free certificate generator, such as +letsencrypt.org or +sslforfree.com

    • +
    +
  • +
  • Corporate trusted certificates — A certificate trusted by browsers and +clients in your organization. Typically, a site’s IT department runs a local +certificate authority and preconfigures clients and servers +to trust this CA.

  • +
  • Generated self-signed certificates — A certificate generated just for +Trino that is not automatically trusted by any client. Before using, make sure +you understand the limitations of self-signed certificates.

  • +
+

The most convenient option and strongly recommended option is a globally trusted +certificate. It may require a little more work up front, but it is worth it to +not have to configure every single client.

+
+
+

Keys and certificates#

+

Trino can read certificates and private keys encoded in PEM encoded PKCS #1, PEM +encoded PKCS #8, PKCS #12, and the legacy Java KeyStore (JKS) format. +Certificates and private keys encoded in a binary format such as DER must be +converted.

+

Make sure you obtain a certificate that is validated by a recognized +certificate authority.

+
+
+

Inspect received certificates#

+

Before installing your certificate, inspect and validate the received key and +certificate files to make sure they reference the correct information to access +your Trino server. Much unnecessary debugging time is saved by taking the time +to validate your certificates before proceeding to configure the server.

+

Inspect PEM-encoded files as described in Inspect PEM files.

+

Inspect PKCS # 12 and JKS keystores as described in Inspect JKS files.

+
+
+

Invalid certificates#

+

If your certificate does not pass validation, or does not show the expected +information on inspection, contact the group or vendor who provided it for a +replacement.

+
+
+

Place the certificate file#

+

There are no location requirements for a certificate file as long as:

+
    +
  • The file can be read by the Trino coordinator server process.

  • +
  • The location is secure from copying or tampering by malicious actors.

  • +
+

You can place your file in the Trino coordinator’s etc directory, which +allows you to use a relative path reference in configuration files. However, +this location can require you to keep track of the certificate file, and move it +to a new etc directory when you upgrade your Trino version.

+
+
+

Configure the coordinator#

+

On the coordinator, add the following lines to the config properties file to enable TLS/HTTPS support for the server.

+
+

Note

+

Legacy keystore and truststore wording is used in property names, even +when directly using PEM-encoded certificates.

+
+
http-server.https.enabled=true
+http-server.https.port=8443
+http-server.https.keystore.path=etc/clustercoord.pem
+
+
+

Possible alternatives for the third line include:

+
http-server.https.keystore.path=etc/clustercoord.jks
+http-server.https.keystore.path=/usr/local/certs/clustercoord.p12
+
+
+

Relative paths are relative to the Trino server’s root directory. In a +tar.gz installation, the root directory is one level above etc.

+

JKS keystores always require a password, while PEM files with passwords are not +supported by Trino. For JKS, add the following line to the configuration:

+
http-server.https.keystore.key=<keystore-password>
+
+
+

It is possible for a key inside a keystore to have its own password, +independent of the keystore’s password. In this case, specify the key’s password +with the following property:

+
http-server.https.keymanager.password=<key-password>
+
+
+

When your Trino coordinator has an authenticator enabled along with HTTPS +enabled, HTTP access is automatically disabled for all clients, including the +Web UI. Although not recommended, you can +re-enable it by setting:

+
http-server.authentication.allow-insecure-over-http=true
+
+
+
+
+

Verify configuration#

+

To verify TLS/HTTPS configuration, log in to the Web UI, and send a query with the Trino CLI.

+
    +
  • Connect to the Web UI from your browser using a URL that uses HTTPS, such as +https://trino.example.com:8443. Enter any username into the Username +text box, and log in to the UI. The Password box is disabled while +authentication is not configured.

  • +
  • Connect with the Trino CLI using a URL that uses HTTPS, such as +https://trino.example.com:8443:

  • +
+
./trino --server https://trino.example.com:8443
+
+
+

Send a query to test the connection:

+
trino> SELECT 'rocks' AS trino;
+
+trino
+-------
+rocks
+(1 row)
+
+Query 20220919_113804_00017_54qfi, FINISHED, 1 node
+Splits: 1 total, 1 done (100.00%)
+0.12 [0 rows, 0B] [0 rows/s, 0B/s]
+
+
+
+
+
+

Limitations of self-signed certificates#

+

It is possible to generate a self-signed certificate with the openssl, +keytool, or on Linux, certtool commands. Self-signed certificates can be +useful during development of a cluster for internal use only. We recommend never +using a self-signed certificate for a production Trino server.

+

Self-signed certificates are not trusted by anyone. They are typically created +by an administrator for expediency, because they do not require getting trust +signoff from anyone.

+

To use a self-signed certificate while developing your cluster requires:

+
    +
  • distributing to every client a local truststore that validates the certificate

  • +
  • configuring every client to use this certificate

  • +
+

However, even with this client configuration, modern browsers reject these +certificates, which makes self-signed servers difficult to work with.

+

There is a difference between self-signed and unsigned certificates. Both types +are created with the same tools, but unsigned certificates are meant to be +forwarded to a CA with a Certificate Signing Request (CSR). The CA returns the +certificate signed by the CA and now globally trusted.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/security/user-mapping.html b/430/security/user-mapping.html new file mode 100644 index 000000000..505495f5f --- /dev/null +++ b/430/security/user-mapping.html @@ -0,0 +1,706 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + User mapping — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

User mapping#

+

User mapping defines rules for mapping from users in the authentication method to Trino users. This +mapping is particularly important for Kerberos or +certificate authentication where the user names +are complex, such as alice@example or CN=Alice Smith,OU=Finance,O=Acme,C=US.

+

There are two ways to map the username format of a given authentication +provider into the simple username format of Trino users:

+ +
+

Pattern mapping rule#

+

If you can map all of your authentication method’s usernames with a single +reqular expression, consider using a Pattern mapping rule.

+

For example, your authentication method uses all usernames in the form +alice@example.com, with no exceptions. In this case, choose a regex that +breaks incoming usernames into at least two regex capture groups, such that the +first capture group includes only the name before the @ sign. You can use +the simple regex (.*)(@.*) for this case.

+

Trino automatically uses the first capture group – the $1 group – as the +username to emit after the regex substitution. If the regular expression does +not match the incoming username, authentication is denied.

+

Specify your regex pattern in the appropriate property in your coordinator’s +config.properties file, using one of the *user-mapping.pattern +properties from the table below that matches the authentication type of your +configured authentication provider. For example, for an LDAP authentication provider:

+
http-server.authentication.password.user-mapping.pattern=(.*)(@.*)
+
+
+

Remember that an authentication type +represents a category, such as PASSWORD, OAUTH2, KERBEROS. More than +one authentication method can have the same authentication type. For example, +the Password file, LDAP, and Salesforce authentication methods all share the +PASSWORD authentication type.

+

You can specify different user mapping patterns for different authentication +types when multiple authentication methods are enabled:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Authentication type

Property

Password (file, LDAP, Salesforce)

http-server.authentication.password.user-mapping.pattern

OAuth2

http-server.authentication.oauth2.user-mapping.pattern

Certificate

http-server.authentication.certificate.user-mapping.pattern

Header

http-server.authentication.header.user-mapping.pattern

JSON Web Token

http-server.authentication.jwt.user-mapping.pattern

Kerberos

http-server.authentication.krb5.user-mapping.pattern

Insecure

http-server.authentication.insecure.user-mapping.pattern

+
+
+

File mapping rules#

+

Use the File mapping rules method if your authentication provider expresses +usernames in a way that cannot be reduced to a single rule, or if you want to +exclude a set of users from accessing the cluster.

+

The rules are loaded from a JSON file identified in a configuration property. +The mapping is based on the first matching rule, processed from top to bottom. +If no rules match, authentication is denied. Each rule is composed of the +following fields:

+
    +
  • pattern (required): regex to match against the authentication method’s +username.

  • +
  • user (optional): replacement string to substitute against pattern. +The default value is $1.

  • +
  • allow (optional): boolean indicating whether authentication is to be +allowed for the current match.

  • +
  • case (optional): one of:

    +
      +
    • keep - keep the matched username as is (default behavior)

    • +
    • lower - lowercase the matched username; thus both Admin and ADMIN become admin

    • +
    • upper - uppercase the matched username; thus both admin and Admin become ADMIN

    • +
    +
  • +
+

The following example maps all usernames in the form alice@example.com to +just alice, except for the test user, which is denied authentication. It +also maps users in the form bob@uk.example.com to bob_uk:

+
{
+    "rules": [
+        {
+            "pattern": "test@example\\.com",
+            "allow": false
+        },
+        {
+            "pattern": "(.+)@example\\.com"
+        },
+        {
+            "pattern": "(?<user>.+)@(?<region>.+)\\.example\\.com",
+            "user": "${user}_${region}"
+        },
+        {
+            "pattern": "(.*)@uppercase.com",
+            "case": "upper"
+        }
+    ]
+}
+
+
+

Set up the preceding example to use the LDAP +authentication method with the PASSWORD +authentication type by adding the following line to your coordinator’s +config.properties file:

+
http-server.authentication.password.user-mapping.file=etc/user-mapping.json
+
+
+

You can place your user mapping JSON file in any local file system location on +the coordinator, but placement in the etc directory is typical. There is no +naming standard for the file or its extension, although using .json as the +extension is traditional. Specify an absolute path or a path relative to the +Trino installation root.

+

You can specify different user mapping files for different authentication +types when multiple authentication methods are enabled:

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

Authentication type

Property

Password (file, LDAP, Salesforce)

http-server.authentication.password.user-mapping.file

OAuth2

http-server.authentication.oauth2.user-mapping.file

Certificate

http-server.authentication.certificate.user-mapping.file

Header

http-server.authentication.header.user-mapping.pattern

JSON Web Token

http-server.authentication.jwt.user-mapping.file

Kerberos

http-server.authentication.krb5.user-mapping.file

Insecure

http-server.authentication.insecure.user-mapping.file

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sitemap.xml b/430/sitemap.xml new file mode 100644 index 000000000..ed47df94a --- /dev/null +++ b/430/sitemap.xml @@ -0,0 +1 @@ +https://trino.io/docs/current/admin.htmlhttps://trino.io/docs/current/admin/dist-sort.htmlhttps://trino.io/docs/current/admin/dynamic-filtering.htmlhttps://trino.io/docs/current/admin/event-listeners-http.htmlhttps://trino.io/docs/current/admin/event-listeners-mysql.htmlhttps://trino.io/docs/current/admin/fault-tolerant-execution.htmlhttps://trino.io/docs/current/admin/graceful-shutdown.htmlhttps://trino.io/docs/current/admin/jmx.htmlhttps://trino.io/docs/current/admin/properties.htmlhttps://trino.io/docs/current/admin/properties-exchange.htmlhttps://trino.io/docs/current/admin/properties-general.htmlhttps://trino.io/docs/current/admin/properties-http-client.htmlhttps://trino.io/docs/current/admin/properties-logging.htmlhttps://trino.io/docs/current/admin/properties-node-scheduler.htmlhttps://trino.io/docs/current/admin/properties-optimizer.htmlhttps://trino.io/docs/current/admin/properties-query-management.htmlhttps://trino.io/docs/current/admin/properties-regexp-function.htmlhttps://trino.io/docs/current/admin/properties-resource-management.htmlhttps://trino.io/docs/current/admin/properties-spilling.htmlhttps://trino.io/docs/current/admin/properties-task.htmlhttps://trino.io/docs/current/admin/properties-web-interface.htmlhttps://trino.io/docs/current/admin/properties-write-partitioning.htmlhttps://trino.io/docs/current/admin/properties-writer-scaling.htmlhttps://trino.io/docs/current/admin/resource-groups.htmlhttps://trino.io/docs/current/admin/session-property-managers.htmlhttps://trino.io/docs/current/admin/spill.htmlhttps://trino.io/docs/current/admin/tuning.htmlhttps://trino.io/docs/current/admin/web-interface.htmlhttps://trino.io/docs/current/appendix.htmlhttps://trino.io/docs/current/appendix/from-hive.htmlhttps://trino.io/docs/current/appendix/legal-notices.htmlhttps://trino.io/docs/current/client.htmlhttps://trino.io/docs/current/connector/elasticsearch.htmlhttps://trino.io/docs/current/client/cli.htmlhttps://trino.io/docs/current/client/jdbc.htmlhttps://trino.io/docs/current/connector/googlesheets.htmlhttps://trino.io/docs/current/connector.htmlhttps://trino.io/docs/current/connector/accumulo.htmlhttps://trino.io/docs/current/connector/hive.htmlhttps://trino.io/docs/current/connector/atop.htmlhttps://trino.io/docs/current/connector/hive-alluxio.htmlhttps://trino.io/docs/current/connector/bigquery.htmlhttps://trino.io/docs/current/connector/hive-azure.htmlhttps://trino.io/docs/current/connector/blackhole.htmlhttps://trino.io/docs/current/connector/hive-caching.htmlhttps://trino.io/docs/current/connector/cassandra.htmlhttps://trino.io/docs/current/connector/hive-cos.htmlhttps://trino.io/docs/current/connector/hive-gcs-tutorial.htmlhttps://trino.io/docs/current/connector/clickhouse.htmlhttps://trino.io/docs/current/connector/hive-s3.htmlhttps://trino.io/docs/current/connector/hive-security.htmlhttps://trino.io/docs/current/develop/event-listener.htmlhttps://trino.io/docs/current/develop/example-http.htmlhttps://trino.io/docs/current/develop/example-jdbc.htmlhttps://trino.io/docs/current/connector/hudi.htmlhttps://trino.io/docs/current/develop/functions.htmlhttps://trino.io/docs/current/connector/delta-lake.htmlhttps://trino.io/docs/current/develop/group-provider.htmlhttps://trino.io/docs/current/develop/header-authenticator.htmlhttps://trino.io/docs/current/connector/iceberg.htmlhttps://trino.io/docs/current/develop/insert.htmlhttps://trino.io/docs/current/develop/password-authenticator.htmlhttps://trino.io/docs/current/develop/spi-overview.htmlhttps://trino.io/docs/current/connector/druid.htmlhttps://trino.io/docs/current/develop/supporting-merge.htmlhttps://trino.io/docs/current/develop/system-access-control.htmlhttps://trino.io/docs/current/connector/ignite.htmlhttps://trino.io/docs/current/develop/table-functions.htmlhttps://trino.io/docs/current/develop/types.htmlhttps://trino.io/docs/current/functions/tdigest.htmlhttps://trino.io/docs/current/functions.htmlhttps://trino.io/docs/current/functions/teradata.htmlhttps://trino.io/docs/current/connector/jmx.htmlhttps://trino.io/docs/current/functions/aggregate.htmlhttps://trino.io/docs/current/functions/url.htmlhttps://trino.io/docs/current/functions/uuid.htmlhttps://trino.io/docs/current/connector/kafka.htmlhttps://trino.io/docs/current/functions/window.htmlhttps://trino.io/docs/current/functions/array.htmlhttps://trino.io/docs/current/functions/binary.htmlhttps://trino.io/docs/current/functions/bitwise.htmlhttps://trino.io/docs/current/functions/color.htmlhttps://trino.io/docs/current/functions/comparison.htmlhttps://trino.io/docs/current/connector/kafka-tutorial.htmlhttps://trino.io/docs/current/glossary.htmlhttps://trino.io/docs/current/functions/conditional.htmlhttps://trino.io/docs/current/index.htmlhttps://trino.io/docs/current/functions/conversion.htmlhttps://trino.io/docs/current/installation.htmlhttps://trino.io/docs/current/installation/containers.htmlhttps://trino.io/docs/current/connector/kinesis.htmlhttps://trino.io/docs/current/installation/deployment.htmlhttps://trino.io/docs/current/functions/datetime.htmlhttps://trino.io/docs/current/installation/kubernetes.htmlhttps://trino.io/docs/current/installation/query-resiliency.htmlhttps://trino.io/docs/current/connector/kudu.htmlhttps://trino.io/docs/current/installation/rpm.htmlhttps://trino.io/docs/current/language.htmlhttps://trino.io/docs/current/language/comments.htmlhttps://trino.io/docs/current/language/reserved.htmlhttps://trino.io/docs/current/functions/decimal.htmlhttps://trino.io/docs/current/connector/localfile.htmlhttps://trino.io/docs/current/language/sql-support.htmlhttps://trino.io/docs/current/functions/geospatial.htmlhttps://trino.io/docs/current/language/types.htmlhttps://trino.io/docs/current/connector/mariadb.htmlhttps://trino.io/docs/current/optimizer.htmlhttps://trino.io/docs/current/functions/hyperloglog.htmlhttps://trino.io/docs/current/optimizer/cost-based-optimizations.htmlhttps://trino.io/docs/current/optimizer/cost-in-explain.htmlhttps://trino.io/docs/current/functions/ipaddress.htmlhttps://trino.io/docs/current/optimizer/pushdown.htmlhttps://trino.io/docs/current/connector/memory.htmlhttps://trino.io/docs/current/optimizer/statistics.htmlhttps://trino.io/docs/current/overview.htmlhttps://trino.io/docs/current/overview/concepts.htmlhttps://trino.io/docs/current/connector/metastores.htmlhttps://trino.io/docs/current/overview/use-cases.htmlhttps://trino.io/docs/current/functions/json.htmlhttps://trino.io/docs/current/release.htmlhttps://trino.io/docs/current/connector/mongodb.htmlhttps://trino.io/docs/current/release/release-0.100.htmlhttps://trino.io/docs/current/connector/mysql.htmlhttps://trino.io/docs/current/release/release-0.101.htmlhttps://trino.io/docs/current/connector/object-storage-file-formats.htmlhttps://trino.io/docs/current/connector/oracle.htmlhttps://trino.io/docs/current/release/release-0.102.htmlhttps://trino.io/docs/current/functions/lambda.htmlhttps://trino.io/docs/current/release/release-0.103.htmlhttps://trino.io/docs/current/connector/phoenix.htmlhttps://trino.io/docs/current/functions/list.htmlhttps://trino.io/docs/current/connector/pinot.htmlhttps://trino.io/docs/current/release/release-0.104.htmlhttps://trino.io/docs/current/connector/postgresql.htmlhttps://trino.io/docs/current/functions/list-by-topic.htmlhttps://trino.io/docs/current/release/release-0.105.htmlhttps://trino.io/docs/current/connector/prometheus.htmlhttps://trino.io/docs/current/functions/logical.htmlhttps://trino.io/docs/current/release/release-0.106.htmlhttps://trino.io/docs/current/functions/map.htmlhttps://trino.io/docs/current/functions/math.htmlhttps://trino.io/docs/current/release/release-0.107.htmlhttps://trino.io/docs/current/functions/ml.htmlhttps://trino.io/docs/current/functions/qdigest.htmlhttps://trino.io/docs/current/functions/regexp.htmlhttps://trino.io/docs/current/connector/redis.htmlhttps://trino.io/docs/current/functions/session.htmlhttps://trino.io/docs/current/functions/setdigest.htmlhttps://trino.io/docs/current/functions/string.htmlhttps://trino.io/docs/current/connector/redshift.htmlhttps://trino.io/docs/current/functions/system.htmlhttps://trino.io/docs/current/connector/singlestore.htmlhttps://trino.io/docs/current/functions/table.htmlhttps://trino.io/docs/current/release/release-0.108.htmlhttps://trino.io/docs/current/sql/explain.htmlhttps://trino.io/docs/current/connector/sqlserver.htmlhttps://trino.io/docs/current/release/release-0.109.htmlhttps://trino.io/docs/current/sql/explain-analyze.htmlhttps://trino.io/docs/current/release/release-0.110.htmlhttps://trino.io/docs/current/connector/system.htmlhttps://trino.io/docs/current/sql/grant.htmlhttps://trino.io/docs/current/sql/grant-roles.htmlhttps://trino.io/docs/current/connector/thrift.htmlhttps://trino.io/docs/current/sql/insert.htmlhttps://trino.io/docs/current/release/release-0.111.htmlhttps://trino.io/docs/current/sql/match-recognize.htmlhttps://trino.io/docs/current/connector/tpcds.htmlhttps://trino.io/docs/current/release/release-0.112.htmlhttps://trino.io/docs/current/connector/tpch.htmlhttps://trino.io/docs/current/sql/merge.htmlhttps://trino.io/docs/current/develop.htmlhttps://trino.io/docs/current/develop/certificate-authenticator.htmlhttps://trino.io/docs/current/sql/pattern-recognition-in-window.htmlhttps://trino.io/docs/current/develop/client-protocol.htmlhttps://trino.io/docs/current/release/release-0.113.htmlhttps://trino.io/docs/current/sql/prepare.htmlhttps://trino.io/docs/current/develop/connectors.htmlhttps://trino.io/docs/current/sql/refresh-materialized-view.htmlhttps://trino.io/docs/current/sql/reset-session.htmlhttps://trino.io/docs/current/release/release-0.114.htmlhttps://trino.io/docs/current/sql/reset-session-authorization.htmlhttps://trino.io/docs/current/sql/revoke.htmlhttps://trino.io/docs/current/security/group-file.htmlhttps://trino.io/docs/current/sql/revoke-roles.htmlhttps://trino.io/docs/current/security/inspect-jks.htmlhttps://trino.io/docs/current/security/inspect-pem.htmlhttps://trino.io/docs/current/release/release-394.htmlhttps://trino.io/docs/current/sql/rollback.htmlhttps://trino.io/docs/current/security/internal-communication.htmlhttps://trino.io/docs/current/security/jwt.htmlhttps://trino.io/docs/current/sql/select.htmlhttps://trino.io/docs/current/security/kerberos.htmlhttps://trino.io/docs/current/security/ldap.htmlhttps://trino.io/docs/current/release/release-395.htmlhttps://trino.io/docs/current/security/oauth2.htmlhttps://trino.io/docs/current/security/overview.htmlhttps://trino.io/docs/current/security/password-file.htmlhttps://trino.io/docs/current/sql/set-path.htmlhttps://trino.io/docs/current/release/release-396.htmlhttps://trino.io/docs/current/security/salesforce.htmlhttps://trino.io/docs/current/sql/set-role.htmlhttps://trino.io/docs/current/security/secrets.htmlhttps://trino.io/docs/current/sql/set-session.htmlhttps://trino.io/docs/current/security/tls.htmlhttps://trino.io/docs/current/sql/set-session-authorization.htmlhttps://trino.io/docs/current/security/user-mapping.htmlhttps://trino.io/docs/current/sql.htmlhttps://trino.io/docs/current/sql/set-time-zone.htmlhttps://trino.io/docs/current/release/release-397.htmlhttps://trino.io/docs/current/sql/show-catalogs.htmlhttps://trino.io/docs/current/sql/alter-materialized-view.htmlhttps://trino.io/docs/current/sql/show-columns.htmlhttps://trino.io/docs/current/sql/alter-schema.htmlhttps://trino.io/docs/current/sql/show-create-materialized-view.htmlhttps://trino.io/docs/current/sql/alter-table.htmlhttps://trino.io/docs/current/release/release-398.htmlhttps://trino.io/docs/current/sql/show-create-schema.htmlhttps://trino.io/docs/current/sql/show-create-table.htmlhttps://trino.io/docs/current/sql/alter-view.htmlhttps://trino.io/docs/current/sql/show-create-view.htmlhttps://trino.io/docs/current/sql/analyze.htmlhttps://trino.io/docs/current/sql/show-functions.htmlhttps://trino.io/docs/current/release/release-399.htmlhttps://trino.io/docs/current/sql/call.htmlhttps://trino.io/docs/current/sql/show-grants.htmlhttps://trino.io/docs/current/sql/comment.htmlhttps://trino.io/docs/current/sql/show-role-grants.htmlhttps://trino.io/docs/current/sql/commit.htmlhttps://trino.io/docs/current/sql/show-roles.htmlhttps://trino.io/docs/current/sql/create-materialized-view.htmlhttps://trino.io/docs/current/sql/show-schemas.htmlhttps://trino.io/docs/current/release/release-400.htmlhttps://trino.io/docs/current/sql/show-session.htmlhttps://trino.io/docs/current/sql/create-role.htmlhttps://trino.io/docs/current/sql/show-stats.htmlhttps://trino.io/docs/current/sql/create-schema.htmlhttps://trino.io/docs/current/sql/show-tables.htmlhttps://trino.io/docs/current/sql/create-table.htmlhttps://trino.io/docs/current/release/release-401.htmlhttps://trino.io/docs/current/sql/start-transaction.htmlhttps://trino.io/docs/current/sql/create-table-as.htmlhttps://trino.io/docs/current/sql/truncate.htmlhttps://trino.io/docs/current/sql/create-view.htmlhttps://trino.io/docs/current/sql/update.htmlhttps://trino.io/docs/current/sql/deallocate-prepare.htmlhttps://trino.io/docs/current/sql/use.htmlhttps://trino.io/docs/current/sql/delete.htmlhttps://trino.io/docs/current/release/release-402.htmlhttps://trino.io/docs/current/sql/values.htmlhttps://trino.io/docs/current/sql/deny.htmlhttps://trino.io/docs/current/sql/describe.htmlhttps://trino.io/docs/current/release/release-352.htmlhttps://trino.io/docs/current/sql/describe-input.htmlhttps://trino.io/docs/current/release/release-403.htmlhttps://trino.io/docs/current/sql/describe-output.htmlhttps://trino.io/docs/current/sql/drop-materialized-view.htmlhttps://trino.io/docs/current/sql/drop-role.htmlhttps://trino.io/docs/current/release/release-353.htmlhttps://trino.io/docs/current/sql/drop-schema.htmlhttps://trino.io/docs/current/release/release-404.htmlhttps://trino.io/docs/current/sql/drop-table.htmlhttps://trino.io/docs/current/sql/drop-view.htmlhttps://trino.io/docs/current/sql/execute.htmlhttps://trino.io/docs/current/release/release-354.htmlhttps://trino.io/docs/current/release/release-405.htmlhttps://trino.io/docs/current/sql/execute-immediate.htmlhttps://trino.io/docs/current/release/release-310.htmlhttps://trino.io/docs/current/release/release-355.htmlhttps://trino.io/docs/current/release/release-406.htmlhttps://trino.io/docs/current/release/release-311.htmlhttps://trino.io/docs/current/release/release-356.htmlhttps://trino.io/docs/current/release/release-407.htmlhttps://trino.io/docs/current/release/release-312.htmlhttps://trino.io/docs/current/release/release-357.htmlhttps://trino.io/docs/current/release/release-408.htmlhttps://trino.io/docs/current/release/release-313.htmlhttps://trino.io/docs/current/release/release-358.htmlhttps://trino.io/docs/current/release/release-314.htmlhttps://trino.io/docs/current/release/release-409.htmlhttps://trino.io/docs/current/release/release-359.htmlhttps://trino.io/docs/current/release/release-315.htmlhttps://trino.io/docs/current/release/release-360.htmlhttps://trino.io/docs/current/release/release-316.htmlhttps://trino.io/docs/current/release/release-361.htmlhttps://trino.io/docs/current/release/release-317.htmlhttps://trino.io/docs/current/release/release-362.htmlhttps://trino.io/docs/current/release/release-318.htmlhttps://trino.io/docs/current/release/release-363.htmlhttps://trino.io/docs/current/release/release-319.htmlhttps://trino.io/docs/current/release/release-364.htmlhttps://trino.io/docs/current/release/release-320.htmlhttps://trino.io/docs/current/release/release-410.htmlhttps://trino.io/docs/current/release/release-365.htmlhttps://trino.io/docs/current/release/release-321.htmlhttps://trino.io/docs/current/release/release-411.htmlhttps://trino.io/docs/current/release/release-322.htmlhttps://trino.io/docs/current/release/release-366.htmlhttps://trino.io/docs/current/release/release-323.htmlhttps://trino.io/docs/current/release/release-412.htmlhttps://trino.io/docs/current/release/release-367.htmlhttps://trino.io/docs/current/release/release-324.htmlhttps://trino.io/docs/current/release/release-413.htmlhttps://trino.io/docs/current/release/release-325.htmlhttps://trino.io/docs/current/release/release-414.htmlhttps://trino.io/docs/current/release/release-415.htmlhttps://trino.io/docs/current/release/release-416.htmlhttps://trino.io/docs/current/release/release-417.htmlhttps://trino.io/docs/current/release/release-418.htmlhttps://trino.io/docs/current/release/release-368.htmlhttps://trino.io/docs/current/release/release-419.htmlhttps://trino.io/docs/current/release/release-369.htmlhttps://trino.io/docs/current/release/release-420.htmlhttps://trino.io/docs/current/release/release-326.htmlhttps://trino.io/docs/current/release/release-370.htmlhttps://trino.io/docs/current/release/release-327.htmlhttps://trino.io/docs/current/release/release-371.htmlhttps://trino.io/docs/current/release/release-421.htmlhttps://trino.io/docs/current/release/release-328.htmlhttps://trino.io/docs/current/release/release-372.htmlhttps://trino.io/docs/current/release/release-422.htmlhttps://trino.io/docs/current/release/release-329.htmlhttps://trino.io/docs/current/release/release-373.htmlhttps://trino.io/docs/current/release/release-423.htmlhttps://trino.io/docs/current/release/release-330.htmlhttps://trino.io/docs/current/release/release-374.htmlhttps://trino.io/docs/current/release/release-424.htmlhttps://trino.io/docs/current/release/release-331.htmlhttps://trino.io/docs/current/release/release-375.htmlhttps://trino.io/docs/current/release/release-425.htmlhttps://trino.io/docs/current/release/release-332.htmlhttps://trino.io/docs/current/release/release-376.htmlhttps://trino.io/docs/current/release/release-426.htmlhttps://trino.io/docs/current/release/release-333.htmlhttps://trino.io/docs/current/release/release-427.htmlhttps://trino.io/docs/current/release/release-334.htmlhttps://trino.io/docs/current/release/release-428.htmlhttps://trino.io/docs/current/release/release-377.htmlhttps://trino.io/docs/current/release/release-335.htmlhttps://trino.io/docs/current/release/release-429.htmlhttps://trino.io/docs/current/release/release-378.htmlhttps://trino.io/docs/current/release/release-430.htmlhttps://trino.io/docs/current/release/release-379.htmlhttps://trino.io/docs/current/release/release-380.htmlhttps://trino.io/docs/current/release/release-336.htmlhttps://trino.io/docs/current/release/release-381.htmlhttps://trino.io/docs/current/release/release-337.htmlhttps://trino.io/docs/current/security.htmlhttps://trino.io/docs/current/security/authentication-types.htmlhttps://trino.io/docs/current/security/built-in-system-access-control.htmlhttps://trino.io/docs/current/release/release-382.htmlhttps://trino.io/docs/current/security/certificate.htmlhttps://trino.io/docs/current/release/release-338.htmlhttps://trino.io/docs/current/security/file-system-access-control.htmlhttps://trino.io/docs/current/release/release-383.htmlhttps://trino.io/docs/current/release/release-339.htmlhttps://trino.io/docs/current/release/release-0.68.htmlhttps://trino.io/docs/current/release/release-384.htmlhttps://trino.io/docs/current/release/release-340.htmlhttps://trino.io/docs/current/release/release-0.69.htmlhttps://trino.io/docs/current/release/release-385.htmlhttps://trino.io/docs/current/release/release-341.htmlhttps://trino.io/docs/current/release/release-0.70.htmlhttps://trino.io/docs/current/release/release-386.htmlhttps://trino.io/docs/current/release/release-342.htmlhttps://trino.io/docs/current/release/release-0.71.htmlhttps://trino.io/docs/current/release/release-387.htmlhttps://trino.io/docs/current/release/release-343.htmlhttps://trino.io/docs/current/release/release-0.72.htmlhttps://trino.io/docs/current/release/release-344.htmlhttps://trino.io/docs/current/release/release-0.73.htmlhttps://trino.io/docs/current/release/release-345.htmlhttps://trino.io/docs/current/release/release-0.74.htmlhttps://trino.io/docs/current/release/release-388.htmlhttps://trino.io/docs/current/release/release-0.75.htmlhttps://trino.io/docs/current/release/release-389.htmlhttps://trino.io/docs/current/release/release-0.76.htmlhttps://trino.io/docs/current/release/release-390.htmlhttps://trino.io/docs/current/release/release-0.77.htmlhttps://trino.io/docs/current/release/release-346.htmlhttps://trino.io/docs/current/release/release-391.htmlhttps://trino.io/docs/current/release/release-0.78.htmlhttps://trino.io/docs/current/release/release-347.htmlhttps://trino.io/docs/current/release/release-0.79.htmlhttps://trino.io/docs/current/release/release-392.htmlhttps://trino.io/docs/current/release/release-348.htmlhttps://trino.io/docs/current/release/release-0.80.htmlhttps://trino.io/docs/current/release/release-393.htmlhttps://trino.io/docs/current/release/release-349.htmlhttps://trino.io/docs/current/release/release-0.81.htmlhttps://trino.io/docs/current/release/release-350.htmlhttps://trino.io/docs/current/release/release-0.188.htmlhttps://trino.io/docs/current/release/release-0.82.htmlhttps://trino.io/docs/current/release/release-351.htmlhttps://trino.io/docs/current/release/release-0.189.htmlhttps://trino.io/docs/current/release/release-0.83.htmlhttps://trino.io/docs/current/release/release-0.190.htmlhttps://trino.io/docs/current/release/release-0.150.htmlhttps://trino.io/docs/current/release/release-0.84.htmlhttps://trino.io/docs/current/release/release-0.191.htmlhttps://trino.io/docs/current/release/release-0.151.htmlhttps://trino.io/docs/current/release/release-0.192.htmlhttps://trino.io/docs/current/release/release-0.152.htmlhttps://trino.io/docs/current/release/release-0.193.htmlhttps://trino.io/docs/current/release/release-0.152.1.htmlhttps://trino.io/docs/current/release/release-0.152.2.htmlhttps://trino.io/docs/current/release/release-0.194.htmlhttps://trino.io/docs/current/release/release-0.152.3.htmlhttps://trino.io/docs/current/release/release-0.195.htmlhttps://trino.io/docs/current/release/release-0.153.htmlhttps://trino.io/docs/current/release/release-0.196.htmlhttps://trino.io/docs/current/release/release-0.154.htmlhttps://trino.io/docs/current/release/release-0.197.htmlhttps://trino.io/docs/current/release/release-0.85.htmlhttps://trino.io/docs/current/release/release-0.155.htmlhttps://trino.io/docs/current/release/release-0.198.htmlhttps://trino.io/docs/current/release/release-0.86.htmlhttps://trino.io/docs/current/release/release-0.156.htmlhttps://trino.io/docs/current/release/release-0.199.htmlhttps://trino.io/docs/current/release/release-0.87.htmlhttps://trino.io/docs/current/release/release-0.157.htmlhttps://trino.io/docs/current/release/release-0.200.htmlhttps://trino.io/docs/current/release/release-0.88.htmlhttps://trino.io/docs/current/release/release-0.157.1.htmlhttps://trino.io/docs/current/release/release-0.201.htmlhttps://trino.io/docs/current/release/release-0.89.htmlhttps://trino.io/docs/current/release/release-0.158.htmlhttps://trino.io/docs/current/release/release-0.202.htmlhttps://trino.io/docs/current/release/release-0.90.htmlhttps://trino.io/docs/current/release/release-0.159.htmlhttps://trino.io/docs/current/release/release-0.91.htmlhttps://trino.io/docs/current/release/release-0.203.htmlhttps://trino.io/docs/current/release/release-0.160.htmlhttps://trino.io/docs/current/release/release-0.92.htmlhttps://trino.io/docs/current/release/release-0.161.htmlhttps://trino.io/docs/current/release/release-0.93.htmlhttps://trino.io/docs/current/release/release-0.162.htmlhttps://trino.io/docs/current/release/release-0.94.htmlhttps://trino.io/docs/current/release/release-0.95.htmlhttps://trino.io/docs/current/release/release-0.96.htmlhttps://trino.io/docs/current/release/release-0.97.htmlhttps://trino.io/docs/current/release/release-0.204.htmlhttps://trino.io/docs/current/release/release-0.98.htmlhttps://trino.io/docs/current/release/release-0.205.htmlhttps://trino.io/docs/current/release/release-0.99.htmlhttps://trino.io/docs/current/release/release-0.206.htmlhttps://trino.io/docs/current/release/release-0.163.htmlhttps://trino.io/docs/current/release/release-300.htmlhttps://trino.io/docs/current/release/release-0.207.htmlhttps://trino.io/docs/current/release/release-0.164.htmlhttps://trino.io/docs/current/release/release-301.htmlhttps://trino.io/docs/current/release/release-0.208.htmlhttps://trino.io/docs/current/release/release-0.165.htmlhttps://trino.io/docs/current/release/release-0.209.htmlhttps://trino.io/docs/current/release/release-0.166.htmlhttps://trino.io/docs/current/release/release-0.210.htmlhttps://trino.io/docs/current/release/release-0.167.htmlhttps://trino.io/docs/current/release/release-302.htmlhttps://trino.io/docs/current/release/release-0.211.htmlhttps://trino.io/docs/current/release/release-303.htmlhttps://trino.io/docs/current/release/release-304.htmlhttps://trino.io/docs/current/release/release-0.168.htmlhttps://trino.io/docs/current/release/release-0.212.htmlhttps://trino.io/docs/current/release/release-305.htmlhttps://trino.io/docs/current/release/release-0.169.htmlhttps://trino.io/docs/current/release/release-0.213.htmlhttps://trino.io/docs/current/release/release-306.htmlhttps://trino.io/docs/current/release/release-0.170.htmlhttps://trino.io/docs/current/release/release-0.214.htmlhttps://trino.io/docs/current/release/release-307.htmlhttps://trino.io/docs/current/release/release-0.171.htmlhttps://trino.io/docs/current/release/release-0.215.htmlhttps://trino.io/docs/current/release/release-308.htmlhttps://trino.io/docs/current/release/release-0.172.htmlhttps://trino.io/docs/current/release/release-0.54.htmlhttps://trino.io/docs/current/release/release-309.htmlhttps://trino.io/docs/current/release/release-0.173.htmlhttps://trino.io/docs/current/release/release-0.55.htmlhttps://trino.io/docs/current/release/release-0.115.htmlhttps://trino.io/docs/current/release/release-0.174.htmlhttps://trino.io/docs/current/release/release-0.116.htmlhttps://trino.io/docs/current/release/release-0.175.htmlhttps://trino.io/docs/current/release/release-0.117.htmlhttps://trino.io/docs/current/release/release-0.56.htmlhttps://trino.io/docs/current/release/release-0.176.htmlhttps://trino.io/docs/current/release/release-0.118.htmlhttps://trino.io/docs/current/release/release-0.57.htmlhttps://trino.io/docs/current/release/release-0.177.htmlhttps://trino.io/docs/current/release/release-0.58.htmlhttps://trino.io/docs/current/release/release-0.119.htmlhttps://trino.io/docs/current/release/release-0.178.htmlhttps://trino.io/docs/current/release/release-0.59.htmlhttps://trino.io/docs/current/release/release-0.120.htmlhttps://trino.io/docs/current/release/release-0.179.htmlhttps://trino.io/docs/current/release/release-0.60.htmlhttps://trino.io/docs/current/release/release-0.121.htmlhttps://trino.io/docs/current/release/release-0.61.htmlhttps://trino.io/docs/current/release/release-0.122.htmlhttps://trino.io/docs/current/release/release-0.62.htmlhttps://trino.io/docs/current/release/release-0.180.htmlhttps://trino.io/docs/current/release/release-0.123.htmlhttps://trino.io/docs/current/release/release-0.63.htmlhttps://trino.io/docs/current/release/release-0.181.htmlhttps://trino.io/docs/current/release/release-0.124.htmlhttps://trino.io/docs/current/release/release-0.182.htmlhttps://trino.io/docs/current/release/release-0.125.htmlhttps://trino.io/docs/current/release/release-0.183.htmlhttps://trino.io/docs/current/release/release-0.64.htmlhttps://trino.io/docs/current/release/release-0.126.htmlhttps://trino.io/docs/current/release/release-0.184.htmlhttps://trino.io/docs/current/release/release-0.65.htmlhttps://trino.io/docs/current/release/release-0.127.htmlhttps://trino.io/docs/current/release/release-0.185.htmlhttps://trino.io/docs/current/release/release-0.66.htmlhttps://trino.io/docs/current/release/release-0.128.htmlhttps://trino.io/docs/current/release/release-0.186.htmlhttps://trino.io/docs/current/release/release-0.67.htmlhttps://trino.io/docs/current/release/release-0.129.htmlhttps://trino.io/docs/current/release/release-0.130.htmlhttps://trino.io/docs/current/release/release-0.187.htmlhttps://trino.io/docs/current/release/release-0.131.htmlhttps://trino.io/docs/current/release/release-0.132.htmlhttps://trino.io/docs/current/release/release-0.133.htmlhttps://trino.io/docs/current/release/release-0.134.htmlhttps://trino.io/docs/current/release/release-0.135.htmlhttps://trino.io/docs/current/release/release-0.136.htmlhttps://trino.io/docs/current/release/release-0.137.htmlhttps://trino.io/docs/current/release/release-0.138.htmlhttps://trino.io/docs/current/release/release-0.139.htmlhttps://trino.io/docs/current/release/release-0.140.htmlhttps://trino.io/docs/current/release/release-0.141.htmlhttps://trino.io/docs/current/release/release-0.142.htmlhttps://trino.io/docs/current/release/release-0.143.htmlhttps://trino.io/docs/current/release/release-0.144.htmlhttps://trino.io/docs/current/release/release-0.144.1.htmlhttps://trino.io/docs/current/release/release-0.144.2.htmlhttps://trino.io/docs/current/release/release-0.144.3.htmlhttps://trino.io/docs/current/release/release-0.144.4.htmlhttps://trino.io/docs/current/release/release-0.144.5.htmlhttps://trino.io/docs/current/release/release-0.144.6.htmlhttps://trino.io/docs/current/release/release-0.144.7.htmlhttps://trino.io/docs/current/release/release-0.145.htmlhttps://trino.io/docs/current/release/release-0.146.htmlhttps://trino.io/docs/current/release/release-0.147.htmlhttps://trino.io/docs/current/release/release-0.148.htmlhttps://trino.io/docs/current/release/release-0.149.htmlhttps://trino.io/docs/current/genindex.htmlhttps://trino.io/docs/current/search.html \ No newline at end of file diff --git a/430/sql.html b/430/sql.html new file mode 100644 index 000000000..8179aa7ec --- /dev/null +++ b/430/sql.html @@ -0,0 +1,958 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SQL statement syntax — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ + +
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/alter-materialized-view.html b/430/sql/alter-materialized-view.html new file mode 100644 index 000000000..8e55824cc --- /dev/null +++ b/430/sql/alter-materialized-view.html @@ -0,0 +1,964 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ALTER MATERIALIZED VIEW — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

ALTER MATERIALIZED VIEW#

+
+

Synopsis#

+
ALTER MATERIALIZED VIEW [ IF EXISTS ] name RENAME TO new_name
+ALTER MATERIALIZED VIEW name SET PROPERTIES property_name = expression [, ...]
+
+
+
+
+

Description#

+

Change the name of an existing materialized view.

+

The optional IF EXISTS clause causes the error to be suppressed if the +materialized view does not exist. The error is not suppressed if the +materialized view does not exist, but a table or view with the given name +exists.

+
+

SET PROPERTIES#

+

The ALTER MATERIALIZED VIEW SET PROPERTIES statement followed by some number +of property_name and expression pairs applies the specified properties +and values to a materialized view. Ommitting an already-set property from this +statement leaves that property unchanged in the materialized view.

+

A property in a SET PROPERTIES statement can be set to DEFAULT, which +reverts its value back to the default in that materialized view.

+

Support for ALTER MATERIALIZED VIEW SET PROPERTIES varies between +connectors. Refer to the connector documentation for more details.

+
+
+
+

Examples#

+

Rename materialized view people to users in the current schema:

+
ALTER MATERIALIZED VIEW people RENAME TO users;
+
+
+

Rename materialized view people to users, if materialized view +people exists in the current catalog and schema:

+
ALTER MATERIALIZED VIEW IF EXISTS people RENAME TO users;
+
+
+

Set view properties (x = y) in materialized view people:

+
ALTER MATERIALIZED VIEW people SET PROPERTIES x = 'y';
+
+
+

Set multiple view properties (foo = 123 and foo bar = 456) in +materialized view people:

+
ALTER MATERIALIZED VIEW people SET PROPERTIES foo = 123, "foo bar" = 456;
+
+
+

Set view property x to its default value in materialized view people:

+
ALTER MATERIALIZED VIEW people SET PROPERTIES x = DEFAULT;
+
+
+
+
+

See also#

+ +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/alter-schema.html b/430/sql/alter-schema.html new file mode 100644 index 000000000..5036d06a5 --- /dev/null +++ b/430/sql/alter-schema.html @@ -0,0 +1,927 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ALTER SCHEMA — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

ALTER SCHEMA#

+
+

Synopsis#

+
ALTER SCHEMA name RENAME TO new_name
+ALTER SCHEMA name SET AUTHORIZATION ( user | USER user | ROLE role )
+
+
+
+
+

Description#

+

Change the definition of an existing schema.

+
+
+

Examples#

+

Rename schema web to traffic:

+
ALTER SCHEMA web RENAME TO traffic
+
+
+

Change owner of schema web to user alice:

+
ALTER SCHEMA web SET AUTHORIZATION alice
+
+
+

Allow everyone to drop schema and create tables in schema web:

+
ALTER SCHEMA web SET AUTHORIZATION ROLE PUBLIC
+
+
+
+
+

See Also#

+

CREATE SCHEMA

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/alter-table.html b/430/sql/alter-table.html new file mode 100644 index 000000000..0752cb0da --- /dev/null +++ b/430/sql/alter-table.html @@ -0,0 +1,1024 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ALTER TABLE — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

ALTER TABLE#

+
+

Synopsis#

+
ALTER TABLE [ IF EXISTS ] name RENAME TO new_name
+ALTER TABLE [ IF EXISTS ] name ADD COLUMN [ IF NOT EXISTS ] column_name data_type
+  [ NOT NULL ] [ COMMENT comment ]
+  [ WITH ( property_name = expression [, ...] ) ]
+ALTER TABLE [ IF EXISTS ] name DROP COLUMN [ IF EXISTS ] column_name
+ALTER TABLE [ IF EXISTS ] name RENAME COLUMN [ IF EXISTS ] old_name TO new_name
+ALTER TABLE [ IF EXISTS ] name ALTER COLUMN column_name SET DATA TYPE new_type
+ALTER TABLE name SET AUTHORIZATION ( user | USER user | ROLE role )
+ALTER TABLE name SET PROPERTIES property_name = expression [, ...]
+ALTER TABLE name EXECUTE command [ ( parameter => expression [, ... ] ) ]
+    [ WHERE expression ]
+
+
+
+
+

Description#

+

Change the definition of an existing table.

+

The optional IF EXISTS (when used before the table name) clause causes the error to be suppressed if the table does not exists.

+

The optional IF EXISTS (when used before the column name) clause causes the error to be suppressed if the column does not exists.

+

The optional IF NOT EXISTS clause causes the error to be suppressed if the column already exists.

+
+

SET PROPERTIES#

+

The ALTER TABLE SET PROPERTIES statement followed by some number +of property_name and expression pairs applies the specified properties +and values to a table. Ommitting an already-set property from this +statement leaves that property unchanged in the table.

+

A property in a SET PROPERTIES statement can be set to DEFAULT, which +reverts its value back to the default in that table.

+

Support for ALTER TABLE SET PROPERTIES varies between +connectors, as not all connectors support modifying table properties.

+
+
+

EXECUTE#

+

The ALTER TABLE EXECUTE statement followed by a command and +parameters modifies the table according to the specified command and +parameters. ALTER TABLE EXECUTE supports different commands on a +per-connector basis.

+

You can use the => operator for passing named parameter values. +The left side is the name of the parameter, the right side is the value being passed:

+
ALTER TABLE hive.schema.test_table EXECUTE optimize(file_size_threshold => '10MB')
+
+
+
+
+
+

Examples#

+

Rename table users to people:

+
ALTER TABLE users RENAME TO people;
+
+
+

Rename table users to people if table users exists:

+
ALTER TABLE IF EXISTS users RENAME TO people;
+
+
+

Add column zip to the users table:

+
ALTER TABLE users ADD COLUMN zip varchar;
+
+
+

Add column zip to the users table if table users exists and column zip not already exists:

+
ALTER TABLE IF EXISTS users ADD COLUMN IF NOT EXISTS zip varchar;
+
+
+

Drop column zip from the users table:

+
ALTER TABLE users DROP COLUMN zip;
+
+
+

Drop column zip from the users table if table users and column zip exists:

+
ALTER TABLE IF EXISTS users DROP COLUMN IF EXISTS zip;
+
+
+

Rename column id to user_id in the users table:

+
ALTER TABLE users RENAME COLUMN id TO user_id;
+
+
+

Rename column id to user_id in the users table if table users and column id exists:

+
ALTER TABLE IF EXISTS users RENAME column IF EXISTS id to user_id;
+
+
+

Change type of column id to bigint in the users table:

+
ALTER TABLE users ALTER COLUMN id SET DATA TYPE bigint;
+
+
+

Change owner of table people to user alice:

+
ALTER TABLE people SET AUTHORIZATION alice
+
+
+

Allow everyone with role public to drop and alter table people:

+
ALTER TABLE people SET AUTHORIZATION ROLE PUBLIC
+
+
+

Set table properties (x = y) in table people:

+
ALTER TABLE people SET PROPERTIES x = 'y';
+
+
+

Set multiple table properties (foo = 123 and foo bar = 456) in +table people:

+
ALTER TABLE people SET PROPERTIES foo = 123, "foo bar" = 456;
+
+
+

Set table property x to its default value in table``people``:

+
ALTER TABLE people SET PROPERTIES x = DEFAULT;
+
+
+

Collapse files in a table that are over 10 megabytes in size, as supported by +the Hive connector:

+
ALTER TABLE hive.schema.test_table EXECUTE optimize(file_size_threshold => '10MB')
+
+
+
+
+

See also#

+

CREATE TABLE

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/alter-view.html b/430/sql/alter-view.html new file mode 100644 index 000000000..5e0f94800 --- /dev/null +++ b/430/sql/alter-view.html @@ -0,0 +1,923 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ALTER VIEW — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

ALTER VIEW#

+
+

Synopsis#

+
ALTER VIEW name RENAME TO new_name
+ALTER VIEW name SET AUTHORIZATION ( user | USER user | ROLE role )
+
+
+
+
+

Description#

+

Change the definition of an existing view.

+
+
+

Examples#

+

Rename view people to users:

+
ALTER VIEW people RENAME TO users
+
+
+

Change owner of VIEW people to user alice:

+
ALTER VIEW people SET AUTHORIZATION alice
+
+
+
+
+

See also#

+

CREATE VIEW

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/analyze.html b/430/sql/analyze.html new file mode 100644 index 000000000..27a73050b --- /dev/null +++ b/430/sql/analyze.html @@ -0,0 +1,934 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ANALYZE — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

ANALYZE#

+
+

Synopsis#

+
ANALYZE table_name [ WITH ( property_name = expression [, ...] ) ]
+
+
+
+
+

Description#

+

Collects table and column statistics for a given table.

+

The optional WITH clause can be used to provide connector-specific properties. +To list all available properties, run the following query:

+
SELECT * FROM system.metadata.analyze_properties
+
+
+
+
+

Examples#

+

Analyze table web to collect table and column statistics:

+
ANALYZE web;
+
+
+

Analyze table stores in catalog hive and schema default:

+
ANALYZE hive.default.stores;
+
+
+

Analyze partitions '1992-01-01', '1992-01-02' from a Hive partitioned table sales:

+
ANALYZE hive.default.sales WITH (partitions = ARRAY[ARRAY['1992-01-01'], ARRAY['1992-01-02']]);
+
+
+

Analyze partitions with complex partition key (state and city columns) from a Hive partitioned table customers:

+
ANALYZE hive.default.customers WITH (partitions = ARRAY[ARRAY['CA', 'San Francisco'], ARRAY['NY', 'NY']]);
+
+
+

Analyze only columns department and product_id for partitions '1992-01-01', '1992-01-02' from a Hive partitioned +table sales:

+
ANALYZE hive.default.sales WITH (
+    partitions = ARRAY[ARRAY['1992-01-01'], ARRAY['1992-01-02']],
+    columns = ARRAY['department', 'product_id']);
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/call.html b/430/sql/call.html new file mode 100644 index 000000000..c2b335b58 --- /dev/null +++ b/430/sql/call.html @@ -0,0 +1,926 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + CALL — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

CALL#

+
+

Synopsis#

+
CALL procedure_name ( [ name => ] expression [, ...] )
+
+
+
+
+

Description#

+

Call a procedure.

+

Procedures can be provided by connectors to perform data manipulation or +administrative tasks. For example, the System connector defines a +procedure for killing a running query.

+

Some connectors, such as the PostgreSQL connector, are for systems +that have their own stored procedures. These stored procedures are separate +from the connector-defined procedures discussed here and thus are not +directly callable via CALL.

+

See connector documentation for details on available procedures.

+
+
+

Examples#

+

Call a procedure using positional arguments:

+
CALL test(123, 'apple');
+
+
+

Call a procedure using named arguments:

+
CALL test(name => 'apple', id => 123);
+
+
+

Call a procedure using a fully qualified name:

+
CALL catalog.schema.test();
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/comment.html b/430/sql/comment.html new file mode 100644 index 000000000..096dfa2c6 --- /dev/null +++ b/430/sql/comment.html @@ -0,0 +1,926 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + COMMENT — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

COMMENT#

+
+

Synopsis#

+
COMMENT ON ( TABLE | VIEW | COLUMN ) name IS 'comments'
+
+
+
+
+

Description#

+

Set the comment for a object. The comment can be removed by setting the comment to NULL.

+
+
+

Examples#

+

Change the comment for the users table to be master table:

+
COMMENT ON TABLE users IS 'master table';
+
+
+

Change the comment for the users view to be master view:

+
COMMENT ON VIEW users IS 'master view';
+
+
+

Change the comment for the users.name column to be full name:

+
COMMENT ON COLUMN users.name IS 'full name';
+
+
+
+
+

See also#

+

Comments

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/commit.html b/430/sql/commit.html new file mode 100644 index 000000000..74da3764a --- /dev/null +++ b/430/sql/commit.html @@ -0,0 +1,918 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + COMMIT — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

COMMIT#

+
+

Synopsis#

+
COMMIT [ WORK ]
+
+
+
+
+

Description#

+

Commit the current transaction.

+
+
+

Examples#

+
COMMIT;
+COMMIT WORK;
+
+
+
+
+

See also#

+

ROLLBACK, START TRANSACTION

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/create-materialized-view.html b/430/sql/create-materialized-view.html new file mode 100644 index 000000000..cb918f32c --- /dev/null +++ b/430/sql/create-materialized-view.html @@ -0,0 +1,999 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + CREATE MATERIALIZED VIEW — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

CREATE MATERIALIZED VIEW#

+
+

Synopsis#

+
CREATE [ OR REPLACE ] MATERIALIZED VIEW
+[ IF NOT EXISTS ] view_name
+[ GRACE PERIOD interval ]
+[ COMMENT string ]
+[ WITH properties ]
+AS query
+
+
+
+
+

Description#

+

Create and validate the definition of a new materialized view view_name of a +SELECT query. You need to run the REFRESH MATERIALIZED VIEW +statement after the creation to populate the materialized view with data. This +materialized view is a physical manifestation of the query results at time of +refresh. The data is stored, and can be referenced by future queries.

+

Queries accessing materialized views are typically faster than retrieving data +from a view created with the same query. Any computation, aggregation, and other +operation to create the data is performed once during refresh of the +materialized views, as compared to each time of accessing the view. Multiple +reads of view data over time, or by multiple users, all trigger repeated +processing. This is avoided for materialized views.

+

The optional OR REPLACE clause causes the materialized view to be replaced +if it already exists rather than raising an error.

+

The optional IF NOT EXISTS clause causes the materialized view only to be +created if it does not exist yet.

+

Note that OR REPLACE and IF NOT EXISTS are mutually exclusive clauses.

+

The optional GRACE PERIOD clause specifies how long the query materialization +is used for querying. If the time elapsed since last materialized view refresh +is greater than the grace period, the materialized view acts as a normal view and +the materialized data is not used. If not specified, the grace period defaults to +infinity. See REFRESH MATERIALIZED VIEW for more about refreshing +materialized views.

+

The optional COMMENT clause causes a string comment to be stored with +the metadata about the materialized view. The comment is displayed with the +SHOW CREATE MATERIALIZED VIEW statement and is available in the table +system.metadata.materialized_view_properties.

+

The optional WITH clause is used to define properties for the materialized +view creation. Separate multiple property/value pairs by commas. The connector +uses the properties as input parameters for the materialized view refresh +operation. The supported properties are different for each connector and +detailed in the SQL support section of the specific connector’s documentation.

+

After successful creation, all metadata about the materialized view is available +in a system table.

+
+
+

Examples#

+

Create a simple materialized view cancelled_orders over the orders table +that only includes cancelled orders. Note that orderstatus is a numeric +value that is potentially meaningless to a consumer, yet the name of the view +clarifies the content:

+
CREATE MATERIALIZED VIEW cancelled_orders
+AS
+    SELECT orderkey, totalprice
+    FROM orders
+    WHERE orderstatus = 3;
+
+
+

Create or replace a materialized view order_totals_by_date that summarizes +orders across all orders from all customers:

+
CREATE OR REPLACE MATERIALIZED VIEW order_totals_by_date
+AS
+    SELECT orderdate, sum(totalprice) AS price
+    FROM orders
+    GROUP BY orderdate;
+
+
+

Create a materialized view for a catalog using the Iceberg connector, with a +comment and partitioning on two fields in the storage:

+
CREATE MATERIALIZED VIEW orders_nation_mkgsegment
+COMMENT 'Orders with nation and market segment data'
+WITH ( partitioning = ARRAY['mktsegment', 'nationkey'] )
+AS
+    SELECT o.*, c.nationkey, c.mktsegment
+    FROM orders AS o
+    JOIN customer AS c
+    ON o.custkey = c.custkey;
+
+
+

Set multiple properties:

+
WITH ( format = 'ORC', partitioning = ARRAY['_date'] )
+
+
+

Show defined materialized view properties for all catalogs:

+
SELECT * FROM system.metadata.materialized_view_properties;
+
+
+

Show metadata about the materialized views in all catalogs:

+
SELECT * FROM system.metadata.materialized_views;
+
+
+
+
+

See also#

+ +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/create-role.html b/430/sql/create-role.html new file mode 100644 index 000000000..ac697af44 --- /dev/null +++ b/430/sql/create-role.html @@ -0,0 +1,939 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + CREATE ROLE — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

CREATE ROLE#

+
+

Synopsis#

+
CREATE ROLE role_name
+[ WITH ADMIN ( user | USER user | ROLE role | CURRENT_USER | CURRENT_ROLE ) ]
+[ IN catalog ]
+
+
+
+
+

Description#

+

CREATE ROLE creates the specified role.

+

The optional WITH ADMIN clause causes the role to be created with +the specified user as a role admin. A role admin has permission to drop +or grant a role. If the optional WITH ADMIN clause is not +specified, the role is created with current user as admin.

+

The optional IN catalog clause creates the role in a catalog as opposed +to a system role.

+
+
+

Examples#

+

Create role admin

+
CREATE ROLE admin;
+
+
+

Create role moderator with admin bob:

+
CREATE ROLE moderator WITH ADMIN USER bob;
+
+
+
+
+

Limitations#

+

Some connectors do not support role management. +See connector documentation for more details.

+
+
+

See also#

+

DROP ROLE, SET ROLE, GRANT ROLES, REVOKE ROLES

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/create-schema.html b/430/sql/create-schema.html new file mode 100644 index 000000000..28f495805 --- /dev/null +++ b/430/sql/create-schema.html @@ -0,0 +1,958 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + CREATE SCHEMA — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

CREATE SCHEMA#

+
+

Synopsis#

+
CREATE SCHEMA [ IF NOT EXISTS ] schema_name
+[ AUTHORIZATION ( user | USER user | ROLE role ) ]
+[ WITH ( property_name = expression [, ...] ) ]
+
+
+
+
+

Description#

+

Create a new, empty schema. A schema is a container that +holds tables, views and other database objects.

+

The optional IF NOT EXISTS clause causes the error to be +suppressed if the schema already exists.

+

The optional AUTHORIZATION clause can be used to set the +owner of the newly created schema to a user or role.

+

The optional WITH clause can be used to set properties +on the newly created schema. To list all available schema +properties, run the following query:

+
SELECT * FROM system.metadata.schema_properties
+
+
+
+
+

Examples#

+

Create a new schema web in the current catalog:

+
CREATE SCHEMA web
+
+
+

Create a new schema sales in the hive catalog:

+
CREATE SCHEMA hive.sales
+
+
+

Create the schema traffic if it does not already exist:

+
CREATE SCHEMA IF NOT EXISTS traffic
+
+
+

Create a new schema web and set the owner to user alice:

+
CREATE SCHEMA web AUTHORIZATION alice
+
+
+

Create a new schema web, set the LOCATION property to /hive/data/web +and set the owner to user alice:

+
CREATE SCHEMA web AUTHORIZATION alice WITH ( LOCATION = '/hive/data/web' )
+
+
+

Create a new schema web and allow everyone to drop schema and create tables +in schema web:

+
CREATE SCHEMA web AUTHORIZATION ROLE PUBLIC
+
+
+

Create a new schema web, set the LOCATION property to /hive/data/web +and allow everyone to drop schema and create tables in schema web:

+
CREATE SCHEMA web AUTHORIZATION ROLE PUBLIC WITH ( LOCATION = '/hive/data/web' )
+
+
+
+
+

See also#

+

ALTER SCHEMA, DROP SCHEMA

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/create-table-as.html b/430/sql/create-table-as.html new file mode 100644 index 000000000..6f1342ff2 --- /dev/null +++ b/430/sql/create-table-as.html @@ -0,0 +1,958 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + CREATE TABLE AS — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

CREATE TABLE AS#

+
+

Synopsis#

+
CREATE TABLE [ IF NOT EXISTS ] table_name [ ( column_alias, ... ) ]
+[ COMMENT table_comment ]
+[ WITH ( property_name = expression [, ...] ) ]
+AS query
+[ WITH [ NO ] DATA ]
+
+
+
+
+

Description#

+

Create a new table containing the result of a SELECT query. +Use CREATE TABLE to create an empty table.

+

The optional IF NOT EXISTS clause causes the error to be +suppressed if the table already exists.

+

The optional WITH clause can be used to set properties +on the newly created table. To list all available table +properties, run the following query:

+
SELECT * FROM system.metadata.table_properties
+
+
+
+
+

Examples#

+

Create a new table orders_column_aliased with the results of a query and the given column names:

+
CREATE TABLE orders_column_aliased (order_date, total_price)
+AS
+SELECT orderdate, totalprice
+FROM orders
+
+
+

Create a new table orders_by_date that summarizes orders:

+
CREATE TABLE orders_by_date
+COMMENT 'Summary of orders by date'
+WITH (format = 'ORC')
+AS
+SELECT orderdate, sum(totalprice) AS price
+FROM orders
+GROUP BY orderdate
+
+
+

Create the table orders_by_date if it does not already exist:

+
CREATE TABLE IF NOT EXISTS orders_by_date AS
+SELECT orderdate, sum(totalprice) AS price
+FROM orders
+GROUP BY orderdate
+
+
+

Create a new empty_nation table with the same schema as nation and no data:

+
CREATE TABLE empty_nation AS
+SELECT *
+FROM nation
+WITH NO DATA
+
+
+
+
+

See also#

+

CREATE TABLE, SELECT

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/create-table.html b/430/sql/create-table.html new file mode 100644 index 000000000..3e402faf6 --- /dev/null +++ b/430/sql/create-table.html @@ -0,0 +1,976 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + CREATE TABLE — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

CREATE TABLE#

+
+

Synopsis#

+
CREATE TABLE [ IF NOT EXISTS ]
+table_name (
+  { column_name data_type [ NOT NULL ]
+      [ COMMENT comment ]
+      [ WITH ( property_name = expression [, ...] ) ]
+  | LIKE existing_table_name
+      [ { INCLUDING | EXCLUDING } PROPERTIES ]
+  }
+  [, ...]
+)
+[ COMMENT table_comment ]
+[ WITH ( property_name = expression [, ...] ) ]
+
+
+
+
+

Description#

+

Create a new, empty table with the specified columns. +Use CREATE TABLE AS to create a table with data.

+

The optional IF NOT EXISTS clause causes the error to be +suppressed if the table already exists.

+

The optional WITH clause can be used to set properties +on the newly created table or on single columns. To list all available table +properties, run the following query:

+
SELECT * FROM system.metadata.table_properties
+
+
+

To list all available column properties, run the following query:

+
SELECT * FROM system.metadata.column_properties
+
+
+

The LIKE clause can be used to include all the column definitions from +an existing table in the new table. Multiple LIKE clauses may be +specified, which allows copying the columns from multiple tables.

+

If INCLUDING PROPERTIES is specified, all of the table properties are +copied to the new table. If the WITH clause specifies the same property +name as one of the copied properties, the value from the WITH clause +will be used. The default behavior is EXCLUDING PROPERTIES. The +INCLUDING PROPERTIES option maybe specified for at most one table.

+
+
+

Examples#

+

Create a new table orders:

+
CREATE TABLE orders (
+  orderkey bigint,
+  orderstatus varchar,
+  totalprice double,
+  orderdate date
+)
+WITH (format = 'ORC')
+
+
+

Create the table orders if it does not already exist, adding a table comment +and a column comment:

+
CREATE TABLE IF NOT EXISTS orders (
+  orderkey bigint,
+  orderstatus varchar,
+  totalprice double COMMENT 'Price in cents.',
+  orderdate date
+)
+COMMENT 'A table to keep track of orders.'
+
+
+

Create the table bigger_orders using the columns from orders +plus additional columns at the start and end:

+
CREATE TABLE bigger_orders (
+  another_orderkey bigint,
+  LIKE orders,
+  another_orderdate date
+)
+
+
+
+
+

See also#

+

ALTER TABLE, DROP TABLE, CREATE TABLE AS, SHOW CREATE TABLE

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/create-view.html b/430/sql/create-view.html new file mode 100644 index 000000000..37480c4aa --- /dev/null +++ b/430/sql/create-view.html @@ -0,0 +1,967 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + CREATE VIEW — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

CREATE VIEW#

+
+

Synopsis#

+
CREATE [ OR REPLACE ] VIEW view_name
+[ COMMENT view_comment ]
+[ SECURITY { DEFINER | INVOKER } ]
+AS query
+
+
+
+
+

Description#

+

Create a new view of a SELECT query. The view is a logical table +that can be referenced by future queries. Views do not contain any data. +Instead, the query stored by the view is executed every time the view is +referenced by another query.

+

The optional OR REPLACE clause causes the view to be replaced if it +already exists rather than raising an error.

+
+
+

Security#

+

In the default DEFINER security mode, tables referenced in the view +are accessed using the permissions of the view owner (the creator or +definer of the view) rather than the user executing the query. This +allows providing restricted access to the underlying tables, for which +the user may not be allowed to access directly.

+

In the INVOKER security mode, tables referenced in the view are accessed +using the permissions of the user executing the query (the invoker of the view). +A view created in this mode is simply a stored query.

+

Regardless of the security mode, the current_user function will +always return the user executing the query and thus may be used +within views to filter out rows or otherwise restrict access.

+
+
+

Examples#

+

Create a simple view test over the orders table:

+
CREATE VIEW test AS
+SELECT orderkey, orderstatus, totalprice / 2 AS half
+FROM orders
+
+
+

Create a view test_with_comment with a view comment:

+
CREATE VIEW test_with_comment
+COMMENT 'A view to keep track of orders.'
+AS
+SELECT orderkey, orderstatus, totalprice
+FROM orders
+
+
+

Create a view orders_by_date that summarizes orders:

+
CREATE VIEW orders_by_date AS
+SELECT orderdate, sum(totalprice) AS price
+FROM orders
+GROUP BY orderdate
+
+
+

Create a view that replaces an existing view:

+
CREATE OR REPLACE VIEW test AS
+SELECT orderkey, orderstatus, totalprice / 4 AS quarter
+FROM orders
+
+
+
+
+

See also#

+

DROP VIEW, SHOW CREATE VIEW

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/deallocate-prepare.html b/430/sql/deallocate-prepare.html new file mode 100644 index 000000000..f12c6f3a8 --- /dev/null +++ b/430/sql/deallocate-prepare.html @@ -0,0 +1,919 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + DEALLOCATE PREPARE — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

DEALLOCATE PREPARE#

+
+

Synopsis#

+
DEALLOCATE PREPARE statement_name
+
+
+
+
+

Description#

+

Removes a statement with the name statement_name from the list of prepared +statements in a session.

+
+
+

Examples#

+

Deallocate a statement with the name my_query:

+
DEALLOCATE PREPARE my_query;
+
+
+
+
+

See also#

+

PREPARE, EXECUTE, EXECUTE IMMEDIATE

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/delete.html b/430/sql/delete.html new file mode 100644 index 000000000..3ee1eb85d --- /dev/null +++ b/430/sql/delete.html @@ -0,0 +1,929 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + DELETE — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

DELETE#

+
+

Synopsis#

+
DELETE FROM table_name [ WHERE condition ]
+
+
+
+
+

Description#

+

Delete rows from a table. If the WHERE clause is specified, only the +matching rows are deleted. Otherwise, all rows from the table are deleted.

+
+
+

Examples#

+

Delete all line items shipped by air:

+
DELETE FROM lineitem WHERE shipmode = 'AIR';
+
+
+

Delete all line items for low priority orders:

+
DELETE FROM lineitem
+WHERE orderkey IN (SELECT orderkey FROM orders WHERE priority = 'LOW');
+
+
+

Delete all orders:

+
DELETE FROM orders;
+
+
+
+
+

Limitations#

+

Some connectors have limited or no support for DELETE. +See connector documentation for more details.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/deny.html b/430/sql/deny.html new file mode 100644 index 000000000..be48c6e7e --- /dev/null +++ b/430/sql/deny.html @@ -0,0 +1,942 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + DENY — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

DENY#

+
+

Synopsis#

+
DENY ( privilege [, ...] | ( ALL PRIVILEGES ) )
+ON ( table_name | TABLE table_name | SCHEMA schema_name)
+TO ( user | USER user | ROLE role )
+
+
+
+
+

Description#

+

Denies the specified privileges to the specified grantee.

+

Deny on a table rejects the specified privilege on all current and future +columns of the table.

+

Deny on a schema rejects the specified privilege on all current and future +columns of all current and future tables of the schema.

+
+
+

Examples#

+

Deny INSERT and SELECT privileges on the table orders +to user alice:

+
DENY INSERT, SELECT ON orders TO alice;
+
+
+

Deny DELETE privilege on the schema finance to user bob:

+
DENY DELETE ON SCHEMA finance TO bob;
+
+
+

Deny SELECT privilege on the table orders to everyone:

+
DENY SELECT ON orders TO ROLE PUBLIC;
+
+
+
+
+

Limitations#

+

The system access controls as well as the connectors provided by default +in Trino have no support for DENY.

+
+
+

See also#

+

GRANT, REVOKE, SHOW GRANTS

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/describe-input.html b/430/sql/describe-input.html new file mode 100644 index 000000000..6f0d69384 --- /dev/null +++ b/430/sql/describe-input.html @@ -0,0 +1,945 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + DESCRIBE INPUT — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

DESCRIBE INPUT#

+
+

Synopsis#

+
DESCRIBE INPUT statement_name
+
+
+
+
+

Description#

+

Lists the input parameters of a prepared statement along with the +position and type of each parameter. Parameter types that cannot be +determined will appear as unknown.

+
+
+

Examples#

+

Prepare and describe a query with three parameters:

+
PREPARE my_select1 FROM
+SELECT ? FROM nation WHERE regionkey = ? AND name < ?;
+
+
+
DESCRIBE INPUT my_select1;
+
+
+
 Position | Type
+--------------------
+        0 | unknown
+        1 | bigint
+        2 | varchar
+(3 rows)
+
+
+

Prepare and describe a query with no parameters:

+
PREPARE my_select2 FROM
+SELECT * FROM nation;
+
+
+
DESCRIBE INPUT my_select2;
+
+
+
 Position | Type
+-----------------
+(0 rows)
+
+
+
+
+

See also#

+

PREPARE

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/describe-output.html b/430/sql/describe-output.html new file mode 100644 index 000000000..7f5c1efb9 --- /dev/null +++ b/430/sql/describe-output.html @@ -0,0 +1,962 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + DESCRIBE OUTPUT — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

DESCRIBE OUTPUT#

+
+

Synopsis#

+
DESCRIBE OUTPUT statement_name
+
+
+
+
+

Description#

+

List the output columns of a prepared statement, including the +column name (or alias), catalog, schema, table, type, type size in +bytes, and a boolean indicating if the column is aliased.

+
+
+

Examples#

+

Prepare and describe a query with four output columns:

+
PREPARE my_select1 FROM
+SELECT * FROM nation;
+
+
+
DESCRIBE OUTPUT my_select1;
+
+
+
 Column Name | Catalog | Schema | Table  |  Type   | Type Size | Aliased
+-------------+---------+--------+--------+---------+-----------+---------
+ nationkey   | tpch    | sf1    | nation | bigint  |         8 | false
+ name        | tpch    | sf1    | nation | varchar |         0 | false
+ regionkey   | tpch    | sf1    | nation | bigint  |         8 | false
+ comment     | tpch    | sf1    | nation | varchar |         0 | false
+(4 rows)
+
+
+

Prepare and describe a query whose output columns are expressions:

+
PREPARE my_select2 FROM
+SELECT count(*) as my_count, 1+2 FROM nation;
+
+
+
DESCRIBE OUTPUT my_select2;
+
+
+
 Column Name | Catalog | Schema | Table |  Type  | Type Size | Aliased
+-------------+---------+--------+-------+--------+-----------+---------
+ my_count    |         |        |       | bigint |         8 | true
+ _col1       |         |        |       | bigint |         8 | false
+(2 rows)
+
+
+

Prepare and describe a row count query:

+
PREPARE my_create FROM
+CREATE TABLE foo AS SELECT * FROM nation;
+
+
+
DESCRIBE OUTPUT my_create;
+
+
+
 Column Name | Catalog | Schema | Table |  Type  | Type Size | Aliased
+-------------+---------+--------+-------+--------+-----------+---------
+ rows        |         |        |       | bigint |         8 | false
+(1 row)
+
+
+
+
+

See also#

+

PREPARE

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/describe.html b/430/sql/describe.html new file mode 100644 index 000000000..ffc142d15 --- /dev/null +++ b/430/sql/describe.html @@ -0,0 +1,899 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + DESCRIBE — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

DESCRIBE#

+
+

Synopsis#

+
DESCRIBE table_name
+
+
+
+
+

Description#

+

DESCRIBE is an alias for SHOW COLUMNS.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/drop-materialized-view.html b/430/sql/drop-materialized-view.html new file mode 100644 index 000000000..e54a866e8 --- /dev/null +++ b/430/sql/drop-materialized-view.html @@ -0,0 +1,928 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + DROP MATERIALIZED VIEW — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

DROP MATERIALIZED VIEW#

+
+

Synopsis#

+
DROP MATERIALIZED VIEW [ IF EXISTS ] view_name
+
+
+
+
+

Description#

+

Drop an existing materialized view view_name.

+

The optional IF EXISTS clause causes the error to be suppressed if +the materialized view does not exist.

+
+
+

Examples#

+

Drop the materialized view orders_by_date:

+
DROP MATERIALIZED VIEW orders_by_date;
+
+
+

Drop the materialized view orders_by_date if it exists:

+
DROP MATERIALIZED VIEW IF EXISTS orders_by_date;
+
+
+
+
+

See also#

+ +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/drop-role.html b/430/sql/drop-role.html new file mode 100644 index 000000000..ea5d5d6b3 --- /dev/null +++ b/430/sql/drop-role.html @@ -0,0 +1,932 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + DROP ROLE — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

DROP ROLE#

+
+

Synopsis#

+
DROP ROLE role_name
+[ IN catalog ]
+
+
+
+
+

Description#

+

DROP ROLE drops the specified role.

+

For DROP ROLE statement to succeed, the user executing it should possess +admin privileges for the given role.

+

The optional IN catalog clause drops the role in a catalog as opposed +to a system role.

+
+
+

Examples#

+

Drop role admin

+
DROP ROLE admin;
+
+
+
+
+

Limitations#

+

Some connectors do not support role management. +See connector documentation for more details.

+
+
+

See also#

+

CREATE ROLE, SET ROLE, GRANT ROLES, REVOKE ROLES

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/drop-schema.html b/430/sql/drop-schema.html new file mode 100644 index 000000000..35e0bdaa1 --- /dev/null +++ b/430/sql/drop-schema.html @@ -0,0 +1,932 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + DROP SCHEMA — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

DROP SCHEMA#

+
+

Synopsis#

+
DROP SCHEMA [ IF EXISTS ] schema_name [ CASCADE | RESTRICT ]
+
+
+
+
+

Description#

+

Drop an existing schema. The schema must be empty.

+

The optional IF EXISTS clause causes the error to be suppressed if +the schema does not exist.

+
+
+

Examples#

+

Drop the schema web:

+
DROP SCHEMA web
+
+
+

Drop the schema sales if it exists:

+
DROP SCHEMA IF EXISTS sales
+
+
+

Drop the schema archive, along with everything it contains:

+
DROP SCHEMA archive CASCADE
+
+
+

Drop the schema archive, only if there are no objects contained in the schema:

+
DROP SCHEMA archive RESTRICT
+
+
+
+
+

See also#

+

ALTER SCHEMA, CREATE SCHEMA

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/drop-table.html b/430/sql/drop-table.html new file mode 100644 index 000000000..5a5742ec0 --- /dev/null +++ b/430/sql/drop-table.html @@ -0,0 +1,924 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + DROP TABLE — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

DROP TABLE#

+
+

Synopsis#

+
DROP TABLE  [ IF EXISTS ] table_name
+
+
+
+
+

Description#

+

Drops an existing table.

+

The optional IF EXISTS clause causes the error to be suppressed if +the table does not exist.

+
+
+

Examples#

+

Drop the table orders_by_date:

+
DROP TABLE orders_by_date
+
+
+

Drop the table orders_by_date if it exists:

+
DROP TABLE IF EXISTS orders_by_date
+
+
+
+
+

See also#

+

ALTER TABLE, CREATE TABLE

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/drop-view.html b/430/sql/drop-view.html new file mode 100644 index 000000000..63398cc22 --- /dev/null +++ b/430/sql/drop-view.html @@ -0,0 +1,924 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + DROP VIEW — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

DROP VIEW#

+
+

Synopsis#

+
DROP VIEW [ IF EXISTS ] view_name
+
+
+
+
+

Description#

+

Drop an existing view.

+

The optional IF EXISTS clause causes the error to be suppressed if +the view does not exist.

+
+
+

Examples#

+

Drop the view orders_by_date:

+
DROP VIEW orders_by_date
+
+
+

Drop the view orders_by_date if it exists:

+
DROP VIEW IF EXISTS orders_by_date
+
+
+
+
+

See also#

+

CREATE VIEW

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/execute-immediate.html b/430/sql/execute-immediate.html new file mode 100644 index 000000000..f8c74dbbb --- /dev/null +++ b/430/sql/execute-immediate.html @@ -0,0 +1,932 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + EXECUTE IMMEDIATE — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

EXECUTE IMMEDIATE#

+
+

Synopsis#

+
EXECUTE IMMEDIATE `statement` [ USING parameter1 [ , parameter2, ... ] ]
+
+
+
+
+

Description#

+

Executes a statement without the need to prepare or deallocate the statement. +Parameter values are defined in the USING clause.

+
+
+

Examples#

+

Execute a query with no parameters:

+
EXECUTE IMMEDIATE
+'SELECT name FROM nation';
+
+
+

Execute a query with two parameters:

+
EXECUTE IMMEDIATE
+'SELECT name FROM nation WHERE regionkey = ? and nationkey < ?'
+USING 1, 3;
+
+
+

This is equivalent to:

+
PREPARE statement_name FROM SELECT name FROM nation WHERE regionkey = ? and nationkey < ?
+EXECUTE statement_name USING 1, 3
+DEALLOCATE PREPARE statement_name
+
+
+
+
+

See also#

+

EXECUTE, PREPARE, DEALLOCATE PREPARE

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/execute.html b/430/sql/execute.html new file mode 100644 index 000000000..82ea7d5c4 --- /dev/null +++ b/430/sql/execute.html @@ -0,0 +1,935 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + EXECUTE — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

EXECUTE#

+
+

Synopsis#

+
EXECUTE statement_name [ USING parameter1 [ , parameter2, ... ] ]
+
+
+
+
+

Description#

+

Executes a prepared statement with the name statement_name. Parameter values +are defined in the USING clause.

+
+
+

Examples#

+

Prepare and execute a query with no parameters:

+
PREPARE my_select1 FROM
+SELECT name FROM nation;
+
+
+
EXECUTE my_select1;
+
+
+

Prepare and execute a query with two parameters:

+
PREPARE my_select2 FROM
+SELECT name FROM nation WHERE regionkey = ? and nationkey < ?;
+
+
+
EXECUTE my_select2 USING 1, 3;
+
+
+

This is equivalent to:

+
SELECT name FROM nation WHERE regionkey = 1 AND nationkey < 3;
+
+
+
+
+

See also#

+

PREPARE, DEALLOCATE PREPARE, EXECUTE IMMEDIATE

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/explain-analyze.html b/430/sql/explain-analyze.html new file mode 100644 index 000000000..ba49f1820 --- /dev/null +++ b/430/sql/explain-analyze.html @@ -0,0 +1,1005 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + EXPLAIN ANALYZE — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

EXPLAIN ANALYZE#

+
+

Synopsis#

+
EXPLAIN ANALYZE [VERBOSE] statement
+
+
+
+
+

Description#

+

Execute the statement and show the distributed execution plan of the statement +along with the cost of each operation.

+

The VERBOSE option will give more detailed information and low-level statistics; +understanding these may require knowledge of Trino internals and implementation details.

+
+

Note

+

The stats may not be entirely accurate, especially for queries that complete quickly.

+
+
+
+

Examples#

+

In the example below, you can see the CPU time spent in each stage, as well as the relative +cost of each plan node in the stage. Note that the relative cost of the plan nodes is based on +wall time, which may or may not be correlated to CPU time. For each plan node you can see +some additional statistics (e.g: average input per node instance). Such statistics are useful +when one wants to detect data anomalies for a query (e.g: skewness).

+
EXPLAIN ANALYZE SELECT count(*), clerk FROM orders
+WHERE orderdate > date '1995-01-01' GROUP BY clerk;
+
+
+
                                          Query Plan
+-----------------------------------------------------------------------------------------------
+Trino version: version
+Queued: 374.17us, Analysis: 190.96ms, Planning: 179.03ms, Execution: 3.06s
+Fragment 1 [HASH]
+    CPU: 22.58ms, Scheduled: 96.72ms, Blocked 46.21s (Input: 23.06s, Output: 0.00ns), Input: 1000 rows (37.11kB); per task: avg.: 1000.00 std.dev.: 0.00, Output: 1000 rows (28.32kB)
+    Output layout: [clerk, count]
+    Output partitioning: SINGLE []
+    Project[]
+    │   Layout: [clerk:varchar(15), count:bigint]
+    │   Estimates: {rows: ? (?), cpu: ?, memory: 0B, network: 0B}
+    │   CPU: 8.00ms (3.51%), Scheduled: 63.00ms (15.11%), Blocked: 0.00ns (0.00%), Output: 1000 rows (28.32kB)
+    │   Input avg.: 15.63 rows, Input std.dev.: 24.36%
+    └─ Aggregate[type = FINAL, keys = [clerk], hash = [$hashvalue]]
+       │   Layout: [clerk:varchar(15), $hashvalue:bigint, count:bigint]
+       │   Estimates: {rows: ? (?), cpu: ?, memory: ?, network: 0B}
+       │   CPU: 8.00ms (3.51%), Scheduled: 22.00ms (5.28%), Blocked: 0.00ns (0.00%), Output: 1000 rows (37.11kB)
+       │   Input avg.: 15.63 rows, Input std.dev.: 24.36%
+       │   count := count("count_0")
+       └─ LocalExchange[partitioning = HASH, hashColumn = [$hashvalue], arguments = ["clerk"]]
+          │   Layout: [clerk:varchar(15), count_0:bigint, $hashvalue:bigint]
+          │   Estimates: {rows: ? (?), cpu: ?, memory: 0B, network: 0B}
+          │   CPU: 2.00ms (0.88%), Scheduled: 4.00ms (0.96%), Blocked: 23.15s (50.10%), Output: 1000 rows (37.11kB)
+          │   Input avg.: 15.63 rows, Input std.dev.: 793.73%
+          └─ RemoteSource[sourceFragmentIds = [2]]
+                 Layout: [clerk:varchar(15), count_0:bigint, $hashvalue_1:bigint]
+                 CPU: 0.00ns (0.00%), Scheduled: 0.00ns (0.00%), Blocked: 23.06s (49.90%), Output: 1000 rows (37.11kB)
+                 Input avg.: 15.63 rows, Input std.dev.: 793.73%
+
+Fragment 2 [SOURCE]
+    CPU: 210.60ms, Scheduled: 327.92ms, Blocked 0.00ns (Input: 0.00ns, Output: 0.00ns), Input: 1500000 rows (18.17MB); per task: avg.: 1500000.00 std.dev.: 0.00, Output: 1000 rows (37.11kB)
+    Output layout: [clerk, count_0, $hashvalue_2]
+    Output partitioning: HASH [clerk][$hashvalue_2]
+    Aggregate[type = PARTIAL, keys = [clerk], hash = [$hashvalue_2]]
+    │   Layout: [clerk:varchar(15), $hashvalue_2:bigint, count_0:bigint]
+    │   CPU: 30.00ms (13.16%), Scheduled: 30.00ms (7.19%), Blocked: 0.00ns (0.00%), Output: 1000 rows (37.11kB)
+    │   Input avg.: 818058.00 rows, Input std.dev.: 0.00%
+    │   count_0 := count(*)
+    └─ ScanFilterProject[table = hive:sf1:orders, filterPredicate = ("orderdate" > DATE '1995-01-01')]
+           Layout: [clerk:varchar(15), $hashvalue_2:bigint]
+           Estimates: {rows: 1500000 (41.48MB), cpu: 35.76M, memory: 0B, network: 0B}/{rows: 816424 (22.58MB), cpu: 35.76M, memory: 0B, network: 0B}/{rows: 816424 (22.58MB), cpu: 22.58M, memory: 0B, network: 0B}
+           CPU: 180.00ms (78.95%), Scheduled: 298.00ms (71.46%), Blocked: 0.00ns (0.00%), Output: 818058 rows (12.98MB)
+           Input avg.: 1500000.00 rows, Input std.dev.: 0.00%
+           $hashvalue_2 := combine_hash(bigint '0', COALESCE("$operator$hash_code"("clerk"), 0))
+           clerk := clerk:varchar(15):REGULAR
+           orderdate := orderdate:date:REGULAR
+           Input: 1500000 rows (18.17MB), Filtered: 45.46%, Physical Input: 4.51MB
+
+
+

When the VERBOSE option is used, some operators may report additional information. +For example, the window function operator will output the following:

+
EXPLAIN ANALYZE VERBOSE SELECT count(clerk) OVER() FROM orders
+WHERE orderdate > date '1995-01-01';
+
+
+
                                          Query Plan
+-----------------------------------------------------------------------------------------------
+  ...
+         ─ Window[]
+           │   Layout: [clerk:varchar(15), count:bigint]
+           │   CPU: 157.00ms (53.40%), Scheduled: 158.00ms (37.71%), Blocked: 0.00ns (0.00%), Output: 818058 rows (22.62MB)
+           │   metrics:
+           │     'CPU time distribution (s)' = {count=1.00, p01=0.16, p05=0.16, p10=0.16, p25=0.16, p50=0.16, p75=0.16, p90=0.16, p95=0.16, p99=0.16, min=0.16, max=0.16}
+           │     'Input rows distribution' = {count=1.00, p01=818058.00, p05=818058.00, p10=818058.00, p25=818058.00, p50=818058.00, p75=818058.00, p90=818058.00, p95=818058.00, p99=818058.00, min=818058.00, max=818058.00}
+           │     'Scheduled time distribution (s)' = {count=1.00, p01=0.16, p05=0.16, p10=0.16, p25=0.16, p50=0.16, p75=0.16, p90=0.16, p95=0.16, p99=0.16, min=0.16, max=0.16}
+           │   Input avg.: 818058.00 rows, Input std.dev.: 0.00%
+           │   Active Drivers: [ 1 / 1 ]
+           │   Index size: std.dev.: 0.00 bytes, 0.00 rows
+           │   Index count per driver: std.dev.: 0.00
+           │   Rows per driver: std.dev.: 0.00
+           │   Size of partition: std.dev.: 0.00
+           │   count := count("clerk") RANGE UNBOUNDED_PRECEDING CURRENT_ROW
+ ...
+
+
+
+
+

See also#

+

EXPLAIN

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/explain.html b/430/sql/explain.html new file mode 100644 index 000000000..11479279e --- /dev/null +++ b/430/sql/explain.html @@ -0,0 +1,1679 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + EXPLAIN — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

EXPLAIN#

+
+

Synopsis#

+
EXPLAIN [ ( option [, ...] ) ] statement
+
+
+

where option can be one of:

+
FORMAT { TEXT | GRAPHVIZ | JSON }
+TYPE { LOGICAL | DISTRIBUTED | VALIDATE | IO }
+
+
+
+
+

Description#

+

Show the logical or distributed execution plan of a statement, or validate the statement. +The distributed plan is shown by default. Each plan fragment of the distributed plan is executed by +a single or multiple Trino nodes. Fragments separation represent the data exchange between Trino nodes. +Fragment type specifies how the fragment is executed by Trino nodes and how the data is +distributed between fragments:

+
+
SINGLE

Fragment is executed on a single node.

+
+
HASH

Fragment is executed on a fixed number of nodes with the input data +distributed using a hash function.

+
+
ROUND_ROBIN

Fragment is executed on a fixed number of nodes with the input data +distributed in a round-robin fashion.

+
+
BROADCAST

Fragment is executed on a fixed number of nodes with the input data +broadcasted to all nodes.

+
+
SOURCE

Fragment is executed on nodes where input splits are accessed.

+
+
+
+
+

Examples#

+
+

EXPLAIN (TYPE LOGICAL)#

+

Process the supplied query statement and create a logical plan in text format:

+
EXPLAIN (TYPE LOGICAL) SELECT regionkey, count(*) FROM nation GROUP BY 1;
+
+
+
                                                   Query Plan
+-----------------------------------------------------------------------------------------------------------------
+ Trino version: version
+ Output[regionkey, _col1]
+ │   Layout: [regionkey:bigint, count:bigint]
+ │   Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?}
+ │   _col1 := count
+ └─ RemoteExchange[GATHER]
+    │   Layout: [regionkey:bigint, count:bigint]
+    │   Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?}
+    └─ Aggregate(FINAL)[regionkey]
+       │   Layout: [regionkey:bigint, count:bigint]
+       │   Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?}
+       │   count := count("count_8")
+       └─ LocalExchange[HASH][$hashvalue] ("regionkey")
+          │   Layout: [regionkey:bigint, count_8:bigint, $hashvalue:bigint]
+          │   Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?}
+          └─ RemoteExchange[REPARTITION][$hashvalue_9]
+             │   Layout: [regionkey:bigint, count_8:bigint, $hashvalue_9:bigint]
+             │   Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?}
+             └─ Project[]
+                │   Layout: [regionkey:bigint, count_8:bigint, $hashvalue_10:bigint]
+                │   Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?}
+                │   $hashvalue_10 := "combine_hash"(bigint '0', COALESCE("$operator$hash_code"("regionkey"), 0))
+                └─ Aggregate(PARTIAL)[regionkey]
+                   │   Layout: [regionkey:bigint, count_8:bigint]
+                   │   count_8 := count(*)
+                   └─ TableScan[tpch:nation:sf0.01]
+                          Layout: [regionkey:bigint]
+                          Estimates: {rows: 25 (225B), cpu: 225, memory: 0B, network: 0B}
+                          regionkey := tpch:regionkey
+
+
+
+
+

EXPLAIN (TYPE LOGICAL, FORMAT JSON)#

+
+

Warning

+

The output format is not guaranteed to be backward compatible across Trino versions.

+
+

Process the supplied query statement and create a logical plan in JSON format:

+
EXPLAIN (TYPE LOGICAL, FORMAT JSON) SELECT regionkey, count(*) FROM nation GROUP BY 1;
+
+
+
{
+   "id": "9",
+   "name": "Output",
+   "descriptor": {
+      "columnNames": "[regionkey, _col1]"
+   },
+   "outputs": [
+      {
+         "symbol": "regionkey",
+         "type": "bigint"
+      },
+      {
+         "symbol": "count",
+         "type": "bigint"
+      }
+   ],
+   "details": [
+      "_col1 := count"
+   ],
+   "estimates": [
+      {
+         "outputRowCount": "NaN",
+         "outputSizeInBytes": "NaN",
+         "cpuCost": "NaN",
+         "memoryCost": "NaN",
+         "networkCost": "NaN"
+      }
+   ],
+   "children": [
+      {
+         "id": "145",
+         "name": "RemoteExchange",
+         "descriptor": {
+            "type": "GATHER",
+            "isReplicateNullsAndAny": "",
+            "hashColumn": ""
+         },
+         "outputs": [
+            {
+               "symbol": "regionkey",
+               "type": "bigint"
+            },
+            {
+               "symbol": "count",
+               "type": "bigint"
+            }
+         ],
+         "details": [
+
+         ],
+         "estimates": [
+            {
+               "outputRowCount": "NaN",
+               "outputSizeInBytes": "NaN",
+               "cpuCost": "NaN",
+               "memoryCost": "NaN",
+               "networkCost": "NaN"
+            }
+         ],
+         "children": [
+            {
+               "id": "4",
+               "name": "Aggregate",
+               "descriptor": {
+                  "type": "FINAL",
+                  "keys": "[regionkey]",
+                  "hash": ""
+               },
+               "outputs": [
+                  {
+                     "symbol": "regionkey",
+                     "type": "bigint"
+                  },
+                  {
+                     "symbol": "count",
+                     "type": "bigint"
+                  }
+               ],
+               "details": [
+                  "count := count(\"count_0\")"
+               ],
+               "estimates": [
+                  {
+                     "outputRowCount": "NaN",
+                     "outputSizeInBytes": "NaN",
+                     "cpuCost": "NaN",
+                     "memoryCost": "NaN",
+                     "networkCost": "NaN"
+                  }
+               ],
+               "children": [
+                  {
+                     "id": "194",
+                     "name": "LocalExchange",
+                     "descriptor": {
+                        "partitioning": "HASH",
+                        "isReplicateNullsAndAny": "",
+                        "hashColumn": "[$hashvalue]",
+                        "arguments": "[\"regionkey\"]"
+                     },
+                     "outputs": [
+                        {
+                           "symbol": "regionkey",
+                           "type": "bigint"
+                        },
+                        {
+                           "symbol": "count_0",
+                           "type": "bigint"
+                        },
+                        {
+                           "symbol": "$hashvalue",
+                           "type": "bigint"
+                        }
+                     ],
+                     "details":[],
+                     "estimates": [
+                        {
+                           "outputRowCount": "NaN",
+                           "outputSizeInBytes": "NaN",
+                           "cpuCost": "NaN",
+                           "memoryCost": "NaN",
+                           "networkCost": "NaN"
+                        }
+                     ],
+                     "children": [
+                        {
+                           "id": "200",
+                           "name": "RemoteExchange",
+                           "descriptor": {
+                              "type": "REPARTITION",
+                              "isReplicateNullsAndAny": "",
+                              "hashColumn": "[$hashvalue_1]"
+                           },
+                           "outputs": [
+                              {
+                                 "symbol": "regionkey",
+                                 "type": "bigint"
+                              },
+                              {
+                                 "symbol": "count_0",
+                                 "type": "bigint"
+                              },
+                              {
+                                 "symbol": "$hashvalue_1",
+                                 "type": "bigint"
+                              }
+                           ],
+                           "details":[],
+                           "estimates": [
+                              {
+                                 "outputRowCount": "NaN",
+                                 "outputSizeInBytes": "NaN",
+                                 "cpuCost": "NaN",
+                                 "memoryCost": "NaN",
+                                 "networkCost": "NaN"
+                              }
+                           ],
+                           "children": [
+                              {
+                                 "id": "226",
+                                 "name": "Project",
+                                 "descriptor": {}
+                                 "outputs": [
+                                    {
+                                       "symbol": "regionkey",
+                                       "type": "bigint"
+                                    },
+                                    {
+                                       "symbol": "count_0",
+                                       "type": "bigint"
+                                    },
+                                    {
+                                       "symbol": "$hashvalue_2",
+                                       "type": "bigint"
+                                    }
+                                 ],
+                                 "details": [
+                                    "$hashvalue_2 := combine_hash(bigint '0', COALESCE(\"$operator$hash_code\"(\"regionkey\"), 0))"
+                                 ],
+                                 "estimates": [
+                                    {
+                                       "outputRowCount": "NaN",
+                                       "outputSizeInBytes": "NaN",
+                                       "cpuCost": "NaN",
+                                       "memoryCost": "NaN",
+                                       "networkCost": "NaN"
+                                    }
+                                 ],
+                                 "children": [
+                                    {
+                                       "id": "198",
+                                       "name": "Aggregate",
+                                       "descriptor": {
+                                          "type": "PARTIAL",
+                                          "keys": "[regionkey]",
+                                          "hash": ""
+                                       },
+                                       "outputs": [
+                                          {
+                                             "symbol": "regionkey",
+                                             "type": "bigint"
+                                          },
+                                          {
+                                             "symbol": "count_0",
+                                             "type": "bigint"
+                                          }
+                                       ],
+                                       "details": [
+                                          "count_0 := count(*)"
+                                       ],
+                                       "estimates":[],
+                                       "children": [
+                                          {
+                                             "id": "0",
+                                             "name": "TableScan",
+                                             "descriptor": {
+                                                "table": "hive:tpch_sf1_orc_part:nation"
+                                             },
+                                             "outputs": [
+                                                {
+                                                   "symbol": "regionkey",
+                                                   "type": "bigint"
+                                                }
+                                             ],
+                                             "details": [
+                                                "regionkey := regionkey:bigint:REGULAR"
+                                             ],
+                                             "estimates": [
+                                                {
+                                                   "outputRowCount": 25,
+                                                   "outputSizeInBytes": 225,
+                                                   "cpuCost": 225,
+                                                   "memoryCost": 0,
+                                                   "networkCost": 0
+                                                }
+                                             ],
+                                             "children": []
+                                          }
+                                       ]
+                                    }
+                                 ]
+                              }
+                           ]
+                        }
+                     ]
+                  }
+               ]
+            }
+         ]
+      }
+   ]
+}
+
+
+
+
+

EXPLAIN (TYPE DISTRIBUTED)#

+

Process the supplied query statement and create a distributed plan in text +format. The distributed plan splits the logical plan into stages, and therefore +explicitly shows the data exchange between workers:

+
EXPLAIN (TYPE DISTRIBUTED) SELECT regionkey, count(*) FROM nation GROUP BY 1;
+
+
+
                                              Query Plan
+------------------------------------------------------------------------------------------------------
+ Trino version: version
+ Fragment 0 [SINGLE]
+     Output layout: [regionkey, count]
+     Output partitioning: SINGLE []
+     Output[regionkey, _col1]
+     │   Layout: [regionkey:bigint, count:bigint]
+     │   Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?}
+     │   _col1 := count
+     └─ RemoteSource[1]
+            Layout: [regionkey:bigint, count:bigint]
+
+ Fragment 1 [HASH]
+     Output layout: [regionkey, count]
+     Output partitioning: SINGLE []
+     Aggregate(FINAL)[regionkey]
+     │   Layout: [regionkey:bigint, count:bigint]
+     │   Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?}
+     │   count := count("count_8")
+     └─ LocalExchange[HASH][$hashvalue] ("regionkey")
+        │   Layout: [regionkey:bigint, count_8:bigint, $hashvalue:bigint]
+        │   Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?}
+        └─ RemoteSource[2]
+               Layout: [regionkey:bigint, count_8:bigint, $hashvalue_9:bigint]
+
+ Fragment 2 [SOURCE]
+     Output layout: [regionkey, count_8, $hashvalue_10]
+     Output partitioning: HASH [regionkey][$hashvalue_10]
+     Project[]
+     │   Layout: [regionkey:bigint, count_8:bigint, $hashvalue_10:bigint]
+     │   Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?}
+     │   $hashvalue_10 := "combine_hash"(bigint '0', COALESCE("$operator$hash_code"("regionkey"), 0))
+     └─ Aggregate(PARTIAL)[regionkey]
+        │   Layout: [regionkey:bigint, count_8:bigint]
+        │   count_8 := count(*)
+        └─ TableScan[tpch:nation:sf0.01, grouped = false]
+               Layout: [regionkey:bigint]
+               Estimates: {rows: 25 (225B), cpu: 225, memory: 0B, network: 0B}
+               regionkey := tpch:regionkey
+
+
+
+
+

EXPLAIN (TYPE DISTRIBUTED, FORMAT JSON)#

+
+

Warning

+

The output format is not guaranteed to be backward compatible across Trino versions.

+
+

Process the supplied query statement and create a distributed plan in JSON +format. The distributed plan splits the logical plan into stages, and therefore +explicitly shows the data exchange between workers:

+
EXPLAIN (TYPE DISTRIBUTED, FORMAT JSON) SELECT regionkey, count(*) FROM nation GROUP BY 1;
+
+
+
{
+   "0" : {
+      "id" : "9",
+      "name" : "Output",
+      "descriptor" : {
+         "columnNames" : "[regionkey, _col1]"
+      },
+      "outputs" : [ {
+         "symbol" : "regionkey",
+         "type" : "bigint"
+      }, {
+         "symbol" : "count",
+         "type" : "bigint"
+      } ],
+      "details" : [ "_col1 := count" ],
+      "estimates" : [ {
+         "outputRowCount" : "NaN",
+         "outputSizeInBytes" : "NaN",
+         "cpuCost" : "NaN",
+         "memoryCost" : "NaN",
+         "networkCost" : "NaN"
+      } ],
+      "children" : [ {
+         "id" : "145",
+         "name" : "RemoteSource",
+         "descriptor" : {
+            "sourceFragmentIds" : "[1]"
+         },
+         "outputs" : [ {
+            "symbol" : "regionkey",
+            "type" : "bigint"
+         }, {
+            "symbol" : "count",
+            "type" : "bigint"
+         } ],
+         "details" : [ ],
+         "estimates" : [ ],
+         "children" : [ ]
+      } ]
+   },
+   "1" : {
+      "id" : "4",
+      "name" : "Aggregate",
+      "descriptor" : {
+         "type" : "FINAL",
+         "keys" : "[regionkey]",
+         "hash" : "[]"
+      },
+      "outputs" : [ {
+         "symbol" : "regionkey",
+         "type" : "bigint"
+      }, {
+         "symbol" : "count",
+         "type" : "bigint"
+      } ],
+      "details" : [ "count := count(\"count_0\")" ],
+      "estimates" : [ {
+         "outputRowCount" : "NaN",
+         "outputSizeInBytes" : "NaN",
+         "cpuCost" : "NaN",
+         "memoryCost" : "NaN",
+         "networkCost" : "NaN"
+      } ],
+      "children" : [ {
+         "id" : "194",
+         "name" : "LocalExchange",
+         "descriptor" : {
+            "partitioning" : "SINGLE",
+            "isReplicateNullsAndAny" : "",
+            "hashColumn" : "[]",
+            "arguments" : "[]"
+         },
+         "outputs" : [ {
+            "symbol" : "regionkey",
+            "type" : "bigint"
+         }, {
+            "symbol" : "count_0",
+            "type" : "bigint"
+         } ],
+         "details" : [ ],
+         "estimates" : [ {
+            "outputRowCount" : "NaN",
+            "outputSizeInBytes" : "NaN",
+            "cpuCost" : "NaN",
+            "memoryCost" : "NaN",
+            "networkCost" : "NaN"
+         } ],
+         "children" : [ {
+            "id" : "227",
+            "name" : "Project",
+            "descriptor" : { },
+            "outputs" : [ {
+               "symbol" : "regionkey",
+               "type" : "bigint"
+            }, {
+               "symbol" : "count_0",
+               "type" : "bigint"
+            } ],
+            "details" : [ ],
+            "estimates" : [ {
+               "outputRowCount" : "NaN",
+               "outputSizeInBytes" : "NaN",
+               "cpuCost" : "NaN",
+               "memoryCost" : "NaN",
+               "networkCost" : "NaN"
+            } ],
+            "children" : [ {
+               "id" : "200",
+               "name" : "RemoteSource",
+               "descriptor" : {
+                  "sourceFragmentIds" : "[2]"
+               },
+               "outputs" : [ {
+                  "symbol" : "regionkey",
+                  "type" : "bigint"
+               }, {
+                  "symbol" : "count_0",
+                  "type" : "bigint"
+               }, {
+                  "symbol" : "$hashvalue",
+                  "type" : "bigint"
+               } ],
+               "details" : [ ],
+               "estimates" : [ ],
+               "children" : [ ]
+            } ]
+         } ]
+      } ]
+   },
+   "2" : {
+      "id" : "226",
+      "name" : "Project",
+      "descriptor" : { },
+      "outputs" : [ {
+         "symbol" : "regionkey",
+         "type" : "bigint"
+      }, {
+         "symbol" : "count_0",
+         "type" : "bigint"
+      }, {
+         "symbol" : "$hashvalue_1",
+         "type" : "bigint"
+      } ],
+      "details" : [ "$hashvalue_1 := combine_hash(bigint '0', COALESCE(\"$operator$hash_code\"(\"regionkey\"), 0))" ],
+      "estimates" : [ {
+         "outputRowCount" : "NaN",
+         "outputSizeInBytes" : "NaN",
+         "cpuCost" : "NaN",
+         "memoryCost" : "NaN",
+         "networkCost" : "NaN"
+      } ],
+      "children" : [ {
+         "id" : "198",
+         "name" : "Aggregate",
+         "descriptor" : {
+            "type" : "PARTIAL",
+            "keys" : "[regionkey]",
+            "hash" : "[]"
+         },
+         "outputs" : [ {
+            "symbol" : "regionkey",
+            "type" : "bigint"
+         }, {
+            "symbol" : "count_0",
+            "type" : "bigint"
+         } ],
+         "details" : [ "count_0 := count(*)" ],
+         "estimates" : [ ],
+         "children" : [ {
+            "id" : "0",
+            "name" : "TableScan",
+            "descriptor" : {
+               "table" : "tpch:tiny:nation"
+            },
+            "outputs" : [ {
+               "symbol" : "regionkey",
+               "type" : "bigint"
+            } ],
+            "details" : [ "regionkey := tpch:regionkey" ],
+            "estimates" : [ {
+               "outputRowCount" : 25.0,
+               "outputSizeInBytes" : 225.0,
+               "cpuCost" : 225.0,
+               "memoryCost" : 0.0,
+               "networkCost" : 0.0
+            } ],
+            "children" : [ ]
+         } ]
+      } ]
+   }
+}
+
+
+
+
+

EXPLAIN (TYPE VALIDATE)#

+

Validate the supplied query statement for syntactical and semantic correctness. +Returns true if the statement is valid:

+
EXPLAIN (TYPE VALIDATE) SELECT regionkey, count(*) FROM nation GROUP BY 1;
+
+
+
 Valid
+-------
+ true
+
+
+

If the statement is not correct because a syntax error, such as an unknown +keyword, is found the error message details the problem:

+
EXPLAIN (TYPE VALIDATE) SELET 1=0;
+
+
+
Query 20220929_234840_00001_vjwxj failed: line 1:25: mismatched input 'SELET'.
+Expecting: 'ALTER', 'ANALYZE', 'CALL', 'COMMENT', 'COMMIT', 'CREATE',
+'DEALLOCATE', 'DELETE', 'DENY', 'DESC', 'DESCRIBE', 'DROP', 'EXECUTE',
+'EXPLAIN', 'GRANT', 'INSERT', 'MERGE', 'PREPARE', 'REFRESH', 'RESET',
+'REVOKE', 'ROLLBACK', 'SET', 'SHOW', 'START', 'TRUNCATE', 'UPDATE', 'USE',
+<query>
+
+
+

Similarly if semantic issues are detected, such as an invalid object name +nations instead of nation, the error message returns useful +information:

+
EXPLAIN(TYPE VALIDATE) SELECT * FROM tpch.tiny.nations;
+
+
+
Query 20220929_235059_00003_vjwxj failed: line 1:15: Table 'tpch.tiny.nations' does not exist
+SELECT * FROM tpch.tiny.nations
+
+
+
+
+

EXPLAIN (TYPE IO)#

+

Process the supplied query statement and create a plan with input and output +details about the accessed objects in JSON format:

+
EXPLAIN (TYPE IO, FORMAT JSON) INSERT INTO test_lineitem
+SELECT * FROM lineitem WHERE shipdate = '2020-02-01' AND quantity > 10;
+
+
+
            Query Plan
+-----------------------------------
+{
+   inputTableColumnInfos: [
+      {
+         table: {
+            catalog: "hive",
+            schemaTable: {
+               schema: "tpch",
+               table: "test_orders"
+            }
+         },
+         columnConstraints: [
+            {
+               columnName: "orderkey",
+               type: "bigint",
+               domain: {
+                  nullsAllowed: false,
+                  ranges: [
+                     {
+                        low: {
+                           value: "1",
+                           bound: "EXACTLY"
+                        },
+                        high: {
+                           value: "1",
+                           bound: "EXACTLY"
+                        }
+                     },
+                     {
+                        low: {
+                           value: "2",
+                           bound: "EXACTLY"
+                        },
+                        high: {
+                           value: "2",
+                           bound: "EXACTLY"
+                        }
+                     }
+                  ]
+               }
+            },
+            {
+               columnName: "processing",
+               type: "boolean",
+               domain: {
+                  nullsAllowed: false,
+                  ranges: [
+                     {
+                        low: {
+                           value: "false",
+                           bound: "EXACTLY"
+                        },
+                        high: {
+                           value: "false",
+                           bound: "EXACTLY"
+                        }
+                     }
+                  ]
+               }
+            },
+            {
+               columnName: "custkey",
+               type: "bigint",
+               domain: {
+                  nullsAllowed: false,
+                  ranges: [
+                     {
+                        low: {
+                           bound: "ABOVE"
+                        },
+                        high: {
+                           value: "10",
+                           bound: "EXACTLY"
+                        }
+                     }
+                  ]
+               }
+            }
+         ],
+         estimate: {
+            outputRowCount: 2,
+            outputSizeInBytes: 40,
+            cpuCost: 40,
+            maxMemory: 0,
+            networkCost: 0
+         }
+      }
+   ],
+   outputTable: {
+      catalog: "hive",
+      schemaTable: {
+         schema: "tpch",
+         table: "test_orders"
+      }
+   },
+   estimate: {
+      outputRowCount: "NaN",
+      outputSizeInBytes: "NaN",
+      cpuCost: "NaN",
+      maxMemory: "NaN",
+      networkCost: "NaN"
+   }
+}
+
+
+
+
+
+

See also#

+

EXPLAIN ANALYZE

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/grant-roles.html b/430/sql/grant-roles.html new file mode 100644 index 000000000..c65faf23a --- /dev/null +++ b/430/sql/grant-roles.html @@ -0,0 +1,944 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + GRANT ROLES — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

GRANT ROLES#

+
+

Synopsis#

+
GRANT role [, ...]
+TO ( user | USER user | ROLE role) [, ...]
+[ GRANTED BY ( user | USER user | ROLE role | CURRENT_USER | CURRENT_ROLE ) ]
+[ WITH ADMIN OPTION ]
+[ IN catalog ]
+
+
+
+
+

Description#

+

Grants the specified role(s) to the specified principal(s).

+

If the WITH ADMIN OPTION clause is specified, the role(s) are granted +to the users with GRANT option.

+

For the GRANT statement for roles to succeed, the user executing it either should +be the role admin or should possess the GRANT option for the given role.

+

The optional GRANTED BY clause causes the role(s) to be granted with +the specified principal as a grantor. If the GRANTED BY clause is not +specified, the roles are granted with the current user as a grantor.

+

The optional IN catalog clause grants the roles in a catalog as opposed +to a system roles.

+
+
+

Examples#

+

Grant role bar to user foo

+
GRANT bar TO USER foo;
+
+
+

Grant roles bar and foo to user baz and role qux with admin option

+
GRANT bar, foo TO USER baz, ROLE qux WITH ADMIN OPTION;
+
+
+
+
+

Limitations#

+

Some connectors do not support role management. +See connector documentation for more details.

+
+
+

See also#

+

CREATE ROLE, DROP ROLE, SET ROLE, REVOKE ROLES

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/grant.html b/430/sql/grant.html new file mode 100644 index 000000000..cd5d68703 --- /dev/null +++ b/430/sql/grant.html @@ -0,0 +1,948 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + GRANT — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

GRANT#

+
+

Synopsis#

+
GRANT ( privilege [, ...] | ( ALL PRIVILEGES ) )
+ON ( table_name | TABLE table_name | SCHEMA schema_name)
+TO ( user | USER user | ROLE role )
+[ WITH GRANT OPTION ]
+
+
+
+
+

Description#

+

Grants the specified privileges to the specified grantee.

+

Specifying ALL PRIVILEGES grants DELETE, INSERT, UPDATE and SELECT privileges.

+

Specifying ROLE PUBLIC grants privileges to the PUBLIC role and hence to all users.

+

The optional WITH GRANT OPTION clause allows the grantee to grant these same privileges to others.

+

For GRANT statement to succeed, the user executing it should possess the specified privileges as well as the GRANT OPTION for those privileges.

+

Grant on a table grants the specified privilege on all current and future columns of the table.

+

Grant on a schema grants the specified privilege on all current and future columns of all current and future tables of the schema.

+
+
+

Examples#

+

Grant INSERT and SELECT privileges on the table orders to user alice:

+
GRANT INSERT, SELECT ON orders TO alice;
+
+
+

Grant DELETE privilege on the schema finance to user bob:

+
GRANT DELETE ON SCHEMA finance TO bob;
+
+
+

Grant SELECT privilege on the table nation to user alice, additionally allowing alice to grant SELECT privilege to others:

+
GRANT SELECT ON nation TO alice WITH GRANT OPTION;
+
+
+

Grant SELECT privilege on the table orders to everyone:

+
GRANT SELECT ON orders TO ROLE PUBLIC;
+
+
+
+
+

Limitations#

+

Some connectors have no support for GRANT. +See connector documentation for more details.

+
+
+

See also#

+

DENY, REVOKE, SHOW GRANTS

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/insert.html b/430/sql/insert.html new file mode 100644 index 000000000..aabb940d9 --- /dev/null +++ b/430/sql/insert.html @@ -0,0 +1,943 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + INSERT — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

INSERT#

+
+

Synopsis#

+
INSERT INTO table_name [ ( column [, ... ] ) ] query
+
+
+
+
+

Description#

+

Insert new rows into a table.

+

If the list of column names is specified, they must exactly match the list +of columns produced by the query. Each column in the table not present in the +column list will be filled with a null value. Otherwise, if the list of +columns is not specified, the columns produced by the query must exactly match +the columns in the table being inserted into.

+
+
+

Examples#

+

Load additional rows into the orders table from the new_orders table:

+
INSERT INTO orders
+SELECT * FROM new_orders;
+
+
+

Insert a single row into the cities table:

+
INSERT INTO cities VALUES (1, 'San Francisco');
+
+
+

Insert multiple rows into the cities table:

+
INSERT INTO cities VALUES (2, 'San Jose'), (3, 'Oakland');
+
+
+

Insert a single row into the nation table with the specified column list:

+
INSERT INTO nation (nationkey, name, regionkey, comment)
+VALUES (26, 'POLAND', 3, 'no comment');
+
+
+

Insert a row without specifying the comment column. +That column will be null:

+
INSERT INTO nation (nationkey, name, regionkey)
+VALUES (26, 'POLAND', 3);
+
+
+
+
+

See also#

+

VALUES

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/match-recognize.html b/430/sql/match-recognize.html new file mode 100644 index 000000000..e7fdf2ac7 --- /dev/null +++ b/430/sql/match-recognize.html @@ -0,0 +1,1688 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + MATCH_RECOGNIZE — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

MATCH_RECOGNIZE#

+
+

Synopsis#

+
MATCH_RECOGNIZE (
+  [ PARTITION BY column [, ...] ]
+  [ ORDER BY column [, ...] ]
+  [ MEASURES measure_definition [, ...] ]
+  [ rows_per_match ]
+  [ AFTER MATCH skip_to ]
+  PATTERN ( row_pattern )
+  [ SUBSET subset_definition [, ...] ]
+  DEFINE variable_definition [, ...]
+  )
+
+
+
+
+

Description#

+

The MATCH_RECOGNIZE clause is an optional subclause of the FROM clause. +It is used to detect patterns in a set of rows. Patterns of interest are +specified using row pattern syntax based on regular expressions. The input to +pattern matching is a table, a view or a subquery. For each detected match, one +or more rows are returned. They contain requested information about the match.

+

Row pattern matching is a powerful tool when analyzing complex sequences of +events. The following examples show some of the typical use cases:

+
    +
  • in trade applications, tracking trends or identifying customers with specific +behavioral patterns

  • +
  • in shipping applications, tracking packages through all possible valid paths,

  • +
  • in financial applications, detecting unusual incidents, which might signal +fraud

  • +
+
+
+

Example#

+

In the following example, the pattern describes a V-shape over the +totalprice column. A match is found whenever orders made by a customer +first decrease in price, and then increase past the starting point:

+
SELECT * FROM orders MATCH_RECOGNIZE(
+     PARTITION BY custkey
+     ORDER BY orderdate
+     MEASURES
+              A.totalprice AS starting_price,
+              LAST(B.totalprice) AS bottom_price,
+              LAST(U.totalprice) AS top_price
+     ONE ROW PER MATCH
+     AFTER MATCH SKIP PAST LAST ROW
+     PATTERN (A B+ C+ D+)
+     SUBSET U = (C, D)
+     DEFINE
+              B AS totalprice < PREV(totalprice),
+              C AS totalprice > PREV(totalprice) AND totalprice <= A.totalprice,
+              D AS totalprice > PREV(totalprice)
+     )
+
+
+

In the following sections, all subclauses of the MATCH_RECOGNIZE clause are +explained with this example query.

+
+
+

Partitioning and ordering#

+
PARTITION BY custkey
+
+
+

The PARTITION BY clause allows you to break up the input table into +separate sections, that are independently processed for pattern matching. +Without a partition declaration, the whole input table is used. This behavior +is analogous to the semantics of PARTITION BY clause in window specification. In the example, the orders table is +partitioned by the custkey value, so that pattern matching is performed for +all orders of a specific customer independently from orders of other +customers.

+
ORDER BY orderdate
+
+
+

The optional ORDER BY clause is generally useful to allow matching on an +ordered data set. For example, sorting the input by orderdate allows for +matching on a trend of changes over time.

+
+
+

Row pattern measures#

+

The MEASURES clause allows to specify what information is retrieved from a +matched sequence of rows.

+
MEASURES measure_expression AS measure_name [, ...]
+
+
+

A measure expression is a scalar expression whose value is computed based on a +match. In the example, three row pattern measures are specified:

+

A.totalprice AS starting_price returns the price in the first row of the +match, which is the only row associated with A according to the pattern.

+

LAST(B.totalprice) AS bottom_price returns the lowest price (corresponding +to the bottom of the “V” in the pattern). It is the price in the last row +associated with B, which is the last row of the descending section.

+

LAST(U.totalprice) AS top_price returns the highest price in the match. It +is the price in the last row associated with C or D, which is also the +final row of the match.

+

Measure expressions can refer to the columns of the input table. They also +allow special syntax to combine the input information with the details of the +match (see Row pattern recognition expressions).

+

Each measure defines an output column of the pattern recognition. The column +can be referenced with the measure_name.

+

The MEASURES clause is optional. When no measures are specified, certain +input columns (depending on ROWS PER MATCH clause) are +the output of the pattern recognition.

+
+
+

Rows per match#

+

This clause can be used to specify the quantity of output rows. There are two +main options:

+
ONE ROW PER MATCH
+
+
+

and

+
ALL ROWS PER MATCH
+
+
+

ONE ROW PER MATCH is the default option. For every match, a single row of +output is produced. Output consists of PARTITION BY columns and measures. +The output is also produced for empty matches, based on their starting rows. +Rows that are unmatched (that is, neither included in some non-empty match, nor +being the starting row of an empty match), are not included in the output.

+

For ALL ROWS PER MATCH, every row of a match produces an output row, unless +it is excluded from the output by the exclusion syntax. Output consists +of PARTITION BY columns, ORDER BY columns, measures and remaining +columns from the input table. By default, empty matches are shown and unmatched +rows are skipped, similarly as with the ONE ROW PER MATCH option. However, +this behavior can be changed by modifiers:

+
ALL ROWS PER MATCH SHOW EMPTY MATCHES
+
+
+

shows empty matches and skips unmatched rows, like the default.

+
ALL ROWS PER MATCH OMIT EMPTY MATCHES
+
+
+

excludes empty matches from the output.

+
ALL ROWS PER MATCH WITH UNMATCHED ROWS
+
+
+

shows empty matches and produces additional output row for each unmatched row.

+

There are special rules for computing row pattern measures for empty matches +and unmatched rows. They are explained in +Evaluating expressions in empty matches and unmatched rows.

+

Unmatched rows can only occur when the pattern does not allow an empty match. +Otherwise, they are considered as starting rows of empty matches. The option +ALL ROWS PER MATCH WITH UNMATCHED ROWS is recommended when pattern +recognition is expected to pass all input rows, and it is not certain whether +the pattern allows an empty match.

+
+
+

After match skip#

+

The AFTER MATCH SKIP clause specifies where pattern matching resumes after +a non-empty match is found.

+

The default option is:

+
AFTER MATCH SKIP PAST LAST ROW
+
+
+

With this option, pattern matching starts from the row after the last row of +the match. Overlapping matches are not detected.

+

With the following option, pattern matching starts from the second row of the +match:

+
AFTER MATCH SKIP TO NEXT ROW
+
+
+

In the example, if a V-shape is detected, further overlapping matches are +found, starting from consecutive rows on the descending slope of the “V”. +Skipping to the next row is the default behavior after detecting an empty match +or unmatched row.

+

The following AFTER MATCH SKIP options allow to resume pattern matching +based on the components of the pattern. Pattern matching starts from the last +(default) or first row matched to a certain row pattern variable. It can be +either a primary pattern variable (they are explained in +Row pattern syntax) or a +union variable:

+
AFTER MATCH SKIP TO [ FIRST | LAST ] pattern_variable
+
+
+

It is forbidden to skip to the first row of the current match, because it +results in an infinite loop. For example specifying AFTER MATCH SKIP TO A +fails, because A is the first element of the pattern, and jumping back to +it creates an infinite loop. Similarly, skipping to a pattern variable which is +not present in the match causes failure.

+

All other options than the default AFTER MATCH SKIP PAST LAST ROW allow +detection of overlapping matches. The combination of ALL ROWS PER MATCH WITH UNMATCHED ROWS with AFTER MATCH SKIP PAST LAST ROW is the only +configuration that guarantees exactly one output row for each input row.

+
+
+

Row pattern syntax#

+

Row pattern is a form of a regular expression with some syntactical extensions +specific to row pattern recognition. It is specified in the PATTERN +clause:

+
PATTERN ( row_pattern )
+
+
+

The basic element of row pattern is a primary pattern variable. Like pattern +matching in character strings searches for characters, pattern matching in row +sequences searches for rows which can be “labeled” with certain primary pattern +variables. A primary pattern variable has a form of an identifier and is +defined by a boolean condition. This +condition determines whether a particular input row can be mapped to this +variable and take part in the match.

+

In the example PATTERN (A B+ C+ D+), there are four primary pattern +variables: A, B, C, and D.

+

Row pattern syntax includes the following usage:

+
+

concatenation#

+
A B+ C+ D+
+
+
+

It is a sequence of components without operators between them. All components +are matched in the same order as they are specified.

+
+
+

alternation#

+
A | B | C
+
+
+

It is a sequence of components separated by |. Exactly one of the +components is matched. In case when multiple components can be matched, the +leftmost matching component is chosen.

+
+
+

permutation#

+
PERMUTE(A, B, C)
+
+
+

It is equivalent to alternation of all permutations of its components. All +components are matched in some order. If multiple matches are possible for +different orderings of the components, the match is chosen based on the +lexicographical order established by the order of components in the PERMUTE +list. In the above example, the most preferred option is A B C, and the +least preferred option is C B A.

+
+
+

grouping#

+
(A B C)
+
+
+
+
+

partition start anchor#

+
^
+
+
+
+
+

partition end anchor#

+
$
+
+
+
+
+

empty pattern#

+
()
+
+
+
+
+

exclusion syntax#

+
{- row_pattern -}
+
+
+

Exclusion syntax is used to specify portions of the match to exclude from the +output. It is useful in combination with the ALL ROWS PER MATCH option, +when only certain sections of the match are interesting.

+

If you change the example to use ALL ROWS PER MATCH, and the pattern is +modified to PATTERN (A {- B+ C+ -} D+), the result consists of the initial +matched row and the trailing section of rows.

+

Specifying pattern exclusions does not affect the computation of expressions in +MEASURES and DEFINE clauses. Exclusions also do not affect pattern +matching. They have the same semantics as regular grouping with parentheses.

+

It is forbidden to specify pattern exclusions with the option ALL ROWS PER MATCH WITH UNMATCHED ROWS.

+
+
+

quantifiers#

+

Pattern quantifiers allow to specify the desired number of repetitions of a +sub-pattern in a match. They are appended after the relevant pattern +component:

+
(A | B)*
+
+
+

There are following row pattern quantifiers:

+
    +
  • zero or more repetitions:

  • +
+
*
+
+
+
    +
  • one or more repetitions:

  • +
+
+
+
+
+
    +
  • zero or one repetition:

  • +
+
?
+
+
+
    +
  • exact number of repetitions, specified by a non-negative integer number:

  • +
+
{n}
+
+
+
    +
  • number of repetitions ranging between bounds, specified by non-negative +integer numbers:

  • +
+
{m, n}
+
+
+

Specifying bounds is optional. If the left bound is omitted, it defaults to +0. So, {, 5} can be described as “between zero and five repetitions”. +If the right bound is omitted, the number of accepted repetitions is unbounded. +So, {5, } can be described as “at least five repetitions”. Also, {,} is +equivalent to *.

+

Quantifiers are greedy by default. It means that higher number of repetitions +is preferred over lower number. This behavior can be changed to reluctant by +appending ? immediately after the quantifier. With {3, 5}, 3 +repetitions is the least desired option and 5 repetitions – the most desired. +With {3, 5}?, 3 repetitions are most desired. Similarly, ? prefers 1 +repetition, while ?? prefers 0 repetitions.

+
+
+
+

Row pattern union variables#

+

As explained in Row pattern syntax, primary pattern variables are the +basic elements of row pattern. In addition to primary pattern variables, you +can define union variables. They are introduced in the SUBSET clause:

+
SUBSET U = (C, D), ...
+
+
+

In the preceding example, union variable U is defined as union of primary +variables C and D. Union variables are useful in MEASURES, +DEFINE and AFTER MATCH SKIP clauses. They allow you to refer to set of +rows matched to either primary variable from a subset.

+

With the pattern: PATTERN((A | B){5} C+) it cannot be determined upfront if +the match contains any A or any B. A union variable can be used to +access the last row matched to either A or B. Define SUBSET U = (A, B), and the expression LAST(U.totalprice) returns the value of the +totalprice column from the last row mapped to either A or B. Also, +AFTER MATCH SKIP TO LAST A or AFTER MATCH SKIP TO LAST B can result in +failure if A or B is not present in the match. AFTER MATCH SKIP TO LAST U does not fail.

+
+
+

Row pattern variable definitions#

+

The DEFINE clause is where row pattern primary variables are defined. Each +variable is associated with a boolean condition:

+
DEFINE B AS totalprice < PREV(totalprice), ...
+
+
+

During pattern matching, when a certain variable is considered for the next +step of the match, the boolean condition is evaluated in context of the current +match. If the result is true, then the current row, “labeled” with the +variable, becomes part of the match.

+

In the preceding example, assume that the pattern allows to match B at some +point. There are some rows already matched to some pattern variables. Now, +variable B is being considered for the current row. Before the match is +made, the defining condition for B is evaluated. In this example, it is +only true if the value of the totalprice column in the current row is lower +than totalprice in the preceding row.

+

The mechanism of matching variables to rows shows the difference between +pattern matching in row sequences and regular expression matching in text. In +text, characters remain constantly in their positions. In row pattern matching, +a row can be mapped to different variables in different matches, depending on +the preceding part of the match, and even on the match number.

+

It is not required that every primary variable has a definition in the +DEFINE clause. Variables not mentioned in the DEFINE clause are +implicitly associated with true condition, which means that they can be +matched to every row.

+

Boolean expressions in the DEFINE clause allow the same special syntax as +expressions in the MEASURES clause. Details are explained in +Row pattern recognition expressions.

+
+
+

Row pattern recognition expressions#

+

Expressions in MEASURES and +DEFINE clauses are scalar expressions +evaluated over rows of the input table. They support special syntax, specific +to pattern recognition context. They can combine input information with the +information about the current match. Special syntax allows to access pattern +variables assigned to rows, browse rows based on how they are matched, and +refer to the sequential number of the match.

+
+

pattern variable references#

+
A.totalprice
+
+U.orderdate
+
+orderstatus
+
+
+

A column name prefixed with a pattern variable refers to values of this column +in all rows matched to this variable, or to any variable from the subset in +case of union variable. If a column name is not prefixed, it is considered as +prefixed with the universal pattern variable, defined as union of all +primary pattern variables. In other words, a non-prefixed column name refers to +all rows of the current match.

+

It is forbidden to prefix a column name with a table name in the pattern +recognition context.

+
+
+

classifier function#

+
CLASSIFIER()
+
+CLASSIFIER(A)
+
+CLASSIFIER(U)
+
+
+

The classifier function returns the primary pattern variable associated +with the row. The return type is varchar. The optional argument is a +pattern variable. It limits the rows of interest, the same way as with prefixed +column references. The classifier function is particularly useful with a +union variable as the argument. It allows you to determine which variable from +the subset actually matched.

+
+
+

match_number function#

+
MATCH_NUMBER()
+
+
+

The match_number function returns the sequential number of the match within +partition, starting from 1. Empty matches are assigned sequential numbers +as well as non-empty matches. The return type is bigint.

+
+
+

logical navigation functions#

+
FIRST(A.totalprice, 2)
+
+
+

In the above example, the first function navigates to the first row matched +to pattern variable A, and then searches forward until it finds two more +occurrences of variable A within the match. The result is the value of the +totalprice column in that row.

+
LAST(A.totalprice, 2)
+
+
+

In the above example, the last function navigates to the last row matched +to pattern variable A, and then searches backwards until it finds two more +occurrences of variable A within the match. The result is the value of the +totalprice column in that row.

+

With the first and last functions the result is null, if the +searched row is not found in the mach.

+

The second argument is optional. The default value is 0, which means that +by default these functions navigate to the first or last row of interest. If +specified, the second argument must be a non-negative integer number.

+
+
+

physical navigation functions#

+
PREV(A.totalprice, 2)
+
+
+

In the above example, the prev function navigates to the last row matched +to pattern variable A, and then searches two rows backward. The result is +the value of the totalprice column in that row.

+
NEXT(A.totalprice, 2)
+
+
+

In the above example, the next function navigates to the last row matched +to pattern variable A, and then searches two rows forward. The result is +the value of the totalprice column in that row.

+

With the prev and next functions, it is possible to navigate and +retrieve values outside the match. If the navigation goes beyond partition +bounds, the result is null.

+

The second argument is optional. The default value is 1, which means that +by default these functions navigate to previous or next row. If specified, the +second argument must be a non-negative integer number.

+
+
+

nesting of navigation functions#

+

It is possible to nest logical navigation functions within physical navigation +functions:

+
PREV(FIRST(A.totalprice, 3), 2)
+
+
+

In case of nesting, first the logical navigation is performed. It establishes +the starting row for the physical navigation. When both navigation operations +succeed, the value is retrieved from the designated row.

+

Pattern navigation functions require at least one column reference or +classifier function inside of their first argument. The following examples +are correct:

+
LAST("pattern_variable_" || CLASSIFIER())
+
+NEXT(U.totalprice + 10)
+
+
+

This is incorrect:

+
LAST(1)
+
+
+

It is also required that all column references and all classifier calls +inside a pattern navigation function are consistent in referred pattern +variables. They must all refer either to the same primary variable, the same +union variable, or to the implicit universal pattern variable. The following +examples are correct:

+
LAST(CLASSIFIER() = 'A' OR totalprice > 10) /* universal pattern variable */
+
+LAST(CLASSIFIER(U) = 'A' OR U.totalprice > 10) /* pattern variable U */
+
+
+

This is incorrect:

+
LAST(A.totalprice + B.totalprice)
+
+
+
+
+

Aggregate functions#

+

It is allowed to use aggregate functions in a row pattern recognition context. +Aggregate functions are evaluated over all rows of the current match or over a +subset of rows based on the matched pattern variables. The +running and final semantics are supported, with +running as the default.

+

The following expression returns the average value of the totalprice column +for all rows matched to pattern variable A:

+
avg(A.totalprice)
+
+
+

The following expression returns the average value of the totalprice column +for all rows matched to pattern variables from subset U:

+
avg(U.totalprice)
+
+
+

The following expression returns the average value of the totalprice column +for all rows of the match:

+
avg(totalprice)
+
+
+
+

Aggregation arguments#

+

In case when the aggregate function has multiple arguments, it is required that +all arguments refer consistently to the same set of rows:

+
max_by(totalprice, tax) /* aggregate over all rows of the match */
+
+max_by(CLASSIFIER(A), A.tax) /* aggregate over all rows matched to A */
+
+
+

This is incorrect:

+
max_by(A.totalprice, tax)
+
+max_by(A.totalprice, A.tax + B.tax)
+
+
+

If an aggregate argument does not contain any column reference or +classifier function, it does not refer to any pattern variable. In such a +case other aggregate arguments determine the set of rows to aggregate over. If +none of the arguments contains a pattern variable reference, the universal row +pattern variable is implicit. This means that the aggregate function applies to +all rows of the match:

+
count(1) /* aggregate over all rows of the match */
+
+min_by(1, 2) /* aggregate over all rows of the match */
+
+min_by(1, totalprice) /* aggregate over all rows of the match */
+
+min_by(totalprice, 1) /* aggregate over all rows of the match */
+
+min_by(A.totalprice, 1) /* aggregate over all rows matched to A */
+
+max_by(1, A.totalprice) /* aggregate over all rows matched to A */
+
+
+
+
+

Nesting of aggregate functions#

+

Aggregate function arguments must not contain pattern navigation functions. +Similarly, aggregate functions cannot be nested in pattern navigation +functions.

+
+
+

Usage of the classifier and match_number functions#

+

It is allowed to use the classifier and match_number functions in +aggregate function arguments. The following expression returns an array +containing all matched pattern variables:

+
array_agg(CLASSIFIER())
+
+
+

This is particularly useful in combination with the option +ONE ROW PER MATCH. It allows to get all the components of the match while +keeping the output size reduced.

+
+
+

Row pattern count aggregation#

+

Like other aggregate functions in a row pattern recognition context, the +count function can be applied to all rows of the match, or to rows +associated with certain row pattern variables:

+
count(*), count() /* count all rows of the match */
+
+count(totalprice) /* count non-null values of the totalprice column
+                     in all rows of the match */
+
+count(A.totalprice) /* count non-null values of the totalprice column
+                       in all rows matched to A */
+
+
+

The count function in a row pattern recognition context allows special syntax +to support the count(*) behavior over a limited set of rows:

+
count(A.*) /* count rows matched to A */
+
+count(U.*) /* count rows matched to pattern variables from subset U */
+
+
+
+
+
+

RUNNING and FINAL semantics#

+

During pattern matching in a sequence of rows, one row after another is +examined to determine if it fits the pattern. At any step, a partial match is +known, but it is not yet known what rows will be added in the future or what +pattern variables they will be mapped to. So, when evaluating a boolean +condition in the DEFINE clause for the current row, only the preceding part +of the match (plus the current row) is “visible”. This is the running +semantics.

+

When evaluating expressions in the MEASURES clause, the match is complete. +It is then possible to apply the final semantics. In the final +semantics, the whole match is “visible” as from the position of the final row.

+

In the MEASURES clause, the running semantics can also be applied. When +outputting information row by row (as in ALL ROWS PER MATCH), the +running semantics evaluate expressions from the positions of consecutive +rows.

+

The running and final semantics are denoted by the keywords: +RUNNING and FINAL, preceding a logical navigation function first or +last, or an aggregate function:

+
RUNNING LAST(A.totalprice)
+
+FINAL LAST(A.totalprice)
+
+RUNNING avg(A.totalprice)
+
+FINAL count(A.*)
+
+
+

The running semantics is default in MEASURES and DEFINE clauses. +FINAL can only be specified in the MEASURES clause.

+

With the option ONE ROW PER MATCH, row pattern measures are evaluated from +the position of the final row in the match. Therefore, running and +final semantics are the same.

+
+
+
+

Evaluating expressions in empty matches and unmatched rows#

+

An empty match occurs when the row pattern is successfully matched, but no +pattern variables are assigned. The following pattern produces an empty match +for every row:

+
PATTERN(())
+
+
+

When evaluating row pattern measures for an empty match:

+
    +
  • all column references return null

  • +
  • all navigation operations return null

  • +
  • classifier function returns null

  • +
  • match_number function returns the sequential number of the match

  • +
  • all aggregate functions are evaluated over an empty set of rows

  • +
+

Like every match, an empty match has its starting row. All input values which +are to be output along with the measures (as explained in +Rows per match), are the values from the starting row.

+

An unmatched row is a row that is neither part of any non-empty match nor the +starting row of an empty match. With the option ALL ROWS PER MATCH WITH UNMATCHED ROWS, a single output row is produced. In that row, all row pattern +measures are null. All input values which are to be output along with the +measures (as explained in Rows per match), are the values from the +unmatched row. Using the match_number function as a measure can help +differentiate between an empty match and unmatched row.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/merge.html b/430/sql/merge.html new file mode 100644 index 000000000..15bd4725e --- /dev/null +++ b/430/sql/merge.html @@ -0,0 +1,982 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + MERGE — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

MERGE#

+
+

Synopsis#

+
MERGE INTO target_table [ [ AS ]  target_alias ]
+USING { source_table | query } [ [ AS ] source_alias ]
+ON search_condition
+when_clause [...]
+
+
+

where when_clause is one of

+
WHEN MATCHED [ AND condition ]
+    THEN DELETE
+
+
+
WHEN MATCHED [ AND condition ]
+    THEN UPDATE SET ( column = expression [, ...] )
+
+
+
WHEN NOT MATCHED [ AND condition ]
+    THEN INSERT [ column_list ] VALUES (expression, ...)
+
+
+
+
+

Description#

+

Conditionally update and/or delete rows of a table and/or insert new +rows into a table.

+

MERGE supports an arbitrary number of WHEN clauses with different +MATCHED conditions, executing the DELETE, UPDATE or INSERT +operation in the first WHEN clause selected by the MATCHED +state and the match condition.

+

For each source row, the WHEN clauses are processed in order. Only +the first first matching WHEN clause is executed and subsequent clauses +are ignored. A MERGE_TARGET_ROW_MULTIPLE_MATCHES exception is +raised when a single target table row matches more than one source row.

+

If a source row is not matched by any WHEN clause and there is no +WHEN NOT MATCHED clause, the source row is ignored.

+

In WHEN clauses with UPDATE operations, the column value expressions +can depend on any field of the target or the source. In the NOT MATCHED +case, the INSERT expressions can depend on any field of the source.

+
+
+

Examples#

+

Delete all customers mentioned in the source table:

+
MERGE INTO accounts t USING monthly_accounts_update s
+    ON t.customer = s.customer
+    WHEN MATCHED
+        THEN DELETE
+
+
+

For matching customer rows, increment the purchases, and if there is no +match, insert the row from the source table:

+
MERGE INTO accounts t USING monthly_accounts_update s
+    ON (t.customer = s.customer)
+    WHEN MATCHED
+        THEN UPDATE SET purchases = s.purchases + t.purchases
+    WHEN NOT MATCHED
+        THEN INSERT (customer, purchases, address)
+              VALUES(s.customer, s.purchases, s.address)
+
+
+

MERGE into the target table from the source table, deleting any matching +target row for which the source address is Centreville. For all other +matching rows, add the source purchases and set the address to the source +address, if there is no match in the target table, insert the source +table row:

+
MERGE INTO accounts t USING monthly_accounts_update s
+    ON (t.customer = s.customer)
+    WHEN MATCHED AND s.address = 'Centreville'
+        THEN DELETE
+    WHEN MATCHED
+        THEN UPDATE
+            SET purchases = s.purchases + t.purchases, address = s.address
+    WHEN NOT MATCHED
+        THEN INSERT (customer, purchases, address)
+              VALUES(s.customer, s.purchases, s.address)
+
+
+
+
+

Limitations#

+

Any connector can be used as a source table for a MERGE statement. +Only connectors which support the MERGE statement can be the target of a +merge operation. See the connector documentation for more +information.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/pattern-recognition-in-window.html b/430/sql/pattern-recognition-in-window.html new file mode 100644 index 000000000..2b4c91cf5 --- /dev/null +++ b/430/sql/pattern-recognition-in-window.html @@ -0,0 +1,1132 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Row pattern recognition in window structures — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

Row pattern recognition in window structures#

+

A window structure can be defined in the WINDOW clause or in the OVER +clause of a window operation. In both cases, the window specification can +include row pattern recognition clauses. They are part of the window frame. The +syntax and semantics of row pattern recognition in window are similar to those +of the MATCH_RECOGNIZE clause.

+

This section explains the details of row pattern recognition in window +structures, and highlights the similarities and the differences between both +pattern recognition mechanisms.

+
+

Window with row pattern recognition#

+

Window specification:

+
(
+[ existing_window_name ]
+[ PARTITION BY column [, ...] ]
+[ ORDER BY column [, ...] ]
+[ window_frame ]
+)
+
+
+

Window frame:

+
[ MEASURES measure_definition [, ...] ]
+frame_extent
+[ AFTER MATCH skip_to ]
+[ INITIAL | SEEK ]
+[ PATTERN ( row_pattern ) ]
+[ SUBSET subset_definition [, ...] ]
+[ DEFINE variable_definition [, ...] ]
+
+
+

Generally, a window frame specifies the frame_extent, which defines the +“sliding window” of rows to be processed by a window function. It can be +defined in terms of ROWS, RANGE or GROUPS.

+

A window frame with row pattern recognition involves many other syntactical +components, mandatory or optional, and enforces certain limitations on the +frame_extent.

+

Window frame with row pattern recognition:

+
[ MEASURES measure_definition [, ...] ]
+ROWS BETWEEN CURRENT ROW AND frame_end
+[ AFTER MATCH skip_to ]
+[ INITIAL | SEEK ]
+PATTERN ( row_pattern )
+[ SUBSET subset_definition [, ...] ]
+DEFINE variable_definition [, ...]
+
+
+
+
+

Description of the pattern recognition clauses#

+

The frame_extent with row pattern recognition must be defined in terms of +ROWS. The frame start must be at the CURRENT ROW, which limits the +allowed frame extent values to the following:

+
ROWS BETWEEN CURRENT ROW AND CURRENT ROW
+
+ROWS BETWEEN CURRENT ROW AND <expression> FOLLOWING
+
+ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING
+
+
+

For every input row processed by the window, the portion of rows enclosed by +the frame_extent limits the search area for row pattern recognition. Unlike +in MATCH_RECOGNIZE, where the pattern search can explore all rows until the +partition end, and all rows of the partition are available for computations, in +window structures the pattern matching can neither match rows nor retrieve +input values outside the frame.

+

Besides the frame_extent, pattern matching requires the PATTERN and +DEFINE clauses.

+

The PATTERN clause specifies a row pattern, which is a form of a regular +expression with some syntactical extensions. The row pattern syntax is similar +to the row pattern syntax in MATCH_RECOGNIZE. +However, the anchor patterns ^ and $ are not allowed in a window +specification.

+

The DEFINE clause defines the row pattern primary variables in terms of +boolean conditions that must be satisfied. It is similar to the +DEFINE clause of MATCH_RECOGNIZE. +The only difference is that the window syntax does not support the +MATCH_NUMBER function.

+

The MEASURES clause is syntactically similar to the +MEASURES clause of MATCH_RECOGNIZE. The only +limitation is that the MATCH_NUMBER function is not allowed. However, the +semantics of this clause differs between MATCH_RECOGNIZE and window. +While in MATCH_RECOGNIZE every measure produces an output column, the +measures in window should be considered as definitions associated with the +window structure. They can be called over the window, in the same manner as +regular window functions:

+
SELECT cust_key, value OVER w, label OVER w
+    FROM orders
+    WINDOW w AS (
+                 PARTITION BY cust_key
+                 ORDER BY order_date
+                 MEASURES
+                        RUNNING LAST(total_price) AS value,
+                        CLASSIFIER() AS label
+                 ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING
+                 PATTERN (A B+ C+)
+                 DEFINE
+                        B AS B.value < PREV (B.value),
+                        C AS C.value > PREV (C.value)
+                )
+
+
+

Measures defined in a window can be referenced in the SELECT clause and in +the ORDER BY clause of the enclosing query.

+

The RUNNING and FINAL keywords are allowed in the MEASURES clause. +They can precede a logical navigation function FIRST or LAST, or an +aggregate function. However, they have no effect. Every computation is +performed from the position of the final row of the match, so the semantics is +effectively FINAL.

+

The AFTER MATCH SKIP clause has the same syntax as the +AFTER MATCH SKIP clause of MATCH_RECOGNIZE.

+

The INITIAL or SEEK modifier is specific to row pattern recognition in +window. With INITIAL, which is the default, the pattern match for an input +row can only be found starting from that row. With SEEK, if there is no +match starting from the current row, the engine tries to find a match starting +from subsequent rows within the frame. As a result, it is possible to associate +an input row with a match which is detached from that row.

+

The SUBSET clause is used to define union variables as sets of primary pattern variables. You can +use union variables to refer to a set of rows matched to any primary pattern +variable from the subset:

+
SUBSET U = (A, B)
+
+
+

The following expression returns the total_price value from the last row +matched to either A or B:

+
LAST(U.total_price)
+
+
+

If you want to refer to all rows of the match, there is no need to define a +SUBSET containing all pattern variables. There is an implicit universal +pattern variable applied to any non prefixed column name and any +CLASSIFIER call without an argument. The following expression returns the +total_price value from the last matched row:

+
LAST(total_price)
+
+
+

The following call returns the primary pattern variable of the first matched +row:

+
FIRST(CLASSIFIER())
+
+
+

In window, unlike in MATCH_RECOGNIZE, you cannot specify ONE ROW PER MATCH or ALL ROWS PER MATCH. This is because all calls over window, +whether they are regular window functions or measures, must comply with the +window semantics. A call over window is supposed to produce exactly one output +row for every input row. And so, the output mode of pattern recognition in +window is a combination of ONE ROW PER MATCH and WITH UNMATCHED ROWS.

+
+
+

Processing input with row pattern recognition#

+

Pattern recognition in window processes input rows in two different cases:

+
    +
  • upon a row pattern measure call over the window:

    +
    some_measure OVER w
    +
    +
    +
  • +
  • upon a window function call over the window:

    +
    sum(total_price) OVER w
    +
    +
    +
  • +
+

The output row produced for each input row, consists of:

+
    +
  • all values from the input row

  • +
  • the value of the called measure or window function, computed with respect to +the pattern match associated with the row

  • +
+

Processing the input can be described as the following sequence of steps:

+
    +
  • Partition the input data accordingly to PARTITION BY

  • +
  • Order each partition by the ORDER BY expressions

  • +
  • +
    For every row of the ordered partition:
    +
    If the row is ‘skipped’ by a match of some previous row:
      +
    • For a measure, produce a one-row output as for an unmatched row

    • +
    • For a window function, evaluate the function over an empty frame +and produce a one-row output

    • +
    +
    +
    Otherwise:
      +
    • Determine the frame extent

    • +
    • Try match the row pattern starting from the current row within +the frame extent

    • +
    • If no match is found, and SEEK is specified, try to find a match +starting from subsequent rows within the frame extent

    • +
    +
    +
    If no match is found:
      +
    • For a measure, produce a one-row output for an unmatched row

    • +
    • For a window function, evaluate the function over an empty +frame and produce a one-row output

    • +
    +
    +
    Otherwise:
      +
    • For a measure, produce a one-row output for the match

    • +
    • For a window function, evaluate the function over a frame +limited to the matched rows sequence and produce a one-row +output

    • +
    • Evaluate the AFTER MATCH SKIP clause, and mark the ‘skipped’ +rows

    • +
    +
    +
    +
    +
    +
    +
    +
  • +
+
+
+

Empty matches and unmatched rows#

+

If no match can be associated with a particular input row, the row is +unmatched. This happens when no match can be found for the row. This also +happens when no match is attempted for the row, because it is skipped by the +AFTER MATCH SKIP clause of some preceding row. For an unmatched row, +every row pattern measure is null. Every window function is evaluated over +an empty frame.

+

An empty match is a successful match which does not involve any pattern +variables. In other words, an empty match does not contain any rows. If an +empty match is associated with an input row, every row pattern measure for that +row is evaluated over an empty sequence of rows. All navigation operations and +the CLASSIFIER function return null. Every window function is evaluated +over an empty frame.

+

In most cases, the results for empty matches and unmatched rows are the same. +A constant measure can be helpful to distinguish between them:

+

The following call returns 'matched' for every matched row, including empty +matches, and null for every unmatched row:

+
matched OVER (
+              ...
+              MEASURES 'matched' AS matched
+              ...
+             )
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/prepare.html b/430/sql/prepare.html new file mode 100644 index 000000000..9fe16e254 --- /dev/null +++ b/430/sql/prepare.html @@ -0,0 +1,933 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + PREPARE — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

PREPARE#

+
+

Synopsis#

+
PREPARE statement_name FROM statement
+
+
+
+
+

Description#

+

Prepares a statement for execution at a later time. Prepared statements are +queries that are saved in a session with a given name. The statement can +include parameters in place of literals to be replaced at execution time. +Parameters are represented by question marks.

+
+
+

Examples#

+

Prepare a select query:

+
PREPARE my_select1 FROM
+SELECT * FROM nation;
+
+
+

Prepare a select query that includes parameters. The values to compare with +regionkey and nationkey will be filled in with the EXECUTE statement:

+
PREPARE my_select2 FROM
+SELECT name FROM nation WHERE regionkey = ? AND nationkey < ?;
+
+
+

Prepare an insert query:

+
PREPARE my_insert FROM
+INSERT INTO cities VALUES (1, 'San Francisco');
+
+
+
+
+

See also#

+

EXECUTE, DEALLOCATE PREPARE, EXECUTE IMMEDIATE, DESCRIBE INPUT, DESCRIBE OUTPUT

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/refresh-materialized-view.html b/430/sql/refresh-materialized-view.html new file mode 100644 index 000000000..56b66c9fe --- /dev/null +++ b/430/sql/refresh-materialized-view.html @@ -0,0 +1,921 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + REFRESH MATERIALIZED VIEW — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

REFRESH MATERIALIZED VIEW#

+
+

Synopsis#

+
REFRESH MATERIALIZED VIEW view_name
+
+
+
+
+

Description#

+

Initially populate or refresh the data stored in the materialized view +view_name. The materialized view must be defined with +CREATE MATERIALIZED VIEW. Data is retrieved from the underlying tables +accessed by the defined query.

+

The initial population of the materialized view is typically processing +intensive since it reads the data from the source tables and performs physical +write operations.

+

The refresh operation can be less intensive, if the underlying data has not +changed and the connector has implemented a mechanism to be aware of that. The +specific implementation and performance varies by connector used to create the +materialized view.

+
+
+

See also#

+ +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/reset-session-authorization.html b/430/sql/reset-session-authorization.html new file mode 100644 index 000000000..7d31cd7b1 --- /dev/null +++ b/430/sql/reset-session-authorization.html @@ -0,0 +1,909 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + RESET SESSION AUTHORIZATION — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

RESET SESSION AUTHORIZATION#

+
+

Synopsis#

+
RESET SESSION AUTHORIZATION
+
+
+
+
+

Description#

+

Resets the current authorization user back to the original user. +The original user is usually the authenticated user (principal), +or it can be the session user when the session user is provided by the client.

+
+
+

See Also#

+

SET SESSION AUTHORIZATION

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/reset-session.html b/430/sql/reset-session.html new file mode 100644 index 000000000..55fc342b6 --- /dev/null +++ b/430/sql/reset-session.html @@ -0,0 +1,920 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + RESET SESSION — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

RESET SESSION#

+
+

Synopsis#

+
RESET SESSION name
+RESET SESSION catalog.name
+
+
+
+
+

Description#

+

Reset a session property value to the +default value.

+
+
+

Examples#

+
RESET SESSION optimize_hash_generation;
+RESET SESSION hive.optimized_reader_enabled;
+
+
+
+
+

See also#

+

SET SESSION, SHOW SESSION

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/revoke-roles.html b/430/sql/revoke-roles.html new file mode 100644 index 000000000..4f896e8a6 --- /dev/null +++ b/430/sql/revoke-roles.html @@ -0,0 +1,945 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + REVOKE ROLES — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

REVOKE ROLES#

+
+

Synopsis#

+
REVOKE
+[ ADMIN OPTION FOR ]
+role [, ...]
+FROM ( user | USER user | ROLE role) [, ...]
+[ GRANTED BY ( user | USER user | ROLE role | CURRENT_USER | CURRENT_ROLE ) ]
+[ IN catalog ]
+
+
+
+
+

Description#

+

Revokes the specified role(s) from the specified principal(s).

+

If the ADMIN OPTION FOR clause is specified, the GRANT permission is +revoked instead of the role.

+

For the REVOKE statement for roles to succeed, the user executing it either should +be the role admin or should possess the GRANT option for the given role.

+

The optional GRANTED BY clause causes the role(s) to be revoked with +the specified principal as a revoker. If the GRANTED BY clause is not +specified, the roles are revoked by the current user as a revoker.

+

The optional IN catalog clause revokes the roles in a catalog as opposed +to a system roles.

+
+
+

Examples#

+

Revoke role bar from user foo

+
REVOKE bar FROM USER foo;
+
+
+

Revoke admin option for roles bar and foo from user baz and role qux

+
REVOKE ADMIN OPTION FOR bar, foo FROM USER baz, ROLE qux;
+
+
+
+
+

Limitations#

+

Some connectors do not support role management. +See connector documentation for more details.

+
+
+

See also#

+

CREATE ROLE, DROP ROLE, SET ROLE, GRANT ROLES

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/revoke.html b/430/sql/revoke.html new file mode 100644 index 000000000..6474b1450 --- /dev/null +++ b/430/sql/revoke.html @@ -0,0 +1,949 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + REVOKE — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

REVOKE#

+
+

Synopsis#

+
REVOKE [ GRANT OPTION FOR ]
+( privilege [, ...] | ALL PRIVILEGES )
+ON ( table_name | TABLE table_name | SCHEMA schema_name )
+FROM ( user | USER user | ROLE role )
+
+
+
+
+

Description#

+

Revokes the specified privileges from the specified grantee.

+

Specifying ALL PRIVILEGES revokes DELETE, INSERT and SELECT privileges.

+

Specifying ROLE PUBLIC revokes privileges from the PUBLIC role. Users will retain privileges assigned to them directly or via other roles.

+

If the optional GRANT OPTION FOR clause is specified, only the GRANT OPTION +is removed. Otherwise, both the GRANT and GRANT OPTION are revoked.

+

For REVOKE statement to succeed, the user executing it should possess the specified privileges as well as the GRANT OPTION for those privileges.

+

Revoke on a table revokes the specified privilege on all columns of the table.

+

Revoke on a schema revokes the specified privilege on all columns of all tables of the schema.

+
+
+

Examples#

+

Revoke INSERT and SELECT privileges on the table orders from user alice:

+
REVOKE INSERT, SELECT ON orders FROM alice;
+
+
+

Revoke DELETE privilege on the schema finance from user bob:

+
REVOKE DELETE ON SCHEMA finance FROM bob;
+
+
+

Revoke SELECT privilege on the table nation from everyone, additionally revoking the privilege to grant SELECT privilege:

+
REVOKE GRANT OPTION FOR SELECT ON nation FROM ROLE PUBLIC;
+
+
+

Revoke all privileges on the table test from user alice:

+
REVOKE ALL PRIVILEGES ON test FROM alice;
+
+
+
+
+

Limitations#

+

Some connectors have no support for REVOKE. +See connector documentation for more details.

+
+
+

See also#

+

DENY, GRANT, SHOW GRANTS

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/rollback.html b/430/sql/rollback.html new file mode 100644 index 000000000..d08f3b340 --- /dev/null +++ b/430/sql/rollback.html @@ -0,0 +1,918 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ROLLBACK — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

ROLLBACK#

+
+

Synopsis#

+
ROLLBACK [ WORK ]
+
+
+
+
+

Description#

+

Rollback the current transaction.

+
+
+

Examples#

+
ROLLBACK;
+ROLLBACK WORK;
+
+
+
+
+

See also#

+

COMMIT, START TRANSACTION

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/select.html b/430/sql/select.html new file mode 100644 index 000000000..5e4445ee4 --- /dev/null +++ b/430/sql/select.html @@ -0,0 +1,2240 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SELECT — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+ + +
+
+ +
+

SELECT#

+
+

Synopsis#

+
[ WITH [ RECURSIVE ] with_query [, ...] ]
+SELECT [ ALL | DISTINCT ] select_expression [, ...]
+[ FROM from_item [, ...] ]
+[ WHERE condition ]
+[ GROUP BY [ ALL | DISTINCT ] grouping_element [, ...] ]
+[ HAVING condition]
+[ WINDOW window_definition_list]
+[ { UNION | INTERSECT | EXCEPT } [ ALL | DISTINCT ] select ]
+[ ORDER BY expression [ ASC | DESC ] [, ...] ]
+[ OFFSET count [ ROW | ROWS ] ]
+[ LIMIT { count | ALL } ]
+[ FETCH { FIRST | NEXT } [ count ] { ROW | ROWS } { ONLY | WITH TIES } ]
+
+
+

where from_item is one of

+
table_name [ [ AS ] alias [ ( column_alias [, ...] ) ] ]
+
+
+
from_item join_type from_item
+  [ ON join_condition | USING ( join_column [, ...] ) ]
+
+
+
table_name [ [ AS ] alias [ ( column_alias [, ...] ) ] ]
+  MATCH_RECOGNIZE pattern_recognition_specification
+    [ [ AS ] alias [ ( column_alias [, ...] ) ] ]
+
+
+

For detailed description of MATCH_RECOGNIZE clause, see pattern recognition in FROM clause.

+
TABLE (table_function_invocation) [ [ AS ] alias [ ( column_alias [, ...] ) ] ]
+
+
+

For description of table functions usage, see table functions.

+

and join_type is one of

+
[ INNER ] JOIN
+LEFT [ OUTER ] JOIN
+RIGHT [ OUTER ] JOIN
+FULL [ OUTER ] JOIN
+CROSS JOIN
+
+
+

and grouping_element is one of

+
()
+expression
+GROUPING SETS ( ( column [, ...] ) [, ...] )
+CUBE ( column [, ...] )
+ROLLUP ( column [, ...] )
+
+
+
+
+

Description#

+

Retrieve rows from zero or more tables.

+
+
+

WITH clause#

+

The WITH clause defines named relations for use within a query. +It allows flattening nested queries or simplifying subqueries. +For example, the following queries are equivalent:

+
SELECT a, b
+FROM (
+  SELECT a, MAX(b) AS b FROM t GROUP BY a
+) AS x;
+
+WITH x AS (SELECT a, MAX(b) AS b FROM t GROUP BY a)
+SELECT a, b FROM x;
+
+
+

This also works with multiple subqueries:

+
WITH
+  t1 AS (SELECT a, MAX(b) AS b FROM x GROUP BY a),
+  t2 AS (SELECT a, AVG(d) AS d FROM y GROUP BY a)
+SELECT t1.*, t2.*
+FROM t1
+JOIN t2 ON t1.a = t2.a;
+
+
+

Additionally, the relations within a WITH clause can chain:

+
WITH
+  x AS (SELECT a FROM t),
+  y AS (SELECT a AS b FROM x),
+  z AS (SELECT b AS c FROM y)
+SELECT c FROM z;
+
+
+
+

Warning

+

Currently, the SQL for the WITH clause will be inlined anywhere the named +relation is used. This means that if the relation is used more than once and the query +is non-deterministic, the results may be different each time.

+
+
+
+

WITH RECURSIVE clause#

+

The WITH RECURSIVE clause is a variant of the WITH clause. It defines +a list of queries to process, including recursive processing of suitable +queries.

+
+

Warning

+

This feature is experimental only. Proceed to use it only if you understand +potential query failures and the impact of the recursion processing on your +workload.

+
+

A recursive WITH-query must be shaped as a UNION of two relations. The +first relation is called the recursion base, and the second relation is called +the recursion step. Trino supports recursive WITH-queries with a single +recursive reference to a WITH-query from within the query. The name T of +the query T can be mentioned once in the FROM clause of the recursion +step relation.

+

The following listing shows a simple example, that displays a commonly used +form of a single query in the list:

+
WITH RECURSIVE t(n) AS (
+    VALUES (1)
+    UNION ALL
+    SELECT n + 1 FROM t WHERE n < 4
+)
+SELECT sum(n) FROM t;
+
+
+

In the preceding query the simple assignment VALUES (1) defines the +recursion base relation. SELECT n + 1 FROM t WHERE n < 4 defines the +recursion step relation. The recursion processing performs these steps:

+
    +
  • recursive base yields 1

  • +
  • first recursion yields 1 + 1 = 2

  • +
  • second recursion uses the result from the first and adds one: 2 + 1 = 3

  • +
  • third recursion uses the result from the second and adds one again: +3 + 1 = 4

  • +
  • fourth recursion aborts since n = 4

  • +
  • this results in t having values 1, 2, 3 and 4

  • +
  • the final statement performs the sum operation of these elements with the +final result value 10

  • +
+

The types of the returned columns are those of the base relation. Therefore it +is required that types in the step relation can be coerced to base relation +types.

+

The RECURSIVE clause applies to all queries in the WITH list, but not +all of them must be recursive. If a WITH-query is not shaped according to +the rules mentioned above or it does not contain a recursive reference, it is +processed like a regular WITH-query. Column aliases are mandatory for all +the queries in the recursive WITH list.

+

The following limitations apply as a result of following the SQL standard and +due to implementation choices, in addition to WITH clause limitations:

+
    +
  • only single-element recursive cycles are supported. Like in regular +WITH-queries, references to previous queries in the WITH list are +allowed. References to following queries are forbidden.

  • +
  • usage of outer joins, set operations, limit clause, and others is not always +allowed in the step relation

  • +
  • recursion depth is fixed, defaults to 10, and doesn’t depend on the actual +query results

  • +
+

You can adjust the recursion depth with the session property max_recursion_depth. When changing the value consider +that the size of the query plan growth is quadratic with the recursion depth.

+
+
+

SELECT clause#

+

The SELECT clause specifies the output of the query. Each select_expression +defines a column or columns to be included in the result.

+
SELECT [ ALL | DISTINCT ] select_expression [, ...]
+
+
+

The ALL and DISTINCT quantifiers determine whether duplicate rows +are included in the result set. If the argument ALL is specified, +all rows are included. If the argument DISTINCT is specified, only unique +rows are included in the result set. In this case, each output column must +be of a type that allows comparison. If neither argument is specified, +the behavior defaults to ALL.

+
+

Select expressions#

+

Each select_expression must be in one of the following forms:

+
expression [ [ AS ] column_alias ]
+
+
+
row_expression.* [ AS ( column_alias [, ...] ) ]
+
+
+
relation.*
+
+
+
*
+
+
+

In the case of expression [ [ AS ] column_alias ], a single output column +is defined.

+

In the case of row_expression.* [ AS ( column_alias [, ...] ) ], +the row_expression is an arbitrary expression of type ROW. +All fields of the row define output columns to be included in the result set.

+

In the case of relation.*, all columns of relation are included +in the result set. In this case column aliases are not allowed.

+

In the case of *, all columns of the relation defined by the query +are included in the result set.

+

In the result set, the order of columns is the same as the order of their +specification by the select expressions. If a select expression returns multiple +columns, they are ordered the same way they were ordered in the source +relation or row type expression.

+

If column aliases are specified, they override any preexisting column +or row field names:

+
SELECT (CAST(ROW(1, true) AS ROW(field1 bigint, field2 boolean))).* AS (alias1, alias2);
+
+
+
 alias1 | alias2
+--------+--------
+      1 | true
+(1 row)
+
+
+

Otherwise, the existing names are used:

+
SELECT (CAST(ROW(1, true) AS ROW(field1 bigint, field2 boolean))).*;
+
+
+
 field1 | field2
+--------+--------
+      1 | true
+(1 row)
+
+
+

and in their absence, anonymous columns are produced:

+
SELECT (ROW(1, true)).*;
+
+
+
 _col0 | _col1
+-------+-------
+     1 | true
+(1 row)
+
+
+
+
+
+

GROUP BY clause#

+

The GROUP BY clause divides the output of a SELECT statement into +groups of rows containing matching values. A simple GROUP BY clause may +contain any expression composed of input columns or it may be an ordinal +number selecting an output column by position (starting at one).

+

The following queries are equivalent. They both group the output by +the nationkey input column with the first query using the ordinal +position of the output column and the second query using the input +column name:

+
SELECT count(*), nationkey FROM customer GROUP BY 2;
+
+SELECT count(*), nationkey FROM customer GROUP BY nationkey;
+
+
+

GROUP BY clauses can group output by input column names not appearing in +the output of a select statement. For example, the following query generates +row counts for the customer table using the input column mktsegment:

+
SELECT count(*) FROM customer GROUP BY mktsegment;
+
+
+
 _col0
+-------
+ 29968
+ 30142
+ 30189
+ 29949
+ 29752
+(5 rows)
+
+
+

When a GROUP BY clause is used in a SELECT statement all output +expressions must be either aggregate functions or columns present in +the GROUP BY clause.

+
+

Complex grouping operations#

+

Trino also supports complex aggregations using the GROUPING SETS, CUBE +and ROLLUP syntax. This syntax allows users to perform analysis that requires +aggregation on multiple sets of columns in a single query. Complex grouping +operations do not support grouping on expressions composed of input columns. +Only column names are allowed.

+

Complex grouping operations are often equivalent to a UNION ALL of simple +GROUP BY expressions, as shown in the following examples. This equivalence +does not apply, however, when the source of data for the aggregation +is non-deterministic.

+
+
+

GROUPING SETS#

+

Grouping sets allow users to specify multiple lists of columns to group on. +The columns not part of a given sublist of grouping columns are set to NULL.

+
SELECT * FROM shipping;
+
+
+
 origin_state | origin_zip | destination_state | destination_zip | package_weight
+--------------+------------+-------------------+-----------------+----------------
+ California   |      94131 | New Jersey        |            8648 |             13
+ California   |      94131 | New Jersey        |            8540 |             42
+ New Jersey   |       7081 | Connecticut       |            6708 |            225
+ California   |      90210 | Connecticut       |            6927 |           1337
+ California   |      94131 | Colorado          |           80302 |              5
+ New York     |      10002 | New Jersey        |            8540 |              3
+(6 rows)
+
+
+

GROUPING SETS semantics are demonstrated by this example query:

+
SELECT origin_state, origin_zip, destination_state, sum(package_weight)
+FROM shipping
+GROUP BY GROUPING SETS (
+    (origin_state),
+    (origin_state, origin_zip),
+    (destination_state));
+
+
+
 origin_state | origin_zip | destination_state | _col0
+--------------+------------+-------------------+-------
+ New Jersey   | NULL       | NULL              |   225
+ California   | NULL       | NULL              |  1397
+ New York     | NULL       | NULL              |     3
+ California   |      90210 | NULL              |  1337
+ California   |      94131 | NULL              |    60
+ New Jersey   |       7081 | NULL              |   225
+ New York     |      10002 | NULL              |     3
+ NULL         | NULL       | Colorado          |     5
+ NULL         | NULL       | New Jersey        |    58
+ NULL         | NULL       | Connecticut       |  1562
+(10 rows)
+
+
+

The preceding query may be considered logically equivalent to a UNION ALL of +multiple GROUP BY queries:

+
SELECT origin_state, NULL, NULL, sum(package_weight)
+FROM shipping GROUP BY origin_state
+
+UNION ALL
+
+SELECT origin_state, origin_zip, NULL, sum(package_weight)
+FROM shipping GROUP BY origin_state, origin_zip
+
+UNION ALL
+
+SELECT NULL, NULL, destination_state, sum(package_weight)
+FROM shipping GROUP BY destination_state;
+
+
+

However, the query with the complex grouping syntax (GROUPING SETS, CUBE +or ROLLUP) will only read from the underlying data source once, while the +query with the UNION ALL reads the underlying data three times. This is why +queries with a UNION ALL may produce inconsistent results when the data +source is not deterministic.

+
+
+

CUBE#

+

The CUBE operator generates all possible grouping sets (i.e. a power set) +for a given set of columns. For example, the query:

+
SELECT origin_state, destination_state, sum(package_weight)
+FROM shipping
+GROUP BY CUBE (origin_state, destination_state);
+
+
+

is equivalent to:

+
SELECT origin_state, destination_state, sum(package_weight)
+FROM shipping
+GROUP BY GROUPING SETS (
+    (origin_state, destination_state),
+    (origin_state),
+    (destination_state),
+    ()
+);
+
+
+
 origin_state | destination_state | _col0
+--------------+-------------------+-------
+ California   | New Jersey        |    55
+ California   | Colorado          |     5
+ New York     | New Jersey        |     3
+ New Jersey   | Connecticut       |   225
+ California   | Connecticut       |  1337
+ California   | NULL              |  1397
+ New York     | NULL              |     3
+ New Jersey   | NULL              |   225
+ NULL         | New Jersey        |    58
+ NULL         | Connecticut       |  1562
+ NULL         | Colorado          |     5
+ NULL         | NULL              |  1625
+(12 rows)
+
+
+
+
+

ROLLUP#

+

The ROLLUP operator generates all possible subtotals for a given set of +columns. For example, the query:

+
SELECT origin_state, origin_zip, sum(package_weight)
+FROM shipping
+GROUP BY ROLLUP (origin_state, origin_zip);
+
+
+
 origin_state | origin_zip | _col2
+--------------+------------+-------
+ California   |      94131 |    60
+ California   |      90210 |  1337
+ New Jersey   |       7081 |   225
+ New York     |      10002 |     3
+ California   | NULL       |  1397
+ New York     | NULL       |     3
+ New Jersey   | NULL       |   225
+ NULL         | NULL       |  1625
+(8 rows)
+
+
+

is equivalent to:

+
SELECT origin_state, origin_zip, sum(package_weight)
+FROM shipping
+GROUP BY GROUPING SETS ((origin_state, origin_zip), (origin_state), ());
+
+
+
+
+

Combining multiple grouping expressions#

+

Multiple grouping expressions in the same query are interpreted as having +cross-product semantics. For example, the following query:

+
SELECT origin_state, destination_state, origin_zip, sum(package_weight)
+FROM shipping
+GROUP BY
+    GROUPING SETS ((origin_state, destination_state)),
+    ROLLUP (origin_zip);
+
+
+

which can be rewritten as:

+
SELECT origin_state, destination_state, origin_zip, sum(package_weight)
+FROM shipping
+GROUP BY
+    GROUPING SETS ((origin_state, destination_state)),
+    GROUPING SETS ((origin_zip), ());
+
+
+

is logically equivalent to:

+
SELECT origin_state, destination_state, origin_zip, sum(package_weight)
+FROM shipping
+GROUP BY GROUPING SETS (
+    (origin_state, destination_state, origin_zip),
+    (origin_state, destination_state)
+);
+
+
+
 origin_state | destination_state | origin_zip | _col3
+--------------+-------------------+------------+-------
+ New York     | New Jersey        |      10002 |     3
+ California   | New Jersey        |      94131 |    55
+ New Jersey   | Connecticut       |       7081 |   225
+ California   | Connecticut       |      90210 |  1337
+ California   | Colorado          |      94131 |     5
+ New York     | New Jersey        | NULL       |     3
+ New Jersey   | Connecticut       | NULL       |   225
+ California   | Colorado          | NULL       |     5
+ California   | Connecticut       | NULL       |  1337
+ California   | New Jersey        | NULL       |    55
+(10 rows)
+
+
+

The ALL and DISTINCT quantifiers determine whether duplicate grouping +sets each produce distinct output rows. This is particularly useful when +multiple complex grouping sets are combined in the same query. For example, the +following query:

+
SELECT origin_state, destination_state, origin_zip, sum(package_weight)
+FROM shipping
+GROUP BY ALL
+    CUBE (origin_state, destination_state),
+    ROLLUP (origin_state, origin_zip);
+
+
+

is equivalent to:

+
SELECT origin_state, destination_state, origin_zip, sum(package_weight)
+FROM shipping
+GROUP BY GROUPING SETS (
+    (origin_state, destination_state, origin_zip),
+    (origin_state, origin_zip),
+    (origin_state, destination_state, origin_zip),
+    (origin_state, origin_zip),
+    (origin_state, destination_state),
+    (origin_state),
+    (origin_state, destination_state),
+    (origin_state),
+    (origin_state, destination_state),
+    (origin_state),
+    (destination_state),
+    ()
+);
+
+
+

However, if the query uses the DISTINCT quantifier for the GROUP BY:

+
SELECT origin_state, destination_state, origin_zip, sum(package_weight)
+FROM shipping
+GROUP BY DISTINCT
+    CUBE (origin_state, destination_state),
+    ROLLUP (origin_state, origin_zip);
+
+
+

only unique grouping sets are generated:

+
SELECT origin_state, destination_state, origin_zip, sum(package_weight)
+FROM shipping
+GROUP BY GROUPING SETS (
+    (origin_state, destination_state, origin_zip),
+    (origin_state, origin_zip),
+    (origin_state, destination_state),
+    (origin_state),
+    (destination_state),
+    ()
+);
+
+
+

The default set quantifier is ALL.

+
+
+

GROUPING operation#

+

grouping(col1, ..., colN) -> bigint

+

The grouping operation returns a bit set converted to decimal, indicating which columns are present in a +grouping. It must be used in conjunction with GROUPING SETS, ROLLUP, CUBE or GROUP BY +and its arguments must match exactly the columns referenced in the corresponding GROUPING SETS, +ROLLUP, CUBE or GROUP BY clause.

+

To compute the resulting bit set for a particular row, bits are assigned to the argument columns with +the rightmost column being the least significant bit. For a given grouping, a bit is set to 0 if the +corresponding column is included in the grouping and to 1 otherwise. For example, consider the query +below:

+
SELECT origin_state, origin_zip, destination_state, sum(package_weight),
+       grouping(origin_state, origin_zip, destination_state)
+FROM shipping
+GROUP BY GROUPING SETS (
+    (origin_state),
+    (origin_state, origin_zip),
+    (destination_state)
+);
+
+
+
origin_state | origin_zip | destination_state | _col3 | _col4
+--------------+------------+-------------------+-------+-------
+California   | NULL       | NULL              |  1397 |     3
+New Jersey   | NULL       | NULL              |   225 |     3
+New York     | NULL       | NULL              |     3 |     3
+California   |      94131 | NULL              |    60 |     1
+New Jersey   |       7081 | NULL              |   225 |     1
+California   |      90210 | NULL              |  1337 |     1
+New York     |      10002 | NULL              |     3 |     1
+NULL         | NULL       | New Jersey        |    58 |     6
+NULL         | NULL       | Connecticut       |  1562 |     6
+NULL         | NULL       | Colorado          |     5 |     6
+(10 rows)
+
+
+

The first grouping in the above result only includes the origin_state column and excludes +the origin_zip and destination_state columns. The bit set constructed for that grouping +is 011 where the most significant bit represents origin_state.

+
+
+
+

HAVING clause#

+

The HAVING clause is used in conjunction with aggregate functions and +the GROUP BY clause to control which groups are selected. A HAVING +clause eliminates groups that do not satisfy the given conditions. +HAVING filters groups after groups and aggregates are computed.

+

The following example queries the customer table and selects groups +with an account balance greater than the specified value:

+
SELECT count(*), mktsegment, nationkey,
+       CAST(sum(acctbal) AS bigint) AS totalbal
+FROM customer
+GROUP BY mktsegment, nationkey
+HAVING sum(acctbal) > 5700000
+ORDER BY totalbal DESC;
+
+
+
 _col0 | mktsegment | nationkey | totalbal
+-------+------------+-----------+----------
+  1272 | AUTOMOBILE |        19 |  5856939
+  1253 | FURNITURE  |        14 |  5794887
+  1248 | FURNITURE  |         9 |  5784628
+  1243 | FURNITURE  |        12 |  5757371
+  1231 | HOUSEHOLD  |         3 |  5753216
+  1251 | MACHINERY  |         2 |  5719140
+  1247 | FURNITURE  |         8 |  5701952
+(7 rows)
+
+
+
+
+

WINDOW clause#

+

The WINDOW clause is used to define named window specifications. The defined named +window specifications can be referred to in the SELECT and ORDER BY clauses +of the enclosing query:

+
SELECT orderkey, clerk, totalprice,
+      rank() OVER w AS rnk
+FROM orders
+WINDOW w AS (PARTITION BY clerk ORDER BY totalprice DESC)
+ORDER BY count() OVER w, clerk, rnk
+
+
+

The window definition list of WINDOW clause can contain one or multiple named window +specifications of the form

+
window_name AS (window_specification)
+
+
+

A window specification has the following components:

+
    +
  • The existing window name, which refers to a named window specification in the +WINDOW clause. The window specification associated with the referenced name +is the basis of the current specification.

  • +
  • The partition specification, which separates the input rows into different +partitions. This is analogous to how the GROUP BY clause separates rows +into different groups for aggregate functions.

  • +
  • The ordering specification, which determines the order in which input rows +will be processed by the window function.

  • +
  • The window frame, which specifies a sliding window of rows to be processed +by the function for a given row. If the frame is not specified, it defaults +to RANGE UNBOUNDED PRECEDING, which is the same as +RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW. This frame contains all +rows from the start of the partition up to the last peer of the current row. +In the absence of ORDER BY, all rows are considered peers, so RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW is equivalent to BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING. The window frame syntax +supports additional clauses for row pattern recognition. If the row pattern +recognition clauses are specified, the window frame for a particular row +consists of the rows matched by a pattern starting from that row. +Additionally, if the frame specifies row pattern measures, they can be +called over the window, similarly to window functions. For more details, see +Row pattern recognition in window structures .

  • +
+

Each window component is optional. If a window specification does not specify +window partitioning, ordering or frame, those components are obtained from +the window specification referenced by the existing window name, or from +another window specification in the reference chain. In case when there is no +existing window name specified, or none of the referenced window +specifications contains the component, the default value is used.

+
+
+

Set operations#

+

UNION INTERSECT and EXCEPT are all set operations. These clauses are used +to combine the results of more than one select statement into a single result set:

+
query UNION [ALL | DISTINCT] query
+
+
+
query INTERSECT [ALL | DISTINCT] query
+
+
+
query EXCEPT [ALL | DISTINCT] query
+
+
+

The argument ALL or DISTINCT controls which rows are included in +the final result set. If the argument ALL is specified all rows are +included even if the rows are identical. If the argument DISTINCT +is specified only unique rows are included in the combined result set. +If neither is specified, the behavior defaults to DISTINCT.

+

Multiple set operations are processed left to right, unless the order is explicitly +specified via parentheses. Additionally, INTERSECT binds more tightly +than EXCEPT and UNION. That means A UNION B INTERSECT C EXCEPT D +is the same as A UNION (B INTERSECT C) EXCEPT D.

+
+

UNION clause#

+

UNION combines all the rows that are in the result set from the +first query with those that are in the result set for the second query. +The following is an example of one of the simplest possible UNION clauses. +It selects the value 13 and combines this result set with a second query +that selects the value 42:

+
SELECT 13
+UNION
+SELECT 42;
+
+
+
 _col0
+-------
+    13
+    42
+(2 rows)
+
+
+

The following query demonstrates the difference between UNION and UNION ALL. +It selects the value 13 and combines this result set with a second query that +selects the values 42 and 13:

+
SELECT 13
+UNION
+SELECT * FROM (VALUES 42, 13);
+
+
+
 _col0
+-------
+    13
+    42
+(2 rows)
+
+
+
SELECT 13
+UNION ALL
+SELECT * FROM (VALUES 42, 13);
+
+
+
 _col0
+-------
+    13
+    42
+    13
+(2 rows)
+
+
+
+
+

INTERSECT clause#

+

INTERSECT returns only the rows that are in the result sets of both the first and +the second queries. The following is an example of one of the simplest +possible INTERSECT clauses. It selects the values 13 and 42 and combines +this result set with a second query that selects the value 13. Since 42 +is only in the result set of the first query, it is not included in the final results.:

+
SELECT * FROM (VALUES 13, 42)
+INTERSECT
+SELECT 13;
+
+
+
 _col0
+-------
+    13
+(2 rows)
+
+
+
+
+

EXCEPT clause#

+

EXCEPT returns the rows that are in the result set of the first query, +but not the second. The following is an example of one of the simplest +possible EXCEPT clauses. It selects the values 13 and 42 and combines +this result set with a second query that selects the value 13. Since 13 +is also in the result set of the second query, it is not included in the final result.:

+
SELECT * FROM (VALUES 13, 42)
+EXCEPT
+SELECT 13;
+
+
+
 _col0
+-------
+   42
+(2 rows)
+
+
+
+
+
+

ORDER BY clause#

+

The ORDER BY clause is used to sort a result set by one or more +output expressions:

+
ORDER BY expression [ ASC | DESC ] [ NULLS { FIRST | LAST } ] [, ...]
+
+
+

Each expression may be composed of output columns, or it may be an ordinal +number selecting an output column by position, starting at one. The +ORDER BY clause is evaluated after any GROUP BY or HAVING clause, +and before any OFFSET, LIMIT or FETCH FIRST clause. +The default null ordering is NULLS LAST, regardless of the ordering direction.

+

Note that, following the SQL specification, an ORDER BY clause only +affects the order of rows for queries that immediately contain the clause. +Trino follows that specification, and drops redundant usage of the clause to +avoid negative performance impacts.

+

In the following example, the clause only applies to the select statement.

+
INSERT INTO some_table
+SELECT * FROM another_table
+ORDER BY field;
+
+
+

Since tables in SQL are inherently unordered, and the ORDER BY clause in +this case does not result in any difference, but negatively impacts performance +of running the overall insert statement, Trino skips the sort operation.

+

Another example where the ORDER BY clause is redundant, and does not affect +the outcome of the overall statement, is a nested query:

+
SELECT *
+FROM some_table
+    JOIN (SELECT * FROM another_table ORDER BY field) u
+    ON some_table.key = u.key;
+
+
+

More background information and details can be found in +a blog post about this optimization.

+
+
+

OFFSET clause#

+

The OFFSET clause is used to discard a number of leading rows +from the result set:

+
OFFSET count [ ROW | ROWS ]
+
+
+

If the ORDER BY clause is present, the OFFSET clause is evaluated +over a sorted result set, and the set remains sorted after the +leading rows are discarded:

+
SELECT name FROM nation ORDER BY name OFFSET 22;
+
+
+
      name
+----------------
+ UNITED KINGDOM
+ UNITED STATES
+ VIETNAM
+(3 rows)
+
+
+

Otherwise, it is arbitrary which rows are discarded. +If the count specified in the OFFSET clause equals or exceeds the size +of the result set, the final result is empty.

+
+
+

LIMIT or FETCH FIRST clause#

+

The LIMIT or FETCH FIRST clause restricts the number of rows +in the result set.

+
LIMIT { count | ALL }
+
+
+
FETCH { FIRST | NEXT } [ count ] { ROW | ROWS } { ONLY | WITH TIES }
+
+
+

The following example queries a large table, but the LIMIT clause +restricts the output to only have five rows (because the query lacks an ORDER BY, +exactly which rows are returned is arbitrary):

+
SELECT orderdate FROM orders LIMIT 5;
+
+
+
 orderdate
+------------
+ 1994-07-25
+ 1993-11-12
+ 1992-10-06
+ 1994-01-04
+ 1997-12-28
+(5 rows)
+
+
+

LIMIT ALL is the same as omitting the LIMIT clause.

+

The FETCH FIRST clause supports either the FIRST or NEXT keywords +and the ROW or ROWS keywords. These keywords are equivalent and +the choice of keyword has no effect on query execution.

+

If the count is not specified in the FETCH FIRST clause, it defaults to 1:

+
SELECT orderdate FROM orders FETCH FIRST ROW ONLY;
+
+
+
 orderdate
+------------
+ 1994-02-12
+(1 row)
+
+
+

If the OFFSET clause is present, the LIMIT or FETCH FIRST clause +is evaluated after the OFFSET clause:

+
SELECT * FROM (VALUES 5, 2, 4, 1, 3) t(x) ORDER BY x OFFSET 2 LIMIT 2;
+
+
+
 x
+---
+ 3
+ 4
+(2 rows)
+
+
+

For the FETCH FIRST clause, the argument ONLY or WITH TIES +controls which rows are included in the result set.

+

If the argument ONLY is specified, the result set is limited to the exact +number of leading rows determined by the count.

+

If the argument WITH TIES is specified, it is required that the ORDER BY +clause be present. The result set consists of the same set of leading rows +and all of the rows in the same peer group as the last of them (‘ties’) +as established by the ordering in the ORDER BY clause. The result set is sorted:

+
SELECT name, regionkey
+FROM nation
+ORDER BY regionkey FETCH FIRST ROW WITH TIES;
+
+
+
    name    | regionkey
+------------+-----------
+ ETHIOPIA   |         0
+ MOROCCO    |         0
+ KENYA      |         0
+ ALGERIA    |         0
+ MOZAMBIQUE |         0
+(5 rows)
+
+
+
+
+

TABLESAMPLE#

+

There are multiple sample methods:

+
+
BERNOULLI

Each row is selected to be in the table sample with a probability of +the sample percentage. When a table is sampled using the Bernoulli +method, all physical blocks of the table are scanned and certain +rows are skipped (based on a comparison between the sample percentage +and a random value calculated at runtime).

+

The probability of a row being included in the result is independent +from any other row. This does not reduce the time required to read +the sampled table from disk. It may have an impact on the total +query time if the sampled output is processed further.

+
+
SYSTEM

This sampling method divides the table into logical segments of data +and samples the table at this granularity. This sampling method either +selects all the rows from a particular segment of data or skips it +(based on a comparison between the sample percentage and a random +value calculated at runtime).

+

The rows selected in a system sampling will be dependent on which +connector is used. For example, when used with Hive, it is dependent +on how the data is laid out on HDFS. This method does not guarantee +independent sampling probabilities.

+
+
+
+

Note

+

Neither of the two methods allow deterministic bounds on the number of rows returned.

+
+

Examples:

+
SELECT *
+FROM users TABLESAMPLE BERNOULLI (50);
+
+SELECT *
+FROM users TABLESAMPLE SYSTEM (75);
+
+
+

Using sampling with joins:

+
SELECT o.*, i.*
+FROM orders o TABLESAMPLE SYSTEM (10)
+JOIN lineitem i TABLESAMPLE BERNOULLI (40)
+  ON o.orderkey = i.orderkey;
+
+
+
+
+

UNNEST#

+

UNNEST can be used to expand an ARRAY or MAP into a relation. +Arrays are expanded into a single column:

+
SELECT * FROM UNNEST(ARRAY[1,2]) AS t(number);
+
+
+
 number
+--------
+      1
+      2
+(2 rows)
+
+
+

Maps are expanded into two columns (key, value):

+
SELECT * FROM UNNEST(
+        map_from_entries(
+            ARRAY[
+                ('SQL',1974),
+                ('Java', 1995)
+            ]
+        )
+) AS t(language, first_appeared_year);
+
+
+
 language | first_appeared_year
+----------+---------------------
+ SQL      |                1974
+ Java     |                1995
+(2 rows)
+
+
+

UNNEST can be used in combination with an ARRAY of ROW structures for expanding each +field of the ROW into a corresponding column:

+
SELECT *
+FROM UNNEST(
+        ARRAY[
+            ROW('Java',  1995),
+            ROW('SQL' , 1974)],
+        ARRAY[
+            ROW(false),
+            ROW(true)]
+) as t(language,first_appeared_year,declarative);
+
+
+
 language | first_appeared_year | declarative
+----------+---------------------+-------------
+ Java     |                1995 | false
+ SQL      |                1974 | true
+(2 rows)
+
+
+

UNNEST can optionally have a WITH ORDINALITY clause, in which case an additional ordinality column +is added to the end:

+
SELECT a, b, rownumber
+FROM UNNEST (
+    ARRAY[2, 5],
+    ARRAY[7, 8, 9]
+     ) WITH ORDINALITY AS t(a, b, rownumber);
+
+
+
  a   | b | rownumber
+------+---+-----------
+    2 | 7 |         1
+    5 | 8 |         2
+ NULL | 9 |         3
+(3 rows)
+
+
+

UNNEST returns zero entries when the array/map is empty:

+
SELECT * FROM UNNEST (ARRAY[]) AS t(value);
+
+
+
 value
+-------
+(0 rows)
+
+
+

UNNEST returns zero entries when the array/map is null:

+
SELECT * FROM UNNEST (CAST(null AS ARRAY(integer))) AS t(number);
+
+
+
 number
+--------
+(0 rows)
+
+
+

UNNEST is normally used with a JOIN, and can reference columns +from relations on the left side of the join:

+
SELECT student, score
+FROM (
+   VALUES
+      ('John', ARRAY[7, 10, 9]),
+      ('Mary', ARRAY[4, 8, 9])
+) AS tests (student, scores)
+CROSS JOIN UNNEST(scores) AS t(score);
+
+
+
 student | score
+---------+-------
+ John    |     7
+ John    |    10
+ John    |     9
+ Mary    |     4
+ Mary    |     8
+ Mary    |     9
+(6 rows)
+
+
+

UNNEST can also be used with multiple arguments, in which case they are expanded into multiple columns, +with as many rows as the highest cardinality argument (the other columns are padded with nulls):

+
SELECT numbers, animals, n, a
+FROM (
+  VALUES
+    (ARRAY[2, 5], ARRAY['dog', 'cat', 'bird']),
+    (ARRAY[7, 8, 9], ARRAY['cow', 'pig'])
+) AS x (numbers, animals)
+CROSS JOIN UNNEST(numbers, animals) AS t (n, a);
+
+
+
  numbers  |     animals      |  n   |  a
+-----------+------------------+------+------
+ [2, 5]    | [dog, cat, bird] |    2 | dog
+ [2, 5]    | [dog, cat, bird] |    5 | cat
+ [2, 5]    | [dog, cat, bird] | NULL | bird
+ [7, 8, 9] | [cow, pig]       |    7 | cow
+ [7, 8, 9] | [cow, pig]       |    8 | pig
+ [7, 8, 9] | [cow, pig]       |    9 | NULL
+(6 rows)
+
+
+

LEFT JOIN is preferable in order to avoid losing the the row containing the array/map field in question +when referenced columns from relations on the left side of the join can be empty or have NULL values:

+
SELECT runner, checkpoint
+FROM (
+   VALUES
+      ('Joe', ARRAY[10, 20, 30, 42]),
+      ('Roger', ARRAY[10]),
+      ('Dave', ARRAY[]),
+      ('Levi', NULL)
+) AS marathon (runner, checkpoints)
+LEFT JOIN UNNEST(checkpoints) AS t(checkpoint) ON TRUE;
+
+
+
 runner | checkpoint
+--------+------------
+ Joe    |         10
+ Joe    |         20
+ Joe    |         30
+ Joe    |         42
+ Roger  |         10
+ Dave   |       NULL
+ Levi   |       NULL
+(7 rows)
+
+
+

Note that in case of using LEFT JOIN the only condition supported by the current implementation is ON TRUE.

+
+
+

Joins#

+

Joins allow you to combine data from multiple relations.

+
+

CROSS JOIN#

+

A cross join returns the Cartesian product (all combinations) of two +relations. Cross joins can either be specified using the explit +CROSS JOIN syntax or by specifying multiple relations in the +FROM clause.

+

Both of the following queries are equivalent:

+
SELECT *
+FROM nation
+CROSS JOIN region;
+
+SELECT *
+FROM nation, region;
+
+
+

The nation table contains 25 rows and the region table contains 5 rows, +so a cross join between the two tables produces 125 rows:

+
SELECT n.name AS nation, r.name AS region
+FROM nation AS n
+CROSS JOIN region AS r
+ORDER BY 1, 2;
+
+
+
     nation     |   region
+----------------+-------------
+ ALGERIA        | AFRICA
+ ALGERIA        | AMERICA
+ ALGERIA        | ASIA
+ ALGERIA        | EUROPE
+ ALGERIA        | MIDDLE EAST
+ ARGENTINA      | AFRICA
+ ARGENTINA      | AMERICA
+...
+(125 rows)
+
+
+
+
+

LATERAL#

+

Subqueries appearing in the FROM clause can be preceded by the keyword LATERAL. +This allows them to reference columns provided by preceding FROM items.

+

A LATERAL join can appear at the top level in the FROM list, or anywhere +within a parenthesized join tree. In the latter case, it can also refer to any items +that are on the left-hand side of a JOIN for which it is on the right-hand side.

+

When a FROM item contains LATERAL cross-references, evaluation proceeds as follows: +for each row of the FROM item providing the cross-referenced columns, +the LATERAL item is evaluated using that row set’s values of the columns. +The resulting rows are joined as usual with the rows they were computed from. +This is repeated for set of rows from the column source tables.

+

LATERAL is primarily useful when the cross-referenced column is necessary for +computing the rows to be joined:

+
SELECT name, x, y
+FROM nation
+CROSS JOIN LATERAL (SELECT name || ' :-' AS x)
+CROSS JOIN LATERAL (SELECT x || ')' AS y);
+
+
+
+
+

Qualifying column names#

+

When two relations in a join have columns with the same name, the column +references must be qualified using the relation alias (if the relation +has an alias), or with the relation name:

+
SELECT nation.name, region.name
+FROM nation
+CROSS JOIN region;
+
+SELECT n.name, r.name
+FROM nation AS n
+CROSS JOIN region AS r;
+
+SELECT n.name, r.name
+FROM nation n
+CROSS JOIN region r;
+
+
+

The following query will fail with the error Column 'name' is ambiguous:

+
SELECT name
+FROM nation
+CROSS JOIN region;
+
+
+
+
+
+

Subqueries#

+

A subquery is an expression which is composed of a query. The subquery +is correlated when it refers to columns outside of the subquery. +Logically, the subquery will be evaluated for each row in the surrounding +query. The referenced columns will thus be constant during any single +evaluation of the subquery.

+
+

Note

+

Support for correlated subqueries is limited. Not every standard form is supported.

+
+
+

EXISTS#

+

The EXISTS predicate determines if a subquery returns any rows:

+
SELECT name
+FROM nation
+WHERE EXISTS (
+     SELECT *
+     FROM region
+     WHERE region.regionkey = nation.regionkey
+);
+
+
+
+
+

IN#

+

The IN predicate determines if any values produced by the subquery +are equal to the provided expression. The result of IN follows the +standard rules for nulls. The subquery must produce exactly one column:

+
SELECT name
+FROM nation
+WHERE regionkey IN (
+     SELECT regionkey
+     FROM region
+     WHERE name = 'AMERICA' OR name = 'AFRICA'
+);
+
+
+
+
+

Scalar subquery#

+

A scalar subquery is a non-correlated subquery that returns zero or +one row. It is an error for the subquery to produce more than one +row. The returned value is NULL if the subquery produces no rows:

+
SELECT name
+FROM nation
+WHERE regionkey = (SELECT max(regionkey) FROM region);
+
+
+
+

Note

+

Currently only single column can be returned from the scalar subquery.

+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/set-path.html b/430/sql/set-path.html new file mode 100644 index 000000000..4d94aefe5 --- /dev/null +++ b/430/sql/set-path.html @@ -0,0 +1,941 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SET PATH — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

SET PATH#

+
+

Synopsis#

+
SET PATH path-element[, ...]
+
+
+
+
+

Description#

+

Define a collection of paths to functions or table functions in specific +catalogs and schemas for the current session.

+

Each path-element uses a period-separated syntax to specify the catalog name and +schema location <catalog>.<schema> of the function, or only the schema +location <schema> in the current catalog. The current catalog is set with +USE, or as part of a client tool connection. Catalog and schema must +exist.

+
+
+

Examples#

+

The following example sets a path to access functions in the system schema +of the example catalog:

+
SET PATH example.system;
+
+
+

The catalog uses the PostgreSQL connector, and you can therefore use the +query table function directly, without the +full catalog and schema qualifiers:

+
SELECT
+  *
+FROM
+  TABLE(
+    query(
+      query => 'SELECT
+        *
+      FROM
+        tpch.nation'
+    )
+  );
+
+
+
+
+

See also#

+

USE

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/set-role.html b/430/sql/set-role.html new file mode 100644 index 000000000..1d37f78aa --- /dev/null +++ b/430/sql/set-role.html @@ -0,0 +1,926 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SET ROLE — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

SET ROLE#

+
+

Synopsis#

+
SET ROLE ( role | ALL | NONE )
+[ IN catalog ]
+
+
+
+
+

Description#

+

SET ROLE sets the enabled role for the current session.

+

SET ROLE role enables a single specified role for the current session. +For the SET ROLE role statement to succeed, the user executing it should +have a grant for the given role.

+

SET ROLE ALL enables all roles that the current user has been granted for the +current session.

+

SET ROLE NONE disables all the roles granted to the current user for the +current session.

+

The optional IN catalog clause sets the role in a catalog as opposed +to a system role.

+
+
+

Limitations#

+

Some connectors do not support role management. +See connector documentation for more details.

+
+
+

See also#

+

CREATE ROLE, DROP ROLE, GRANT ROLES, REVOKE ROLES

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/set-session-authorization.html b/430/sql/set-session-authorization.html new file mode 100644 index 000000000..f9487f217 --- /dev/null +++ b/430/sql/set-session-authorization.html @@ -0,0 +1,936 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SET SESSION AUTHORIZATION — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

SET SESSION AUTHORIZATION#

+
+

Synopsis#

+
SET SESSION AUTHORIZATION username
+
+
+
+
+

Description#

+

Changes the current user of the session. +For the SET SESSION AUTHORIZATION username statement to succeed, +the the original user (that the client connected with) must be able to impersonate the specified user. +User impersonation can be enabled in the system access control.

+
+
+

Examples#

+

In the following example, the original user when the connection to Trino is made is Kevin. +The following sets the session authorization user to John:

+
SET SESSION AUTHORIZATION 'John';
+
+
+

Queries will now execute as John instead of Kevin.

+

All supported syntax to change the session authorization users are shown below.

+

Changing the session authorization with single quotes:

+
SET SESSION AUTHORIZATION 'John';
+
+
+

Changing the session authorization with double quotes:

+
SET SESSION AUTHORIZATION "John";
+
+
+

Changing the session authorization without quotes:

+
SET SESSION AUTHORIZATION John;
+
+
+
+
+

See Also#

+

RESET SESSION AUTHORIZATION

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/set-session.html b/430/sql/set-session.html new file mode 100644 index 000000000..0051c12c4 --- /dev/null +++ b/430/sql/set-session.html @@ -0,0 +1,954 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SET SESSION — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

SET SESSION#

+
+

Synopsis#

+
SET SESSION name = expression
+SET SESSION catalog.name = expression
+
+
+
+
+

Description#

+

Set a session property value or a catalog session property.

+
+
+

Session properties#

+

A session property is a configuration property that +can be temporarily modified by a user for the duration of the current +connection session to the Trino cluster. Many configuration properties have a +corresponding session property that accepts the same values as the config +property.

+

There are two types of session properties:

+
    +
  • System session properties apply to the whole cluster. Most session +properties are system session properties unless specified otherwise.

  • +
  • Catalog session properties are connector-defined session properties that +can be set on a per-catalog basis. These properties must be set separately for +each catalog by including the catalog name as a prefix, such as +catalogname.property_name.

  • +
+

Session properties are tied to the current session, so a user can have multiple +connections to a cluster that each have different values for the same session +properties. Once a session ends, either by disconnecting or creating a new +session, any changes made to session properties during the previous session are +lost.

+
+
+

Examples#

+

The following example sets a system session property to enable optimized hash +generation:

+
SET SESSION optimize_hash_generation = true;
+
+
+

The following example sets the optimize_locality_enabled catalog session +property for an Accumulo catalog named acc01:

+
SET SESSION acc01.optimize_locality_enabled = false;
+
+
+

The example acc01.optimize_locality_enabled catalog session property +does not apply to any other catalog, even if another catalog also uses the +Accumulo connector.

+
+
+

See also#

+

RESET SESSION, SHOW SESSION

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/set-time-zone.html b/430/sql/set-time-zone.html new file mode 100644 index 000000000..11166db04 --- /dev/null +++ b/430/sql/set-time-zone.html @@ -0,0 +1,959 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SET TIME ZONE — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

SET TIME ZONE#

+
+

Synopsis#

+
SET TIME ZONE LOCAL
+SET TIME ZONE expression
+
+
+
+
+

Description#

+

Sets the default time zone for the current session.

+

If the LOCAL option is specified, the time zone for the current session +is set to the initial time zone of the session.

+

If the expression option is specified:

+
    +
  • if the type of the expression is a string, the time zone for the current +session is set to the corresponding region-based time zone ID or the +corresponding zone offset.

  • +
  • if the type of the expression is an interval, the time zone for the +current session is set to the corresponding zone offset relative to UTC. +It must be in the range of [-14,14] hours.

  • +
+
+
+

Examples#

+

Use the default time zone for the current session:

+
SET TIME ZONE LOCAL;
+
+
+

Use a zone offset for specifying the time zone:

+
SET TIME ZONE '-08:00';
+
+
+

Use an interval literal for specifying the time zone:

+
SET TIME ZONE INTERVAL '10' HOUR;
+SET TIME ZONE INTERVAL -'08:00' HOUR TO MINUTE;
+
+
+

Use a region-based time zone identifier for specifying the time zone:

+
SET TIME ZONE 'America/Los_Angeles';
+
+
+

The time zone identifier to be used can be passed as the output of a +function call:

+
SET TIME ZONE concat_ws('/', 'America', 'Los_Angeles');
+
+
+
+
+

Limitations#

+

Setting the default time zone for the session has no effect if +the sql.forced-session-time-zone configuration property is already set.

+
+
+

See also#

+ +
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/show-catalogs.html b/430/sql/show-catalogs.html new file mode 100644 index 000000000..b48745305 --- /dev/null +++ b/430/sql/show-catalogs.html @@ -0,0 +1,905 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SHOW CATALOGS — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

SHOW CATALOGS#

+
+

Synopsis#

+
SHOW CATALOGS [ LIKE pattern ]
+
+
+
+
+

Description#

+

List the available catalogs.

+

Specify a pattern in the optional LIKE clause to +filter the results to the desired subset. For example, the following query +allows you to find catalogs that begin with t:

+
SHOW CATALOGS LIKE 't%'
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/show-columns.html b/430/sql/show-columns.html new file mode 100644 index 000000000..8654be2c3 --- /dev/null +++ b/430/sql/show-columns.html @@ -0,0 +1,922 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SHOW COLUMNS — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

SHOW COLUMNS#

+
+

Synopsis#

+
SHOW COLUMNS FROM table [ LIKE pattern ]
+
+
+
+
+

Description#

+

List the columns in a table along with their data type and other attributes:

+
SHOW COLUMNS FROM nation;
+
+
+
  Column   |     Type     | Extra | Comment
+-----------+--------------+-------+---------
+ nationkey | bigint       |       |
+ name      | varchar(25)  |       |
+ regionkey | bigint       |       |
+ comment   | varchar(152) |       |
+
+
+

Specify a pattern in the optional LIKE clause to +filter the results to the desired subset. For example, the following query +allows you to find columns ending in key:

+
SHOW COLUMNS FROM nation LIKE '%key';
+
+
+
  Column   |     Type     | Extra | Comment
+-----------+--------------+-------+---------
+ nationkey | bigint       |       |
+ regionkey | bigint       |       |
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/show-create-materialized-view.html b/430/sql/show-create-materialized-view.html new file mode 100644 index 000000000..7566eeb07 --- /dev/null +++ b/430/sql/show-create-materialized-view.html @@ -0,0 +1,912 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SHOW CREATE MATERIALIZED VIEW — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+ +
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/show-create-schema.html b/430/sql/show-create-schema.html new file mode 100644 index 000000000..ee0f91057 --- /dev/null +++ b/430/sql/show-create-schema.html @@ -0,0 +1,907 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SHOW CREATE SCHEMA — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

SHOW CREATE SCHEMA#

+
+

Synopsis#

+
SHOW CREATE SCHEMA schema_name
+
+
+
+
+

Description#

+

Show the SQL statement that creates the specified schema.

+
+
+

See also#

+

CREATE SCHEMA

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/show-create-table.html b/430/sql/show-create-table.html new file mode 100644 index 000000000..1656c01c0 --- /dev/null +++ b/430/sql/show-create-table.html @@ -0,0 +1,933 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SHOW CREATE TABLE — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

SHOW CREATE TABLE#

+
+

Synopsis#

+
SHOW CREATE TABLE table_name
+
+
+
+
+

Description#

+

Show the SQL statement that creates the specified table.

+
+
+

Examples#

+

Show the SQL that can be run to create the orders table:

+
SHOW CREATE TABLE sf1.orders;
+
+
+
              Create Table
+-----------------------------------------
+ CREATE TABLE tpch.sf1.orders (
+    orderkey bigint,
+    orderstatus varchar,
+    totalprice double,
+    orderdate varchar
+ )
+ WITH (
+    format = 'ORC',
+    partitioned_by = ARRAY['orderdate']
+ )
+(1 row)
+
+
+
+
+

See also#

+

CREATE TABLE

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/show-create-view.html b/430/sql/show-create-view.html new file mode 100644 index 000000000..3c8118b81 --- /dev/null +++ b/430/sql/show-create-view.html @@ -0,0 +1,907 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SHOW CREATE VIEW — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

SHOW CREATE VIEW#

+
+

Synopsis#

+
SHOW CREATE VIEW view_name
+
+
+
+
+

Description#

+

Show the SQL statement that creates the specified view.

+
+
+

See also#

+

CREATE VIEW

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/show-functions.html b/430/sql/show-functions.html new file mode 100644 index 000000000..7bf25481d --- /dev/null +++ b/430/sql/show-functions.html @@ -0,0 +1,925 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SHOW FUNCTIONS — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

SHOW FUNCTIONS#

+
+

Synopsis#

+
SHOW FUNCTIONS [ FROM schema ] [ LIKE pattern ]
+
+
+
+
+

Description#

+

List functions in schema or all functions in the current session path. +For each function returned, the following information is displayed:

+
    +
  • Function name

  • +
  • Return type

  • +
  • Argument types

  • +
  • Function type

  • +
  • Deterministic

  • +
  • Description

  • +
+

Specify a pattern in the optional LIKE clause to +filter the results to the desired subset. For example, the following query +allows you to find functions beginning with array:

+
SHOW FUNCTIONS LIKE 'array%';
+
+
+

SHOW FUNCTIONS works with built-in functions as well as with custom functions. In the following example, three custom +functions beginning with cf are available:

+
SHOW FUNCTIONS LIKE 'cf%';
+
+     Function      | Return Type | Argument Types | Function Type | Deterministic |               Description
+ ------------------+-------------+----------------+---------------+---------------+-----------------------------------------
+ cf_getgroups      | varchar     |                | scalar        | true          | Returns the current session's groups
+ cf_getprincipal   | varchar     |                | scalar        | true          | Returns the current session's principal
+ cf_getuser        | varchar     |                | scalar        | true          | Returns the current session's user
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/show-grants.html b/430/sql/show-grants.html new file mode 100644 index 000000000..6520e60f8 --- /dev/null +++ b/430/sql/show-grants.html @@ -0,0 +1,937 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SHOW GRANTS — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

SHOW GRANTS#

+
+

Synopsis#

+
SHOW GRANTS [ ON [ TABLE ] table_name ]
+
+
+
+
+

Description#

+

List the grants for the current user on the specified table in the current catalog.

+

If no table name is specified, the command lists the grants for the current user on all the tables in all schemas of the current catalog.

+

The command requires the current catalog to be set.

+
+

Note

+

Ensure that authentication has been enabled before running any of the authorization commands.

+
+
+
+

Examples#

+

List the grants for the current user on table orders:

+
SHOW GRANTS ON TABLE orders;
+
+
+

List the grants for the current user on all the tables in all schemas of the current catalog:

+
SHOW GRANTS;
+
+
+
+
+

Limitations#

+

Some connectors have no support for SHOW GRANTS. +See connector documentation for more details.

+
+
+

See also#

+

GRANT, REVOKE

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/show-role-grants.html b/430/sql/show-role-grants.html new file mode 100644 index 000000000..dc1cba7c8 --- /dev/null +++ b/430/sql/show-role-grants.html @@ -0,0 +1,899 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SHOW ROLE GRANTS — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

SHOW ROLE GRANTS#

+
+

Synopsis#

+
SHOW ROLE GRANTS [ FROM catalog ]
+
+
+
+
+

Description#

+

List non-recursively the system roles or roles in catalog that have been granted to the session user.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/show-roles.html b/430/sql/show-roles.html new file mode 100644 index 000000000..feb37cd40 --- /dev/null +++ b/430/sql/show-roles.html @@ -0,0 +1,900 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SHOW ROLES — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

SHOW ROLES#

+
+

Synopsis#

+
SHOW [CURRENT] ROLES [ FROM catalog ]
+
+
+
+
+

Description#

+

SHOW ROLES lists all the system roles or all the roles in catalog.

+

SHOW CURRENT ROLES lists the enabled system roles or roles in catalog.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/show-schemas.html b/430/sql/show-schemas.html new file mode 100644 index 000000000..ff9f85986 --- /dev/null +++ b/430/sql/show-schemas.html @@ -0,0 +1,905 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SHOW SCHEMAS — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

SHOW SCHEMAS#

+
+

Synopsis#

+
SHOW SCHEMAS [ FROM catalog ] [ LIKE pattern ]
+
+
+
+
+

Description#

+

List the schemas in catalog or in the current catalog.

+

Specify a pattern in the optional LIKE clause to +filter the results to the desired subset. For example, the following query +allows you to find schemas that have 3 as the third character:

+
SHOW SCHEMAS FROM tpch LIKE '__3%'
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/show-session.html b/430/sql/show-session.html new file mode 100644 index 000000000..ab8b2ca20 --- /dev/null +++ b/430/sql/show-session.html @@ -0,0 +1,913 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SHOW SESSION — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

SHOW SESSION#

+
+

Synopsis#

+
SHOW SESSION [ LIKE pattern ]
+
+
+
+
+

Description#

+

List the current session properties.

+

Specify a pattern in the optional LIKE clause to +filter the results to the desired subset. For example, the following query +allows you to find session properties that begin with query:

+
SHOW SESSION LIKE 'query%'
+
+
+
+
+

See also#

+

RESET SESSION, SET SESSION

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/show-stats.html b/430/sql/show-stats.html new file mode 100644 index 000000000..9369a065b --- /dev/null +++ b/430/sql/show-stats.html @@ -0,0 +1,958 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SHOW STATS — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

SHOW STATS#

+
+

Synopsis#

+
SHOW STATS FOR table
+SHOW STATS FOR ( query )
+
+
+
+
+

Description#

+

Returns approximated statistics for the named table or for the results of a +query. Returns NULL for any statistics that are not populated or +unavailable on the data source.

+

Statistics are returned as a row for each column, plus a summary row for +the table (identifiable by a NULL value for column_name). The following +table lists the returned columns and what statistics they represent. Any +additional statistics collected on the data source, other than those listed +here, are not included.

+ + +++++ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Statistics#

Column

Description

Notes

column_name

The name of the column

NULL in the table summary row

data_size

The total size in bytes of all of the values in the column

NULL in the table summary row. Available for columns of +string data types with variable widths.

distinct_values_count

The estimated number of distinct values in the column

NULL in the table summary row

nulls_fractions

The portion of the values in the column that are NULL

NULL in the table summary row.

row_count

The estimated number of rows in the table

NULL in column statistic rows

low_value

The lowest value found in this column

NULL in the table summary row. Available for columns of +DATE, integer, +floating-point, and +fixed-precision data types.

high_value

The highest value found in this column

NULL in the table summary row. Available for columns of +DATE, integer, +floating-point, and +fixed-precision data types.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/show-tables.html b/430/sql/show-tables.html new file mode 100644 index 000000000..386893daf --- /dev/null +++ b/430/sql/show-tables.html @@ -0,0 +1,905 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SHOW TABLES — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

SHOW TABLES#

+
+

Synopsis#

+
SHOW TABLES [ FROM schema ] [ LIKE pattern ]
+
+
+
+
+

Description#

+

List the tables in schema or in the current schema.

+

Specify a pattern in the optional LIKE clause to +filter the results to the desired subset.. For example, the following query +allows you to find tables that begin with p:

+
SHOW TABLES FROM tpch.tiny LIKE 'p%';
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/start-transaction.html b/430/sql/start-transaction.html new file mode 100644 index 000000000..3e96bd9e1 --- /dev/null +++ b/430/sql/start-transaction.html @@ -0,0 +1,926 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + START TRANSACTION — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

START TRANSACTION#

+
+

Synopsis#

+
START TRANSACTION [ mode [, ...] ]
+
+
+

where mode is one of

+
ISOLATION LEVEL { READ UNCOMMITTED | READ COMMITTED | REPEATABLE READ | SERIALIZABLE }
+READ { ONLY | WRITE }
+
+
+
+
+

Description#

+

Start a new transaction for the current session.

+
+
+

Examples#

+
START TRANSACTION;
+START TRANSACTION ISOLATION LEVEL REPEATABLE READ;
+START TRANSACTION READ WRITE;
+START TRANSACTION ISOLATION LEVEL READ COMMITTED, READ ONLY;
+START TRANSACTION READ WRITE, ISOLATION LEVEL SERIALIZABLE;
+
+
+
+
+

See also#

+

COMMIT, ROLLBACK

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/truncate.html b/430/sql/truncate.html new file mode 100644 index 000000000..fe9d0ebb5 --- /dev/null +++ b/430/sql/truncate.html @@ -0,0 +1,910 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + TRUNCATE — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

TRUNCATE#

+
+

Synopsis#

+
TRUNCATE TABLE table_name
+
+
+
+
+

Description#

+

Delete all rows from a table.

+
+
+

Examples#

+

Truncate the table orders:

+
TRUNCATE TABLE orders;
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/update.html b/430/sql/update.html new file mode 100644 index 000000000..1d27df481 --- /dev/null +++ b/430/sql/update.html @@ -0,0 +1,951 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + UPDATE — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

UPDATE#

+
+

Synopsis#

+
UPDATE table_name SET [ ( column = expression [, ... ] ) ] [ WHERE condition ]
+
+
+
+
+

Description#

+

Update selected columns values in existing rows in a table.

+

The columns named in the column = expression assignments will be updated +for all rows that match the WHERE condition. The values of all column update +expressions for a matching row are evaluated before any column value is changed. +When the type of the expression and the type of the column differ, the usual implicit +CASTs, such as widening numeric fields, are applied to the UPDATE expression values.

+
+
+

Examples#

+

Update the status of all purchases that haven’t been assigned a ship date:

+
UPDATE
+  purchases
+SET
+  status = 'OVERDUE'
+WHERE
+  ship_date IS NULL;
+
+
+

Update the account manager and account assign date for all customers:

+
UPDATE
+  customers
+SET
+  account_manager = 'John Henry',
+  assign_date = now();
+
+
+

Update the manager to be the name of the employee who matches the manager ID:

+
UPDATE
+  new_hires
+SET
+  manager = (
+    SELECT
+      e.name
+    FROM
+      employees e
+    WHERE
+      e.employee_id = new_hires.manager_id
+  );
+
+
+
+
+

Limitations#

+

Some connectors have limited or no support for UPDATE. +See connector documentation for more details.

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/use.html b/430/sql/use.html new file mode 100644 index 000000000..796e2e00b --- /dev/null +++ b/430/sql/use.html @@ -0,0 +1,913 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + USE — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

USE#

+
+

Synopsis#

+
USE catalog.schema
+USE schema
+
+
+
+
+

Description#

+

Update the session to use the specified catalog and schema. If a +catalog is not specified, the schema is resolved relative to the +current catalog.

+
+
+

Examples#

+
USE hive.finance;
+USE information_schema;
+
+
+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/430/sql/values.html b/430/sql/values.html new file mode 100644 index 000000000..2a2ff88cf --- /dev/null +++ b/430/sql/values.html @@ -0,0 +1,952 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + VALUES — Trino 430 Documentation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Skip to content + +
+ +
+ + +
+ + + +
+
+ +
+
+
+ +
+
+
+
+
+
+ + +
+
+
+ +
+
+ +
+

VALUES#

+
+

Synopsis#

+
VALUES row [, ...]
+
+
+

where row is a single expression or

+
( column_expression [, ...] )
+
+
+
+
+

Description#

+

Defines a literal inline table.

+

VALUES can be used anywhere a query can be used (e.g., the FROM clause +of a SELECT, an INSERT, or even at the top level). VALUES creates +an anonymous table without column names, but the table and columns can be named +using an AS clause with column aliases.

+
+
+

Examples#

+

Return a table with one column and three rows:

+
VALUES 1, 2, 3
+
+
+

Return a table with two columns and three rows:

+
VALUES
+    (1, 'a'),
+    (2, 'b'),
+    (3, 'c')
+
+
+

Return table with column id and name:

+
SELECT * FROM (
+    VALUES
+        (1, 'a'),
+        (2, 'b'),
+        (3, 'c')
+) AS t (id, name)
+
+
+

Create a new table with column id and name:

+
CREATE TABLE example AS
+SELECT * FROM (
+    VALUES
+        (1, 'a'),
+        (2, 'b'),
+        (3, 'c')
+) AS t (id, name)
+
+
+
+
+

See also#

+

INSERT, SELECT

+
+
+ + +
+
+
+
+
+ + + + + \ No newline at end of file diff --git a/_redirects b/_redirects index abf18bd60..38410fdab 100644 --- a/_redirects +++ b/_redirects @@ -1 +1 @@ -/current/* /429/:splat 200 +/current/* /430/:splat 200 diff --git a/current b/current index 00c22e598..a71e551f3 120000 --- a/current +++ b/current @@ -1 +1 @@ -429 \ No newline at end of file +430 \ No newline at end of file