Hive Integration

Apache Hive has established itself as a focal point of the data warehousing ecosystem. It serves as not only a SQL engine for big data analytics and ETL, but also a data management platform, where data is discovered, defined, and evolved.

Flink offers a two-fold integration with Hive.

The first is to leverage Hive’s Metastore as a persistent catalog with Flink’s HiveCatalog for storing Flink specific metadata across sessions. For example, users can store their Kafka or ElasticSearch tables in Hive Metastore by using HiveCatalog, and reuse them later on in SQL queries.

The second is to offer Flink as an alternative engine for reading and writing Hive tables.

The HiveCatalog is designed to be “out of the box” compatible with existing Hive installations. You do not need to modify your existing Hive Metastore or change the data placement or partitioning of your tables.

Supported Hive Versions

Flink supports the following Hive versions.

  • 1.0
    • 1.0.0
    • 1.0.1
  • 1.1
    • 1.1.0
    • 1.1.1
  • 1.2
    • 1.2.0
    • 1.2.1
    • 1.2.2
  • 2.0
    • 2.0.0
    • 2.0.1
  • 2.1
    • 2.1.0
    • 2.1.1
  • 2.2
    • 2.2.0
  • 2.3
    • 2.3.0
    • 2.3.1
    • 2.3.2
    • 2.3.3
    • 2.3.4
    • 2.3.5
    • 2.3.6
  • 3.1
    • 3.1.0
    • 3.1.1
    • 3.1.2

Please note Hive itself have different features available for different versions, and these issues are not caused by Flink:

  • Hive built-in functions are supported in 1.2.0 and later.
  • Column constraints, i.e. PRIMARY KEY and NOT NULL, are supported in 3.1.0 and later.
  • Altering table statistics is supported in 1.2.0 and later.
  • DATE column statistics are supported in 1.2.0 and later.
  • Writing to ORC tables is not supported in 2.0.x.

Dependencies

To integrate with Hive, you need to add some extra dependencies to the /lib/ directory in Flink distribution to make the integration work in Table API program or SQL in SQL Client. Alternatively, you can put these dependencies in a dedicated folder, and add them to classpath with the -C or -l option for Table API program or SQL Client respectively.

Apache Hive is built on Hadoop, so you need Hadoop dependency first, please refer to Providing Hadoop classes.

There are two ways to add Hive dependencies. First is to use Flink’s bundled Hive jars. You can choose a bundled Hive jar according to the version of the metastore you use. Second is to add each of the required jars separately. The second way can be useful if the Hive version you’re using is not listed here.

Using bundled hive jar

The following tables list all available bundled hive jars. You can pick one to the /lib/ directory in Flink distribution.

Metastore versionMaven dependencySQL Client JAR
1.0.0 - 1.2.2flink-sql-connector-hive-1.2.2Download
2.0.0 - 2.2.0flink-sql-connector-hive-2.2.0Download
2.3.0 - 2.3.6flink-sql-connector-hive-2.3.6Download
3.0.0 - 3.1.2flink-sql-connector-hive-3.1.2Download

User defined dependencies

Please find the required dependencies for different Hive major versions below.

  1. /flink-1.11.0
  2. /lib
  3. // Flink's Hive connector.Contains flink-hadoop-compatibility and flink-orc jars
  4. flink-connector-hive_2.11-1.11.0.jar
  5. // Hive dependencies
  6. hive-exec-2.3.4.jar
  1. /flink-1.11.0
  2. /lib
  3. // Flink's Hive connector
  4. flink-connector-hive_2.11-1.11.0.jar
  5. // Hive dependencies
  6. hive-metastore-1.0.0.jar
  7. hive-exec-1.0.0.jar
  8. libfb303-0.9.0.jar // libfb303 is not packed into hive-exec in some versions, need to add it separately
  9. // Orc dependencies -- required by the ORC vectorized optimizations
  10. orc-core-1.4.3-nohive.jar
  11. aircompressor-0.8.jar // transitive dependency of orc-core
  1. /flink-1.11.0
  2. /lib
  3. // Flink's Hive connector
  4. flink-connector-hive_2.11-1.11.0.jar
  5. // Hive dependencies
  6. hive-metastore-1.1.0.jar
  7. hive-exec-1.1.0.jar
  8. libfb303-0.9.2.jar // libfb303 is not packed into hive-exec in some versions, need to add it separately
  9. // Orc dependencies -- required by the ORC vectorized optimizations
  10. orc-core-1.4.3-nohive.jar
  11. aircompressor-0.8.jar // transitive dependency of orc-core
  1. /flink-1.11.0
  2. /lib
  3. // Flink's Hive connector
  4. flink-connector-hive_2.11-1.11.0.jar
  5. // Hive dependencies
  6. hive-metastore-1.2.1.jar
  7. hive-exec-1.2.1.jar
  8. libfb303-0.9.2.jar // libfb303 is not packed into hive-exec in some versions, need to add it separately
  9. // Orc dependencies -- required by the ORC vectorized optimizations
  10. orc-core-1.4.3-nohive.jar
  11. aircompressor-0.8.jar // transitive dependency of orc-core
  1. /flink-1.11.0
  2. /lib
  3. // Flink's Hive connector
  4. flink-connector-hive_2.11-1.11.0.jar
  5. // Hive dependencies
  6. hive-exec-2.0.0.jar
  1. /flink-1.11.0
  2. /lib
  3. // Flink's Hive connector
  4. flink-connector-hive_2.11-1.11.0.jar
  5. // Hive dependencies
  6. hive-exec-2.1.0.jar
  1. /flink-1.11.0
  2. /lib
  3. // Flink's Hive connector
  4. flink-connector-hive_2.11-1.11.0.jar
  5. // Hive dependencies
  6. hive-exec-2.2.0.jar
  7. // Orc dependencies -- required by the ORC vectorized optimizations
  8. orc-core-1.4.3.jar
  9. aircompressor-0.8.jar // transitive dependency of orc-core
  1. /flink-1.11.0
  2. /lib
  3. // Flink's Hive connector
  4. flink-connector-hive_2.11-1.11.0.jar
  5. // Hive dependencies
  6. hive-exec-3.1.0.jar
  7. libfb303-0.9.3.jar // libfb303 is not packed into hive-exec in some versions, need to add it separately

