Sharing is caring - Catalogs in Flink SQL

July 23, 2020 - Dawid Wysakowicz (@dwysakowicz)

With an ever-growing number of people working with data, it’s a common practice for companies to build self-service platforms with the goal of democratizing their access across different teams and — especially — to enable users from any background to be independent in their data needs. In such environments, metadata management becomes a crucial aspect. Without it, users often work blindly, spending too much time searching for datasets and their location, figuring out data formats and similar cumbersome tasks.

In this blog post, we want to give you a high level overview of catalogs in Flink. We’ll describe why you should consider using them and what you can achieve with one in place. To round it up, we’ll also showcase how simple it is to combine catalogs and Flink, in the form of an end-to-end example that you can try out yourself.

Why do I need a catalog? #

Frequently, companies start building a data platform with a metastore, catalog, or schema registry of some sort already in place. Those let you clearly separate making the data available from consuming it. That separation has a few benefits:

  • Improved productivity - The most obvious one. Making data reusable and shifting the focus on building new models/pipelines rather than data cleansing and discovery.
  • Security - You can control the access to certain features of the data. For example, you can make the schema of the dataset publicly available, but limit the actual access to the underlying data only to particular teams.
  • Compliance - If you have all the metadata in a central entity, it’s much easier to ensure compliance with GDPR and similar regulations and legal requirements.

What is stored in a catalog? #

Almost all data sets can be described by certain properties that must be known in order to consume them. Those include:

  • Schema - It describes the actual contents of the data, what columns it has, what are the constraints (e.g. keys) on which the updates should be performed, which fields can act as time attributes, what are the rules for watermark generation and so on.

  • Location - Does the data come from Kafka or a file in a filesystem? How do you connect to the external system? Which topic or file name do you use?

  • Format - Is the data serialized as JSON, CSV, or maybe Avro records?

  • Statistics - You can also store additional information that can be useful when creating an execution plan of your query. For example, you can choose the best join algorithm, based on the number of rows in joined datasets.

Catalogs don’t have to be limited to the metadata of datasets. You can usually store other objects that can be reused in different scenarios, such as:

  • Functions - It’s very common to have domain specific functions that can be helpful in different use cases. Instead of having to create them in each place separately, you can just create them once and share them with others.

  • Queries - Those can be useful when you don’t want to persist a data set, but want to provide a recipe for creating it from other sources instead.

Starting from version 1.9, Flink has a set of Catalog APIs that allows to integrate Flink with various catalog implementations. With the help of those APIs, you can query tables in Flink that were created in your external catalogs (e.g. Hive Metastore). Additionally, depending on the catalog implementation, you can create new objects such as tables or views from Flink, reuse them across different jobs, and possibly even use them in other tools compatible with that catalog. In other words, you can see catalogs as having a two-fold purpose:

  • Provide an out-of-the box integration with ecosystems such as RDBMSs or Hive that allows you to query external objects like tables, views, or functions with no additional connector configuration. The connector properties are automatically derived from the catalog itself.

  • Act as a persistent store for Flink-specific metadata. In this mode, we additionally store connector properties alongside the logical metadata (e.g. schema, object name). That approach enables you to, for example, store a full definition of a Kafka-backed table with records serialized with Avro in Hive that can be later on used by Flink. However, as it incorporates Flink-specific properties, it can not be used by other tools that leverage Hive Metastore.

As of Flink 1.11, there are two catalog implementations supported by the community:

  1. A comprehensive Hive catalog

  2. A Postgres catalog (preview, read-only, for now)

Note Flink does not store data at rest; it is a compute engine and requires other systems to consume input from and write its output. This means that Flink does not own the lifecycle of the data. Integration with Catalogs does not change that. Flink uses catalogs for metadata management only.

All you need to do to start querying your tables defined in either of these metastores is to create the corresponding catalogs with connection parameters. Once this is done, you can use them the way you would in any relational database management system.

-- create a catalog which gives access to the backing Postgres installation
CREATE CATALOG postgres WITH (
    'type'='jdbc',
    'property-version'='1',
    'base-url'='jdbc:postgresql://postgres:5432/',
    'default-database'='postgres',
    'username'='postgres',
    'password'='example'
);

-- create a catalog which gives access to the backing Hive installation
CREATE CATALOG hive WITH (
    'type'='hive',
    'property-version'='1',
    'hive-version'='2.3.6',
    'hive-conf-dir'='/opt/hive-conf'
);

After creating the catalogs, you can confirm that they are available to Flink and also list the databases or tables in each of these catalogs:

> show catalogs;
default_catalog
hive
postgres

-- switch the default catalog to Hive
> use catalog hive;
> show databases;
default -- hive's default database

> show tables;
dev_orders

> use catalog postgres;
> show tables;
prod_customer
prod_nation
prod_rates
prod_region
region_stats

