This documentation is for an out-of-date version of Apache Flink. We recommend you use the latest stable version.

Hive Integration Beta

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 for storing Flink specific metadata across sessions. The second is to offer Flink as an alternative engine for reading and writing Hive tables.

The hive catalog 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 Version’s

Flink supports Hive 2.3.4 and 1.2.1 and relies on Hive’s compatibility guarantee’s for other minor versions.

If you use a different minor Hive version such as 1.2.2 or 2.3.1, it should also be ok to choose the closest version 1.2.1 (for 1.2.2) or 2.3.4 (for 2.3.1) to workaround. For example, you want to use Flink to integrate 2.3.1 hive version in sql client, just set the hive-version to 2.3.4 in YAML config. Similarly pass the version string when creating HiveCatalog instance via Table API.

Users are welcome to try out different versions with this workaround. Since only 2.3.4 and 1.2.1 have been tested, there might be unexpected issues. We will test and support more versions in future releases.

Depedencies

To integrate with Hive, users need the following dependencies in their project.

<dependency>
  <groupId>org.apache.flink</groupId>
  <artifactId>flink-connector-hive_2.11</artifactId>
  <version>1.9.0</version>
  <scope>provided</scope>
</dependency>

<!-- Hadoop Dependencies -->

<dependency>
  <groupId>org.apache.flink</groupId>
  <artifactId>flink-hadoop-compatibility_2.11</artifactId>
  <version>1.9.0</version>
  <scope>provided</scope>
</dependency>

<!-- Hive 2.3.4 is built with Hadoop 2.7.2. We pick 2.7.5 which flink-shaded-hadoop is pre-built with, but users can pick their own hadoop version, as long as it's compatible with Hadoop 2.7.2 -->

<dependency>
  <groupId>org.apache.flink</groupId>
  <artifactId>flink-shaded-hadoop-2-uber</artifactId>
  <version>2.7.5-8.0</version>
  <scope>provided</scope>
</dependency>

<!-- Hive Metastore -->
<dependency>
    <groupId>org.apache.hive</groupId>
    <artifactId>hive-exec</artifactId>
    <version>2.3.4</version>
</dependency>
<dependency>
  <groupId>org.apache.flink</groupId>
  <artifactId>flink-connector-hive_2.11</artifactId>
  <version>1.9.0</version>
  <scope>provided</scope>
</dependency>

<!-- Hadoop Dependencies -->

<dependency>
  <groupId>org.apache.flink</groupId>
  <artifactId>flink-hadoop-compatibility_2.11</artifactId>
  <version>1.9.0</version>
  <scope>provided</scope>
</dependency>

<!-- Hive 1.2.1 is built with Hadoop 2.6.0. We pick 2.6.5 which flink-shaded-hadoop is pre-built with, but users can pick their own hadoop version, as long as it's compatible with Hadoop 2.6.0 -->

<dependency>
  <groupId>org.apache.flink</groupId>
  <artifactId>flink-shaded-hadoop-2-uber</artifactId>
  <version>2.6.5-8.0</version>
  <scope>provided</scope>
</dependency>

<!-- Hive Metastore -->
<dependency>
    <groupId>org.apache.hive</groupId>
    <artifactId>hive-metastore</artifactId>
    <version>1.2.1</version>
</dependency>

<dependency>
    <groupId>org.apache.hive</groupId>
    <artifactId>hive-exec</artifactId>
    <version>1.2.1</version>
</dependency>

<dependency>
    <groupId>org.apache.thrift</groupId>
    <artifactId>libfb303</artifactId>
    <version>0.9.3</version>
</dependency>

Connecting To Hive

Connect to an existing Hive installation using the Hive Catalog through the table environment or YAML configuration.

String name            = "myhive";
String defaultDatabase = "mydatabase";
String hiveConfDir     = "/opt/hive-conf";
String version         = "2.3.4"; // or 1.2.1

HiveCatalog hive = new HiveCatalog(name, defaultDatabase, hiveConfDir, version);
tableEnv.registerCatalog("myhive", hive);
val name            = "myhive"
val defaultDatabase = "mydatabase"
val hiveConfDir     = "/opt/hive-conf"
val version         = "2.3.4" // or 1.2.1

val hive = new HiveCatalog(name, defaultDatabase, hiveConfDir, version)
tableEnv.registerCatalog("myhive", hive)
catalogs:
   - name: myhive
     type: hive
     property-version: 1
     hive-conf-dir: /opt/hive-conf
     hive-version: 2.3.4 # or 1.2.1

Supported Types

Currently HiveCatalog supports most Flink data types with the following mapping:

Flink Data Type Hive Data Type
CHAR(p) CHAR(p)
VARCHAR(p) VARCHAR(p)
STRING STRING
BOOLEAN BOOLEAN
TINYINT TINYINT
SMALLINT SMALLINT
INT INT
BIGINT LONG
FLOAT FLOAT
DOUBLE DOUBLE
DECIMAL(p, s) DECIMAL(p, s)
DATE DATE
BYTES BINARY
ARRAY<T> LIST<T>
MAP<K, V> MAP<K, V>
ROW STRUCT

Limitations

The following limitations in Hive’s data types impact the mapping between Flink and Hive:

  • CHAR(p) has a maximum length of 255
  • VARCHAR(p) has a maximum length of 65535
  • Hive’s MAP only supports primitive key types while Flink’s MAP can be any data type
  • Hive’s UNION type is not supported
  • Flink’s INTERVAL type cannot be mapped to Hive INTERVAL type
  • Flink’s TIMESTAMP_WITH_TIME_ZONE and TIMESTAMP_WITH_LOCAL_TIME_ZONE are not supported by Hive
  • Flink’s TIMESTAMP_WITHOUT_TIME_ZONE type cannot be mapped to Hive’s TIMESTAMP type due to precision difference.
  • Flink’s MULTISET is not supported by Hive