随着FLink在流处理中的广泛采用,Flink也显示了其在批处理中的潜力。改进Flink的批处理,尤其是在SQL方面的能力,将使Flink在流处理之外得到更多的应用,并为用户提供流和批处理需求的全套解决方案。
另一方面,Hive已将其重点放在大数据技术及其完整的生态系统上。对于大多数大数据用户而言,Hive不仅是大数据分析和ETL领域SQL引擎,还是一个数据管理平台,可以在Hive平台上,发现,定义和演化数据。也就是说,Hive是Hadoop上大数据事实上的标准。
因此,Flink与Hive生态系统集成势在必行,进一步,FLink可以获得更多的Hive批处理和SQL用户支持。为此,Flink需要与Hive元数据(metadata)和数据(data)集成。
1.Hive的meta-object(元数据对象:例如表和视图)可供Flink使用,并且Flink也能够在Hive中创建此类meta-object(元数据对象)
2.使用Hive Metastore作为持久化存储,使Flink的meta-object(元数据对象:表,视图和UDF)持久化。
本文档是Flink和Hive生态系统集成的三个部分之一。不仅涉到FLink与Hive集成,还涉及到重构catalog interface接口,以及TableEnvironment内部catalog及外部catalog规整,长远目标是能够在catalog中,存储批处理和流处理connector连接器信息(不仅仅是Hive,还包括是Kafka, Elasticsearch等连接器)。
在下一节中可以看到,本文档主要介绍了一组与catalog相关的API,以替换现有的,不完整的和未使用的API。更具体地说,我们提出了一系列接口,用于管理和统一catalog内部的表,视图,函数等,这些接口在要么在外部系统(例如Hive),或者在内部系统(例如Flink内部SQL Client定义YAML文件)。我们还为catalog通常依赖的meta-object(元数据对象)定义了接口,例如表,视图和函数(UDF)。由于Flink允许多个catalog共存于一个用户会话中,因此需要一个管理类CatalogManager,让用户代码更为简洁和方便。简而言之,这些接口主要与SQL和Table API有关。
Catalog可以在客户端程序(例如SQL Client)中配置。因此,我们还支持YAML方式配置Catalog。
在当前的Flink代码库中,已经为外部catalog定义了一组接口。但是,这些API尚不稳定,需要适应我们后续的工作计划。
实现方案的层次结构如下:
在图1中,ReadableCatalog、ReadableWritableCatalog和CatalogManager是我们定义的主要接口。其他的只是实现类或接口调用者。
这个类来自重命名现有的ExternalCatalog 类。删除“External”(外部)关键字的原因是内部和外部之间没有明确的区别,因为外部catalog也可以用于存储Flink的元对象。
我们需要调整现有的API,以容纳其他meta-objects(元数据对象),例如Flink中存在的表和视图,它们在典型的数据库catalog中也很常见。我们还恢复了schema/database概念,而不是非标准的子目录(sub-catalog)术语。
/**
* All methods in this class can throw a `CatalogException` if the communication
* fails or the API is not used as intended.
*/
public interface ReadableCatalog {
/**
* Open and close methods for establishing or tearing down connections
* to external systems.
*/
void open();
void close();
/**
* Get a table factory instance that can convert catalog tables of this catalog
* to sources or sinks. This allows a catalog to provide connectors directly without
* going through a discovery mechanism. If a table factory isn’t provided, the
* current discovery mechanism will be used with Java Service Providers.
*/
Optional getTableFactory();
/**
* List and get instances of databases.
*/
List listDatabases();
/**
* Return information about the database (description, last modified date, etc.).
* Throw a `DatabaseNotFound` exception if database isn’t found.
*/
CatalogDatabase getDatabase(String databaseName);
/**
* List table names under the given database. Throw a `DatabaseNotFound` exception
* if database isn’t found.
*/
List listTables(String databaseName);
/**
* List view names under the given database. Throw a `DatabaseNotFound` exception
* if database isn’t found.
*/
List listViews(String databaseName);
/**
* Returns both physical or virtual tables (aka tables or views).
*/
CatalogCommonTable getTable(ObjectPath tableOrViewName);
/**
* List function names under the given database. Throw a `DatabaseNotFound` exception if
* database isn’t found.
*/
List listFunctions(String databaseName);
/**
* Returns a function definition.
*/
CatalogFunction getFunction(ObjectPath functionName);
}
变更包括:
1、添加open()和close()。它们被添加到ReadableCatalog 接口,以兼容外部连接(external connections)。他们可能需要一些运行时上下文,但是我们暂时不做介绍。
2、添加了view/ UDF相关的读取接口
3、定义了表(table)和视图(view)之间的关系(见图2)
视图是一种特殊类型的表。更具体地说,视图通过SQL语句,在其他表或视图之上定义出来的虚拟表。
这表示schema/database对象。它目前是subcatalog的模型,来自FLINK-6574。有关更多讨论,请参见“Additional Notes”附加说明部分。
请注意,许多meta-object类(包括CatalogDatabase ,CatalogTable 和CatalogView ),都有一个称为properties的成员变量。它们之所以出现,是因为external catalog可能允许用户指定任何常规属性,例如所有者,creation_time,last_access_time等。
public final class CatalogDatabase {
private final Map properties;
public CatalogDatabase(Map properties) {
this.properties = properties;
}
public Map getProperties() {
return properties;
}
}
这表示catalog中表/视图/函数的完全限定路径。
public final class ObjectPath {
private final String databaseName;
private final String objectName;
public ObjectPath(String databaseName, String objectName) {
this.databaseName = databaseName;
this.objectName = objectName;
}
public String getDatabaseName() {
return databaseName;
}
public String getObjectName() {
return objectName;
}
}
CatalogTable从ExternalCatalogTable修改而来 。它定义了Properties Map,在该属Map中,关于表的所有stats和schema,都被编码为键值对(descriptor),或者仅仅是一个POJO类,具有table schema, table stats, 和 table properties。
public interface CatalogCommonTable {
public TableSchema getSchema();
public Map getProperties();
}
public interface CatalogTable extends CatalogCommonTable {
public TableStatistics getTableStatistics();
}
注意:当前,TableSchema仅包含有关表的基本信息,例如字段名称和类型。由于还正在进行SQL DDL讨论,因此将来可能会扩展此类,以表示CREATE TABLE语句中声明的所有内容(schema, computed columns, partitioning, and key constraints约束).。TableSchema与table properties一起,factory 类具有全量的信息,用于创建Flink连接器。
这表示Hive目录中的一个表。
Public class HiveTable implements CatalogTable {
Public TableSchema getSchema() {
// get from Hive megastore
}
Public TableStats getStats() {
// get from Hive megastore
}
/**
* Hive table properties (not contain schema or stats)
*/
Public TableStats getProperties() {
// get from Hive megastore
}
}
此类表示Flink中当前定义的表,这些表没有外部定义。此类表当前存储在内存中,但可以存储在永久性存储中,以实现跨用户会话的持久化。
public class GenericCatalogTable implements CatalogTable {
// All table info (schema, stat, and table properties) is encoded as properties
private Map properties;
private TableSchema tableSchema;
prviate TableStats tableStats;
public TableSchema getSchema() {
return tableSchema
}
Public TableStats getStats() {
return tableStats;
}
public Map getProperties() {
return properties;
}
}
CatalogView是CommonTable的一种特殊类型。View视图由查询语句(query statement, )定义,查询语句的扩展形式也需要存储,以记住查询上下文(例如当前database)。
public interface CatalogView extends CommonTable {
// Original text of the view definition.
String getOriginalQuery();
// Expanded text of the original view definition
// This is needed because the context such as current DB is
// lost after the session, in which view is defined, is gone.
// Expanded query text takes care of the this, as an example.
String getExpandedQuery();
}
此类表示在catalog中定义的函数(UDF)。它现在仅用作占位,因为许多细节需要明确出来。而且CatalogFunction需要涵盖Flink功能和Hive功能。
/**
* The detailed definition of this class needs to be further sorted
* out
*/
public class CatalogFunction {
private Enum from; // source of the function (can only be "CLASS" for now)
private String clazz; // fully qualified class name of the function
...
private Map properties;
public CatalogFunction(String from, String clazz, Map properties) {
this.properties = properties;
}
public Map getProperties() {
return properties;
}
}
该接口来自重命名CrudExternalCatalog 类。我们添加了与view和function相关的方法。
public interface ReadableWritableCatalog extends ReadableCatalog {
void createDatabase(String databaseName, CatalogDatabase database, boolean ignoreIfExists);
void alterDatabase(String databaseName, CatalogDatabase database, boolean ignoreIfNotExists);
void renameDatabase(String databaseName,String newDatabaseName, boolean ignoreIfNotExists);
void dropDatabase(String databaseName, boolean ignoreIfNotExists);
void createTable(ObjectPath tableName, CatalogTable table, boolean ignoreIfExists);
/**
* dropTable() also covers views.
* @param tableName
* @param ignoreIfNotExists
*/
void dropTable(ObjectPath tableName, boolean ignoreIfNotExists);
void renameTable(ObjectPath tableName, String newTableName, boolean ignoreIfNotExists);
void alterTable(ObjectPath tableName, CatalogTable table, boolean ignoreIfNotExists):
void createView(ObjectPath viewName, CatalogView view, boolean ignoreIfExists);
void alterView(ObjectPath viewName, CatalogView view, boolean ignoreIfNotExists);
void createFunction(ObjectPath funcName, CatalogFunction function, boolean ignoreIfExists);
void renameFunction(ObjectPath funcName, String newFuncName, boolean ignoreIfNotExists);
void dropFunction(ObjectPath funcName, boolean ignoreIfNotExists);
void alterFunction(ObjectPath funcName, CatalogFunction function, boolean ignoreIfNotExists);
}
这是HiveCatalog和GenericHiveMetastoreCatalog的父类。
abstract class HiveCatalogBase implements ReadableWritableCatalog {
Private HiveMetastoreClient hmsClient;
// implementation for reading metadata from or writing metadata to
// Hive metastore
// Any utility methods that are common to both HiveCatalog and
// FlinkHmsCatalog
}
HiveCatalog类是HiveCatalogBase类的扩展,是存储在Hive Metastore中的Hive meta-objects对应的catalog。
class HiveCatalog extends HiveCatalogBase {
public TableFactory getTableFactory() {
return new HiveTableFactory();
}
// Implementation of other methods that are not implemented yet.
}
这是catalog的实现类,用于保存Flink当前定义的table(view/function)。这个类利用Hive元存储作为持久性存储。
class GenericHiveMetastoreCatalog extends HiveCatalogBase {
public TableFactory getTableFactory() {
return null; // Use table factory discovery mechanism
}
// Implementation of other methods that are not implemented yet.
}
我们引入CatalogManager类来管理TableEnvironment中所有已注册的ReadableCatalog 实例。它还具有默认catalog的概念,当在meta-object引用中未提供catalog名称时,将选择默认catalog。
public class CatalogManager {
// The catalog to hold all registered and translated tables
// We disable caching here to prevent side effects
private CalciteSchema internalSchema = CalciteSchema.createRootSchema(true, false);
private SchemaPlus rootSchema = internalSchema.plus();
// A list of named catalogs.
private Map catalogs;
// The name of the default catalog
private String defaultCatalog = null;
public CatalogManager(Map catalogs, String defaultCatalog) {
// make sure that defaultCatalog is in catalogs.keySet().
this.catalogs = catalogs;
this.defaultCatalog = defaultCatalog;
}
public void registerCatalog(String catalogName, ReadableCatalog catalog) {
catalogs.put(catalogName, catalog);
}
public ReadableCatalog getCatalog(String catalogName) {
return catalogs.get(catalogName);
}
public Set getCatalogs() {
return this.catalogs.keySet();
}
public void setDefaultCatalog(String catName) {
// validate
this.defaultCatalog = catqName;
}
public ReadableCatalog getDefaultCatalog() {
return this.catalogs.get(defaultCatalog);
}
}
除了ReadableCatalogs列表之外,CatalogManger还封装了Calcite的schema框架,这样除了parser需要所有的catalog之外,CatalogManager之外的任何代码都不需要与Calcite的schema交互。(所有的catalog都将被添加到calcite schema中,以便所有的external外部表,可以被calcite在查询和解析期间使用)
这是table API中的已有的类,该类现在具有对CatalogManager 实例的引用,该引用(CatalogManager实例),将用来添加或者替换 in-memory meta-objects、registered catalogs。
abstract class TableEnvironment(val config: TableConfig) {
…
private val catalogManager: CatalogManager;
// This is an existing class with only argument type change
def registerCatalog(name: String, catalog: ReadableCatalog): Unit
// Set the default catalog
def setDefaultCatalog(catName: String);
// Set the default database
Def setDefaultDatabase(catName: String, databaseName: String): unit
}
TableEnvironment 类当前具有一些特殊的registerTable 方法,例如TableSource ,TableSink 和非公共类(例如Table RelTable 和Table InlineTable) 。这些API将保持不变。但是,为了利用catalog的持久化,可能会更改其实现。详细信息将在本设计系列的第2部分中提供。
以下是Flink中catalog配置的示例。可用的catalog类型有:flink-in-memory, generic-hive-metastore, and hive。每个实现类和相应的工厂类的详细信息将在后续的设计文档中提供。在这里,我们仅关注如何在YAML文件中指定catalog。
catalogs:
- name: hive1
catalog:
type: hive
is-default: false
default-db: default
connection-params:
hive.metastore.uris: “thrift://host1:10000,thrift://host2:10000”
hive.metastore.username: “flink”
- name: flink1
catalog:
type: generic-hive-metastore
is-default: true
Default-db: default
connection-params:
hive.metastore.uris: “thrift://host1:10000,thrift://host2:10000”
hive.metastore.username: “flink”
Hive.metastore.db: flink
这是现有接口。
interface TableFactory {
Map requiredContext();
List supportedProperties();
}
The following is a utility class providing implementations for conversions between CatallogTable and property map.
public class TableFactoryUtils {
public static CatalogTable convertToCatalogTable(Map properties) {
// default implementation
}
Public static Map convertToProperties(CatalogTable
table) {
// implementation
}
}
Interface StreamTableSourceFactory extends TableFactory {
// this one is existing one, which will be deprecated.
@Deprecated
StreamTableSource createStreamTableSource(Map properties);
// This one is new with default implementation.
Default StreamTableSource createStreamTableSource(CatalogTable table) {
return createStreamTableSource(
TableFactoryUtils.convertToProperties(table) );
}
}
Interface StreamTableSinkFactory extends TableFactory {
// this one is existing one
StreamTableSink createStreamSinkSource(Map properties);
// This one is new.
Default StreamTableSink createStreamSinkSource(CatalogTable table) {
return createStreamTableSink(
TableFactoryUtils.convertToProperties(table) );
}
}
Interface BatchTableSourceFactory extends TableFactory {
// this one is existing one
BatchTableSource createBatchTableSource(Map properties);
// This one is new.
Default BatchTableSource createBatchTableSource(CatalogTable table) {
return createBatchTableSource(
TableFactoryUtils.convertToProperties(table) );
}
}
Interface BatchTableSinkFactory extends TableFactory {
// this one is existing one
BatchTableSink createBatchTableSink(Map properties);
// This one is new.
BatchTableSink createBatchTableSink(CatalogTable table) {
return createBatchTableSink(
TableFactoryUtils.convertToProperties(table) );
}
}
HiveTableFactory是TableFactory新的实现类。我们目前仅支持batch模式。
Public class HiveTableFactory implements BatchTableSourceFactory, BatchTableSinkFactory {
Map requiredContext() {
// return an empty map to indicate that auto discovery is not needed.
return new HashMap<>().
}
List supportedProperties() {
// Return an empty list to indicate that no check is needed.
Return new ArrayList<>();
}
BatchTableSource createBatchTableSource(Map properties) {
// convert properties to catalogtable and call the other version of this method.
// It’s fine not to support this method.
}
BatchTableSource createBatchTableSink(Map properties) {
// convert properties to catalogtable and call the other version of this method.
// It’s fine not to support this method.
}
BatchTableSource createBatchTableSource(CatalogTable table) {
Assert (table instanceof HiveTable);
HiveTable hiveTable = (HiveTable)table;
// create a table source based on HiveTable
// This is specific implementation for Hive tables.
}
BatchTableSource createBatchTableSink(CatalogTable table) {
Assert (table instanceof HiveTable);
HiveTable hiveTable = (HiveTable)table;
// create a table sink based on HiveTable
// This is specific implementation for Hive tables.
}
}
如果一个catalog(例如上面的GenericHiveMetastoreCatalog )从其getTableFactory()中返回null ,则框架将利用Java Service Provider interfaces(SPI)自动发现真实的table factory。这是Flink中定义的所有Table的现有机制。
HiveTable类是CatalogTable的实现,它表示Hive中的表。
在一个系统具有多个catalog的情况下,必须通过 catalog name, schema/database name, table name来标识表(Table)。因此,表引用需要包括catalog name, schema name, and table name,例如hive1.risk_db.user_events 。如果缺少catalog名称,则假定它表示默认catalog(无论当前默认catalog设置的是哪一个)和默认database。
我们在Flink SQL中引入了默认数据库(default database)概念。这对应于SQL“ use xxx”,其中将schema(database) 设置为当前架构,而没有database/schema前缀的任何表都引用默认架构。由于Flink具有多个catalog,因此语法将为“ use cat1.db1”,其中cat1将是默认目录,而db1将是默认数据库。给定一个表名,目录管理器必须将其解析为全名,以便正确识别该表。
这与FLINK-6574中所做的更改形成鲜明对比,后者试图减少指定目录名称的需要。从理论上讲,这是不可行的,因为需要支持多个catalog。初步测试表明,FLINK-6574未能达到预期的效果。相反,它造成了极大的概念混乱。因此,我们将审查并调整此工作中的更改。
1、创建ReadableCatalog ,ReadableWritableCatalog 和相关接口。弃用现有的ExternalCatalog 和CrudExternalCatalog 接口。
2、调整现有InMemoryExternalCatalog 到GenericInMemoryCatalog 类。
3、在TableEnvironment中创建CatalogManager 来管理注册的(可能是多个)catalog,并封装Calcite schema管理。
4、在SQL客户端YAML文件中定义catalog 入口,并处理这些DDL的创建和注册。
5、YAML文件作为table的入口保持向后兼容性。
6、实现HiveCatalog 和HiveTableFactory 。
7、将HiveTableFactory 与现有table factory discovery关联起来。
8、实现GenericHiveMetastoreCatalog 。
这里的设计完全基于JAVA,因此实现取决于将当前TableEnvironment 和相关类移植到JAVA 的工作是否完成。[ FLIP-28 ],[ FLINK-11067 ]
1、兼容性不应该成为问题,因为要更改的接口不完整,不稳定且未使用。
2、不鼓励在SQL Client YAML文件中定义表的方式,但是为了向后兼容,仅当catalog中不存在表时才创建它们。
3、catalog相关的那些旧接口和类将被简单地删除或修改,而无需弃用过程。
4、无需迁移。
由于更改主要涉及一组大多数未使用的API,因此测试主要是关于:当我们实际实现Hive connector以及Flink中定义的catalog和meta-object(元数据对象),对API有效性进行验证。在此过程中,可能会进一步完善API,以满足这些实现中的特殊要求。