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.
Setting up a HiveCatalog
in Flink requires the same dependencies
as those of an overall Flink-Hive integration.
Setting up a HiveCatalog
in Flink requires the same configuration
as those of an overall Flink-Hive integration.
Once configured properly, HiveCatalog
should just work out of box. Users can create Flink meta-objects with DDL, and shoud
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.
Flink uses the property ‘is_generic’ to tell whether a table is Hive-compatible or generic. When creating a table with
HiveCatalog
, it’s by default considered generic. If you’d like to create a Hive-compatible table, make sure to set
is_generic
to false in your table properties.
As stated above, generic tables shouldn’t be used from Hive. In Hive CLI, you can call DESCRIBE FORMATTED
for a table and
decide whether it’s generic or not by checking the is_generic
property. Generic tables will have is_generic=true
.
We will walk through a simple example here.
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:
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.
Add all Hive dependencies to /lib
dir in Flink distribution, and modify SQL CLI’s yaml config file sql-cli-defaults.yaml
as following:
Bootstrap a local Kafka 2.3.0 cluster with a topic named “test”, and produce some simple data to the topic as tuple of name and age.
These message can be seen by starting a Kafka console consumer.
Start Flink SQL Client, create a simple Kafka 2.3.0 table via DDL, and verify its schema.
Verify the table is also visible to Hive via Hive Cli, and note that the table has property is_generic=true
:
Run a simple select query from Flink SQL Client in a Flink cluster, either standalone or yarn-session.
Produce some more messages in the Kafka topic
You should see results produced by Flink in SQL Client now, as:
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 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 |
TIMESTAMP(9) | TIMESTAMP |
BYTES | BINARY |
ARRAY<T> | LIST<T> |
MAP<K, V> | MAP<K, V> |
ROW | STRUCT |
Something to note about the type mapping:
CHAR(p)
has a maximum length of 255VARCHAR(p)
has a maximum length of 65535MAP
only supports primitive key types while Flink’s MAP
can be any data typeUNION
type is not supportedTIMESTAMP
always has precision 9 and doesn’t support other precisions. Hive UDFs, on the other hand, can process TIMESTAMP
values with a precision <= 9.TIMESTAMP_WITH_TIME_ZONE
, TIMESTAMP_WITH_LOCAL_TIME_ZONE
, and MULTISET
INTERVAL
type cannot be mapped to Hive INTERVAL
type yet注意:目前 blink planner 还不能很好的支持 Scala Shell,因此 不 建议在 Scala Shell 中使用 Hive 连接器。