本文基于Flink 1.11,主要讲解最新的基于Flink StreamingFileSink的FilesystemConnector和HiveConnector,包括理论、配置和源码分析。
关于StreamingFileSink,可参考
可读写本地或分布式文件系统(如HDFS)。注意,File System Connector做流处理目前还是试验阶段。
Flink 1.11.0开始实现了File System Connector,可直接使用FlinkSql写出支持分区的流式读写本地或分布式文件系统程序。
该Connector为内嵌,不需要任何依赖。
Flink当前具体对文件系统支持情况,请参考File Systems
Flink支持标准hive分区格式,但并非必须提前在Catalog注册分区,因为Flink可通过目录结构推断。
比如以下目录可推断出分区包含datetime
和 hour
支持append和overwrite两种模式写入分区表,请参阅INSERT Statement
目前支持
CSV
未压缩的
JSON
采用的是未压缩的、按换行符分隔了的JSON
Avro
Apache Avro. 支持通过avro.codec
配置压缩
Parquet
Apache Parquet. 与Hive兼容。
相关配置可以看类ParquetOutputFormat
,配置如下:
public static final String JOB_SUMMARY_LEVEL = "parquet.summary.metadata.level";
public static final String BLOCK_SIZE = "parquet.block.size";
public static final String PAGE_SIZE = "parquet.page.size";
public static final String COMPRESSION = "parquet.compression";
public static final String WRITE_SUPPORT_CLASS = "parquet.write.support.class";
public static final String DICTIONARY_PAGE_SIZE = "parquet.dictionary.page.size";
public static final String ENABLE_DICTIONARY = "parquet.enable.dictionary";
public static final String VALIDATION = "parquet.validation";
public static final String WRITER_VERSION = "parquet.writer.version";
public static final String MEMORY_POOL_RATIO = "parquet.memory.pool.ratio";
public static final String MIN_MEMORY_ALLOCATION = "parquet.memory.min.chunk.size";
public static final String MAX_PADDING_BYTES = "parquet.writer.max-padding";
public static final String MIN_ROW_COUNT_FOR_PAGE_SIZE_CHECK = "parquet.page.size.row.check.min";
public static final String MAX_ROW_COUNT_FOR_PAGE_SIZE_CHECK = "parquet.page.size.row.check.max";
public static final String ESTIMATE_PAGE_SIZE_CHECK = "parquet.page.size.check.estimate";
Orc
Apache Orc. 与Hive兼容。
Flink FileSystemConnector支持StreamingSink,依赖于Streaming File Sink写文件,还可参考:
Flink 1.11 可支持的编码和文件格式:
目前Flink Sql支持分区表和非分区表
分区目录内会有很多Part File,每个写某个分区的Sink子任务至少在该分区有一个文件。
in-progress
文件滚动配置如下:默认 | 类型 | 描述 | |
---|---|---|---|
sink.rolling-policy.file-size | 128MB | MemorySize | 文件滚动前最大大小 |
sink.rolling-policy.rollover-interval | 30 min | Duration | 文件滚动前最长打开时间 |
sink.rolling-policy.check-interval | 1 min | Duration | 检测按时间滚动策略的检测间隔时间 |
in-progress
文件滚动取决于Checkpoint间隔时间。
in-progress
变为pending
,此时不会再写入该文件,但也对读不可见finished
,对读可见。execution.checkpointing.interval
参数设置注意:本条目只适用于动态分区写入
写完一个分区后,可能需要通知下游应用,比如添加该分区到Hive元数据或在特定目录写一个_SUCCESS
文件。本Connector可允许配置自定义提交策略,依赖如下组件:
ProcessingTime
_SUCCESS
文件和元数据,也可以自己实现策略如合并小文件等。PartitionCommitTrigger
决定何时提交分区,分区提交的时间由从分区提取的时间生成的水位或者ProcessingTime
配置项 | 默认 | Type | 描述 |
---|---|---|---|
sink.partition-commit.trigger | process-time | String | 1.process-tim,表示依赖机器时间,无需水位。一旦当前系统时间超过分区创建时间+delay就提交分区。如果数据延迟会导致分区过早提交 2.partition-time,依赖从分区记录提取的时间戳,需要水位。一旦水位超过从分区值中提取的时间+delay,就提交分区 |
sink.partition-commit.delay | 0 s | Duration | 分区提交延迟时间,可填1d 天级分区,1h 表示小时级分区 |
process-time
,当watermark
> partition-time
+ sink.partition-commit.delay
时提交分区,partition-time
从PartitionTimeExtractor
获取分区代表的时间。
// 快照发生时调用
@Override
public void snapshotState(long checkpointId, long watermark) throws Exception {
// 清理老的pending分区状态
pendingPartitionsState.clear();
// 将当前pending分区加入状态,待放入快照
pendingPartitionsState.add(new ArrayList<>(pendingPartitions));
// 在本地内存记录该次checkpoin和水位时间戳对应关系
watermarks.put(checkpointId, watermark);
// 清理旧的水位状态
watermarksState.clear();
// 将新的checkpoint和水位对应关系放入状态,待放入快照
watermarksState.add(new HashMap<>(watermarks));
}
// 添加分区为pending待提交状态
// 比如 partitioned by (d string,e string)
// 则partition为 d=2020-05-03/e=8/
@Override
public void addPartition(String partition) {
if (!StringUtils.isNullOrWhitespaceOnly(partition)) {
// 将当前要提交的分区放入内存Set记录下来
this.pendingPartitions.add(partition);
}
}
// 获取checkpointId对应的可提交的分区
@Override
public List<String> committablePartitions(long checkpointId) {
if (!watermarks.containsKey(checkpointId)) {
throw new IllegalArgumentException(String.format(
"Checkpoint(%d) has not been snapshot. The watermark information is: %s.",
checkpointId, watermarks));
}
long watermark = watermarks.get(checkpointId);
// 将checkpointId小于等于当前checkpointId的全部从内存找哪个移除
watermarks.headMap(checkpointId, true).clear();
List<String> needCommit = new ArrayList<>();
// 遍历所有pending状态分区
Iterator<String> iter = pendingPartitions.iterator();
while (iter.hasNext()) {
String partition = iter.next();
// 传入分区列名list和分区列值解析后得到的list
// 得到无时间分区的LocalDateTime,如 2020-05-03T08:00
LocalDateTime partTime = extractor.extract(
partitionKeys, extractPartitionValues(new Path(partition)));
// 若当前水位大于 (partTime + commitDelay)就说明该分区需要提交
// 比如commitDelay为1h,而水位升到了2020-05-03 09:00:01,
// 则需要提交 2020-05-03T08:00 分区
if (watermark > toMills(partTime) + commitDelay) {
needCommit.add(partition);
iter.remove();
}
}
return needCommit;
}
current processing time
> partition creation time
+ sink.partition-commit.delay
时提交分区。
@Override
public List<String> committablePartitions(long checkpointId) {
List<String> needCommit = new ArrayList<>();
// 获取当前系统时间
long currentProcTime = procTimeService.getCurrentProcessingTime();
// 遍历pending状态的分区
Iterator<Map.Entry<String, Long>> iter = pendingPartitions.entrySet().iterator();
while (iter.hasNext()) {
Map.Entry<String, Long> entry = iter.next();
// value为addPartition时的系统时间
long creationTime = entry.getValue();
// sink.partition-commit.delay为0
// 或者 currentProcTime > creationTime + commitDelay
// 说明需要提交该分区,并从pending状态的分区记录中移除
if (commitDelay == 0 || currentProcTime > creationTime + commitDelay) {
needCommit.add(entry.getKey());
iter.remove();
}
}
// 最后返回需要提交的分区list
return needCommit;
}
配置项 | 默认 | Type | 描述 |
---|---|---|---|
partition.time-extractor.kind | default | String | 1.default,可配置时间提取格式 2.custom,配置时间提取器实现类 |
partition.time-extractor.class | (none) | String | 时间提取器实现类,实现自PartitionTimeExtractor |
partition.time-extractor.timestamp-pattern | (none) | String | 指定分区时间提取格式,默认支持从第一个字段以yyyy-mm-dd hh:mm:ss 提取。比如想从 dt 字段中提取,就配置为$dt 多个字段时,可以设为 $year-$month-$day $hour:00:00 |
PartitionTimeExtractor的默认实现类为org.apache.flink.table.filesystem.DefaultPartTimeExtractor
。
// partitionKeys为分区列名组成的list,如partitioned by (d string,e string)那就是 {d, e}
// partitionValues是由记录的pending分区列的值解析后构成的个分区列的值组成的list
// 如 d=2020-05-03/e=8/ ,解析为 {2020-05-03, 8}
@Override
public LocalDateTime extract(List<String> partitionKeys, List<String> partitionValues) {
String timestampString;
// partition.time-extractor.timestamp-pattern ,如 $d $e:00:00
if (pattern == null) {
// 如果不配置pattern,则取首个分区列的值作为timestampString
timestampString = partitionValues.get(0);
} else {
// 配置了pattern
timestampString = pattern;
// 将pattern内的所有分区列名替换为分区列的值
// 如 $d $e:00:00 -> 2020-05-03 8:00:00
for (int i = 0; i < partitionKeys.size(); i++) {
timestampString = timestampString.replaceAll(
"\\$" + partitionKeys.get(i),
partitionValues.get(i));
}
}
// 最后得到无时间分区的LocalDateTime,如 2020-05-03T08:00
return toLocalDateTime(timestampString);
}
public class HourPartTimeExtractor implements PartitionTimeExtractor {
@Override
public LocalDateTime extract(List<String> keys, List<String> values) {
String dt = values.get(0);
String hour = values.get(1);
return Timestamp.valueOf(dt + " " + hour + ":00:00").toLocalDateTime();
}
}
定义partition提交时的行为
配置项 | 默认 | Type | 描述 |
---|---|---|---|
sink.partition-commit.policy.kind | (none) | String | 当分区提交后通知下游应用分区可读的策略。 1.metastore,将分区添加到元数据中,仅支持Hive表 2.success-file,增加 _SUCCESS 文件到目录3.配置逗号分隔的多个策略,如 metastore,success-file 4.custom |
sink.partition-commit.policy.class | (none) | String | custom 时分区提交策略类,实现自PartitionCommitPolicy |
sink.partition-commit.success-file.name | (none) | String | 指定分区成功提交后生成的空文件名,默认_SUCCESS |
自带实现有:
_SUCCESS
可自定义分区提交后行为策略,实现PartitionCommitPolicy
接口,例子:
public class AnalysisCommitPolicy implements PartitionCommitPolicy {
private HiveShell hiveShell;
@Override
public void commit(Context context) throws Exception {
if (hiveShell == null) {
hiveShell = createHiveShell(context.catalogName());
}
hiveShell.execute(String.format("ALTER TABLE %s ADD IF NOT EXISTS PARTITION (%s = '%s') location '%s'",
context.tableName(),
context.partitionKeys().get(0),
context.partitionValues().get(0),
context.partitionPath()));
hiveShell.execute(String.format(
"ANALYZE TABLE %s PARTITION (%s = '%s') COMPUTE STATISTICS FOR COLUMNS",
context.tableName(),
context.partitionKeys().get(0),
context.partitionValues().get(0)));
}
}
StreamingFileWriter
流式写文件的时候,用此类来做分区提交。
本类是一个单实例、非并行的task,搜集从上游发来的所有分区信息,当判断已经收集到某个Checkpoint的所有task实例的分区信息时触发分区提交。
处理步骤为:
processElement
方法,接收到来自上游的分区信息,将ready的分区trigger.addPartition(partition)
TaskTracker#add
方法判断是否已经接收到所有属于某个CheckpointId
的上游task的分区数据trigger.committablePartitions
从trigger提取可提交的partition信息PartitionCommitPolicy链
内的所有PartitionCommitPolicy#commit
,执行用户定义的分区提交行为,如提交元数据到Hive、创建SUCCESS_FILE
文件等可以通过HiveTableSinkITCase#testPartStreamingWrite
测试用例来debug流式写入分区表。
也可以自己写demo来调试。
@Override
public void initializeState(StateInitializationContext context) throws Exception {
super.initializeState(context);
currentWatermark = Long.MIN_VALUE;
// 设置分区提交触发器,由 sink.partition-commit.trigger 指定
// 默认为ProcTimeCommitTigger
this.trigger = PartitionCommitTrigger.create(
context.isRestored(),
context.getOperatorStateStore(),
conf,
getUserCodeClassloader(),
partitionKeys,
getProcessingTimeService());
// 设置分区提交时的行为策略链,由 sink.partition-commit.policy.kind 指定
// 默认为空
this.policies = PartitionCommitPolicy.createPolicyChain(
getUserCodeClassloader(),
conf.get(SINK_PARTITION_COMMIT_POLICY_KIND),
conf.get(SINK_PARTITION_COMMIT_POLICY_CLASS),
conf.get(SINK_PARTITION_COMMIT_SUCCESS_FILE_NAME),
() -> {
try {
return fsFactory.create(locationPath.toUri());
} catch (IOException e) {
throw new RuntimeException(e);
}
});
}
PartitionCommitTrigger#snapshotState
,记录pendingPartition、水位状态@Override
public void snapshotState(StateSnapshotContext context) throws Exception {
super.snapshotState(context);
// 调用
trigger.snapshotState(context.getCheckpointId(), currentWatermark);
}
@Override
public void processWatermark(Watermark mark) throws Exception {
super.processWatermark(mark);
this.currentWatermark = mark.getTimestamp();
}
@Override
public void processElement(StreamRecord<CommitMessage> element) throws Exception {
// 写完一个分区后会发送一个message到这里
CommitMessage message = element.getValue();
// 将ready的分区添加到内存中的pendingPartitions
for (String partition : message.partitions) {
trigger.addPartition(partition);
}
// 用来监控上游任务,以决定是否已经接收到所有属于某个Checkpoint的上游数据
if (taskTracker == null) {
taskTracker = new TaskTracker(message.numberOfTasks);
}
// 当该checkpointId应该被提交时返回true
boolean needCommit = taskTracker.add(message.checkpointId, message.taskId);
if (needCommit) {
// 提交该partition
commitPartitions(message.checkpointId);
}
}
private void commitPartitions(long checkpointId) throws Exception {
// 得到可提交的分区,并清理内存中的pendingPartition
// 如果task结束,会发送值为Long.MAX_VALUE的checkpointId,调用endInput方法,以保证所有pendingPartition被提交
// 其他时候走committablePartitions,通过具体策略判断分区是否应该被提交
List<String> partitions = checkpointId == Long.MAX_VALUE ?
trigger.endInput() :
trigger.committablePartitions(checkpointId);
if (partitions.isEmpty()) {
return;
}
// 如HiveTableMetaStore
try (TableMetaStoreFactory.TableMetaStore metaStore = metaStoreFactory.createTableMetaStore()) {
for (String partition : partitions) {
// 得到所有分区列和值的映射关系组成的list
LinkedHashMap<String, String> partSpec = extractPartitionSpecFromPath(new Path(partition));
LOG.info("Partition {} of table {} is ready to be committed", partSpec, tableIdentifier);
// locationPath为表的location根目录 file:/var/warehouse/db1.db/sink_table
// 第二个参数为生成的分区路径,如 d=2020-05-03/e=8/
// 组合后的结果例子 file:/var/warehouse/db1.db/sink_table/d=2020-05-03/e=8
Path path = new Path(locationPath, generatePartitionPath(partSpec));
// 组合分区列和值以及分区路径
PartitionCommitPolicy.Context context = new PolicyContext(
new ArrayList<>(partSpec.values()), path);
for (PartitionCommitPolicy policy : policies) {
if (policy instanceof MetastoreCommitPolicy) {
// 如果有MetastoreCommitPolicy,就将如HiveTableMetaStore传递给他
((MetastoreCommitPolicy) policy).setMetastore(metaStore);
}
// 调用PartitionCommitPolicy#commit,做用户指定的分区提交行为
// 比如创建SUCCESS_FILE、向Hive元数据Server提交创建分区请求等
policy.commit(context);
}
}
}
}
从Kafka流式读取数据,流式写入FileSystem,并从fs_table
流式查询
CREATE TABLE kafka_table (
user_id STRING,
order_amount DOUBLE,
log_ts TIMESTAMP(3),
WATERMARK FOR log_ts AS log_ts - INTERVAL '5' SECOND
) WITH (...);
CREATE TABLE fs_table (
user_id STRING,
order_amount DOUBLE,
dt STRING,
hour STRING
) PARTITIONED BY (dt, hour) WITH (
'connector'='filesystem',
'path'='...',
'format'='parquet',
'sink.partition-commit.delay'='1 h',
'sink.partition-commit.policy.kind'='success-file'
);
-- streaming sql, insert into file system table
INSERT INTO TABLE fs_table SELECT user_id, order_amount, DATE_FORMAT(log_ts, 'yyyy-MM-dd'), DATE_FORMAT(log_ts, 'HH') FROM kafka_table;
-- batch sql, select with partition pruning
SELECT * FROM fs_table WHERE dt='2020-05-20' and hour='12';
目前Flink支持Hive相关功能有:
Flink兼容Hive 1.0到3.1等不同版本,所以具体功能受限于Hive版本:
在Flink使用Hive需要:
Hadoop相关依赖
将HADOOP_CLASSPATH
添加到环境变量,以备Flink启动Client、JM、TM等组件时使用
export HADOOP_CLASSPATH=hadoop classpath
添加依赖到$FLINK_HOME/lib
:
flink-sql-connector-hive-2.3.6.jar
hive-site.xml
推荐大家使用Apache Zeppelin访问,图形化界面,十分方便。参考:Flink-Zeppelin On FlinkSql
从Flink 1.11.0开始,Flink可直接使用Hive SQL,只要动态开启hive dialect选项即可:
set table.sql-dialect=hive;
使用hive方言,这样,Flink就可以和Hive互操作。
set table.sql-dialect=default;
使用默认的Flink Sql方言
目前支持的语法请参考:Hive Dialect
注意点:
is_generic=false
.Flink既可以批量读写Hive,也支持流式读写Hive,还能支持流式Join Hive。
HiveStreamingSink利用Filesystem Streaming Sink,集成了Hadoop OutputFormat/RecordWriter(批量编码、在每个Checkpoint滚动文件)来进行StreamingSink。
-- 采用hive sql方言
SET table.sql-dialect=hive;
-- 建立Hive表
CREATE TABLE hive_table (
user_id STRING,
order_amount DOUBLE
) PARTITIONED BY (dt STRING, hr STRING) STORED AS parquet TBLPROPERTIES (
'partition.time-extractor.timestamp-pattern'='$dt $hr:00:00',
'sink.partition-commit.trigger'='partition-time',
'sink.partition-commit.delay'='1 h',
'sink.partition-commit.policy.kind'='metastore,success-file'
);
-- 使用Flink sql方言
SET table.sql-dialect=default;
CREATE TABLE kafka_table (
user_id STRING,
order_amount DOUBLE,
log_ts TIMESTAMP(3),
WATERMARK FOR log_ts AS log_ts - INTERVAL '5' SECOND
) WITH (...);
-- streaming sql, insert into hive table
INSERT INTO TABLE hive_table SELECT user_id, order_amount, DATE_FORMAT(log_ts, 'yyyy-MM-dd'), DATE_FORMAT(log_ts, 'HH') FROM kafka_table;
-- batch sql, select with partition pruning
SELECT * FROM hive_table WHERE dt='2020-05-20' and hr='12';
使用如下语句建表
create table flink_meta.test_hive_20200729 (
f_sequence INT,
f_random INT,
f_random_str STRING
) PARTITIONED BY (dt STRING, hr STRING, mi STRING)
STORED AS parquet
TBLPROPERTIES (
'partition.time-extractor.timestamp-pattern'='$dt $hr:$mi:00',
'sink.partition-commit.trigger'='partition-time',
'sink.partition-commit.delay'='1 min',
'sink.partition-commit.policy.kind'='metastore,success-file'
);
建好后,可以直接在hive上查看该表:
createtab_stmt
CREATE TABLE `flink_meta.test_hive_20200729`(
`f_sequence` int,
`f_random` int,
`f_random_str` string)
PARTITIONED BY (
`dt` string,
`hr` string,
`mi` string)
ROW FORMAT SERDE
'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
STORED AS INPUTFORMAT
'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
OUTPUTFORMAT
'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
LOCATION
'hdfs://xxx/hive/flink_meta.db/test_hive_20200729'
TBLPROPERTIES (
'is_generic'='false',
'partition.time-extractor.timestamp-pattern'='$dt $hr:$mi:00',
'sink.partition-commit.delay'='1 min',
'sink.partition-commit.policy.kind'='metastore,success-file',
'sink.partition-commit.trigger'='partition-time',
'transient_lastDdlTime'='1596019921')
注意到该表is_generic
属性为false
,表示不是Flink专用表,我们用hive也可以查。
写入的数据在hdfs情况如下:
使用zeppelin执行flink查询该分区数据:
使用hive查询该分区数据:
select * from flink_meta.test_hive_20200729 where dt='2020-07-29'
Flink 1.11支持流式读Hive表,助力Hive实时数仓实现:
相关配置如下:
配置项 | 默认 | Type | 描述 |
---|---|---|---|
streaming-source.enable | false | Boolean | 是否启用HiveStreamingSource。 注意:必须确保每个分区/文件均应原子写入,否则可能会读到不完整的数据 |
streaming-source.monitor-interval | 1 m | Duration | 连续监控分区/文件的间隔时间 |
streaming-source.consume-order | create-time | String | 消费StreamingSource的顺序 1.create-time, 比较分区/文件在文件系统上的 modification time 。非分区表只能用该模式2. partition-time,比较分区名字代表的时间 |
streaming-source.consume-start-offset | 1970-00-00 | String | 消费起点分区。consume-order为create-time 和 partition-time 时使用时间戳字符串,格式为yyyy-[m]m-[d]d [hh:mm:ss] 。如果是partition-time ,会使用分区时间提取器来从分区中提取时间。 |
注意事项:
从Hive表的2020-05-20
分区开始增量流读取例子:
SELECT * FROM hive_table /*+ OPTIONS('streaming-source.enable'='true', 'streaming-source.consume-start-offset'='2020-05-20') */;
Hive表作为临时维表参与join时,数据会被缓存到TaskManager的内存中。需要join的Stream的每条数据都会从缓存中匹配Hive数据。
可以配合Hive数据缓存在TM中的TTL时间,当超过该事件会将缓存失效,并读取最新数据:
配置项 | 默认 | Type | 描述 |
---|---|---|---|
lookup.join.cache.ttl | 60 min | Duration | TM中缓存临时表TTL时间 |
注意事项:
lookup.join.cache.ttl
不能设的太小,否则频繁更新会造成性能问题当联用BlinkPlanner、HiveCatalog和Flink Hive Connector时,可直接读写Hive,这也是即MR、Tez、Spark后的又一个Hive执行引擎选项。
常用语句:
show catalogs;
use catalog myhive;
show databases;
show tables;
describe mytable;
SELECT * FROM mytable;
查询Hive视图:
insert into xxx select …:
-- append数据
INSERT INTO mytable SELECT 'Tom', 25;
-- 覆盖所有现存数据
INSERT OVERWRITE mytable SELECT 'Tom', 25;
插入数据到分区表,支持静态和动态分区。
以下例子中,Hive表有四列:name, age, my_type, my_date
,分区字段为my_type
my_date
:
# ------ Insert with static partition ------
Flink SQL> INSERT OVERWRITE myparttable PARTITION (my_type='type_1', my_date='2019-08-08') SELECT 'Tom', 25;
# ------ Insert with dynamic partition ------
Flink SQL> INSERT OVERWRITE myparttable SELECT 'Tom', 25, 'type_1', '2019-08-08';
# ------ Insert with static(my_type) and dynamic(my_date) partition ------
Flink SQL> INSERT OVERWRITE myparttable PARTITION (my_type='type_1') SELECT 'Tom', 25, '2019-08-08';
默认下Flink依据Split数量来推断FlinkHiveSource并发量,Split数量取决于目标文件数量和文件Block数。
可通过TableConfig
调整以下参数,需要注意这些参数会影响该job的所有Source:
limit
时,尽量让limit下推,以最大程度地减少跨网络传输的数据量。table.exec.hive.fallback-mapred-reader
这里我们使用HiveTableSinkITCase#testPartStreamingWrite
测试用例来进行debug,代码如下:
@Test(timeout = 120000)
public void testPartStreamingWrite() throws Exception {
testStreamingWrite(true, false, false, this::checkSuccessFiles);
}
testStreamingWrite:
private void testStreamingWrite(
boolean part,
boolean useMr,
boolean defaultSer,
Consumer<String> pathConsumer) throws Exception {
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setParallelism(1);
env.enableCheckpointing(100);
StreamTableEnvironment tEnv = HiveTestUtils.createTableEnvWithBlinkPlannerStreamMode(env);
tEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog);
tEnv.useCatalog(hiveCatalog.getName());
tEnv.getConfig().setSqlDialect(SqlDialect.HIVE);
if (useMr) {
tEnv.getConfig().getConfiguration().set(
HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_WRITER, true);
} else {
tEnv.getConfig().getConfiguration().set(
HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_WRITER, false);
}
try {
tEnv.executeSql("create database db1");
tEnv.useDatabase("db1");
// prepare source
List<Row> data = Arrays.asList(
Row.of(1, "a", "b", "2020-05-03", "7"),
Row.of(2, "p", "q", "2020-05-03", "8"),
Row.of(3, "x", "y", "2020-05-03", "9"),
Row.of(4, "x", "y", "2020-05-03", "10"),
Row.of(5, "x", "y", "2020-05-03", "11"));
DataStream<Row> stream = env.addSource(
new FiniteTestSource<>(data),
new RowTypeInfo(Types.INT, Types.STRING, Types.STRING, Types.STRING, Types.STRING));
tEnv.createTemporaryView("my_table", stream, $("a"), $("b"), $("c"), $("d"), $("e"));
// DDL
tEnv.executeSql("create external table sink_table (a int,b string,c string" +
(part ? "" : ",d string,e string") +
") " +
(part ? "partitioned by (d string,e string) " : "") +
(defaultSer ? "" : " stored as parquet") +
" TBLPROPERTIES (" +
// "'" + SINK_PARTITION_COMMIT_TRIGGER.key() + "'='partition-time'," +
"'" + PARTITION_TIME_EXTRACTOR_TIMESTAMP_PATTERN.key() + "'='$d $e:00:00'," +
"'" + SINK_PARTITION_COMMIT_DELAY.key() + "'='1h'," +
"'" + SINK_PARTITION_COMMIT_POLICY_KIND.key() + "'='metastore,success-file'," +
"'" + SINK_PARTITION_COMMIT_SUCCESS_FILE_NAME.key() + "'='_MY_SUCCESS'" +
")");
TableEnvUtil.execInsertTableAndWaitResult(
tEnv.sqlQuery("select * from my_table"),
"sink_table");
assertBatch("db1.sink_table", Arrays.asList(
"1,a,b,2020-05-03,7",
"1,a,b,2020-05-03,7",
"2,p,q,2020-05-03,8",
"2,p,q,2020-05-03,8",
"3,x,y,2020-05-03,9",
"3,x,y,2020-05-03,9",
"4,x,y,2020-05-03,10",
"4,x,y,2020-05-03,10",
"5,x,y,2020-05-03,11",
"5,x,y,2020-05-03,11"));
// using batch table env to query.
List<String> results = new ArrayList<>();
TableEnvironment batchTEnv = HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode();
batchTEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog);
batchTEnv.useCatalog(hiveCatalog.getName());
batchTEnv.executeSql("select * from db1.sink_table").collect()
.forEachRemaining(r -> results.add(r.toString()));
results.sort(String::compareTo);
Assert.assertEquals(
Arrays.asList(
"1,a,b,2020-05-03,7",
"1,a,b,2020-05-03,7",
"2,p,q,2020-05-03,8",
"2,p,q,2020-05-03,8",
"3,x,y,2020-05-03,9",
"3,x,y,2020-05-03,9",
"4,x,y,2020-05-03,10",
"4,x,y,2020-05-03,10",
"5,x,y,2020-05-03,11",
"5,x,y,2020-05-03,11"),
results);
pathConsumer.accept(URI.create(hiveCatalog.getHiveTable(
ObjectPath.fromString("db1.sink_table")).getSd().getLocation()).getPath());
} finally {
tEnv.executeSql("drop database db1 cascade");
}
}
其中
TableEnvUtil.execInsertTableAndWaitResult(
tEnv.sqlQuery("select * from my_table"),
"sink_table")
这一句就会开始流式地从临时视图表my_table
读取数据,然后流式写入Hive表sink_table
。
如该测试用例中的:
tEnv.sqlQuery("select * from my_table")
会在TableEnvironmentImpl#sqlQuery
中被解析:
parser#parse如下(使用blink):
@Override
public List<Operation> parse(String statement) {
CalciteParser parser = calciteParserSupplier.get();
FlinkPlannerImpl planner = validatorSupplier.get();
// parse the sql query
SqlNode parsed = parser.parse(statement);
Operation operation = SqlToOperationConverter.convert(planner, catalogManager, parsed)
.orElseThrow(() -> new TableException("Unsupported query: " + statement));
return Collections.singletonList(operation);
}
解析后,我们的原始sql转为了带有元数据和逻辑计划(calciteTree)的PlannerQueryOperation
对象。
随后利用该operation来构建一张动态表。
protected TableImpl createTable(QueryOperation tableOperation) {
return TableImpl.createTable(
this,
tableOperation,
operationTreeBuilder,
functionCatalog.asLookup(parser::parseIdentifier));
}
LookupCallResolver
解析完Source表以后,开始往Sink表插入数据。
经过一些列翻译和转换,会利用已有元数据信息来使用HiveTableFactory
创建HiveTableSink
。
会利用该工厂来创建HiveTableSink。
这里注意,我们的Hive表的is_generic
属性为false,表示不是flink专用表,则说明Hive的其他引擎也可以直接查该表。
会喂一个DataStream,返回消费该DataStream的DataStreamSink用来设置资源参数
这个方法内容很多,摘一部分:
@Override
public final DataStreamSink consumeDataStream(DataStream dataStream) {
// 分区列
String[] partitionColumns = getPartitionKeys().toArray(new String[0]);
// 库
String dbName = identifier.getDatabaseName();
// 表
String tableName = identifier.getObjectName();
// 创建到HiveMetastore的Client
try (HiveMetastoreClientWrapper client = HiveMetastoreClientFactory.create(
new HiveConf(jobConf, HiveConf.class), hiveVersion)) {
// 拿到该表在Hive中的元数据
Table table = client.getTable(dbName, tableName);
// 拿到该表的存储相关元数据,如location、input/outputFormat、压缩信息、分桶列、排序列、数据倾斜信息等
StorageDescriptor sd = table.getSd();
HiveTableMetaStoreFactory msFactory = new HiveTableMetaStoreFactory(
jobConf, hiveVersion, dbName, tableName);
HadoopFileSystemFactory fsFactory = new HadoopFileSystemFactory(jobConf);
Class hiveOutputFormatClz = hiveShim.getHiveOutputFormatClass(
Class.forName(sd.getOutputFormat()));
boolean isCompressed = jobConf.getBoolean(HiveConf.ConfVars.COMPRESSRESULT.varname, false);
// 创建写入时用到的RecordWriter 和 converter 工厂类
HiveWriterFactory recordWriterFactory = new HiveWriterFactory(
jobConf,
hiveOutputFormatClz,
sd.getSerdeInfo(),
tableSchema,
partitionColumns,
HiveReflectionUtils.getTableMetadata(hiveShim, table),
hiveShim,
isCompressed);
String extension = Utilities.getFileExtension(jobConf, isCompressed,
(HiveOutputFormat<?, ?>) hiveOutputFormatClz.newInstance());
// 决定输出的part-file的随机前缀(如part-6a836c0c-a9e2-4c32-9032-7f1732b2a7f6)和后缀
OutputFileConfig outputFileConfig = OutputFileConfig.builder()
.withPartPrefix("part-" + UUID.randomUUID().toString())
.withPartSuffix(extension == null ? "" : extension)
.build();
if (isBounded) {
// 有界数据处理
...
} else {
//无界数据处理
// 该sink表的属性,如sink.partition-commit.success-file.name -> _MY_SUCCESS
org.apache.flink.configuration.Configuration conf = new org.apache.flink.configuration.Configuration();
catalogTable.getOptions().forEach(conf::setString);
// Hive表分区计算器,有一个generatePartValues方法用来从RowData得到该条记录对应分区的值
// 还有个父类RowDataPartitionComputer的projectColumnsToWrite方法将原始数据剔除分区列数据
HiveRowDataPartitionComputer partComputer = new HiveRowDataPartitionComputer(
hiveShim,
jobConf.get(
HiveConf.ConfVars.DEFAULTPARTITIONNAME.varname,
HiveConf.ConfVars.DEFAULTPARTITIONNAME.defaultStrVal),
tableSchema.getFieldNames(),
tableSchema.getFieldDataTypes(),
partitionColumns);
// 文件系统Bucket分配器,包含了partComputer。从分区列的值来计算BucketId。
// StreamingFileSink支持自定义,但FlinkSql不支持
TableBucketAssigner assigner = new TableBucketAssigner(partComputer);
// 继承自CheckpointRollingPolicy,在每个Checkpoint时滚动part-file,以使之可读
// 不同的地方见后文分析
TableRollingPolicy rollingPolicy = new TableRollingPolicy(
true,
conf.get(SINK_ROLLING_POLICY_FILE_SIZE).getBytes(),
conf.get(SINK_ROLLING_POLICY_ROLLOVER_INTERVAL).toMillis());
// 批量模式下的Wrtier工厂,如ParquetWriterFactory,可创建输出Writer
// 目前StreamingFileS
Optional<BulkWriter.Factory<RowData>> bulkFactory = createBulkWriterFactory(partitionColumns, sd);
BucketsBuilder<RowData, String, ? extends BucketsBuilder<RowData, ?, ?>> builder;
if (userMrWriter || !bulkFactory.isPresent()) {
HiveBulkWriterFactory hadoopBulkFactory = new HiveBulkWriterFactory(recordWriterFactory);
builder = new HadoopPathBasedBulkFormatBuilder<>(
new Path(sd.getLocation()), hadoopBulkFactory, jobConf, assigner)
.withRollingPolicy(rollingPolicy)
.withOutputFileConfig(outputFileConfig);
LOG.info("Hive streaming sink: Use MapReduce RecordWriter writer.");
} else {
builder = StreamingFileSink.forBulkFormat(
new org.apache.flink.core.fs.Path(sd.getLocation()),
new FileSystemTableSink.ProjectionBulkFactory(bulkFactory.get(), partComputer))
.withBucketAssigner(assigner)
.withRollingPolicy(rollingPolicy)
.withOutputFileConfig(outputFileConfig);
LOG.info("Hive streaming sink: Use native parquet&orc writer.");
}
return FileSystemTableSink.createStreamingSink(
conf,
new org.apache.flink.core.fs.Path(sd.getLocation()),
getPartitionKeys(),
identifier,
overwrite,
dataStream,
builder,
msFactory,
fsFactory,
conf.get(SINK_ROLLING_POLICY_CHECK_INTERVAL).toMillis());
}
} catch (TException e) {
...
}
}
继承自CheckpointRollingPolicy,在每个Checkpoint时滚动part-file,以使之可读。
和DataStream API里的实现OnCheckpointRollingPolicy
有所不同:
sink.rolling-policy.file-size
(默认128MB),而非总是truereturn rollOnCheckpoint || partFileState.getSize() > rollingFileSize;
shouldRollOnEvent
根据partFile大小是否超过配置的sink.rolling-policy.file-size
(默认128MB),而非总是false。
会在每条数据写入的时候判断,具体见Bucket#write
return partFileState.getSize() > rollingFileSize;
shouldRollOnProcessingTime
根据当前时间减去partFile的创建时间否超过配置的sink.rolling-policy.rollover-interval
(默认30min),而非总是false。
根据sink.rolling-policy.check-interval
配置的时间,StreamingFileSinkHelper
会调用ProcessingTimeService#registerTimer(currentProcessingTime + bucketCheckInterval, this)
,来定时调用buckets.onProcessingTime(currentTime)
方法,内部就会判断活跃Bucket中的正在写入的partFile是否应该更新,具体可见Bucket#onProcessingTime
@Override
public boolean shouldRollOnProcessingTime(
PartFileInfo<String> partFileState,
long currentTime) {
return currentTime - partFileState.getCreationTime() >= rollingTimeInterval;
}