If you use the hive version of HDP or CDH, you need to refer to the dependency in the previous section and select a similar version.

And you need to specify selected and supported “hive-version” in yaml, HiveCatalog and HiveModule.

Program maven

If you are building your own program, you need the following dependencies in your mvn file. It’s recommended not to include these dependencies in the resulting jar file. You’re supposed to add dependencies as stated above at runtime.

  1. <!-- Flink Dependency -->
  2. <dependency>
  3. <groupId>org.apache.flink</groupId>
  4. <artifactId>flink-connector-hive_2.11</artifactId>
  5. <version>1.11.0</version>
  6. <scope>provided</scope>
  7. </dependency>
  8. <dependency>
  9. <groupId>org.apache.flink</groupId>
  10. <artifactId>flink-table-api-java-bridge_2.11</artifactId>
  11. <version>1.11.0</version>
  12. <scope>provided</scope>
  13. </dependency>
  14. <!-- Hive Dependency -->
  15. <dependency>
  16. <groupId>org.apache.hive</groupId>
  17. <artifactId>hive-exec</artifactId>
  18. <version>${hive.version}</version>
  19. <scope>provided</scope>
  20. </dependency>

Connecting To Hive

Connect to an existing Hive installation using the catalog interface and HiveCatalog through the table environment or YAML configuration.

If the hive-conf/hive-site.xml file is stored in remote storage system, users should download the hive configuration file to their local environment first.

Please note while HiveCatalog doesn’t require a particular planner, reading/writing Hive tables only works with blink planner. Therefore it’s highly recommended that you use blink planner when connecting to your Hive warehouse.

Take Hive version 2.3.4 for example:

  1. EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build();
  2. TableEnvironment tableEnv = TableEnvironment.create(settings);
  3. String name = "myhive";
  4. String defaultDatabase = "mydatabase";
  5. String hiveConfDir = "/opt/hive-conf"; // a local path
  6. String version = "2.3.4";
  7. HiveCatalog hive = new HiveCatalog(name, defaultDatabase, hiveConfDir, version);
  8. tableEnv.registerCatalog("myhive", hive);
  9. // set the HiveCatalog as the current catalog of the session
  10. tableEnv.useCatalog("myhive");
  1. val settings = EnvironmentSettings.newInstance().inBatchMode().build()
  2. val tableEnv = TableEnvironment.create(settings)
  3. val name = "myhive"
  4. val defaultDatabase = "mydatabase"
  5. val hiveConfDir = "/opt/hive-conf" // a local path
  6. val version = "2.3.4"
  7. val hive = new HiveCatalog(name, defaultDatabase, hiveConfDir, version)
  8. tableEnv.registerCatalog("myhive", hive)
  9. // set the HiveCatalog as the current catalog of the session
  10. tableEnv.useCatalog("myhive")
  1. execution:
  2. planner: blink
  3. ...
  4. current-catalog: myhive # set the HiveCatalog as the current catalog of the session
  5. current-database: mydatabase
  6. catalogs:
  7. - name: myhive
  8. type: hive
  9. hive-conf-dir: /opt/hive-conf

DDL

It’s recommended to use Hive dialect to execute DDLs to create Hive tables, views, partitions, functions within Flink.

DML

Flink supports DML writing to Hive tables. Please refer to details in Reading & Writing Hive Tables