-- desribe a schema of a table in Postgres, the Postgres types are automatically mapped to
-- Flink's type system
> describe prod_customer
root
 |-- c_custkey: INT NOT NULL
 |-- c_name: VARCHAR(25) NOT NULL
 |-- c_address: VARCHAR(40) NOT NULL
 |-- c_nationkey: INT NOT NULL
 |-- c_phone: CHAR(15) NOT NULL
 |-- c_acctbal: DOUBLE NOT NULL
 |-- c_mktsegment: CHAR(10) NOT NULL
 |-- c_comment: VARCHAR(117) NOT NULL

Now that you know which tables are available, you can write your first query. In this scenario, we keep customer orders in Hive (dev_orders) because of their volume, and reference customer data in Postgres (prod_customer) to be able to easily update it. Let’s write a query that shows customers and their orders by region and order priority for a specific day.

USE CATALOG postgres;
SELECT
  r_name AS `region`,
  o_orderpriority AS `priority`,
  COUNT(DISTINCT c_custkey) AS `number_of_customers`,
  COUNT(o_orderkey) AS `number_of_orders`
FROM `hive`.`default`.dev_orders -- we need to fully qualify the table in hive because we set the
                                 -- current catalog to Postgres
JOIN prod_customer ON o_custkey = c_custkey
JOIN prod_nation ON c_nationkey = n_nationkey
JOIN prod_region ON n_regionkey = r_regionkey
WHERE
  FLOOR(o_ordertime TO DAY) = TIMESTAMP '2020-04-01 0:00:00.000'
  AND NOT o_orderpriority = '4-NOT SPECIFIED'
GROUP BY r_name, o_orderpriority
ORDER BY r_name, o_orderpriority;

Flink’s catalog support also covers storing Flink-specific objects in external catalogs that might not be fully usable by the corresponding external tools. The most notable use case for this is, for example, storing a table that describes a Kafka topic in a Hive catalog. Take the following DDL statement, that contains a watermark declaration as well as a set of connector properties that are not recognizable by Hive. You won’t be able to query the table with Hive, but it will be persisted and can be reused by different Flink jobs.

USE CATALOG hive;
CREATE TABLE prod_lineitem (
  l_orderkey INTEGER,
  l_partkey INTEGER,
  l_suppkey INTEGER,
  l_linenumber INTEGER,
  l_quantity DOUBLE,
  l_extendedprice DOUBLE,
  l_discount DOUBLE,
  l_tax DOUBLE,
  l_currency STRING,
  l_returnflag STRING,
  l_linestatus STRING,
  l_ordertime TIMESTAMP(3),
  l_shipinstruct STRING,
  l_shipmode STRING,
  l_comment STRING,
  l_proctime AS PROCTIME(),
  WATERMARK FOR l_ordertime AS l_ordertime - INTERVAL '5' SECONDS
) WITH (
  'connector'='kafka',
  'topic'='lineitem',
  'scan.startup.mode'='earliest-offset',
  'properties.bootstrap.servers'='kafka:9092',
  'properties.group.id'='testGroup',
  'format'='csv',
  'csv.field-delimiter'='|'
);

With prod_lineitem stored in Hive, you can now write a query that will enrich the incoming stream with static data kept in Postgres. To illustrate how this works, let’s calculate the item prices based on the current currency rates:

USE CATALOG postgres;
SELECT
  l_proctime AS `querytime`,
  l_orderkey AS `order`,
  l_linenumber AS `linenumber`,
  l_currency AS `currency`,
  rs_rate AS `cur_rate`,
  (l_extendedprice * (1 - l_discount) * (1 + l_tax)) / rs_rate AS `open_in_euro`
FROM hive.`default`.prod_lineitem
JOIN prod_rates FOR SYSTEM_TIME AS OF l_proctime ON rs_symbol = l_currency
WHERE
  l_linestatus = 'O';

The query above uses a SYSTEM AS OF clause for executing a temporal join. If you’d like to learn more about the different kind of joins you can do in Flink I highly encourage you to check this documentation page.

Conclusion #

Catalogs can be extremely powerful when building data platforms aimed at reusing the work of different teams in an organization. Centralizing the metadata is a common practice for improving productivity, security, and compliance when working with data.

Flink provides flexible metadata management capabilities, that aim at reducing the cumbersome, repetitive work needed before querying the data such as defining schemas, connection properties etc. As of version 1.11, Flink provides a native, comprehensive integration with Hive Metastore and a read-only version for Postgres catalogs.

You can get started with Flink and catalogs by reading the docs. If you want to play around with Flink SQL (e.g. try out how catalogs work in Flink yourself), you can check this demo prepared by our colleagues Fabian and Timo — it runs in a dockerized environment, and we used it for the examples in this blog post.