Hive Catalog

Hive Metastore has evolved into the de facto metadata hub over the years in Hadoop ecosystem. Many companies have a single Hive Metastore service instance in their production to manage all of their metadata, either Hive metadata or non-Hive metadata, as the source of truth.

For users who have both Hive and Flink deployments, HiveCatalog enables them to use Hive Metastore to manage Flink’s metadata.

For users who have just Flink deployment, HiveCatalog is the only persistent catalog provided out-of-box by Flink. Without a persistent catalog, users using Flink SQL CREATE DDL have to repeatedly create meta-objects like a Kafka table in each session, which wastes a lot of time. HiveCatalog fills this gap by empowering users to create tables and other meta-objects only once, and reference and manage them with convenience later on across sessions.

Set up HiveCatalog

Dependencies

Setting up a HiveCatalog in Flink requires the same dependencies as those of an overall Flink-Hive integration.

Configuration

Setting up a HiveCatalog in Flink requires the same configuration as those of an overall Flink-Hive integration.

How to use HiveCatalog

Once configured properly, HiveCatalog should just work out of box. Users can create Flink meta-objects with DDL, and should see them immediately afterwards.

HiveCatalog can be used to handle two kinds of tables: Hive-compatible tables and generic tables. Hive-compatible tables are those stored in a Hive-compatible way, in terms of both metadata and data in the storage layer. Therefore, Hive-compatible tables created via Flink can be queried from Hive side.

Generic tables, on the other hand, are specific to Flink. When creating generic tables with HiveCatalog, we’re just using HMS to persist the metadata. While these tables are visible to Hive, it’s unlikely Hive is able to understand the metadata. And therefore using such tables in Hive leads to undefined behavior.

It’s recommended to switch to Hive dialect to create Hive-compatible tables. If you want to create Hive-compatible tables with default dialect, make sure to set 'connector'='hive' in your table properties, otherwise a table is considered generic by default in HiveCatalog. Note that the connector property is not required if you use Hive dialect.

Example

We will walk through a simple example here.

step 1: set up a Hive Metastore

Have a Hive Metastore running.

Here, we set up a local Hive Metastore and our hive-site.xml file in local path /opt/hive-conf/hive-site.xml. We have some configs like the following:

  1. <configuration>
  2. <property>
  3. <name>javax.jdo.option.ConnectionURL</name>
  4. <value>jdbc:mysql://localhost/metastore?createDatabaseIfNotExist=true</value>
  5. <description>metadata is stored in a MySQL server</description>
  6. </property>
  7. <property>
  8. <name>javax.jdo.option.ConnectionDriverName</name>
  9. <value>com.mysql.jdbc.Driver</value>
  10. <description>MySQL JDBC driver class</description>
  11. </property>
  12. <property>
  13. <name>javax.jdo.option.ConnectionUserName</name>
  14. <value>...</value>
  15. <description>user name for connecting to mysql server</description>
  16. </property>
  17. <property>
  18. <name>javax.jdo.option.ConnectionPassword</name>
  19. <value>...</value>
  20. <description>password for connecting to mysql server</description>
  21. </property>
  22. <property>
  23. <name>hive.metastore.uris</name>
  24. <value>thrift://localhost:9083</value>
  25. <description>IP address (or fully-qualified domain name) and port of the metastore host</description>
  26. </property>
  27. <property>
  28. <name>hive.metastore.schema.verification</name>
  29. <value>true</value>
  30. </property>
  31. </configuration>

Test connection to the HMS with Hive Cli. Running some commands, we can see we have a database named default and there’s no table in it.

  1. hive> show databases;
  2. OK
  3. default
  4. Time taken: 0.032 seconds, Fetched: 1 row(s)
  5. hive> show tables;
  6. OK
  7. Time taken: 0.028 seconds, Fetched: 0 row(s)

Add all Hive dependencies to /lib dir in Flink distribution, and create a Hive catalog in Flink SQL CLI as following:

  1. Flink SQL> CREATE CATALOG myhive WITH (
  2. 'type' = 'hive',
  3. 'hive-conf-dir' = '/opt/hive-conf'
  4. );

step 3: set up a Kafka cluster

Bootstrap a local Kafka cluster with a topic named “test”, and produce some simple data to the topic as tuple of name and age.

  1. localhost$ bin/kafka-console-producer.sh --broker-list localhost:9092 --topic test
  2. >tom,15
  3. >john,21

These message can be seen by starting a Kafka console consumer.

  1. localhost$ bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic test --from-beginning
  2. tom,15
  3. john,21

Create a simple Kafka table with Flink SQL DDL, and verify its schema.

  1. Flink SQL> USE CATALOG myhive;
  2. Flink SQL> CREATE TABLE mykafka (name String, age Int) WITH (
  3. 'connector.type' = 'kafka',
  4. 'connector.version' = 'universal',
  5. 'connector.topic' = 'test',
  6. 'connector.properties.bootstrap.servers' = 'localhost:9092',
  7. 'format.type' = 'csv',
  8. 'update-mode' = 'append'
  9. );
  10. [INFO] Table has been created.
  11. Flink SQL> DESCRIBE mykafka;
  12. root
  13. |-- name: STRING
  14. |-- age: INT

Verify the table is also visible to Hive via Hive Cli:

  1. hive> show tables;
  2. OK
  3. mykafka
  4. Time taken: 0.038 seconds, Fetched: 1 row(s)

Run a simple select query from Flink SQL Client in a Flink cluster, either standalone or yarn-session.

  1. Flink SQL> select * from mykafka;

Produce some more messages in the Kafka topic

  1. localhost$ bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic test --from-beginning
  2. tom,15
  3. john,21
  4. kitty,30
  5. amy,24
  6. kaiky,18

You should see results produced by Flink in SQL Client now, as:

  1. SQL Query Result (Table)
  2. Refresh: 1 s Page: Last of 1
  3. name age
  4. tom 15
  5. john 21
  6. kitty 30
  7. amy 24
  8. kaiky 18

Supported Types

HiveCatalog supports all Flink types for generic tables.

For Hive-compatible tables, HiveCatalog needs to map Flink data types to corresponding Hive types as described in the following table:

Flink Data TypeHive Data Type
CHAR(p)CHAR(p)
VARCHAR(p)VARCHAR(p)
STRINGSTRING
BOOLEANBOOLEAN
TINYINTTINYINT
SMALLINTSMALLINT
INTINT
BIGINTLONG
FLOATFLOAT
DOUBLEDOUBLE
DECIMAL(p, s)DECIMAL(p, s)
DATEDATE
TIMESTAMP(9)TIMESTAMP
BYTESBINARY
ARRAY<T>LIST<T>
MAPMAP
ROWSTRUCT

Something to note about the type mapping:

  • Hive’s CHAR(p) has a maximum length of 255
  • Hive’s 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
  • Hive’s TIMESTAMP always has precision 9 and doesn’t support other precisions. Hive UDFs, on the other hand, can process TIMESTAMP values with a precision <= 9.
  • Hive doesn’t support Flink’s TIMESTAMP_WITH_TIME_ZONE, TIMESTAMP_WITH_LOCAL_TIME_ZONE, and MULTISET
  • Flink’s INTERVAL type cannot be mapped to Hive INTERVAL type yet