Flink如何创建同Hive兼容的表

曾使用Flink API创建Hive表,然后查询时碰到过这个问题:

hive>select * from  test_db.test_flink_hive_table;
FAILED: SemanticException Line 0:-1 Invalid column reference 'TOK_ALLCOLREF'

但是如果使用Flink SQL去查询却是正常的。切换其他引擎,比如使用Trino按照相同的方式创建表,也是正常查询,那问题自然就是Flink的处理方式不同了。

用Hive SQL打印表的详情来研究下:

hive>desc formatted test_db.test_flink_hive_table;
OK
# col_name            	data_type           	comment             
		 
# Detailed Table Information		 
Database:           	test_db             	 
OwnerType:          	USER                	 
Owner:              	null                	 
LastAccessTime:     	UNKNOWN             	 
Retention:          	0                   	 
Location:           	file:/tmp/test_db/test_flink_hive_table	 
Table Type:         	MANAGED_TABLE       	 
Table Parameters:		 
	flink.comment       	test crreate flink hive table
	flink.schema.0.data-type	INT                 
	flink.schema.0.name 	id                  
	flink.schema.1.data-type	VARCHAR(2147483647) 
	flink.schema.1.name 	data                
	is_generic          	true                
	transient_lastDdlTime	1678861999          
		 
# Storage Information		 
SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
OutputFormat:       	org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat	 
Compressed:         	No                  	 
Num Buckets:        	0                   	 
Bucket Columns:     	[]                  	 
Sort Columns:       	[]                  	 
Storage Desc Params:		 
	serialization.format	1

注意到打印的第4行为空,没有字段!正常的Hive表,都是有字段的。但是在第13行的表参数中多了以flink.schema为前缀的属性,里面有字段信息。我们用Flink SQL再来看看:

CREATE TABLE test_db.test_flink_hive_table_1 (ID BIGINT)  
with (
	'connector'='hive',
	'is_generic' = 'false'
)

再次用Hive SQL打印表的详情来比较下:

hive> desc formatted test_db.test_flink_hive_table_1;
OK
# col_name            	data_type           	comment             
id                  	bigint              	                    
	 	 
# Detailed Table Information	 	 
Database:           	test_db             	 
OwnerType:          	USER                	 
Owner:              	null                	 
LastAccessTime:     	UNKNOWN             	 
Retention:          	0                   	 
Location:           	file:/tmp/test_db/test_flink_hive_table_1	 
Table Type:         	MANAGED_TABLE       	 
Table Parameters:	 	 
	bucketing_version   	2                   
	is_generic          	false               
	numFiles            	0                   
	totalSize           	0                   
	transient_lastDdlTime	1678852266          
	 	 
# Storage Information	 	 
SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
OutputFormat:       	org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat	 
Compressed:         	No                  	 
Num Buckets:        	0                   	 
Bucket Columns:     	[]                  	 
Sort Columns:       	[]                  	 
Storage Desc Params:	 	 
	serialization.format	1

打印的第4行的字段出来了,同时第14行的Table Parameters中没有了flink.schema为前缀的属性了。

再比较系Flink API创建表和Flink SQL创建表有啥不一样?

ObjectPath tablePath=new ObjectPath("test_db",table);
TableSchema tableSchema=TableSchema.builder().field("id",DataTypes.INT()).build();
CatalogTable catalogTable=new CatalogTableImpl(tableSchema, new HashMap<>(),"test crreate flink hive table");
hiveCatalog.createTable(tablePath,catalogTable,false);

从创建表传递的参数来看,通过API创建的表缺少了'connector'='hive',那问题基本就认定是它了,为什么会这样?这得从Flink的建表行为来找原因了,这里是Flink CatalogManager触发HiveCatalog建表的起点:

//CatalogManager.java
public void createTable(
            CatalogBaseTable table, ObjectIdentifier objectIdentifier, boolean ignoreIfExists) {
        execute(
                (catalog, path) ->
                        catalog.createTable(path, resolveCatalogBaseTable(table), ignoreIfExists),
                objectIdentifier,
                false,
                "CreateTable");
    }

在HiveCatalog的HiveTableUtil的实例化Table的方法中我们找到了明确的答案:

//HiveTableUtil.java
public static Table instantiateHiveTable(
            ObjectPath tablePath, CatalogBaseTable table, HiveConf hiveConf) {
      final boolean isView = table instanceof CatalogView;
      // let Hive set default parameters for us, e.g. serialization.format
      Table hiveTable =
              org.apache.hadoop.hive.ql.metadata.Table.getEmptyTable(
                      tablePath.getDatabaseName(), tablePath.getObjectName());
      hiveTable.setCreateTime((int) (System.currentTimeMillis() / 1000));

      Map<String, String> properties = new HashMap<>(table.getOptions());
      // Table comment
      if (table.getComment() != null) {
          properties.put(HiveCatalogConfig.COMMENT, table.getComment());
      }

      boolean isHiveTable = HiveCatalog.isHiveTable(properties);
      // Hive table's StorageDescriptor
      StorageDescriptor sd = hiveTable.getSd();
      HiveTableUtil.setDefaultStorageFormat(sd, hiveConf);
      if (isHiveTable && !isView) {
  					// Table columns and partition keys
            if (table instanceof CatalogTable) {
                CatalogTable catalogTable = (CatalogTable) table;

                if (catalogTable.isPartitioned()) {
                    int partitionKeySize = catalogTable.getPartitionKeys().size();
                    List<FieldSchema> regularColumns =
                            allColumns.subList(0, allColumns.size() - partitionKeySize);
                    List<FieldSchema> partitionColumns =
                            allColumns.subList(
                                    allColumns.size() - partitionKeySize, allColumns.size());

                    sd.setCols(regularColumns);
                    hiveTable.setPartitionKeys(partitionColumns);
                } else {
                    sd.setCols(allColumns);
                    hiveTable.setPartitionKeys(new ArrayList<>());
                }
            } else {
                sd.setCols(allColumns);
            }
            // Table properties
            hiveTable.getParameters().putAll(properties);
      } else {
          DescriptorProperties tableSchemaProps = new DescriptorProperties(true);
          tableSchemaProps.putTableSchema(Schema.SCHEMA, table.getSchema());

          if (table instanceof CatalogTable) {
              tableSchemaProps.putPartitionKeys(((CatalogTable) table).getPartitionKeys());
          }

          properties.putAll(tableSchemaProps.asMap());
          properties = maskFlinkProperties(properties);
          hiveTable.setParameters(properties);
          return hiveTable;
  }

从代码可见,Flink根据当前创建的表是否是HiveTable采取不同的逻辑:

  1. 如果是HiveTable,创建常规的Hive Table对象,包括设置表的字段、分区、参数、序列化参数、存储路径等;

  2. 如果不是HiveTable,则没有上述过程,只是将Table的字段等信息打包封装进表参数里面。这就解释了上面为何API创建的Hive表的字段信息都跑到表的以flink.schema为前缀的参数中了。

通过这个案例可知,要创建同Hive兼容的Flink 表,connector属性还是不可丢,否则只能Flink 自己能识别了(这就是is_generic=true的含义)。但是加一个connector还是略显麻烦,有没有其他不需要指定该参数的方式呢?

这就用到了Flink 的SQL方言了:

hive>set table.sql-dialect=hive;
hive>CREATE TABLE test_db.test_flink_hive_table_2 (ID BIGINT);

打印表详情结果,会发现Flink自动加了connector参数。

  • 0
    点赞
  • 0
    收藏
    觉得还不错? 一键收藏
  • 0
    评论

“相关推荐”对你有帮助么?

  • 非常没帮助
  • 没帮助
  • 一般
  • 有帮助
  • 非常有帮助
提交
评论
添加红包

请填写红包祝福语或标题

红包个数最小为10个

红包金额最低5元

当前余额3.43前往充值 >
需支付:10.00
成就一亿技术人!
领取后你会自动成为博主和红包主的粉丝 规则
hope_wisdom
发出的红包
实付
使用余额支付
点击重新获取
扫码支付
钱包余额 0

抵扣说明:

1.余额是钱包充值的虚拟货币,按照1:1的比例进行支付金额的抵扣。
2.余额无法直接购买下载,可以购买VIP、付费专栏及课程。

余额充值