You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iceberg.apache.org by pv...@apache.org on 2022/06/30 09:34:35 UTC

[iceberg] branch master updated: Docs: Update Hive doc page with the 4.0.0-alpha-1 features (#5161)

This is an automated email from the ASF dual-hosted git repository.

pvary pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iceberg.git


The following commit(s) were added to refs/heads/master by this push:
     new dd5b35067b Docs: Update Hive doc page with the 4.0.0-alpha-1 features (#5161)
dd5b35067b is described below

commit dd5b35067baac5eb004141d51d32251d2f8e808c
Author: Samuel Redai <43...@users.noreply.github.com>
AuthorDate: Thu Jun 30 05:34:29 2022 -0400

    Docs: Update Hive doc page with the 4.0.0-alpha-1 features (#5161)
---
 docs/hive.md | 379 +++++++++++++++++++++++++++++++++++++++++++----------------
 1 file changed, 277 insertions(+), 102 deletions(-)

diff --git a/docs/hive.md b/docs/hive.md
index d9cc3b041b..718797310c 100644
--- a/docs/hive.md
+++ b/docs/hive.md
@@ -24,20 +24,63 @@ menu: main
 # Hive
 
 Iceberg supports reading and writing Iceberg tables through [Hive](https://hive.apache.org) by using
-a [StorageHandler](https://cwiki.apache.org/confluence/display/Hive/StorageHandlers). Here is the current compatibility
-matrix for Iceberg Hive support:
+a [StorageHandler](https://cwiki.apache.org/confluence/display/Hive/StorageHandlers).
 
-| Feature                  | Hive 2.x               | Hive 3.1.2             |
-| ------------------------ | ---------------------- | ---------------------- |
-| CREATE EXTERNAL TABLE    | ✔️                     | ✔️                     |
-| CREATE TABLE             | ✔️                     | ✔️                     |
-| DROP TABLE               | ✔️                     | ✔️                     |
-| SELECT                   | ✔️ (MapReduce and Tez) | ✔️ (MapReduce and Tez) |
-| INSERT INTO              | ✔️ (MapReduce only)️    | ✔️ (MapReduce only)    |
+## Feature support
+Iceberg compatibility with Hive 2.x and Hive 3.1.2/3 supports the following features:
+
+* Creating a table
+* Dropping a table
+* Reading a table
+* Inserting into a table (INSERT INTO)
+
+{{< hint warning >}}
+DML operations work only with MapReduce execution engine.
+{{< /hint >}}
+
+With Hive version 4.0.0-alpha-1 and above,
+the Iceberg integration when using HiveCatalog supports the following additional features:
+
+* Creating an Iceberg identity-partitioned table
+* Creating an Iceberg table with any partition spec, including the various transforms supported by Iceberg
+* Creating a table from an existing table (CTAS table)
+* Altering a table while keeping Iceberg and Hive schemas in sync
+* Altering the partition schema (updating columns)
+* Altering the partition schema by specifying partition transforms
+* Truncating a table
+* Migrating tables in Avro, Parquet, or ORC (Non-ACID) format to Iceberg
+* Reading the schema of a table
+* Querying Iceberg metadata tables
+* Time travel applications
+* Inserting into a table (INSERT INTO)
+* Inserting data overwriting existing data (INSERT OVERWRITE)
+
+{{< hint warning >}}
+DML operations work only with Tez execution engine.
+{{< /hint >}}
 
 ## Enabling Iceberg support in Hive
 
-### Loading runtime jar
+### Hive 4.0.0-alpha-1
+
+Hive 4.0.0-alpha-1 comes with the Iceberg 0.13.1 included. No additional downloads or jars are needed.
+
+### Hive 2.3.x, Hive 3.1.x
+
+You need to do the following steps:
+* Loading runtime jar
+* Enabling support
+Collaborator
+@samredai samredai 26 days ago
+This could just be one line right?
+
+In order to use Hive 2.3.x or Hive 3.1.x, you must load the Iceberg-Hive runtime jar and enable Iceberg support, either globally or for an individual table using a table property.
+
+and then jump into the sections below that provide more details.
+
+@samredai	Reply...
+
+#### Loading runtime jar
 
 To enable Iceberg support in Hive, the `HiveIcebergStorageHandler` and supporting classes need to be made available on
 Hive's classpath. These are provided by the `iceberg-hive-runtime` jar file. For example, if using the Hive shell, this
@@ -50,7 +93,7 @@ add jar /path/to/iceberg-hive-runtime.jar;
 There are many others ways to achieve this including adding the jar file to Hive's auxiliary classpath so it is
 available by default. Please refer to Hive's documentation for more information.
 
-### Enabling support
+#### Enabling support
 
 If the Iceberg storage handler is not in Hive's classpath, then Hive cannot load or update the metadata for an Iceberg
 table when the storage handler is set. To avoid the appearance of broken tables in Hive, Iceberg will not add the
@@ -58,7 +101,7 @@ storage handler to a table unless Hive support is enabled. The storage handler i
 time Hive engine support for the table is updated, i.e. turned on or off in the table properties. There are two ways to
 enable Hive support: globally in Hadoop Configuration and per-table using a table property.
 
-#### Hadoop configuration
+##### Hadoop configuration
 
 To enable Hive support globally for an application, set `iceberg.engine.hive.enabled=true` in its Hadoop configuration.
 For example, setting this in the `hive-site.xml` loaded by Spark will enable the storage handler for all tables created
@@ -67,7 +110,7 @@ by Spark.
 {{< hint danger >}} Starting with Apache Iceberg `0.11.0`, when using Hive with Tez you also have to disable
 vectorization (`hive.vectorized.execution.enabled=false`). {{< /hint >}}
 
-#### Table property configuration
+##### Table property configuration
 
 Alternatively, the property `engine.hive.enabled` can be set to `true` and added to the table properties when creating
 the Iceberg table. Here is an example of doing it programmatically:
@@ -81,10 +124,10 @@ Catalog catalog=...;
 
 The table level configuration overrides the global Hadoop configuration.
 
-#### Hive on Tez configuration
+##### Hive on Tez configuration
+
+To use the Tez engine on Hive `3.1.2` or later, Tez needs to be upgraded to >= `0.10.1` which contains a necessary fix [TEZ-4248](https://issues.apache.org/jira/browse/TEZ-4248).
 
-To use the Tez engine on Hive `3.1.2` or later, Tez needs to be upgraded to >= `0.10.1` which contains a necessary
-fix [Tez-4248](https://issues.apache.org/jira/browse/TEZ-4248).
 
 To use the Tez engine on Hive `2.3.x`, you will need to manually build Tez from the `branch-0.9` branch due to a
 backwards incompatibility issue with Tez `0.10.1`.
@@ -154,7 +197,108 @@ SET iceberg.catalog.glue.lock.table=myGlueLockTable;
 
 ## DDL Commands
 
-### CREATE EXTERNAL TABLE
+Not all the features below are supported with Hive 2.3.x and Hive 3.1.x. Please refer to the
+[Feature support](#feature-support) paragraph for further details.
+
+One generally applicable difference is that Hive 4.0.0-alpha-1 provides the possibility to use
+`STORED BY ICEBERG` instead of the old `STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler'`
+
+### CREATE TABLE
+
+#### Non partitioned tables
+
+The Hive `CREATE EXTERNAL TABLE` command creates an Iceberg table when you specify the storage handler as follows:
+
+```sql
+CREATE EXTERNAL TABLE x (i int) STORED BY ICEBERG;
+```
+
+If you want to create external tables using CREATE TABLE, configure the MetaStoreMetadataTransformer on the cluster,
+and `CREATE TABLE` commands are transformed to create external tables. For example:
+
+```sql
+CREATE TABLE x (i int) STORED BY ICEBERG;
+```
+
+You can specify the default file format (Avro, Parquet, ORC) at the time of the table creation.
+The default is Parquet:
+
+```sql
+CREATE TABLE x (i int) STORED BY ICEBERG STORED AS ORC;
+```
+
+#### Partitioned tables
+You can create Iceberg partitioned tables using a command familiar to those who create non-Iceberg tables:
+
+```sql
+CREATE TABLE x (i int) PARTITIONED BY (j int) STORED BY ICEBERG;
+```
+
+{{< hint info >}}
+The resulting table does not create partitions in HMS, but instead, converts partition data into Iceberg identity partitions.
+{{< /hint >}}
+
+Use the DESCRIBE command to get information about the Iceberg identity partitions:
+
+```sql
+DESCRIBE x;
+```
+The result is:
+
+| col_name                           | data_type      | comment
+| ---------------------------------- | -------------- | -------
+| i                                  | int            |
+| j                                  | int            |
+|                                    | NULL           | NULL
+| # Partition Transform Information  | NULL           | NULL
+| # col_name                         | transform_type | NULL
+| j                                  | IDENTITY       | NULL
+
+You can create Iceberg partitions using the following Iceberg partition specification syntax
+(supported only in Hive 4.0.0-alpha-1):
+
+```sql
+CREATE TABLE x (i int, ts timestamp) PARTITIONED BY SPEC (month(ts), bucket(2, i)) STORED AS ICEBERG;
+DESCRIBE x;
+```
+The result is:
+
+| col_name                           | data_type      | comment
+| ---------------------------------- | -------------- | -------
+| i                                  | int            |
+| ts                                 | timestamp      |
+|                                    | NULL           | NULL
+| # Partition Transform Information  | NULL           | NULL
+| # col_name                         | transform_type | NULL
+| ts                                 | MONTH          | NULL
+| i                                  | BUCKET\[2\]    | NULL
+
+The supported transformations for Hive are the same as for Spark:
+* years(ts): partition by year
+* months(ts): partition by month
+* days(ts) or date(ts): equivalent to dateint partitioning
+* hours(ts) or date_hour(ts): equivalent to dateint and hour partitioning
+* bucket(N, col): partition by hashed value mod N buckets
+* truncate(L, col): partition by value truncated to L
+     - Strings are truncated to the given length
+     - Integers and longs truncate to bins: truncate(10, i) produces partitions 0, 10, 20, 30,
+
+{{< hint info >}}
+The resulting table does not create partitions in HMS, but instead, converts partition data into Iceberg partitions.
+{{< /hint >}}
+
+### CREATE TABLE AS SELECT
+
+`CREATE TABLE AS SELECT` operation resembles the native Hive operation with a single important difference.
+The Iceberg table and the corresponding Hive table are created at the beginning of the query execution.
+The data is inserted / committed when the query finishes. So for a transient period the table already exists but contains no data.
+
+```sql
+CREATE TABLE target PARTITIONED BY SPEC (year(year_field), identity_field) STORED BY ICEBERG AS
+    SELECT * FROM source;
+```
+
+### CREATE EXTERNAL TABLE overlaying an existing Iceberg table
 
 The `CREATE EXTERNAL TABLE` command is used to overlay a Hive table "on top of" an existing Iceberg table. Iceberg
 tables are created using either a [`Catalog`](../../../javadoc/{{% icebergVersion
@@ -201,29 +345,7 @@ LOCATION 'hdfs://some_bucket/some_path/table_a'
 TBLPROPERTIES ('iceberg.catalog'='location_based_table');
 ```
 
-### CREATE TABLE
-
-Hive also supports directly creating a new Iceberg table through `CREATE TABLE` statement. For example:
-
-```sql
-CREATE TABLE database_a.table_a
-(
-    id   bigint,
-    name string
-) PARTITIONED BY (
-  dept string
-) STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler';
-```
-
-{{< hint info >}} to Hive, the table appears to be unpartitioned although the underlying Iceberg table is partitioned.
-{{< /hint >}}
-
-{{< hint info >}} Due to the limitation of Hive `PARTITIONED BY` syntax, if you use Hive `CREATE TABLE`, currently you
-can only partition by columns, which is translated to Iceberg identity partition transform. You cannot partition by
-other Iceberg partition transforms such as `days(timestamp)`. To create table with all partition transforms, you need to
-create the table with other engines like Spark or Flink. {{< /hint >}}
-
-#### Custom catalog table
+#### CREATE TABLE overlaying an existing Iceberg table
 
 You can also create a new table that is managed by a custom catalog. For example, the following code creates a table in
 a custom Hadoop catalog:
@@ -249,78 +371,101 @@ table. This means technically you can omit the `EXTERNAL` keyword when creating
 recommended** because creating managed overlay tables could pose a risk to the shared data files in case of accidental
 drop table commands from the Hive side, which would unintentionally remove all the data in the table. {{< /hint >}}
 
-### DROP TABLE
-
-Tables can be dropped using the `DROP TABLE` command:
+### ALTER TABLE
+#### Table properties
+For HiveCatalog tables the Iceberg table properties and the Hive table properties stored in HMS are kept in sync.
 
+{{< hint info >}}
+IMPORTANT: This feature is not available for other Catalog implementations.
+{{< /hint >}}
 ```sql
-DROP TABLE [IF EXISTS] table_name [PURGE];
+ALTER TABLE t SET TBLPROPERTIES('...'='...');
 ```
 
-You can configure purge behavior through global Hadoop configuration or Hive metastore table properties:
-
-| Config key                  | Default                    | Description                                                     |
-| ----------------------------| ---------------------------| --------------------------------------------------------------- |
-| external.table.purge        | true                       | if all data and metadata should be purged in a table by default |
-
-Each Iceberg table's default purge behavior can also be configured through Iceberg table properties:
+#### Schema evolution
+The Hive table schema is kept in sync with the Iceberg table. If an outside source (Impala/Spark/Java API/etc)
+changes the schema, the Hive table immediately reflects the changes. You alter the table schema using Hive commands:
 
-| Property                    | Default                    | Description                                                       |
-| ----------------------------| ---------------------------| ----------------------------------------------------------------- |
-| gc.enabled                  | true                       | if all data and metadata should be purged in the table by default |
-
-When changing `gc.enabled` on the Iceberg table via `UpdateProperties`, `external.table.purge` is also updated on HMS
-table accordingly. When setting `external.table.purge` as a table prop during Hive `CREATE TABLE`, `gc.enabled` is
-pushed down accordingly to the Iceberg table properties. This makes sure that the 2 properties are always consistent at
-table level between Hive and Iceberg.
-
-{{< hint danger >}} Changing `external.table.purge` via Hive `ALTER TABLE SET TBLPROPERTIES` does not
-update `gc.enabled` on the Iceberg table. This is a limitation on Hive 3.1.2 because the `HiveMetaHook` doesn't have all
-the hooks for alter tables yet. {{< /hint >}}
-
-## Querying with SQL
-
-Here are the features highlights for Iceberg Hive read support:
-
-1. **Predicate pushdown**: Pushdown of the Hive SQL `WHERE` clause has been implemented so that these filters are used
-   at the Iceberg `TableScan` level as well as by the Parquet and ORC Readers.
-2. **Column projection**: Columns from the Hive SQL `SELECT` clause are projected down to the Iceberg readers to reduce
-   the number of columns read.
-3. **Hive query engines**: Both the MapReduce and Tez query execution engines are supported.
-
-### Configurations
+* Add a column
+```sql
+ALTER TABLE orders ADD COLUMNS (nickname string);
+```
+* Rename a column
+```sql
+ALTER TABLE orders CHANGE COLUMN item fruit string;
+```
+* Reorder columns
+```sql
+ALTER TABLE orders CHANGE COLUMN quantity quantity int AFTER price;
+```
+* Change a column type - only if the Iceberg defined the column type change as safe
+```sql
+ALTER TABLE orders CHANGE COLUMN price price long;
+```
+* Drop column by using REPLACE COLUMN to remove the old column
+```sql
+ALTER TABLE orders REPLACE COLUMNS (remaining string);
+```
+{{< hint info >}}
+Note, that dropping columns is only thing REPLACE COLUMNS can be used for
+i.e. if columns are specified out-of-order an error will be thrown signalling this limitation.
+{{< /hint >}}
 
-Here are the Hadoop configurations that one can adjust for the Hive reader:
+#### Partition evolution
+You change the partitioning schema using the following commands:
+* Change the partitioning schema to new identity partitions:
+```sql
+ALTER TABLE default.customers SET PARTITION SPEC (last_name);
+```
+* Alternatively, provide a partition specification:
+```sql
+ALTER TABLE order SET PARTITION SPEC (month(ts));
+```
+#### Table migration
+You can migrate Avro / Parquet / ORC external tables to Iceberg tables using the following command:
+```sql
+ALTER TABLE t SET TBLPROPERTIES ('storage_handler'='org.apache.iceberg.mr.hive.HiveIcebergStorageHandler');
+```
+During the migration the data files are not changed, only the appropriate Iceberg metadata files are created.
+After the migration, handle the table as a normal Iceberg table.
 
-| Config key                   | Default                 | Description                                            |
-| ---------------------------- | ----------------------- | ------------------------------------------------------ |
-| iceberg.mr.reuse.containers  | false                   | if Avro reader should reuse containers                 |
-| iceberg.mr.case.sensitive    | true                    | if the query is case-sensitive                         |
+### TRUNCATE TABLE
+The following command truncates the Iceberg table:
+```sql
+TRUNCATE TABLE t;
+```
+Using a partition specification is not allowed.
 
-### SELECT
+### DROP TABLE
 
-You should now be able to issue Hive SQL `SELECT` queries and see the results returned from the underlying Iceberg
-table, for example:
+Tables can be dropped using the `DROP TABLE` command:
 
 ```sql
-SELECT *
-from table_a;
+DROP TABLE [IF EXISTS] table_name [PURGE];
 ```
 
-## Writing with SQL
+## DML Commands
 
-### Configurations
+### SELECT
+Select statements work the same on Iceberg tables in Hive. You will see the Iceberg benefits over Hive in compilation and execution:
+* **No file system listings** - especially important on blob stores, like S3
+* **No partition listing from** the Metastore
+* **Advanced partition filtering** - the partition keys are not needed in the queries when they could be calculated
+* Could handle **higher number of partitions** than normal Hive tables
 
-Here are the Hadoop configurations that one can adjust for the Hive writer:
+Here are the features highlights for Iceberg Hive read support:
+1. **Predicate pushdown**: Pushdown of the Hive SQL `WHERE` clause has been implemented so that these filters are used at the Iceberg `TableScan` level as well as by the Parquet and ORC Readers.
+2. **Column projection**: Columns from the Hive SQL `SELECT` clause are projected down to the Iceberg readers to reduce the number of columns read.
+3. **Hive query engines**:
+   - With Hive 2.3.x, 3.1.x both the MapReduce and Tez query execution engines are supported.
+   - With Hive 4.0.0-alpha-1 Tez query execution engine is supported.
 
-| Config key                                        | Default                                  | Description                                            |
-| ------------------------------------------------- | ---------------------------------------- | ------------------------------------------------------ |
-| iceberg.mr.commit.table.thread.pool.size          | 10                                       | the number of threads of a shared thread pool to execute parallel commits for output tables |
-| iceberg.mr.commit.file.thread.pool.size           | 10                                       | the number of threads of a shared thread pool to execute parallel commits for files in each output table |
+Some of the advanced / little used optimizations are not yet implemented for Iceberg tables, so you should check your individual queries.
+Also currently the statistics stored in the MetaStore are used for query planning. This is something we are planning to improve in the future.
 
 ### INSERT INTO
 
-Hive supports the standard single-table `INSERT INTO` operation:
+Hive supports the standard single-table INSERT INTO operation:
 
 ```sql
 INSERT INTO table_a
@@ -329,20 +474,50 @@ INSERT INTO table_a
 SELECT...;
 ```
 
-Multi-table insert is also supported, but it will not be atomic and are committed one table at a time. Partial changes
-will be visible during the commit process and failures can leave partial changes committed. Changes within a single
-table will remain atomic.
+Multi-table insert is also supported, but it will not be atomic. Commits occur one table at a time.
+Partial changes will be visible during the commit process and failures can leave partial changes committed.
+Changes within a single table will remain atomic.
 
 Here is an example of inserting into multiple tables at once in Hive SQL:
 
 ```sql
 FROM customers
-    INSERT INTO target1
-SELECT customer_id,
-       first_name
-    INSERT
-INTO target2
-SELECT last_name, customer_id;
+   INSERT INTO target1 SELECT customer_id, first_name
+   INSERT INTO target2 SELECT last_name, customer_id;
+```
+
+### INSERT OVERWRITE
+INSERT OVERWRITE can replace data in the table with the result of a query. Overwrites are atomic operations for Iceberg tables.
+For nonpartitioned tables the content of the table is always removed. For partitioned tables the partitions
+that have rows produced by the SELECT query will be replaced.
+```sql
+INSERT OVERWRITE TABLE target SELECT * FROM source;
+```
+
+### QUERYING METADATA TABLES
+Hive supports querying of the Iceberg Metadata tables. The tables could be used as normal
+Hive tables, so it is possible to use projections / joins / filters / etc.
+To reference a metadata table the full name of the table should be used, like:
+<DB_NAME>.<TABLE_NAME>.<METADATA_TABLE_NAME>.
+
+Currently the following metadata tables are available in Hive:
+* files
+* entries
+* snapshots
+* manifests
+* partitions
+
+```sql
+SELECT * FROM default.table_a.files;
+```
+
+### TIMETRAVEL
+Hive supports snapshot id based and time base timetravel queries.
+For these views it is possible to use projections / joins / filters / etc.
+The function is available with the following syntax:
+```sql
+SELECT * FROM table_a FOR SYSTEM_TIME AS OF '2021-08-09 10:35:57';
+SELECT * FROM table_a FOR SYSTEM_VERSION AS OF 1234567;
 ```
 
 ## Type compatibility