[HUDI-3665] Support flink multiple versions (#5072)
This commit is contained in:
364
hudi-flink-datasource/hudi-flink/pom.xml
Normal file
364
hudi-flink-datasource/hudi-flink/pom.xml
Normal file
@@ -0,0 +1,364 @@
|
||||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
|
||||
<!--
|
||||
Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
contributor license agreements. See the NOTICE file distributed with
|
||||
this work for additional information regarding copyright ownership.
|
||||
The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
(the "License"); you may not use this file except in compliance with
|
||||
the License. You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
-->
|
||||
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<parent>
|
||||
<artifactId>hudi-flink-datasource</artifactId>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<version>0.11.0-SNAPSHOT</version>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
<artifactId>hudi-flink</artifactId>
|
||||
<version>0.11.0-SNAPSHOT</version>
|
||||
<packaging>jar</packaging>
|
||||
|
||||
<properties>
|
||||
<main.basedir>${project.parent.parent.basedir}</main.basedir>
|
||||
<parquet.version>1.11.1</parquet.version>
|
||||
</properties>
|
||||
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.jacoco</groupId>
|
||||
<artifactId>jacoco-maven-plugin</artifactId>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-compiler-plugin</artifactId>
|
||||
<configuration>
|
||||
<source>1.8</source>
|
||||
<target>1.8</target>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-jar-plugin</artifactId>
|
||||
<version>3.1.2</version>
|
||||
<executions>
|
||||
<execution>
|
||||
<goals>
|
||||
<goal>test-jar</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.rat</groupId>
|
||||
<artifactId>apache-rat-plugin</artifactId>
|
||||
</plugin>
|
||||
</plugins>
|
||||
|
||||
<resources>
|
||||
<resource>
|
||||
<directory>src/main/resources</directory>
|
||||
</resource>
|
||||
<resource>
|
||||
<directory>src/test/resources</directory>
|
||||
</resource>
|
||||
</resources>
|
||||
</build>
|
||||
|
||||
<dependencies>
|
||||
<!-- Hoodie -->
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-common</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-client-common</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-flink-client</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-hadoop-mr</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-hive-sync</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-sync-common</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>${hudi.flink.module}</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
|
||||
<!-- Flink -->
|
||||
<dependency>
|
||||
<groupId>org.apache.flink</groupId>
|
||||
<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.flink</groupId>
|
||||
<artifactId>flink-clients_${scala.binary.version}</artifactId>
|
||||
<scope>compile</scope>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>com.esotericsoftware.kryo</groupId>
|
||||
<artifactId>kryo</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>com.esotericsoftware.minlog</groupId>
|
||||
<artifactId>minlog</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.flink</groupId>
|
||||
<artifactId>flink-connector-kafka_${scala.binary.version}</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.kafka</groupId>
|
||||
<artifactId>kafka-clients</artifactId>
|
||||
<version>${kafka.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.flink</groupId>
|
||||
<artifactId>flink-hadoop-compatibility_${scala.binary.version}</artifactId>
|
||||
<version>${flink.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.flink</groupId>
|
||||
<artifactId>flink-parquet_${scala.binary.version}</artifactId>
|
||||
<version>${flink.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.flink</groupId>
|
||||
<artifactId>flink-json</artifactId>
|
||||
<version>${flink.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.flink</groupId>
|
||||
<artifactId>flink-table-common</artifactId>
|
||||
<version>${flink.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.flink</groupId>
|
||||
<artifactId>${flink.table.runtime.artifactId}</artifactId>
|
||||
<version>${flink.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.flink</groupId>
|
||||
<artifactId>${flink.table.planner.artifactId}</artifactId>
|
||||
<version>${flink.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.flink</groupId>
|
||||
<artifactId>flink-statebackend-rocksdb_${scala.binary.version}</artifactId>
|
||||
<version>${flink.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.parquet</groupId>
|
||||
<artifactId>parquet-hadoop</artifactId>
|
||||
<version>${parquet.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.xerial.snappy</groupId>
|
||||
<artifactId>snappy-java</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<!-- Parquet -->
|
||||
<dependency>
|
||||
<groupId>org.apache.parquet</groupId>
|
||||
<artifactId>parquet-avro</artifactId>
|
||||
<version>${parquet.version}</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<!-- Avro -->
|
||||
<dependency>
|
||||
<groupId>org.apache.avro</groupId>
|
||||
<artifactId>avro</artifactId>
|
||||
<!-- Override the version to be same with Flink avro -->
|
||||
<version>1.10.0</version>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
|
||||
<!-- Hadoop -->
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-mapreduce-client-core</artifactId>
|
||||
<scope>compile</scope>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>slf4j-log4j12</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>com.beust</groupId>
|
||||
<artifactId>jcommander</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.twitter</groupId>
|
||||
<artifactId>bijection-avro_${scala.binary.version}</artifactId>
|
||||
<version>0.9.7</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>joda-time</groupId>
|
||||
<artifactId>joda-time</artifactId>
|
||||
<version>2.5</version>
|
||||
</dependency>
|
||||
<!-- Hive -->
|
||||
<dependency>
|
||||
<groupId>${hive.groupid}</groupId>
|
||||
<artifactId>hive-exec</artifactId>
|
||||
<version>${hive.version}</version>
|
||||
<classifier>${hive.exec.classifier}</classifier>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>javax.mail</groupId>
|
||||
<artifactId>mail</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.eclipse.jetty.aggregate</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<!-- Test dependencies -->
|
||||
|
||||
<!-- Junit 5 dependencies -->
|
||||
<dependency>
|
||||
<groupId>org.junit.jupiter</groupId>
|
||||
<artifactId>junit-jupiter-api</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.junit.jupiter</groupId>
|
||||
<artifactId>junit-jupiter-engine</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.junit.vintage</groupId>
|
||||
<artifactId>junit-vintage-engine</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.junit.jupiter</groupId>
|
||||
<artifactId>junit-jupiter-params</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<!-- Hoodie dependencies -->
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-common</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<classifier>tests</classifier>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-client-common</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<classifier>tests</classifier>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-flink-client</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<classifier>tests</classifier>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>${hudi.flink.module}</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<classifier>tests</classifier>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<!-- Flink dependencies -->
|
||||
<dependency>
|
||||
<groupId>org.apache.flink</groupId>
|
||||
<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
|
||||
<version>${flink.version}</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.flink</groupId>
|
||||
<artifactId>${flink.runtime.artifactId}</artifactId>
|
||||
<version>${flink.version}</version>
|
||||
<scope>test</scope>
|
||||
<type>test-jar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.flink</groupId>
|
||||
<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
|
||||
<version>${flink.version}</version>
|
||||
<scope>test</scope>
|
||||
<type>test-jar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.flink</groupId>
|
||||
<artifactId>${flink.table.runtime.artifactId}</artifactId>
|
||||
<version>${flink.version}</version>
|
||||
<scope>test</scope>
|
||||
<type>test-jar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.flink</groupId>
|
||||
<artifactId>flink-json</artifactId>
|
||||
<version>${flink.version}</version>
|
||||
<scope>test</scope>
|
||||
<type>test-jar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.flink</groupId>
|
||||
<artifactId>flink-csv</artifactId>
|
||||
<version>${flink.version}</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
||||
@@ -0,0 +1,785 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.configuration;
|
||||
|
||||
import org.apache.hudi.common.config.ConfigClassProperty;
|
||||
import org.apache.hudi.common.config.ConfigGroups;
|
||||
import org.apache.hudi.common.config.HoodieConfig;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorType;
|
||||
|
||||
import org.apache.flink.configuration.ConfigOption;
|
||||
import org.apache.flink.configuration.ConfigOptions;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
|
||||
import java.lang.reflect.Field;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Hoodie Flink config options.
|
||||
*
|
||||
* <p>It has the options for Hoodie table read and write. It also defines some utilities.
|
||||
*/
|
||||
@ConfigClassProperty(name = "Flink Options",
|
||||
groupName = ConfigGroups.Names.FLINK_SQL,
|
||||
description = "Flink jobs using the SQL can be configured through the options in WITH clause."
|
||||
+ " The actual datasource level configs are listed below.")
|
||||
public class FlinkOptions extends HoodieConfig {
|
||||
private FlinkOptions() {
|
||||
}
|
||||
|
||||
// ------------------------------------------------------------------------
|
||||
// Base Options
|
||||
// ------------------------------------------------------------------------
|
||||
public static final ConfigOption<String> PATH = ConfigOptions
|
||||
.key("path")
|
||||
.stringType()
|
||||
.noDefaultValue()
|
||||
.withDescription("Base path for the target hoodie table.\n"
|
||||
+ "The path would be created if it does not exist,\n"
|
||||
+ "otherwise a Hoodie table expects to be initialized successfully");
|
||||
|
||||
// ------------------------------------------------------------------------
|
||||
// Common Options
|
||||
// ------------------------------------------------------------------------
|
||||
|
||||
public static final ConfigOption<String> PARTITION_DEFAULT_NAME = ConfigOptions
|
||||
.key("partition.default_name")
|
||||
.stringType()
|
||||
.defaultValue("default") // keep sync with hoodie style
|
||||
.withDescription("The default partition name in case the dynamic partition"
|
||||
+ " column value is null/empty string");
|
||||
|
||||
public static final ConfigOption<Boolean> CHANGELOG_ENABLED = ConfigOptions
|
||||
.key("changelog.enabled")
|
||||
.booleanType()
|
||||
.defaultValue(false)
|
||||
.withDescription("Whether to keep all the intermediate changes, "
|
||||
+ "we try to keep all the changes of a record when enabled:\n"
|
||||
+ "1). The sink accept the UPDATE_BEFORE message;\n"
|
||||
+ "2). The source try to emit every changes of a record.\n"
|
||||
+ "The semantics is best effort because the compaction job would finally merge all changes of a record into one.\n"
|
||||
+ " default false to have UPSERT semantics");
|
||||
|
||||
// ------------------------------------------------------------------------
|
||||
// Metadata table Options
|
||||
// ------------------------------------------------------------------------
|
||||
|
||||
public static final ConfigOption<Boolean> METADATA_ENABLED = ConfigOptions
|
||||
.key("metadata.enabled")
|
||||
.booleanType()
|
||||
.defaultValue(false)
|
||||
.withDescription("Enable the internal metadata table which serves table metadata like level file listings, default false");
|
||||
|
||||
public static final ConfigOption<Integer> METADATA_COMPACTION_DELTA_COMMITS = ConfigOptions
|
||||
.key("metadata.compaction.delta_commits")
|
||||
.intType()
|
||||
.defaultValue(10)
|
||||
.withDescription("Max delta commits for metadata table to trigger compaction, default 24");
|
||||
|
||||
// ------------------------------------------------------------------------
|
||||
// Index Options
|
||||
// ------------------------------------------------------------------------
|
||||
public static final ConfigOption<String> INDEX_TYPE = ConfigOptions
|
||||
.key("index.type")
|
||||
.stringType()
|
||||
.defaultValue(HoodieIndex.IndexType.FLINK_STATE.name())
|
||||
.withDescription("Index type of Flink write job, default is using state backed index.");
|
||||
|
||||
public static final ConfigOption<Boolean> INDEX_BOOTSTRAP_ENABLED = ConfigOptions
|
||||
.key("index.bootstrap.enabled")
|
||||
.booleanType()
|
||||
.defaultValue(false)
|
||||
.withDescription("Whether to bootstrap the index state from existing hoodie table, default false");
|
||||
|
||||
public static final ConfigOption<Double> INDEX_STATE_TTL = ConfigOptions
|
||||
.key("index.state.ttl")
|
||||
.doubleType()
|
||||
.defaultValue(0D)
|
||||
.withDescription("Index state ttl in days, default stores the index permanently");
|
||||
|
||||
public static final ConfigOption<Boolean> INDEX_GLOBAL_ENABLED = ConfigOptions
|
||||
.key("index.global.enabled")
|
||||
.booleanType()
|
||||
.defaultValue(true)
|
||||
.withDescription("Whether to update index for the old partition path\n"
|
||||
+ "if same key record with different partition path came in, default true");
|
||||
|
||||
public static final ConfigOption<String> INDEX_PARTITION_REGEX = ConfigOptions
|
||||
.key("index.partition.regex")
|
||||
.stringType()
|
||||
.defaultValue(".*")
|
||||
.withDescription("Whether to load partitions in state if partition path matching, default *");
|
||||
|
||||
// ------------------------------------------------------------------------
|
||||
// Read Options
|
||||
// ------------------------------------------------------------------------
|
||||
public static final ConfigOption<Integer> READ_TASKS = ConfigOptions
|
||||
.key("read.tasks")
|
||||
.intType()
|
||||
.defaultValue(4)
|
||||
.withDescription("Parallelism of tasks that do actual read, default is 4");
|
||||
|
||||
public static final ConfigOption<String> SOURCE_AVRO_SCHEMA_PATH = ConfigOptions
|
||||
.key("source.avro-schema.path")
|
||||
.stringType()
|
||||
.noDefaultValue()
|
||||
.withDescription("Source avro schema file path, the parsed schema is used for deserialization");
|
||||
|
||||
public static final ConfigOption<String> SOURCE_AVRO_SCHEMA = ConfigOptions
|
||||
.key("source.avro-schema")
|
||||
.stringType()
|
||||
.noDefaultValue()
|
||||
.withDescription("Source avro schema string, the parsed schema is used for deserialization");
|
||||
|
||||
public static final String QUERY_TYPE_SNAPSHOT = "snapshot";
|
||||
public static final String QUERY_TYPE_READ_OPTIMIZED = "read_optimized";
|
||||
public static final String QUERY_TYPE_INCREMENTAL = "incremental";
|
||||
public static final ConfigOption<String> QUERY_TYPE = ConfigOptions
|
||||
.key("hoodie.datasource.query.type")
|
||||
.stringType()
|
||||
.defaultValue(QUERY_TYPE_SNAPSHOT)
|
||||
.withDescription("Decides how data files need to be read, in\n"
|
||||
+ "1) Snapshot mode (obtain latest view, based on row & columnar data);\n"
|
||||
+ "2) incremental mode (new data since an instantTime);\n"
|
||||
+ "3) Read Optimized mode (obtain latest view, based on columnar data)\n."
|
||||
+ "Default: snapshot");
|
||||
|
||||
public static final String REALTIME_SKIP_MERGE = "skip_merge";
|
||||
public static final String REALTIME_PAYLOAD_COMBINE = "payload_combine";
|
||||
public static final ConfigOption<String> MERGE_TYPE = ConfigOptions
|
||||
.key("hoodie.datasource.merge.type")
|
||||
.stringType()
|
||||
.defaultValue(REALTIME_PAYLOAD_COMBINE)
|
||||
.withDescription("For Snapshot query on merge on read table. Use this key to define how the payloads are merged, in\n"
|
||||
+ "1) skip_merge: read the base file records plus the log file records;\n"
|
||||
+ "2) payload_combine: read the base file records first, for each record in base file, checks whether the key is in the\n"
|
||||
+ " log file records(combines the two records with same key for base and log file records), then read the left log file records");
|
||||
|
||||
public static final ConfigOption<Boolean> UTC_TIMEZONE = ConfigOptions
|
||||
.key("read.utc-timezone")
|
||||
.booleanType()
|
||||
.defaultValue(true)
|
||||
.withDescription("Use UTC timezone or local timezone to the conversion between epoch"
|
||||
+ " time and LocalDateTime. Hive 0.x/1.x/2.x use local timezone. But Hive 3.x"
|
||||
+ " use UTC timezone, by default true");
|
||||
|
||||
public static final ConfigOption<Boolean> READ_AS_STREAMING = ConfigOptions
|
||||
.key("read.streaming.enabled")
|
||||
.booleanType()
|
||||
.defaultValue(false)// default read as batch
|
||||
.withDescription("Whether to read as streaming source, default false");
|
||||
|
||||
public static final ConfigOption<Integer> READ_STREAMING_CHECK_INTERVAL = ConfigOptions
|
||||
.key("read.streaming.check-interval")
|
||||
.intType()
|
||||
.defaultValue(60)// default 1 minute
|
||||
.withDescription("Check interval for streaming read of SECOND, default 1 minute");
|
||||
|
||||
// this option is experimental
|
||||
public static final ConfigOption<Boolean> READ_STREAMING_SKIP_COMPACT = ConfigOptions
|
||||
.key("read.streaming.skip_compaction")
|
||||
.booleanType()
|
||||
.defaultValue(false)// default read as batch
|
||||
.withDescription("Whether to skip compaction instants for streaming read,\n"
|
||||
+ "there are two cases that this option can be used to avoid reading duplicates:\n"
|
||||
+ "1) you are definitely sure that the consumer reads faster than any compaction instants, "
|
||||
+ "usually with delta time compaction strategy that is long enough, for e.g, one week;\n"
|
||||
+ "2) changelog mode is enabled, this option is a solution to keep data integrity");
|
||||
|
||||
public static final String START_COMMIT_EARLIEST = "earliest";
|
||||
public static final ConfigOption<String> READ_START_COMMIT = ConfigOptions
|
||||
.key("read.start-commit")
|
||||
.stringType()
|
||||
.noDefaultValue()
|
||||
.withDescription("Start commit instant for reading, the commit time format should be 'yyyyMMddHHmmss', "
|
||||
+ "by default reading from the latest instant for streaming read");
|
||||
|
||||
public static final ConfigOption<String> READ_END_COMMIT = ConfigOptions
|
||||
.key("read.end-commit")
|
||||
.stringType()
|
||||
.noDefaultValue()
|
||||
.withDescription("End commit instant for reading, the commit time format should be 'yyyyMMddHHmmss'");
|
||||
|
||||
// ------------------------------------------------------------------------
|
||||
// Write Options
|
||||
// ------------------------------------------------------------------------
|
||||
public static final ConfigOption<String> TABLE_NAME = ConfigOptions
|
||||
.key(HoodieWriteConfig.TBL_NAME.key())
|
||||
.stringType()
|
||||
.noDefaultValue()
|
||||
.withDescription("Table name to register to Hive metastore");
|
||||
|
||||
public static final String TABLE_TYPE_COPY_ON_WRITE = HoodieTableType.COPY_ON_WRITE.name();
|
||||
public static final String TABLE_TYPE_MERGE_ON_READ = HoodieTableType.MERGE_ON_READ.name();
|
||||
public static final ConfigOption<String> TABLE_TYPE = ConfigOptions
|
||||
.key("table.type")
|
||||
.stringType()
|
||||
.defaultValue(TABLE_TYPE_COPY_ON_WRITE)
|
||||
.withDescription("Type of table to write. COPY_ON_WRITE (or) MERGE_ON_READ");
|
||||
|
||||
public static final ConfigOption<Boolean> INSERT_CLUSTER = ConfigOptions
|
||||
.key("write.insert.cluster")
|
||||
.booleanType()
|
||||
.defaultValue(false)
|
||||
.withDescription("Whether to merge small files for insert mode, "
|
||||
+ "if true, the write throughput will decrease because the read/write of existing small file, "
|
||||
+ "only valid for COW table, default false");
|
||||
|
||||
public static final ConfigOption<String> OPERATION = ConfigOptions
|
||||
.key("write.operation")
|
||||
.stringType()
|
||||
.defaultValue("upsert")
|
||||
.withDescription("The write operation, that this write should do");
|
||||
|
||||
public static final String NO_PRE_COMBINE = "no_precombine";
|
||||
public static final ConfigOption<String> PRECOMBINE_FIELD = ConfigOptions
|
||||
.key("write.precombine.field")
|
||||
.stringType()
|
||||
.defaultValue("ts")
|
||||
.withDescription("Field used in preCombining before actual write. When two records have the same\n"
|
||||
+ "key value, we will pick the one with the largest value for the precombine field,\n"
|
||||
+ "determined by Object.compareTo(..)");
|
||||
|
||||
public static final ConfigOption<String> PAYLOAD_CLASS_NAME = ConfigOptions
|
||||
.key("write.payload.class")
|
||||
.stringType()
|
||||
.defaultValue(OverwriteWithLatestAvroPayload.class.getName())
|
||||
.withDescription("Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting.\n"
|
||||
+ "This will render any value set for the option in-effective");
|
||||
|
||||
/**
|
||||
* Flag to indicate whether to drop duplicates before insert/upsert.
|
||||
* By default false to gain extra performance.
|
||||
*/
|
||||
public static final ConfigOption<Boolean> PRE_COMBINE = ConfigOptions
|
||||
.key("write.precombine")
|
||||
.booleanType()
|
||||
.defaultValue(false)
|
||||
.withDescription("Flag to indicate whether to drop duplicates before insert/upsert.\n"
|
||||
+ "By default these cases will accept duplicates, to gain extra performance:\n"
|
||||
+ "1) insert operation;\n"
|
||||
+ "2) upsert for MOR table, the MOR table deduplicate on reading");
|
||||
|
||||
public static final ConfigOption<Integer> RETRY_TIMES = ConfigOptions
|
||||
.key("write.retry.times")
|
||||
.intType()
|
||||
.defaultValue(3)
|
||||
.withDescription("Flag to indicate how many times streaming job should retry for a failed checkpoint batch.\n"
|
||||
+ "By default 3");
|
||||
|
||||
public static final ConfigOption<Long> RETRY_INTERVAL_MS = ConfigOptions
|
||||
.key("write.retry.interval.ms")
|
||||
.longType()
|
||||
.defaultValue(2000L)
|
||||
.withDescription("Flag to indicate how long (by millisecond) before a retry should issued for failed checkpoint batch.\n"
|
||||
+ "By default 2000 and it will be doubled by every retry");
|
||||
|
||||
public static final ConfigOption<Boolean> IGNORE_FAILED = ConfigOptions
|
||||
.key("write.ignore.failed")
|
||||
.booleanType()
|
||||
.defaultValue(true)
|
||||
.withDescription("Flag to indicate whether to ignore any non exception error (e.g. writestatus error). within a checkpoint batch.\n"
|
||||
+ "By default true (in favor of streaming progressing over data integrity)");
|
||||
|
||||
public static final ConfigOption<String> RECORD_KEY_FIELD = ConfigOptions
|
||||
.key(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key())
|
||||
.stringType()
|
||||
.defaultValue("uuid")
|
||||
.withDescription("Record key field. Value to be used as the `recordKey` component of `HoodieKey`.\n"
|
||||
+ "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using "
|
||||
+ "the dot notation eg: `a.b.c`");
|
||||
|
||||
public static final ConfigOption<String> INDEX_KEY_FIELD = ConfigOptions
|
||||
.key(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key())
|
||||
.stringType()
|
||||
.defaultValue("")
|
||||
.withDescription("Index key field. Value to be used as hashing to find the bucket ID. Should be a subset of or equal to the recordKey fields.\n"
|
||||
+ "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using "
|
||||
+ "the dot notation eg: `a.b.c`");
|
||||
|
||||
public static final ConfigOption<Integer> BUCKET_INDEX_NUM_BUCKETS = ConfigOptions
|
||||
.key(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key())
|
||||
.intType()
|
||||
.defaultValue(4) // default 4 buckets per partition
|
||||
.withDescription("Hudi bucket number per partition. Only affected if using Hudi bucket index.");
|
||||
|
||||
public static final ConfigOption<String> PARTITION_PATH_FIELD = ConfigOptions
|
||||
.key(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key())
|
||||
.stringType()
|
||||
.defaultValue("")
|
||||
.withDescription("Partition path field. Value to be used at the `partitionPath` component of `HoodieKey`.\n"
|
||||
+ "Actual value obtained by invoking .toString(), default ''");
|
||||
|
||||
public static final ConfigOption<Boolean> URL_ENCODE_PARTITIONING = ConfigOptions
|
||||
.key(KeyGeneratorOptions.URL_ENCODE_PARTITIONING.key())
|
||||
.booleanType()
|
||||
.defaultValue(false)
|
||||
.withDescription("Whether to encode the partition path url, default false");
|
||||
|
||||
public static final ConfigOption<Boolean> HIVE_STYLE_PARTITIONING = ConfigOptions
|
||||
.key(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key())
|
||||
.booleanType()
|
||||
.defaultValue(false)
|
||||
.withDescription("Whether to use Hive style partitioning.\n"
|
||||
+ "If set true, the names of partition folders follow <partition_column_name>=<partition_value> format.\n"
|
||||
+ "By default false (the names of partition folders are only partition values)");
|
||||
|
||||
public static final ConfigOption<String> KEYGEN_CLASS_NAME = ConfigOptions
|
||||
.key(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key())
|
||||
.stringType()
|
||||
.defaultValue("")
|
||||
.withDescription("Key generator class, that implements will extract the key out of incoming record");
|
||||
|
||||
public static final ConfigOption<String> KEYGEN_TYPE = ConfigOptions
|
||||
.key(HoodieWriteConfig.KEYGENERATOR_TYPE.key())
|
||||
.stringType()
|
||||
.defaultValue(KeyGeneratorType.SIMPLE.name())
|
||||
.withDescription("Key generator type, that implements will extract the key out of incoming record");
|
||||
|
||||
public static final String PARTITION_FORMAT_HOUR = "yyyyMMddHH";
|
||||
public static final String PARTITION_FORMAT_DAY = "yyyyMMdd";
|
||||
public static final ConfigOption<String> PARTITION_FORMAT = ConfigOptions
|
||||
.key("write.partition.format")
|
||||
.stringType()
|
||||
.noDefaultValue()
|
||||
.withDescription("Partition path format, only valid when 'write.datetime.partitioning' is true, default is:\n"
|
||||
+ "1) 'yyyyMMddHH' for timestamp(3) WITHOUT TIME ZONE, LONG, FLOAT, DOUBLE, DECIMAL;\n"
|
||||
+ "2) 'yyyyMMdd' for DAY and INT.");
|
||||
|
||||
public static final ConfigOption<Integer> INDEX_BOOTSTRAP_TASKS = ConfigOptions
|
||||
.key("write.index_bootstrap.tasks")
|
||||
.intType()
|
||||
.noDefaultValue()
|
||||
.withDescription("Parallelism of tasks that do index bootstrap, default is the parallelism of the execution environment");
|
||||
|
||||
public static final ConfigOption<Integer> BUCKET_ASSIGN_TASKS = ConfigOptions
|
||||
.key("write.bucket_assign.tasks")
|
||||
.intType()
|
||||
.noDefaultValue()
|
||||
.withDescription("Parallelism of tasks that do bucket assign, default is the parallelism of the execution environment");
|
||||
|
||||
public static final ConfigOption<Integer> WRITE_TASKS = ConfigOptions
|
||||
.key("write.tasks")
|
||||
.intType()
|
||||
.defaultValue(4)
|
||||
.withDescription("Parallelism of tasks that do actual write, default is 4");
|
||||
|
||||
public static final ConfigOption<Double> WRITE_TASK_MAX_SIZE = ConfigOptions
|
||||
.key("write.task.max.size")
|
||||
.doubleType()
|
||||
.defaultValue(1024D) // 1GB
|
||||
.withDescription("Maximum memory in MB for a write task, when the threshold hits,\n"
|
||||
+ "it flushes the max size data bucket to avoid OOM, default 1GB");
|
||||
|
||||
public static final ConfigOption<Long> WRITE_RATE_LIMIT = ConfigOptions
|
||||
.key("write.rate.limit")
|
||||
.longType()
|
||||
.defaultValue(0L) // default no limit
|
||||
.withDescription("Write record rate limit per second to prevent traffic jitter and improve stability, default 0 (no limit)");
|
||||
|
||||
public static final ConfigOption<Double> WRITE_BATCH_SIZE = ConfigOptions
|
||||
.key("write.batch.size")
|
||||
.doubleType()
|
||||
.defaultValue(256D) // 256MB
|
||||
.withDescription("Batch buffer size in MB to flush data into the underneath filesystem, default 256MB");
|
||||
|
||||
public static final ConfigOption<Integer> WRITE_LOG_BLOCK_SIZE = ConfigOptions
|
||||
.key("write.log_block.size")
|
||||
.intType()
|
||||
.defaultValue(128)
|
||||
.withDescription("Max log block size in MB for log file, default 128MB");
|
||||
|
||||
public static final ConfigOption<Long> WRITE_LOG_MAX_SIZE = ConfigOptions
|
||||
.key("write.log.max.size")
|
||||
.longType()
|
||||
.defaultValue(1024L)
|
||||
.withDescription("Maximum size allowed in MB for a log file before it is rolled over to the next version, default 1GB");
|
||||
|
||||
public static final ConfigOption<Integer> WRITE_PARQUET_BLOCK_SIZE = ConfigOptions
|
||||
.key("write.parquet.block.size")
|
||||
.intType()
|
||||
.defaultValue(120)
|
||||
.withDescription("Parquet RowGroup size. It's recommended to make this large enough that scan costs can be"
|
||||
+ " amortized by packing enough column values into a single row group.");
|
||||
|
||||
public static final ConfigOption<Integer> WRITE_PARQUET_MAX_FILE_SIZE = ConfigOptions
|
||||
.key("write.parquet.max.file.size")
|
||||
.intType()
|
||||
.defaultValue(120)
|
||||
.withDescription("Target size for parquet files produced by Hudi write phases. "
|
||||
+ "For DFS, this needs to be aligned with the underlying filesystem block size for optimal performance.");
|
||||
|
||||
public static final ConfigOption<Integer> WRITE_PARQUET_PAGE_SIZE = ConfigOptions
|
||||
.key("write.parquet.page.size")
|
||||
.intType()
|
||||
.defaultValue(1)
|
||||
.withDescription("Parquet page size. Page is the unit of read within a parquet file. "
|
||||
+ "Within a block, pages are compressed separately.");
|
||||
|
||||
public static final ConfigOption<Integer> WRITE_MERGE_MAX_MEMORY = ConfigOptions
|
||||
.key("write.merge.max_memory")
|
||||
.intType()
|
||||
.defaultValue(100) // default 100 MB
|
||||
.withDescription("Max memory in MB for merge, default 100MB");
|
||||
|
||||
// this is only for internal use
|
||||
public static final ConfigOption<Long> WRITE_COMMIT_ACK_TIMEOUT = ConfigOptions
|
||||
.key("write.commit.ack.timeout")
|
||||
.longType()
|
||||
.defaultValue(-1L) // default at least once
|
||||
.withDescription("Timeout limit for a writer task after it finishes a checkpoint and\n"
|
||||
+ "waits for the instant commit success, only for internal use");
|
||||
|
||||
public static final ConfigOption<Boolean> WRITE_BULK_INSERT_SHUFFLE_BY_PARTITION = ConfigOptions
|
||||
.key("write.bulk_insert.shuffle_by_partition")
|
||||
.booleanType()
|
||||
.defaultValue(true)
|
||||
.withDescription("Whether to shuffle the inputs by partition path for bulk insert tasks, default true");
|
||||
|
||||
public static final ConfigOption<Boolean> WRITE_BULK_INSERT_SORT_BY_PARTITION = ConfigOptions
|
||||
.key("write.bulk_insert.sort_by_partition")
|
||||
.booleanType()
|
||||
.defaultValue(true)
|
||||
.withDescription("Whether to sort the inputs by partition path for bulk insert tasks, default true");
|
||||
|
||||
public static final ConfigOption<Integer> WRITE_SORT_MEMORY = ConfigOptions
|
||||
.key("write.sort.memory")
|
||||
.intType()
|
||||
.defaultValue(128)
|
||||
.withDescription("Sort memory in MB, default 128MB");
|
||||
|
||||
// ------------------------------------------------------------------------
|
||||
// Compaction Options
|
||||
// ------------------------------------------------------------------------
|
||||
|
||||
public static final ConfigOption<Boolean> COMPACTION_SCHEDULE_ENABLED = ConfigOptions
|
||||
.key("compaction.schedule.enabled")
|
||||
.booleanType()
|
||||
.defaultValue(true) // default true for MOR write
|
||||
.withDescription("Schedule the compaction plan, enabled by default for MOR");
|
||||
|
||||
public static final ConfigOption<Boolean> COMPACTION_ASYNC_ENABLED = ConfigOptions
|
||||
.key("compaction.async.enabled")
|
||||
.booleanType()
|
||||
.defaultValue(true) // default true for MOR write
|
||||
.withDescription("Async Compaction, enabled by default for MOR");
|
||||
|
||||
public static final ConfigOption<Integer> COMPACTION_TASKS = ConfigOptions
|
||||
.key("compaction.tasks")
|
||||
.intType()
|
||||
.defaultValue(4) // default WRITE_TASKS * COMPACTION_DELTA_COMMITS * 0.2 (assumes 5 commits generate one bucket)
|
||||
.withDescription("Parallelism of tasks that do actual compaction, default is 4");
|
||||
|
||||
public static final String NUM_COMMITS = "num_commits";
|
||||
public static final String TIME_ELAPSED = "time_elapsed";
|
||||
public static final String NUM_AND_TIME = "num_and_time";
|
||||
public static final String NUM_OR_TIME = "num_or_time";
|
||||
public static final ConfigOption<String> COMPACTION_TRIGGER_STRATEGY = ConfigOptions
|
||||
.key("compaction.trigger.strategy")
|
||||
.stringType()
|
||||
.defaultValue(NUM_COMMITS) // default true for MOR write
|
||||
.withDescription("Strategy to trigger compaction, options are 'num_commits': trigger compaction when reach N delta commits;\n"
|
||||
+ "'time_elapsed': trigger compaction when time elapsed > N seconds since last compaction;\n"
|
||||
+ "'num_and_time': trigger compaction when both NUM_COMMITS and TIME_ELAPSED are satisfied;\n"
|
||||
+ "'num_or_time': trigger compaction when NUM_COMMITS or TIME_ELAPSED is satisfied.\n"
|
||||
+ "Default is 'num_commits'");
|
||||
|
||||
public static final ConfigOption<Integer> COMPACTION_DELTA_COMMITS = ConfigOptions
|
||||
.key("compaction.delta_commits")
|
||||
.intType()
|
||||
.defaultValue(5)
|
||||
.withDescription("Max delta commits needed to trigger compaction, default 5 commits");
|
||||
|
||||
public static final ConfigOption<Integer> COMPACTION_DELTA_SECONDS = ConfigOptions
|
||||
.key("compaction.delta_seconds")
|
||||
.intType()
|
||||
.defaultValue(3600) // default 1 hour
|
||||
.withDescription("Max delta seconds time needed to trigger compaction, default 1 hour");
|
||||
|
||||
public static final ConfigOption<Integer> COMPACTION_TIMEOUT_SECONDS = ConfigOptions
|
||||
.key("compaction.timeout.seconds")
|
||||
.intType()
|
||||
.defaultValue(1200) // default 20 minutes
|
||||
.withDescription("Max timeout time in seconds for online compaction to rollback, default 20 minutes");
|
||||
|
||||
public static final ConfigOption<Integer> COMPACTION_MAX_MEMORY = ConfigOptions
|
||||
.key("compaction.max_memory")
|
||||
.intType()
|
||||
.defaultValue(100) // default 100 MB
|
||||
.withDescription("Max memory in MB for compaction spillable map, default 100MB");
|
||||
|
||||
public static final ConfigOption<Long> COMPACTION_TARGET_IO = ConfigOptions
|
||||
.key("compaction.target_io")
|
||||
.longType()
|
||||
.defaultValue(500 * 1024L) // default 500 GB
|
||||
.withDescription("Target IO per compaction (both read and write), default 500 GB");
|
||||
|
||||
public static final ConfigOption<Boolean> CLEAN_ASYNC_ENABLED = ConfigOptions
|
||||
.key("clean.async.enabled")
|
||||
.booleanType()
|
||||
.defaultValue(true)
|
||||
.withDescription("Whether to cleanup the old commits immediately on new commits, enabled by default");
|
||||
|
||||
public static final ConfigOption<Integer> CLEAN_RETAIN_COMMITS = ConfigOptions
|
||||
.key("clean.retain_commits")
|
||||
.intType()
|
||||
.defaultValue(30)// default 30 commits
|
||||
.withDescription("Number of commits to retain. So data will be retained for num_of_commits * time_between_commits (scheduled).\n"
|
||||
+ "This also directly translates into how much you can incrementally pull on this table, default 30");
|
||||
|
||||
public static final ConfigOption<Integer> ARCHIVE_MAX_COMMITS = ConfigOptions
|
||||
.key("archive.max_commits")
|
||||
.intType()
|
||||
.defaultValue(50)// default max 50 commits
|
||||
.withDescription("Max number of commits to keep before archiving older commits into a sequential log, default 50");
|
||||
|
||||
public static final ConfigOption<Integer> ARCHIVE_MIN_COMMITS = ConfigOptions
|
||||
.key("archive.min_commits")
|
||||
.intType()
|
||||
.defaultValue(40)// default min 40 commits
|
||||
.withDescription("Min number of commits to keep before archiving older commits into a sequential log, default 40");
|
||||
|
||||
// ------------------------------------------------------------------------
|
||||
// Hive Sync Options
|
||||
// ------------------------------------------------------------------------
|
||||
public static final ConfigOption<Boolean> HIVE_SYNC_ENABLED = ConfigOptions
|
||||
.key("hive_sync.enable")
|
||||
.booleanType()
|
||||
.defaultValue(false)
|
||||
.withDescription("Asynchronously sync Hive meta to HMS, default false");
|
||||
|
||||
public static final ConfigOption<String> HIVE_SYNC_DB = ConfigOptions
|
||||
.key("hive_sync.db")
|
||||
.stringType()
|
||||
.defaultValue("default")
|
||||
.withDescription("Database name for hive sync, default 'default'");
|
||||
|
||||
public static final ConfigOption<String> HIVE_SYNC_TABLE = ConfigOptions
|
||||
.key("hive_sync.table")
|
||||
.stringType()
|
||||
.defaultValue("unknown")
|
||||
.withDescription("Table name for hive sync, default 'unknown'");
|
||||
|
||||
public static final ConfigOption<String> HIVE_SYNC_FILE_FORMAT = ConfigOptions
|
||||
.key("hive_sync.file_format")
|
||||
.stringType()
|
||||
.defaultValue("PARQUET")
|
||||
.withDescription("File format for hive sync, default 'PARQUET'");
|
||||
|
||||
public static final ConfigOption<String> HIVE_SYNC_MODE = ConfigOptions
|
||||
.key("hive_sync.mode")
|
||||
.stringType()
|
||||
.defaultValue("jdbc")
|
||||
.withDescription("Mode to choose for Hive ops. Valid values are hms, jdbc and hiveql, default 'jdbc'");
|
||||
|
||||
public static final ConfigOption<String> HIVE_SYNC_USERNAME = ConfigOptions
|
||||
.key("hive_sync.username")
|
||||
.stringType()
|
||||
.defaultValue("hive")
|
||||
.withDescription("Username for hive sync, default 'hive'");
|
||||
|
||||
public static final ConfigOption<String> HIVE_SYNC_PASSWORD = ConfigOptions
|
||||
.key("hive_sync.password")
|
||||
.stringType()
|
||||
.defaultValue("hive")
|
||||
.withDescription("Password for hive sync, default 'hive'");
|
||||
|
||||
public static final ConfigOption<String> HIVE_SYNC_JDBC_URL = ConfigOptions
|
||||
.key("hive_sync.jdbc_url")
|
||||
.stringType()
|
||||
.defaultValue("jdbc:hive2://localhost:10000")
|
||||
.withDescription("Jdbc URL for hive sync, default 'jdbc:hive2://localhost:10000'");
|
||||
|
||||
public static final ConfigOption<String> HIVE_SYNC_METASTORE_URIS = ConfigOptions
|
||||
.key("hive_sync.metastore.uris")
|
||||
.stringType()
|
||||
.defaultValue("")
|
||||
.withDescription("Metastore uris for hive sync, default ''");
|
||||
|
||||
public static final ConfigOption<String> HIVE_SYNC_PARTITION_FIELDS = ConfigOptions
|
||||
.key("hive_sync.partition_fields")
|
||||
.stringType()
|
||||
.defaultValue("")
|
||||
.withDescription("Partition fields for hive sync, default ''");
|
||||
|
||||
public static final ConfigOption<String> HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME = ConfigOptions
|
||||
.key("hive_sync.partition_extractor_class")
|
||||
.stringType()
|
||||
.defaultValue(SlashEncodedDayPartitionValueExtractor.class.getCanonicalName())
|
||||
.withDescription("Tool to extract the partition value from HDFS path, "
|
||||
+ "default 'SlashEncodedDayPartitionValueExtractor'");
|
||||
|
||||
public static final ConfigOption<Boolean> HIVE_SYNC_ASSUME_DATE_PARTITION = ConfigOptions
|
||||
.key("hive_sync.assume_date_partitioning")
|
||||
.booleanType()
|
||||
.defaultValue(false)
|
||||
.withDescription("Assume partitioning is yyyy/mm/dd, default false");
|
||||
|
||||
public static final ConfigOption<Boolean> HIVE_SYNC_USE_JDBC = ConfigOptions
|
||||
.key("hive_sync.use_jdbc")
|
||||
.booleanType()
|
||||
.defaultValue(true)
|
||||
.withDescription("Use JDBC when hive synchronization is enabled, default true");
|
||||
|
||||
public static final ConfigOption<Boolean> HIVE_SYNC_AUTO_CREATE_DB = ConfigOptions
|
||||
.key("hive_sync.auto_create_db")
|
||||
.booleanType()
|
||||
.defaultValue(true)
|
||||
.withDescription("Auto create hive database if it does not exists, default true");
|
||||
|
||||
public static final ConfigOption<Boolean> HIVE_SYNC_IGNORE_EXCEPTIONS = ConfigOptions
|
||||
.key("hive_sync.ignore_exceptions")
|
||||
.booleanType()
|
||||
.defaultValue(false)
|
||||
.withDescription("Ignore exceptions during hive synchronization, default false");
|
||||
|
||||
public static final ConfigOption<Boolean> HIVE_SYNC_SKIP_RO_SUFFIX = ConfigOptions
|
||||
.key("hive_sync.skip_ro_suffix")
|
||||
.booleanType()
|
||||
.defaultValue(false)
|
||||
.withDescription("Skip the _ro suffix for Read optimized table when registering, default false");
|
||||
|
||||
public static final ConfigOption<Boolean> HIVE_SYNC_SUPPORT_TIMESTAMP = ConfigOptions
|
||||
.key("hive_sync.support_timestamp")
|
||||
.booleanType()
|
||||
.defaultValue(false)
|
||||
.withDescription("INT64 with original type TIMESTAMP_MICROS is converted to hive timestamp type.\n"
|
||||
+ "Disabled by default for backward compatibility.");
|
||||
|
||||
public static final ConfigOption<String> HIVE_SYNC_TABLE_PROPERTIES = ConfigOptions
|
||||
.key("hive_sync.table_properties")
|
||||
.stringType()
|
||||
.noDefaultValue()
|
||||
.withDescription("Additional properties to store with table, the data format is k1=v1\nk2=v2");
|
||||
|
||||
public static final ConfigOption<String> HIVE_SYNC_TABLE_SERDE_PROPERTIES = ConfigOptions
|
||||
.key("hive_sync.serde_properties")
|
||||
.stringType()
|
||||
.noDefaultValue()
|
||||
.withDescription("Serde properties to hive table, the data format is k1=v1\nk2=v2");
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Utilities
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
// Prefix for Hoodie specific properties.
|
||||
private static final String PROPERTIES_PREFIX = "properties.";
|
||||
|
||||
/**
|
||||
* Collects the config options that start with 'properties.' into a 'key'='value' list.
|
||||
*/
|
||||
public static Map<String, String> getHoodieProperties(Map<String, String> options) {
|
||||
return getHoodiePropertiesWithPrefix(options, PROPERTIES_PREFIX);
|
||||
}
|
||||
|
||||
/**
|
||||
* Collects the config options that start with specified prefix {@code prefix} into a 'key'='value' list.
|
||||
*/
|
||||
public static Map<String, String> getHoodiePropertiesWithPrefix(Map<String, String> options, String prefix) {
|
||||
final Map<String, String> hoodieProperties = new HashMap<>();
|
||||
|
||||
if (hasPropertyOptions(options)) {
|
||||
options.keySet().stream()
|
||||
.filter(key -> key.startsWith(PROPERTIES_PREFIX))
|
||||
.forEach(key -> {
|
||||
final String value = options.get(key);
|
||||
final String subKey = key.substring((prefix).length());
|
||||
hoodieProperties.put(subKey, value);
|
||||
});
|
||||
}
|
||||
return hoodieProperties;
|
||||
}
|
||||
|
||||
/**
|
||||
* Collects all the config options, the 'properties.' prefix would be removed if the option key starts with it.
|
||||
*/
|
||||
public static Configuration flatOptions(Configuration conf) {
|
||||
final Map<String, String> propsMap = new HashMap<>();
|
||||
|
||||
conf.toMap().forEach((key, value) -> {
|
||||
final String subKey = key.startsWith(PROPERTIES_PREFIX)
|
||||
? key.substring((PROPERTIES_PREFIX).length())
|
||||
: key;
|
||||
propsMap.put(subKey, value);
|
||||
});
|
||||
return fromMap(propsMap);
|
||||
}
|
||||
|
||||
private static boolean hasPropertyOptions(Map<String, String> options) {
|
||||
return options.keySet().stream().anyMatch(k -> k.startsWith(PROPERTIES_PREFIX));
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new configuration that is initialized with the options of the given map.
|
||||
*/
|
||||
public static Configuration fromMap(Map<String, String> map) {
|
||||
final Configuration configuration = new Configuration();
|
||||
map.forEach(configuration::setString);
|
||||
return configuration;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether the given conf defines default value for the option {@code option}.
|
||||
*/
|
||||
public static <T> boolean isDefaultValueDefined(Configuration conf, ConfigOption<T> option) {
|
||||
return !conf.getOptional(option).isPresent()
|
||||
|| conf.get(option).equals(option.defaultValue());
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all the optional config options.
|
||||
*/
|
||||
public static Set<ConfigOption<?>> optionalOptions() {
|
||||
Set<ConfigOption<?>> options = new HashSet<>(allOptions());
|
||||
options.remove(PATH);
|
||||
return options;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all the config options.
|
||||
*/
|
||||
public static List<ConfigOption<?>> allOptions() {
|
||||
Field[] declaredFields = FlinkOptions.class.getDeclaredFields();
|
||||
List<ConfigOption<?>> options = new ArrayList<>();
|
||||
for (Field field : declaredFields) {
|
||||
if (java.lang.reflect.Modifier.isStatic(field.getModifiers())
|
||||
&& field.getType().equals(ConfigOption.class)) {
|
||||
try {
|
||||
options.add((ConfigOption<?>) field.get(ConfigOption.class));
|
||||
} catch (IllegalAccessException e) {
|
||||
throw new HoodieException("Error while fetching static config option", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
return options;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,118 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.configuration;
|
||||
|
||||
import org.apache.hudi.common.model.DefaultHoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.table.format.FilePathUtils;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
|
||||
import java.util.Locale;
|
||||
|
||||
/**
|
||||
* Tool helping to resolve the flink options {@link FlinkOptions}.
|
||||
*/
|
||||
public class OptionsResolver {
|
||||
/**
|
||||
* Returns whether insert clustering is allowed with given configuration {@code conf}.
|
||||
*/
|
||||
public static boolean insertClustering(Configuration conf) {
|
||||
return isCowTable(conf) && isInsertOperation(conf) && conf.getBoolean(FlinkOptions.INSERT_CLUSTER);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether the insert is clustering disabled with given configuration {@code conf}.
|
||||
*/
|
||||
public static boolean isAppendMode(Configuration conf) {
|
||||
return isCowTable(conf) && isInsertOperation(conf) && !conf.getBoolean(FlinkOptions.INSERT_CLUSTER);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether the table operation is 'insert'.
|
||||
*/
|
||||
public static boolean isInsertOperation(Configuration conf) {
|
||||
WriteOperationType operationType = WriteOperationType.fromValue(conf.getString(FlinkOptions.OPERATION));
|
||||
return operationType == WriteOperationType.INSERT;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether it is a MERGE_ON_READ table.
|
||||
*/
|
||||
public static boolean isMorTable(Configuration conf) {
|
||||
return conf.getString(FlinkOptions.TABLE_TYPE)
|
||||
.toUpperCase(Locale.ROOT)
|
||||
.equals(FlinkOptions.TABLE_TYPE_MERGE_ON_READ);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether it is a COPY_ON_WRITE table.
|
||||
*/
|
||||
public static boolean isCowTable(Configuration conf) {
|
||||
return conf.getString(FlinkOptions.TABLE_TYPE)
|
||||
.toUpperCase(Locale.ROOT)
|
||||
.equals(FlinkOptions.TABLE_TYPE_COPY_ON_WRITE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether the payload clazz is {@link DefaultHoodieRecordPayload}.
|
||||
*/
|
||||
public static boolean isDefaultHoodieRecordPayloadClazz(Configuration conf) {
|
||||
return conf.getString(FlinkOptions.PAYLOAD_CLASS_NAME).contains(DefaultHoodieRecordPayload.class.getSimpleName());
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the preCombine field
|
||||
* or null if the value is set as {@link FlinkOptions#NO_PRE_COMBINE}.
|
||||
*/
|
||||
public static String getPreCombineField(Configuration conf) {
|
||||
final String preCombineField = conf.getString(FlinkOptions.PRECOMBINE_FIELD);
|
||||
return preCombineField.equals(FlinkOptions.NO_PRE_COMBINE) ? null : preCombineField;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether the compaction strategy is based on elapsed delta time.
|
||||
*/
|
||||
public static boolean isDeltaTimeCompaction(Configuration conf) {
|
||||
final String strategy = conf.getString(FlinkOptions.COMPACTION_TRIGGER_STRATEGY).toLowerCase(Locale.ROOT);
|
||||
return FlinkOptions.TIME_ELAPSED.equals(strategy) || FlinkOptions.NUM_OR_TIME.equals(strategy);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether the table is partitioned.
|
||||
*/
|
||||
public static boolean isPartitionedTable(Configuration conf) {
|
||||
return FilePathUtils.extractPartitionKeys(conf).length > 0;
|
||||
}
|
||||
|
||||
public static boolean isBucketIndexType(Configuration conf) {
|
||||
return conf.getString(FlinkOptions.INDEX_TYPE).equals(HoodieIndex.IndexType.BUCKET.name());
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether the source should emit changelog.
|
||||
*
|
||||
* @return true if the source is read as streaming with changelog mode enabled
|
||||
*/
|
||||
public static boolean emitChangelog(Configuration conf) {
|
||||
return conf.getBoolean(FlinkOptions.READ_AS_STREAMING)
|
||||
&& conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,89 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.schema;
|
||||
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
|
||||
/**
|
||||
* A simple schema provider, that reads off files on DFS.
|
||||
*/
|
||||
public class FilebasedSchemaProvider extends SchemaProvider {
|
||||
|
||||
/**
|
||||
* Configs supported.
|
||||
*/
|
||||
public static class Config {
|
||||
private static final String SOURCE_SCHEMA_FILE_PROP = "hoodie.deltastreamer.schemaprovider.source.schema.file";
|
||||
private static final String TARGET_SCHEMA_FILE_PROP = "hoodie.deltastreamer.schemaprovider.target.schema.file";
|
||||
}
|
||||
|
||||
private final Schema sourceSchema;
|
||||
|
||||
private Schema targetSchema;
|
||||
|
||||
public FilebasedSchemaProvider(TypedProperties props) {
|
||||
StreamerUtil.checkRequiredProperties(props, Collections.singletonList(Config.SOURCE_SCHEMA_FILE_PROP));
|
||||
FileSystem fs = FSUtils.getFs(props.getString(Config.SOURCE_SCHEMA_FILE_PROP), StreamerUtil.getHadoopConf());
|
||||
try {
|
||||
this.sourceSchema = new Schema.Parser().parse(fs.open(new Path(props.getString(Config.SOURCE_SCHEMA_FILE_PROP))));
|
||||
if (props.containsKey(Config.TARGET_SCHEMA_FILE_PROP)) {
|
||||
this.targetSchema =
|
||||
new Schema.Parser().parse(fs.open(new Path(props.getString(Config.TARGET_SCHEMA_FILE_PROP))));
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException("Error reading schema", ioe);
|
||||
}
|
||||
}
|
||||
|
||||
public FilebasedSchemaProvider(Configuration conf) {
|
||||
final String sourceSchemaPath = conf.getString(FlinkOptions.SOURCE_AVRO_SCHEMA_PATH);
|
||||
final FileSystem fs = FSUtils.getFs(sourceSchemaPath, StreamerUtil.getHadoopConf());
|
||||
try {
|
||||
this.sourceSchema = new Schema.Parser().parse(fs.open(new Path(sourceSchemaPath)));
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException("Error reading schema", ioe);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Schema getSourceSchema() {
|
||||
return sourceSchema;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Schema getTargetSchema() {
|
||||
if (targetSchema != null) {
|
||||
return targetSchema;
|
||||
} else {
|
||||
return super.getTargetSchema();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,36 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.schema;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* Class to provide schema for reading data and also writing into a Hoodie table.
|
||||
*/
|
||||
public abstract class SchemaProvider implements Serializable {
|
||||
|
||||
public abstract Schema getSourceSchema();
|
||||
|
||||
public Schema getTargetSchema() {
|
||||
// by default, use source schema as target for hoodie table as well
|
||||
return getSourceSchema();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,126 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.schema;
|
||||
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.avro.Schema;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.HttpURLConnection;
|
||||
import java.net.URL;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Base64;
|
||||
import java.util.Collections;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
/**
|
||||
* Obtains latest schema from the Confluent/Kafka schema-registry.
|
||||
* <p>
|
||||
* https://github.com/confluentinc/schema-registry
|
||||
*/
|
||||
public class SchemaRegistryProvider extends SchemaProvider {
|
||||
|
||||
private final TypedProperties config;
|
||||
|
||||
|
||||
/**
|
||||
* Configs supported.
|
||||
*/
|
||||
public static class Config {
|
||||
|
||||
private static final String SRC_SCHEMA_REGISTRY_URL_PROP = "hoodie.deltastreamer.schemaprovider.registry.url";
|
||||
private static final String TARGET_SCHEMA_REGISTRY_URL_PROP =
|
||||
"hoodie.deltastreamer.schemaprovider.registry.targetUrl";
|
||||
}
|
||||
|
||||
/**
|
||||
* The method takes the provided url {@code registryUrl} and gets the schema from the schema registry using that url.
|
||||
* If the caller provides userInfo credentials in the url (e.g "https://foo:bar@schemaregistry.org") then the credentials
|
||||
* are extracted the url using the Matcher and the extracted credentials are set on the request as an Authorization
|
||||
* header.
|
||||
*
|
||||
* @param registryUrl
|
||||
* @return the Schema in String form.
|
||||
* @throws IOException
|
||||
*/
|
||||
public String fetchSchemaFromRegistry(String registryUrl) throws IOException {
|
||||
URL registry;
|
||||
HttpURLConnection connection;
|
||||
Matcher matcher = Pattern.compile("://(.*?)@").matcher(registryUrl);
|
||||
if (matcher.find()) {
|
||||
String creds = matcher.group(1);
|
||||
String urlWithoutCreds = registryUrl.replace(creds + "@", "");
|
||||
registry = new URL(urlWithoutCreds);
|
||||
connection = (HttpURLConnection) registry.openConnection();
|
||||
setAuthorizationHeader(matcher.group(1), connection);
|
||||
} else {
|
||||
registry = new URL(registryUrl);
|
||||
connection = (HttpURLConnection) registry.openConnection();
|
||||
}
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
JsonNode node = mapper.readTree(getStream(connection));
|
||||
return node.get("schema").asText();
|
||||
}
|
||||
|
||||
protected void setAuthorizationHeader(String creds, HttpURLConnection connection) {
|
||||
String encodedAuth = Base64.getEncoder().encodeToString(creds.getBytes(StandardCharsets.UTF_8));
|
||||
connection.setRequestProperty("Authorization", "Basic " + encodedAuth);
|
||||
}
|
||||
|
||||
protected InputStream getStream(HttpURLConnection connection) throws IOException {
|
||||
return connection.getInputStream();
|
||||
}
|
||||
|
||||
public SchemaRegistryProvider(TypedProperties props) {
|
||||
this.config = props;
|
||||
StreamerUtil.checkRequiredProperties(props, Collections.singletonList(Config.SRC_SCHEMA_REGISTRY_URL_PROP));
|
||||
}
|
||||
|
||||
private Schema getSchema(String registryUrl) throws IOException {
|
||||
return new Schema.Parser().parse(fetchSchemaFromRegistry(registryUrl));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Schema getSourceSchema() {
|
||||
String registryUrl = config.getString(Config.SRC_SCHEMA_REGISTRY_URL_PROP);
|
||||
try {
|
||||
return getSchema(registryUrl);
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException("Error reading source schema from registry :" + registryUrl, ioe);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Schema getTargetSchema() {
|
||||
String registryUrl = config.getString(Config.SRC_SCHEMA_REGISTRY_URL_PROP);
|
||||
String targetRegistryUrl = config.getString(Config.TARGET_SCHEMA_REGISTRY_URL_PROP, registryUrl);
|
||||
try {
|
||||
return getSchema(targetRegistryUrl);
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException("Error reading target schema from registry :" + registryUrl, ioe);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,168 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink;
|
||||
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.index.bucket.BucketIdentifier;
|
||||
import org.apache.hudi.table.HoodieFlinkTable;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.runtime.state.FunctionInitializationContext;
|
||||
import org.apache.flink.streaming.api.functions.ProcessFunction;
|
||||
import org.apache.flink.util.Collector;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
||||
/**
|
||||
* A stream write function with bucket hash index.
|
||||
*
|
||||
* <p>The task holds a fresh new local index: {(partition + bucket number) &rarr fileId} mapping, this index
|
||||
* is used for deciding whether the incoming records in an UPDATE or INSERT.
|
||||
* The index is local because different partition paths have separate items in the index.
|
||||
*
|
||||
* @param <I> the input type
|
||||
*/
|
||||
public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(BucketStreamWriteFunction.class);
|
||||
|
||||
private int maxParallelism;
|
||||
|
||||
private int parallelism;
|
||||
|
||||
private int bucketNum;
|
||||
|
||||
private transient HoodieFlinkTable<?> table;
|
||||
|
||||
private String indexKeyFields;
|
||||
|
||||
private final HashMap<String, String> bucketToFileIDMap;
|
||||
|
||||
/**
|
||||
* Constructs a BucketStreamWriteFunction.
|
||||
*
|
||||
* @param config The config options
|
||||
*/
|
||||
public BucketStreamWriteFunction(Configuration config) {
|
||||
super(config);
|
||||
this.bucketToFileIDMap = new HashMap<>();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(Configuration parameters) throws IOException {
|
||||
super.open(parameters);
|
||||
this.bucketNum = config.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS);
|
||||
this.indexKeyFields = config.getString(FlinkOptions.INDEX_KEY_FIELD);
|
||||
this.taskID = getRuntimeContext().getIndexOfThisSubtask();
|
||||
this.parallelism = getRuntimeContext().getNumberOfParallelSubtasks();
|
||||
this.maxParallelism = getRuntimeContext().getMaxNumberOfParallelSubtasks();
|
||||
bootstrapIndex();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initializeState(FunctionInitializationContext context) throws Exception {
|
||||
super.initializeState(context);
|
||||
this.table = this.writeClient.getHoodieTable();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processElement(I i, ProcessFunction<I, Object>.Context context, Collector<Object> collector) throws Exception {
|
||||
HoodieRecord<?> record = (HoodieRecord<?>) i;
|
||||
final HoodieKey hoodieKey = record.getKey();
|
||||
final HoodieRecordLocation location;
|
||||
|
||||
final int bucketNum = BucketIdentifier.getBucketId(hoodieKey, indexKeyFields, this.bucketNum);
|
||||
final String partitionBucketId = BucketIdentifier.partitionBucketIdStr(hoodieKey.getPartitionPath(), bucketNum);
|
||||
|
||||
if (bucketToFileIDMap.containsKey(partitionBucketId)) {
|
||||
location = new HoodieRecordLocation("U", bucketToFileIDMap.get(partitionBucketId));
|
||||
} else {
|
||||
String newFileId = BucketIdentifier.newBucketFileIdPrefix(bucketNum);
|
||||
location = new HoodieRecordLocation("I", newFileId);
|
||||
bucketToFileIDMap.put(partitionBucketId, newFileId);
|
||||
}
|
||||
record.unseal();
|
||||
record.setCurrentLocation(location);
|
||||
record.seal();
|
||||
bufferRecord(record);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get partition_bucket -> fileID mapping from the existing hudi table.
|
||||
* This is a required operation for each restart to avoid having duplicate file ids for one bucket.
|
||||
*/
|
||||
private void bootstrapIndex() throws IOException {
|
||||
Option<HoodieInstant> latestCommitTime = table.getFileSystemView().getTimeline().filterCompletedInstants().lastInstant();
|
||||
if (!latestCommitTime.isPresent()) {
|
||||
return;
|
||||
}
|
||||
// bootstrap bucket info from existing file system
|
||||
// bucketNum % totalParallelism == this taskID belongs to this task
|
||||
HashSet<Integer> bucketToLoad = new HashSet<>();
|
||||
for (int i = 0; i < bucketNum; i++) {
|
||||
int partitionOfBucket = BucketIdentifier.mod(i, parallelism);
|
||||
if (partitionOfBucket == taskID) {
|
||||
LOG.info(String.format("Bootstrapping index. Adding bucket %s , "
|
||||
+ "Current parallelism: %s , Max parallelism: %s , Current task id: %s",
|
||||
i, parallelism, maxParallelism, taskID));
|
||||
bucketToLoad.add(i);
|
||||
}
|
||||
}
|
||||
bucketToLoad.forEach(bucket -> LOG.info(String.format("bucketToLoad contains %s", bucket)));
|
||||
|
||||
LOG.info(String.format("Loading Hoodie Table %s, with path %s", table.getMetaClient().getTableConfig().getTableName(),
|
||||
table.getMetaClient().getBasePath()));
|
||||
|
||||
// Iterate through all existing partitions to load existing fileID belongs to this task
|
||||
List<String> partitions = table.getMetadata().getAllPartitionPaths();
|
||||
for (String partitionPath : partitions) {
|
||||
List<FileSlice> latestFileSlices = table.getSliceView()
|
||||
.getLatestFileSlices(partitionPath)
|
||||
.collect(toList());
|
||||
for (FileSlice fileslice : latestFileSlices) {
|
||||
String fileID = fileslice.getFileId();
|
||||
int bucketNumber = BucketIdentifier.bucketIdFromFileId(fileID);
|
||||
if (bucketToLoad.contains(bucketNumber)) {
|
||||
String partitionBucketId = BucketIdentifier.partitionBucketIdStr(partitionPath, bucketNumber);
|
||||
LOG.info(String.format("Should load this partition bucket %s with fileID %s", partitionBucketId, fileID));
|
||||
if (bucketToFileIDMap.containsKey(partitionBucketId)) {
|
||||
throw new RuntimeException(String.format("Duplicate fileID %s from partitionBucket %s found "
|
||||
+ "during the BucketStreamWriteFunction index bootstrap.", fileID, partitionBucketId));
|
||||
} else {
|
||||
LOG.info(String.format("Adding fileID %s to the partition bucket %s.", fileID, partitionBucketId));
|
||||
bucketToFileIDMap.put(partitionBucketId, fileID);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,40 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink;
|
||||
|
||||
import org.apache.hudi.sink.common.AbstractWriteOperator;
|
||||
import org.apache.hudi.sink.common.WriteOperatorFactory;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
|
||||
/**
|
||||
* Operator for {@link BucketStreamWriteFunction}.
|
||||
*
|
||||
* @param <I> The input type
|
||||
*/
|
||||
public class BucketStreamWriteOperator<I> extends AbstractWriteOperator<I> {
|
||||
|
||||
public BucketStreamWriteOperator(Configuration conf) {
|
||||
super(new BucketStreamWriteFunction<>(conf));
|
||||
}
|
||||
|
||||
public static <I> WriteOperatorFactory<I> getFactory(Configuration conf) {
|
||||
return WriteOperatorFactory.instance(conf, new BucketStreamWriteOperator<>(conf));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,118 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink;
|
||||
|
||||
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.sink.utils.NonThrownExecutor;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.api.common.functions.AbstractRichFunction;
|
||||
import org.apache.flink.api.common.state.CheckpointListener;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.runtime.state.FunctionInitializationContext;
|
||||
import org.apache.flink.runtime.state.FunctionSnapshotContext;
|
||||
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
|
||||
import org.apache.flink.streaming.api.functions.sink.SinkFunction;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* Sink function that cleans the old commits.
|
||||
*
|
||||
* <p>It starts a cleaning task on new checkpoints, there is only one cleaning task
|
||||
* at a time, a new task can not be scheduled until the last task finished(fails or normally succeed).
|
||||
* The cleaning task never expects to throw but only log.
|
||||
*/
|
||||
public class CleanFunction<T> extends AbstractRichFunction
|
||||
implements SinkFunction<T>, CheckpointedFunction, CheckpointListener {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(CleanFunction.class);
|
||||
|
||||
private final Configuration conf;
|
||||
|
||||
protected HoodieFlinkWriteClient writeClient;
|
||||
|
||||
private NonThrownExecutor executor;
|
||||
|
||||
private volatile boolean isCleaning;
|
||||
|
||||
public CleanFunction(Configuration conf) {
|
||||
this.conf = conf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(Configuration parameters) throws Exception {
|
||||
super.open(parameters);
|
||||
if (conf.getBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED)) {
|
||||
this.writeClient = StreamerUtil.createWriteClient(conf, getRuntimeContext());
|
||||
this.executor = NonThrownExecutor.builder(LOG).waitForTasksFinish(true).build();
|
||||
|
||||
if (conf.getString(FlinkOptions.OPERATION).equals(WriteOperationType.INSERT_OVERWRITE_TABLE.value())) {
|
||||
String instantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
LOG.info(String.format("exec sync clean with instant time %s...", instantTime));
|
||||
executor.execute(() -> writeClient.clean(instantTime), "wait for sync cleaning finish");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void notifyCheckpointComplete(long l) throws Exception {
|
||||
if (conf.getBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED) && isCleaning) {
|
||||
executor.execute(() -> {
|
||||
try {
|
||||
this.writeClient.waitForCleaningFinish();
|
||||
} finally {
|
||||
// ensure to switch the isCleaning flag
|
||||
this.isCleaning = false;
|
||||
}
|
||||
}, "wait for cleaning finish");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void snapshotState(FunctionSnapshotContext context) throws Exception {
|
||||
if (conf.getBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED) && !isCleaning) {
|
||||
try {
|
||||
this.writeClient.startAsyncCleaning();
|
||||
this.isCleaning = true;
|
||||
} catch (Throwable throwable) {
|
||||
// catch the exception to not affect the normal checkpointing
|
||||
LOG.warn("Error while start async cleaning", throwable);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initializeState(FunctionInitializationContext context) throws Exception {
|
||||
// no operation
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws Exception {
|
||||
if (executor != null) {
|
||||
executor.close();
|
||||
}
|
||||
|
||||
if (this.writeClient != null) {
|
||||
this.writeClient.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,486 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieOperation;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.util.ObjectSizeCalculator;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.sink.common.AbstractStreamWriteFunction;
|
||||
import org.apache.hudi.sink.event.WriteMetadataEvent;
|
||||
import org.apache.hudi.table.action.commit.FlinkWriteHelper;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.annotation.VisibleForTesting;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.streaming.api.functions.ProcessFunction;
|
||||
import org.apache.flink.util.Collector;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.function.BiFunction;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Sink function to write the data to the underneath filesystem.
|
||||
*
|
||||
* <p><h2>Work Flow</h2>
|
||||
*
|
||||
* <p>The function firstly buffers the data as a batch of {@link HoodieRecord}s,
|
||||
* It flushes(write) the records batch when the batch size exceeds the configured size {@link FlinkOptions#WRITE_BATCH_SIZE}
|
||||
* or the total buffer size exceeds the configured size {@link FlinkOptions#WRITE_TASK_MAX_SIZE}
|
||||
* or a Flink checkpoint starts. After a batch has been written successfully,
|
||||
* the function notifies its operator coordinator {@link StreamWriteOperatorCoordinator} to mark a successful write.
|
||||
*
|
||||
* <p><h2>The Semantics</h2>
|
||||
*
|
||||
* <p>The task implements exactly-once semantics by buffering the data between checkpoints. The operator coordinator
|
||||
* starts a new instant on the timeline when a checkpoint triggers, the coordinator checkpoints always
|
||||
* start before its operator, so when this function starts a checkpoint, a REQUESTED instant already exists.
|
||||
*
|
||||
* <p>The function process thread blocks data buffering after the checkpoint thread finishes flushing the existing data buffer until
|
||||
* the current checkpoint succeed and the coordinator starts a new instant. Any error triggers the job failure during the metadata committing,
|
||||
* when the job recovers from a failure, the write function re-send the write metadata to the coordinator to see if these metadata
|
||||
* can re-commit, thus if unexpected error happens during the instant committing, the coordinator would retry to commit when the job
|
||||
* recovers.
|
||||
*
|
||||
* <p><h2>Fault Tolerance</h2>
|
||||
*
|
||||
* <p>The operator coordinator checks and commits the last instant then starts a new one after a checkpoint finished successfully.
|
||||
* It rolls back any inflight instant before it starts a new instant, this means one hoodie instant only span one checkpoint,
|
||||
* the write function blocks data buffer flushing for the configured checkpoint timeout
|
||||
* before it throws exception, any checkpoint failure would finally trigger the job failure.
|
||||
*
|
||||
* <p>Note: The function task requires the input stream be shuffled by the file IDs.
|
||||
*
|
||||
* @param <I> Type of the input record
|
||||
* @see StreamWriteOperatorCoordinator
|
||||
*/
|
||||
public class StreamWriteFunction<I> extends AbstractStreamWriteFunction<I> {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(StreamWriteFunction.class);
|
||||
|
||||
/**
|
||||
* Write buffer as buckets for a checkpoint. The key is bucket ID.
|
||||
*/
|
||||
private transient Map<String, DataBucket> buckets;
|
||||
|
||||
private transient BiFunction<List<HoodieRecord>, String, List<WriteStatus>> writeFunction;
|
||||
|
||||
/**
|
||||
* Total size tracer.
|
||||
*/
|
||||
private transient TotalSizeTracer tracer;
|
||||
|
||||
/**
|
||||
* Constructs a StreamingSinkFunction.
|
||||
*
|
||||
* @param config The config options
|
||||
*/
|
||||
public StreamWriteFunction(Configuration config) {
|
||||
super(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(Configuration parameters) throws IOException {
|
||||
this.tracer = new TotalSizeTracer(this.config);
|
||||
initBuffer();
|
||||
initWriteFunction();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void snapshotState() {
|
||||
// Based on the fact that the coordinator starts the checkpoint first,
|
||||
// it would check the validity.
|
||||
// wait for the buffer data flush out and request a new instant
|
||||
flushRemaining(false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processElement(I value, ProcessFunction<I, Object>.Context ctx, Collector<Object> out) throws Exception {
|
||||
bufferRecord((HoodieRecord<?>) value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
if (this.writeClient != null) {
|
||||
this.writeClient.cleanHandlesGracefully();
|
||||
this.writeClient.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* End input action for batch source.
|
||||
*/
|
||||
public void endInput() {
|
||||
flushRemaining(true);
|
||||
this.writeClient.cleanHandles();
|
||||
this.writeStatuses.clear();
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Getter/Setter
|
||||
// -------------------------------------------------------------------------
|
||||
@VisibleForTesting
|
||||
@SuppressWarnings("rawtypes")
|
||||
public Map<String, List<HoodieRecord>> getDataBuffer() {
|
||||
Map<String, List<HoodieRecord>> ret = new HashMap<>();
|
||||
for (Map.Entry<String, DataBucket> entry : buckets.entrySet()) {
|
||||
ret.put(entry.getKey(), entry.getValue().writeBuffer());
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Utilities
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
private void initBuffer() {
|
||||
this.buckets = new LinkedHashMap<>();
|
||||
}
|
||||
|
||||
private void initWriteFunction() {
|
||||
final String writeOperation = this.config.get(FlinkOptions.OPERATION);
|
||||
switch (WriteOperationType.fromValue(writeOperation)) {
|
||||
case INSERT:
|
||||
this.writeFunction = (records, instantTime) -> this.writeClient.insert(records, instantTime);
|
||||
break;
|
||||
case UPSERT:
|
||||
this.writeFunction = (records, instantTime) -> this.writeClient.upsert(records, instantTime);
|
||||
break;
|
||||
case INSERT_OVERWRITE:
|
||||
this.writeFunction = (records, instantTime) -> this.writeClient.insertOverwrite(records, instantTime);
|
||||
break;
|
||||
case INSERT_OVERWRITE_TABLE:
|
||||
this.writeFunction = (records, instantTime) -> this.writeClient.insertOverwriteTable(records, instantTime);
|
||||
break;
|
||||
default:
|
||||
throw new RuntimeException("Unsupported write operation : " + writeOperation);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Represents a data item in the buffer, this is needed to reduce the
|
||||
* memory footprint.
|
||||
*
|
||||
* <p>A {@link HoodieRecord} was firstly transformed into a {@link DataItem}
|
||||
* for buffering, it then transforms back to the {@link HoodieRecord} before flushing.
|
||||
*/
|
||||
private static class DataItem {
|
||||
private final String key; // record key
|
||||
private final String instant; // 'U' or 'I'
|
||||
private final HoodieRecordPayload<?> data; // record payload
|
||||
private final HoodieOperation operation; // operation
|
||||
|
||||
private DataItem(String key, String instant, HoodieRecordPayload<?> data, HoodieOperation operation) {
|
||||
this.key = key;
|
||||
this.instant = instant;
|
||||
this.data = data;
|
||||
this.operation = operation;
|
||||
}
|
||||
|
||||
public static DataItem fromHoodieRecord(HoodieRecord<?> record) {
|
||||
return new DataItem(
|
||||
record.getRecordKey(),
|
||||
record.getCurrentLocation().getInstantTime(),
|
||||
((HoodieAvroRecord) record).getData(),
|
||||
record.getOperation());
|
||||
}
|
||||
|
||||
public HoodieRecord<?> toHoodieRecord(String partitionPath) {
|
||||
HoodieKey hoodieKey = new HoodieKey(this.key, partitionPath);
|
||||
HoodieRecord<?> record = new HoodieAvroRecord<>(hoodieKey, data, operation);
|
||||
HoodieRecordLocation loc = new HoodieRecordLocation(instant, null);
|
||||
record.setCurrentLocation(loc);
|
||||
return record;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Data bucket.
|
||||
*/
|
||||
private static class DataBucket {
|
||||
private final List<DataItem> records;
|
||||
private final BufferSizeDetector detector;
|
||||
private final String partitionPath;
|
||||
private final String fileID;
|
||||
|
||||
private DataBucket(Double batchSize, HoodieRecord<?> hoodieRecord) {
|
||||
this.records = new ArrayList<>();
|
||||
this.detector = new BufferSizeDetector(batchSize);
|
||||
this.partitionPath = hoodieRecord.getPartitionPath();
|
||||
this.fileID = hoodieRecord.getCurrentLocation().getFileId();
|
||||
}
|
||||
|
||||
/**
|
||||
* Prepare the write data buffer: patch up all the records with correct partition path.
|
||||
*/
|
||||
public List<HoodieRecord> writeBuffer() {
|
||||
// rewrite all the records with new record key
|
||||
return records.stream()
|
||||
.map(record -> record.toHoodieRecord(partitionPath))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets up before flush: patch up the first record with correct partition path and fileID.
|
||||
*
|
||||
* <p>Note: the method may modify the given records {@code records}.
|
||||
*/
|
||||
public void preWrite(List<HoodieRecord> records) {
|
||||
// rewrite the first record with expected fileID
|
||||
HoodieRecord<?> first = records.get(0);
|
||||
HoodieRecord<?> record = new HoodieAvroRecord<>(first.getKey(), (HoodieRecordPayload) first.getData(), first.getOperation());
|
||||
HoodieRecordLocation newLoc = new HoodieRecordLocation(first.getCurrentLocation().getInstantTime(), fileID);
|
||||
record.setCurrentLocation(newLoc);
|
||||
|
||||
records.set(0, record);
|
||||
}
|
||||
|
||||
public void reset() {
|
||||
this.records.clear();
|
||||
this.detector.reset();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Tool to detect if to flush out the existing buffer.
|
||||
* Sampling the record to compute the size with 0.01 percentage.
|
||||
*/
|
||||
private static class BufferSizeDetector {
|
||||
private final Random random = new Random(47);
|
||||
private static final int DENOMINATOR = 100;
|
||||
|
||||
private final double batchSizeBytes;
|
||||
|
||||
private long lastRecordSize = -1L;
|
||||
private long totalSize = 0L;
|
||||
|
||||
BufferSizeDetector(double batchSizeMb) {
|
||||
this.batchSizeBytes = batchSizeMb * 1024 * 1024;
|
||||
}
|
||||
|
||||
boolean detect(Object record) {
|
||||
if (lastRecordSize == -1 || sampling()) {
|
||||
lastRecordSize = ObjectSizeCalculator.getObjectSize(record);
|
||||
}
|
||||
totalSize += lastRecordSize;
|
||||
return totalSize > this.batchSizeBytes;
|
||||
}
|
||||
|
||||
boolean sampling() {
|
||||
// 0.01 sampling percentage
|
||||
return random.nextInt(DENOMINATOR) == 1;
|
||||
}
|
||||
|
||||
void reset() {
|
||||
this.lastRecordSize = -1L;
|
||||
this.totalSize = 0L;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Tool to trace the total buffer size. It computes the maximum buffer size,
|
||||
* if current buffer size is greater than the maximum buffer size, the data bucket
|
||||
* flush triggers.
|
||||
*/
|
||||
private static class TotalSizeTracer {
|
||||
private long bufferSize = 0L;
|
||||
private final double maxBufferSize;
|
||||
|
||||
TotalSizeTracer(Configuration conf) {
|
||||
long mergeReaderMem = 100; // constant 100MB
|
||||
long mergeMapMaxMem = conf.getInteger(FlinkOptions.WRITE_MERGE_MAX_MEMORY);
|
||||
this.maxBufferSize = (conf.getDouble(FlinkOptions.WRITE_TASK_MAX_SIZE) - mergeReaderMem - mergeMapMaxMem) * 1024 * 1024;
|
||||
final String errMsg = String.format("'%s' should be at least greater than '%s' plus merge reader memory(constant 100MB now)",
|
||||
FlinkOptions.WRITE_TASK_MAX_SIZE.key(), FlinkOptions.WRITE_MERGE_MAX_MEMORY.key());
|
||||
ValidationUtils.checkState(this.maxBufferSize > 0, errMsg);
|
||||
}
|
||||
|
||||
/**
|
||||
* Trace the given record size {@code recordSize}.
|
||||
*
|
||||
* @param recordSize The record size
|
||||
* @return true if the buffer size exceeds the maximum buffer size
|
||||
*/
|
||||
boolean trace(long recordSize) {
|
||||
this.bufferSize += recordSize;
|
||||
return this.bufferSize > this.maxBufferSize;
|
||||
}
|
||||
|
||||
void countDown(long size) {
|
||||
this.bufferSize -= size;
|
||||
}
|
||||
|
||||
public void reset() {
|
||||
this.bufferSize = 0;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the bucket ID with the given value {@code value}.
|
||||
*/
|
||||
private String getBucketID(HoodieRecord<?> record) {
|
||||
final String fileId = record.getCurrentLocation().getFileId();
|
||||
return StreamerUtil.generateBucketKey(record.getPartitionPath(), fileId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Buffers the given record.
|
||||
*
|
||||
* <p>Flush the data bucket first if the bucket records size is greater than
|
||||
* the configured value {@link FlinkOptions#WRITE_BATCH_SIZE}.
|
||||
*
|
||||
* <p>Flush the max size data bucket if the total buffer size exceeds the configured
|
||||
* threshold {@link FlinkOptions#WRITE_TASK_MAX_SIZE}.
|
||||
*
|
||||
* @param value HoodieRecord
|
||||
*/
|
||||
protected void bufferRecord(HoodieRecord<?> value) {
|
||||
final String bucketID = getBucketID(value);
|
||||
|
||||
DataBucket bucket = this.buckets.computeIfAbsent(bucketID,
|
||||
k -> new DataBucket(this.config.getDouble(FlinkOptions.WRITE_BATCH_SIZE), value));
|
||||
final DataItem item = DataItem.fromHoodieRecord(value);
|
||||
|
||||
bucket.records.add(item);
|
||||
|
||||
boolean flushBucket = bucket.detector.detect(item);
|
||||
boolean flushBuffer = this.tracer.trace(bucket.detector.lastRecordSize);
|
||||
if (flushBucket) {
|
||||
if (flushBucket(bucket)) {
|
||||
this.tracer.countDown(bucket.detector.totalSize);
|
||||
bucket.reset();
|
||||
}
|
||||
} else if (flushBuffer) {
|
||||
// find the max size bucket and flush it out
|
||||
List<DataBucket> sortedBuckets = this.buckets.values().stream()
|
||||
.sorted((b1, b2) -> Long.compare(b2.detector.totalSize, b1.detector.totalSize))
|
||||
.collect(Collectors.toList());
|
||||
final DataBucket bucketToFlush = sortedBuckets.get(0);
|
||||
if (flushBucket(bucketToFlush)) {
|
||||
this.tracer.countDown(bucketToFlush.detector.totalSize);
|
||||
bucketToFlush.reset();
|
||||
} else {
|
||||
LOG.warn("The buffer size hits the threshold {}, but still flush the max size data bucket failed!", this.tracer.maxBufferSize);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private boolean hasData() {
|
||||
return this.buckets.size() > 0
|
||||
&& this.buckets.values().stream().anyMatch(bucket -> bucket.records.size() > 0);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked, rawtypes")
|
||||
private boolean flushBucket(DataBucket bucket) {
|
||||
String instant = instantToWrite(true);
|
||||
|
||||
if (instant == null) {
|
||||
// in case there are empty checkpoints that has no input data
|
||||
LOG.info("No inflight instant when flushing data, skip.");
|
||||
return false;
|
||||
}
|
||||
|
||||
List<HoodieRecord> records = bucket.writeBuffer();
|
||||
ValidationUtils.checkState(records.size() > 0, "Data bucket to flush has no buffering records");
|
||||
if (config.getBoolean(FlinkOptions.PRE_COMBINE)) {
|
||||
records = FlinkWriteHelper.newInstance().deduplicateRecords(records, (HoodieIndex) null, -1);
|
||||
}
|
||||
bucket.preWrite(records);
|
||||
final List<WriteStatus> writeStatus = new ArrayList<>(writeFunction.apply(records, instant));
|
||||
records.clear();
|
||||
final WriteMetadataEvent event = WriteMetadataEvent.builder()
|
||||
.taskID(taskID)
|
||||
.instantTime(instant) // the write instant may shift but the event still use the currentInstant.
|
||||
.writeStatus(writeStatus)
|
||||
.lastBatch(false)
|
||||
.endInput(false)
|
||||
.build();
|
||||
|
||||
this.eventGateway.sendEventToCoordinator(event);
|
||||
writeStatuses.addAll(writeStatus);
|
||||
return true;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked, rawtypes")
|
||||
private void flushRemaining(boolean endInput) {
|
||||
this.currentInstant = instantToWrite(hasData());
|
||||
if (this.currentInstant == null) {
|
||||
// in case there are empty checkpoints that has no input data
|
||||
throw new HoodieException("No inflight instant when flushing data!");
|
||||
}
|
||||
final List<WriteStatus> writeStatus;
|
||||
if (buckets.size() > 0) {
|
||||
writeStatus = new ArrayList<>();
|
||||
this.buckets.values()
|
||||
// The records are partitioned by the bucket ID and each batch sent to
|
||||
// the writer belongs to one bucket.
|
||||
.forEach(bucket -> {
|
||||
List<HoodieRecord> records = bucket.writeBuffer();
|
||||
if (records.size() > 0) {
|
||||
if (config.getBoolean(FlinkOptions.PRE_COMBINE)) {
|
||||
records = FlinkWriteHelper.newInstance().deduplicateRecords(records, (HoodieIndex) null, -1);
|
||||
}
|
||||
bucket.preWrite(records);
|
||||
writeStatus.addAll(writeFunction.apply(records, currentInstant));
|
||||
records.clear();
|
||||
bucket.reset();
|
||||
}
|
||||
});
|
||||
} else {
|
||||
LOG.info("No data to write in subtask [{}] for instant [{}]", taskID, currentInstant);
|
||||
writeStatus = Collections.emptyList();
|
||||
}
|
||||
final WriteMetadataEvent event = WriteMetadataEvent.builder()
|
||||
.taskID(taskID)
|
||||
.instantTime(currentInstant)
|
||||
.writeStatus(writeStatus)
|
||||
.lastBatch(true)
|
||||
.endInput(endInput)
|
||||
.build();
|
||||
|
||||
this.eventGateway.sendEventToCoordinator(event);
|
||||
this.buckets.clear();
|
||||
this.tracer.reset();
|
||||
this.writeClient.cleanHandles();
|
||||
this.writeStatuses.addAll(writeStatus);
|
||||
// blocks flushing until the coordinator starts a new instant
|
||||
this.confirming = true;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,41 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink;
|
||||
|
||||
import org.apache.hudi.sink.common.AbstractWriteOperator;
|
||||
import org.apache.hudi.sink.common.WriteOperatorFactory;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.streaming.api.operators.StreamSink;
|
||||
|
||||
/**
|
||||
* Operator for {@link StreamSink}.
|
||||
*
|
||||
* @param <I> The input type
|
||||
*/
|
||||
public class StreamWriteOperator<I> extends AbstractWriteOperator<I> {
|
||||
|
||||
public StreamWriteOperator(Configuration conf) {
|
||||
super(new StreamWriteFunction<>(conf));
|
||||
}
|
||||
|
||||
public static <I> WriteOperatorFactory<I> getFactory(Configuration conf) {
|
||||
return WriteOperatorFactory.instance(conf, new StreamWriteOperator<>(conf));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,618 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink;
|
||||
|
||||
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.CommitUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.configuration.OptionsResolver;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.sink.event.CommitAckEvent;
|
||||
import org.apache.hudi.sink.event.WriteMetadataEvent;
|
||||
import org.apache.hudi.sink.meta.CkpMetadata;
|
||||
import org.apache.hudi.sink.utils.HiveSyncContext;
|
||||
import org.apache.hudi.sink.utils.NonThrownExecutor;
|
||||
import org.apache.hudi.util.CompactionUtil;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.annotation.VisibleForTesting;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.runtime.jobgraph.OperatorID;
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
|
||||
import org.apache.flink.runtime.operators.coordination.TaskNotRunningException;
|
||||
import org.jetbrains.annotations.Nullable;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionException;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.util.StreamerUtil.initTableIfNotExists;
|
||||
|
||||
/**
|
||||
* {@link OperatorCoordinator} for {@link StreamWriteFunction}.
|
||||
*
|
||||
* <p>This coordinator starts a new instant when a new checkpoint starts. It commits the instant when all the
|
||||
* operator tasks write the buffer successfully for a round of checkpoint.
|
||||
*
|
||||
* <p>If there is no data for a round of checkpointing, it resets the events buffer and returns early.
|
||||
*
|
||||
* @see StreamWriteFunction for the work flow and semantics
|
||||
*/
|
||||
public class StreamWriteOperatorCoordinator
|
||||
implements OperatorCoordinator {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(StreamWriteOperatorCoordinator.class);
|
||||
|
||||
/**
|
||||
* Config options.
|
||||
*/
|
||||
private final Configuration conf;
|
||||
|
||||
/**
|
||||
* Coordinator context.
|
||||
*/
|
||||
private final Context context;
|
||||
|
||||
/**
|
||||
* Gateways for sending events to sub tasks.
|
||||
*/
|
||||
private transient SubtaskGateway[] gateways;
|
||||
|
||||
/**
|
||||
* Write client.
|
||||
*/
|
||||
private transient HoodieFlinkWriteClient writeClient;
|
||||
|
||||
/**
|
||||
* Meta client.
|
||||
*/
|
||||
private transient HoodieTableMetaClient metaClient;
|
||||
|
||||
/**
|
||||
* Current REQUESTED instant, for validation.
|
||||
*/
|
||||
private volatile String instant = WriteMetadataEvent.BOOTSTRAP_INSTANT;
|
||||
|
||||
/**
|
||||
* Event buffer for one round of checkpointing. When all the elements are non-null and have the same
|
||||
* write instant, then the instant succeed and we can commit it.
|
||||
*/
|
||||
private transient WriteMetadataEvent[] eventBuffer;
|
||||
|
||||
/**
|
||||
* Task number of the operator.
|
||||
*/
|
||||
private final int parallelism;
|
||||
|
||||
/**
|
||||
* A single-thread executor to handle all the asynchronous jobs of the coordinator.
|
||||
*/
|
||||
private NonThrownExecutor executor;
|
||||
|
||||
/**
|
||||
* A single-thread executor to handle asynchronous hive sync.
|
||||
*/
|
||||
private NonThrownExecutor hiveSyncExecutor;
|
||||
|
||||
/**
|
||||
* Context that holds variables for asynchronous hive sync.
|
||||
*/
|
||||
private HiveSyncContext hiveSyncContext;
|
||||
|
||||
/**
|
||||
* The table state.
|
||||
*/
|
||||
private transient TableState tableState;
|
||||
|
||||
/**
|
||||
* The checkpoint metadata.
|
||||
*/
|
||||
private CkpMetadata ckpMetadata;
|
||||
|
||||
/**
|
||||
* Current checkpoint.
|
||||
*/
|
||||
private long checkpointId = -1;
|
||||
|
||||
/**
|
||||
* Constructs a StreamingSinkOperatorCoordinator.
|
||||
*
|
||||
* @param conf The config options
|
||||
* @param context The coordinator context
|
||||
*/
|
||||
public StreamWriteOperatorCoordinator(
|
||||
Configuration conf,
|
||||
Context context) {
|
||||
this.conf = conf;
|
||||
this.context = context;
|
||||
this.parallelism = context.currentParallelism();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start() throws Exception {
|
||||
// setup classloader for APIs that use reflection without taking ClassLoader param
|
||||
// reference: https://stackoverflow.com/questions/1771679/difference-between-threads-context-class-loader-and-normal-classloader
|
||||
Thread.currentThread().setContextClassLoader(getClass().getClassLoader());
|
||||
// initialize event buffer
|
||||
reset();
|
||||
this.gateways = new SubtaskGateway[this.parallelism];
|
||||
// init table, create if not exists.
|
||||
this.metaClient = initTableIfNotExists(this.conf);
|
||||
// the write client must create after the table creation
|
||||
this.writeClient = StreamerUtil.createWriteClient(conf);
|
||||
this.tableState = TableState.create(conf);
|
||||
// start the executor
|
||||
this.executor = NonThrownExecutor.builder(LOG)
|
||||
.exceptionHook((errMsg, t) -> this.context.failJob(new HoodieException(errMsg, t)))
|
||||
.waitForTasksFinish(true).build();
|
||||
// start the executor if required
|
||||
if (tableState.syncHive) {
|
||||
initHiveSync();
|
||||
}
|
||||
if (tableState.syncMetadata) {
|
||||
initMetadataSync();
|
||||
}
|
||||
this.ckpMetadata = CkpMetadata.getInstance(this.metaClient.getFs(), metaClient.getBasePath());
|
||||
this.ckpMetadata.bootstrap(this.metaClient);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws Exception {
|
||||
// teardown the resource
|
||||
if (executor != null) {
|
||||
executor.close();
|
||||
}
|
||||
if (hiveSyncExecutor != null) {
|
||||
hiveSyncExecutor.close();
|
||||
}
|
||||
// the write client must close after the executor service
|
||||
// because the task in the service may send requests to the embedded timeline service.
|
||||
if (writeClient != null) {
|
||||
writeClient.close();
|
||||
}
|
||||
this.eventBuffer = null;
|
||||
if (this.ckpMetadata != null) {
|
||||
this.ckpMetadata.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void checkpointCoordinator(long checkpointId, CompletableFuture<byte[]> result) {
|
||||
this.checkpointId = checkpointId;
|
||||
executor.execute(
|
||||
() -> {
|
||||
try {
|
||||
result.complete(new byte[0]);
|
||||
} catch (Throwable throwable) {
|
||||
// when a checkpoint fails, throws directly.
|
||||
result.completeExceptionally(
|
||||
new CompletionException(
|
||||
String.format("Failed to checkpoint Instant %s for source %s",
|
||||
this.instant, this.getClass().getSimpleName()), throwable));
|
||||
}
|
||||
}, "taking checkpoint %d", checkpointId
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void notifyCheckpointComplete(long checkpointId) {
|
||||
executor.execute(
|
||||
() -> {
|
||||
// The executor thread inherits the classloader of the #notifyCheckpointComplete
|
||||
// caller, which is a AppClassLoader.
|
||||
Thread.currentThread().setContextClassLoader(getClass().getClassLoader());
|
||||
// for streaming mode, commits the ever received events anyway,
|
||||
// the stream write task snapshot and flush the data buffer synchronously in sequence,
|
||||
// so a successful checkpoint subsumes the old one(follows the checkpoint subsuming contract)
|
||||
final boolean committed = commitInstant(this.instant, checkpointId);
|
||||
|
||||
if (tableState.scheduleCompaction) {
|
||||
// if async compaction is on, schedule the compaction
|
||||
CompactionUtil.scheduleCompaction(metaClient, writeClient, tableState.isDeltaTimeCompaction, committed);
|
||||
}
|
||||
|
||||
if (committed) {
|
||||
// start new instant.
|
||||
startInstant();
|
||||
// sync Hive if is enabled
|
||||
syncHiveAsync();
|
||||
}
|
||||
}, "commits the instant %s", this.instant
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void notifyCheckpointAborted(long checkpointId) {
|
||||
if (checkpointId == this.checkpointId) {
|
||||
executor.execute(() -> {
|
||||
this.ckpMetadata.abortInstant(this.instant);
|
||||
}, "abort instant %s", this.instant);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void resetToCheckpoint(long checkpointID, byte[] checkpointData) {
|
||||
// no operation
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handleEventFromOperator(int i, OperatorEvent operatorEvent) {
|
||||
ValidationUtils.checkState(operatorEvent instanceof WriteMetadataEvent,
|
||||
"The coordinator can only handle WriteMetaEvent");
|
||||
WriteMetadataEvent event = (WriteMetadataEvent) operatorEvent;
|
||||
|
||||
if (event.isEndInput()) {
|
||||
// handle end input event synchronously
|
||||
handleEndInputEvent(event);
|
||||
} else {
|
||||
executor.execute(
|
||||
() -> {
|
||||
if (event.isBootstrap()) {
|
||||
handleBootstrapEvent(event);
|
||||
} else {
|
||||
handleWriteMetaEvent(event);
|
||||
}
|
||||
}, "handle write metadata event for instant %s", this.instant
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void subtaskFailed(int i, @Nullable Throwable throwable) {
|
||||
// reset the event
|
||||
this.eventBuffer[i] = null;
|
||||
LOG.warn("Reset the event for task [" + i + "]", throwable);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void subtaskReset(int i, long l) {
|
||||
// no operation
|
||||
}
|
||||
|
||||
@Override
|
||||
public void subtaskReady(int i, SubtaskGateway subtaskGateway) {
|
||||
this.gateways[i] = subtaskGateway;
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Utilities
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
private void initHiveSync() {
|
||||
this.hiveSyncExecutor = NonThrownExecutor.builder(LOG).waitForTasksFinish(true).build();
|
||||
this.hiveSyncContext = HiveSyncContext.create(conf);
|
||||
}
|
||||
|
||||
private void syncHiveAsync() {
|
||||
if (tableState.syncHive) {
|
||||
this.hiveSyncExecutor.execute(this::doSyncHive, "sync hive metadata for instant %s", this.instant);
|
||||
}
|
||||
}
|
||||
|
||||
private void syncHive() {
|
||||
if (tableState.syncHive) {
|
||||
doSyncHive();
|
||||
LOG.info("Sync hive metadata for instant {} success!", this.instant);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sync hoodie table metadata to Hive metastore.
|
||||
*/
|
||||
public void doSyncHive() {
|
||||
hiveSyncContext.hiveSyncTool().syncHoodieTable();
|
||||
}
|
||||
|
||||
private void initMetadataSync() {
|
||||
this.writeClient.initMetadataWriter();
|
||||
}
|
||||
|
||||
private void reset() {
|
||||
this.eventBuffer = new WriteMetadataEvent[this.parallelism];
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks the buffer is ready to commit.
|
||||
*/
|
||||
private boolean allEventsReceived() {
|
||||
return Arrays.stream(eventBuffer)
|
||||
.allMatch(event -> event != null && event.isReady(this.instant));
|
||||
}
|
||||
|
||||
private void addEventToBuffer(WriteMetadataEvent event) {
|
||||
if (this.eventBuffer[event.getTaskID()] != null) {
|
||||
this.eventBuffer[event.getTaskID()].mergeWith(event);
|
||||
} else {
|
||||
this.eventBuffer[event.getTaskID()] = event;
|
||||
}
|
||||
}
|
||||
|
||||
private void startInstant() {
|
||||
// put the assignment in front of metadata generation,
|
||||
// because the instant request from write task is asynchronous.
|
||||
this.instant = this.writeClient.startCommit(tableState.commitAction, this.metaClient);
|
||||
this.metaClient.getActiveTimeline().transitionRequestedToInflight(tableState.commitAction, this.instant);
|
||||
this.ckpMetadata.startInstant(this.instant);
|
||||
LOG.info("Create instant [{}] for table [{}] with type [{}]", this.instant,
|
||||
this.conf.getString(FlinkOptions.TABLE_NAME), conf.getString(FlinkOptions.TABLE_TYPE));
|
||||
}
|
||||
|
||||
/**
|
||||
* Initializes the instant.
|
||||
*
|
||||
* <p>Recommits the last inflight instant if the write metadata checkpoint successfully
|
||||
* but was not committed due to some rare cases.
|
||||
*
|
||||
* <p>Starts a new instant, a writer can not flush data buffer
|
||||
* until it finds a new inflight instant on the timeline.
|
||||
*/
|
||||
private void initInstant(String instant) {
|
||||
HoodieTimeline completedTimeline =
|
||||
StreamerUtil.createMetaClient(conf).getActiveTimeline().filterCompletedInstants();
|
||||
executor.execute(() -> {
|
||||
if (instant.equals("") || completedTimeline.containsInstant(instant)) {
|
||||
// the last instant committed successfully
|
||||
reset();
|
||||
} else {
|
||||
LOG.info("Recommit instant {}", instant);
|
||||
commitInstant(instant);
|
||||
}
|
||||
// starts a new instant
|
||||
startInstant();
|
||||
// upgrade downgrade
|
||||
this.writeClient.upgradeDowngrade(this.instant);
|
||||
}, "initialize instant %s", instant);
|
||||
}
|
||||
|
||||
private void handleBootstrapEvent(WriteMetadataEvent event) {
|
||||
this.eventBuffer[event.getTaskID()] = event;
|
||||
if (Arrays.stream(eventBuffer).allMatch(evt -> evt != null && evt.isBootstrap())) {
|
||||
// start to initialize the instant.
|
||||
initInstant(event.getInstantTime());
|
||||
}
|
||||
}
|
||||
|
||||
private void handleEndInputEvent(WriteMetadataEvent event) {
|
||||
addEventToBuffer(event);
|
||||
if (allEventsReceived()) {
|
||||
// start to commit the instant.
|
||||
commitInstant(this.instant);
|
||||
// The executor thread inherits the classloader of the #handleEventFromOperator
|
||||
// caller, which is a AppClassLoader.
|
||||
Thread.currentThread().setContextClassLoader(getClass().getClassLoader());
|
||||
// sync Hive synchronously if it is enabled in batch mode.
|
||||
syncHive();
|
||||
}
|
||||
}
|
||||
|
||||
private void handleWriteMetaEvent(WriteMetadataEvent event) {
|
||||
// the write task does not block after checkpointing(and before it receives a checkpoint success event),
|
||||
// if it checkpoints succeed then flushes the data buffer again before this coordinator receives a checkpoint
|
||||
// success event, the data buffer would flush with an older instant time.
|
||||
ValidationUtils.checkState(
|
||||
HoodieTimeline.compareTimestamps(this.instant, HoodieTimeline.GREATER_THAN_OR_EQUALS, event.getInstantTime()),
|
||||
String.format("Receive an unexpected event for instant %s from task %d",
|
||||
event.getInstantTime(), event.getTaskID()));
|
||||
|
||||
addEventToBuffer(event);
|
||||
}
|
||||
|
||||
/**
|
||||
* The coordinator reuses the instant if there is no data for this round of checkpoint,
|
||||
* sends the commit ack events to unblock the flushing.
|
||||
*/
|
||||
private void sendCommitAckEvents(long checkpointId) {
|
||||
CompletableFuture<?>[] futures = Arrays.stream(this.gateways).filter(Objects::nonNull)
|
||||
.map(gw -> gw.sendEvent(CommitAckEvent.getInstance(checkpointId)))
|
||||
.toArray(CompletableFuture<?>[]::new);
|
||||
CompletableFuture.allOf(futures).whenComplete((resp, error) -> {
|
||||
if (!sendToFinishedTasks(error)) {
|
||||
throw new HoodieException("Error while waiting for the commit ack events to finish sending", error);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Decides whether the given exception is caused by sending events to FINISHED tasks.
|
||||
*
|
||||
* <p>Ugly impl: the exception may change in the future.
|
||||
*/
|
||||
private static boolean sendToFinishedTasks(Throwable throwable) {
|
||||
return throwable.getCause() instanceof TaskNotRunningException
|
||||
|| throwable.getCause().getMessage().contains("running");
|
||||
}
|
||||
|
||||
/**
|
||||
* Commits the instant.
|
||||
*/
|
||||
private void commitInstant(String instant) {
|
||||
commitInstant(instant, -1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Commits the instant.
|
||||
*
|
||||
* @return true if the write statuses are committed successfully.
|
||||
*/
|
||||
private boolean commitInstant(String instant, long checkpointId) {
|
||||
if (Arrays.stream(eventBuffer).allMatch(Objects::isNull)) {
|
||||
// The last checkpoint finished successfully.
|
||||
return false;
|
||||
}
|
||||
|
||||
List<WriteStatus> writeResults = Arrays.stream(eventBuffer)
|
||||
.filter(Objects::nonNull)
|
||||
.map(WriteMetadataEvent::getWriteStatuses)
|
||||
.flatMap(Collection::stream)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
if (writeResults.size() == 0) {
|
||||
// No data has written, reset the buffer and returns early
|
||||
reset();
|
||||
// Send commit ack event to the write function to unblock the flushing
|
||||
sendCommitAckEvents(checkpointId);
|
||||
return false;
|
||||
}
|
||||
doCommit(instant, writeResults);
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs the actual commit action.
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
private void doCommit(String instant, List<WriteStatus> writeResults) {
|
||||
// commit or rollback
|
||||
long totalErrorRecords = writeResults.stream().map(WriteStatus::getTotalErrorRecords).reduce(Long::sum).orElse(0L);
|
||||
long totalRecords = writeResults.stream().map(WriteStatus::getTotalRecords).reduce(Long::sum).orElse(0L);
|
||||
boolean hasErrors = totalErrorRecords > 0;
|
||||
|
||||
if (!hasErrors || this.conf.getBoolean(FlinkOptions.IGNORE_FAILED)) {
|
||||
HashMap<String, String> checkpointCommitMetadata = new HashMap<>();
|
||||
if (hasErrors) {
|
||||
LOG.warn("Some records failed to merge but forcing commit since commitOnErrors set to true. Errors/Total="
|
||||
+ totalErrorRecords + "/" + totalRecords);
|
||||
}
|
||||
|
||||
final Map<String, List<String>> partitionToReplacedFileIds = tableState.isOverwrite
|
||||
? writeClient.getPartitionToReplacedFileIds(tableState.operationType, writeResults)
|
||||
: Collections.emptyMap();
|
||||
boolean success = writeClient.commit(instant, writeResults, Option.of(checkpointCommitMetadata),
|
||||
tableState.commitAction, partitionToReplacedFileIds);
|
||||
if (success) {
|
||||
reset();
|
||||
this.ckpMetadata.commitInstant(instant);
|
||||
LOG.info("Commit instant [{}] success!", instant);
|
||||
} else {
|
||||
throw new HoodieException(String.format("Commit instant [%s] failed!", instant));
|
||||
}
|
||||
} else {
|
||||
LOG.error("Error when writing. Errors/Total=" + totalErrorRecords + "/" + totalRecords);
|
||||
LOG.error("The first 100 error messages");
|
||||
writeResults.stream().filter(WriteStatus::hasErrors).limit(100).forEach(ws -> {
|
||||
LOG.error("Global error for partition path {} and fileID {}: {}",
|
||||
ws.getGlobalError(), ws.getPartitionPath(), ws.getFileId());
|
||||
if (ws.getErrors().size() > 0) {
|
||||
ws.getErrors().forEach((key, value) -> LOG.trace("Error for key:" + key + " and value " + value));
|
||||
}
|
||||
});
|
||||
// Rolls back instant
|
||||
writeClient.rollback(instant);
|
||||
throw new HoodieException(String.format("Commit instant [%s] failed and rolled back !", instant));
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public WriteMetadataEvent[] getEventBuffer() {
|
||||
return eventBuffer;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public String getInstant() {
|
||||
return instant;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public Context getContext() {
|
||||
return context;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public void setExecutor(NonThrownExecutor executor) throws Exception {
|
||||
if (this.executor != null) {
|
||||
this.executor.close();
|
||||
}
|
||||
this.executor = executor;
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Inner Class
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
/**
|
||||
* Provider for {@link StreamWriteOperatorCoordinator}.
|
||||
*/
|
||||
public static class Provider implements OperatorCoordinator.Provider {
|
||||
private final OperatorID operatorId;
|
||||
private final Configuration conf;
|
||||
|
||||
public Provider(OperatorID operatorId, Configuration conf) {
|
||||
this.operatorId = operatorId;
|
||||
this.conf = conf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public OperatorID getOperatorId() {
|
||||
return this.operatorId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public OperatorCoordinator create(Context context) {
|
||||
return new StreamWriteOperatorCoordinator(this.conf, context);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Remember some table state variables.
|
||||
*/
|
||||
private static class TableState implements Serializable {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
final WriteOperationType operationType;
|
||||
final String commitAction;
|
||||
final boolean isOverwrite;
|
||||
final boolean scheduleCompaction;
|
||||
final boolean syncHive;
|
||||
final boolean syncMetadata;
|
||||
final boolean isDeltaTimeCompaction;
|
||||
|
||||
private TableState(Configuration conf) {
|
||||
this.operationType = WriteOperationType.fromValue(conf.getString(FlinkOptions.OPERATION));
|
||||
this.commitAction = CommitUtils.getCommitActionType(this.operationType,
|
||||
HoodieTableType.valueOf(conf.getString(FlinkOptions.TABLE_TYPE).toUpperCase(Locale.ROOT)));
|
||||
this.isOverwrite = WriteOperationType.isOverwrite(this.operationType);
|
||||
this.scheduleCompaction = StreamerUtil.needsScheduleCompaction(conf);
|
||||
this.syncHive = conf.getBoolean(FlinkOptions.HIVE_SYNC_ENABLED);
|
||||
this.syncMetadata = conf.getBoolean(FlinkOptions.METADATA_ENABLED);
|
||||
this.isDeltaTimeCompaction = OptionsResolver.isDeltaTimeCompaction(conf);
|
||||
}
|
||||
|
||||
public static TableState create(Configuration conf) {
|
||||
return new TableState(conf);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,144 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.append;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.sink.StreamWriteOperatorCoordinator;
|
||||
import org.apache.hudi.sink.bulk.BulkInsertWriterHelper;
|
||||
import org.apache.hudi.sink.common.AbstractStreamWriteFunction;
|
||||
import org.apache.hudi.sink.event.WriteMetadataEvent;
|
||||
|
||||
import org.apache.flink.annotation.VisibleForTesting;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
import org.apache.flink.util.Collector;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Sink function to write the data to the underneath filesystem.
|
||||
*
|
||||
* <p>The function writes base files directly for each checkpoint,
|
||||
* the file may roll over when it’s size hits the configured threshold.
|
||||
*
|
||||
* @param <I> Type of the input record
|
||||
* @see StreamWriteOperatorCoordinator
|
||||
*/
|
||||
public class AppendWriteFunction<I> extends AbstractStreamWriteFunction<I> {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(AppendWriteFunction.class);
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
/**
|
||||
* Helper class for log mode.
|
||||
*/
|
||||
private transient BulkInsertWriterHelper writerHelper;
|
||||
|
||||
/**
|
||||
* Table row type.
|
||||
*/
|
||||
private final RowType rowType;
|
||||
|
||||
/**
|
||||
* Constructs an AppendWriteFunction.
|
||||
*
|
||||
* @param config The config options
|
||||
*/
|
||||
public AppendWriteFunction(Configuration config, RowType rowType) {
|
||||
super(config);
|
||||
this.rowType = rowType;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void snapshotState() {
|
||||
// Based on the fact that the coordinator starts the checkpoint first,
|
||||
// it would check the validity.
|
||||
// wait for the buffer data flush out and request a new instant
|
||||
flushData(false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processElement(I value, Context ctx, Collector<Object> out) throws Exception {
|
||||
if (this.writerHelper == null) {
|
||||
initWriterHelper();
|
||||
}
|
||||
this.writerHelper.write((RowData) value);
|
||||
}
|
||||
|
||||
/**
|
||||
* End input action for batch source.
|
||||
*/
|
||||
public void endInput() {
|
||||
flushData(true);
|
||||
this.writeStatuses.clear();
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// GetterSetter
|
||||
// -------------------------------------------------------------------------
|
||||
@VisibleForTesting
|
||||
public BulkInsertWriterHelper getWriterHelper() {
|
||||
return this.writerHelper;
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Utilities
|
||||
// -------------------------------------------------------------------------
|
||||
private void initWriterHelper() {
|
||||
this.currentInstant = instantToWrite(true);
|
||||
if (this.currentInstant == null) {
|
||||
// in case there are empty checkpoints that has no input data
|
||||
throw new HoodieException("No inflight instant when flushing data!");
|
||||
}
|
||||
this.writerHelper = new BulkInsertWriterHelper(this.config, this.writeClient.getHoodieTable(), this.writeClient.getConfig(),
|
||||
this.currentInstant, this.taskID, getRuntimeContext().getNumberOfParallelSubtasks(), getRuntimeContext().getAttemptNumber(),
|
||||
this.rowType);
|
||||
}
|
||||
|
||||
private void flushData(boolean endInput) {
|
||||
final List<WriteStatus> writeStatus;
|
||||
final String instant;
|
||||
if (this.writerHelper != null) {
|
||||
writeStatus = this.writerHelper.getWriteStatuses(this.taskID);
|
||||
instant = this.writerHelper.getInstantTime();
|
||||
} else {
|
||||
LOG.info("No data to write in subtask [{}] for instant [{}]", taskID, currentInstant);
|
||||
writeStatus = Collections.emptyList();
|
||||
instant = instantToWrite(false);
|
||||
}
|
||||
final WriteMetadataEvent event = WriteMetadataEvent.builder()
|
||||
.taskID(taskID)
|
||||
.instantTime(instant)
|
||||
.writeStatus(writeStatus)
|
||||
.lastBatch(true)
|
||||
.endInput(endInput)
|
||||
.build();
|
||||
this.eventGateway.sendEventToCoordinator(event);
|
||||
// nullify the write helper for next ckp
|
||||
this.writerHelper = null;
|
||||
this.writeStatuses.addAll(writeStatus);
|
||||
// blocks flushing until the coordinator starts a new instant
|
||||
this.confirming = true;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,41 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.append;
|
||||
|
||||
import org.apache.hudi.sink.common.AbstractWriteOperator;
|
||||
import org.apache.hudi.sink.common.WriteOperatorFactory;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
|
||||
/**
|
||||
* Operator for {@link AppendWriteFunction}.
|
||||
*
|
||||
* @param <I> The input type
|
||||
*/
|
||||
public class AppendWriteOperator<I> extends AbstractWriteOperator<I> {
|
||||
|
||||
public AppendWriteOperator(Configuration conf, RowType rowType) {
|
||||
super(new AppendWriteFunction<>(conf, rowType));
|
||||
}
|
||||
|
||||
public static <I> WriteOperatorFactory<I> getFactory(Configuration conf, RowType rowType) {
|
||||
return WriteOperatorFactory.instance(conf, new AppendWriteOperator<>(conf, rowType));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,270 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.bootstrap;
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordGlobalLocation;
|
||||
import org.apache.hudi.common.table.TableSchemaResolver;
|
||||
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.BaseFileUtils;
|
||||
import org.apache.hudi.common.util.ClosableIterator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.sink.bootstrap.aggregate.BootstrapAggFunction;
|
||||
import org.apache.hudi.sink.meta.CkpMetadata;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.format.FormatUtils;
|
||||
import org.apache.hudi.util.FlinkTables;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.flink.annotation.VisibleForTesting;
|
||||
import org.apache.flink.api.common.state.ListState;
|
||||
import org.apache.flink.api.common.state.ListStateDescriptor;
|
||||
import org.apache.flink.api.common.typeinfo.Types;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
|
||||
import org.apache.flink.runtime.state.StateInitializationContext;
|
||||
import org.apache.flink.runtime.state.StateSnapshotContext;
|
||||
import org.apache.flink.runtime.taskexecutor.GlobalAggregateManager;
|
||||
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
|
||||
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
|
||||
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
import static org.apache.hudi.util.StreamerUtil.isValidFile;
|
||||
|
||||
/**
|
||||
* The operator to load index from existing hoodieTable.
|
||||
*
|
||||
* <p>Each subtask of the function triggers the index bootstrap when the first element came in,
|
||||
* the record cannot be sent until all the index records have been sent.
|
||||
*
|
||||
* <p>The output records should then shuffle by the recordKey and thus do scalable write.
|
||||
*/
|
||||
public class BootstrapOperator<I, O extends HoodieRecord<?>>
|
||||
extends AbstractStreamOperator<O> implements OneInputStreamOperator<I, O> {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(BootstrapOperator.class);
|
||||
|
||||
protected HoodieTable<?, ?, ?, ?> hoodieTable;
|
||||
|
||||
private CkpMetadata ckpMetadata;
|
||||
|
||||
protected final Configuration conf;
|
||||
|
||||
protected transient org.apache.hadoop.conf.Configuration hadoopConf;
|
||||
protected transient HoodieWriteConfig writeConfig;
|
||||
|
||||
private transient GlobalAggregateManager aggregateManager;
|
||||
|
||||
private transient ListState<String> instantState;
|
||||
private final Pattern pattern;
|
||||
private String lastInstantTime;
|
||||
|
||||
public BootstrapOperator(Configuration conf) {
|
||||
this.conf = conf;
|
||||
this.pattern = Pattern.compile(conf.getString(FlinkOptions.INDEX_PARTITION_REGEX));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void snapshotState(StateSnapshotContext context) throws Exception {
|
||||
lastInstantTime = this.ckpMetadata.lastPendingInstant();
|
||||
instantState.update(Collections.singletonList(lastInstantTime));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initializeState(StateInitializationContext context) throws Exception {
|
||||
ListStateDescriptor<String> instantStateDescriptor = new ListStateDescriptor<>(
|
||||
"instantStateDescriptor",
|
||||
Types.STRING
|
||||
);
|
||||
instantState = context.getOperatorStateStore().getListState(instantStateDescriptor);
|
||||
|
||||
if (context.isRestored()) {
|
||||
Iterator<String> instantIterator = instantState.get().iterator();
|
||||
if (instantIterator.hasNext()) {
|
||||
lastInstantTime = instantIterator.next();
|
||||
}
|
||||
}
|
||||
|
||||
this.hadoopConf = StreamerUtil.getHadoopConf();
|
||||
this.writeConfig = StreamerUtil.getHoodieClientConfig(this.conf, true);
|
||||
this.hoodieTable = FlinkTables.createTable(writeConfig, hadoopConf, getRuntimeContext());
|
||||
this.ckpMetadata = CkpMetadata.getInstance(hoodieTable.getMetaClient().getFs(), this.writeConfig.getBasePath());
|
||||
this.aggregateManager = getRuntimeContext().getGlobalAggregateManager();
|
||||
|
||||
preLoadIndexRecords();
|
||||
}
|
||||
|
||||
/**
|
||||
* Load the index records before {@link #processElement}.
|
||||
*/
|
||||
protected void preLoadIndexRecords() throws Exception {
|
||||
String basePath = hoodieTable.getMetaClient().getBasePath();
|
||||
int taskID = getRuntimeContext().getIndexOfThisSubtask();
|
||||
LOG.info("Start loading records in table {} into the index state, taskId = {}", basePath, taskID);
|
||||
for (String partitionPath : FSUtils.getAllFoldersWithPartitionMetaFile(FSUtils.getFs(basePath, hadoopConf), basePath)) {
|
||||
if (pattern.matcher(partitionPath).matches()) {
|
||||
loadRecords(partitionPath);
|
||||
}
|
||||
}
|
||||
|
||||
LOG.info("Finish sending index records, taskId = {}.", getRuntimeContext().getIndexOfThisSubtask());
|
||||
|
||||
// wait for the other bootstrap tasks finish bootstrapping.
|
||||
waitForBootstrapReady(getRuntimeContext().getIndexOfThisSubtask());
|
||||
}
|
||||
|
||||
/**
|
||||
* Wait for other bootstrap tasks to finish the index bootstrap.
|
||||
*/
|
||||
private void waitForBootstrapReady(int taskID) {
|
||||
int taskNum = getRuntimeContext().getNumberOfParallelSubtasks();
|
||||
int readyTaskNum = 1;
|
||||
while (taskNum != readyTaskNum) {
|
||||
try {
|
||||
readyTaskNum = aggregateManager.updateGlobalAggregate(BootstrapAggFunction.NAME, taskID, new BootstrapAggFunction());
|
||||
LOG.info("Waiting for other bootstrap tasks to complete, taskId = {}.", taskID);
|
||||
|
||||
TimeUnit.SECONDS.sleep(5);
|
||||
} catch (Exception e) {
|
||||
LOG.warn("Update global task bootstrap summary error", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public void processElement(StreamRecord<I> element) throws Exception {
|
||||
output.collect((StreamRecord<O>) element);
|
||||
}
|
||||
|
||||
/**
|
||||
* Loads all the indices of give partition path into the backup state.
|
||||
*
|
||||
* @param partitionPath The partition path
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
protected void loadRecords(String partitionPath) throws Exception {
|
||||
long start = System.currentTimeMillis();
|
||||
|
||||
final int parallelism = getRuntimeContext().getNumberOfParallelSubtasks();
|
||||
final int maxParallelism = getRuntimeContext().getMaxNumberOfParallelSubtasks();
|
||||
final int taskID = getRuntimeContext().getIndexOfThisSubtask();
|
||||
|
||||
HoodieTimeline commitsTimeline = this.hoodieTable.getMetaClient().getCommitsTimeline();
|
||||
if (!StringUtils.isNullOrEmpty(lastInstantTime)) {
|
||||
commitsTimeline = commitsTimeline.findInstantsAfter(lastInstantTime);
|
||||
}
|
||||
Option<HoodieInstant> latestCommitTime = commitsTimeline.filterCompletedInstants().lastInstant();
|
||||
|
||||
if (latestCommitTime.isPresent()) {
|
||||
BaseFileUtils fileUtils = BaseFileUtils.getInstance(this.hoodieTable.getBaseFileFormat());
|
||||
Schema schema = new TableSchemaResolver(this.hoodieTable.getMetaClient()).getTableAvroSchema();
|
||||
|
||||
List<FileSlice> fileSlices = this.hoodieTable.getSliceView()
|
||||
.getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.get().getTimestamp(), true)
|
||||
.collect(toList());
|
||||
|
||||
for (FileSlice fileSlice : fileSlices) {
|
||||
if (!shouldLoadFile(fileSlice.getFileId(), maxParallelism, parallelism, taskID)) {
|
||||
continue;
|
||||
}
|
||||
LOG.info("Load records from {}.", fileSlice);
|
||||
|
||||
// load parquet records
|
||||
fileSlice.getBaseFile().ifPresent(baseFile -> {
|
||||
// filter out crushed files
|
||||
if (!isValidFile(baseFile.getFileStatus())) {
|
||||
return;
|
||||
}
|
||||
try (ClosableIterator<HoodieKey> iterator = fileUtils.getHoodieKeyIterator(this.hadoopConf, new Path(baseFile.getPath()))) {
|
||||
iterator.forEachRemaining(hoodieKey -> {
|
||||
output.collect(new StreamRecord(new IndexRecord(generateHoodieRecord(hoodieKey, fileSlice))));
|
||||
});
|
||||
}
|
||||
});
|
||||
|
||||
// load avro log records
|
||||
List<String> logPaths = fileSlice.getLogFiles()
|
||||
// filter out crushed files
|
||||
.filter(logFile -> isValidFile(logFile.getFileStatus()))
|
||||
.map(logFile -> logFile.getPath().toString())
|
||||
.collect(toList());
|
||||
HoodieMergedLogRecordScanner scanner = FormatUtils.logScanner(logPaths, schema, latestCommitTime.get().getTimestamp(),
|
||||
writeConfig, hadoopConf);
|
||||
|
||||
try {
|
||||
for (String recordKey : scanner.getRecords().keySet()) {
|
||||
output.collect(new StreamRecord(new IndexRecord(generateHoodieRecord(new HoodieKey(recordKey, partitionPath), fileSlice))));
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(String.format("Error when loading record keys from files: %s", logPaths), e);
|
||||
} finally {
|
||||
scanner.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
long cost = System.currentTimeMillis() - start;
|
||||
LOG.info("Task [{}}:{}}] finish loading the index under partition {} and sending them to downstream, time cost: {} milliseconds.",
|
||||
this.getClass().getSimpleName(), taskID, partitionPath, cost);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public static HoodieRecord generateHoodieRecord(HoodieKey hoodieKey, FileSlice fileSlice) {
|
||||
HoodieRecord hoodieRecord = new HoodieAvroRecord(hoodieKey, null);
|
||||
hoodieRecord.setCurrentLocation(new HoodieRecordGlobalLocation(hoodieKey.getPartitionPath(), fileSlice.getBaseInstantTime(), fileSlice.getFileId()));
|
||||
hoodieRecord.seal();
|
||||
|
||||
return hoodieRecord;
|
||||
}
|
||||
|
||||
protected boolean shouldLoadFile(String fileId,
|
||||
int maxParallelism,
|
||||
int parallelism,
|
||||
int taskID) {
|
||||
return KeyGroupRangeAssignment.assignKeyToParallelOperator(
|
||||
fileId, maxParallelism, parallelism) == taskID;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public boolean isAlreadyBootstrap() throws Exception {
|
||||
return instantState.get().iterator().hasNext();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,39 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.bootstrap;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
|
||||
/**
|
||||
* The index record.
|
||||
*/
|
||||
public class IndexRecord<T extends HoodieRecordPayload> extends HoodieAvroRecord<T> {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public IndexRecord(HoodieRecord<T> record) {
|
||||
super(record);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieRecord<T> newInstance() {
|
||||
return new IndexRecord<>(this);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,53 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.bootstrap.aggregate;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Bootstrap ready task id accumulator.
|
||||
*/
|
||||
public class BootstrapAccumulator implements Serializable {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
private final Set<Integer> readyTaskSet;
|
||||
|
||||
public BootstrapAccumulator() {
|
||||
this.readyTaskSet = new HashSet<>();
|
||||
}
|
||||
|
||||
public void update(int taskId) {
|
||||
readyTaskSet.add(taskId);
|
||||
}
|
||||
|
||||
public int readyTaskNum() {
|
||||
return readyTaskSet.size();
|
||||
}
|
||||
|
||||
public BootstrapAccumulator merge(BootstrapAccumulator acc) {
|
||||
if (acc == null) {
|
||||
return this;
|
||||
}
|
||||
|
||||
readyTaskSet.addAll(acc.readyTaskSet);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,50 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.bootstrap.aggregate;
|
||||
|
||||
import org.apache.flink.api.common.functions.AggregateFunction;
|
||||
|
||||
/**
|
||||
* Aggregate function that accumulates the loaded task number of
|
||||
* function {@link org.apache.hudi.sink.bootstrap.BootstrapOperator}.
|
||||
*/
|
||||
public class BootstrapAggFunction implements AggregateFunction<Integer, BootstrapAccumulator, Integer> {
|
||||
public static final String NAME = BootstrapAggFunction.class.getSimpleName();
|
||||
|
||||
@Override
|
||||
public BootstrapAccumulator createAccumulator() {
|
||||
return new BootstrapAccumulator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public BootstrapAccumulator add(Integer taskId, BootstrapAccumulator bootstrapAccumulator) {
|
||||
bootstrapAccumulator.update(taskId);
|
||||
return bootstrapAccumulator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Integer getResult(BootstrapAccumulator bootstrapAccumulator) {
|
||||
return bootstrapAccumulator.readyTaskNum();
|
||||
}
|
||||
|
||||
@Override
|
||||
public BootstrapAccumulator merge(BootstrapAccumulator bootstrapAccumulator, BootstrapAccumulator acc) {
|
||||
return bootstrapAccumulator.merge(acc);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,84 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.bootstrap.batch;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.sink.bootstrap.BootstrapOperator;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
|
||||
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* The operator to load index from existing hoodieTable.
|
||||
*
|
||||
* <p>This function should only be used for bounded source.
|
||||
*
|
||||
* <p>When a record comes in, the function firstly checks whether the partition path of the record is already loaded,
|
||||
* if the partition is not loaded yet, loads the entire partition and sends the index records to downstream operators
|
||||
* before it sends the input record; if the partition is loaded already, sends the input record directly.
|
||||
*
|
||||
* <p>The input records should shuffle by the partition path to avoid repeated loading.
|
||||
*/
|
||||
public class BatchBootstrapOperator<I, O extends HoodieRecord<?>>
|
||||
extends BootstrapOperator<I, O> {
|
||||
|
||||
private Set<String> partitionPathSet;
|
||||
private boolean haveSuccessfulCommits;
|
||||
|
||||
public BatchBootstrapOperator(Configuration conf) {
|
||||
super(conf);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open() throws Exception {
|
||||
super.open();
|
||||
this.partitionPathSet = new HashSet<>();
|
||||
this.haveSuccessfulCommits = StreamerUtil.haveSuccessfulCommits(hoodieTable.getMetaClient());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void preLoadIndexRecords() {
|
||||
// no operation
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public void processElement(StreamRecord<I> element) throws Exception {
|
||||
final HoodieRecord<?> record = (HoodieRecord<?>) element.getValue();
|
||||
final String partitionPath = record.getKey().getPartitionPath();
|
||||
|
||||
if (haveSuccessfulCommits && !partitionPathSet.contains(partitionPath)) {
|
||||
loadRecords(partitionPath);
|
||||
partitionPathSet.add(partitionPath);
|
||||
}
|
||||
|
||||
// send the trigger record
|
||||
output.collect((StreamRecord<O>) element);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean shouldLoadFile(String fileId, int maxParallelism, int parallelism, int taskID) {
|
||||
// load all the file groups in the partition
|
||||
return true;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,212 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.bulk;
|
||||
|
||||
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.sink.StreamWriteOperatorCoordinator;
|
||||
import org.apache.hudi.sink.common.AbstractWriteFunction;
|
||||
import org.apache.hudi.sink.event.WriteMetadataEvent;
|
||||
import org.apache.hudi.sink.meta.CkpMetadata;
|
||||
import org.apache.hudi.sink.utils.TimeWait;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorEventGateway;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
import org.apache.flink.util.Collector;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Sink function to write the data to the underneath filesystem.
|
||||
*
|
||||
* <p>The function should only be used in operation type {@link WriteOperationType#BULK_INSERT}.
|
||||
*
|
||||
* <p>Note: The function task requires the input stream be shuffled by partition path.
|
||||
*
|
||||
* @param <I> Type of the input record
|
||||
* @see StreamWriteOperatorCoordinator
|
||||
*/
|
||||
public class BulkInsertWriteFunction<I>
|
||||
extends AbstractWriteFunction<I> {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(BulkInsertWriteFunction.class);
|
||||
|
||||
/**
|
||||
* Helper class for bulk insert mode.
|
||||
*/
|
||||
private transient BulkInsertWriterHelper writerHelper;
|
||||
|
||||
/**
|
||||
* Config options.
|
||||
*/
|
||||
private final Configuration config;
|
||||
|
||||
/**
|
||||
* Table row type.
|
||||
*/
|
||||
private final RowType rowType;
|
||||
|
||||
/**
|
||||
* Id of current subtask.
|
||||
*/
|
||||
private int taskID;
|
||||
|
||||
/**
|
||||
* Write Client.
|
||||
*/
|
||||
private transient HoodieFlinkWriteClient writeClient;
|
||||
|
||||
/**
|
||||
* The initial inflight instant when start up.
|
||||
*/
|
||||
private volatile String initInstant;
|
||||
|
||||
/**
|
||||
* Gateway to send operator events to the operator coordinator.
|
||||
*/
|
||||
private transient OperatorEventGateway eventGateway;
|
||||
|
||||
/**
|
||||
* Checkpoint metadata.
|
||||
*/
|
||||
private CkpMetadata ckpMetadata;
|
||||
|
||||
/**
|
||||
* Constructs a StreamingSinkFunction.
|
||||
*
|
||||
* @param config The config options
|
||||
*/
|
||||
public BulkInsertWriteFunction(Configuration config, RowType rowType) {
|
||||
this.config = config;
|
||||
this.rowType = rowType;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(Configuration parameters) throws IOException {
|
||||
this.taskID = getRuntimeContext().getIndexOfThisSubtask();
|
||||
this.writeClient = StreamerUtil.createWriteClient(this.config, getRuntimeContext());
|
||||
this.ckpMetadata = CkpMetadata.getInstance(config.getString(FlinkOptions.PATH));
|
||||
this.initInstant = lastPendingInstant();
|
||||
sendBootstrapEvent();
|
||||
initWriterHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processElement(I value, Context ctx, Collector<Object> out) throws IOException {
|
||||
this.writerHelper.write((RowData) value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
if (this.writeClient != null) {
|
||||
this.writeClient.cleanHandlesGracefully();
|
||||
this.writeClient.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* End input action for batch source.
|
||||
*/
|
||||
public void endInput() {
|
||||
final List<WriteStatus> writeStatus = this.writerHelper.getWriteStatuses(this.taskID);
|
||||
|
||||
final WriteMetadataEvent event = WriteMetadataEvent.builder()
|
||||
.taskID(taskID)
|
||||
.instantTime(this.writerHelper.getInstantTime())
|
||||
.writeStatus(writeStatus)
|
||||
.lastBatch(true)
|
||||
.endInput(true)
|
||||
.build();
|
||||
this.eventGateway.sendEventToCoordinator(event);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handleOperatorEvent(OperatorEvent event) {
|
||||
// no operation
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Getter/Setter
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
public void setOperatorEventGateway(OperatorEventGateway operatorEventGateway) {
|
||||
this.eventGateway = operatorEventGateway;
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Utilities
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
private void initWriterHelper() {
|
||||
String instant = instantToWrite();
|
||||
this.writerHelper = new BulkInsertWriterHelper(this.config, this.writeClient.getHoodieTable(), this.writeClient.getConfig(),
|
||||
instant, this.taskID, getRuntimeContext().getNumberOfParallelSubtasks(), getRuntimeContext().getAttemptNumber(),
|
||||
this.rowType);
|
||||
}
|
||||
|
||||
private void sendBootstrapEvent() {
|
||||
WriteMetadataEvent event = WriteMetadataEvent.builder()
|
||||
.taskID(taskID)
|
||||
.writeStatus(Collections.emptyList())
|
||||
.instantTime("")
|
||||
.bootstrap(true)
|
||||
.build();
|
||||
this.eventGateway.sendEventToCoordinator(event);
|
||||
LOG.info("Send bootstrap write metadata event to coordinator, task[{}].", taskID);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the last pending instant time.
|
||||
*/
|
||||
protected String lastPendingInstant() {
|
||||
return this.ckpMetadata.lastPendingInstant();
|
||||
}
|
||||
|
||||
private String instantToWrite() {
|
||||
String instant = lastPendingInstant();
|
||||
// if exactly-once semantics turns on,
|
||||
// waits for the checkpoint notification until the checkpoint timeout threshold hits.
|
||||
TimeWait timeWait = TimeWait.builder()
|
||||
.timeout(config.getLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT))
|
||||
.action("instant initialize")
|
||||
.build();
|
||||
while (instant == null || instant.equals(this.initInstant)) {
|
||||
// wait condition:
|
||||
// 1. there is no inflight instant
|
||||
// 2. the inflight instant does not change
|
||||
// sleep for a while
|
||||
timeWait.waitFor();
|
||||
// refresh the inflight instant
|
||||
instant = lastPendingInstant();
|
||||
}
|
||||
return instant;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,50 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.bulk;
|
||||
|
||||
import org.apache.hudi.sink.common.AbstractWriteOperator;
|
||||
import org.apache.hudi.sink.common.WriteOperatorFactory;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
|
||||
import org.apache.flink.streaming.api.operators.BoundedOneInput;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
|
||||
/**
|
||||
* Operator for bulk insert mode sink.
|
||||
*
|
||||
* @param <I> The input type
|
||||
*/
|
||||
public class BulkInsertWriteOperator<I>
|
||||
extends AbstractWriteOperator<I>
|
||||
implements BoundedOneInput {
|
||||
|
||||
public BulkInsertWriteOperator(Configuration conf, RowType rowType) {
|
||||
super(new BulkInsertWriteFunction<>(conf, rowType));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handleOperatorEvent(OperatorEvent event) {
|
||||
// no operation
|
||||
}
|
||||
|
||||
public static <I> WriteOperatorFactory<I> getFactory(Configuration conf, RowType rowType) {
|
||||
return WriteOperatorFactory.instance(conf, new BulkInsertWriteOperator<>(conf, rowType));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,201 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.bulk;
|
||||
|
||||
import org.apache.hudi.client.HoodieInternalWriteStatus;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.io.storage.row.HoodieRowDataCreateHandle;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.table.api.DataTypes;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.types.logical.LogicalType;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Helper class for bulk insert used by Flink.
|
||||
*/
|
||||
public class BulkInsertWriterHelper {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(BulkInsertWriterHelper.class);
|
||||
|
||||
private final String instantTime;
|
||||
private final int taskPartitionId;
|
||||
private final long taskId;
|
||||
private final long taskEpochId;
|
||||
private final HoodieTable hoodieTable;
|
||||
private final HoodieWriteConfig writeConfig;
|
||||
private final RowType rowType;
|
||||
private final Boolean arePartitionRecordsSorted;
|
||||
private final List<HoodieInternalWriteStatus> writeStatusList = new ArrayList<>();
|
||||
private HoodieRowDataCreateHandle handle;
|
||||
private String lastKnownPartitionPath = null;
|
||||
private final String fileIdPrefix;
|
||||
private int numFilesWritten = 0;
|
||||
private final Map<String, HoodieRowDataCreateHandle> handles = new HashMap<>();
|
||||
private final RowDataKeyGen keyGen;
|
||||
|
||||
public BulkInsertWriterHelper(Configuration conf, HoodieTable hoodieTable, HoodieWriteConfig writeConfig,
|
||||
String instantTime, int taskPartitionId, long taskId, long taskEpochId, RowType rowType) {
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.writeConfig = writeConfig;
|
||||
this.instantTime = instantTime;
|
||||
this.taskPartitionId = taskPartitionId;
|
||||
this.taskId = taskId;
|
||||
this.taskEpochId = taskEpochId;
|
||||
this.rowType = addMetadataFields(rowType, writeConfig.allowOperationMetadataField()); // patch up with metadata fields
|
||||
this.arePartitionRecordsSorted = conf.getBoolean(FlinkOptions.WRITE_BULK_INSERT_SORT_BY_PARTITION);
|
||||
this.fileIdPrefix = UUID.randomUUID().toString();
|
||||
this.keyGen = RowDataKeyGen.instance(conf, rowType);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the write instant time.
|
||||
*/
|
||||
public String getInstantTime() {
|
||||
return this.instantTime;
|
||||
}
|
||||
|
||||
public void write(RowData record) throws IOException {
|
||||
try {
|
||||
String recordKey = keyGen.getRecordKey(record);
|
||||
String partitionPath = keyGen.getPartitionPath(record);
|
||||
|
||||
if ((lastKnownPartitionPath == null) || !lastKnownPartitionPath.equals(partitionPath) || !handle.canWrite()) {
|
||||
LOG.info("Creating new file for partition path " + partitionPath);
|
||||
handle = getRowCreateHandle(partitionPath);
|
||||
lastKnownPartitionPath = partitionPath;
|
||||
}
|
||||
handle.write(recordKey, partitionPath, record);
|
||||
} catch (Throwable t) {
|
||||
LOG.error("Global error thrown while trying to write records in HoodieRowCreateHandle ", t);
|
||||
throw t;
|
||||
}
|
||||
}
|
||||
|
||||
public List<HoodieInternalWriteStatus> getHoodieWriteStatuses() throws IOException {
|
||||
close();
|
||||
return writeStatusList;
|
||||
}
|
||||
|
||||
private HoodieRowDataCreateHandle getRowCreateHandle(String partitionPath) throws IOException {
|
||||
if (!handles.containsKey(partitionPath)) { // if there is no handle corresponding to the partition path
|
||||
// if records are sorted, we can close all existing handles
|
||||
if (arePartitionRecordsSorted) {
|
||||
close();
|
||||
}
|
||||
HoodieRowDataCreateHandle rowCreateHandle = new HoodieRowDataCreateHandle(hoodieTable, writeConfig, partitionPath, getNextFileId(),
|
||||
instantTime, taskPartitionId, taskId, taskEpochId, rowType);
|
||||
handles.put(partitionPath, rowCreateHandle);
|
||||
} else if (!handles.get(partitionPath).canWrite()) {
|
||||
// even if there is a handle to the partition path, it could have reached its max size threshold. So, we close the handle here and
|
||||
// create a new one.
|
||||
writeStatusList.add(handles.remove(partitionPath).close());
|
||||
HoodieRowDataCreateHandle rowCreateHandle = new HoodieRowDataCreateHandle(hoodieTable, writeConfig, partitionPath, getNextFileId(),
|
||||
instantTime, taskPartitionId, taskId, taskEpochId, rowType);
|
||||
handles.put(partitionPath, rowCreateHandle);
|
||||
}
|
||||
return handles.get(partitionPath);
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
for (HoodieRowDataCreateHandle rowCreateHandle : handles.values()) {
|
||||
writeStatusList.add(rowCreateHandle.close());
|
||||
}
|
||||
handles.clear();
|
||||
handle = null;
|
||||
}
|
||||
|
||||
private String getNextFileId() {
|
||||
return String.format("%s-%d", fileIdPrefix, numFilesWritten++);
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds the Hoodie metadata fields to the given row type.
|
||||
*/
|
||||
private static RowType addMetadataFields(RowType rowType, boolean withOperationField) {
|
||||
List<RowType.RowField> mergedFields = new ArrayList<>();
|
||||
|
||||
LogicalType metadataFieldType = DataTypes.STRING().getLogicalType();
|
||||
RowType.RowField commitTimeField =
|
||||
new RowType.RowField(HoodieRecord.COMMIT_TIME_METADATA_FIELD, metadataFieldType, "commit time");
|
||||
RowType.RowField commitSeqnoField =
|
||||
new RowType.RowField(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, metadataFieldType, "commit seqno");
|
||||
RowType.RowField recordKeyField =
|
||||
new RowType.RowField(HoodieRecord.RECORD_KEY_METADATA_FIELD, metadataFieldType, "record key");
|
||||
RowType.RowField partitionPathField =
|
||||
new RowType.RowField(HoodieRecord.PARTITION_PATH_METADATA_FIELD, metadataFieldType, "partition path");
|
||||
RowType.RowField fileNameField =
|
||||
new RowType.RowField(HoodieRecord.FILENAME_METADATA_FIELD, metadataFieldType, "field name");
|
||||
|
||||
mergedFields.add(commitTimeField);
|
||||
mergedFields.add(commitSeqnoField);
|
||||
mergedFields.add(recordKeyField);
|
||||
mergedFields.add(partitionPathField);
|
||||
mergedFields.add(fileNameField);
|
||||
|
||||
if (withOperationField) {
|
||||
RowType.RowField operationField =
|
||||
new RowType.RowField(HoodieRecord.OPERATION_METADATA_FIELD, metadataFieldType, "operation");
|
||||
mergedFields.add(operationField);
|
||||
}
|
||||
|
||||
mergedFields.addAll(rowType.getFields());
|
||||
|
||||
return new RowType(false, mergedFields);
|
||||
}
|
||||
|
||||
public List<WriteStatus> getWriteStatuses(int taskID) {
|
||||
try {
|
||||
return getHoodieWriteStatuses().stream()
|
||||
.map(BulkInsertWriterHelper::toWriteStatus).collect(Collectors.toList());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Error collect the write status for task [" + taskID + "]", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Tool to convert {@link HoodieInternalWriteStatus} into {@link WriteStatus}.
|
||||
*/
|
||||
private static WriteStatus toWriteStatus(HoodieInternalWriteStatus internalWriteStatus) {
|
||||
WriteStatus writeStatus = new WriteStatus(false, 0.1);
|
||||
writeStatus.setStat(internalWriteStatus.getStat());
|
||||
writeStatus.setFileId(internalWriteStatus.getFileId());
|
||||
writeStatus.setGlobalError(internalWriteStatus.getGlobalError());
|
||||
writeStatus.setTotalRecords(internalWriteStatus.getTotalRecords());
|
||||
writeStatus.setTotalErrorRecords(internalWriteStatus.getTotalErrorRecords());
|
||||
return writeStatus;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,264 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.bulk;
|
||||
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.PartitionPathEncodeUtils;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.exception.HoodieKeyException;
|
||||
import org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator;
|
||||
import org.apache.hudi.util.RowDataProjection;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.data.TimestampData;
|
||||
import org.apache.flink.table.types.logical.LogicalType;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Key generator for {@link RowData}.
|
||||
*/
|
||||
public class RowDataKeyGen implements Serializable {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
// reference: NonpartitionedAvroKeyGenerator
|
||||
private static final String EMPTY_PARTITION = "";
|
||||
|
||||
// reference: org.apache.hudi.keygen.KeyGenUtils
|
||||
private static final String NULL_RECORDKEY_PLACEHOLDER = "__null__";
|
||||
private static final String EMPTY_RECORDKEY_PLACEHOLDER = "__empty__";
|
||||
|
||||
private static final String DEFAULT_PARTITION_PATH = "default";
|
||||
private static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
|
||||
|
||||
private final String[] recordKeyFields;
|
||||
private final String[] partitionPathFields;
|
||||
|
||||
private final RowDataProjection recordKeyProjection;
|
||||
private final RowDataProjection partitionPathProjection;
|
||||
|
||||
private final boolean hiveStylePartitioning;
|
||||
private final boolean encodePartitionPath;
|
||||
|
||||
private final Option<TimestampBasedAvroKeyGenerator> keyGenOpt;
|
||||
|
||||
// efficient code path
|
||||
private boolean simpleRecordKey = false;
|
||||
private RowData.FieldGetter recordKeyFieldGetter;
|
||||
|
||||
private boolean simplePartitionPath = false;
|
||||
private RowData.FieldGetter partitionPathFieldGetter;
|
||||
|
||||
private boolean nonPartitioned;
|
||||
|
||||
private RowDataKeyGen(
|
||||
String recordKeys,
|
||||
String partitionFields,
|
||||
RowType rowType,
|
||||
boolean hiveStylePartitioning,
|
||||
boolean encodePartitionPath,
|
||||
Option<TimestampBasedAvroKeyGenerator> keyGenOpt) {
|
||||
this.recordKeyFields = recordKeys.split(",");
|
||||
this.partitionPathFields = partitionFields.split(",");
|
||||
List<String> fieldNames = rowType.getFieldNames();
|
||||
List<LogicalType> fieldTypes = rowType.getChildren();
|
||||
|
||||
this.hiveStylePartitioning = hiveStylePartitioning;
|
||||
this.encodePartitionPath = encodePartitionPath;
|
||||
if (this.recordKeyFields.length == 1) {
|
||||
// efficient code path
|
||||
this.simpleRecordKey = true;
|
||||
int recordKeyIdx = fieldNames.indexOf(this.recordKeyFields[0]);
|
||||
this.recordKeyFieldGetter = RowData.createFieldGetter(fieldTypes.get(recordKeyIdx), recordKeyIdx);
|
||||
this.recordKeyProjection = null;
|
||||
} else {
|
||||
this.recordKeyProjection = getProjection(this.recordKeyFields, fieldNames, fieldTypes);
|
||||
}
|
||||
if (this.partitionPathFields.length == 1) {
|
||||
// efficient code path
|
||||
if (this.partitionPathFields[0].equals("")) {
|
||||
this.nonPartitioned = true;
|
||||
} else {
|
||||
this.simplePartitionPath = true;
|
||||
int partitionPathIdx = fieldNames.indexOf(this.partitionPathFields[0]);
|
||||
this.partitionPathFieldGetter = RowData.createFieldGetter(fieldTypes.get(partitionPathIdx), partitionPathIdx);
|
||||
}
|
||||
this.partitionPathProjection = null;
|
||||
} else {
|
||||
this.partitionPathProjection = getProjection(this.partitionPathFields, fieldNames, fieldTypes);
|
||||
}
|
||||
this.keyGenOpt = keyGenOpt;
|
||||
}
|
||||
|
||||
public static RowDataKeyGen instance(Configuration conf, RowType rowType) {
|
||||
Option<TimestampBasedAvroKeyGenerator> keyGeneratorOpt = Option.empty();
|
||||
if (conf.getString(FlinkOptions.KEYGEN_CLASS_NAME).equals(TimestampBasedAvroKeyGenerator.class.getName())) {
|
||||
try {
|
||||
keyGeneratorOpt = Option.of(new TimestampBasedAvroKeyGenerator(StreamerUtil.flinkConf2TypedProperties(conf)));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieKeyException("Initialize TimestampBasedAvroKeyGenerator error", e);
|
||||
}
|
||||
}
|
||||
return new RowDataKeyGen(conf.getString(FlinkOptions.RECORD_KEY_FIELD), conf.getString(FlinkOptions.PARTITION_PATH_FIELD),
|
||||
rowType, conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING), conf.getBoolean(FlinkOptions.URL_ENCODE_PARTITIONING),
|
||||
keyGeneratorOpt);
|
||||
}
|
||||
|
||||
public String getRecordKey(RowData rowData) {
|
||||
if (this.simpleRecordKey) {
|
||||
return getRecordKey(recordKeyFieldGetter.getFieldOrNull(rowData), this.recordKeyFields[0]);
|
||||
} else {
|
||||
Object[] keyValues = this.recordKeyProjection.projectAsValues(rowData);
|
||||
return getRecordKey(keyValues, this.recordKeyFields);
|
||||
}
|
||||
}
|
||||
|
||||
public String getPartitionPath(RowData rowData) {
|
||||
if (this.simplePartitionPath) {
|
||||
return getPartitionPath(partitionPathFieldGetter.getFieldOrNull(rowData),
|
||||
this.partitionPathFields[0], this.hiveStylePartitioning, this.encodePartitionPath, this.keyGenOpt);
|
||||
} else if (this.nonPartitioned) {
|
||||
return EMPTY_PARTITION;
|
||||
} else {
|
||||
Object[] partValues = this.partitionPathProjection.projectAsValues(rowData);
|
||||
return getRecordPartitionPath(partValues, this.partitionPathFields, this.hiveStylePartitioning, this.encodePartitionPath);
|
||||
}
|
||||
}
|
||||
|
||||
// reference: org.apache.hudi.keygen.KeyGenUtils.getRecordPartitionPath
|
||||
private static String getRecordKey(Object[] keyValues, String[] keyFields) {
|
||||
boolean keyIsNullEmpty = true;
|
||||
StringBuilder recordKey = new StringBuilder();
|
||||
for (int i = 0; i < keyValues.length; i++) {
|
||||
String recordKeyField = keyFields[i];
|
||||
String recordKeyValue = StringUtils.objToString(keyValues[i]);
|
||||
if (recordKeyValue == null) {
|
||||
recordKey.append(recordKeyField).append(":").append(NULL_RECORDKEY_PLACEHOLDER).append(",");
|
||||
} else if (recordKeyValue.isEmpty()) {
|
||||
recordKey.append(recordKeyField).append(":").append(EMPTY_RECORDKEY_PLACEHOLDER).append(",");
|
||||
} else {
|
||||
recordKey.append(recordKeyField).append(":").append(recordKeyValue).append(",");
|
||||
keyIsNullEmpty = false;
|
||||
}
|
||||
}
|
||||
recordKey.deleteCharAt(recordKey.length() - 1);
|
||||
if (keyIsNullEmpty) {
|
||||
throw new HoodieKeyException("recordKey values: \"" + recordKey + "\" for fields: "
|
||||
+ Arrays.toString(keyFields) + " cannot be entirely null or empty.");
|
||||
}
|
||||
return recordKey.toString();
|
||||
}
|
||||
|
||||
// reference: org.apache.hudi.keygen.KeyGenUtils.getRecordPartitionPath
|
||||
private static String getRecordPartitionPath(
|
||||
Object[] partValues,
|
||||
String[] partFields,
|
||||
boolean hiveStylePartitioning,
|
||||
boolean encodePartitionPath) {
|
||||
StringBuilder partitionPath = new StringBuilder();
|
||||
for (int i = 0; i < partFields.length; i++) {
|
||||
String partField = partFields[i];
|
||||
String partValue = StringUtils.objToString(partValues[i]);
|
||||
if (partValue == null || partValue.isEmpty()) {
|
||||
partitionPath.append(hiveStylePartitioning ? partField + "=" + DEFAULT_PARTITION_PATH
|
||||
: DEFAULT_PARTITION_PATH);
|
||||
} else {
|
||||
if (encodePartitionPath) {
|
||||
partValue = PartitionPathEncodeUtils.escapePathName(partValue);
|
||||
}
|
||||
partitionPath.append(hiveStylePartitioning ? partField + "=" + partValue : partValue);
|
||||
}
|
||||
partitionPath.append(DEFAULT_PARTITION_PATH_SEPARATOR);
|
||||
}
|
||||
partitionPath.deleteCharAt(partitionPath.length() - 1);
|
||||
return partitionPath.toString();
|
||||
}
|
||||
|
||||
// reference: org.apache.hudi.keygen.KeyGenUtils.getRecordKey
|
||||
public static String getRecordKey(Object recordKeyValue, String recordKeyField) {
|
||||
String recordKey = StringUtils.objToString(recordKeyValue);
|
||||
if (recordKey == null || recordKey.isEmpty()) {
|
||||
throw new HoodieKeyException("recordKey value: \"" + recordKey + "\" for field: \"" + recordKeyField + "\" cannot be null or empty.");
|
||||
}
|
||||
return recordKey;
|
||||
}
|
||||
|
||||
// reference: org.apache.hudi.keygen.KeyGenUtils.getPartitionPath
|
||||
public static String getPartitionPath(
|
||||
Object partValue,
|
||||
String partField,
|
||||
boolean hiveStylePartitioning,
|
||||
boolean encodePartitionPath,
|
||||
Option<TimestampBasedAvroKeyGenerator> keyGenOpt) {
|
||||
if (keyGenOpt.isPresent()) {
|
||||
TimestampBasedAvroKeyGenerator keyGenerator = keyGenOpt.get();
|
||||
return keyGenerator.getPartitionPath(toEpochMilli(partValue, keyGenerator));
|
||||
}
|
||||
String partitionPath = StringUtils.objToString(partValue);
|
||||
if (partitionPath == null || partitionPath.isEmpty()) {
|
||||
partitionPath = DEFAULT_PARTITION_PATH;
|
||||
}
|
||||
if (encodePartitionPath) {
|
||||
partitionPath = PartitionPathEncodeUtils.escapePathName(partitionPath);
|
||||
}
|
||||
if (hiveStylePartitioning) {
|
||||
partitionPath = partField + "=" + partitionPath;
|
||||
}
|
||||
return partitionPath;
|
||||
}
|
||||
|
||||
private static Object toEpochMilli(Object val, TimestampBasedAvroKeyGenerator keyGenerator) {
|
||||
if (val instanceof TimestampData) {
|
||||
return ((TimestampData) val).toInstant().toEpochMilli();
|
||||
}
|
||||
if (val == null) {
|
||||
// should match the default partition path when STRING partition path re-format is supported
|
||||
return keyGenerator.getDefaultPartitionVal();
|
||||
}
|
||||
return val;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the row data projection for the given field names and table schema.
|
||||
*
|
||||
* @param fields The projected field names
|
||||
* @param schemaFields The table schema names
|
||||
* @param schemaTypes The table schema types
|
||||
* @return the row data projection for the fields
|
||||
*/
|
||||
private static RowDataProjection getProjection(String[] fields, List<String> schemaFields, List<LogicalType> schemaTypes) {
|
||||
int[] positions = getFieldPositions(fields, schemaFields);
|
||||
LogicalType[] types = Arrays.stream(positions).mapToObj(schemaTypes::get).toArray(LogicalType[]::new);
|
||||
return RowDataProjection.instance(types, positions);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the field positions of the given fields {@code fields} among all the fields {@code allFields}.
|
||||
*/
|
||||
private static int[] getFieldPositions(String[] fields, List<String> allFields) {
|
||||
return Arrays.stream(fields).mapToInt(allFields::indexOf).toArray();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,125 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.bulk.sort;
|
||||
|
||||
import org.apache.flink.metrics.Gauge;
|
||||
import org.apache.flink.runtime.memory.MemoryManager;
|
||||
import org.apache.flink.streaming.api.operators.BoundedOneInput;
|
||||
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
|
||||
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.data.binary.BinaryRowData;
|
||||
import org.apache.flink.table.runtime.generated.GeneratedNormalizedKeyComputer;
|
||||
import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
|
||||
import org.apache.flink.table.runtime.generated.NormalizedKeyComputer;
|
||||
import org.apache.flink.table.runtime.generated.RecordComparator;
|
||||
import org.apache.flink.table.runtime.operators.TableStreamOperator;
|
||||
import org.apache.flink.table.runtime.operators.sort.BinaryExternalSorter;
|
||||
import org.apache.flink.table.runtime.typeutils.AbstractRowDataSerializer;
|
||||
import org.apache.flink.table.runtime.typeutils.BinaryRowDataSerializer;
|
||||
import org.apache.flink.table.runtime.util.StreamRecordCollector;
|
||||
import org.apache.flink.util.MutableObjectIterator;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* Operator for batch sort.
|
||||
*
|
||||
* <p>Copied from org.apache.flink.table.runtime.operators.sort.SortOperator to change the annotation.
|
||||
*/
|
||||
public class SortOperator extends TableStreamOperator<RowData>
|
||||
implements OneInputStreamOperator<RowData, RowData>, BoundedOneInput {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(SortOperator.class);
|
||||
|
||||
private GeneratedNormalizedKeyComputer gComputer;
|
||||
private GeneratedRecordComparator gComparator;
|
||||
|
||||
private transient BinaryExternalSorter sorter;
|
||||
private transient StreamRecordCollector<RowData> collector;
|
||||
private transient BinaryRowDataSerializer binarySerializer;
|
||||
|
||||
public SortOperator(
|
||||
GeneratedNormalizedKeyComputer gComputer, GeneratedRecordComparator gComparator) {
|
||||
this.gComputer = gComputer;
|
||||
this.gComparator = gComparator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open() throws Exception {
|
||||
super.open();
|
||||
LOG.info("Opening SortOperator");
|
||||
|
||||
ClassLoader cl = getContainingTask().getUserCodeClassLoader();
|
||||
|
||||
AbstractRowDataSerializer inputSerializer =
|
||||
(AbstractRowDataSerializer)
|
||||
getOperatorConfig().getTypeSerializerIn1(getUserCodeClassloader());
|
||||
this.binarySerializer = new BinaryRowDataSerializer(inputSerializer.getArity());
|
||||
|
||||
NormalizedKeyComputer computer = gComputer.newInstance(cl);
|
||||
RecordComparator comparator = gComparator.newInstance(cl);
|
||||
gComputer = null;
|
||||
gComparator = null;
|
||||
|
||||
MemoryManager memManager = getContainingTask().getEnvironment().getMemoryManager();
|
||||
this.sorter =
|
||||
new BinaryExternalSorter(
|
||||
this.getContainingTask(),
|
||||
memManager,
|
||||
computeMemorySize(),
|
||||
this.getContainingTask().getEnvironment().getIOManager(),
|
||||
inputSerializer,
|
||||
binarySerializer,
|
||||
computer,
|
||||
comparator,
|
||||
getContainingTask().getJobConfiguration());
|
||||
this.sorter.startThreads();
|
||||
|
||||
collector = new StreamRecordCollector<>(output);
|
||||
|
||||
// register the the metrics.
|
||||
getMetricGroup().gauge("memoryUsedSizeInBytes", (Gauge<Long>) sorter::getUsedMemoryInBytes);
|
||||
getMetricGroup().gauge("numSpillFiles", (Gauge<Long>) sorter::getNumSpillFiles);
|
||||
getMetricGroup().gauge("spillInBytes", (Gauge<Long>) sorter::getSpillInBytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processElement(StreamRecord<RowData> element) throws Exception {
|
||||
this.sorter.write(element.getValue());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void endInput() throws Exception {
|
||||
BinaryRowData row = binarySerializer.createInstance();
|
||||
MutableObjectIterator<BinaryRowData> iterator = sorter.getIterator();
|
||||
while ((row = iterator.next(row)) != null) {
|
||||
collector.collect(row);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws Exception {
|
||||
LOG.info("Closing SortOperator");
|
||||
super.close();
|
||||
if (sorter != null) {
|
||||
sorter.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,56 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.bulk.sort;
|
||||
|
||||
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
|
||||
import org.apache.flink.table.api.TableConfig;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.planner.codegen.sort.SortCodeGenerator;
|
||||
import org.apache.flink.table.planner.plan.nodes.exec.spec.SortSpec;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
/**
|
||||
* Tools to generate the sort operator.
|
||||
*/
|
||||
public class SortOperatorGen {
|
||||
private final int[] sortIndices;
|
||||
private final RowType rowType;
|
||||
private final TableConfig tableConfig = new TableConfig();
|
||||
|
||||
public SortOperatorGen(RowType rowType, String[] sortFields) {
|
||||
this.sortIndices = Arrays.stream(sortFields).mapToInt(rowType::getFieldIndex).toArray();
|
||||
this.rowType = rowType;
|
||||
}
|
||||
|
||||
public OneInputStreamOperator<RowData, RowData> createSortOperator() {
|
||||
SortCodeGenerator codeGen = createSortCodeGenerator();
|
||||
return new SortOperator(
|
||||
codeGen.generateNormalizedKeyComputer("SortComputer"),
|
||||
codeGen.generateRecordComparator("SortComparator"));
|
||||
}
|
||||
|
||||
private SortCodeGenerator createSortCodeGenerator() {
|
||||
SortSpec.SortSpecBuilder builder = SortSpec.builder();
|
||||
IntStream.range(0, sortIndices.length).forEach(i -> builder.addField(i, true, true));
|
||||
return new SortCodeGenerator(tableConfig, rowType, builder.build());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,261 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.common;
|
||||
|
||||
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.sink.StreamWriteOperatorCoordinator;
|
||||
import org.apache.hudi.sink.event.CommitAckEvent;
|
||||
import org.apache.hudi.sink.event.WriteMetadataEvent;
|
||||
import org.apache.hudi.sink.meta.CkpMetadata;
|
||||
import org.apache.hudi.sink.utils.TimeWait;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.annotation.VisibleForTesting;
|
||||
import org.apache.flink.api.common.state.ListState;
|
||||
import org.apache.flink.api.common.state.ListStateDescriptor;
|
||||
import org.apache.flink.api.common.typeinfo.TypeInformation;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorEventGateway;
|
||||
import org.apache.flink.runtime.state.FunctionInitializationContext;
|
||||
import org.apache.flink.runtime.state.FunctionSnapshotContext;
|
||||
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Base infrastructures for streaming writer function.
|
||||
*
|
||||
* @param <I> Type of the input record
|
||||
* @see StreamWriteOperatorCoordinator
|
||||
*/
|
||||
public abstract class AbstractStreamWriteFunction<I>
|
||||
extends AbstractWriteFunction<I>
|
||||
implements CheckpointedFunction {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(AbstractStreamWriteFunction.class);
|
||||
|
||||
/**
|
||||
* Config options.
|
||||
*/
|
||||
protected final Configuration config;
|
||||
|
||||
/**
|
||||
* Id of current subtask.
|
||||
*/
|
||||
protected int taskID;
|
||||
|
||||
/**
|
||||
* Meta Client.
|
||||
*/
|
||||
protected transient HoodieTableMetaClient metaClient;
|
||||
|
||||
/**
|
||||
* Write Client.
|
||||
*/
|
||||
protected transient HoodieFlinkWriteClient writeClient;
|
||||
|
||||
/**
|
||||
* The REQUESTED instant we write the data.
|
||||
*/
|
||||
protected volatile String currentInstant;
|
||||
|
||||
/**
|
||||
* Gateway to send operator events to the operator coordinator.
|
||||
*/
|
||||
protected transient OperatorEventGateway eventGateway;
|
||||
|
||||
/**
|
||||
* Flag saying whether the write task is waiting for the checkpoint success notification
|
||||
* after it finished a checkpoint.
|
||||
*
|
||||
* <p>The flag is needed because the write task does not block during the waiting time interval,
|
||||
* some data buckets still flush out with old instant time. There are two cases that the flush may produce
|
||||
* corrupted files if the old instant is committed successfully:
|
||||
* 1) the write handle was writing data but interrupted, left a corrupted parquet file;
|
||||
* 2) the write handle finished the write but was not closed, left an empty parquet file.
|
||||
*
|
||||
* <p>To solve, when this flag was set to true, we block the data flushing thus the #processElement method,
|
||||
* the flag was reset to false if the task receives the checkpoint success event or the latest inflight instant
|
||||
* time changed(the last instant committed successfully).
|
||||
*/
|
||||
protected volatile boolean confirming = false;
|
||||
|
||||
/**
|
||||
* List state of the write metadata events.
|
||||
*/
|
||||
private transient ListState<WriteMetadataEvent> writeMetadataState;
|
||||
|
||||
/**
|
||||
* Write status list for the current checkpoint.
|
||||
*/
|
||||
protected List<WriteStatus> writeStatuses;
|
||||
|
||||
/**
|
||||
* The checkpoint metadata.
|
||||
*/
|
||||
private transient CkpMetadata ckpMetadata;
|
||||
|
||||
/**
|
||||
* Constructs a StreamWriteFunctionBase.
|
||||
*
|
||||
* @param config The config options
|
||||
*/
|
||||
public AbstractStreamWriteFunction(Configuration config) {
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initializeState(FunctionInitializationContext context) throws Exception {
|
||||
this.taskID = getRuntimeContext().getIndexOfThisSubtask();
|
||||
this.metaClient = StreamerUtil.createMetaClient(this.config);
|
||||
this.writeClient = StreamerUtil.createWriteClient(this.config, getRuntimeContext());
|
||||
this.writeStatuses = new ArrayList<>();
|
||||
this.writeMetadataState = context.getOperatorStateStore().getListState(
|
||||
new ListStateDescriptor<>(
|
||||
"write-metadata-state",
|
||||
TypeInformation.of(WriteMetadataEvent.class)
|
||||
));
|
||||
|
||||
this.ckpMetadata = CkpMetadata.getInstance(this.metaClient.getFs(), this.metaClient.getBasePath());
|
||||
this.currentInstant = lastPendingInstant();
|
||||
if (context.isRestored()) {
|
||||
restoreWriteMetadata();
|
||||
} else {
|
||||
sendBootstrapEvent();
|
||||
}
|
||||
// blocks flushing until the coordinator starts a new instant
|
||||
this.confirming = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void snapshotState(FunctionSnapshotContext functionSnapshotContext) throws Exception {
|
||||
snapshotState();
|
||||
// Reload the snapshot state as the current state.
|
||||
reloadWriteMetaState();
|
||||
}
|
||||
|
||||
public abstract void snapshotState();
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Getter/Setter
|
||||
// -------------------------------------------------------------------------
|
||||
@VisibleForTesting
|
||||
public boolean isConfirming() {
|
||||
return this.confirming;
|
||||
}
|
||||
|
||||
public void setOperatorEventGateway(OperatorEventGateway operatorEventGateway) {
|
||||
this.eventGateway = operatorEventGateway;
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Utilities
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
private void restoreWriteMetadata() throws Exception {
|
||||
String lastInflight = lastPendingInstant();
|
||||
boolean eventSent = false;
|
||||
for (WriteMetadataEvent event : this.writeMetadataState.get()) {
|
||||
if (Objects.equals(lastInflight, event.getInstantTime())) {
|
||||
// The checkpoint succeed but the meta does not commit,
|
||||
// re-commit the inflight instant
|
||||
this.eventGateway.sendEventToCoordinator(event);
|
||||
LOG.info("Send uncommitted write metadata event to coordinator, task[{}].", taskID);
|
||||
eventSent = true;
|
||||
}
|
||||
}
|
||||
if (!eventSent) {
|
||||
sendBootstrapEvent();
|
||||
}
|
||||
}
|
||||
|
||||
private void sendBootstrapEvent() {
|
||||
this.eventGateway.sendEventToCoordinator(WriteMetadataEvent.emptyBootstrap(taskID));
|
||||
LOG.info("Send bootstrap write metadata event to coordinator, task[{}].", taskID);
|
||||
}
|
||||
|
||||
/**
|
||||
* Reload the write metadata state as the current checkpoint.
|
||||
*/
|
||||
private void reloadWriteMetaState() throws Exception {
|
||||
this.writeMetadataState.clear();
|
||||
WriteMetadataEvent event = WriteMetadataEvent.builder()
|
||||
.taskID(taskID)
|
||||
.instantTime(currentInstant)
|
||||
.writeStatus(new ArrayList<>(writeStatuses))
|
||||
.bootstrap(true)
|
||||
.build();
|
||||
this.writeMetadataState.add(event);
|
||||
writeStatuses.clear();
|
||||
}
|
||||
|
||||
public void handleOperatorEvent(OperatorEvent event) {
|
||||
ValidationUtils.checkArgument(event instanceof CommitAckEvent,
|
||||
"The write function can only handle CommitAckEvent");
|
||||
this.confirming = false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the last pending instant time.
|
||||
*/
|
||||
protected String lastPendingInstant() {
|
||||
return this.ckpMetadata.lastPendingInstant();
|
||||
}
|
||||
|
||||
/**
|
||||
* Prepares the instant time to write with for next checkpoint.
|
||||
*
|
||||
* @param hasData Whether the task has buffering data
|
||||
* @return The instant time
|
||||
*/
|
||||
protected String instantToWrite(boolean hasData) {
|
||||
String instant = lastPendingInstant();
|
||||
// if exactly-once semantics turns on,
|
||||
// waits for the checkpoint notification until the checkpoint timeout threshold hits.
|
||||
TimeWait timeWait = TimeWait.builder()
|
||||
.timeout(config.getLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT))
|
||||
.action("instant initialize")
|
||||
.build();
|
||||
while (confirming) {
|
||||
// wait condition:
|
||||
// 1. there is no inflight instant
|
||||
// 2. the inflight instant does not change and the checkpoint has buffering data
|
||||
if (instant == null || (instant.equals(this.currentInstant) && hasData && !this.ckpMetadata.isAborted(instant))) {
|
||||
// sleep for a while
|
||||
timeWait.waitFor();
|
||||
// refresh the inflight instant
|
||||
instant = lastPendingInstant();
|
||||
} else {
|
||||
// the pending instant changed, that means the last instant was committed
|
||||
// successfully.
|
||||
confirming = false;
|
||||
}
|
||||
}
|
||||
return instant;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,48 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.common;
|
||||
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorEventGateway;
|
||||
import org.apache.flink.streaming.api.functions.ProcessFunction;
|
||||
import org.apache.flink.streaming.api.operators.BoundedOneInput;
|
||||
|
||||
/**
|
||||
* Base class for write function.
|
||||
*
|
||||
* @param <I> the input type
|
||||
*/
|
||||
public abstract class AbstractWriteFunction<I> extends ProcessFunction<I, Object> implements BoundedOneInput {
|
||||
/**
|
||||
* Sets up the event gateway.
|
||||
*/
|
||||
public abstract void setOperatorEventGateway(OperatorEventGateway operatorEventGateway);
|
||||
|
||||
/**
|
||||
* Invoked when bounded source ends up.
|
||||
*/
|
||||
public abstract void endInput();
|
||||
|
||||
/**
|
||||
* Handles the operator event sent by the coordinator.
|
||||
*
|
||||
* @param event The event
|
||||
*/
|
||||
public abstract void handleOperatorEvent(OperatorEvent event);
|
||||
}
|
||||
@@ -0,0 +1,55 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.common;
|
||||
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorEventGateway;
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
|
||||
import org.apache.flink.streaming.api.operators.BoundedOneInput;
|
||||
import org.apache.flink.streaming.api.operators.ProcessOperator;
|
||||
|
||||
/**
|
||||
* Base class for write operator.
|
||||
*
|
||||
* @param <I> the input type
|
||||
*/
|
||||
public abstract class AbstractWriteOperator<I>
|
||||
extends ProcessOperator<I, Object>
|
||||
implements OperatorEventHandler, BoundedOneInput {
|
||||
private final AbstractWriteFunction<I> function;
|
||||
|
||||
public AbstractWriteOperator(AbstractWriteFunction<I> function) {
|
||||
super(function);
|
||||
this.function = function;
|
||||
}
|
||||
|
||||
public void setOperatorEventGateway(OperatorEventGateway operatorEventGateway) {
|
||||
this.function.setOperatorEventGateway(operatorEventGateway);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void endInput() {
|
||||
this.function.endInput();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handleOperatorEvent(OperatorEvent evt) {
|
||||
this.function.handleOperatorEvent(evt);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,72 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.common;
|
||||
|
||||
import org.apache.hudi.sink.StreamWriteOperator;
|
||||
import org.apache.hudi.sink.StreamWriteOperatorCoordinator;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.runtime.jobgraph.OperatorID;
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorEventDispatcher;
|
||||
import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
|
||||
import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
|
||||
import org.apache.flink.streaming.api.operators.SimpleUdfStreamOperatorFactory;
|
||||
import org.apache.flink.streaming.api.operators.StreamOperator;
|
||||
import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
|
||||
|
||||
/**
|
||||
* Factory class for {@link StreamWriteOperator}.
|
||||
*/
|
||||
public class WriteOperatorFactory<I>
|
||||
extends SimpleUdfStreamOperatorFactory<Object>
|
||||
implements CoordinatedOperatorFactory<Object>, OneInputStreamOperatorFactory<I, Object> {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
private final AbstractWriteOperator<I> operator;
|
||||
private final Configuration conf;
|
||||
|
||||
public WriteOperatorFactory(Configuration conf, AbstractWriteOperator<I> operator) {
|
||||
super(operator);
|
||||
this.operator = operator;
|
||||
this.conf = conf;
|
||||
}
|
||||
|
||||
public static <I> WriteOperatorFactory<I> instance(Configuration conf, AbstractWriteOperator<I> operator) {
|
||||
return new WriteOperatorFactory<>(conf, operator);
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public <T extends StreamOperator<Object>> T createStreamOperator(StreamOperatorParameters<Object> parameters) {
|
||||
final OperatorID operatorID = parameters.getStreamConfig().getOperatorID();
|
||||
final OperatorEventDispatcher eventDispatcher = parameters.getOperatorEventDispatcher();
|
||||
|
||||
this.operator.setOperatorEventGateway(eventDispatcher.getOperatorEventGateway(operatorID));
|
||||
this.operator.setup(parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput());
|
||||
this.operator.setProcessingTimeService(this.processingTimeService);
|
||||
eventDispatcher.registerEventHandler(operatorID, operator);
|
||||
return (T) operator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public OperatorCoordinator.Provider getCoordinatorProvider(String s, OperatorID operatorID) {
|
||||
return new StreamWriteOperatorCoordinator.Provider(operatorID, this.conf);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,132 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.compact;
|
||||
|
||||
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.sink.utils.NonThrownExecutor;
|
||||
import org.apache.hudi.table.HoodieFlinkCopyOnWriteTable;
|
||||
import org.apache.hudi.table.action.compact.HoodieFlinkMergeOnReadTableCompactor;
|
||||
import org.apache.hudi.util.CompactionUtil;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.annotation.VisibleForTesting;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.streaming.api.functions.ProcessFunction;
|
||||
import org.apache.flink.util.Collector;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Function to execute the actual compaction task assigned by the compaction plan task.
|
||||
* In order to execute scalable, the input should shuffle by the compact event {@link CompactionPlanEvent}.
|
||||
*/
|
||||
public class CompactFunction extends ProcessFunction<CompactionPlanEvent, CompactionCommitEvent> {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(CompactFunction.class);
|
||||
|
||||
/**
|
||||
* Config options.
|
||||
*/
|
||||
private final Configuration conf;
|
||||
|
||||
/**
|
||||
* Write Client.
|
||||
*/
|
||||
private transient HoodieFlinkWriteClient<?> writeClient;
|
||||
|
||||
/**
|
||||
* Whether to execute compaction asynchronously.
|
||||
*/
|
||||
private final boolean asyncCompaction;
|
||||
|
||||
/**
|
||||
* Id of current subtask.
|
||||
*/
|
||||
private int taskID;
|
||||
|
||||
/**
|
||||
* Executor service to execute the compaction task.
|
||||
*/
|
||||
private transient NonThrownExecutor executor;
|
||||
|
||||
public CompactFunction(Configuration conf) {
|
||||
this.conf = conf;
|
||||
this.asyncCompaction = StreamerUtil.needsAsyncCompaction(conf);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(Configuration parameters) throws Exception {
|
||||
this.taskID = getRuntimeContext().getIndexOfThisSubtask();
|
||||
this.writeClient = StreamerUtil.createWriteClient(conf, getRuntimeContext());
|
||||
if (this.asyncCompaction) {
|
||||
this.executor = NonThrownExecutor.builder(LOG).build();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processElement(CompactionPlanEvent event, Context context, Collector<CompactionCommitEvent> collector) throws Exception {
|
||||
final String instantTime = event.getCompactionInstantTime();
|
||||
final CompactionOperation compactionOperation = event.getOperation();
|
||||
if (asyncCompaction) {
|
||||
// executes the compaction task asynchronously to not block the checkpoint barrier propagate.
|
||||
executor.execute(
|
||||
() -> doCompaction(instantTime, compactionOperation, collector, reloadWriteConfig()),
|
||||
(errMsg, t) -> collector.collect(new CompactionCommitEvent(instantTime, compactionOperation.getFileId(), taskID)),
|
||||
"Execute compaction for instant %s from task %d", instantTime, taskID);
|
||||
} else {
|
||||
// executes the compaction task synchronously for batch mode.
|
||||
LOG.info("Execute compaction for instant {} from task {}", instantTime, taskID);
|
||||
doCompaction(instantTime, compactionOperation, collector, writeClient.getConfig());
|
||||
}
|
||||
}
|
||||
|
||||
private void doCompaction(String instantTime,
|
||||
CompactionOperation compactionOperation,
|
||||
Collector<CompactionCommitEvent> collector,
|
||||
HoodieWriteConfig writeConfig) throws IOException {
|
||||
HoodieFlinkMergeOnReadTableCompactor<?> compactor = new HoodieFlinkMergeOnReadTableCompactor<>();
|
||||
List<WriteStatus> writeStatuses = compactor.compact(
|
||||
new HoodieFlinkCopyOnWriteTable<>(
|
||||
writeConfig,
|
||||
writeClient.getEngineContext(),
|
||||
writeClient.getHoodieTable().getMetaClient()),
|
||||
writeClient.getHoodieTable().getMetaClient(),
|
||||
writeClient.getConfig(),
|
||||
compactionOperation,
|
||||
instantTime,
|
||||
writeClient.getHoodieTable().getTaskContextSupplier());
|
||||
collector.collect(new CompactionCommitEvent(instantTime, compactionOperation.getFileId(), writeStatuses, taskID));
|
||||
}
|
||||
|
||||
private HoodieWriteConfig reloadWriteConfig() throws Exception {
|
||||
HoodieWriteConfig writeConfig = writeClient.getConfig();
|
||||
CompactionUtil.setAvroSchema(writeConfig, writeClient.getHoodieTable().getMetaClient());
|
||||
return writeConfig;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public void setExecutor(NonThrownExecutor executor) {
|
||||
this.executor = executor;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,107 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.compact;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Represents a commit event from the compaction task {@link CompactFunction}.
|
||||
*/
|
||||
public class CompactionCommitEvent implements Serializable {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
/**
|
||||
* The compaction commit instant time.
|
||||
*/
|
||||
private String instant;
|
||||
|
||||
/**
|
||||
* The file ID.
|
||||
*/
|
||||
private String fileId;
|
||||
|
||||
/**
|
||||
* The write statuses.
|
||||
*/
|
||||
private List<WriteStatus> writeStatuses;
|
||||
/**
|
||||
* The compaction task identifier.
|
||||
*/
|
||||
private int taskID;
|
||||
|
||||
public CompactionCommitEvent() {
|
||||
}
|
||||
|
||||
/**
|
||||
* An event with NULL write statuses that represents a failed compaction.
|
||||
*/
|
||||
public CompactionCommitEvent(String instant, String fileId, int taskID) {
|
||||
this(instant, fileId, null, taskID);
|
||||
}
|
||||
|
||||
public CompactionCommitEvent(String instant, String fileId, List<WriteStatus> writeStatuses, int taskID) {
|
||||
this.instant = instant;
|
||||
this.fileId = fileId;
|
||||
this.writeStatuses = writeStatuses;
|
||||
this.taskID = taskID;
|
||||
}
|
||||
|
||||
public boolean isFailed() {
|
||||
return this.writeStatuses == null;
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Getter/Setter
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
public void setInstant(String instant) {
|
||||
this.instant = instant;
|
||||
}
|
||||
|
||||
public void setFileId(String fileId) {
|
||||
this.fileId = fileId;
|
||||
}
|
||||
|
||||
public void setWriteStatuses(List<WriteStatus> writeStatuses) {
|
||||
this.writeStatuses = writeStatuses;
|
||||
}
|
||||
|
||||
public void setTaskID(int taskID) {
|
||||
this.taskID = taskID;
|
||||
}
|
||||
|
||||
public String getInstant() {
|
||||
return instant;
|
||||
}
|
||||
|
||||
public String getFileId() {
|
||||
return fileId;
|
||||
}
|
||||
|
||||
public List<WriteStatus> getWriteStatuses() {
|
||||
return writeStatuses;
|
||||
}
|
||||
|
||||
public int getTaskID() {
|
||||
return taskID;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,169 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.compact;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.data.HoodieList;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.sink.CleanFunction;
|
||||
import org.apache.hudi.table.HoodieFlinkTable;
|
||||
import org.apache.hudi.table.action.compact.CompactHelpers;
|
||||
import org.apache.hudi.util.CompactionUtil;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Function to check and commit the compaction action.
|
||||
*
|
||||
* <p> Each time after receiving a compaction commit event {@link CompactionCommitEvent},
|
||||
* it loads and checks the compaction plan {@link HoodieCompactionPlan},
|
||||
* if all the compaction operations {@link org.apache.hudi.common.model.CompactionOperation}
|
||||
* of the plan are finished, tries to commit the compaction action.
|
||||
*
|
||||
* <p>It also inherits the {@link CleanFunction} cleaning ability. This is needed because
|
||||
* the SQL API does not allow multiple sinks in one table sink provider.
|
||||
*/
|
||||
public class CompactionCommitSink extends CleanFunction<CompactionCommitEvent> {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(CompactionCommitSink.class);
|
||||
|
||||
/**
|
||||
* Config options.
|
||||
*/
|
||||
private final Configuration conf;
|
||||
|
||||
/**
|
||||
* Buffer to collect the event from each compact task {@code CompactFunction}.
|
||||
*
|
||||
* <p>Stores the mapping of instant_time -> file_id -> event. Use a map to collect the
|
||||
* events because the rolling back of intermediate compaction tasks generates corrupt
|
||||
* events.
|
||||
*/
|
||||
private transient Map<String, Map<String, CompactionCommitEvent>> commitBuffer;
|
||||
|
||||
/**
|
||||
* Cache to store compaction plan for each instant.
|
||||
* Stores the mapping of instant_time -> compactionPlan.
|
||||
*/
|
||||
private transient Map<String, HoodieCompactionPlan> compactionPlanCache;
|
||||
|
||||
/**
|
||||
* The hoodie table.
|
||||
*/
|
||||
private transient HoodieFlinkTable<?> table;
|
||||
|
||||
public CompactionCommitSink(Configuration conf) {
|
||||
super(conf);
|
||||
this.conf = conf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(Configuration parameters) throws Exception {
|
||||
super.open(parameters);
|
||||
if (writeClient == null) {
|
||||
this.writeClient = StreamerUtil.createWriteClient(conf, getRuntimeContext());
|
||||
}
|
||||
this.commitBuffer = new HashMap<>();
|
||||
this.compactionPlanCache = new HashMap<>();
|
||||
this.table = this.writeClient.getHoodieTable();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void invoke(CompactionCommitEvent event, Context context) throws Exception {
|
||||
final String instant = event.getInstant();
|
||||
if (event.isFailed()) {
|
||||
// handle failure case
|
||||
CompactionUtil.rollbackCompaction(table, event.getInstant());
|
||||
return;
|
||||
}
|
||||
commitBuffer.computeIfAbsent(instant, k -> new HashMap<>())
|
||||
.put(event.getFileId(), event);
|
||||
commitIfNecessary(instant, commitBuffer.get(instant).values());
|
||||
}
|
||||
|
||||
/**
|
||||
* Condition to commit: the commit buffer has equal size with the compaction plan operations
|
||||
* and all the compact commit event {@link CompactionCommitEvent} has the same compaction instant time.
|
||||
*
|
||||
* @param instant Compaction commit instant time
|
||||
* @param events Commit events ever received for the instant
|
||||
*/
|
||||
private void commitIfNecessary(String instant, Collection<CompactionCommitEvent> events) throws IOException {
|
||||
HoodieCompactionPlan compactionPlan = compactionPlanCache.computeIfAbsent(instant, k -> {
|
||||
try {
|
||||
return CompactionUtils.getCompactionPlan(
|
||||
this.writeClient.getHoodieTable().getMetaClient(), instant);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
});
|
||||
|
||||
boolean isReady = compactionPlan.getOperations().size() == events.size();
|
||||
if (!isReady) {
|
||||
return;
|
||||
}
|
||||
try {
|
||||
doCommit(instant, events);
|
||||
} catch (Throwable throwable) {
|
||||
// make it fail-safe
|
||||
LOG.error("Error while committing compaction instant: " + instant, throwable);
|
||||
} finally {
|
||||
// reset the status
|
||||
reset(instant);
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private void doCommit(String instant, Collection<CompactionCommitEvent> events) throws IOException {
|
||||
List<WriteStatus> statuses = events.stream()
|
||||
.map(CompactionCommitEvent::getWriteStatuses)
|
||||
.flatMap(Collection::stream)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
HoodieCommitMetadata metadata = CompactHelpers.getInstance().createCompactionMetadata(
|
||||
table, instant, HoodieList.of(statuses), writeClient.getConfig().getSchema());
|
||||
|
||||
// commit the compaction
|
||||
this.writeClient.commitCompaction(instant, metadata, Option.empty());
|
||||
|
||||
// Whether to clean up the old log file when compaction
|
||||
if (!conf.getBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED)) {
|
||||
this.writeClient.clean();
|
||||
}
|
||||
}
|
||||
|
||||
private void reset(String instant) {
|
||||
this.commitBuffer.remove(instant);
|
||||
this.compactionPlanCache.remove(instant);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,58 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.compact;
|
||||
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* Represents a compact command from the compaction plan task {@link CompactionPlanOperator}.
|
||||
*/
|
||||
public class CompactionPlanEvent implements Serializable {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
private String compactionInstantTime;
|
||||
|
||||
private CompactionOperation operation;
|
||||
|
||||
public CompactionPlanEvent() {
|
||||
}
|
||||
|
||||
public CompactionPlanEvent(String instantTime, CompactionOperation operation) {
|
||||
this.compactionInstantTime = instantTime;
|
||||
this.operation = operation;
|
||||
}
|
||||
|
||||
public void setCompactionInstantTime(String compactionInstantTime) {
|
||||
this.compactionInstantTime = compactionInstantTime;
|
||||
}
|
||||
|
||||
public void setOperation(CompactionOperation operation) {
|
||||
this.operation = operation;
|
||||
}
|
||||
|
||||
public String getCompactionInstantTime() {
|
||||
return compactionInstantTime;
|
||||
}
|
||||
|
||||
public CompactionOperation getOperation() {
|
||||
return operation;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,140 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.compact;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.table.HoodieFlinkTable;
|
||||
import org.apache.hudi.util.CompactionUtil;
|
||||
import org.apache.hudi.util.FlinkTables;
|
||||
|
||||
import org.apache.flink.annotation.VisibleForTesting;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
|
||||
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
|
||||
import org.apache.flink.streaming.api.operators.Output;
|
||||
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
||||
/**
|
||||
* Operator that generates the compaction plan with pluggable strategies on finished checkpoints.
|
||||
*
|
||||
* <p>It should be singleton to avoid conflicts.
|
||||
*/
|
||||
public class CompactionPlanOperator extends AbstractStreamOperator<CompactionPlanEvent>
|
||||
implements OneInputStreamOperator<Object, CompactionPlanEvent> {
|
||||
|
||||
/**
|
||||
* Config options.
|
||||
*/
|
||||
private final Configuration conf;
|
||||
|
||||
/**
|
||||
* Meta Client.
|
||||
*/
|
||||
@SuppressWarnings("rawtypes")
|
||||
private transient HoodieFlinkTable table;
|
||||
|
||||
public CompactionPlanOperator(Configuration conf) {
|
||||
this.conf = conf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open() throws Exception {
|
||||
super.open();
|
||||
this.table = FlinkTables.createTable(conf, getRuntimeContext());
|
||||
// when starting up, rolls back all the inflight compaction instants if there exists,
|
||||
// these instants are in priority for scheduling task because the compaction instants are
|
||||
// scheduled from earliest(FIFO sequence).
|
||||
CompactionUtil.rollbackCompaction(table);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processElement(StreamRecord<Object> streamRecord) {
|
||||
// no operation
|
||||
}
|
||||
|
||||
@Override
|
||||
public void notifyCheckpointComplete(long checkpointId) {
|
||||
try {
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
// There is no good way to infer when the compaction task for an instant crushed
|
||||
// or is still undergoing. So we use a configured timeout threshold to control the rollback:
|
||||
// {@code FlinkOptions.COMPACTION_TIMEOUT_SECONDS},
|
||||
// when the earliest inflight instant has timed out, assumes it has failed
|
||||
// already and just rolls it back.
|
||||
|
||||
// comment out: do we really need the timeout rollback ?
|
||||
// CompactionUtil.rollbackEarliestCompaction(table, conf);
|
||||
scheduleCompaction(table, checkpointId);
|
||||
} catch (Throwable throwable) {
|
||||
// make it fail-safe
|
||||
LOG.error("Error while scheduling compaction plan for checkpoint: " + checkpointId, throwable);
|
||||
}
|
||||
}
|
||||
|
||||
private void scheduleCompaction(HoodieFlinkTable<?> table, long checkpointId) throws IOException {
|
||||
// the first instant takes the highest priority.
|
||||
Option<HoodieInstant> firstRequested = table.getActiveTimeline().filterPendingCompactionTimeline()
|
||||
.filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED).firstInstant();
|
||||
if (!firstRequested.isPresent()) {
|
||||
// do nothing.
|
||||
LOG.info("No compaction plan for checkpoint " + checkpointId);
|
||||
return;
|
||||
}
|
||||
|
||||
String compactionInstantTime = firstRequested.get().getTimestamp();
|
||||
|
||||
// generate compaction plan
|
||||
// should support configurable commit metadata
|
||||
HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan(
|
||||
table.getMetaClient(), compactionInstantTime);
|
||||
|
||||
if (compactionPlan == null || (compactionPlan.getOperations() == null)
|
||||
|| (compactionPlan.getOperations().isEmpty())) {
|
||||
// do nothing.
|
||||
LOG.info("Empty compaction plan for instant " + compactionInstantTime);
|
||||
} else {
|
||||
HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime);
|
||||
// Mark instant as compaction inflight
|
||||
table.getActiveTimeline().transitionCompactionRequestedToInflight(instant);
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
|
||||
List<CompactionOperation> operations = compactionPlan.getOperations().stream()
|
||||
.map(CompactionOperation::convertFromAvroRecordInstance).collect(toList());
|
||||
LOG.info("Execute compaction plan for instant {} as {} file groups", compactionInstantTime, operations.size());
|
||||
for (CompactionOperation operation : operations) {
|
||||
output.collect(new StreamRecord<>(new CompactionPlanEvent(compactionInstantTime, operation)));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public void setOutput(Output<StreamRecord<CompactionPlanEvent>> output) {
|
||||
this.output = output;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,94 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.compact;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
|
||||
import org.apache.flink.api.common.functions.AbstractRichFunction;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.streaming.api.functions.source.SourceFunction;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
||||
/**
|
||||
* Flink hudi compaction source function.
|
||||
*
|
||||
* <P>This function read the compaction plan as {@link CompactionOperation}s then assign the compaction task
|
||||
* event {@link CompactionPlanEvent} to downstream operators.
|
||||
*
|
||||
* <p>The compaction instant time is specified explicitly with strategies:
|
||||
*
|
||||
* <ul>
|
||||
* <li>If the timeline has no inflight instants,
|
||||
* use {@link org.apache.hudi.common.table.timeline.HoodieActiveTimeline#createNewInstantTime()}
|
||||
* as the instant time;</li>
|
||||
* <li>If the timeline has inflight instants,
|
||||
* use the median instant time between [last complete instant time, earliest inflight instant time]
|
||||
* as the instant time.</li>
|
||||
* </ul>
|
||||
*/
|
||||
public class CompactionPlanSourceFunction extends AbstractRichFunction implements SourceFunction<CompactionPlanEvent> {
|
||||
|
||||
protected static final Logger LOG = LoggerFactory.getLogger(CompactionPlanSourceFunction.class);
|
||||
|
||||
/**
|
||||
* Compaction instant time.
|
||||
*/
|
||||
private final String compactionInstantTime;
|
||||
|
||||
/**
|
||||
* The compaction plan.
|
||||
*/
|
||||
private final HoodieCompactionPlan compactionPlan;
|
||||
|
||||
public CompactionPlanSourceFunction(HoodieCompactionPlan compactionPlan, String compactionInstantTime) {
|
||||
this.compactionPlan = compactionPlan;
|
||||
this.compactionInstantTime = compactionInstantTime;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(Configuration parameters) throws Exception {
|
||||
// no operation
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run(SourceContext sourceContext) throws Exception {
|
||||
List<CompactionOperation> operations = this.compactionPlan.getOperations().stream()
|
||||
.map(CompactionOperation::convertFromAvroRecordInstance).collect(toList());
|
||||
LOG.info("CompactionPlanFunction compacting " + operations + " files");
|
||||
for (CompactionOperation operation : operations) {
|
||||
sourceContext.collect(new CompactionPlanEvent(compactionInstantTime, operation));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws Exception {
|
||||
// no operation
|
||||
}
|
||||
|
||||
@Override
|
||||
public void cancel() {
|
||||
// no operation
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,138 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.compact;
|
||||
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
|
||||
import com.beust.jcommander.Parameter;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
|
||||
/**
|
||||
* Configurations for Hoodie Flink compaction.
|
||||
*/
|
||||
public class FlinkCompactionConfig extends Configuration {
|
||||
|
||||
@Parameter(names = {"--help", "-h"}, help = true)
|
||||
public Boolean help = false;
|
||||
|
||||
// ------------------------------------------------------------------------
|
||||
// Hudi Write Options
|
||||
// ------------------------------------------------------------------------
|
||||
|
||||
@Parameter(names = {"--path"}, description = "Base path for the target hoodie table.", required = true)
|
||||
public String path;
|
||||
|
||||
// ------------------------------------------------------------------------
|
||||
// Compaction Options
|
||||
// ------------------------------------------------------------------------
|
||||
|
||||
public static final String NUM_COMMITS = "num_commits";
|
||||
public static final String TIME_ELAPSED = "time_elapsed";
|
||||
public static final String NUM_AND_TIME = "num_and_time";
|
||||
public static final String NUM_OR_TIME = "num_or_time";
|
||||
@Parameter(names = {"--compaction-trigger-strategy"},
|
||||
description = "Strategy to trigger compaction, options are 'num_commits': trigger compaction when reach N delta commits;\n"
|
||||
+ "'time_elapsed': trigger compaction when time elapsed > N seconds since last compaction;\n"
|
||||
+ "'num_and_time': trigger compaction when both NUM_COMMITS and TIME_ELAPSED are satisfied;\n"
|
||||
+ "'num_or_time': trigger compaction when NUM_COMMITS or TIME_ELAPSED is satisfied.\n"
|
||||
+ "Default is 'num_commits'",
|
||||
required = false)
|
||||
public String compactionTriggerStrategy = NUM_COMMITS;
|
||||
|
||||
@Parameter(names = {"--compaction-delta-commits"}, description = "Max delta commits needed to trigger compaction, default 5 commits", required = false)
|
||||
public Integer compactionDeltaCommits = 1;
|
||||
|
||||
@Parameter(names = {"--compaction-delta-seconds"}, description = "Max delta seconds time needed to trigger compaction, default 1 hour", required = false)
|
||||
public Integer compactionDeltaSeconds = 3600;
|
||||
|
||||
@Parameter(names = {"--clean-async-enabled"}, description = "Whether to cleanup the old commits immediately on new commits, enabled by default", required = false)
|
||||
public Boolean cleanAsyncEnable = false;
|
||||
|
||||
@Parameter(names = {"--clean-retain-commits"},
|
||||
description = "Number of commits to retain. So data will be retained for num_of_commits * time_between_commits (scheduled).\n"
|
||||
+ "This also directly translates into how much you can incrementally pull on this table, default 10",
|
||||
required = false)
|
||||
public Integer cleanRetainCommits = 10;
|
||||
|
||||
@Parameter(names = {"--archive-min-commits"},
|
||||
description = "Min number of commits to keep before archiving older commits into a sequential log, default 20.",
|
||||
required = false)
|
||||
public Integer archiveMinCommits = 20;
|
||||
|
||||
@Parameter(names = {"--archive-max-commits"},
|
||||
description = "Max number of commits to keep before archiving older commits into a sequential log, default 30.",
|
||||
required = false)
|
||||
public Integer archiveMaxCommits = 30;
|
||||
|
||||
@Parameter(names = {"--compaction-max-memory"}, description = "Max memory in MB for compaction spillable map, default 100MB.", required = false)
|
||||
public Integer compactionMaxMemory = 100;
|
||||
|
||||
@Parameter(names = {"--compaction-target-io"}, description = "Target IO per compaction (both read and write) for batching compaction, default 512000M.", required = false)
|
||||
public Long compactionTargetIo = 512000L;
|
||||
|
||||
@Parameter(names = {"--compaction-tasks"}, description = "Parallelism of tasks that do actual compaction, default is -1", required = false)
|
||||
public Integer compactionTasks = -1;
|
||||
|
||||
@Parameter(names = {"--schedule", "-sc"}, description = "Not recommended. Schedule the compaction plan in this job.\n"
|
||||
+ "There is a risk of losing data when scheduling compaction outside the writer job.\n"
|
||||
+ "Scheduling compaction in the writer job and only let this job do the compaction execution is recommended.\n"
|
||||
+ "Default is false", required = false)
|
||||
public Boolean schedule = false;
|
||||
|
||||
public static final String SEQ_FIFO = "FIFO";
|
||||
public static final String SEQ_LIFO = "LIFO";
|
||||
@Parameter(names = {"--seq"}, description = "Compaction plan execution sequence, two options are supported:\n"
|
||||
+ "1). FIFO: execute the oldest plan first;\n"
|
||||
+ "2). LIFO: execute the latest plan first, by default LIFO", required = false)
|
||||
public String compactionSeq = SEQ_LIFO;
|
||||
|
||||
@Parameter(names = {"--service"}, description = "Flink Compaction runs in service mode, disable by default")
|
||||
public Boolean serviceMode = false;
|
||||
|
||||
@Parameter(names = {"--min-compaction-interval-seconds"},
|
||||
description = "Min compaction interval of async compaction service, default 10 minutes")
|
||||
public Integer minCompactionIntervalSeconds = 600;
|
||||
|
||||
/**
|
||||
* Transforms a {@code HoodieFlinkCompaction.config} into {@code Configuration}.
|
||||
* The latter is more suitable for the table APIs. It reads all the properties
|
||||
* in the properties file (set by `--props` option) and cmd line options
|
||||
* (set by `--hoodie-conf` option).
|
||||
*/
|
||||
public static org.apache.flink.configuration.Configuration toFlinkConfig(FlinkCompactionConfig config) {
|
||||
org.apache.flink.configuration.Configuration conf = new Configuration();
|
||||
|
||||
conf.setString(FlinkOptions.PATH, config.path);
|
||||
conf.setString(FlinkOptions.COMPACTION_TRIGGER_STRATEGY, config.compactionTriggerStrategy);
|
||||
conf.setInteger(FlinkOptions.ARCHIVE_MAX_COMMITS, config.archiveMaxCommits);
|
||||
conf.setInteger(FlinkOptions.ARCHIVE_MIN_COMMITS, config.archiveMinCommits);
|
||||
conf.setInteger(FlinkOptions.CLEAN_RETAIN_COMMITS, config.cleanRetainCommits);
|
||||
conf.setInteger(FlinkOptions.COMPACTION_DELTA_COMMITS, config.compactionDeltaCommits);
|
||||
conf.setInteger(FlinkOptions.COMPACTION_DELTA_SECONDS, config.compactionDeltaSeconds);
|
||||
conf.setInteger(FlinkOptions.COMPACTION_MAX_MEMORY, config.compactionMaxMemory);
|
||||
conf.setLong(FlinkOptions.COMPACTION_TARGET_IO, config.compactionTargetIo);
|
||||
conf.setInteger(FlinkOptions.COMPACTION_TASKS, config.compactionTasks);
|
||||
conf.setBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED, config.cleanAsyncEnable);
|
||||
// use synchronous compaction always
|
||||
conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, false);
|
||||
conf.setBoolean(FlinkOptions.COMPACTION_SCHEDULE_ENABLED, config.schedule);
|
||||
|
||||
return conf;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,305 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.compact;
|
||||
|
||||
import org.apache.hudi.async.HoodieAsyncTableService;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.table.HoodieFlinkTable;
|
||||
import org.apache.hudi.util.CompactionUtil;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import com.beust.jcommander.JCommander;
|
||||
import org.apache.flink.annotation.VisibleForTesting;
|
||||
import org.apache.flink.api.common.typeinfo.TypeInformation;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
|
||||
import org.apache.flink.streaming.api.operators.ProcessOperator;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
|
||||
/**
|
||||
* Flink hudi compaction program that can be executed manually.
|
||||
*/
|
||||
public class HoodieFlinkCompactor {
|
||||
|
||||
protected static final Logger LOG = LoggerFactory.getLogger(HoodieFlinkCompactor.class);
|
||||
|
||||
/**
|
||||
* Flink Execution Environment.
|
||||
*/
|
||||
private final AsyncCompactionService compactionScheduleService;
|
||||
|
||||
public HoodieFlinkCompactor(AsyncCompactionService service) {
|
||||
this.compactionScheduleService = service;
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
|
||||
|
||||
FlinkCompactionConfig cfg = getFlinkCompactionConfig(args);
|
||||
Configuration conf = FlinkCompactionConfig.toFlinkConfig(cfg);
|
||||
|
||||
AsyncCompactionService service = new AsyncCompactionService(cfg, conf, env);
|
||||
|
||||
new HoodieFlinkCompactor(service).start(cfg.serviceMode);
|
||||
}
|
||||
|
||||
/**
|
||||
* Main method to start compaction service.
|
||||
*/
|
||||
public void start(boolean serviceMode) throws Exception {
|
||||
if (serviceMode) {
|
||||
compactionScheduleService.start(null);
|
||||
try {
|
||||
compactionScheduleService.waitForShutdown();
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e.getMessage(), e);
|
||||
} finally {
|
||||
LOG.info("Shut down hoodie flink compactor");
|
||||
}
|
||||
} else {
|
||||
LOG.info("Hoodie Flink Compactor running only single round");
|
||||
try {
|
||||
compactionScheduleService.compact();
|
||||
} catch (Exception e) {
|
||||
LOG.error("Got error running delta sync once. Shutting down", e);
|
||||
throw e;
|
||||
} finally {
|
||||
LOG.info("Shut down hoodie flink compactor");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static FlinkCompactionConfig getFlinkCompactionConfig(String[] args) {
|
||||
FlinkCompactionConfig cfg = new FlinkCompactionConfig();
|
||||
JCommander cmd = new JCommander(cfg, null, args);
|
||||
if (cfg.help || args.length == 0) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
return cfg;
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Inner Class
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
/**
|
||||
* Schedules compaction in service.
|
||||
*/
|
||||
public static class AsyncCompactionService extends HoodieAsyncTableService {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
/**
|
||||
* Flink Compaction Config.
|
||||
*/
|
||||
private final FlinkCompactionConfig cfg;
|
||||
|
||||
/**
|
||||
* Flink Config.
|
||||
*/
|
||||
private final Configuration conf;
|
||||
|
||||
/**
|
||||
* Meta Client.
|
||||
*/
|
||||
private final HoodieTableMetaClient metaClient;
|
||||
|
||||
/**
|
||||
* Write Client.
|
||||
*/
|
||||
private final HoodieFlinkWriteClient<?> writeClient;
|
||||
|
||||
/**
|
||||
* The hoodie table.
|
||||
*/
|
||||
private final HoodieFlinkTable<?> table;
|
||||
|
||||
/**
|
||||
* Flink Execution Environment.
|
||||
*/
|
||||
private final StreamExecutionEnvironment env;
|
||||
|
||||
/**
|
||||
* Executor Service.
|
||||
*/
|
||||
private final ExecutorService executor;
|
||||
|
||||
public AsyncCompactionService(FlinkCompactionConfig cfg, Configuration conf, StreamExecutionEnvironment env) throws Exception {
|
||||
this.cfg = cfg;
|
||||
this.conf = conf;
|
||||
this.env = env;
|
||||
this.executor = Executors.newFixedThreadPool(1);
|
||||
|
||||
// create metaClient
|
||||
this.metaClient = StreamerUtil.createMetaClient(conf);
|
||||
|
||||
// get the table name
|
||||
conf.setString(FlinkOptions.TABLE_NAME, metaClient.getTableConfig().getTableName());
|
||||
|
||||
// set table schema
|
||||
CompactionUtil.setAvroSchema(conf, metaClient);
|
||||
|
||||
// infer changelog mode
|
||||
CompactionUtil.inferChangelogMode(conf, metaClient);
|
||||
|
||||
this.writeClient = StreamerUtil.createWriteClient(conf);
|
||||
this.writeConfig = writeClient.getConfig();
|
||||
this.table = writeClient.getHoodieTable();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Pair<CompletableFuture, ExecutorService> startService() {
|
||||
return Pair.of(CompletableFuture.supplyAsync(() -> {
|
||||
boolean error = false;
|
||||
|
||||
try {
|
||||
while (!isShutdownRequested()) {
|
||||
try {
|
||||
compact();
|
||||
Thread.sleep(cfg.minCompactionIntervalSeconds * 1000);
|
||||
} catch (Exception e) {
|
||||
LOG.error("Shutting down compaction service due to exception", e);
|
||||
error = true;
|
||||
throw new HoodieException(e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
shutdownAsyncService(error);
|
||||
}
|
||||
return true;
|
||||
}, executor), executor);
|
||||
}
|
||||
|
||||
private void compact() throws Exception {
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
|
||||
// checks the compaction plan and do compaction.
|
||||
if (cfg.schedule) {
|
||||
Option<String> compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(metaClient);
|
||||
if (compactionInstantTimeOption.isPresent()) {
|
||||
boolean scheduled = writeClient.scheduleCompactionAtInstant(compactionInstantTimeOption.get(), Option.empty());
|
||||
if (!scheduled) {
|
||||
// do nothing.
|
||||
LOG.info("No compaction plan for this job ");
|
||||
return;
|
||||
}
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
}
|
||||
}
|
||||
|
||||
// fetch the instant based on the configured execution sequence
|
||||
HoodieTimeline timeline = table.getActiveTimeline().filterPendingCompactionTimeline();
|
||||
Option<HoodieInstant> requested = CompactionUtil.isLIFO(cfg.compactionSeq) ? timeline.lastInstant() : timeline.firstInstant();
|
||||
if (!requested.isPresent()) {
|
||||
// do nothing.
|
||||
LOG.info("No compaction plan scheduled, turns on the compaction plan schedule with --schedule option");
|
||||
return;
|
||||
}
|
||||
|
||||
String compactionInstantTime = requested.get().getTimestamp();
|
||||
|
||||
HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime);
|
||||
if (timeline.containsInstant(inflightInstant)) {
|
||||
LOG.info("Rollback inflight compaction instant: [" + compactionInstantTime + "]");
|
||||
table.rollbackInflightCompaction(inflightInstant);
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
}
|
||||
|
||||
// generate compaction plan
|
||||
// should support configurable commit metadata
|
||||
HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan(
|
||||
table.getMetaClient(), compactionInstantTime);
|
||||
|
||||
if (compactionPlan == null || (compactionPlan.getOperations() == null)
|
||||
|| (compactionPlan.getOperations().isEmpty())) {
|
||||
// No compaction plan, do nothing and return.
|
||||
LOG.info("No compaction plan for instant " + compactionInstantTime);
|
||||
return;
|
||||
}
|
||||
|
||||
HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime);
|
||||
HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline();
|
||||
if (!pendingCompactionTimeline.containsInstant(instant)) {
|
||||
// this means that the compaction plan was written to auxiliary path(.tmp)
|
||||
// but not the meta path(.hoodie), this usually happens when the job crush
|
||||
// exceptionally.
|
||||
|
||||
// clean the compaction plan in auxiliary path and cancels the compaction.
|
||||
|
||||
LOG.warn("The compaction plan was fetched through the auxiliary path(.tmp) but not the meta path(.hoodie).\n"
|
||||
+ "Clean the compaction plan in auxiliary path and cancels the compaction");
|
||||
CompactionUtil.cleanInstant(table.getMetaClient(), instant);
|
||||
return;
|
||||
}
|
||||
|
||||
// get compactionParallelism.
|
||||
int compactionParallelism = conf.getInteger(FlinkOptions.COMPACTION_TASKS) == -1
|
||||
? compactionPlan.getOperations().size() : conf.getInteger(FlinkOptions.COMPACTION_TASKS);
|
||||
|
||||
LOG.info("Start to compaction for instant " + compactionInstantTime);
|
||||
|
||||
// Mark instant as compaction inflight
|
||||
table.getActiveTimeline().transitionCompactionRequestedToInflight(instant);
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
|
||||
env.addSource(new CompactionPlanSourceFunction(compactionPlan, compactionInstantTime))
|
||||
.name("compaction_source")
|
||||
.uid("uid_compaction_source")
|
||||
.rebalance()
|
||||
.transform("compact_task",
|
||||
TypeInformation.of(CompactionCommitEvent.class),
|
||||
new ProcessOperator<>(new CompactFunction(conf)))
|
||||
.setParallelism(compactionParallelism)
|
||||
.addSink(new CompactionCommitSink(conf))
|
||||
.name("clean_commits")
|
||||
.uid("uid_clean_commits")
|
||||
.setParallelism(1);
|
||||
|
||||
env.execute("flink_hudi_compaction_" + compactionInstantTime);
|
||||
}
|
||||
|
||||
/**
|
||||
* Shutdown async services like compaction/clustering as DeltaSync is shutdown.
|
||||
*/
|
||||
public void shutdownAsyncService(boolean error) {
|
||||
LOG.info("Gracefully shutting down compactor. Error ?" + error);
|
||||
executor.shutdown();
|
||||
writeClient.close();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public void shutDown() {
|
||||
shutdownAsyncService(false);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,50 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.event;
|
||||
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
|
||||
|
||||
/**
|
||||
* An operator event to mark successful instant commit.
|
||||
*/
|
||||
public class CommitAckEvent implements OperatorEvent {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
private long checkpointId;
|
||||
|
||||
public CommitAckEvent(long checkpointId) {
|
||||
this.checkpointId = checkpointId;
|
||||
}
|
||||
|
||||
// default constructor for efficient serialization
|
||||
public CommitAckEvent() {
|
||||
}
|
||||
|
||||
public long getCheckpointId() {
|
||||
return checkpointId;
|
||||
}
|
||||
|
||||
public void setCheckpointId(long checkpointId) {
|
||||
this.checkpointId = checkpointId;
|
||||
}
|
||||
|
||||
public static CommitAckEvent getInstance(long checkpointId) {
|
||||
return new CommitAckEvent(checkpointId);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,237 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.event;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* An operator event to mark successful checkpoint batch write.
|
||||
*/
|
||||
public class WriteMetadataEvent implements OperatorEvent {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public static final String BOOTSTRAP_INSTANT = "";
|
||||
|
||||
private List<WriteStatus> writeStatuses;
|
||||
private int taskID;
|
||||
private String instantTime;
|
||||
private boolean lastBatch;
|
||||
|
||||
/**
|
||||
* Flag saying whether the event comes from the end of input, e.g. the source
|
||||
* is bounded, there are two cases in which this flag should be set to true:
|
||||
* 1. batch execution mode
|
||||
* 2. bounded stream source such as VALUES
|
||||
*/
|
||||
private boolean endInput;
|
||||
|
||||
/**
|
||||
* Flag saying whether the event comes from bootstrap of a write function.
|
||||
*/
|
||||
private boolean bootstrap;
|
||||
|
||||
/**
|
||||
* Creates an event.
|
||||
*
|
||||
* @param taskID The task ID
|
||||
* @param instantTime The instant time under which to write the data
|
||||
* @param writeStatuses The write statues list
|
||||
* @param lastBatch Whether the event reports the last batch
|
||||
* within an checkpoint interval,
|
||||
* if true, the whole data set of the checkpoint
|
||||
* has been flushed successfully
|
||||
* @param bootstrap Whether the event comes from the bootstrap
|
||||
*/
|
||||
private WriteMetadataEvent(
|
||||
int taskID,
|
||||
String instantTime,
|
||||
List<WriteStatus> writeStatuses,
|
||||
boolean lastBatch,
|
||||
boolean endInput,
|
||||
boolean bootstrap) {
|
||||
this.taskID = taskID;
|
||||
this.instantTime = instantTime;
|
||||
this.writeStatuses = new ArrayList<>(writeStatuses);
|
||||
this.lastBatch = lastBatch;
|
||||
this.endInput = endInput;
|
||||
this.bootstrap = bootstrap;
|
||||
}
|
||||
|
||||
// default constructor for efficient serialization
|
||||
public WriteMetadataEvent() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the builder for {@link WriteMetadataEvent}.
|
||||
*/
|
||||
public static Builder builder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
public List<WriteStatus> getWriteStatuses() {
|
||||
return writeStatuses;
|
||||
}
|
||||
|
||||
public void setWriteStatuses(List<WriteStatus> writeStatuses) {
|
||||
this.writeStatuses = writeStatuses;
|
||||
}
|
||||
|
||||
public int getTaskID() {
|
||||
return taskID;
|
||||
}
|
||||
|
||||
public void setTaskID(int taskID) {
|
||||
this.taskID = taskID;
|
||||
}
|
||||
|
||||
public String getInstantTime() {
|
||||
return instantTime;
|
||||
}
|
||||
|
||||
public void setInstantTime(String instantTime) {
|
||||
this.instantTime = instantTime;
|
||||
}
|
||||
|
||||
public boolean isEndInput() {
|
||||
return endInput;
|
||||
}
|
||||
|
||||
public void setEndInput(boolean endInput) {
|
||||
this.endInput = endInput;
|
||||
}
|
||||
|
||||
public boolean isBootstrap() {
|
||||
return bootstrap;
|
||||
}
|
||||
|
||||
public void setBootstrap(boolean bootstrap) {
|
||||
this.bootstrap = bootstrap;
|
||||
}
|
||||
|
||||
public boolean isLastBatch() {
|
||||
return lastBatch;
|
||||
}
|
||||
|
||||
public void setLastBatch(boolean lastBatch) {
|
||||
this.lastBatch = lastBatch;
|
||||
}
|
||||
|
||||
/**
|
||||
* Merges this event with given {@link WriteMetadataEvent} {@code other}.
|
||||
*
|
||||
* @param other The event to be merged
|
||||
*/
|
||||
public void mergeWith(WriteMetadataEvent other) {
|
||||
ValidationUtils.checkArgument(this.taskID == other.taskID);
|
||||
// the instant time could be monotonically increasing
|
||||
this.instantTime = other.instantTime;
|
||||
this.lastBatch |= other.lastBatch; // true if one of the event lastBatch is true
|
||||
List<WriteStatus> statusList = new ArrayList<>();
|
||||
statusList.addAll(this.writeStatuses);
|
||||
statusList.addAll(other.writeStatuses);
|
||||
this.writeStatuses = statusList;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether the event is ready to commit.
|
||||
*/
|
||||
public boolean isReady(String currentInstant) {
|
||||
return lastBatch && this.instantTime.equals(currentInstant);
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Utilities
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
/**
|
||||
* Creates empty bootstrap event for task {@code taskId}.
|
||||
*
|
||||
* <p>The event indicates that the new instant can start directly,
|
||||
* there is no old instant write statuses to recover.
|
||||
*/
|
||||
public static WriteMetadataEvent emptyBootstrap(int taskId) {
|
||||
return WriteMetadataEvent.builder()
|
||||
.taskID(taskId)
|
||||
.instantTime(BOOTSTRAP_INSTANT)
|
||||
.writeStatus(Collections.emptyList())
|
||||
.bootstrap(true)
|
||||
.build();
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Builder
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
/**
|
||||
* Builder for {@link WriteMetadataEvent}.
|
||||
*/
|
||||
public static class Builder {
|
||||
private List<WriteStatus> writeStatus;
|
||||
private Integer taskID;
|
||||
private String instantTime;
|
||||
private boolean lastBatch = false;
|
||||
private boolean endInput = false;
|
||||
private boolean bootstrap = false;
|
||||
|
||||
public WriteMetadataEvent build() {
|
||||
Objects.requireNonNull(taskID);
|
||||
Objects.requireNonNull(instantTime);
|
||||
Objects.requireNonNull(writeStatus);
|
||||
return new WriteMetadataEvent(taskID, instantTime, writeStatus, lastBatch, endInput, bootstrap);
|
||||
}
|
||||
|
||||
public Builder taskID(int taskID) {
|
||||
this.taskID = taskID;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder instantTime(String instantTime) {
|
||||
this.instantTime = instantTime;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder writeStatus(List<WriteStatus> writeStatus) {
|
||||
this.writeStatus = writeStatus;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder lastBatch(boolean lastBatch) {
|
||||
this.lastBatch = lastBatch;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder endInput(boolean endInput) {
|
||||
this.endInput = endInput;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bootstrap(boolean bootstrap) {
|
||||
this.bootstrap = bootstrap;
|
||||
return this;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,113 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.meta;
|
||||
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.jetbrains.annotations.NotNull;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* A checkpoint message.
|
||||
*/
|
||||
public class CkpMessage implements Serializable, Comparable<CkpMessage> {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public static final Comparator<CkpMessage> COMPARATOR = Comparator.comparing(CkpMessage::getInstant)
|
||||
.thenComparing(CkpMessage::getState);
|
||||
|
||||
private final String instant; // the instant time
|
||||
private final State state; // the checkpoint state
|
||||
|
||||
public CkpMessage(String instant, String state) {
|
||||
this.instant = instant;
|
||||
this.state = State.valueOf(state);
|
||||
}
|
||||
|
||||
public CkpMessage(FileStatus fileStatus) {
|
||||
String fileName = fileStatus.getPath().getName();
|
||||
String[] nameAndExt = fileName.split("\\.");
|
||||
ValidationUtils.checkState(nameAndExt.length == 2);
|
||||
String name = nameAndExt[0];
|
||||
String ext = nameAndExt[1];
|
||||
|
||||
this.instant = name;
|
||||
this.state = State.valueOf(ext);
|
||||
}
|
||||
|
||||
public String getInstant() {
|
||||
return instant;
|
||||
}
|
||||
|
||||
public State getState() {
|
||||
return state;
|
||||
}
|
||||
|
||||
public boolean isAborted() {
|
||||
return State.ABORTED == this.state;
|
||||
}
|
||||
|
||||
public boolean isComplete() {
|
||||
return State.COMPLETED == this.state;
|
||||
}
|
||||
|
||||
public boolean isInflight() {
|
||||
return State.INFLIGHT == this.state;
|
||||
}
|
||||
|
||||
public static String getFileName(String instant, State state) {
|
||||
return instant + "." + state.name();
|
||||
}
|
||||
|
||||
public static List<String> getAllFileNames(String instant) {
|
||||
return Arrays.stream(State.values())
|
||||
.map(state -> getFileName(instant, state))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(@NotNull CkpMessage o) {
|
||||
return COMPARATOR.compare(this, o);
|
||||
}
|
||||
|
||||
/**
|
||||
* Instant State.
|
||||
*/
|
||||
public enum State {
|
||||
// Inflight instant
|
||||
INFLIGHT,
|
||||
// Aborted instant
|
||||
// An instant can be aborted then be reused again, so it has lower priority
|
||||
// than COMPLETED
|
||||
ABORTED,
|
||||
// Committed instant
|
||||
COMPLETED
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "Ckp{" + "instant='" + instant + '\'' + ", state='" + state + '\'' + '}';
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,227 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.meta;
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* The checkpoint metadata for bookkeeping the checkpoint messages.
|
||||
*
|
||||
* <p>Each time the driver starts a new instant, it writes a commit message into the metadata, the write tasks
|
||||
* then consume the message and unblock the data flushing.
|
||||
*
|
||||
* <p>Why we use the DFS based message queue instead of sending
|
||||
* the {@link org.apache.flink.runtime.operators.coordination.OperatorEvent} ?
|
||||
* The write task handles the operator event using the main mailbox executor which has the lowest priority for mails,
|
||||
* it is also used to process the inputs. When the write task blocks and waits for the operator event to ack the valid instant to write,
|
||||
* it actually blocks all the subsequent events in the mailbox, the operator event would never be consumed then it causes deadlock.
|
||||
*
|
||||
* <p>The checkpoint metadata is also more lightweight than the active timeline.
|
||||
*
|
||||
* <p>NOTE: should be removed in the future if we have good manner to handle the async notifications from driver.
|
||||
*/
|
||||
public class CkpMetadata implements Serializable {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(CkpMetadata.class);
|
||||
|
||||
protected static final int MAX_RETAIN_CKP_NUM = 3;
|
||||
|
||||
// the ckp metadata directory
|
||||
private static final String CKP_META = "ckp_meta";
|
||||
|
||||
private final FileSystem fs;
|
||||
protected final Path path;
|
||||
|
||||
private List<CkpMessage> messages;
|
||||
private List<String> instantCache;
|
||||
|
||||
private CkpMetadata(String basePath) {
|
||||
this(FSUtils.getFs(basePath, StreamerUtil.getHadoopConf()), basePath);
|
||||
}
|
||||
|
||||
private CkpMetadata(FileSystem fs, String basePath) {
|
||||
this.fs = fs;
|
||||
this.path = new Path(ckpMetaPath(basePath));
|
||||
}
|
||||
|
||||
public void close() {
|
||||
this.instantCache = null;
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// WRITE METHODS
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
/**
|
||||
* Initialize the message bus, would clean all the messages and publish the last pending instant.
|
||||
*
|
||||
* <p>This expects to be called by the driver.
|
||||
*/
|
||||
public void bootstrap(HoodieTableMetaClient metaClient) throws IOException {
|
||||
fs.delete(path, true);
|
||||
fs.mkdirs(path);
|
||||
metaClient.getActiveTimeline().getCommitsTimeline().filterPendingExcludingCompaction()
|
||||
.lastInstant().ifPresent(instant -> startInstant(instant.getTimestamp()));
|
||||
}
|
||||
|
||||
public void startInstant(String instant) {
|
||||
Path path = fullPath(CkpMessage.getFileName(instant, CkpMessage.State.INFLIGHT));
|
||||
try {
|
||||
fs.createNewFile(path);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Exception while adding checkpoint start metadata for instant: " + instant);
|
||||
}
|
||||
// cleaning
|
||||
clean(instant);
|
||||
}
|
||||
|
||||
private void clean(String newInstant) {
|
||||
if (this.instantCache == null) {
|
||||
this.instantCache = new ArrayList<>();
|
||||
}
|
||||
this.instantCache.add(newInstant);
|
||||
if (instantCache.size() > MAX_RETAIN_CKP_NUM) {
|
||||
final String instant = instantCache.get(0);
|
||||
boolean[] error = new boolean[1];
|
||||
CkpMessage.getAllFileNames(instant).stream().map(this::fullPath).forEach(path -> {
|
||||
try {
|
||||
fs.delete(path, false);
|
||||
} catch (IOException e) {
|
||||
error[0] = true;
|
||||
LOG.warn("Exception while cleaning the checkpoint meta file: " + path);
|
||||
}
|
||||
});
|
||||
if (!error[0]) {
|
||||
instantCache.remove(0);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a checkpoint commit message.
|
||||
*
|
||||
* @param instant The committed instant
|
||||
*/
|
||||
public void commitInstant(String instant) {
|
||||
Path path = fullPath(CkpMessage.getFileName(instant, CkpMessage.State.COMPLETED));
|
||||
try {
|
||||
fs.createNewFile(path);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Exception while adding checkpoint commit metadata for instant: " + instant);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Add an aborted checkpoint message.
|
||||
*/
|
||||
public void abortInstant(String instant) {
|
||||
Path path = fullPath(CkpMessage.getFileName(instant, CkpMessage.State.ABORTED));
|
||||
try {
|
||||
fs.createNewFile(path);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Exception while adding checkpoint abort metadata for instant: " + instant);
|
||||
}
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// READ METHODS
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
private void load() {
|
||||
try {
|
||||
this.messages = scanCkpMetadata(this.path);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Exception while scanning the checkpoint meta files under path: " + this.path);
|
||||
}
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public String lastPendingInstant() {
|
||||
load();
|
||||
for (int i = this.messages.size() - 1; i >= 0; i--) {
|
||||
CkpMessage ckpMsg = this.messages.get(i);
|
||||
// consider 'aborted' as pending too to reuse the instant
|
||||
if (!ckpMsg.isComplete()) {
|
||||
return ckpMsg.getInstant();
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
public List<CkpMessage> getMessages() {
|
||||
load();
|
||||
return messages;
|
||||
}
|
||||
|
||||
public boolean isAborted(String instant) {
|
||||
ValidationUtils.checkState(this.messages != null, "The checkpoint metadata should #load first");
|
||||
return this.messages.stream().anyMatch(ckpMsg -> instant.equals(ckpMsg.getInstant()) && ckpMsg.isAborted());
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Utilities
|
||||
// -------------------------------------------------------------------------
|
||||
public static CkpMetadata getInstance(String basePath) {
|
||||
return new CkpMetadata(basePath);
|
||||
}
|
||||
|
||||
public static CkpMetadata getInstance(FileSystem fs, String basePath) {
|
||||
return new CkpMetadata(fs, basePath);
|
||||
}
|
||||
|
||||
protected static String ckpMetaPath(String basePath) {
|
||||
return basePath + Path.SEPARATOR + HoodieTableMetaClient.AUXILIARYFOLDER_NAME + Path.SEPARATOR + CKP_META;
|
||||
}
|
||||
|
||||
private Path fullPath(String fileName) {
|
||||
return new Path(path, fileName);
|
||||
}
|
||||
|
||||
private List<CkpMessage> scanCkpMetadata(Path ckpMetaPath) throws IOException {
|
||||
return Arrays.stream(this.fs.listStatus(ckpMetaPath)).map(CkpMessage::new)
|
||||
.collect(Collectors.groupingBy(CkpMessage::getInstant)).values().stream()
|
||||
.map(messages -> messages.stream().reduce((x, y) -> {
|
||||
// Pick the one with the highest state
|
||||
if (x.getState().compareTo(y.getState()) >= 0) {
|
||||
return x;
|
||||
}
|
||||
return y;
|
||||
}).get())
|
||||
.sorted().collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,242 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.partitioner;
|
||||
|
||||
import org.apache.hudi.client.FlinkTaskContextSupplier;
|
||||
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
|
||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||
import org.apache.hudi.common.model.BaseAvroPayload;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordGlobalLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.sink.bootstrap.IndexRecord;
|
||||
import org.apache.hudi.sink.utils.PayloadCreation;
|
||||
import org.apache.hudi.table.action.commit.BucketInfo;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.api.common.state.CheckpointListener;
|
||||
import org.apache.flink.api.common.state.StateTtlConfig;
|
||||
import org.apache.flink.api.common.state.ValueState;
|
||||
import org.apache.flink.api.common.state.ValueStateDescriptor;
|
||||
import org.apache.flink.api.common.time.Time;
|
||||
import org.apache.flink.api.common.typeinfo.TypeInformation;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.runtime.state.FunctionInitializationContext;
|
||||
import org.apache.flink.runtime.state.FunctionSnapshotContext;
|
||||
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
|
||||
import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
|
||||
import org.apache.flink.util.Collector;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* The function to build the write profile incrementally for records within a checkpoint,
|
||||
* it then assigns the bucket with ID using the {@link BucketAssigner}.
|
||||
*
|
||||
* <p>All the records are tagged with HoodieRecordLocation, instead of real instant time,
|
||||
* INSERT record uses "I" and UPSERT record uses "U" as instant time. There is no need to keep
|
||||
* the "real" instant time for each record, the bucket ID (partition path & fileID) actually decides
|
||||
* where the record should write to. The "I" and "U" tags are only used for downstream to decide whether
|
||||
* the data bucket is an INSERT or an UPSERT, we should factor the tags out when the underneath writer
|
||||
* supports specifying the bucket type explicitly.
|
||||
*
|
||||
* <p>The output records should then shuffle by the bucket ID and thus do scalable write.
|
||||
*
|
||||
* @see BucketAssigner
|
||||
*/
|
||||
public class BucketAssignFunction<K, I, O extends HoodieRecord<?>>
|
||||
extends KeyedProcessFunction<K, I, O>
|
||||
implements CheckpointedFunction, CheckpointListener {
|
||||
|
||||
/**
|
||||
* Index cache(speed-up) state for the underneath file based(BloomFilter) indices.
|
||||
* When a record came in, we do these check:
|
||||
*
|
||||
* <ul>
|
||||
* <li>Try to load all the records in the partition path where the record belongs to</li>
|
||||
* <li>Checks whether the state contains the record key</li>
|
||||
* <li>If it does, tag the record with the location</li>
|
||||
* <li>If it does not, use the {@link BucketAssigner} to generate a new bucket ID</li>
|
||||
* </ul>
|
||||
*/
|
||||
private ValueState<HoodieRecordGlobalLocation> indexState;
|
||||
|
||||
/**
|
||||
* Bucket assigner to assign new bucket IDs or reuse existing ones.
|
||||
*/
|
||||
private BucketAssigner bucketAssigner;
|
||||
|
||||
private final Configuration conf;
|
||||
|
||||
private final boolean isChangingRecords;
|
||||
|
||||
/**
|
||||
* Used to create DELETE payload.
|
||||
*/
|
||||
private PayloadCreation payloadCreation;
|
||||
|
||||
/**
|
||||
* If the index is global, update the index for the old partition path
|
||||
* if same key record with different partition path came in.
|
||||
*/
|
||||
private final boolean globalIndex;
|
||||
|
||||
public BucketAssignFunction(Configuration conf) {
|
||||
this.conf = conf;
|
||||
this.isChangingRecords = WriteOperationType.isChangingRecords(
|
||||
WriteOperationType.fromValue(conf.getString(FlinkOptions.OPERATION)));
|
||||
this.globalIndex = conf.getBoolean(FlinkOptions.INDEX_GLOBAL_ENABLED)
|
||||
&& !conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(Configuration parameters) throws Exception {
|
||||
super.open(parameters);
|
||||
HoodieWriteConfig writeConfig = StreamerUtil.getHoodieClientConfig(this.conf, true);
|
||||
HoodieFlinkEngineContext context = new HoodieFlinkEngineContext(
|
||||
new SerializableConfiguration(StreamerUtil.getHadoopConf()),
|
||||
new FlinkTaskContextSupplier(getRuntimeContext()));
|
||||
this.bucketAssigner = BucketAssigners.create(
|
||||
getRuntimeContext().getIndexOfThisSubtask(),
|
||||
getRuntimeContext().getMaxNumberOfParallelSubtasks(),
|
||||
getRuntimeContext().getNumberOfParallelSubtasks(),
|
||||
ignoreSmallFiles(),
|
||||
HoodieTableType.valueOf(conf.getString(FlinkOptions.TABLE_TYPE)),
|
||||
context,
|
||||
writeConfig);
|
||||
this.payloadCreation = PayloadCreation.instance(this.conf);
|
||||
}
|
||||
|
||||
private boolean ignoreSmallFiles() {
|
||||
WriteOperationType operationType = WriteOperationType.fromValue(conf.getString(FlinkOptions.OPERATION));
|
||||
return WriteOperationType.isOverwrite(operationType);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void snapshotState(FunctionSnapshotContext context) {
|
||||
this.bucketAssigner.reset();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initializeState(FunctionInitializationContext context) {
|
||||
ValueStateDescriptor<HoodieRecordGlobalLocation> indexStateDesc =
|
||||
new ValueStateDescriptor<>(
|
||||
"indexState",
|
||||
TypeInformation.of(HoodieRecordGlobalLocation.class));
|
||||
double ttl = conf.getDouble(FlinkOptions.INDEX_STATE_TTL) * 24 * 60 * 60 * 1000;
|
||||
if (ttl > 0) {
|
||||
indexStateDesc.enableTimeToLive(StateTtlConfig.newBuilder(Time.milliseconds((long) ttl)).build());
|
||||
}
|
||||
indexState = context.getKeyedStateStore().getState(indexStateDesc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processElement(I value, Context ctx, Collector<O> out) throws Exception {
|
||||
if (value instanceof IndexRecord) {
|
||||
IndexRecord<?> indexRecord = (IndexRecord<?>) value;
|
||||
this.indexState.update((HoodieRecordGlobalLocation) indexRecord.getCurrentLocation());
|
||||
} else {
|
||||
processRecord((HoodieRecord<?>) value, out);
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private void processRecord(HoodieRecord<?> record, Collector<O> out) throws Exception {
|
||||
// 1. put the record into the BucketAssigner;
|
||||
// 2. look up the state for location, if the record has a location, just send it out;
|
||||
// 3. if it is an INSERT, decide the location using the BucketAssigner then send it out.
|
||||
final HoodieKey hoodieKey = record.getKey();
|
||||
final String recordKey = hoodieKey.getRecordKey();
|
||||
final String partitionPath = hoodieKey.getPartitionPath();
|
||||
final HoodieRecordLocation location;
|
||||
|
||||
// Only changing records need looking up the index for the location,
|
||||
// append only records are always recognized as INSERT.
|
||||
HoodieRecordGlobalLocation oldLoc = indexState.value();
|
||||
if (isChangingRecords && oldLoc != null) {
|
||||
// Set up the instant time as "U" to mark the bucket as an update bucket.
|
||||
if (!Objects.equals(oldLoc.getPartitionPath(), partitionPath)) {
|
||||
if (globalIndex) {
|
||||
// if partition path changes, emit a delete record for old partition path,
|
||||
// then update the index state using location with new partition path.
|
||||
HoodieRecord<?> deleteRecord = new HoodieAvroRecord<>(new HoodieKey(recordKey, oldLoc.getPartitionPath()),
|
||||
payloadCreation.createDeletePayload((BaseAvroPayload) record.getData()));
|
||||
deleteRecord.setCurrentLocation(oldLoc.toLocal("U"));
|
||||
deleteRecord.seal();
|
||||
out.collect((O) deleteRecord);
|
||||
}
|
||||
location = getNewRecordLocation(partitionPath);
|
||||
updateIndexState(partitionPath, location);
|
||||
} else {
|
||||
location = oldLoc.toLocal("U");
|
||||
this.bucketAssigner.addUpdate(partitionPath, location.getFileId());
|
||||
}
|
||||
} else {
|
||||
location = getNewRecordLocation(partitionPath);
|
||||
}
|
||||
// always refresh the index
|
||||
if (isChangingRecords) {
|
||||
updateIndexState(partitionPath, location);
|
||||
}
|
||||
record.setCurrentLocation(location);
|
||||
out.collect((O) record);
|
||||
}
|
||||
|
||||
private HoodieRecordLocation getNewRecordLocation(String partitionPath) {
|
||||
final BucketInfo bucketInfo = this.bucketAssigner.addInsert(partitionPath);
|
||||
final HoodieRecordLocation location;
|
||||
switch (bucketInfo.getBucketType()) {
|
||||
case INSERT:
|
||||
// This is an insert bucket, use HoodieRecordLocation instant time as "I".
|
||||
// Downstream operators can then check the instant time to know whether
|
||||
// a record belongs to an insert bucket.
|
||||
location = new HoodieRecordLocation("I", bucketInfo.getFileIdPrefix());
|
||||
break;
|
||||
case UPDATE:
|
||||
location = new HoodieRecordLocation("U", bucketInfo.getFileIdPrefix());
|
||||
break;
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
return location;
|
||||
}
|
||||
|
||||
private void updateIndexState(
|
||||
String partitionPath,
|
||||
HoodieRecordLocation localLoc) throws Exception {
|
||||
this.indexState.update(HoodieRecordGlobalLocation.fromLocal(partitionPath, localLoc));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void notifyCheckpointComplete(long checkpointId) {
|
||||
// Refresh the table state when there are new commits.
|
||||
this.bucketAssigner.reload(checkpointId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws Exception {
|
||||
this.bucketAssigner.close();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,348 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.partitioner;
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.sink.partitioner.profile.WriteProfile;
|
||||
import org.apache.hudi.sink.partitioner.profile.WriteProfiles;
|
||||
import org.apache.hudi.table.action.commit.BucketInfo;
|
||||
import org.apache.hudi.table.action.commit.BucketType;
|
||||
import org.apache.hudi.table.action.commit.SmallFile;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.annotation.VisibleForTesting;
|
||||
import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
|
||||
import org.apache.flink.util.Preconditions;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Bucket assigner that assigns the data buffer of one checkpoint into buckets.
|
||||
*
|
||||
* <p>This assigner assigns the record one by one.
|
||||
* If the record is an update, checks and reuse existing UPDATE bucket or generates a new one;
|
||||
* If the record is an insert, checks the record partition for small files first, try to find a small file
|
||||
* that has space to append new records and reuse the small file's data bucket, if
|
||||
* there is no small file(or no left space for new records), generates an INSERT bucket.
|
||||
*
|
||||
* <p>Use {partition}_{fileId} as the bucket identifier, so that the bucket is unique
|
||||
* within and among partitions.
|
||||
*/
|
||||
public class BucketAssigner implements AutoCloseable {
|
||||
private static final Logger LOG = LogManager.getLogger(BucketAssigner.class);
|
||||
|
||||
/**
|
||||
* Task ID.
|
||||
*/
|
||||
private final int taskID;
|
||||
|
||||
/**
|
||||
* The max parallelism.
|
||||
*/
|
||||
private final int maxParallelism;
|
||||
|
||||
/**
|
||||
* Number of tasks.
|
||||
*/
|
||||
private final int numTasks;
|
||||
|
||||
/**
|
||||
* Remembers what type each bucket is for later.
|
||||
*/
|
||||
private final HashMap<String, BucketInfo> bucketInfoMap;
|
||||
|
||||
/**
|
||||
* The write config.
|
||||
*/
|
||||
protected final HoodieWriteConfig config;
|
||||
|
||||
/**
|
||||
* Write profile.
|
||||
*/
|
||||
private final WriteProfile writeProfile;
|
||||
|
||||
/**
|
||||
* Partition path to small file assign mapping.
|
||||
*/
|
||||
private final Map<String, SmallFileAssign> smallFileAssignMap;
|
||||
|
||||
/**
|
||||
* Bucket ID(partition + fileId) -> new file assign state.
|
||||
*/
|
||||
private final Map<String, NewFileAssignState> newFileAssignStates;
|
||||
|
||||
/**
|
||||
* Num of accumulated successful checkpoints, used for cleaning the new file assign state.
|
||||
*/
|
||||
private int accCkp = 0;
|
||||
|
||||
public BucketAssigner(
|
||||
int taskID,
|
||||
int maxParallelism,
|
||||
int numTasks,
|
||||
WriteProfile profile,
|
||||
HoodieWriteConfig config) {
|
||||
this.taskID = taskID;
|
||||
this.maxParallelism = maxParallelism;
|
||||
this.numTasks = numTasks;
|
||||
this.config = config;
|
||||
this.writeProfile = profile;
|
||||
|
||||
this.bucketInfoMap = new HashMap<>();
|
||||
this.smallFileAssignMap = new HashMap<>();
|
||||
this.newFileAssignStates = new HashMap<>();
|
||||
}
|
||||
|
||||
/**
|
||||
* Reset the states of this assigner, should do once for each checkpoint,
|
||||
* all the states are accumulated within one checkpoint interval.
|
||||
*/
|
||||
public void reset() {
|
||||
bucketInfoMap.clear();
|
||||
}
|
||||
|
||||
public BucketInfo addUpdate(String partitionPath, String fileIdHint) {
|
||||
final String key = StreamerUtil.generateBucketKey(partitionPath, fileIdHint);
|
||||
if (!bucketInfoMap.containsKey(key)) {
|
||||
BucketInfo bucketInfo = new BucketInfo(BucketType.UPDATE, fileIdHint, partitionPath);
|
||||
bucketInfoMap.put(key, bucketInfo);
|
||||
}
|
||||
// else do nothing because the bucket already exists.
|
||||
return bucketInfoMap.get(key);
|
||||
}
|
||||
|
||||
public BucketInfo addInsert(String partitionPath) {
|
||||
// for new inserts, compute buckets depending on how many records we have for each partition
|
||||
SmallFileAssign smallFileAssign = getSmallFileAssign(partitionPath);
|
||||
|
||||
// first try packing this into one of the smallFiles
|
||||
if (smallFileAssign != null && smallFileAssign.assign()) {
|
||||
return new BucketInfo(BucketType.UPDATE, smallFileAssign.getFileId(), partitionPath);
|
||||
}
|
||||
|
||||
// if we have anything more, create new insert buckets, like normal
|
||||
if (newFileAssignStates.containsKey(partitionPath)) {
|
||||
NewFileAssignState newFileAssignState = newFileAssignStates.get(partitionPath);
|
||||
if (newFileAssignState.canAssign()) {
|
||||
newFileAssignState.assign();
|
||||
final String key = StreamerUtil.generateBucketKey(partitionPath, newFileAssignState.fileId);
|
||||
if (bucketInfoMap.containsKey(key)) {
|
||||
// the newFileAssignStates is cleaned asynchronously when received the checkpoint success notification,
|
||||
// the records processed within the time range:
|
||||
// (start checkpoint, checkpoint success(and instant committed))
|
||||
// should still be assigned to the small buckets of last checkpoint instead of new one.
|
||||
|
||||
// the bucketInfoMap is cleaned when checkpoint starts.
|
||||
|
||||
// A promotion: when the HoodieRecord can record whether it is an UPDATE or INSERT,
|
||||
// we can always return an UPDATE BucketInfo here, and there is no need to record the
|
||||
// UPDATE bucket through calling #addUpdate.
|
||||
return bucketInfoMap.get(key);
|
||||
}
|
||||
return new BucketInfo(BucketType.UPDATE, newFileAssignState.fileId, partitionPath);
|
||||
}
|
||||
}
|
||||
BucketInfo bucketInfo = new BucketInfo(BucketType.INSERT, createFileIdOfThisTask(), partitionPath);
|
||||
final String key = StreamerUtil.generateBucketKey(partitionPath, bucketInfo.getFileIdPrefix());
|
||||
bucketInfoMap.put(key, bucketInfo);
|
||||
NewFileAssignState newFileAssignState = new NewFileAssignState(bucketInfo.getFileIdPrefix(), writeProfile.getRecordsPerBucket());
|
||||
newFileAssignState.assign();
|
||||
newFileAssignStates.put(partitionPath, newFileAssignState);
|
||||
return bucketInfo;
|
||||
}
|
||||
|
||||
private synchronized SmallFileAssign getSmallFileAssign(String partitionPath) {
|
||||
if (smallFileAssignMap.containsKey(partitionPath)) {
|
||||
return smallFileAssignMap.get(partitionPath);
|
||||
}
|
||||
List<SmallFile> smallFiles = smallFilesOfThisTask(writeProfile.getSmallFiles(partitionPath));
|
||||
if (smallFiles.size() > 0) {
|
||||
LOG.info("For partitionPath : " + partitionPath + " Small Files => " + smallFiles);
|
||||
SmallFileAssignState[] states = smallFiles.stream()
|
||||
.map(smallFile -> new SmallFileAssignState(config.getParquetMaxFileSize(), smallFile, writeProfile.getAvgSize()))
|
||||
.toArray(SmallFileAssignState[]::new);
|
||||
SmallFileAssign assign = new SmallFileAssign(states);
|
||||
smallFileAssignMap.put(partitionPath, assign);
|
||||
return assign;
|
||||
}
|
||||
smallFileAssignMap.put(partitionPath, null);
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Refresh the table state like TableFileSystemView and HoodieTimeline.
|
||||
*/
|
||||
public synchronized void reload(long checkpointId) {
|
||||
this.accCkp += 1;
|
||||
if (this.accCkp > 1) {
|
||||
// do not clean the new file assignment state for the first checkpoint,
|
||||
// this #reload calling is triggered by checkpoint success event, the coordinator
|
||||
// also relies on the checkpoint success event to commit the inflight instant,
|
||||
// and very possibly this component receives the notification before the coordinator,
|
||||
// if we do the cleaning, the records processed within the time range:
|
||||
// (start checkpoint, checkpoint success(and instant committed))
|
||||
// would be assigned to a fresh new data bucket which is not the right behavior.
|
||||
this.newFileAssignStates.clear();
|
||||
this.accCkp = 0;
|
||||
}
|
||||
this.smallFileAssignMap.clear();
|
||||
this.writeProfile.reload(checkpointId);
|
||||
}
|
||||
|
||||
private boolean fileIdOfThisTask(String fileId) {
|
||||
// the file id can shuffle to this task
|
||||
return KeyGroupRangeAssignment.assignKeyToParallelOperator(fileId, maxParallelism, numTasks) == taskID;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public String createFileIdOfThisTask() {
|
||||
String newFileIdPfx = FSUtils.createNewFileIdPfx();
|
||||
while (!fileIdOfThisTask(newFileIdPfx)) {
|
||||
newFileIdPfx = FSUtils.createNewFileIdPfx();
|
||||
}
|
||||
return newFileIdPfx;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public List<SmallFile> smallFilesOfThisTask(List<SmallFile> smallFiles) {
|
||||
// computes the small files to write inserts for this task.
|
||||
return smallFiles.stream()
|
||||
.filter(smallFile -> fileIdOfThisTask(smallFile.location.getFileId()))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public void close() {
|
||||
reset();
|
||||
WriteProfiles.clean(config.getBasePath());
|
||||
}
|
||||
|
||||
/**
|
||||
* Assigns the record to one of the small files under one partition.
|
||||
*
|
||||
* <p> The tool is initialized with an array of {@link SmallFileAssignState}s.
|
||||
* A pointer points to the current small file we are ready to assign,
|
||||
* if the current small file can not be assigned anymore (full assigned), the pointer
|
||||
* move to next small file.
|
||||
* <pre>
|
||||
* | ->
|
||||
* V
|
||||
* | smallFile_1 | smallFile_2 | smallFile_3 | ... | smallFile_N |
|
||||
* </pre>
|
||||
*
|
||||
* <p>If all the small files are full assigned, a flag {@code noSpace} was marked to true, and
|
||||
* we can return early for future check.
|
||||
*/
|
||||
private static class SmallFileAssign {
|
||||
final SmallFileAssignState[] states;
|
||||
int assignIdx = 0;
|
||||
boolean noSpace = false;
|
||||
|
||||
SmallFileAssign(SmallFileAssignState[] states) {
|
||||
this.states = states;
|
||||
}
|
||||
|
||||
public boolean assign() {
|
||||
if (noSpace) {
|
||||
return false;
|
||||
}
|
||||
SmallFileAssignState state = states[assignIdx];
|
||||
while (!state.canAssign()) {
|
||||
assignIdx += 1;
|
||||
if (assignIdx >= states.length) {
|
||||
noSpace = true;
|
||||
return false;
|
||||
}
|
||||
// move to next slot if possible
|
||||
state = states[assignIdx];
|
||||
}
|
||||
state.assign();
|
||||
return true;
|
||||
}
|
||||
|
||||
public String getFileId() {
|
||||
return states[assignIdx].fileId;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Candidate bucket state for small file. It records the total number of records
|
||||
* that the bucket can append and the current number of assigned records.
|
||||
*/
|
||||
private static class SmallFileAssignState {
|
||||
long assigned;
|
||||
long totalUnassigned;
|
||||
final String fileId;
|
||||
|
||||
SmallFileAssignState(long parquetMaxFileSize, SmallFile smallFile, long averageRecordSize) {
|
||||
this.assigned = 0;
|
||||
this.totalUnassigned = (parquetMaxFileSize - smallFile.sizeBytes) / averageRecordSize;
|
||||
this.fileId = smallFile.location.getFileId();
|
||||
}
|
||||
|
||||
public boolean canAssign() {
|
||||
return this.totalUnassigned > 0 && this.totalUnassigned > this.assigned;
|
||||
}
|
||||
|
||||
/**
|
||||
* Remembers to invoke {@link #canAssign()} first.
|
||||
*/
|
||||
public void assign() {
|
||||
Preconditions.checkState(canAssign(),
|
||||
"Can not assign insert to small file: assigned => "
|
||||
+ this.assigned + " totalUnassigned => " + this.totalUnassigned);
|
||||
this.assigned++;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Candidate bucket state for a new file. It records the total number of records
|
||||
* that the bucket can append and the current number of assigned records.
|
||||
*/
|
||||
private static class NewFileAssignState {
|
||||
long assigned;
|
||||
long totalUnassigned;
|
||||
final String fileId;
|
||||
|
||||
NewFileAssignState(String fileId, long insertRecordsPerBucket) {
|
||||
this.fileId = fileId;
|
||||
this.assigned = 0;
|
||||
this.totalUnassigned = insertRecordsPerBucket;
|
||||
}
|
||||
|
||||
public boolean canAssign() {
|
||||
return this.totalUnassigned > 0 && this.totalUnassigned > this.assigned;
|
||||
}
|
||||
|
||||
/**
|
||||
* Remembers to invoke {@link #canAssign()} first.
|
||||
*/
|
||||
public void assign() {
|
||||
Preconditions.checkState(canAssign(),
|
||||
"Can not assign insert to new file: assigned => "
|
||||
+ this.assigned + " totalUnassigned => " + this.totalUnassigned);
|
||||
this.assigned++;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,59 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.partitioner;
|
||||
|
||||
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.sink.partitioner.profile.WriteProfile;
|
||||
import org.apache.hudi.sink.partitioner.profile.WriteProfiles;
|
||||
|
||||
/**
|
||||
* Utilities for {@code BucketAssigner}.
|
||||
*/
|
||||
public abstract class BucketAssigners {
|
||||
|
||||
private BucketAssigners() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a {@code BucketAssigner}.
|
||||
*
|
||||
* @param taskID The task ID
|
||||
* @param maxParallelism The max parallelism
|
||||
* @param numTasks The number of tasks
|
||||
* @param ignoreSmallFiles Whether to ignore the small files
|
||||
* @param tableType The table type
|
||||
* @param context The engine context
|
||||
* @param config The configuration
|
||||
* @return the bucket assigner instance
|
||||
*/
|
||||
public static BucketAssigner create(
|
||||
int taskID,
|
||||
int maxParallelism,
|
||||
int numTasks,
|
||||
boolean ignoreSmallFiles,
|
||||
HoodieTableType tableType,
|
||||
HoodieFlinkEngineContext context,
|
||||
HoodieWriteConfig config) {
|
||||
boolean delta = tableType.equals(HoodieTableType.MERGE_ON_READ);
|
||||
WriteProfile writeProfile = WriteProfiles.singleton(ignoreSmallFiles, delta, config, context);
|
||||
return new BucketAssigner(taskID, maxParallelism, numTasks, writeProfile, config);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,47 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.partitioner;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.index.bucket.BucketIdentifier;
|
||||
|
||||
import org.apache.flink.api.common.functions.Partitioner;
|
||||
|
||||
/**
|
||||
* Bucket index input partitioner.
|
||||
* The fields to hash can be a subset of the primary key fields.
|
||||
*
|
||||
* @param <T> The type of obj to hash
|
||||
*/
|
||||
public class BucketIndexPartitioner<T extends HoodieKey> implements Partitioner<T> {
|
||||
|
||||
private final int bucketNum;
|
||||
private final String indexKeyFields;
|
||||
|
||||
public BucketIndexPartitioner(int bucketNum, String indexKeyFields) {
|
||||
this.bucketNum = bucketNum;
|
||||
this.indexKeyFields = indexKeyFields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int partition(HoodieKey key, int numPartitions) {
|
||||
int curBucket = BucketIdentifier.getBucketId(key, indexKeyFields, bucketNum);
|
||||
return BucketIdentifier.mod(curBucket, numPartitions);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,119 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.partitioner.profile;
|
||||
|
||||
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.view.SyncableFileSystemView;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.action.commit.SmallFile;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* WriteProfile for MERGE_ON_READ table type, this allows auto correction of small parquet files to larger ones
|
||||
* without the need for an index in the logFile.
|
||||
*
|
||||
* <p>Note: assumes the index can always index log files for Flink write.
|
||||
*/
|
||||
public class DeltaWriteProfile extends WriteProfile {
|
||||
public DeltaWriteProfile(HoodieWriteConfig config, HoodieFlinkEngineContext context) {
|
||||
super(config, context);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<SmallFile> smallFilesProfile(String partitionPath) {
|
||||
// smallFiles only for partitionPath
|
||||
List<SmallFile> smallFileLocations = new ArrayList<>();
|
||||
|
||||
// Init here since this class (and member variables) might not have been initialized
|
||||
HoodieTimeline commitTimeline = metaClient.getCommitsTimeline().filterCompletedInstants();
|
||||
|
||||
// Find out all eligible small file slices
|
||||
if (!commitTimeline.empty()) {
|
||||
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
|
||||
// find the smallest file in partition and append to it
|
||||
List<FileSlice> allSmallFileSlices = new ArrayList<>();
|
||||
// If we can index log files, we can add more inserts to log files for fileIds including those under
|
||||
// pending compaction.
|
||||
List<FileSlice> allFileSlices = fsView.getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp(), false)
|
||||
.collect(Collectors.toList());
|
||||
for (FileSlice fileSlice : allFileSlices) {
|
||||
if (isSmallFile(fileSlice)) {
|
||||
allSmallFileSlices.add(fileSlice);
|
||||
}
|
||||
}
|
||||
// Create SmallFiles from the eligible file slices
|
||||
for (FileSlice smallFileSlice : allSmallFileSlices) {
|
||||
SmallFile sf = new SmallFile();
|
||||
if (smallFileSlice.getBaseFile().isPresent()) {
|
||||
// TODO : Move logic of file name, file id, base commit time handling inside file slice
|
||||
String filename = smallFileSlice.getBaseFile().get().getFileName();
|
||||
sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename), FSUtils.getFileId(filename));
|
||||
sf.sizeBytes = getTotalFileSize(smallFileSlice);
|
||||
smallFileLocations.add(sf);
|
||||
} else {
|
||||
smallFileSlice.getLogFiles().findFirst().ifPresent(logFile -> {
|
||||
// in case there is something error, and the file slice has no log file
|
||||
sf.location = new HoodieRecordLocation(FSUtils.getBaseCommitTimeFromLogPath(logFile.getPath()),
|
||||
FSUtils.getFileIdFromLogPath(logFile.getPath()));
|
||||
sf.sizeBytes = getTotalFileSize(smallFileSlice);
|
||||
smallFileLocations.add(sf);
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
return smallFileLocations;
|
||||
}
|
||||
|
||||
protected SyncableFileSystemView getFileSystemView() {
|
||||
return (SyncableFileSystemView) getTable().getSliceView();
|
||||
}
|
||||
|
||||
private long getTotalFileSize(FileSlice fileSlice) {
|
||||
if (!fileSlice.getBaseFile().isPresent()) {
|
||||
return convertLogFilesSizeToExpectedParquetSize(fileSlice.getLogFiles().collect(Collectors.toList()));
|
||||
} else {
|
||||
return fileSlice.getBaseFile().get().getFileSize()
|
||||
+ convertLogFilesSizeToExpectedParquetSize(fileSlice.getLogFiles().collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
|
||||
private boolean isSmallFile(FileSlice fileSlice) {
|
||||
long totalSize = getTotalFileSize(fileSlice);
|
||||
return totalSize < config.getParquetMaxFileSize();
|
||||
}
|
||||
|
||||
// TODO (NA) : Make this static part of utility
|
||||
public long convertLogFilesSizeToExpectedParquetSize(List<HoodieLogFile> hoodieLogFiles) {
|
||||
long totalSizeOfLogFiles = hoodieLogFiles.stream().map(HoodieLogFile::getFileSize)
|
||||
.filter(size -> size > 0).reduce(Long::sum).orElse(0L);
|
||||
// Here we assume that if there is no base parquet file, all log files contain only inserts.
|
||||
// We can then just get the parquet equivalent size of these log files, compare that with
|
||||
// {@link config.getParquetMaxFileSize()} and decide if there is scope to insert more rows
|
||||
return (long) (totalSizeOfLogFiles * config.getLogFileToParquetCompressionRatio());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,45 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.partitioner.profile;
|
||||
|
||||
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.action.commit.SmallFile;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* WriteProfile that always return empty small files.
|
||||
*
|
||||
* <p>This write profile is used for INSERT OVERWRITE and INSERT OVERWRITE TABLE operations,
|
||||
* the existing small files are ignored because of the 'OVERWRITE' semantics.
|
||||
*
|
||||
* <p>Note: assumes the index can always index log files for Flink write.
|
||||
*/
|
||||
public class EmptyWriteProfile extends WriteProfile {
|
||||
public EmptyWriteProfile(HoodieWriteConfig config, HoodieFlinkEngineContext context) {
|
||||
super(config, context);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<SmallFile> smallFilesProfile(String partitionPath) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,270 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.partitioner.profile;
|
||||
|
||||
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.view.SyncableFileSystemView;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.sink.partitioner.BucketAssigner;
|
||||
import org.apache.hudi.table.HoodieFlinkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.commit.SmallFile;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.annotation.VisibleForTesting;
|
||||
import org.apache.flink.core.fs.Path;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* Profiling of write statistics for {@link BucketAssigner},
|
||||
* such as the average record size and small files.
|
||||
*
|
||||
* <p>The profile is re-constructed when there are new commits on the timeline.
|
||||
*/
|
||||
public class WriteProfile {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(WriteProfile.class);
|
||||
|
||||
/**
|
||||
* The write config.
|
||||
*/
|
||||
protected final HoodieWriteConfig config;
|
||||
|
||||
/**
|
||||
* Table base path.
|
||||
*/
|
||||
private final Path basePath;
|
||||
|
||||
/**
|
||||
* The meta client.
|
||||
*/
|
||||
protected final HoodieTableMetaClient metaClient;
|
||||
|
||||
/**
|
||||
* The average record size.
|
||||
*/
|
||||
private long avgSize = -1L;
|
||||
|
||||
/**
|
||||
* Total records to write for each bucket based on
|
||||
* the config option {@link org.apache.hudi.config.HoodieStorageConfig#PARQUET_MAX_FILE_SIZE}.
|
||||
*/
|
||||
private long recordsPerBucket;
|
||||
|
||||
/**
|
||||
* Partition path to small files mapping.
|
||||
*/
|
||||
private final Map<String, List<SmallFile>> smallFilesMap;
|
||||
|
||||
/**
|
||||
* Checkpoint id to avoid redundant reload.
|
||||
*/
|
||||
private long reloadedCheckpointId;
|
||||
|
||||
/**
|
||||
* The file system view cache for one checkpoint interval.
|
||||
*/
|
||||
protected SyncableFileSystemView fsView;
|
||||
|
||||
/**
|
||||
* Metadata cache to reduce IO of metadata files.
|
||||
*/
|
||||
private final Map<String, HoodieCommitMetadata> metadataCache;
|
||||
|
||||
/**
|
||||
* The engine context.
|
||||
*/
|
||||
private final HoodieFlinkEngineContext context;
|
||||
|
||||
public WriteProfile(HoodieWriteConfig config, HoodieFlinkEngineContext context) {
|
||||
this.config = config;
|
||||
this.context = context;
|
||||
this.basePath = new Path(config.getBasePath());
|
||||
this.smallFilesMap = new HashMap<>();
|
||||
this.recordsPerBucket = config.getCopyOnWriteInsertSplitSize();
|
||||
this.metaClient = StreamerUtil.createMetaClient(config.getBasePath(), context.getHadoopConf().get());
|
||||
this.metadataCache = new HashMap<>();
|
||||
this.fsView = getFileSystemView();
|
||||
// profile the record statistics on construction
|
||||
recordProfile();
|
||||
}
|
||||
|
||||
public long getAvgSize() {
|
||||
return avgSize;
|
||||
}
|
||||
|
||||
public long getRecordsPerBucket() {
|
||||
return recordsPerBucket;
|
||||
}
|
||||
|
||||
public HoodieTableMetaClient getMetaClient() {
|
||||
return this.metaClient;
|
||||
}
|
||||
|
||||
protected HoodieTable<?, ?, ?, ?> getTable() {
|
||||
return HoodieFlinkTable.create(config, context);
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtains the average record size based on records written during previous commits. Used for estimating how many
|
||||
* records pack into one file.
|
||||
*/
|
||||
private long averageBytesPerRecord() {
|
||||
long avgSize = config.getCopyOnWriteRecordSizeEstimate();
|
||||
long fileSizeThreshold = (long) (config.getRecordSizeEstimationThreshold() * config.getParquetSmallFileLimit());
|
||||
HoodieTimeline commitTimeline = metaClient.getCommitsTimeline().filterCompletedInstants();
|
||||
if (!commitTimeline.empty()) {
|
||||
// Go over the reverse ordered commits to get a more recent estimate of average record size.
|
||||
Iterator<HoodieInstant> instants = commitTimeline.getReverseOrderedInstants().iterator();
|
||||
while (instants.hasNext()) {
|
||||
HoodieInstant instant = instants.next();
|
||||
final HoodieCommitMetadata commitMetadata =
|
||||
this.metadataCache.computeIfAbsent(
|
||||
instant.getTimestamp(),
|
||||
k -> WriteProfiles.getCommitMetadataSafely(config.getTableName(), basePath, instant, commitTimeline)
|
||||
.orElse(null));
|
||||
if (commitMetadata == null) {
|
||||
continue;
|
||||
}
|
||||
long totalBytesWritten = commitMetadata.fetchTotalBytesWritten();
|
||||
long totalRecordsWritten = commitMetadata.fetchTotalRecordsWritten();
|
||||
if (totalBytesWritten > fileSizeThreshold && totalRecordsWritten > 0) {
|
||||
avgSize = (long) Math.ceil((1.0 * totalBytesWritten) / totalRecordsWritten);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
LOG.info("Refresh average bytes per record => " + avgSize);
|
||||
return avgSize;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a list of small files in the given partition path.
|
||||
*
|
||||
* <p>Note: This method should be thread safe.
|
||||
*/
|
||||
public synchronized List<SmallFile> getSmallFiles(String partitionPath) {
|
||||
// lookup the cache first
|
||||
if (smallFilesMap.containsKey(partitionPath)) {
|
||||
return smallFilesMap.get(partitionPath);
|
||||
}
|
||||
|
||||
List<SmallFile> smallFiles = new ArrayList<>();
|
||||
if (config.getParquetSmallFileLimit() <= 0) {
|
||||
this.smallFilesMap.put(partitionPath, smallFiles);
|
||||
return smallFiles;
|
||||
}
|
||||
|
||||
smallFiles = smallFilesProfile(partitionPath);
|
||||
this.smallFilesMap.put(partitionPath, smallFiles);
|
||||
return smallFiles;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a list of small files in the given partition path from the latest filesystem view.
|
||||
*/
|
||||
protected List<SmallFile> smallFilesProfile(String partitionPath) {
|
||||
// smallFiles only for partitionPath
|
||||
List<SmallFile> smallFileLocations = new ArrayList<>();
|
||||
|
||||
HoodieTimeline commitTimeline = metaClient.getCommitsTimeline().filterCompletedInstants();
|
||||
|
||||
if (!commitTimeline.empty()) { // if we have some commits
|
||||
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
|
||||
List<HoodieBaseFile> allFiles = fsView
|
||||
.getLatestBaseFilesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).collect(Collectors.toList());
|
||||
|
||||
for (HoodieBaseFile file : allFiles) {
|
||||
// filter out the corrupted files.
|
||||
if (file.getFileSize() < config.getParquetSmallFileLimit() && file.getFileSize() > 0) {
|
||||
String filename = file.getFileName();
|
||||
SmallFile sf = new SmallFile();
|
||||
sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename), FSUtils.getFileId(filename));
|
||||
sf.sizeBytes = file.getFileSize();
|
||||
smallFileLocations.add(sf);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return smallFileLocations;
|
||||
}
|
||||
|
||||
protected SyncableFileSystemView getFileSystemView() {
|
||||
return (SyncableFileSystemView) getTable().getBaseFileOnlyView();
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove the overdue metadata from the cache
|
||||
* whose instant does not belong to the given instants {@code instants}.
|
||||
*/
|
||||
private void cleanMetadataCache(Stream<HoodieInstant> instants) {
|
||||
Set<String> timestampSet = instants.map(HoodieInstant::getTimestamp).collect(Collectors.toSet());
|
||||
this.metadataCache.keySet().retainAll(timestampSet);
|
||||
}
|
||||
|
||||
private void recordProfile() {
|
||||
this.avgSize = averageBytesPerRecord();
|
||||
if (config.shouldAllowMultiWriteOnSameInstant()) {
|
||||
this.recordsPerBucket = config.getParquetMaxFileSize() / avgSize;
|
||||
LOG.info("Refresh insert records per bucket => " + recordsPerBucket);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Reload the write profile, should do once for each checkpoint.
|
||||
*
|
||||
* <p>We do these things: i). reload the timeline; ii). re-construct the record profile;
|
||||
* iii) clean the small files cache.
|
||||
*
|
||||
* <p>Note: This method should be thread safe.
|
||||
*/
|
||||
public synchronized void reload(long checkpointId) {
|
||||
if (this.reloadedCheckpointId >= checkpointId) {
|
||||
// already reloaded
|
||||
return;
|
||||
}
|
||||
this.metaClient.reloadActiveTimeline();
|
||||
this.fsView.sync();
|
||||
recordProfile();
|
||||
cleanMetadataCache(this.metaClient.getCommitsTimeline().filterCompletedInstants().getInstants());
|
||||
this.smallFilesMap.clear();
|
||||
this.reloadedCheckpointId = checkpointId;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public Map<String, HoodieCommitMetadata> getMetadataCache() {
|
||||
return this.metadataCache;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,201 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.partitioner.profile;
|
||||
|
||||
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.core.fs.Path;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Factory for {@link WriteProfile}.
|
||||
*/
|
||||
public class WriteProfiles {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(WriteProfiles.class);
|
||||
|
||||
private static final Map<String, WriteProfile> PROFILES = new HashMap<>();
|
||||
|
||||
private WriteProfiles() {
|
||||
}
|
||||
|
||||
public static synchronized WriteProfile singleton(
|
||||
boolean ignoreSmallFiles,
|
||||
boolean delta,
|
||||
HoodieWriteConfig config,
|
||||
HoodieFlinkEngineContext context) {
|
||||
return PROFILES.computeIfAbsent(config.getBasePath(),
|
||||
k -> getWriteProfile(ignoreSmallFiles, delta, config, context));
|
||||
}
|
||||
|
||||
private static WriteProfile getWriteProfile(
|
||||
boolean ignoreSmallFiles,
|
||||
boolean delta,
|
||||
HoodieWriteConfig config,
|
||||
HoodieFlinkEngineContext context) {
|
||||
if (ignoreSmallFiles) {
|
||||
return new EmptyWriteProfile(config, context);
|
||||
} else if (delta) {
|
||||
return new DeltaWriteProfile(config, context);
|
||||
} else {
|
||||
return new WriteProfile(config, context);
|
||||
}
|
||||
}
|
||||
|
||||
public static void clean(String path) {
|
||||
PROFILES.remove(path);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all the incremental write file statuses with the given commits metadata.
|
||||
*
|
||||
* @param basePath Table base path
|
||||
* @param hadoopConf The hadoop conf
|
||||
* @param metadataList The commits metadata
|
||||
* @param tableType The table type
|
||||
* @return the file status array
|
||||
*/
|
||||
public static FileStatus[] getWritePathsOfInstants(
|
||||
Path basePath,
|
||||
Configuration hadoopConf,
|
||||
List<HoodieCommitMetadata> metadataList,
|
||||
HoodieTableType tableType) {
|
||||
FileSystem fs = FSUtils.getFs(basePath.toString(), hadoopConf);
|
||||
Map<String, FileStatus> uniqueIdToFileStatus = new HashMap<>();
|
||||
metadataList.forEach(metadata ->
|
||||
uniqueIdToFileStatus.putAll(getFilesToReadOfInstant(basePath, metadata, fs, tableType)));
|
||||
return uniqueIdToFileStatus.values().toArray(new FileStatus[0]);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the commit file status info with given metadata.
|
||||
*
|
||||
* @param basePath Table base path
|
||||
* @param metadata The metadata
|
||||
* @param fs The filesystem
|
||||
* @param tableType The table type
|
||||
* @return the commit file status info grouping by specific ID
|
||||
*/
|
||||
private static Map<String, FileStatus> getFilesToReadOfInstant(
|
||||
Path basePath,
|
||||
HoodieCommitMetadata metadata,
|
||||
FileSystem fs,
|
||||
HoodieTableType tableType) {
|
||||
return getFilesToRead(fs.getConf(), metadata, basePath.toString(), tableType).entrySet().stream()
|
||||
// filter out the file paths that does not exist, some files may be cleaned by
|
||||
// the cleaner.
|
||||
.filter(entry -> {
|
||||
try {
|
||||
return fs.exists(entry.getValue().getPath());
|
||||
} catch (IOException e) {
|
||||
LOG.error("Checking exists of path: {} error", entry.getValue().getPath());
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
})
|
||||
.filter(entry -> StreamerUtil.isValidFile(entry.getValue()))
|
||||
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
|
||||
}
|
||||
|
||||
private static Map<String, FileStatus> getFilesToRead(
|
||||
Configuration hadoopConf,
|
||||
HoodieCommitMetadata metadata,
|
||||
String basePath,
|
||||
HoodieTableType tableType
|
||||
) {
|
||||
switch (tableType) {
|
||||
case COPY_ON_WRITE:
|
||||
return metadata.getFileIdToFileStatus(hadoopConf, basePath);
|
||||
case MERGE_ON_READ:
|
||||
return metadata.getFullPathToFileStatus(hadoopConf, basePath);
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the commit metadata of the given instant safely.
|
||||
*
|
||||
* @param tableName The table name
|
||||
* @param basePath The table base path
|
||||
* @param instant The hoodie instant
|
||||
* @param timeline The timeline
|
||||
* @return the commit metadata or empty if any error occurs
|
||||
*/
|
||||
public static Option<HoodieCommitMetadata> getCommitMetadataSafely(
|
||||
String tableName,
|
||||
Path basePath,
|
||||
HoodieInstant instant,
|
||||
HoodieTimeline timeline) {
|
||||
try {
|
||||
byte[] data = timeline.getInstantDetails(instant).get();
|
||||
return Option.of(HoodieCommitMetadata.fromBytes(data, HoodieCommitMetadata.class));
|
||||
} catch (FileNotFoundException fe) {
|
||||
// make this fail safe.
|
||||
LOG.warn("Instant {} was deleted by the cleaner, ignore", instant.getTimestamp());
|
||||
return Option.empty();
|
||||
} catch (Throwable throwable) {
|
||||
LOG.error("Get write metadata for table {} with instant {} and path: {} error",
|
||||
tableName, instant.getTimestamp(), basePath);
|
||||
return Option.empty();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the commit metadata of the given instant.
|
||||
*
|
||||
* @param tableName The table name
|
||||
* @param basePath The table base path
|
||||
* @param instant The hoodie instant
|
||||
* @param timeline The timeline
|
||||
* @return the commit metadata
|
||||
*/
|
||||
public static HoodieCommitMetadata getCommitMetadata(
|
||||
String tableName,
|
||||
Path basePath,
|
||||
HoodieInstant instant,
|
||||
HoodieTimeline timeline) {
|
||||
try {
|
||||
return HoodieInputFormatUtils.getCommitMetadata(instant, timeline);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Get write metadata for table {} with instant {} and path: {} error",
|
||||
tableName, instant.getTimestamp(), basePath);
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,51 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.transform;
|
||||
|
||||
import org.apache.flink.streaming.api.datastream.DataStream;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* A {@link Transformer} to chain other {@link Transformer}s and apply sequentially.
|
||||
*/
|
||||
public class ChainedTransformer implements Transformer {
|
||||
|
||||
private List<Transformer> transformers;
|
||||
|
||||
public ChainedTransformer(List<Transformer> transformers) {
|
||||
this.transformers = transformers;
|
||||
}
|
||||
|
||||
public List<String> getTransformersNames() {
|
||||
return transformers.stream().map(t -> t.getClass().getName()).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataStream<RowData> apply(DataStream<RowData> source) {
|
||||
DataStream<RowData> dataStream = source;
|
||||
for (Transformer t : transformers) {
|
||||
dataStream = t.apply(dataStream);
|
||||
}
|
||||
|
||||
return dataStream;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,116 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.transform;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieOperation;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.keygen.KeyGenerator;
|
||||
import org.apache.hudi.keygen.factory.HoodieAvroKeyGeneratorFactory;
|
||||
import org.apache.hudi.sink.utils.PayloadCreation;
|
||||
import org.apache.hudi.util.RowDataToAvroConverters;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.flink.api.common.functions.RichMapFunction;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.apache.hudi.util.StreamerUtil.flinkConf2TypedProperties;
|
||||
|
||||
/**
|
||||
* Function that transforms RowData to HoodieRecord.
|
||||
*/
|
||||
public class RowDataToHoodieFunction<I extends RowData, O extends HoodieRecord>
|
||||
extends RichMapFunction<I, O> {
|
||||
/**
|
||||
* Row type of the input.
|
||||
*/
|
||||
private final RowType rowType;
|
||||
|
||||
/**
|
||||
* Avro schema of the input.
|
||||
*/
|
||||
private transient Schema avroSchema;
|
||||
|
||||
/**
|
||||
* RowData to Avro record converter.
|
||||
*/
|
||||
private transient RowDataToAvroConverters.RowDataToAvroConverter converter;
|
||||
|
||||
/**
|
||||
* HoodieKey generator.
|
||||
*/
|
||||
private transient KeyGenerator keyGenerator;
|
||||
|
||||
/**
|
||||
* Utilities to create hoodie pay load instance.
|
||||
*/
|
||||
private transient PayloadCreation payloadCreation;
|
||||
|
||||
/**
|
||||
* Config options.
|
||||
*/
|
||||
private final Configuration config;
|
||||
|
||||
public RowDataToHoodieFunction(RowType rowType, Configuration config) {
|
||||
this.rowType = rowType;
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(Configuration parameters) throws Exception {
|
||||
super.open(parameters);
|
||||
this.avroSchema = StreamerUtil.getSourceSchema(this.config);
|
||||
this.converter = RowDataToAvroConverters.createConverter(this.rowType);
|
||||
this.keyGenerator =
|
||||
HoodieAvroKeyGeneratorFactory
|
||||
.createKeyGenerator(flinkConf2TypedProperties(this.config));
|
||||
this.payloadCreation = PayloadCreation.instance(config);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public O map(I i) throws Exception {
|
||||
return (O) toHoodieRecord(i);
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts the give record to a {@link HoodieRecord}.
|
||||
*
|
||||
* @param record The input record
|
||||
* @return HoodieRecord based on the configuration
|
||||
* @throws IOException if error occurs
|
||||
*/
|
||||
@SuppressWarnings("rawtypes")
|
||||
private HoodieRecord toHoodieRecord(I record) throws Exception {
|
||||
GenericRecord gr = (GenericRecord) this.converter.convert(this.avroSchema, record);
|
||||
final HoodieKey hoodieKey = keyGenerator.getKey(gr);
|
||||
|
||||
HoodieRecordPayload payload = payloadCreation.createPayload(gr);
|
||||
HoodieOperation operation = HoodieOperation.fromValue(record.getRowKind().toByteValue());
|
||||
return new HoodieAvroRecord<>(hoodieKey, payload, operation);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,61 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.transform;
|
||||
|
||||
import org.apache.hudi.adapter.RateLimiterAdapter;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
|
||||
/**
|
||||
* Function that transforms RowData to a HoodieRecord with RateLimit.
|
||||
*/
|
||||
public class RowDataToHoodieFunctionWithRateLimit<I extends RowData, O extends HoodieRecord>
|
||||
extends RowDataToHoodieFunction<I, O> {
|
||||
/**
|
||||
* Total rate limit per second for this job.
|
||||
*/
|
||||
private final double totalLimit;
|
||||
|
||||
/**
|
||||
* Rate limit per second for per task.
|
||||
*/
|
||||
private transient RateLimiterAdapter rateLimiter;
|
||||
|
||||
public RowDataToHoodieFunctionWithRateLimit(RowType rowType, Configuration config) {
|
||||
super(rowType, config);
|
||||
this.totalLimit = config.getLong(FlinkOptions.WRITE_RATE_LIMIT);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(Configuration parameters) throws Exception {
|
||||
super.open(parameters);
|
||||
this.rateLimiter =
|
||||
RateLimiterAdapter.create(totalLimit / getRuntimeContext().getNumberOfParallelSubtasks());
|
||||
}
|
||||
|
||||
@Override
|
||||
public O map(I i) throws Exception {
|
||||
rateLimiter.acquire();
|
||||
return super.map(i);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,46 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.transform;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
|
||||
/**
|
||||
* Utilities for {@link RowDataToHoodieFunction}.
|
||||
*/
|
||||
public abstract class RowDataToHoodieFunctions {
|
||||
private RowDataToHoodieFunctions() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a {@link RowDataToHoodieFunction} instance based on the given configuration.
|
||||
*/
|
||||
@SuppressWarnings("rawtypes")
|
||||
public static RowDataToHoodieFunction<RowData, HoodieRecord> create(RowType rowType, Configuration conf) {
|
||||
if (conf.getLong(FlinkOptions.WRITE_RATE_LIMIT) > 0) {
|
||||
return new RowDataToHoodieFunctionWithRateLimit<>(rowType, conf);
|
||||
} else {
|
||||
return new RowDataToHoodieFunction<>(rowType, conf);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,36 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.transform;
|
||||
|
||||
import org.apache.flink.streaming.api.datastream.DataStream;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
|
||||
/**
|
||||
* Transform source stream to target stream before writing.
|
||||
*/
|
||||
public interface Transformer {
|
||||
|
||||
/**
|
||||
* Transform source DataStream to target DataStream.
|
||||
*
|
||||
* @param source
|
||||
*/
|
||||
DataStream<RowData> apply(DataStream<RowData> source);
|
||||
|
||||
}
|
||||
@@ -0,0 +1,93 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.utils;
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.hive.HiveSyncConfig;
|
||||
import org.apache.hudi.hive.HiveSyncTool;
|
||||
import org.apache.hudi.table.format.FilePathUtils;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
/**
|
||||
* Hive synchronization context.
|
||||
*
|
||||
* <p>Use this context to create the {@link HiveSyncTool} for synchronization.
|
||||
*/
|
||||
public class HiveSyncContext {
|
||||
private final HiveSyncConfig syncConfig;
|
||||
private final HiveConf hiveConf;
|
||||
private final FileSystem fs;
|
||||
|
||||
private HiveSyncContext(HiveSyncConfig syncConfig, HiveConf hiveConf, FileSystem fs) {
|
||||
this.syncConfig = syncConfig;
|
||||
this.hiveConf = hiveConf;
|
||||
this.fs = fs;
|
||||
}
|
||||
|
||||
public HiveSyncTool hiveSyncTool() {
|
||||
return new HiveSyncTool(this.syncConfig, this.hiveConf, this.fs);
|
||||
}
|
||||
|
||||
public static HiveSyncContext create(Configuration conf) {
|
||||
HiveSyncConfig syncConfig = buildSyncConfig(conf);
|
||||
org.apache.hadoop.conf.Configuration hadoopConf = StreamerUtil.getHadoopConf();
|
||||
String path = conf.getString(FlinkOptions.PATH);
|
||||
FileSystem fs = FSUtils.getFs(path, hadoopConf);
|
||||
HiveConf hiveConf = new HiveConf();
|
||||
if (!FlinkOptions.isDefaultValueDefined(conf, FlinkOptions.HIVE_SYNC_METASTORE_URIS)) {
|
||||
hadoopConf.set(HiveConf.ConfVars.METASTOREURIS.varname, conf.getString(FlinkOptions.HIVE_SYNC_METASTORE_URIS));
|
||||
}
|
||||
hiveConf.addResource(hadoopConf);
|
||||
return new HiveSyncContext(syncConfig, hiveConf, fs);
|
||||
}
|
||||
|
||||
private static HiveSyncConfig buildSyncConfig(Configuration conf) {
|
||||
HiveSyncConfig hiveSyncConfig = new HiveSyncConfig();
|
||||
hiveSyncConfig.basePath = conf.getString(FlinkOptions.PATH);
|
||||
hiveSyncConfig.baseFileFormat = conf.getString(FlinkOptions.HIVE_SYNC_FILE_FORMAT);
|
||||
hiveSyncConfig.usePreApacheInputFormat = false;
|
||||
hiveSyncConfig.databaseName = conf.getString(FlinkOptions.HIVE_SYNC_DB);
|
||||
hiveSyncConfig.tableName = conf.getString(FlinkOptions.HIVE_SYNC_TABLE);
|
||||
hiveSyncConfig.syncMode = conf.getString(FlinkOptions.HIVE_SYNC_MODE);
|
||||
hiveSyncConfig.hiveUser = conf.getString(FlinkOptions.HIVE_SYNC_USERNAME);
|
||||
hiveSyncConfig.hivePass = conf.getString(FlinkOptions.HIVE_SYNC_PASSWORD);
|
||||
hiveSyncConfig.tableProperties = conf.getString(FlinkOptions.HIVE_SYNC_TABLE_PROPERTIES);
|
||||
hiveSyncConfig.serdeProperties = conf.getString(FlinkOptions.HIVE_SYNC_TABLE_SERDE_PROPERTIES);
|
||||
hiveSyncConfig.jdbcUrl = conf.getString(FlinkOptions.HIVE_SYNC_JDBC_URL);
|
||||
hiveSyncConfig.partitionFields = Arrays.asList(FilePathUtils.extractPartitionKeys(conf));
|
||||
hiveSyncConfig.partitionValueExtractorClass = conf.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME);
|
||||
hiveSyncConfig.useJdbc = conf.getBoolean(FlinkOptions.HIVE_SYNC_USE_JDBC);
|
||||
hiveSyncConfig.useFileListingFromMetadata = conf.getBoolean(FlinkOptions.METADATA_ENABLED);
|
||||
hiveSyncConfig.ignoreExceptions = conf.getBoolean(FlinkOptions.HIVE_SYNC_IGNORE_EXCEPTIONS);
|
||||
hiveSyncConfig.supportTimestamp = conf.getBoolean(FlinkOptions.HIVE_SYNC_SUPPORT_TIMESTAMP);
|
||||
hiveSyncConfig.autoCreateDatabase = conf.getBoolean(FlinkOptions.HIVE_SYNC_AUTO_CREATE_DB);
|
||||
hiveSyncConfig.decodePartition = conf.getBoolean(FlinkOptions.URL_ENCODE_PARTITIONING);
|
||||
hiveSyncConfig.skipROSuffix = conf.getBoolean(FlinkOptions.HIVE_SYNC_SKIP_RO_SUFFIX);
|
||||
hiveSyncConfig.assumeDatePartitioning = conf.getBoolean(FlinkOptions.HIVE_SYNC_ASSUME_DATE_PARTITION);
|
||||
hiveSyncConfig.withOperationField = conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED);
|
||||
return hiveSyncConfig;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,156 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.utils;
|
||||
|
||||
import org.apache.flink.annotation.VisibleForTesting;
|
||||
import org.apache.flink.util.ExceptionUtils;
|
||||
import org.apache.flink.util.function.ThrowingRunnable;
|
||||
import org.slf4j.Logger;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
* An executor service that catches all the throwable with logging.
|
||||
*
|
||||
* <p>A post-exception hook {@link ExceptionHook} can be defined on construction
|
||||
* or on each execution.
|
||||
*/
|
||||
public class NonThrownExecutor implements AutoCloseable {
|
||||
private final Logger logger;
|
||||
|
||||
/**
|
||||
* A single-thread executor to handle all the asynchronous jobs.
|
||||
*/
|
||||
private final ExecutorService executor;
|
||||
|
||||
/**
|
||||
* Exception hook for post-exception handling.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
protected final ExceptionHook exceptionHook;
|
||||
|
||||
/**
|
||||
* Flag saying whether to wait for the tasks finish on #close.
|
||||
*/
|
||||
private final boolean waitForTasksFinish;
|
||||
|
||||
@VisibleForTesting
|
||||
protected NonThrownExecutor(Logger logger, @Nullable ExceptionHook exceptionHook, boolean waitForTasksFinish) {
|
||||
this.executor = Executors.newSingleThreadExecutor();
|
||||
this.logger = logger;
|
||||
this.exceptionHook = exceptionHook;
|
||||
this.waitForTasksFinish = waitForTasksFinish;
|
||||
}
|
||||
|
||||
public static Builder builder(Logger logger) {
|
||||
return new Builder(logger);
|
||||
}
|
||||
|
||||
/**
|
||||
* Run the action in a loop.
|
||||
*/
|
||||
public void execute(
|
||||
final ThrowingRunnable<Throwable> action,
|
||||
final String actionName,
|
||||
final Object... actionParams) {
|
||||
execute(action, this.exceptionHook, actionName, actionParams);
|
||||
}
|
||||
|
||||
/**
|
||||
* Run the action in a loop.
|
||||
*/
|
||||
public void execute(
|
||||
final ThrowingRunnable<Throwable> action,
|
||||
final ExceptionHook hook,
|
||||
final String actionName,
|
||||
final Object... actionParams) {
|
||||
|
||||
executor.execute(
|
||||
() -> {
|
||||
final String actionString = String.format(actionName, actionParams);
|
||||
try {
|
||||
action.run();
|
||||
logger.info("Executor executes action [{}] success!", actionString);
|
||||
} catch (Throwable t) {
|
||||
// if we have a JVM critical error, promote it immediately, there is a good
|
||||
// chance the
|
||||
// logging or job failing will not succeed any more
|
||||
ExceptionUtils.rethrowIfFatalErrorOrOOM(t);
|
||||
final String errMsg = String.format("Executor executes action [%s] error", actionString);
|
||||
logger.error(errMsg, t);
|
||||
if (hook != null) {
|
||||
hook.apply(errMsg, t);
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws Exception {
|
||||
if (executor != null) {
|
||||
if (waitForTasksFinish) {
|
||||
executor.shutdown();
|
||||
} else {
|
||||
executor.shutdownNow();
|
||||
}
|
||||
// We do not expect this to actually block for long. At this point, there should
|
||||
// be very few task running in the executor, if any.
|
||||
executor.awaitTermination(Long.MAX_VALUE, TimeUnit.SECONDS);
|
||||
}
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Inner Class
|
||||
// -------------------------------------------------------------------------
|
||||
public interface ExceptionHook {
|
||||
void apply(String errMsg, Throwable t);
|
||||
}
|
||||
|
||||
/**
|
||||
* Builder for {@link NonThrownExecutor}.
|
||||
*/
|
||||
public static class Builder {
|
||||
private final Logger logger;
|
||||
private ExceptionHook exceptionHook;
|
||||
private boolean waitForTasksFinish = false;
|
||||
|
||||
private Builder(Logger logger) {
|
||||
this.logger = Objects.requireNonNull(logger);
|
||||
}
|
||||
|
||||
public NonThrownExecutor build() {
|
||||
return new NonThrownExecutor(logger, exceptionHook, waitForTasksFinish);
|
||||
}
|
||||
|
||||
public Builder exceptionHook(ExceptionHook exceptionHook) {
|
||||
this.exceptionHook = exceptionHook;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder waitForTasksFinish(boolean waitForTasksFinish) {
|
||||
this.waitForTasksFinish = waitForTasksFinish;
|
||||
return this;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,94 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.utils;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.model.BaseAvroPayload;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.configuration.OptionsResolver;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.lang.reflect.Constructor;
|
||||
|
||||
/**
|
||||
* Util to create hoodie pay load instance.
|
||||
*/
|
||||
public class PayloadCreation implements Serializable {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
private final boolean shouldCombine;
|
||||
private final Constructor<?> constructor;
|
||||
private final String preCombineField;
|
||||
|
||||
private PayloadCreation(
|
||||
boolean shouldCombine,
|
||||
Constructor<?> constructor,
|
||||
@Nullable String preCombineField) {
|
||||
this.shouldCombine = shouldCombine;
|
||||
this.constructor = constructor;
|
||||
this.preCombineField = preCombineField;
|
||||
}
|
||||
|
||||
public static PayloadCreation instance(Configuration conf) throws Exception {
|
||||
String preCombineField = OptionsResolver.getPreCombineField(conf);
|
||||
boolean needCombine = conf.getBoolean(FlinkOptions.PRE_COMBINE)
|
||||
|| WriteOperationType.fromValue(conf.getString(FlinkOptions.OPERATION)) == WriteOperationType.UPSERT;
|
||||
boolean shouldCombine = needCombine && preCombineField != null;
|
||||
|
||||
final Class<?>[] argTypes;
|
||||
final Constructor<?> constructor;
|
||||
if (shouldCombine) {
|
||||
argTypes = new Class<?>[] {GenericRecord.class, Comparable.class};
|
||||
} else {
|
||||
argTypes = new Class<?>[] {Option.class};
|
||||
}
|
||||
final String clazz = conf.getString(FlinkOptions.PAYLOAD_CLASS_NAME);
|
||||
constructor = ReflectionUtils.getClass(clazz).getConstructor(argTypes);
|
||||
return new PayloadCreation(shouldCombine, constructor, preCombineField);
|
||||
}
|
||||
|
||||
public HoodieRecordPayload<?> createPayload(GenericRecord record) throws Exception {
|
||||
if (shouldCombine) {
|
||||
ValidationUtils.checkState(preCombineField != null);
|
||||
Comparable<?> orderingVal = (Comparable<?>) HoodieAvroUtils.getNestedFieldVal(record,
|
||||
preCombineField, false, false);
|
||||
return (HoodieRecordPayload<?>) constructor.newInstance(record, orderingVal);
|
||||
} else {
|
||||
return (HoodieRecordPayload<?>) this.constructor.newInstance(Option.of(record));
|
||||
}
|
||||
}
|
||||
|
||||
public HoodieRecordPayload<?> createDeletePayload(BaseAvroPayload payload) throws Exception {
|
||||
if (shouldCombine) {
|
||||
return (HoodieRecordPayload<?>) constructor.newInstance(null, payload.orderingVal);
|
||||
} else {
|
||||
return (HoodieRecordPayload<?>) this.constructor.newInstance(Option.empty());
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,353 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.utils;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.configuration.OptionsResolver;
|
||||
import org.apache.hudi.sink.BucketStreamWriteOperator;
|
||||
import org.apache.hudi.sink.CleanFunction;
|
||||
import org.apache.hudi.sink.StreamWriteOperator;
|
||||
import org.apache.hudi.sink.append.AppendWriteOperator;
|
||||
import org.apache.hudi.sink.bootstrap.BootstrapOperator;
|
||||
import org.apache.hudi.sink.bootstrap.batch.BatchBootstrapOperator;
|
||||
import org.apache.hudi.sink.bulk.BulkInsertWriteOperator;
|
||||
import org.apache.hudi.sink.bulk.RowDataKeyGen;
|
||||
import org.apache.hudi.sink.bulk.sort.SortOperatorGen;
|
||||
import org.apache.hudi.sink.common.WriteOperatorFactory;
|
||||
import org.apache.hudi.sink.compact.CompactFunction;
|
||||
import org.apache.hudi.sink.compact.CompactionCommitEvent;
|
||||
import org.apache.hudi.sink.compact.CompactionCommitSink;
|
||||
import org.apache.hudi.sink.compact.CompactionPlanEvent;
|
||||
import org.apache.hudi.sink.compact.CompactionPlanOperator;
|
||||
import org.apache.hudi.sink.partitioner.BucketAssignFunction;
|
||||
import org.apache.hudi.sink.partitioner.BucketIndexPartitioner;
|
||||
import org.apache.hudi.sink.transform.RowDataToHoodieFunctions;
|
||||
import org.apache.hudi.table.format.FilePathUtils;
|
||||
|
||||
import org.apache.flink.api.common.functions.Partitioner;
|
||||
import org.apache.flink.api.common.typeinfo.TypeInformation;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
|
||||
import org.apache.flink.streaming.api.datastream.DataStream;
|
||||
import org.apache.flink.streaming.api.datastream.DataStreamSink;
|
||||
import org.apache.flink.streaming.api.functions.sink.SinkFunction;
|
||||
import org.apache.flink.streaming.api.graph.StreamGraphGenerator;
|
||||
import org.apache.flink.streaming.api.operators.KeyedProcessOperator;
|
||||
import org.apache.flink.streaming.api.operators.ProcessOperator;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
|
||||
/**
|
||||
* Utilities to generate all kinds of sub-pipelines.
|
||||
*/
|
||||
public class Pipelines {
|
||||
|
||||
/**
|
||||
* Bulk insert the input dataset at once.
|
||||
*
|
||||
* <p>By default, the input dataset would shuffle by the partition path first then
|
||||
* sort by the partition path before passing around to the write function.
|
||||
* The whole pipeline looks like the following:
|
||||
*
|
||||
* <pre>
|
||||
* | input1 | ===\ /=== |sorter| === | task1 | (p1, p2)
|
||||
* shuffle
|
||||
* | input2 | ===/ \=== |sorter| === | task2 | (p3, p4)
|
||||
*
|
||||
* Note: Both input1 and input2's dataset come from partitions: p1, p2, p3, p4
|
||||
* </pre>
|
||||
*
|
||||
* <p>The write task switches to new file handle each time it receives a record
|
||||
* from the different partition path, the shuffle and sort would reduce small files.
|
||||
*
|
||||
* <p>The bulk insert should be run in batch execution mode.
|
||||
*
|
||||
* @param conf The configuration
|
||||
* @param rowType The input row type
|
||||
* @param dataStream The input data stream
|
||||
* @return the bulk insert data stream sink
|
||||
*/
|
||||
public static DataStreamSink<Object> bulkInsert(Configuration conf, RowType rowType, DataStream<RowData> dataStream) {
|
||||
WriteOperatorFactory<RowData> operatorFactory = BulkInsertWriteOperator.getFactory(conf, rowType);
|
||||
|
||||
final String[] partitionFields = FilePathUtils.extractPartitionKeys(conf);
|
||||
if (partitionFields.length > 0) {
|
||||
RowDataKeyGen rowDataKeyGen = RowDataKeyGen.instance(conf, rowType);
|
||||
if (conf.getBoolean(FlinkOptions.WRITE_BULK_INSERT_SHUFFLE_BY_PARTITION)) {
|
||||
|
||||
// shuffle by partition keys
|
||||
// use #partitionCustom instead of #keyBy to avoid duplicate sort operations,
|
||||
// see BatchExecutionUtils#applyBatchExecutionSettings for details.
|
||||
Partitioner<String> partitioner = (key, channels) ->
|
||||
KeyGroupRangeAssignment.assignKeyToParallelOperator(key, StreamGraphGenerator.DEFAULT_LOWER_BOUND_MAX_PARALLELISM, channels);
|
||||
dataStream = dataStream.partitionCustom(partitioner, rowDataKeyGen::getPartitionPath);
|
||||
}
|
||||
if (conf.getBoolean(FlinkOptions.WRITE_BULK_INSERT_SORT_BY_PARTITION)) {
|
||||
SortOperatorGen sortOperatorGen = new SortOperatorGen(rowType, partitionFields);
|
||||
// sort by partition keys
|
||||
dataStream = dataStream
|
||||
.transform("partition_key_sorter",
|
||||
TypeInformation.of(RowData.class),
|
||||
sortOperatorGen.createSortOperator())
|
||||
.setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS));
|
||||
ExecNodeUtil.setManagedMemoryWeight(dataStream.getTransformation(),
|
||||
conf.getInteger(FlinkOptions.WRITE_SORT_MEMORY) * 1024L * 1024L);
|
||||
}
|
||||
}
|
||||
return dataStream
|
||||
.transform("hoodie_bulk_insert_write",
|
||||
TypeInformation.of(Object.class),
|
||||
operatorFactory)
|
||||
// follow the parallelism of upstream operators to avoid shuffle
|
||||
.setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS))
|
||||
.addSink(DummySink.INSTANCE)
|
||||
.name("dummy");
|
||||
}
|
||||
|
||||
/**
|
||||
* Insert the dataset with append mode(no upsert or deduplication).
|
||||
*
|
||||
* <p>The input dataset would be rebalanced among the write tasks:
|
||||
*
|
||||
* <pre>
|
||||
* | input1 | ===\ /=== | task1 | (p1, p2, p3, p4)
|
||||
* shuffle
|
||||
* | input2 | ===/ \=== | task2 | (p1, p2, p3, p4)
|
||||
*
|
||||
* Note: Both input1 and input2's dataset come from partitions: p1, p2, p3, p4
|
||||
* </pre>
|
||||
*
|
||||
* <p>The write task switches to new file handle each time it receives a record
|
||||
* from the different partition path, so there may be many small files.
|
||||
*
|
||||
* @param conf The configuration
|
||||
* @param rowType The input row type
|
||||
* @param dataStream The input data stream
|
||||
* @return the appending data stream sink
|
||||
*/
|
||||
public static DataStreamSink<Object> append(Configuration conf, RowType rowType, DataStream<RowData> dataStream) {
|
||||
WriteOperatorFactory<RowData> operatorFactory = AppendWriteOperator.getFactory(conf, rowType);
|
||||
|
||||
return dataStream
|
||||
.transform("hoodie_append_write", TypeInformation.of(Object.class), operatorFactory)
|
||||
.uid("uid_hoodie_stream_write" + conf.getString(FlinkOptions.TABLE_NAME))
|
||||
.setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS))
|
||||
.addSink(DummySink.INSTANCE)
|
||||
.name("dummy");
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructs bootstrap pipeline as streaming.
|
||||
* The bootstrap operator loads the existing data index (primary key to file id mapping),
|
||||
* then sends the indexing data set to subsequent operator(usually the bucket assign operator).
|
||||
*/
|
||||
public static DataStream<HoodieRecord> bootstrap(
|
||||
Configuration conf,
|
||||
RowType rowType,
|
||||
int defaultParallelism,
|
||||
DataStream<RowData> dataStream) {
|
||||
return bootstrap(conf, rowType, defaultParallelism, dataStream, false, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructs bootstrap pipeline.
|
||||
* The bootstrap operator loads the existing data index (primary key to file id mapping),
|
||||
* then send the indexing data set to subsequent operator(usually the bucket assign operator).
|
||||
*
|
||||
* @param conf The configuration
|
||||
* @param rowType The row type
|
||||
* @param defaultParallelism The default parallelism
|
||||
* @param dataStream The data stream
|
||||
* @param bounded Whether the source is bounded
|
||||
* @param overwrite Whether it is insert overwrite
|
||||
*/
|
||||
public static DataStream<HoodieRecord> bootstrap(
|
||||
Configuration conf,
|
||||
RowType rowType,
|
||||
int defaultParallelism,
|
||||
DataStream<RowData> dataStream,
|
||||
boolean bounded,
|
||||
boolean overwrite) {
|
||||
final boolean globalIndex = conf.getBoolean(FlinkOptions.INDEX_GLOBAL_ENABLED);
|
||||
if (overwrite || OptionsResolver.isBucketIndexType(conf)) {
|
||||
return rowDataToHoodieRecord(conf, rowType, dataStream);
|
||||
} else if (bounded && !globalIndex && OptionsResolver.isPartitionedTable(conf)) {
|
||||
return boundedBootstrap(conf, rowType, defaultParallelism, dataStream);
|
||||
} else {
|
||||
return streamBootstrap(conf, rowType, defaultParallelism, dataStream, bounded);
|
||||
}
|
||||
}
|
||||
|
||||
private static DataStream<HoodieRecord> streamBootstrap(
|
||||
Configuration conf,
|
||||
RowType rowType,
|
||||
int defaultParallelism,
|
||||
DataStream<RowData> dataStream,
|
||||
boolean bounded) {
|
||||
DataStream<HoodieRecord> dataStream1 = rowDataToHoodieRecord(conf, rowType, dataStream);
|
||||
|
||||
if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED) || bounded) {
|
||||
dataStream1 = dataStream1
|
||||
.transform(
|
||||
"index_bootstrap",
|
||||
TypeInformation.of(HoodieRecord.class),
|
||||
new BootstrapOperator<>(conf))
|
||||
.setParallelism(conf.getOptional(FlinkOptions.INDEX_BOOTSTRAP_TASKS).orElse(defaultParallelism))
|
||||
.uid("uid_index_bootstrap_" + conf.getString(FlinkOptions.TABLE_NAME));
|
||||
}
|
||||
|
||||
return dataStream1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructs bootstrap pipeline for batch execution mode.
|
||||
* The indexing data set is loaded before the actual data write
|
||||
* in order to support batch UPSERT.
|
||||
*/
|
||||
private static DataStream<HoodieRecord> boundedBootstrap(
|
||||
Configuration conf,
|
||||
RowType rowType,
|
||||
int defaultParallelism,
|
||||
DataStream<RowData> dataStream) {
|
||||
final RowDataKeyGen rowDataKeyGen = RowDataKeyGen.instance(conf, rowType);
|
||||
// shuffle by partition keys
|
||||
dataStream = dataStream
|
||||
.keyBy(rowDataKeyGen::getPartitionPath);
|
||||
|
||||
return rowDataToHoodieRecord(conf, rowType, dataStream)
|
||||
.transform(
|
||||
"batch_index_bootstrap",
|
||||
TypeInformation.of(HoodieRecord.class),
|
||||
new BatchBootstrapOperator<>(conf))
|
||||
.setParallelism(conf.getOptional(FlinkOptions.INDEX_BOOTSTRAP_TASKS).orElse(defaultParallelism))
|
||||
.uid("uid_batch_index_bootstrap_" + conf.getString(FlinkOptions.TABLE_NAME));
|
||||
}
|
||||
|
||||
/**
|
||||
* Transforms the row data to hoodie records.
|
||||
*/
|
||||
public static DataStream<HoodieRecord> rowDataToHoodieRecord(Configuration conf, RowType rowType, DataStream<RowData> dataStream) {
|
||||
return dataStream.map(RowDataToHoodieFunctions.create(rowType, conf), TypeInformation.of(HoodieRecord.class))
|
||||
.setParallelism(dataStream.getParallelism()).name("row_data_to_hoodie_record");
|
||||
}
|
||||
|
||||
/**
|
||||
* The streaming write pipeline.
|
||||
*
|
||||
* <p>The input dataset shuffles by the primary key first then
|
||||
* shuffles by the file group ID before passing around to the write function.
|
||||
* The whole pipeline looks like the following:
|
||||
*
|
||||
* <pre>
|
||||
* | input1 | ===\ /=== | bucket assigner | ===\ /=== | task1 |
|
||||
* shuffle(by PK) shuffle(by bucket ID)
|
||||
* | input2 | ===/ \=== | bucket assigner | ===/ \=== | task2 |
|
||||
*
|
||||
* Note: a file group must be handled by one write task to avoid write conflict.
|
||||
* </pre>
|
||||
*
|
||||
* <p>The bucket assigner assigns the inputs to suitable file groups, the write task caches
|
||||
* and flushes the data set to disk.
|
||||
*
|
||||
* @param conf The configuration
|
||||
* @param defaultParallelism The default parallelism
|
||||
* @param dataStream The input data stream
|
||||
* @return the stream write data stream pipeline
|
||||
*/
|
||||
public static DataStream<Object> hoodieStreamWrite(Configuration conf, int defaultParallelism, DataStream<HoodieRecord> dataStream) {
|
||||
if (OptionsResolver.isBucketIndexType(conf)) {
|
||||
WriteOperatorFactory<HoodieRecord> operatorFactory = BucketStreamWriteOperator.getFactory(conf);
|
||||
int bucketNum = conf.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS);
|
||||
String indexKeyFields = conf.getString(FlinkOptions.INDEX_KEY_FIELD);
|
||||
BucketIndexPartitioner<HoodieKey> partitioner = new BucketIndexPartitioner<>(bucketNum, indexKeyFields);
|
||||
return dataStream.partitionCustom(partitioner, HoodieRecord::getKey)
|
||||
.transform("bucket_write", TypeInformation.of(Object.class), operatorFactory)
|
||||
.uid("uid_bucket_write" + conf.getString(FlinkOptions.TABLE_NAME))
|
||||
.setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS));
|
||||
} else {
|
||||
WriteOperatorFactory<HoodieRecord> operatorFactory = StreamWriteOperator.getFactory(conf);
|
||||
return dataStream
|
||||
// Key-by record key, to avoid multiple subtasks write to a bucket at the same time
|
||||
.keyBy(HoodieRecord::getRecordKey)
|
||||
.transform(
|
||||
"bucket_assigner",
|
||||
TypeInformation.of(HoodieRecord.class),
|
||||
new KeyedProcessOperator<>(new BucketAssignFunction<>(conf)))
|
||||
.uid("uid_bucket_assigner_" + conf.getString(FlinkOptions.TABLE_NAME))
|
||||
.setParallelism(conf.getOptional(FlinkOptions.BUCKET_ASSIGN_TASKS).orElse(defaultParallelism))
|
||||
// shuffle by fileId(bucket id)
|
||||
.keyBy(record -> record.getCurrentLocation().getFileId())
|
||||
.transform("stream_write", TypeInformation.of(Object.class), operatorFactory)
|
||||
.uid("uid_stream_write" + conf.getString(FlinkOptions.TABLE_NAME))
|
||||
.setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The compaction tasks pipeline.
|
||||
*
|
||||
* <p>The compaction plan operator monitors the new compaction plan on the timeline
|
||||
* then distributes the sub-plans to the compaction tasks. The compaction task then
|
||||
* handle over the metadata to commit task for compaction transaction commit.
|
||||
* The whole pipeline looks like the following:
|
||||
*
|
||||
* <pre>
|
||||
* /=== | task1 | ===\
|
||||
* | plan generation | ===> re-balance | commit |
|
||||
* \=== | task2 | ===/
|
||||
*
|
||||
* Note: both the compaction plan generation task and commission task are singleton.
|
||||
* </pre>
|
||||
*
|
||||
* @param conf The configuration
|
||||
* @param dataStream The input data stream
|
||||
* @return the compaction pipeline
|
||||
*/
|
||||
public static DataStreamSink<CompactionCommitEvent> compact(Configuration conf, DataStream<Object> dataStream) {
|
||||
return dataStream.transform("compact_plan_generate",
|
||||
TypeInformation.of(CompactionPlanEvent.class),
|
||||
new CompactionPlanOperator(conf))
|
||||
.setParallelism(1) // plan generate must be singleton
|
||||
.rebalance()
|
||||
.transform("compact_task",
|
||||
TypeInformation.of(CompactionCommitEvent.class),
|
||||
new ProcessOperator<>(new CompactFunction(conf)))
|
||||
.setParallelism(conf.getInteger(FlinkOptions.COMPACTION_TASKS))
|
||||
.addSink(new CompactionCommitSink(conf))
|
||||
.name("compact_commit")
|
||||
.setParallelism(1); // compaction commit should be singleton
|
||||
}
|
||||
|
||||
public static DataStreamSink<Object> clean(Configuration conf, DataStream<Object> dataStream) {
|
||||
return dataStream.addSink(new CleanFunction<>(conf))
|
||||
.setParallelism(1)
|
||||
.name("clean_commits");
|
||||
}
|
||||
|
||||
/**
|
||||
* Dummy sink that does nothing.
|
||||
*/
|
||||
public static class DummySink implements SinkFunction<Object> {
|
||||
private static final long serialVersionUID = 1L;
|
||||
public static DummySink INSTANCE = new DummySink();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,99 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.utils;
|
||||
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
* Tool used for time waiting.
|
||||
*/
|
||||
public class TimeWait {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(TimeWait.class);
|
||||
|
||||
private final long timeout; // timeout in SECONDS
|
||||
private final long interval; // interval in MILLISECONDS
|
||||
private final String action; // action to report error message
|
||||
|
||||
private long waitingTime = 0L;
|
||||
|
||||
private TimeWait(long timeout, long interval, String action) {
|
||||
this.timeout = timeout;
|
||||
this.interval = interval;
|
||||
this.action = action;
|
||||
}
|
||||
|
||||
public static Builder builder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
/**
|
||||
* Wait for an interval time.
|
||||
*/
|
||||
public void waitFor() {
|
||||
try {
|
||||
if (waitingTime > timeout) {
|
||||
throw new HoodieException("Timeout(" + waitingTime + "ms) while waiting for " + action);
|
||||
}
|
||||
TimeUnit.MILLISECONDS.sleep(interval);
|
||||
waitingTime += interval;
|
||||
} catch (InterruptedException e) {
|
||||
throw new HoodieException("Error while waiting for " + action, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Builder.
|
||||
*/
|
||||
public static class Builder {
|
||||
private long timeout = 5 * 60 * 1000L; // default 5 minutes
|
||||
private long interval = 1000;
|
||||
private String action;
|
||||
|
||||
private Builder() {
|
||||
}
|
||||
|
||||
public Builder timeout(long timeout) {
|
||||
if (timeout > 0) {
|
||||
this.timeout = timeout;
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder interval(long interval) {
|
||||
this.interval = interval;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder action(String action) {
|
||||
this.action = action;
|
||||
return this;
|
||||
}
|
||||
|
||||
public TimeWait build() {
|
||||
Objects.requireNonNull(this.action);
|
||||
return new TimeWait(this.timeout, this.interval, this.action);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,188 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.source;
|
||||
|
||||
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.annotation.VisibleForTesting;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* A file index which supports listing files efficiently through metadata table.
|
||||
*
|
||||
* <p>It caches the partition paths to avoid redundant look up.
|
||||
*/
|
||||
public class FileIndex {
|
||||
private final Path path;
|
||||
private final HoodieMetadataConfig metadataConfig;
|
||||
private List<String> partitionPaths; // cache of partition paths
|
||||
private final boolean tableExists;
|
||||
|
||||
private FileIndex(Path path, Configuration conf) {
|
||||
this.path = path;
|
||||
this.metadataConfig = metadataConfig(conf);
|
||||
this.tableExists = StreamerUtil.tableExists(path.toString(), StreamerUtil.getHadoopConf());
|
||||
}
|
||||
|
||||
public static FileIndex instance(Path path, Configuration conf) {
|
||||
return new FileIndex(path, conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the partition path key and values as a list of map, each map item in the list
|
||||
* is a mapping of the partition key name to its actual partition value. For example, say
|
||||
* there is a file path with partition keys [key1, key2, key3]:
|
||||
*
|
||||
* <p><pre>
|
||||
* -- file:/// ... key1=val1/key2=val2/key3=val3
|
||||
* -- file:/// ... key1=val4/key2=val5/key3=val6
|
||||
* </pre>
|
||||
*
|
||||
* <p>The return list should be [{key1:val1, key2:val2, key3:val3}, {key1:val4, key2:val5, key3:val6}].
|
||||
*
|
||||
* @param partitionKeys The partition key list
|
||||
* @param defaultParName The default partition name for nulls
|
||||
* @param hivePartition Whether the partition path is in Hive style
|
||||
*/
|
||||
public List<Map<String, String>> getPartitions(
|
||||
List<String> partitionKeys,
|
||||
String defaultParName,
|
||||
boolean hivePartition) {
|
||||
if (partitionKeys.size() == 0) {
|
||||
// non partitioned table
|
||||
return Collections.emptyList();
|
||||
}
|
||||
List<String> partitionPaths = getOrBuildPartitionPaths();
|
||||
if (partitionPaths.size() == 1 && partitionPaths.get(0).isEmpty()) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
List<Map<String, String>> partitions = new ArrayList<>();
|
||||
for (String partitionPath : partitionPaths) {
|
||||
String[] paths = partitionPath.split(Path.SEPARATOR);
|
||||
Map<String, String> partitionMapping = new LinkedHashMap<>();
|
||||
if (hivePartition) {
|
||||
Arrays.stream(paths).forEach(p -> {
|
||||
String[] kv = p.split("=");
|
||||
if (kv.length == 2) {
|
||||
partitionMapping.put(kv[0], defaultParName.equals(kv[1]) ? null : kv[1]);
|
||||
}
|
||||
});
|
||||
} else {
|
||||
for (int i = 0; i < partitionKeys.size(); i++) {
|
||||
partitionMapping.put(partitionKeys.get(i), defaultParName.equals(paths[i]) ? null : paths[i]);
|
||||
}
|
||||
}
|
||||
partitions.add(partitionMapping);
|
||||
}
|
||||
return partitions;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all the file statuses under the table base path.
|
||||
*/
|
||||
public FileStatus[] getFilesInPartitions() {
|
||||
if (!tableExists) {
|
||||
return new FileStatus[0];
|
||||
}
|
||||
String[] partitions = getOrBuildPartitionPaths().stream().map(p -> fullPartitionPath(path, p)).toArray(String[]::new);
|
||||
return FSUtils.getFilesInPartitions(HoodieFlinkEngineContext.DEFAULT, metadataConfig, path.toString(),
|
||||
partitions, "/tmp/")
|
||||
.values().stream().flatMap(Arrays::stream).toArray(FileStatus[]::new);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the full partition path.
|
||||
*
|
||||
* @param basePath The base path.
|
||||
* @param partitionPath The relative partition path, may be empty if the table is non-partitioned.
|
||||
* @return The full partition path string
|
||||
*/
|
||||
private static String fullPartitionPath(Path basePath, String partitionPath) {
|
||||
if (partitionPath.isEmpty()) {
|
||||
return basePath.toString();
|
||||
}
|
||||
return new Path(basePath, partitionPath).toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Reset the state of the file index.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public void reset() {
|
||||
this.partitionPaths = null;
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Getter/Setter
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
/**
|
||||
* Sets up explicit partition paths for pruning.
|
||||
*/
|
||||
public void setPartitionPaths(@Nullable Set<String> partitionPaths) {
|
||||
if (partitionPaths != null) {
|
||||
this.partitionPaths = new ArrayList<>(partitionPaths);
|
||||
}
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Utilities
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
/**
|
||||
* Returns all the relative partition paths.
|
||||
*
|
||||
* <p>The partition paths are cached once invoked.
|
||||
*/
|
||||
public List<String> getOrBuildPartitionPaths() {
|
||||
if (this.partitionPaths != null) {
|
||||
return this.partitionPaths;
|
||||
}
|
||||
this.partitionPaths = this.tableExists
|
||||
? FSUtils.getAllPartitionPaths(HoodieFlinkEngineContext.DEFAULT, metadataConfig, path.toString())
|
||||
: Collections.emptyList();
|
||||
return this.partitionPaths;
|
||||
}
|
||||
|
||||
private static HoodieMetadataConfig metadataConfig(org.apache.flink.configuration.Configuration conf) {
|
||||
Properties properties = new Properties();
|
||||
|
||||
// set up metadata.enabled=true in table DDL to enable metadata listing
|
||||
properties.put(HoodieMetadataConfig.ENABLE.key(), conf.getBoolean(FlinkOptions.METADATA_ENABLED));
|
||||
|
||||
return HoodieMetadataConfig.newBuilder().fromProperties(properties).build();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,391 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.source;
|
||||
|
||||
import org.apache.hudi.common.model.BaseFile;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.log.InstantRange;
|
||||
import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
|
||||
import org.apache.hudi.sink.partitioner.profile.WriteProfiles;
|
||||
import org.apache.hudi.table.format.mor.MergeOnReadInputSplit;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.core.fs.Path;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import scala.Serializable;
|
||||
|
||||
import static org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN;
|
||||
import static org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN_OR_EQUALS;
|
||||
import static org.apache.hudi.common.table.timeline.HoodieTimeline.LESSER_THAN_OR_EQUALS;
|
||||
|
||||
/**
|
||||
* Utilities to generate incremental input splits {@link MergeOnReadInputSplit}.
|
||||
* The input splits are used for streaming and incremental read.
|
||||
*
|
||||
* <p>How to generate the input splits:
|
||||
* <ol>
|
||||
* <li>first fetch all the commit metadata for the incremental instants;</li>
|
||||
* <li>resolve the incremental commit file paths;</li>
|
||||
* <li>filter the full file paths by required partitions;</li>
|
||||
* <li>use the file paths from #step 3 as the back-up of the filesystem view.</li>
|
||||
* </ol>
|
||||
*/
|
||||
public class IncrementalInputSplits implements Serializable {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(IncrementalInputSplits.class);
|
||||
private final Configuration conf;
|
||||
private final Path path;
|
||||
private final long maxCompactionMemoryInBytes;
|
||||
// for partition pruning
|
||||
private final Set<String> requiredPartitions;
|
||||
// skip compaction
|
||||
private final boolean skipCompaction;
|
||||
|
||||
private IncrementalInputSplits(
|
||||
Configuration conf,
|
||||
Path path,
|
||||
long maxCompactionMemoryInBytes,
|
||||
@Nullable Set<String> requiredPartitions,
|
||||
boolean skipCompaction) {
|
||||
this.conf = conf;
|
||||
this.path = path;
|
||||
this.maxCompactionMemoryInBytes = maxCompactionMemoryInBytes;
|
||||
this.requiredPartitions = requiredPartitions;
|
||||
this.skipCompaction = skipCompaction;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the builder.
|
||||
*/
|
||||
public static Builder builder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the incremental input splits.
|
||||
*
|
||||
* @param metaClient The meta client
|
||||
* @param hadoopConf The hadoop configuration
|
||||
* @return The list of incremental input splits or empty if there are no new instants
|
||||
*/
|
||||
public Result inputSplits(
|
||||
HoodieTableMetaClient metaClient,
|
||||
org.apache.hadoop.conf.Configuration hadoopConf) {
|
||||
return inputSplits(metaClient, hadoopConf, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the incremental input splits.
|
||||
*
|
||||
* @param metaClient The meta client
|
||||
* @param hadoopConf The hadoop configuration
|
||||
* @param issuedInstant The last issued instant, only valid in streaming read
|
||||
* @return The list of incremental input splits or empty if there are no new instants
|
||||
*/
|
||||
public Result inputSplits(
|
||||
HoodieTableMetaClient metaClient,
|
||||
org.apache.hadoop.conf.Configuration hadoopConf,
|
||||
String issuedInstant) {
|
||||
metaClient.reloadActiveTimeline();
|
||||
HoodieTimeline commitTimeline = metaClient.getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants();
|
||||
if (commitTimeline.empty()) {
|
||||
LOG.warn("No splits found for the table under path " + path);
|
||||
return Result.EMPTY;
|
||||
}
|
||||
List<HoodieInstant> instants = filterInstantsWithRange(commitTimeline, issuedInstant);
|
||||
// get the latest instant that satisfies condition
|
||||
final HoodieInstant instantToIssue = instants.size() == 0 ? null : instants.get(instants.size() - 1);
|
||||
final InstantRange instantRange;
|
||||
if (instantToIssue != null) {
|
||||
if (issuedInstant != null) {
|
||||
// the streaming reader may record the last issued instant, if the issued instant is present,
|
||||
// the instant range should be: (issued instant, the latest instant].
|
||||
instantRange = InstantRange.getInstance(issuedInstant, instantToIssue.getTimestamp(),
|
||||
InstantRange.RangeType.OPEN_CLOSE);
|
||||
} else if (this.conf.getOptional(FlinkOptions.READ_START_COMMIT).isPresent()) {
|
||||
// first time consume and has a start commit
|
||||
final String startCommit = this.conf.getString(FlinkOptions.READ_START_COMMIT);
|
||||
instantRange = startCommit.equalsIgnoreCase(FlinkOptions.START_COMMIT_EARLIEST)
|
||||
? null
|
||||
: InstantRange.getInstance(startCommit, instantToIssue.getTimestamp(), InstantRange.RangeType.CLOSE_CLOSE);
|
||||
} else {
|
||||
// first time consume and no start commit, consumes the latest incremental data set.
|
||||
instantRange = InstantRange.getInstance(instantToIssue.getTimestamp(), instantToIssue.getTimestamp(),
|
||||
InstantRange.RangeType.CLOSE_CLOSE);
|
||||
}
|
||||
} else {
|
||||
LOG.info("No new instant found for the table under path " + path + ", skip reading");
|
||||
return Result.EMPTY;
|
||||
}
|
||||
|
||||
String tableName = conf.getString(FlinkOptions.TABLE_NAME);
|
||||
|
||||
Set<String> writePartitions;
|
||||
final FileStatus[] fileStatuses;
|
||||
|
||||
if (instantRange == null) {
|
||||
// reading from the earliest, scans the partitions and files directly.
|
||||
FileIndex fileIndex = FileIndex.instance(new org.apache.hadoop.fs.Path(path.toUri()), conf);
|
||||
if (this.requiredPartitions != null) {
|
||||
// apply partition push down
|
||||
fileIndex.setPartitionPaths(this.requiredPartitions);
|
||||
}
|
||||
writePartitions = new HashSet<>(fileIndex.getOrBuildPartitionPaths());
|
||||
if (writePartitions.size() == 0) {
|
||||
LOG.warn("No partitions found for reading in user provided path.");
|
||||
return Result.EMPTY;
|
||||
}
|
||||
fileStatuses = fileIndex.getFilesInPartitions();
|
||||
} else {
|
||||
List<HoodieCommitMetadata> activeMetadataList = instants.stream()
|
||||
.map(instant -> WriteProfiles.getCommitMetadata(tableName, path, instant, commitTimeline)).collect(Collectors.toList());
|
||||
List<HoodieCommitMetadata> archivedMetadataList = getArchivedMetadata(metaClient, instantRange, commitTimeline, tableName);
|
||||
if (archivedMetadataList.size() > 0) {
|
||||
LOG.warn("\n"
|
||||
+ "--------------------------------------------------------------------------------\n"
|
||||
+ "---------- caution: the reader has fall behind too much from the writer,\n"
|
||||
+ "---------- tweak 'read.tasks' option to add parallelism of read tasks.\n"
|
||||
+ "--------------------------------------------------------------------------------");
|
||||
}
|
||||
List<HoodieCommitMetadata> metadataList = archivedMetadataList.size() > 0
|
||||
// IMPORTANT: the merged metadata list must be in ascending order by instant time
|
||||
? mergeList(archivedMetadataList, activeMetadataList)
|
||||
: activeMetadataList;
|
||||
|
||||
writePartitions = HoodieInputFormatUtils.getWritePartitionPaths(metadataList);
|
||||
// apply partition push down
|
||||
if (this.requiredPartitions != null) {
|
||||
writePartitions = writePartitions.stream()
|
||||
.filter(this.requiredPartitions::contains).collect(Collectors.toSet());
|
||||
}
|
||||
if (writePartitions.size() == 0) {
|
||||
LOG.warn("No partitions found for reading in user provided path.");
|
||||
return Result.EMPTY;
|
||||
}
|
||||
fileStatuses = WriteProfiles.getWritePathsOfInstants(path, hadoopConf, metadataList, metaClient.getTableType());
|
||||
}
|
||||
|
||||
if (fileStatuses.length == 0) {
|
||||
LOG.warn("No files found for reading in user provided path.");
|
||||
return Result.EMPTY;
|
||||
}
|
||||
|
||||
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, commitTimeline, fileStatuses);
|
||||
final String endInstant = instantToIssue.getTimestamp();
|
||||
final AtomicInteger cnt = new AtomicInteger(0);
|
||||
final String mergeType = this.conf.getString(FlinkOptions.MERGE_TYPE);
|
||||
List<MergeOnReadInputSplit> inputSplits = writePartitions.stream()
|
||||
.map(relPartitionPath -> fsView.getLatestMergedFileSlicesBeforeOrOn(relPartitionPath, endInstant)
|
||||
.map(fileSlice -> {
|
||||
Option<List<String>> logPaths = Option.ofNullable(fileSlice.getLogFiles()
|
||||
.sorted(HoodieLogFile.getLogFileComparator())
|
||||
.map(logFile -> logFile.getPath().toString())
|
||||
.collect(Collectors.toList()));
|
||||
String basePath = fileSlice.getBaseFile().map(BaseFile::getPath).orElse(null);
|
||||
return new MergeOnReadInputSplit(cnt.getAndAdd(1),
|
||||
basePath, logPaths, endInstant,
|
||||
metaClient.getBasePath(), maxCompactionMemoryInBytes, mergeType, instantRange);
|
||||
}).collect(Collectors.toList()))
|
||||
.flatMap(Collection::stream)
|
||||
.collect(Collectors.toList());
|
||||
return Result.instance(inputSplits, endInstant);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the archived metadata in case the reader consumes untimely or it wants
|
||||
* to read from the earliest.
|
||||
*
|
||||
* <p>Note: should improve it with metadata table when the metadata table is stable enough.
|
||||
*
|
||||
* @param metaClient The meta client
|
||||
* @param instantRange The instant range to filter the timeline instants
|
||||
* @param commitTimeline The commit timeline
|
||||
* @param tableName The table name
|
||||
* @return the list of archived metadata, or empty if there is no need to read the archived timeline
|
||||
*/
|
||||
private List<HoodieCommitMetadata> getArchivedMetadata(
|
||||
HoodieTableMetaClient metaClient,
|
||||
InstantRange instantRange,
|
||||
HoodieTimeline commitTimeline,
|
||||
String tableName) {
|
||||
if (commitTimeline.isBeforeTimelineStarts(instantRange.getStartInstant())) {
|
||||
// read the archived metadata if the start instant is archived.
|
||||
HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline(instantRange.getStartInstant());
|
||||
HoodieTimeline archivedCompleteTimeline = archivedTimeline.getCommitsTimeline().filterCompletedInstants();
|
||||
if (!archivedCompleteTimeline.empty()) {
|
||||
Stream<HoodieInstant> instantStream = archivedCompleteTimeline.getInstants();
|
||||
return maySkipCompaction(instantStream)
|
||||
.map(instant -> WriteProfiles.getCommitMetadata(tableName, path, instant, archivedTimeline)).collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the instants with a given issuedInstant to start from.
|
||||
*
|
||||
* @param commitTimeline The completed commits timeline
|
||||
* @param issuedInstant The last issued instant that has already been delivered to downstream
|
||||
* @return the filtered hoodie instants
|
||||
*/
|
||||
private List<HoodieInstant> filterInstantsWithRange(
|
||||
HoodieTimeline commitTimeline,
|
||||
final String issuedInstant) {
|
||||
HoodieTimeline completedTimeline = commitTimeline.filterCompletedInstants();
|
||||
if (issuedInstant != null) {
|
||||
// returns early for streaming mode
|
||||
return maySkipCompaction(completedTimeline.getInstants())
|
||||
.filter(s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), GREATER_THAN, issuedInstant))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
Stream<HoodieInstant> instantStream = completedTimeline.getInstants();
|
||||
|
||||
if (this.conf.getOptional(FlinkOptions.READ_START_COMMIT).isPresent()
|
||||
&& !this.conf.get(FlinkOptions.READ_START_COMMIT).equalsIgnoreCase(FlinkOptions.START_COMMIT_EARLIEST)) {
|
||||
final String startCommit = this.conf.get(FlinkOptions.READ_START_COMMIT);
|
||||
instantStream = instantStream
|
||||
.filter(s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), GREATER_THAN_OR_EQUALS, startCommit));
|
||||
}
|
||||
if (this.conf.getOptional(FlinkOptions.READ_END_COMMIT).isPresent()) {
|
||||
final String endCommit = this.conf.get(FlinkOptions.READ_END_COMMIT);
|
||||
instantStream = instantStream.filter(s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), LESSER_THAN_OR_EQUALS, endCommit));
|
||||
}
|
||||
return maySkipCompaction(instantStream).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
private Stream<HoodieInstant> maySkipCompaction(Stream<HoodieInstant> instants) {
|
||||
return this.skipCompaction
|
||||
? instants.filter(instant -> !instant.getAction().equals(HoodieTimeline.COMMIT_ACTION))
|
||||
: instants;
|
||||
}
|
||||
|
||||
private static <T> List<T> mergeList(List<T> list1, List<T> list2) {
|
||||
List<T> merged = new ArrayList<>(list1);
|
||||
merged.addAll(list2);
|
||||
return merged;
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Inner Class
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
/**
|
||||
* Represents a result of calling {@link #inputSplits}.
|
||||
*/
|
||||
public static class Result {
|
||||
private final List<MergeOnReadInputSplit> inputSplits; // input splits
|
||||
private final String endInstant; // end instant to consume to
|
||||
|
||||
public static final Result EMPTY = instance(Collections.emptyList(), "");
|
||||
|
||||
public boolean isEmpty() {
|
||||
return this.inputSplits.size() == 0;
|
||||
}
|
||||
|
||||
public List<MergeOnReadInputSplit> getInputSplits() {
|
||||
return this.inputSplits;
|
||||
}
|
||||
|
||||
public String getEndInstant() {
|
||||
return this.endInstant;
|
||||
}
|
||||
|
||||
private Result(List<MergeOnReadInputSplit> inputSplits, String endInstant) {
|
||||
this.inputSplits = inputSplits;
|
||||
this.endInstant = endInstant;
|
||||
}
|
||||
|
||||
public static Result instance(List<MergeOnReadInputSplit> inputSplits, String endInstant) {
|
||||
return new Result(inputSplits, endInstant);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Builder for {@link IncrementalInputSplits}.
|
||||
*/
|
||||
public static class Builder {
|
||||
private Configuration conf;
|
||||
private Path path;
|
||||
private long maxCompactionMemoryInBytes;
|
||||
// for partition pruning
|
||||
private Set<String> requiredPartitions;
|
||||
// skip compaction
|
||||
private boolean skipCompaction = false;
|
||||
|
||||
public Builder() {
|
||||
}
|
||||
|
||||
public Builder conf(Configuration conf) {
|
||||
this.conf = conf;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder path(Path path) {
|
||||
this.path = path;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder maxCompactionMemoryInBytes(long maxCompactionMemoryInBytes) {
|
||||
this.maxCompactionMemoryInBytes = maxCompactionMemoryInBytes;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder requiredPartitions(@Nullable Set<String> requiredPartitions) {
|
||||
this.requiredPartitions = requiredPartitions;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder skipCompaction(boolean skipCompaction) {
|
||||
this.skipCompaction = skipCompaction;
|
||||
return this;
|
||||
}
|
||||
|
||||
public IncrementalInputSplits build() {
|
||||
return new IncrementalInputSplits(Objects.requireNonNull(this.conf), Objects.requireNonNull(this.path),
|
||||
this.maxCompactionMemoryInBytes, this.requiredPartitions, this.skipCompaction);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,254 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.source;
|
||||
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.table.format.mor.MergeOnReadInputSplit;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.annotation.VisibleForTesting;
|
||||
import org.apache.flink.api.common.state.ListState;
|
||||
import org.apache.flink.api.common.state.ListStateDescriptor;
|
||||
import org.apache.flink.api.common.typeutils.base.StringSerializer;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.core.fs.Path;
|
||||
import org.apache.flink.runtime.state.FunctionInitializationContext;
|
||||
import org.apache.flink.runtime.state.FunctionSnapshotContext;
|
||||
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
|
||||
import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
|
||||
import org.apache.flink.streaming.api.functions.source.SourceFunction;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
* This is the single (non-parallel) monitoring task which takes a {@link MergeOnReadInputSplit}
|
||||
* , it is responsible for:
|
||||
*
|
||||
* <ol>
|
||||
* <li>Monitoring a user-provided hoodie table path.</li>
|
||||
* <li>Deciding which files(or split) should be further read and processed.</li>
|
||||
* <li>Creating the {@link MergeOnReadInputSplit splits} corresponding to those files.</li>
|
||||
* <li>Assigning them to downstream tasks for further processing.</li>
|
||||
* </ol>
|
||||
*
|
||||
* <p>The splits to be read are forwarded to the downstream {@link StreamReadOperator}
|
||||
* which can have parallelism greater than one.
|
||||
*
|
||||
* <p><b>IMPORTANT NOTE: </b> Splits are forwarded downstream for reading in ascending instant commits time order,
|
||||
* in each downstream task, the splits are also read in receiving sequence. We do not ensure split consuming sequence
|
||||
* among the downstream tasks.
|
||||
*/
|
||||
public class StreamReadMonitoringFunction
|
||||
extends RichSourceFunction<MergeOnReadInputSplit> implements CheckpointedFunction {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(StreamReadMonitoringFunction.class);
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
/**
|
||||
* The path to monitor.
|
||||
*/
|
||||
private final Path path;
|
||||
|
||||
/**
|
||||
* The interval between consecutive path scans.
|
||||
*/
|
||||
private final long interval;
|
||||
|
||||
private transient Object checkpointLock;
|
||||
|
||||
private volatile boolean isRunning = true;
|
||||
|
||||
private String issuedInstant;
|
||||
|
||||
private transient ListState<String> instantState;
|
||||
|
||||
private final Configuration conf;
|
||||
|
||||
private transient org.apache.hadoop.conf.Configuration hadoopConf;
|
||||
|
||||
private HoodieTableMetaClient metaClient;
|
||||
|
||||
private final IncrementalInputSplits incrementalInputSplits;
|
||||
|
||||
public StreamReadMonitoringFunction(
|
||||
Configuration conf,
|
||||
Path path,
|
||||
long maxCompactionMemoryInBytes,
|
||||
@Nullable Set<String> requiredPartitionPaths) {
|
||||
this.conf = conf;
|
||||
this.path = path;
|
||||
this.interval = conf.getInteger(FlinkOptions.READ_STREAMING_CHECK_INTERVAL);
|
||||
this.incrementalInputSplits = IncrementalInputSplits.builder()
|
||||
.conf(conf)
|
||||
.path(path)
|
||||
.maxCompactionMemoryInBytes(maxCompactionMemoryInBytes)
|
||||
.requiredPartitions(requiredPartitionPaths)
|
||||
.skipCompaction(conf.getBoolean(FlinkOptions.READ_STREAMING_SKIP_COMPACT))
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initializeState(FunctionInitializationContext context) throws Exception {
|
||||
|
||||
ValidationUtils.checkState(this.instantState == null,
|
||||
"The " + getClass().getSimpleName() + " has already been initialized.");
|
||||
|
||||
this.instantState = context.getOperatorStateStore().getListState(
|
||||
new ListStateDescriptor<>(
|
||||
"file-monitoring-state",
|
||||
StringSerializer.INSTANCE
|
||||
)
|
||||
);
|
||||
|
||||
if (context.isRestored()) {
|
||||
LOG.info("Restoring state for the class {} with table {} and base path {}.",
|
||||
getClass().getSimpleName(), conf.getString(FlinkOptions.TABLE_NAME), path);
|
||||
|
||||
List<String> retrievedStates = new ArrayList<>();
|
||||
for (String entry : this.instantState.get()) {
|
||||
retrievedStates.add(entry);
|
||||
}
|
||||
|
||||
ValidationUtils.checkArgument(retrievedStates.size() <= 1,
|
||||
getClass().getSimpleName() + " retrieved invalid state.");
|
||||
|
||||
if (retrievedStates.size() == 1 && issuedInstant != null) {
|
||||
// this is the case where we have both legacy and new state.
|
||||
// the two should be mutually exclusive for the operator, thus we throw the exception.
|
||||
|
||||
throw new IllegalArgumentException(
|
||||
"The " + getClass().getSimpleName() + " has already restored from a previous Flink version.");
|
||||
|
||||
} else if (retrievedStates.size() == 1) {
|
||||
this.issuedInstant = retrievedStates.get(0);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("{} retrieved a issued instant of time {} for table {} with path {}.",
|
||||
getClass().getSimpleName(), issuedInstant, conf.get(FlinkOptions.TABLE_NAME), path);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(Configuration parameters) throws Exception {
|
||||
super.open(parameters);
|
||||
this.hadoopConf = StreamerUtil.getHadoopConf();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run(SourceFunction.SourceContext<MergeOnReadInputSplit> context) throws Exception {
|
||||
checkpointLock = context.getCheckpointLock();
|
||||
while (isRunning) {
|
||||
synchronized (checkpointLock) {
|
||||
monitorDirAndForwardSplits(context);
|
||||
}
|
||||
TimeUnit.SECONDS.sleep(interval);
|
||||
}
|
||||
}
|
||||
|
||||
@Nullable
|
||||
private HoodieTableMetaClient getOrCreateMetaClient() {
|
||||
if (this.metaClient != null) {
|
||||
return this.metaClient;
|
||||
}
|
||||
if (StreamerUtil.tableExists(this.path.toString(), hadoopConf)) {
|
||||
this.metaClient = StreamerUtil.createMetaClient(this.path.toString(), hadoopConf);
|
||||
return this.metaClient;
|
||||
}
|
||||
// fallback
|
||||
return null;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public void monitorDirAndForwardSplits(SourceContext<MergeOnReadInputSplit> context) {
|
||||
HoodieTableMetaClient metaClient = getOrCreateMetaClient();
|
||||
if (metaClient == null) {
|
||||
// table does not exist
|
||||
return;
|
||||
}
|
||||
IncrementalInputSplits.Result result =
|
||||
incrementalInputSplits.inputSplits(metaClient, this.hadoopConf, this.issuedInstant);
|
||||
if (result.isEmpty()) {
|
||||
// no new instants, returns early
|
||||
return;
|
||||
}
|
||||
|
||||
for (MergeOnReadInputSplit split : result.getInputSplits()) {
|
||||
context.collect(split);
|
||||
}
|
||||
// update the issues instant time
|
||||
this.issuedInstant = result.getEndInstant();
|
||||
LOG.info("\n"
|
||||
+ "------------------------------------------------------------\n"
|
||||
+ "---------- consumed to instant: {}\n"
|
||||
+ "------------------------------------------------------------",
|
||||
this.issuedInstant);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws Exception {
|
||||
super.close();
|
||||
|
||||
if (checkpointLock != null) {
|
||||
synchronized (checkpointLock) {
|
||||
issuedInstant = null;
|
||||
isRunning = false;
|
||||
}
|
||||
}
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Closed File Monitoring Source for path: " + path + ".");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void cancel() {
|
||||
if (checkpointLock != null) {
|
||||
// this is to cover the case where cancel() is called before the run()
|
||||
synchronized (checkpointLock) {
|
||||
issuedInstant = null;
|
||||
isRunning = false;
|
||||
}
|
||||
} else {
|
||||
issuedInstant = null;
|
||||
isRunning = false;
|
||||
}
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Checkpointing
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
@Override
|
||||
public void snapshotState(FunctionSnapshotContext context) throws Exception {
|
||||
this.instantState.clear();
|
||||
if (this.issuedInstant != null) {
|
||||
this.instantState.add(this.issuedInstant);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,258 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.source;
|
||||
|
||||
import org.apache.hudi.adapter.AbstractStreamOperatorAdapter;
|
||||
import org.apache.hudi.adapter.AbstractStreamOperatorFactoryAdapter;
|
||||
import org.apache.hudi.adapter.MailboxExecutorAdapter;
|
||||
import org.apache.hudi.adapter.Utils;
|
||||
import org.apache.hudi.table.format.mor.MergeOnReadInputFormat;
|
||||
import org.apache.hudi.table.format.mor.MergeOnReadInputSplit;
|
||||
|
||||
import org.apache.flink.api.common.state.ListState;
|
||||
import org.apache.flink.api.common.state.ListStateDescriptor;
|
||||
import org.apache.flink.runtime.state.JavaSerializer;
|
||||
import org.apache.flink.runtime.state.StateInitializationContext;
|
||||
import org.apache.flink.runtime.state.StateSnapshotContext;
|
||||
import org.apache.flink.streaming.api.functions.source.SourceFunction;
|
||||
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
|
||||
import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
|
||||
import org.apache.flink.streaming.api.operators.StreamOperator;
|
||||
import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
|
||||
import org.apache.flink.streaming.api.watermark.Watermark;
|
||||
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
|
||||
import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.util.Preconditions;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Queue;
|
||||
import java.util.concurrent.LinkedBlockingDeque;
|
||||
|
||||
/**
|
||||
* The operator that reads the {@link MergeOnReadInputSplit splits} received from the preceding {@link
|
||||
* StreamReadMonitoringFunction}. Contrary to the {@link StreamReadMonitoringFunction} which has a parallelism of 1,
|
||||
* this operator can have multiple parallelism.
|
||||
*
|
||||
* <p>As soon as an input split {@link MergeOnReadInputSplit} is received, it is put into a queue,
|
||||
* the {@code MailboxExecutor} read the actual data of the split.
|
||||
* This architecture allows the separation of split reading from processing the checkpoint barriers,
|
||||
* thus removing any potential back-pressure.
|
||||
*/
|
||||
public class StreamReadOperator extends AbstractStreamOperatorAdapter<RowData>
|
||||
implements OneInputStreamOperator<MergeOnReadInputSplit, RowData> {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(StreamReadOperator.class);
|
||||
|
||||
private static final int MINI_BATCH_SIZE = 2048;
|
||||
|
||||
// It's the same thread that runs this operator and checkpoint actions. Use this executor to schedule only
|
||||
// splits for subsequent reading, so that a new checkpoint could be triggered without blocking a long time
|
||||
// for exhausting all scheduled split reading tasks.
|
||||
private final MailboxExecutorAdapter executor;
|
||||
|
||||
private MergeOnReadInputFormat format;
|
||||
|
||||
private transient SourceFunction.SourceContext<RowData> sourceContext;
|
||||
|
||||
private transient ListState<MergeOnReadInputSplit> inputSplitsState;
|
||||
|
||||
private transient Queue<MergeOnReadInputSplit> splits;
|
||||
|
||||
// Splits are read by the same thread that calls #processElement. Each read task is submitted to that thread by adding
|
||||
// them to the executor. This state is used to ensure that only one read task is in that splits queue at a time, so that
|
||||
// read tasks do not accumulate ahead of checkpoint tasks. When there is a read task in the queue, this is set to RUNNING.
|
||||
// When there are no more files to read, this will be set to IDLE.
|
||||
private transient volatile SplitState currentSplitState;
|
||||
|
||||
private StreamReadOperator(MergeOnReadInputFormat format, ProcessingTimeService timeService,
|
||||
MailboxExecutorAdapter mailboxExecutor) {
|
||||
this.format = Preconditions.checkNotNull(format, "The InputFormat should not be null.");
|
||||
this.processingTimeService = timeService;
|
||||
this.executor = Preconditions.checkNotNull(mailboxExecutor, "The mailboxExecutor should not be null.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initializeState(StateInitializationContext context) throws Exception {
|
||||
super.initializeState(context);
|
||||
|
||||
// TODO Replace Java serialization with Avro approach to keep state compatibility.
|
||||
inputSplitsState = context.getOperatorStateStore().getListState(
|
||||
new ListStateDescriptor<>("splits", new JavaSerializer<>()));
|
||||
|
||||
// Initialize the current split state to IDLE.
|
||||
currentSplitState = SplitState.IDLE;
|
||||
|
||||
// Recover splits state from flink state backend if possible.
|
||||
splits = new LinkedBlockingDeque<>();
|
||||
if (context.isRestored()) {
|
||||
int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask();
|
||||
LOG.info("Restoring state for operator {} (task ID: {}).", getClass().getSimpleName(), subtaskIdx);
|
||||
|
||||
for (MergeOnReadInputSplit split : inputSplitsState.get()) {
|
||||
splits.add(split);
|
||||
}
|
||||
}
|
||||
|
||||
this.sourceContext = Utils.getSourceContext(
|
||||
getOperatorConfig().getTimeCharacteristic(),
|
||||
getProcessingTimeService(),
|
||||
getContainingTask(),
|
||||
output,
|
||||
getRuntimeContext().getExecutionConfig().getAutoWatermarkInterval());
|
||||
|
||||
// Enqueue to process the recovered input splits.
|
||||
enqueueProcessSplits();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void snapshotState(StateSnapshotContext context) throws Exception {
|
||||
super.snapshotState(context);
|
||||
|
||||
inputSplitsState.clear();
|
||||
inputSplitsState.addAll(new ArrayList<>(splits));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processElement(StreamRecord<MergeOnReadInputSplit> element) {
|
||||
splits.add(element.getValue());
|
||||
enqueueProcessSplits();
|
||||
}
|
||||
|
||||
private void enqueueProcessSplits() {
|
||||
if (currentSplitState == SplitState.IDLE && !splits.isEmpty()) {
|
||||
currentSplitState = SplitState.RUNNING;
|
||||
executor.execute(this::processSplits, "process input split");
|
||||
}
|
||||
}
|
||||
|
||||
private void processSplits() throws IOException {
|
||||
MergeOnReadInputSplit split = splits.peek();
|
||||
if (split == null) {
|
||||
currentSplitState = SplitState.IDLE;
|
||||
return;
|
||||
}
|
||||
|
||||
// 1. open a fresh new input split and start reading as mini-batch
|
||||
// 2. if the input split has remaining records to read, switches to another runnable to handle
|
||||
// 3. if the input split reads to the end, close the format and remove the split from the queue #splits
|
||||
// 4. for each runnable, reads at most #MINI_BATCH_SIZE number of records
|
||||
if (format.isClosed()) {
|
||||
// This log is important to indicate the consuming process,
|
||||
// there is only one log message for one data bucket.
|
||||
LOG.info("Processing input split : {}", split);
|
||||
format.open(split);
|
||||
}
|
||||
try {
|
||||
consumeAsMiniBatch(split);
|
||||
} finally {
|
||||
currentSplitState = SplitState.IDLE;
|
||||
}
|
||||
|
||||
// Re-schedule to process the next split.
|
||||
enqueueProcessSplits();
|
||||
}
|
||||
|
||||
/**
|
||||
* Consumes at most {@link #MINI_BATCH_SIZE} number of records
|
||||
* for the given input split {@code split}.
|
||||
*
|
||||
* <p>Note: close the input format and remove the input split for the queue {@link #splits}
|
||||
* if the split reads to the end.
|
||||
*
|
||||
* @param split The input split
|
||||
*/
|
||||
private void consumeAsMiniBatch(MergeOnReadInputSplit split) throws IOException {
|
||||
for (int i = 0; i < MINI_BATCH_SIZE; i++) {
|
||||
if (!format.reachedEnd()) {
|
||||
sourceContext.collect(format.nextRecord(null));
|
||||
split.consume();
|
||||
} else {
|
||||
// close the input format
|
||||
format.close();
|
||||
// remove the split
|
||||
splits.poll();
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processWatermark(Watermark mark) {
|
||||
// we do nothing because we emit our own watermarks if needed.
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws Exception {
|
||||
super.close();
|
||||
|
||||
if (format != null) {
|
||||
format.close();
|
||||
format.closeInputFormat();
|
||||
format = null;
|
||||
}
|
||||
|
||||
sourceContext = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finish() throws Exception {
|
||||
super.finish();
|
||||
output.close();
|
||||
if (sourceContext != null) {
|
||||
sourceContext.emitWatermark(Watermark.MAX_WATERMARK);
|
||||
sourceContext.close();
|
||||
sourceContext = null;
|
||||
}
|
||||
}
|
||||
|
||||
public static OneInputStreamOperatorFactory<MergeOnReadInputSplit, RowData> factory(MergeOnReadInputFormat format) {
|
||||
return new OperatorFactory(format);
|
||||
}
|
||||
|
||||
private enum SplitState {
|
||||
IDLE, RUNNING
|
||||
}
|
||||
|
||||
private static class OperatorFactory extends AbstractStreamOperatorFactoryAdapter<RowData>
|
||||
implements OneInputStreamOperatorFactory<MergeOnReadInputSplit, RowData> {
|
||||
|
||||
private final MergeOnReadInputFormat format;
|
||||
|
||||
private OperatorFactory(MergeOnReadInputFormat format) {
|
||||
this.format = format;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public <O extends StreamOperator<RowData>> O createStreamOperator(StreamOperatorParameters<RowData> parameters) {
|
||||
StreamReadOperator operator = new StreamReadOperator(format, processingTimeService, getMailboxExecutorAdapter());
|
||||
operator.setup(parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput());
|
||||
return (O) operator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class<? extends StreamOperator> getStreamOperatorClass(ClassLoader classLoader) {
|
||||
return StreamReadOperator.class;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,417 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.streamer;
|
||||
|
||||
import org.apache.flink.runtime.state.StateBackend;
|
||||
import org.apache.flink.runtime.state.hashmap.HashMapStateBackend;
|
||||
import org.apache.hudi.client.utils.OperationConverter;
|
||||
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorType;
|
||||
import org.apache.hudi.util.FlinkStateBackendConverter;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import com.beust.jcommander.Parameter;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.apache.hudi.configuration.FlinkOptions.PARTITION_FORMAT_DAY;
|
||||
|
||||
/**
|
||||
* Configurations for Hoodie Flink streamer.
|
||||
*/
|
||||
public class FlinkStreamerConfig extends Configuration {
|
||||
@Parameter(names = {"--kafka-topic"}, description = "Kafka topic name.", required = true)
|
||||
public String kafkaTopic;
|
||||
|
||||
@Parameter(names = {"--kafka-group-id"}, description = "Kafka consumer group id.", required = true)
|
||||
public String kafkaGroupId;
|
||||
|
||||
@Parameter(names = {"--kafka-bootstrap-servers"}, description = "Kafka bootstrap.servers.", required = true)
|
||||
public String kafkaBootstrapServers;
|
||||
|
||||
@Parameter(names = {"--flink-checkpoint-path"}, description = "Flink checkpoint path.")
|
||||
public String flinkCheckPointPath;
|
||||
|
||||
@Parameter(names = {"--flink-state-backend-type"}, description = "Flink state backend type, support only hashmap and rocksdb by now,"
|
||||
+ " default hashmap.", converter = FlinkStateBackendConverter.class)
|
||||
public StateBackend stateBackend = new HashMapStateBackend();
|
||||
|
||||
@Parameter(names = {"--instant-retry-times"}, description = "Times to retry when latest instant has not completed.")
|
||||
public String instantRetryTimes = "10";
|
||||
|
||||
@Parameter(names = {"--instant-retry-interval"}, description = "Seconds between two tries when latest instant has not completed.")
|
||||
public String instantRetryInterval = "1";
|
||||
|
||||
@Parameter(names = {"--target-base-path"},
|
||||
description = "Base path for the target hoodie table. "
|
||||
+ "(Will be created if did not exist first time around. If exists, expected to be a hoodie table).",
|
||||
required = true)
|
||||
public String targetBasePath;
|
||||
|
||||
@Parameter(names = {"--target-table"}, description = "Name of the target table in Hive.", required = true)
|
||||
public String targetTableName;
|
||||
|
||||
@Parameter(names = {"--table-type"}, description = "Type of table. COPY_ON_WRITE (or) MERGE_ON_READ.", required = true)
|
||||
public String tableType;
|
||||
|
||||
@Parameter(names = {"--insert-cluster"}, description = "Whether to merge small files for insert mode, "
|
||||
+ "if true, the write throughput will decrease because the read/write of existing small file, default false.")
|
||||
public Boolean insertCluster = false;
|
||||
|
||||
@Parameter(names = {"--props"}, description = "Path to properties file on localfs or dfs, with configurations for "
|
||||
+ "hoodie client, schema provider, key generator and data source. For hoodie client props, sane defaults are "
|
||||
+ "used, but recommend use to provide basic things like metrics endpoints, hive configs etc. For sources, refer"
|
||||
+ "to individual classes, for supported properties.")
|
||||
public String propsFilePath = "";
|
||||
|
||||
@Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file "
|
||||
+ "(using the CLI parameter \"--props\") can also be passed command line using this parameter.")
|
||||
public List<String> configs = new ArrayList<>();
|
||||
|
||||
@Parameter(names = {"--record-key-field"}, description = "Record key field. Value to be used as the `recordKey` component of `HoodieKey`.\n"
|
||||
+ "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using "
|
||||
+ "the dot notation eg: `a.b.c`. By default `uuid`.")
|
||||
public String recordKeyField = "uuid";
|
||||
|
||||
@Parameter(names = {"--partition-path-field"}, description = "Partition path field. Value to be used at \n"
|
||||
+ "the `partitionPath` component of `HoodieKey`. Actual value obtained by invoking .toString(). By default `partitionpath`.")
|
||||
public String partitionPathField = "partitionpath";
|
||||
|
||||
@Parameter(names = {"--keygen-class"}, description = "Key generator class, that implements will extract the key out of incoming record.")
|
||||
public String keygenClass;
|
||||
|
||||
@Parameter(names = {"--keygen-type"}, description = "Key generator type, that implements will extract the key out of incoming record \n"
|
||||
+ "By default `SIMPLE`.")
|
||||
public String keygenType = KeyGeneratorType.SIMPLE.name();
|
||||
|
||||
@Parameter(names = {"--source-ordering-field"}, description = "Field within source record to decide how"
|
||||
+ " to break ties between records with same key in input data. Default: 'ts' holding unix timestamp of record.")
|
||||
public String sourceOrderingField = "ts";
|
||||
|
||||
@Parameter(names = {"--payload-class"}, description = "Subclass of HoodieRecordPayload, that works off "
|
||||
+ "a GenericRecord. Implement your own, if you want to do something other than overwriting existing value.")
|
||||
public String payloadClassName = OverwriteWithLatestAvroPayload.class.getName();
|
||||
|
||||
@Parameter(names = {"--op"}, description = "Takes one of these values : UPSERT (default), INSERT (use when input "
|
||||
+ "is purely new data/inserts to gain speed).", converter = OperationConverter.class)
|
||||
public WriteOperationType operation = WriteOperationType.UPSERT;
|
||||
|
||||
@Parameter(names = {"--filter-dupes"},
|
||||
description = "Should duplicate records from source be dropped/filtered out before insert/bulk-insert.")
|
||||
public Boolean preCombine = false;
|
||||
|
||||
@Parameter(names = {"--commit-on-errors"}, description = "Commit even when some records failed to be written.")
|
||||
public Boolean commitOnErrors = false;
|
||||
|
||||
@Parameter(names = {"--transformer-class"},
|
||||
description = "A subclass or a list of subclasses of org.apache.hudi.sink.transform.Transformer"
|
||||
+ ". Allows transforming raw source DataStream to a target DataStream (conforming to target schema) before "
|
||||
+ "writing. Default : Not set. Pass a comma-separated list of subclass names to chain the transformations.")
|
||||
public List<String> transformerClassNames = null;
|
||||
|
||||
@Parameter(names = {"--metadata-enabled"}, description = "Enable the internal metadata table which serves table metadata like level file listings, default false.")
|
||||
public Boolean metadataEnabled = false;
|
||||
|
||||
@Parameter(names = {"--metadata-compaction-delta_commits"}, description = "Max delta commits for metadata table to trigger compaction, default 10.")
|
||||
public Integer metadataCompactionDeltaCommits = 10;
|
||||
|
||||
@Parameter(names = {"--write-partition-format"}, description = "Partition path format, default is 'yyyyMMdd'.")
|
||||
public String writePartitionFormat = PARTITION_FORMAT_DAY;
|
||||
|
||||
@Parameter(names = {"--write-rate-limit"}, description = "Write record rate limit per second to prevent traffic jitter and improve stability, default 0 (no limit).")
|
||||
public Long writeRateLimit = 0L;
|
||||
|
||||
@Parameter(names = {"--write-parquet-block-size"}, description = "Parquet RowGroup size. It's recommended to make this large enough that scan costs can be"
|
||||
+ " amortized by packing enough column values into a single row group.")
|
||||
public Integer writeParquetBlockSize = 120;
|
||||
|
||||
@Parameter(names = {"--write-parquet-max-file-size"}, description = "Target size for parquet files produced by Hudi write phases. "
|
||||
+ "For DFS, this needs to be aligned with the underlying filesystem block size for optimal performance.")
|
||||
public Integer writeParquetMaxFileSize = 120;
|
||||
|
||||
@Parameter(names = {"--parquet-page-size"}, description = "Parquet page size. Page is the unit of read within a parquet file. "
|
||||
+ "Within a block, pages are compressed separately.")
|
||||
public Integer parquetPageSize = 1;
|
||||
|
||||
/**
|
||||
* Flink checkpoint interval.
|
||||
*/
|
||||
@Parameter(names = {"--checkpoint-interval"}, description = "Flink checkpoint interval.")
|
||||
public Long checkpointInterval = 1000 * 5L;
|
||||
|
||||
@Parameter(names = {"--help", "-h"}, help = true)
|
||||
public Boolean help = false;
|
||||
|
||||
@Parameter(names = {"--index-bootstrap-num"}, description = "Parallelism of tasks that do bucket assign, default is 4.")
|
||||
public Integer indexBootstrapNum = 4;
|
||||
|
||||
@Parameter(names = {"--bucket-assign-num"}, description = "Parallelism of tasks that do bucket assign, default is 4.")
|
||||
public Integer bucketAssignNum = 4;
|
||||
|
||||
@Parameter(names = {"--write-task-num"}, description = "Parallelism of tasks that do actual write, default is 4.")
|
||||
public Integer writeTaskNum = 4;
|
||||
|
||||
@Parameter(names = {"--partition-default-name"},
|
||||
description = "The default partition name in case the dynamic partition column value is null/empty string")
|
||||
public String partitionDefaultName = "default";
|
||||
|
||||
@Parameter(names = {"--index-bootstrap-enabled"},
|
||||
description = "Whether to bootstrap the index state from existing hoodie table, default false")
|
||||
public Boolean indexBootstrapEnabled = false;
|
||||
|
||||
@Parameter(names = {"--index-state-ttl"}, description = "Index state ttl in days, default stores the index permanently")
|
||||
public Double indexStateTtl = 0D;
|
||||
|
||||
@Parameter(names = {"--index-global-enabled"}, description = "Whether to update index for the old partition path "
|
||||
+ "if same key record with different partition path came in, default true")
|
||||
public Boolean indexGlobalEnabled = true;
|
||||
|
||||
@Parameter(names = {"--index-partition-regex"},
|
||||
description = "Whether to load partitions in state if partition path matching, default *")
|
||||
public String indexPartitionRegex = ".*";
|
||||
|
||||
@Parameter(names = {"--source-avro-schema-path"}, description = "Source avro schema file path, the parsed schema is used for deserialization")
|
||||
public String sourceAvroSchemaPath = "";
|
||||
|
||||
@Parameter(names = {"--source-avro-schema"}, description = "Source avro schema string, the parsed schema is used for deserialization")
|
||||
public String sourceAvroSchema = "";
|
||||
|
||||
@Parameter(names = {"--utc-timezone"}, description = "Use UTC timezone or local timezone to the conversion between epoch"
|
||||
+ " time and LocalDateTime. Hive 0.x/1.x/2.x use local timezone. But Hive 3.x"
|
||||
+ " use UTC timezone, by default true")
|
||||
public Boolean utcTimezone = true;
|
||||
|
||||
@Parameter(names = {"--write-partition-url-encode"}, description = "Whether to encode the partition path url, default false")
|
||||
public Boolean writePartitionUrlEncode = false;
|
||||
|
||||
@Parameter(names = {"--hive-style-partitioning"}, description = "Whether to use Hive style partitioning.\n"
|
||||
+ "If set true, the names of partition folders follow <partition_column_name>=<partition_value> format.\n"
|
||||
+ "By default false (the names of partition folders are only partition values)")
|
||||
public Boolean hiveStylePartitioning = false;
|
||||
|
||||
@Parameter(names = {"--write-task-max-size"}, description = "Maximum memory in MB for a write task, when the threshold hits,\n"
|
||||
+ "it flushes the max size data bucket to avoid OOM, default 1GB")
|
||||
public Double writeTaskMaxSize = 1024D;
|
||||
|
||||
@Parameter(names = {"--write-batch-size"},
|
||||
description = "Batch buffer size in MB to flush data into the underneath filesystem, default 256MB")
|
||||
public Double writeBatchSize = 256D;
|
||||
|
||||
@Parameter(names = {"--write-log-block-size"}, description = "Max log block size in MB for log file, default 128MB")
|
||||
public Integer writeLogBlockSize = 128;
|
||||
|
||||
@Parameter(names = {"--write-log-max-size"},
|
||||
description = "Maximum size allowed in MB for a log file before it is rolled over to the next version, default 1GB")
|
||||
public Integer writeLogMaxSize = 1024;
|
||||
|
||||
@Parameter(names = {"--write-merge-max-memory"}, description = "Max memory in MB for merge, default 100MB")
|
||||
public Integer writeMergeMaxMemory = 100;
|
||||
|
||||
@Parameter(names = {"--compaction-async-enabled"}, description = "Async Compaction, enabled by default for MOR")
|
||||
public Boolean compactionAsyncEnabled = true;
|
||||
|
||||
@Parameter(names = {"--compaction-tasks"}, description = "Parallelism of tasks that do actual compaction, default is 10")
|
||||
public Integer compactionTasks = 10;
|
||||
|
||||
@Parameter(names = {"--compaction-trigger-strategy"},
|
||||
description = "Strategy to trigger compaction, options are 'num_commits': trigger compaction when reach N delta commits;\n"
|
||||
+ "'time_elapsed': trigger compaction when time elapsed > N seconds since last compaction;\n"
|
||||
+ "'num_and_time': trigger compaction when both NUM_COMMITS and TIME_ELAPSED are satisfied;\n"
|
||||
+ "'num_or_time': trigger compaction when NUM_COMMITS or TIME_ELAPSED is satisfied.\n"
|
||||
+ "Default is 'num_commits'")
|
||||
public String compactionTriggerStrategy = FlinkOptions.NUM_COMMITS;
|
||||
|
||||
@Parameter(names = {"--compaction-delta-commits"}, description = "Max delta commits needed to trigger compaction, default 5 commits")
|
||||
public Integer compactionDeltaCommits = 5;
|
||||
|
||||
@Parameter(names = {"--compaction-delta-seconds"}, description = "Max delta seconds time needed to trigger compaction, default 1 hour")
|
||||
public Integer compactionDeltaSeconds = 3600;
|
||||
|
||||
@Parameter(names = {"--compaction-max-memory"}, description = "Max memory in MB for compaction spillable map, default 100MB")
|
||||
public Integer compactionMaxMemory = 100;
|
||||
|
||||
@Parameter(names = {"--compaction-target-io"}, description = "Target IO per compaction (both read and write), default 500 GB")
|
||||
public Long compactionTargetIo = 512000L;
|
||||
|
||||
@Parameter(names = {"--clean-async-enabled"}, description = "Whether to cleanup the old commits immediately on new commits, enabled by default")
|
||||
public Boolean cleanAsyncEnabled = true;
|
||||
|
||||
@Parameter(names = {"--clean-retain-commits"},
|
||||
description = "Number of commits to retain. So data will be retained for num_of_commits * time_between_commits (scheduled).\n"
|
||||
+ "This also directly translates into how much you can incrementally pull on this table, default 10")
|
||||
public Integer cleanRetainCommits = 10;
|
||||
|
||||
@Parameter(names = {"--archive-max-commits"},
|
||||
description = "Max number of commits to keep before archiving older commits into a sequential log, default 30")
|
||||
public Integer archiveMaxCommits = 30;
|
||||
|
||||
@Parameter(names = {"--archive-min-commits"},
|
||||
description = "Min number of commits to keep before archiving older commits into a sequential log, default 20")
|
||||
public Integer archiveMinCommits = 20;
|
||||
|
||||
@Parameter(names = {"--hive-sync-enable"}, description = "Asynchronously sync Hive meta to HMS, default false")
|
||||
public Boolean hiveSyncEnabled = false;
|
||||
|
||||
@Parameter(names = {"--hive-sync-db"}, description = "Database name for hive sync, default 'default'")
|
||||
public String hiveSyncDb = "default";
|
||||
|
||||
@Parameter(names = {"--hive-sync-table"}, description = "Table name for hive sync, default 'unknown'")
|
||||
public String hiveSyncTable = "unknown";
|
||||
|
||||
@Parameter(names = {"--hive-sync-file-format"}, description = "File format for hive sync, default 'PARQUET'")
|
||||
public String hiveSyncFileFormat = "PARQUET";
|
||||
|
||||
@Parameter(names = {"--hive-sync-mode"}, description = "Mode to choose for Hive ops. Valid values are hms, jdbc and hiveql, default 'jdbc'")
|
||||
public String hiveSyncMode = "jdbc";
|
||||
|
||||
@Parameter(names = {"--hive-sync-username"}, description = "Username for hive sync, default 'hive'")
|
||||
public String hiveSyncUsername = "hive";
|
||||
|
||||
@Parameter(names = {"--hive-sync-password"}, description = "Password for hive sync, default 'hive'")
|
||||
public String hiveSyncPassword = "hive";
|
||||
|
||||
@Parameter(names = {"--hive-sync-jdbc-url"}, description = "Jdbc URL for hive sync, default 'jdbc:hive2://localhost:10000'")
|
||||
public String hiveSyncJdbcUrl = "jdbc:hive2://localhost:10000";
|
||||
|
||||
@Parameter(names = {"--hive-sync-metastore-uris"}, description = "Metastore uris for hive sync, default ''")
|
||||
public String hiveSyncMetastoreUri = "";
|
||||
|
||||
@Parameter(names = {"--hive-sync-partition-fields"}, description = "Partition fields for hive sync, default ''")
|
||||
public String hiveSyncPartitionFields = "";
|
||||
|
||||
@Parameter(names = {"--hive-sync-partition-extractor-class"}, description = "Tool to extract the partition value from HDFS path, "
|
||||
+ "default 'SlashEncodedDayPartitionValueExtractor'")
|
||||
public String hiveSyncPartitionExtractorClass = SlashEncodedDayPartitionValueExtractor.class.getCanonicalName();
|
||||
|
||||
@Parameter(names = {"--hive-sync-assume-date-partitioning"}, description = "Assume partitioning is yyyy/mm/dd, default false")
|
||||
public Boolean hiveSyncAssumeDatePartition = false;
|
||||
|
||||
@Parameter(names = {"--hive-sync-use-jdbc"}, description = "Use JDBC when hive synchronization is enabled, default true")
|
||||
public Boolean hiveSyncUseJdbc = true;
|
||||
|
||||
@Parameter(names = {"--hive-sync-auto-create-db"}, description = "Auto create hive database if it does not exists, default true")
|
||||
public Boolean hiveSyncAutoCreateDb = true;
|
||||
|
||||
@Parameter(names = {"--hive-sync-ignore-exceptions"}, description = "Ignore exceptions during hive synchronization, default false")
|
||||
public Boolean hiveSyncIgnoreExceptions = false;
|
||||
|
||||
@Parameter(names = {"--hive-sync-skip-ro-suffix"}, description = "Skip the _ro suffix for Read optimized table when registering, default false")
|
||||
public Boolean hiveSyncSkipRoSuffix = false;
|
||||
|
||||
@Parameter(names = {"--hive-sync-support-timestamp"}, description = "INT64 with original type TIMESTAMP_MICROS is converted to hive timestamp type.\n"
|
||||
+ "Disabled by default for backward compatibility.")
|
||||
public Boolean hiveSyncSupportTimestamp = false;
|
||||
|
||||
|
||||
/**
|
||||
* Transforms a {@code HoodieFlinkStreamer.Config} into {@code Configuration}.
|
||||
* The latter is more suitable for the table APIs. It reads all the properties
|
||||
* in the properties file (set by `--props` option) and cmd line options
|
||||
* (set by `--hoodie-conf` option).
|
||||
*/
|
||||
@SuppressWarnings("unchecked, rawtypes")
|
||||
public static org.apache.flink.configuration.Configuration toFlinkConfig(FlinkStreamerConfig config) {
|
||||
Map<String, String> propsMap = new HashMap<String, String>((Map) StreamerUtil.getProps(config));
|
||||
org.apache.flink.configuration.Configuration conf = fromMap(propsMap);
|
||||
|
||||
conf.setString(FlinkOptions.PATH, config.targetBasePath);
|
||||
conf.setString(FlinkOptions.TABLE_NAME, config.targetTableName);
|
||||
// copy_on_write works same as COPY_ON_WRITE
|
||||
conf.setString(FlinkOptions.TABLE_TYPE, config.tableType.toUpperCase());
|
||||
conf.setBoolean(FlinkOptions.INSERT_CLUSTER, config.insertCluster);
|
||||
conf.setString(FlinkOptions.OPERATION, config.operation.value());
|
||||
conf.setString(FlinkOptions.PRECOMBINE_FIELD, config.sourceOrderingField);
|
||||
conf.setString(FlinkOptions.PAYLOAD_CLASS_NAME, config.payloadClassName);
|
||||
conf.setBoolean(FlinkOptions.PRE_COMBINE, config.preCombine);
|
||||
conf.setInteger(FlinkOptions.RETRY_TIMES, Integer.parseInt(config.instantRetryTimes));
|
||||
conf.setLong(FlinkOptions.RETRY_INTERVAL_MS, Long.parseLong(config.instantRetryInterval));
|
||||
conf.setBoolean(FlinkOptions.IGNORE_FAILED, config.commitOnErrors);
|
||||
conf.setString(FlinkOptions.RECORD_KEY_FIELD, config.recordKeyField);
|
||||
conf.setString(FlinkOptions.PARTITION_PATH_FIELD, config.partitionPathField);
|
||||
conf.setBoolean(FlinkOptions.METADATA_ENABLED, config.metadataEnabled);
|
||||
conf.setInteger(FlinkOptions.METADATA_COMPACTION_DELTA_COMMITS, config.metadataCompactionDeltaCommits);
|
||||
conf.setString(FlinkOptions.PARTITION_FORMAT, config.writePartitionFormat);
|
||||
conf.setLong(FlinkOptions.WRITE_RATE_LIMIT, config.writeRateLimit);
|
||||
conf.setInteger(FlinkOptions.WRITE_PARQUET_BLOCK_SIZE, config.writeParquetBlockSize);
|
||||
conf.setInteger(FlinkOptions.WRITE_PARQUET_MAX_FILE_SIZE, config.writeParquetMaxFileSize);
|
||||
conf.setInteger(FlinkOptions.WRITE_PARQUET_PAGE_SIZE, config.parquetPageSize);
|
||||
if (!StringUtils.isNullOrEmpty(config.keygenClass)) {
|
||||
conf.setString(FlinkOptions.KEYGEN_CLASS_NAME, config.keygenClass);
|
||||
} else {
|
||||
conf.setString(FlinkOptions.KEYGEN_TYPE, config.keygenType);
|
||||
}
|
||||
conf.setInteger(FlinkOptions.INDEX_BOOTSTRAP_TASKS, config.indexBootstrapNum);
|
||||
conf.setInteger(FlinkOptions.BUCKET_ASSIGN_TASKS, config.bucketAssignNum);
|
||||
conf.setInteger(FlinkOptions.WRITE_TASKS, config.writeTaskNum);
|
||||
conf.setString(FlinkOptions.PARTITION_DEFAULT_NAME, config.partitionDefaultName);
|
||||
conf.setBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED, config.indexBootstrapEnabled);
|
||||
conf.setDouble(FlinkOptions.INDEX_STATE_TTL, config.indexStateTtl);
|
||||
conf.setBoolean(FlinkOptions.INDEX_GLOBAL_ENABLED, config.indexGlobalEnabled);
|
||||
conf.setString(FlinkOptions.INDEX_PARTITION_REGEX, config.indexPartitionRegex);
|
||||
if (!StringUtils.isNullOrEmpty(config.sourceAvroSchemaPath)) {
|
||||
conf.setString(FlinkOptions.SOURCE_AVRO_SCHEMA_PATH, config.sourceAvroSchemaPath);
|
||||
}
|
||||
conf.setString(FlinkOptions.SOURCE_AVRO_SCHEMA, config.sourceAvroSchema);
|
||||
conf.setBoolean(FlinkOptions.UTC_TIMEZONE, config.utcTimezone);
|
||||
conf.setBoolean(FlinkOptions.URL_ENCODE_PARTITIONING, config.writePartitionUrlEncode);
|
||||
conf.setBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING, config.hiveStylePartitioning);
|
||||
conf.setDouble(FlinkOptions.WRITE_TASK_MAX_SIZE, config.writeTaskMaxSize);
|
||||
conf.setDouble(FlinkOptions.WRITE_BATCH_SIZE, config.writeBatchSize);
|
||||
conf.setInteger(FlinkOptions.WRITE_LOG_BLOCK_SIZE, config.writeLogBlockSize);
|
||||
conf.setLong(FlinkOptions.WRITE_LOG_MAX_SIZE, config.writeLogMaxSize);
|
||||
conf.setInteger(FlinkOptions.WRITE_MERGE_MAX_MEMORY, config.writeMergeMaxMemory);
|
||||
conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, config.compactionAsyncEnabled);
|
||||
conf.setInteger(FlinkOptions.COMPACTION_TASKS, config.compactionTasks);
|
||||
conf.setString(FlinkOptions.COMPACTION_TRIGGER_STRATEGY, config.compactionTriggerStrategy);
|
||||
conf.setInteger(FlinkOptions.COMPACTION_DELTA_COMMITS, config.compactionDeltaCommits);
|
||||
conf.setInteger(FlinkOptions.COMPACTION_DELTA_SECONDS, config.compactionDeltaSeconds);
|
||||
conf.setInteger(FlinkOptions.COMPACTION_MAX_MEMORY, config.compactionMaxMemory);
|
||||
conf.setLong(FlinkOptions.COMPACTION_TARGET_IO, config.compactionTargetIo);
|
||||
conf.setBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED, config.cleanAsyncEnabled);
|
||||
conf.setInteger(FlinkOptions.CLEAN_RETAIN_COMMITS, config.cleanRetainCommits);
|
||||
conf.setInteger(FlinkOptions.ARCHIVE_MAX_COMMITS, config.archiveMaxCommits);
|
||||
conf.setInteger(FlinkOptions.ARCHIVE_MIN_COMMITS, config.archiveMinCommits);
|
||||
conf.setBoolean(FlinkOptions.HIVE_SYNC_ENABLED, config.hiveSyncEnabled);
|
||||
conf.setString(FlinkOptions.HIVE_SYNC_DB, config.hiveSyncDb);
|
||||
conf.setString(FlinkOptions.HIVE_SYNC_TABLE, config.hiveSyncTable);
|
||||
conf.setString(FlinkOptions.HIVE_SYNC_FILE_FORMAT, config.hiveSyncFileFormat);
|
||||
conf.setString(FlinkOptions.HIVE_SYNC_MODE, config.hiveSyncMode);
|
||||
conf.setString(FlinkOptions.HIVE_SYNC_USERNAME, config.hiveSyncUsername);
|
||||
conf.setString(FlinkOptions.HIVE_SYNC_PASSWORD, config.hiveSyncPassword);
|
||||
conf.setString(FlinkOptions.HIVE_SYNC_JDBC_URL, config.hiveSyncJdbcUrl);
|
||||
conf.setString(FlinkOptions.HIVE_SYNC_METASTORE_URIS, config.hiveSyncMetastoreUri);
|
||||
conf.setString(FlinkOptions.HIVE_SYNC_PARTITION_FIELDS, config.hiveSyncPartitionFields);
|
||||
conf.setString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME, config.hiveSyncPartitionExtractorClass);
|
||||
conf.setBoolean(FlinkOptions.HIVE_SYNC_ASSUME_DATE_PARTITION, config.hiveSyncAssumeDatePartition);
|
||||
conf.setBoolean(FlinkOptions.HIVE_SYNC_USE_JDBC, config.hiveSyncUseJdbc);
|
||||
conf.setBoolean(FlinkOptions.HIVE_SYNC_AUTO_CREATE_DB, config.hiveSyncAutoCreateDb);
|
||||
conf.setBoolean(FlinkOptions.HIVE_SYNC_IGNORE_EXCEPTIONS, config.hiveSyncIgnoreExceptions);
|
||||
conf.setBoolean(FlinkOptions.HIVE_SYNC_SKIP_RO_SUFFIX, config.hiveSyncSkipRoSuffix);
|
||||
conf.setBoolean(FlinkOptions.HIVE_SYNC_SUPPORT_TIMESTAMP, config.hiveSyncSupportTimestamp);
|
||||
return conf;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,110 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.streamer;
|
||||
|
||||
import org.apache.hudi.common.config.DFSPropertiesConfiguration;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.sink.transform.Transformer;
|
||||
import org.apache.hudi.sink.utils.Pipelines;
|
||||
import org.apache.hudi.util.AvroSchemaConverter;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import com.beust.jcommander.JCommander;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.formats.common.TimestampFormat;
|
||||
import org.apache.flink.formats.json.JsonRowDataDeserializationSchema;
|
||||
import org.apache.flink.streaming.api.datastream.DataStream;
|
||||
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
|
||||
import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
|
||||
/**
|
||||
* A utility which can incrementally consume data from Kafka and apply it to the target table.
|
||||
* It has the similar functionality with SQL data source except that the source is bind to Kafka
|
||||
* and the format is bind to JSON.
|
||||
*/
|
||||
public class HoodieFlinkStreamer {
|
||||
public static void main(String[] args) throws Exception {
|
||||
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
|
||||
|
||||
final FlinkStreamerConfig cfg = new FlinkStreamerConfig();
|
||||
JCommander cmd = new JCommander(cfg, null, args);
|
||||
if (cfg.help || args.length == 0) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
env.enableCheckpointing(cfg.checkpointInterval);
|
||||
env.getConfig().setGlobalJobParameters(cfg);
|
||||
// We use checkpoint to trigger write operation, including instant generating and committing,
|
||||
// There can only be one checkpoint at one time.
|
||||
env.getCheckpointConfig().setMaxConcurrentCheckpoints(1);
|
||||
|
||||
env.setStateBackend(cfg.stateBackend);
|
||||
if (cfg.flinkCheckPointPath != null) {
|
||||
env.getCheckpointConfig().setCheckpointStorage(cfg.flinkCheckPointPath);
|
||||
}
|
||||
|
||||
TypedProperties kafkaProps = DFSPropertiesConfiguration.getGlobalProps();
|
||||
kafkaProps.putAll(StreamerUtil.appendKafkaProps(cfg));
|
||||
|
||||
// Read from kafka source
|
||||
RowType rowType =
|
||||
(RowType) AvroSchemaConverter.convertToDataType(StreamerUtil.getSourceSchema(cfg))
|
||||
.getLogicalType();
|
||||
|
||||
Configuration conf = FlinkStreamerConfig.toFlinkConfig(cfg);
|
||||
long ckpTimeout = env.getCheckpointConfig().getCheckpointTimeout();
|
||||
int parallelism = env.getParallelism();
|
||||
conf.setLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT, ckpTimeout);
|
||||
|
||||
DataStream<RowData> dataStream = env.addSource(new FlinkKafkaConsumer<>(
|
||||
cfg.kafkaTopic,
|
||||
new JsonRowDataDeserializationSchema(
|
||||
rowType,
|
||||
InternalTypeInfo.of(rowType),
|
||||
false,
|
||||
true,
|
||||
TimestampFormat.ISO_8601
|
||||
), kafkaProps))
|
||||
.name("kafka_source")
|
||||
.uid("uid_kafka_source");
|
||||
|
||||
if (cfg.transformerClassNames != null && !cfg.transformerClassNames.isEmpty()) {
|
||||
Option<Transformer> transformer = StreamerUtil.createTransformer(cfg.transformerClassNames);
|
||||
if (transformer.isPresent()) {
|
||||
dataStream = transformer.get().apply(dataStream);
|
||||
}
|
||||
}
|
||||
|
||||
DataStream<HoodieRecord> hoodieRecordDataStream = Pipelines.bootstrap(conf, rowType, parallelism, dataStream);
|
||||
DataStream<Object> pipeline = Pipelines.hoodieStreamWrite(conf, parallelism, hoodieRecordDataStream);
|
||||
if (StreamerUtil.needsAsyncCompaction(conf)) {
|
||||
Pipelines.compact(conf, pipeline);
|
||||
} else {
|
||||
Pipelines.clean(conf, pipeline);
|
||||
}
|
||||
|
||||
env.execute(cfg.targetTableName);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,327 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table;
|
||||
|
||||
import org.apache.hudi.common.model.DefaultHoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.EventTimeAvroPayload;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.configuration.OptionsResolver;
|
||||
import org.apache.hudi.exception.HoodieValidationException;
|
||||
import org.apache.hudi.hive.MultiPartKeysValueExtractor;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.keygen.ComplexAvroKeyGenerator;
|
||||
import org.apache.hudi.keygen.NonpartitionedAvroKeyGenerator;
|
||||
import org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
|
||||
import org.apache.hudi.util.AvroSchemaConverter;
|
||||
import org.apache.hudi.util.DataTypeUtils;
|
||||
|
||||
import org.apache.flink.configuration.ConfigOption;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.table.api.ValidationException;
|
||||
import org.apache.flink.table.api.constraints.UniqueConstraint;
|
||||
import org.apache.flink.table.catalog.CatalogTable;
|
||||
import org.apache.flink.table.catalog.ResolvedSchema;
|
||||
import org.apache.flink.table.connector.sink.DynamicTableSink;
|
||||
import org.apache.flink.table.connector.source.DynamicTableSource;
|
||||
import org.apache.flink.table.factories.DynamicTableSinkFactory;
|
||||
import org.apache.flink.table.factories.DynamicTableSourceFactory;
|
||||
import org.apache.flink.table.types.DataType;
|
||||
import org.apache.flink.table.types.logical.LogicalType;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Hoodie data source/sink factory.
|
||||
*/
|
||||
public class HoodieTableFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(HoodieTableFactory.class);
|
||||
|
||||
public static final String FACTORY_ID = "hudi";
|
||||
|
||||
@Override
|
||||
public DynamicTableSource createDynamicTableSource(Context context) {
|
||||
Configuration conf = FlinkOptions.fromMap(context.getCatalogTable().getOptions());
|
||||
ResolvedSchema schema = context.getCatalogTable().getResolvedSchema();
|
||||
sanityCheck(conf, schema);
|
||||
setupConfOptions(conf, context.getObjectIdentifier().getObjectName(), context.getCatalogTable(), schema);
|
||||
|
||||
Path path = new Path(conf.getOptional(FlinkOptions.PATH).orElseThrow(() ->
|
||||
new ValidationException("Option [path] should not be empty.")));
|
||||
return new HoodieTableSource(
|
||||
schema,
|
||||
path,
|
||||
context.getCatalogTable().getPartitionKeys(),
|
||||
conf.getString(FlinkOptions.PARTITION_DEFAULT_NAME),
|
||||
conf);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DynamicTableSink createDynamicTableSink(Context context) {
|
||||
Configuration conf = FlinkOptions.fromMap(context.getCatalogTable().getOptions());
|
||||
ResolvedSchema schema = context.getCatalogTable().getResolvedSchema();
|
||||
sanityCheck(conf, schema);
|
||||
setupConfOptions(conf, context.getObjectIdentifier().getObjectName(), context.getCatalogTable(), schema);
|
||||
return new HoodieTableSink(conf, schema);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String factoryIdentifier() {
|
||||
return FACTORY_ID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<ConfigOption<?>> requiredOptions() {
|
||||
return Collections.singleton(FlinkOptions.PATH);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<ConfigOption<?>> optionalOptions() {
|
||||
return FlinkOptions.optionalOptions();
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Utilities
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
/**
|
||||
* The sanity check.
|
||||
*
|
||||
* @param conf The table options
|
||||
* @param schema The table schema
|
||||
*/
|
||||
private void sanityCheck(Configuration conf, ResolvedSchema schema) {
|
||||
List<String> fields = schema.getColumnNames();
|
||||
|
||||
// validate record key in pk absence.
|
||||
if (!schema.getPrimaryKey().isPresent()) {
|
||||
String[] recordKeys = conf.get(FlinkOptions.RECORD_KEY_FIELD).split(",");
|
||||
if (recordKeys.length == 1
|
||||
&& FlinkOptions.RECORD_KEY_FIELD.defaultValue().equals(recordKeys[0])
|
||||
&& !fields.contains(recordKeys[0])) {
|
||||
throw new HoodieValidationException("Primary key definition is required, use either PRIMARY KEY syntax "
|
||||
+ "or option '" + FlinkOptions.RECORD_KEY_FIELD.key() + "' to specify.");
|
||||
}
|
||||
|
||||
Arrays.stream(recordKeys)
|
||||
.filter(field -> !fields.contains(field))
|
||||
.findAny()
|
||||
.ifPresent(f -> {
|
||||
throw new HoodieValidationException("Field '" + f + "' specified in option "
|
||||
+ "'" + FlinkOptions.RECORD_KEY_FIELD.key() + "' does not exist in the table schema.");
|
||||
});
|
||||
}
|
||||
|
||||
// validate pre_combine key
|
||||
String preCombineField = conf.get(FlinkOptions.PRECOMBINE_FIELD);
|
||||
if (!fields.contains(preCombineField)) {
|
||||
if (OptionsResolver.isDefaultHoodieRecordPayloadClazz(conf)) {
|
||||
throw new HoodieValidationException("Option '" + FlinkOptions.PRECOMBINE_FIELD.key()
|
||||
+ "' is required for payload class: " + DefaultHoodieRecordPayload.class.getName());
|
||||
}
|
||||
if (preCombineField.equals(FlinkOptions.PRECOMBINE_FIELD.defaultValue())) {
|
||||
conf.setString(FlinkOptions.PRECOMBINE_FIELD, FlinkOptions.NO_PRE_COMBINE);
|
||||
} else {
|
||||
throw new HoodieValidationException("Field " + preCombineField + " does not exist in the table schema."
|
||||
+ "Please check '" + FlinkOptions.PRECOMBINE_FIELD.key() + "' option.");
|
||||
}
|
||||
} else if (FlinkOptions.isDefaultValueDefined(conf, FlinkOptions.PAYLOAD_CLASS_NAME)) {
|
||||
// if precombine field is specified but payload clazz is default,
|
||||
// use DefaultHoodieRecordPayload to make sure the precombine field is always taken for
|
||||
// comparing.
|
||||
conf.setString(FlinkOptions.PAYLOAD_CLASS_NAME, EventTimeAvroPayload.class.getName());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets up the config options based on the table definition, for e.g the table name, primary key.
|
||||
*
|
||||
* @param conf The configuration to setup
|
||||
* @param tableName The table name
|
||||
* @param table The catalog table
|
||||
* @param schema The physical schema
|
||||
*/
|
||||
private static void setupConfOptions(
|
||||
Configuration conf,
|
||||
String tableName,
|
||||
CatalogTable table,
|
||||
ResolvedSchema schema) {
|
||||
// table name
|
||||
conf.setString(FlinkOptions.TABLE_NAME.key(), tableName);
|
||||
// hoodie key about options
|
||||
setupHoodieKeyOptions(conf, table);
|
||||
// compaction options
|
||||
setupCompactionOptions(conf);
|
||||
// hive options
|
||||
setupHiveOptions(conf);
|
||||
// read options
|
||||
setupReadOptions(conf);
|
||||
// write options
|
||||
setupWriteOptions(conf);
|
||||
// infer avro schema from physical DDL schema
|
||||
inferAvroSchema(conf, schema.toPhysicalRowDataType().notNull().getLogicalType());
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets up the hoodie key options (e.g. record key and partition key) from the table definition.
|
||||
*/
|
||||
private static void setupHoodieKeyOptions(Configuration conf, CatalogTable table) {
|
||||
List<String> pkColumns = table.getSchema().getPrimaryKey()
|
||||
.map(UniqueConstraint::getColumns).orElse(Collections.emptyList());
|
||||
if (pkColumns.size() > 0) {
|
||||
// the PRIMARY KEY syntax always has higher priority than option FlinkOptions#RECORD_KEY_FIELD
|
||||
String recordKey = String.join(",", pkColumns);
|
||||
conf.setString(FlinkOptions.RECORD_KEY_FIELD, recordKey);
|
||||
}
|
||||
List<String> partitionKeys = table.getPartitionKeys();
|
||||
if (partitionKeys.size() > 0) {
|
||||
// the PARTITIONED BY syntax always has higher priority than option FlinkOptions#PARTITION_PATH_FIELD
|
||||
conf.setString(FlinkOptions.PARTITION_PATH_FIELD, String.join(",", partitionKeys));
|
||||
}
|
||||
// set index key for bucket index if not defined
|
||||
if (conf.getString(FlinkOptions.INDEX_TYPE).equals(HoodieIndex.IndexType.BUCKET.name())
|
||||
&& conf.getString(FlinkOptions.INDEX_KEY_FIELD).isEmpty()) {
|
||||
conf.setString(FlinkOptions.INDEX_KEY_FIELD, conf.getString(FlinkOptions.RECORD_KEY_FIELD));
|
||||
}
|
||||
// tweak the key gen class if possible
|
||||
final String[] partitions = conf.getString(FlinkOptions.PARTITION_PATH_FIELD).split(",");
|
||||
final String[] pks = conf.getString(FlinkOptions.RECORD_KEY_FIELD).split(",");
|
||||
if (partitions.length == 1) {
|
||||
final String partitionField = partitions[0];
|
||||
if (partitionField.isEmpty()) {
|
||||
conf.setString(FlinkOptions.KEYGEN_CLASS_NAME, NonpartitionedAvroKeyGenerator.class.getName());
|
||||
LOG.info("Table option [{}] is reset to {} because this is a non-partitioned table",
|
||||
FlinkOptions.KEYGEN_CLASS_NAME.key(), NonpartitionedAvroKeyGenerator.class.getName());
|
||||
return;
|
||||
}
|
||||
DataType partitionFieldType = table.getSchema().getFieldDataType(partitionField)
|
||||
.orElseThrow(() -> new HoodieValidationException("Field " + partitionField + " does not exist"));
|
||||
if (pks.length <= 1 && DataTypeUtils.isDatetimeType(partitionFieldType)) {
|
||||
// timestamp based key gen only supports simple primary key
|
||||
setupTimestampKeygenOptions(conf, partitionFieldType);
|
||||
return;
|
||||
}
|
||||
}
|
||||
boolean complexHoodieKey = pks.length > 1 || partitions.length > 1;
|
||||
if (complexHoodieKey && FlinkOptions.isDefaultValueDefined(conf, FlinkOptions.KEYGEN_CLASS_NAME)) {
|
||||
conf.setString(FlinkOptions.KEYGEN_CLASS_NAME, ComplexAvroKeyGenerator.class.getName());
|
||||
LOG.info("Table option [{}] is reset to {} because record key or partition path has two or more fields",
|
||||
FlinkOptions.KEYGEN_CLASS_NAME.key(), ComplexAvroKeyGenerator.class.getName());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets up the keygen options when the partition path is datetime type.
|
||||
*
|
||||
* <p>The UTC timezone is used as default.
|
||||
*/
|
||||
public static void setupTimestampKeygenOptions(Configuration conf, DataType fieldType) {
|
||||
conf.setString(FlinkOptions.KEYGEN_CLASS_NAME, TimestampBasedAvroKeyGenerator.class.getName());
|
||||
LOG.info("Table option [{}] is reset to {} because datetime partitioning turns on",
|
||||
FlinkOptions.KEYGEN_CLASS_NAME.key(), TimestampBasedAvroKeyGenerator.class.getName());
|
||||
if (DataTypeUtils.isTimestampType(fieldType)) {
|
||||
int precision = DataTypeUtils.precision(fieldType.getLogicalType());
|
||||
if (precision == 0) {
|
||||
// seconds
|
||||
conf.setString(KeyGeneratorOptions.Config.TIMESTAMP_TYPE_FIELD_PROP,
|
||||
TimestampBasedAvroKeyGenerator.TimestampType.UNIX_TIMESTAMP.name());
|
||||
} else if (precision == 3) {
|
||||
// milliseconds
|
||||
conf.setString(KeyGeneratorOptions.Config.TIMESTAMP_TYPE_FIELD_PROP,
|
||||
TimestampBasedAvroKeyGenerator.TimestampType.EPOCHMILLISECONDS.name());
|
||||
}
|
||||
String partitionFormat = conf.getOptional(FlinkOptions.PARTITION_FORMAT).orElse(FlinkOptions.PARTITION_FORMAT_HOUR);
|
||||
conf.setString(KeyGeneratorOptions.Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, partitionFormat);
|
||||
} else {
|
||||
conf.setString(KeyGeneratorOptions.Config.TIMESTAMP_TYPE_FIELD_PROP,
|
||||
TimestampBasedAvroKeyGenerator.TimestampType.DATE_STRING.name());
|
||||
String partitionFormat = conf.getOptional(FlinkOptions.PARTITION_FORMAT).orElse(FlinkOptions.PARTITION_FORMAT_DAY);
|
||||
conf.setString(KeyGeneratorOptions.Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, partitionFormat);
|
||||
}
|
||||
conf.setString(KeyGeneratorOptions.Config.TIMESTAMP_OUTPUT_TIMEZONE_FORMAT_PROP, "UTC");
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets up the compaction options from the table definition.
|
||||
*/
|
||||
private static void setupCompactionOptions(Configuration conf) {
|
||||
int commitsToRetain = conf.getInteger(FlinkOptions.CLEAN_RETAIN_COMMITS);
|
||||
int minCommitsToKeep = conf.getInteger(FlinkOptions.ARCHIVE_MIN_COMMITS);
|
||||
if (commitsToRetain >= minCommitsToKeep) {
|
||||
LOG.info("Table option [{}] is reset to {} to be greater than {}={},\n"
|
||||
+ "to avoid risk of missing data from few instants in incremental pull",
|
||||
FlinkOptions.ARCHIVE_MIN_COMMITS.key(), commitsToRetain + 10,
|
||||
FlinkOptions.CLEAN_RETAIN_COMMITS.key(), commitsToRetain);
|
||||
conf.setInteger(FlinkOptions.ARCHIVE_MIN_COMMITS, commitsToRetain + 10);
|
||||
conf.setInteger(FlinkOptions.ARCHIVE_MAX_COMMITS, commitsToRetain + 20);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets up the hive options from the table definition.
|
||||
*/
|
||||
private static void setupHiveOptions(Configuration conf) {
|
||||
if (!conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING)
|
||||
&& FlinkOptions.isDefaultValueDefined(conf, FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME)) {
|
||||
conf.setString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME, MultiPartKeysValueExtractor.class.getName());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets up the read options from the table definition.
|
||||
*/
|
||||
private static void setupReadOptions(Configuration conf) {
|
||||
if (!conf.getBoolean(FlinkOptions.READ_AS_STREAMING)
|
||||
&& (conf.getOptional(FlinkOptions.READ_START_COMMIT).isPresent() || conf.getOptional(FlinkOptions.READ_END_COMMIT).isPresent())) {
|
||||
conf.setString(FlinkOptions.QUERY_TYPE, FlinkOptions.QUERY_TYPE_INCREMENTAL);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets up the write options from the table definition.
|
||||
*/
|
||||
private static void setupWriteOptions(Configuration conf) {
|
||||
if (FlinkOptions.isDefaultValueDefined(conf, FlinkOptions.OPERATION)
|
||||
&& OptionsResolver.isCowTable(conf)) {
|
||||
conf.setBoolean(FlinkOptions.PRE_COMBINE, true);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Inferences the deserialization Avro schema from the table schema (e.g. the DDL)
|
||||
* if both options {@link FlinkOptions#SOURCE_AVRO_SCHEMA_PATH} and
|
||||
* {@link FlinkOptions#SOURCE_AVRO_SCHEMA} are not specified.
|
||||
*
|
||||
* @param conf The configuration
|
||||
* @param rowType The specified table row type
|
||||
*/
|
||||
private static void inferAvroSchema(Configuration conf, LogicalType rowType) {
|
||||
if (!conf.getOptional(FlinkOptions.SOURCE_AVRO_SCHEMA_PATH).isPresent()
|
||||
&& !conf.getOptional(FlinkOptions.SOURCE_AVRO_SCHEMA).isPresent()) {
|
||||
String inferredSchema = AvroSchemaConverter.convertToSchema(rowType).toString();
|
||||
conf.setString(FlinkOptions.SOURCE_AVRO_SCHEMA, inferredSchema);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,140 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.configuration.OptionsResolver;
|
||||
import org.apache.hudi.sink.utils.Pipelines;
|
||||
import org.apache.hudi.util.ChangelogModes;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.annotation.VisibleForTesting;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.streaming.api.datastream.DataStream;
|
||||
import org.apache.flink.table.catalog.ResolvedSchema;
|
||||
import org.apache.flink.table.connector.ChangelogMode;
|
||||
import org.apache.flink.table.connector.sink.DataStreamSinkProvider;
|
||||
import org.apache.flink.table.connector.sink.DynamicTableSink;
|
||||
import org.apache.flink.table.connector.sink.abilities.SupportsOverwrite;
|
||||
import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Hoodie table sink.
|
||||
*/
|
||||
public class HoodieTableSink implements DynamicTableSink, SupportsPartitioning, SupportsOverwrite {
|
||||
|
||||
private final Configuration conf;
|
||||
private final ResolvedSchema schema;
|
||||
private boolean overwrite = false;
|
||||
|
||||
public HoodieTableSink(Configuration conf, ResolvedSchema schema) {
|
||||
this.conf = conf;
|
||||
this.schema = schema;
|
||||
}
|
||||
|
||||
public HoodieTableSink(Configuration conf, ResolvedSchema schema, boolean overwrite) {
|
||||
this.conf = conf;
|
||||
this.schema = schema;
|
||||
this.overwrite = overwrite;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
|
||||
return (DataStreamSinkProvider) dataStream -> {
|
||||
|
||||
// setup configuration
|
||||
long ckpTimeout = dataStream.getExecutionEnvironment()
|
||||
.getCheckpointConfig().getCheckpointTimeout();
|
||||
conf.setLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT, ckpTimeout);
|
||||
|
||||
RowType rowType = (RowType) schema.toSourceRowDataType().notNull().getLogicalType();
|
||||
|
||||
// bulk_insert mode
|
||||
final String writeOperation = this.conf.get(FlinkOptions.OPERATION);
|
||||
if (WriteOperationType.fromValue(writeOperation) == WriteOperationType.BULK_INSERT) {
|
||||
return context.isBounded() ? Pipelines.bulkInsert(conf, rowType, dataStream) : Pipelines.append(conf, rowType, dataStream);
|
||||
}
|
||||
|
||||
// Append mode
|
||||
if (OptionsResolver.isAppendMode(conf)) {
|
||||
return Pipelines.append(conf, rowType, dataStream);
|
||||
}
|
||||
|
||||
// default parallelism
|
||||
int parallelism = dataStream.getExecutionConfig().getParallelism();
|
||||
DataStream<Object> pipeline;
|
||||
// bootstrap
|
||||
final DataStream<HoodieRecord> hoodieRecordDataStream =
|
||||
Pipelines.bootstrap(conf, rowType, parallelism, dataStream, context.isBounded(), overwrite);
|
||||
// write pipeline
|
||||
pipeline = Pipelines.hoodieStreamWrite(conf, parallelism, hoodieRecordDataStream);
|
||||
// compaction
|
||||
if (StreamerUtil.needsAsyncCompaction(conf)) {
|
||||
return Pipelines.compact(conf, pipeline);
|
||||
} else {
|
||||
return Pipelines.clean(conf, pipeline);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public Configuration getConf() {
|
||||
return this.conf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChangelogMode getChangelogMode(ChangelogMode changelogMode) {
|
||||
if (conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED)) {
|
||||
return ChangelogModes.FULL;
|
||||
} else {
|
||||
return ChangelogModes.UPSERT;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public DynamicTableSink copy() {
|
||||
return new HoodieTableSink(this.conf, this.schema, this.overwrite);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String asSummaryString() {
|
||||
return "HoodieTableSink";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void applyStaticPartition(Map<String, String> partitions) {
|
||||
// #applyOverwrite should have been invoked.
|
||||
if (this.overwrite && partitions.size() > 0) {
|
||||
this.conf.setString(FlinkOptions.OPERATION, WriteOperationType.INSERT_OVERWRITE.value());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void applyOverwrite(boolean overwrite) {
|
||||
this.overwrite = overwrite;
|
||||
// set up the operation as INSERT_OVERWRITE_TABLE first,
|
||||
// if there are explicit partitions, #applyStaticPartition would overwrite the option.
|
||||
this.conf.setString(FlinkOptions.OPERATION, WriteOperationType.INSERT_OVERWRITE_TABLE.value());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,505 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.model.BaseFile;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.TableSchemaResolver;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.configuration.OptionsResolver;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.hadoop.HoodieROTablePathFilter;
|
||||
import org.apache.hudi.source.FileIndex;
|
||||
import org.apache.hudi.source.IncrementalInputSplits;
|
||||
import org.apache.hudi.source.StreamReadMonitoringFunction;
|
||||
import org.apache.hudi.source.StreamReadOperator;
|
||||
import org.apache.hudi.table.format.FilePathUtils;
|
||||
import org.apache.hudi.table.format.cow.CopyOnWriteInputFormat;
|
||||
import org.apache.hudi.table.format.mor.MergeOnReadInputFormat;
|
||||
import org.apache.hudi.table.format.mor.MergeOnReadInputSplit;
|
||||
import org.apache.hudi.table.format.mor.MergeOnReadTableState;
|
||||
import org.apache.hudi.util.AvroSchemaConverter;
|
||||
import org.apache.hudi.util.ChangelogModes;
|
||||
import org.apache.hudi.util.InputFormats;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.flink.annotation.VisibleForTesting;
|
||||
import org.apache.flink.api.common.io.FileInputFormat;
|
||||
import org.apache.flink.api.common.io.FilePathFilter;
|
||||
import org.apache.flink.api.common.io.InputFormat;
|
||||
import org.apache.flink.api.common.typeinfo.TypeInformation;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.streaming.api.datastream.DataStream;
|
||||
import org.apache.flink.streaming.api.datastream.DataStreamSource;
|
||||
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
|
||||
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
|
||||
import org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction;
|
||||
import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
|
||||
import org.apache.flink.table.api.DataTypes;
|
||||
import org.apache.flink.table.catalog.ResolvedSchema;
|
||||
import org.apache.flink.table.connector.ChangelogMode;
|
||||
import org.apache.flink.table.connector.source.DataStreamScanProvider;
|
||||
import org.apache.flink.table.connector.source.DynamicTableSource;
|
||||
import org.apache.flink.table.connector.source.ScanTableSource;
|
||||
import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown;
|
||||
import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown;
|
||||
import org.apache.flink.table.connector.source.abilities.SupportsPartitionPushDown;
|
||||
import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.expressions.Expression;
|
||||
import org.apache.flink.table.expressions.ResolvedExpression;
|
||||
import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter;
|
||||
import org.apache.flink.table.types.DataType;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
import static org.apache.hudi.table.format.FormatUtils.getParquetConf;
|
||||
|
||||
/**
|
||||
* Hoodie batch table source that always read the latest snapshot of the underneath table.
|
||||
*/
|
||||
public class HoodieTableSource implements
|
||||
ScanTableSource,
|
||||
SupportsPartitionPushDown,
|
||||
SupportsProjectionPushDown,
|
||||
SupportsLimitPushDown,
|
||||
SupportsFilterPushDown {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(HoodieTableSource.class);
|
||||
|
||||
private static final int NO_LIMIT_CONSTANT = -1;
|
||||
|
||||
private final transient org.apache.hadoop.conf.Configuration hadoopConf;
|
||||
private final transient HoodieTableMetaClient metaClient;
|
||||
private final long maxCompactionMemoryInBytes;
|
||||
|
||||
private final ResolvedSchema schema;
|
||||
private final Path path;
|
||||
private final List<String> partitionKeys;
|
||||
private final String defaultPartName;
|
||||
private final Configuration conf;
|
||||
private final FileIndex fileIndex;
|
||||
|
||||
private int[] requiredPos;
|
||||
private long limit;
|
||||
private List<Expression> filters;
|
||||
|
||||
private List<Map<String, String>> requiredPartitions;
|
||||
|
||||
public HoodieTableSource(
|
||||
ResolvedSchema schema,
|
||||
Path path,
|
||||
List<String> partitionKeys,
|
||||
String defaultPartName,
|
||||
Configuration conf) {
|
||||
this(schema, path, partitionKeys, defaultPartName, conf, null, null, null, null);
|
||||
}
|
||||
|
||||
public HoodieTableSource(
|
||||
ResolvedSchema schema,
|
||||
Path path,
|
||||
List<String> partitionKeys,
|
||||
String defaultPartName,
|
||||
Configuration conf,
|
||||
@Nullable List<Map<String, String>> requiredPartitions,
|
||||
@Nullable int[] requiredPos,
|
||||
@Nullable Long limit,
|
||||
@Nullable List<Expression> filters) {
|
||||
this.schema = schema;
|
||||
this.path = path;
|
||||
this.partitionKeys = partitionKeys;
|
||||
this.defaultPartName = defaultPartName;
|
||||
this.conf = conf;
|
||||
this.fileIndex = FileIndex.instance(this.path, this.conf);
|
||||
this.requiredPartitions = requiredPartitions;
|
||||
this.requiredPos = requiredPos == null
|
||||
? IntStream.range(0, schema.getColumnCount()).toArray()
|
||||
: requiredPos;
|
||||
this.limit = limit == null ? NO_LIMIT_CONSTANT : limit;
|
||||
this.filters = filters == null ? Collections.emptyList() : filters;
|
||||
this.hadoopConf = StreamerUtil.getHadoopConf();
|
||||
this.metaClient = StreamerUtil.metaClientForReader(conf, hadoopConf);
|
||||
this.maxCompactionMemoryInBytes = StreamerUtil.getMaxCompactionMemoryInBytes(conf);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) {
|
||||
return new DataStreamScanProvider() {
|
||||
|
||||
@Override
|
||||
public boolean isBounded() {
|
||||
return !conf.getBoolean(FlinkOptions.READ_AS_STREAMING);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataStream<RowData> produceDataStream(StreamExecutionEnvironment execEnv) {
|
||||
@SuppressWarnings("unchecked")
|
||||
TypeInformation<RowData> typeInfo =
|
||||
(TypeInformation<RowData>) TypeInfoDataTypeConverter.fromDataTypeToTypeInfo(getProducedDataType());
|
||||
if (conf.getBoolean(FlinkOptions.READ_AS_STREAMING)) {
|
||||
StreamReadMonitoringFunction monitoringFunction = new StreamReadMonitoringFunction(
|
||||
conf, FilePathUtils.toFlinkPath(path), maxCompactionMemoryInBytes, getRequiredPartitionPaths());
|
||||
InputFormat<RowData, ?> inputFormat = getInputFormat(true);
|
||||
OneInputStreamOperatorFactory<MergeOnReadInputSplit, RowData> factory = StreamReadOperator.factory((MergeOnReadInputFormat) inputFormat);
|
||||
SingleOutputStreamOperator<RowData> source = execEnv.addSource(monitoringFunction, getSourceOperatorName("split_monitor"))
|
||||
.setParallelism(1)
|
||||
.transform("split_reader", typeInfo, factory)
|
||||
.setParallelism(conf.getInteger(FlinkOptions.READ_TASKS));
|
||||
return new DataStreamSource<>(source);
|
||||
} else {
|
||||
InputFormatSourceFunction<RowData> func = new InputFormatSourceFunction<>(getInputFormat(), typeInfo);
|
||||
DataStreamSource<RowData> source = execEnv.addSource(func, asSummaryString(), typeInfo);
|
||||
return source.name(getSourceOperatorName("bounded_source")).setParallelism(conf.getInteger(FlinkOptions.READ_TASKS));
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChangelogMode getChangelogMode() {
|
||||
// when read as streaming and changelog mode is enabled, emit as FULL mode;
|
||||
// when all the changes are compacted or read as batch, emit as INSERT mode.
|
||||
return OptionsResolver.emitChangelog(conf) ? ChangelogModes.FULL : ChangelogMode.insertOnly();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DynamicTableSource copy() {
|
||||
return new HoodieTableSource(schema, path, partitionKeys, defaultPartName,
|
||||
conf, requiredPartitions, requiredPos, limit, filters);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String asSummaryString() {
|
||||
return "HudiTableSource";
|
||||
}
|
||||
|
||||
@Override
|
||||
public Result applyFilters(List<ResolvedExpression> filters) {
|
||||
this.filters = new ArrayList<>(filters);
|
||||
// refuse all the filters now
|
||||
return SupportsFilterPushDown.Result.of(Collections.emptyList(), new ArrayList<>(filters));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<List<Map<String, String>>> listPartitions() {
|
||||
List<Map<String, String>> partitions = this.fileIndex.getPartitions(
|
||||
this.partitionKeys, defaultPartName, conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING));
|
||||
return Optional.of(partitions);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void applyPartitions(List<Map<String, String>> partitions) {
|
||||
this.requiredPartitions = partitions;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean supportsNestedProjection() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void applyProjection(int[][] projections) {
|
||||
// nested projection is not supported.
|
||||
this.requiredPos = Arrays.stream(projections).mapToInt(array -> array[0]).toArray();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void applyLimit(long limit) {
|
||||
this.limit = limit;
|
||||
}
|
||||
|
||||
private DataType getProducedDataType() {
|
||||
String[] schemaFieldNames = this.schema.getColumnNames().toArray(new String[0]);
|
||||
DataType[] schemaTypes = this.schema.getColumnDataTypes().toArray(new DataType[0]);
|
||||
|
||||
return DataTypes.ROW(Arrays.stream(this.requiredPos)
|
||||
.mapToObj(i -> DataTypes.FIELD(schemaFieldNames[i], schemaTypes[i]))
|
||||
.toArray(DataTypes.Field[]::new))
|
||||
.bridgedTo(RowData.class);
|
||||
}
|
||||
|
||||
private List<Map<String, String>> getOrFetchPartitions() {
|
||||
if (requiredPartitions == null) {
|
||||
requiredPartitions = listPartitions().orElse(Collections.emptyList());
|
||||
}
|
||||
return requiredPartitions;
|
||||
}
|
||||
|
||||
private String getSourceOperatorName(String operatorName) {
|
||||
String[] schemaFieldNames = this.schema.getColumnNames().toArray(new String[0]);
|
||||
List<String> fields = Arrays.stream(this.requiredPos)
|
||||
.mapToObj(i -> schemaFieldNames[i])
|
||||
.collect(Collectors.toList());
|
||||
StringBuilder sb = new StringBuilder();
|
||||
sb.append(operatorName)
|
||||
.append("(")
|
||||
.append("table=").append(Collections.singletonList(conf.getString(FlinkOptions.TABLE_NAME)))
|
||||
.append(", ")
|
||||
.append("fields=").append(fields)
|
||||
.append(")");
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
private Set<String> getRequiredPartitionPaths() {
|
||||
if (this.requiredPartitions == null) {
|
||||
// returns null for non partition pruning
|
||||
return null;
|
||||
}
|
||||
return FilePathUtils.toRelativePartitionPaths(this.partitionKeys, this.requiredPartitions,
|
||||
conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING));
|
||||
}
|
||||
|
||||
private List<MergeOnReadInputSplit> buildFileIndex() {
|
||||
Set<String> requiredPartitionPaths = getRequiredPartitionPaths();
|
||||
fileIndex.setPartitionPaths(requiredPartitionPaths);
|
||||
List<String> relPartitionPaths = fileIndex.getOrBuildPartitionPaths();
|
||||
if (relPartitionPaths.size() == 0) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
FileStatus[] fileStatuses = fileIndex.getFilesInPartitions();
|
||||
if (fileStatuses.length == 0) {
|
||||
throw new HoodieException("No files found for reading in user provided path.");
|
||||
}
|
||||
|
||||
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient,
|
||||
// file-slice after pending compaction-requested instant-time is also considered valid
|
||||
metaClient.getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants(), fileStatuses);
|
||||
String latestCommit = fsView.getLastInstant().get().getTimestamp();
|
||||
final String mergeType = this.conf.getString(FlinkOptions.MERGE_TYPE);
|
||||
final AtomicInteger cnt = new AtomicInteger(0);
|
||||
// generates one input split for each file group
|
||||
return relPartitionPaths.stream()
|
||||
.map(relPartitionPath -> fsView.getLatestMergedFileSlicesBeforeOrOn(relPartitionPath, latestCommit)
|
||||
.map(fileSlice -> {
|
||||
String basePath = fileSlice.getBaseFile().map(BaseFile::getPath).orElse(null);
|
||||
Option<List<String>> logPaths = Option.ofNullable(fileSlice.getLogFiles()
|
||||
.sorted(HoodieLogFile.getLogFileComparator())
|
||||
.map(logFile -> logFile.getPath().toString())
|
||||
.collect(Collectors.toList()));
|
||||
return new MergeOnReadInputSplit(cnt.getAndAdd(1), basePath, logPaths, latestCommit,
|
||||
metaClient.getBasePath(), maxCompactionMemoryInBytes, mergeType, null);
|
||||
}).collect(Collectors.toList()))
|
||||
.flatMap(Collection::stream)
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public InputFormat<RowData, ?> getInputFormat() {
|
||||
return getInputFormat(false);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public InputFormat<RowData, ?> getInputFormat(boolean isStreaming) {
|
||||
return isStreaming ? getStreamInputFormat() : getBatchInputFormat();
|
||||
}
|
||||
|
||||
private InputFormat<RowData, ?> getBatchInputFormat() {
|
||||
final Schema tableAvroSchema = getTableAvroSchema();
|
||||
final DataType rowDataType = AvroSchemaConverter.convertToDataType(tableAvroSchema);
|
||||
final RowType rowType = (RowType) rowDataType.getLogicalType();
|
||||
final RowType requiredRowType = (RowType) getProducedDataType().notNull().getLogicalType();
|
||||
|
||||
final String queryType = this.conf.getString(FlinkOptions.QUERY_TYPE);
|
||||
switch (queryType) {
|
||||
case FlinkOptions.QUERY_TYPE_SNAPSHOT:
|
||||
final HoodieTableType tableType = HoodieTableType.valueOf(this.conf.getString(FlinkOptions.TABLE_TYPE));
|
||||
switch (tableType) {
|
||||
case MERGE_ON_READ:
|
||||
final List<MergeOnReadInputSplit> inputSplits = buildFileIndex();
|
||||
if (inputSplits.size() == 0) {
|
||||
// When there is no input splits, just return an empty source.
|
||||
LOG.warn("No input splits generate for MERGE_ON_READ input format, returns empty collection instead");
|
||||
return InputFormats.EMPTY_INPUT_FORMAT;
|
||||
}
|
||||
return mergeOnReadInputFormat(rowType, requiredRowType, tableAvroSchema,
|
||||
rowDataType, inputSplits, false);
|
||||
case COPY_ON_WRITE:
|
||||
return baseFileOnlyInputFormat();
|
||||
default:
|
||||
throw new HoodieException("Unexpected table type: " + this.conf.getString(FlinkOptions.TABLE_TYPE));
|
||||
}
|
||||
case FlinkOptions.QUERY_TYPE_READ_OPTIMIZED:
|
||||
return baseFileOnlyInputFormat();
|
||||
case FlinkOptions.QUERY_TYPE_INCREMENTAL:
|
||||
IncrementalInputSplits incrementalInputSplits = IncrementalInputSplits.builder()
|
||||
.conf(conf).path(FilePathUtils.toFlinkPath(path))
|
||||
.maxCompactionMemoryInBytes(maxCompactionMemoryInBytes)
|
||||
.requiredPartitions(getRequiredPartitionPaths()).build();
|
||||
final IncrementalInputSplits.Result result = incrementalInputSplits.inputSplits(metaClient, hadoopConf);
|
||||
if (result.isEmpty()) {
|
||||
// When there is no input splits, just return an empty source.
|
||||
LOG.warn("No input splits generate for incremental read, returns empty collection instead");
|
||||
return InputFormats.EMPTY_INPUT_FORMAT;
|
||||
}
|
||||
return mergeOnReadInputFormat(rowType, requiredRowType, tableAvroSchema,
|
||||
rowDataType, result.getInputSplits(), false);
|
||||
default:
|
||||
String errMsg = String.format("Invalid query type : '%s', options ['%s', '%s', '%s'] are supported now", queryType,
|
||||
FlinkOptions.QUERY_TYPE_SNAPSHOT, FlinkOptions.QUERY_TYPE_READ_OPTIMIZED, FlinkOptions.QUERY_TYPE_INCREMENTAL);
|
||||
throw new HoodieException(errMsg);
|
||||
}
|
||||
}
|
||||
|
||||
private InputFormat<RowData, ?> getStreamInputFormat() {
|
||||
// if table does not exist, use schema from the DDL
|
||||
Schema tableAvroSchema = this.metaClient == null ? inferSchemaFromDdl() : getTableAvroSchema();
|
||||
final DataType rowDataType = AvroSchemaConverter.convertToDataType(tableAvroSchema);
|
||||
final RowType rowType = (RowType) rowDataType.getLogicalType();
|
||||
final RowType requiredRowType = (RowType) getProducedDataType().notNull().getLogicalType();
|
||||
|
||||
final String queryType = this.conf.getString(FlinkOptions.QUERY_TYPE);
|
||||
if (FlinkOptions.QUERY_TYPE_SNAPSHOT.equals(queryType)) {
|
||||
final HoodieTableType tableType = HoodieTableType.valueOf(this.conf.getString(FlinkOptions.TABLE_TYPE));
|
||||
boolean emitDelete = tableType == HoodieTableType.MERGE_ON_READ;
|
||||
return mergeOnReadInputFormat(rowType, requiredRowType, tableAvroSchema,
|
||||
rowDataType, Collections.emptyList(), emitDelete);
|
||||
}
|
||||
String errMsg = String.format("Invalid query type : '%s', options ['%s'] are supported now", queryType,
|
||||
FlinkOptions.QUERY_TYPE_SNAPSHOT);
|
||||
throw new HoodieException(errMsg);
|
||||
}
|
||||
|
||||
private MergeOnReadInputFormat mergeOnReadInputFormat(
|
||||
RowType rowType,
|
||||
RowType requiredRowType,
|
||||
Schema tableAvroSchema,
|
||||
DataType rowDataType,
|
||||
List<MergeOnReadInputSplit> inputSplits,
|
||||
boolean emitDelete) {
|
||||
final MergeOnReadTableState hoodieTableState = new MergeOnReadTableState(
|
||||
rowType,
|
||||
requiredRowType,
|
||||
tableAvroSchema.toString(),
|
||||
AvroSchemaConverter.convertToSchema(requiredRowType).toString(),
|
||||
inputSplits,
|
||||
conf.getString(FlinkOptions.RECORD_KEY_FIELD).split(","));
|
||||
return MergeOnReadInputFormat.builder()
|
||||
.config(this.conf)
|
||||
.tableState(hoodieTableState)
|
||||
// use the explicit fields' data type because the AvroSchemaConverter
|
||||
// is not very stable.
|
||||
.fieldTypes(rowDataType.getChildren())
|
||||
.defaultPartName(conf.getString(FlinkOptions.PARTITION_DEFAULT_NAME))
|
||||
.limit(this.limit)
|
||||
.emitDelete(emitDelete)
|
||||
.build();
|
||||
}
|
||||
|
||||
private InputFormat<RowData, ?> baseFileOnlyInputFormat() {
|
||||
final Path[] paths = getReadPaths();
|
||||
if (paths.length == 0) {
|
||||
return InputFormats.EMPTY_INPUT_FORMAT;
|
||||
}
|
||||
FileInputFormat<RowData> format = new CopyOnWriteInputFormat(
|
||||
FilePathUtils.toFlinkPaths(paths),
|
||||
this.schema.getColumnNames().toArray(new String[0]),
|
||||
this.schema.getColumnDataTypes().toArray(new DataType[0]),
|
||||
this.requiredPos,
|
||||
this.conf.getString(FlinkOptions.PARTITION_DEFAULT_NAME),
|
||||
this.limit == NO_LIMIT_CONSTANT ? Long.MAX_VALUE : this.limit, // ParquetInputFormat always uses the limit value
|
||||
getParquetConf(this.conf, this.hadoopConf),
|
||||
this.conf.getBoolean(FlinkOptions.UTC_TIMEZONE)
|
||||
);
|
||||
format.setFilesFilter(new LatestFileFilter(this.hadoopConf));
|
||||
return format;
|
||||
}
|
||||
|
||||
private Schema inferSchemaFromDdl() {
|
||||
Schema schema = AvroSchemaConverter.convertToSchema(this.schema.toPhysicalRowDataType().getLogicalType());
|
||||
return HoodieAvroUtils.addMetadataFields(schema, conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED));
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public Schema getTableAvroSchema() {
|
||||
try {
|
||||
TableSchemaResolver schemaResolver = new TableSchemaResolver(metaClient);
|
||||
return schemaResolver.getTableAvroSchema();
|
||||
} catch (Throwable e) {
|
||||
// table exists but has no written data
|
||||
LOG.warn("Get table avro schema error, use schema from the DDL instead", e);
|
||||
return inferSchemaFromDdl();
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public HoodieTableMetaClient getMetaClient() {
|
||||
return this.metaClient;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public Configuration getConf() {
|
||||
return this.conf;
|
||||
}
|
||||
|
||||
/**
|
||||
* Reset the state of the table source.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public void reset() {
|
||||
this.metaClient.reloadActiveTimeline();
|
||||
this.requiredPartitions = null;
|
||||
this.fileIndex.reset();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the reader paths with partition path expanded.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public Path[] getReadPaths() {
|
||||
return partitionKeys.isEmpty()
|
||||
? new Path[] {path}
|
||||
: FilePathUtils.partitionPath2ReadPath(path, partitionKeys, getOrFetchPartitions(),
|
||||
conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING));
|
||||
}
|
||||
|
||||
private static class LatestFileFilter extends FilePathFilter {
|
||||
private final HoodieROTablePathFilter hoodieFilter;
|
||||
|
||||
public LatestFileFilter(org.apache.hadoop.conf.Configuration hadoopConf) {
|
||||
this.hoodieFilter = new HoodieROTablePathFilter(hadoopConf);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean filterPath(org.apache.flink.core.fs.Path filePath) {
|
||||
return !this.hoodieFilter.accept(new Path(filePath.toUri()));
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,56 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.catalog;
|
||||
|
||||
import org.apache.flink.configuration.ConfigOption;
|
||||
import org.apache.flink.configuration.ConfigOptions;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.table.catalog.CommonCatalogOptions;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Hoodie catalog options.
|
||||
*/
|
||||
public class CatalogOptions {
|
||||
|
||||
public static final ConfigOption<String> CATALOG_PATH =
|
||||
ConfigOptions.key("catalog.path")
|
||||
.stringType()
|
||||
.noDefaultValue()
|
||||
.withDescription("Catalog base DFS path, used for inferring the sink table path. "
|
||||
+ "The default strategy for a table path is: ${catalog.path}/${db_name}/${table_name}");
|
||||
|
||||
public static final ConfigOption<String> DEFAULT_DATABASE =
|
||||
ConfigOptions.key(CommonCatalogOptions.DEFAULT_DATABASE_KEY)
|
||||
.stringType()
|
||||
.defaultValue("default");
|
||||
|
||||
/**
|
||||
* Returns all the common table options that can be shared.
|
||||
*
|
||||
* @param catalogOptions The catalog options
|
||||
*/
|
||||
public static Map<String, String> tableCommonOptions(Configuration catalogOptions) {
|
||||
Configuration copied = new Configuration(catalogOptions);
|
||||
copied.removeConfig(DEFAULT_DATABASE);
|
||||
copied.removeConfig(CATALOG_PATH);
|
||||
return copied.toMap();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,510 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.catalog;
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.TableSchemaResolver;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.util.AvroSchemaConverter;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.table.catalog.AbstractCatalog;
|
||||
import org.apache.flink.table.catalog.CatalogBaseTable;
|
||||
import org.apache.flink.table.catalog.CatalogDatabase;
|
||||
import org.apache.flink.table.catalog.CatalogDatabaseImpl;
|
||||
import org.apache.flink.table.catalog.CatalogFunction;
|
||||
import org.apache.flink.table.catalog.CatalogPartition;
|
||||
import org.apache.flink.table.catalog.CatalogPartitionSpec;
|
||||
import org.apache.flink.table.catalog.CatalogTable;
|
||||
import org.apache.flink.table.catalog.CatalogView;
|
||||
import org.apache.flink.table.catalog.ObjectPath;
|
||||
import org.apache.flink.table.catalog.ResolvedCatalogTable;
|
||||
import org.apache.flink.table.catalog.ResolvedSchema;
|
||||
import org.apache.flink.table.catalog.exceptions.CatalogException;
|
||||
import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException;
|
||||
import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException;
|
||||
import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
|
||||
import org.apache.flink.table.catalog.exceptions.FunctionAlreadyExistException;
|
||||
import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
|
||||
import org.apache.flink.table.catalog.exceptions.PartitionAlreadyExistsException;
|
||||
import org.apache.flink.table.catalog.exceptions.PartitionNotExistException;
|
||||
import org.apache.flink.table.catalog.exceptions.PartitionSpecInvalidException;
|
||||
import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
|
||||
import org.apache.flink.table.catalog.exceptions.TableNotExistException;
|
||||
import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException;
|
||||
import org.apache.flink.table.catalog.exceptions.TablePartitionedException;
|
||||
import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
|
||||
import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
|
||||
import org.apache.flink.table.expressions.Expression;
|
||||
import org.apache.flink.util.CollectionUtil;
|
||||
import org.apache.flink.util.StringUtils;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.flink.util.Preconditions.checkArgument;
|
||||
import static org.apache.hudi.table.catalog.CatalogOptions.CATALOG_PATH;
|
||||
import static org.apache.hudi.table.catalog.CatalogOptions.DEFAULT_DATABASE;
|
||||
|
||||
/**
|
||||
* Catalog that can set up common options for underneath table.
|
||||
*/
|
||||
public class HoodieCatalog extends AbstractCatalog {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(HoodieCatalog.class);
|
||||
|
||||
private final org.apache.hadoop.conf.Configuration hadoopConf;
|
||||
private final String catalogPathStr;
|
||||
private final Map<String, String> tableCommonOptions;
|
||||
|
||||
private Path catalogPath;
|
||||
private FileSystem fs;
|
||||
|
||||
public HoodieCatalog(String name, Configuration options) {
|
||||
super(name, options.get(DEFAULT_DATABASE));
|
||||
this.catalogPathStr = options.get(CATALOG_PATH);
|
||||
this.hadoopConf = StreamerUtil.getHadoopConf();
|
||||
this.tableCommonOptions = CatalogOptions.tableCommonOptions(options);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open() throws CatalogException {
|
||||
fs = FSUtils.getFs(catalogPathStr, hadoopConf);
|
||||
catalogPath = new Path(catalogPathStr);
|
||||
try {
|
||||
if (!fs.exists(catalogPath)) {
|
||||
throw new CatalogException(String.format("Catalog %s path %s does not exist.", getName(), catalogPathStr));
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new CatalogException(String.format("Checking catalog path %s exists exception.", catalogPathStr), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws CatalogException {
|
||||
try {
|
||||
fs.close();
|
||||
} catch (IOException e) {
|
||||
throw new CatalogException("Closing FileSystem exception.", e);
|
||||
}
|
||||
}
|
||||
|
||||
// ------ databases ------
|
||||
|
||||
@Override
|
||||
public List<String> listDatabases() throws CatalogException {
|
||||
try {
|
||||
FileStatus[] fileStatuses = fs.listStatus(catalogPath);
|
||||
return Arrays.stream(fileStatuses)
|
||||
.filter(FileStatus::isDirectory)
|
||||
.map(fileStatus -> fileStatus.getPath().getName())
|
||||
.collect(Collectors.toList());
|
||||
} catch (IOException e) {
|
||||
throw new CatalogException("Listing database exception.", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistException, CatalogException {
|
||||
if (databaseExists(databaseName)) {
|
||||
return new CatalogDatabaseImpl(Collections.emptyMap(), null);
|
||||
} else {
|
||||
throw new DatabaseNotExistException(getName(), databaseName);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean databaseExists(String databaseName) throws CatalogException {
|
||||
checkArgument(!StringUtils.isNullOrWhitespaceOnly(databaseName));
|
||||
|
||||
return listDatabases().contains(databaseName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void createDatabase(String databaseName, CatalogDatabase catalogDatabase, boolean ignoreIfExists)
|
||||
throws DatabaseAlreadyExistException, CatalogException {
|
||||
if (databaseExists(databaseName)) {
|
||||
if (ignoreIfExists) {
|
||||
return;
|
||||
} else {
|
||||
throw new DatabaseAlreadyExistException(getName(), databaseName);
|
||||
}
|
||||
}
|
||||
|
||||
if (!CollectionUtil.isNullOrEmpty(catalogDatabase.getProperties())) {
|
||||
throw new CatalogException("Hudi catalog doesn't support to create database with options.");
|
||||
}
|
||||
|
||||
Path dbPath = new Path(catalogPath, databaseName);
|
||||
try {
|
||||
fs.mkdirs(dbPath);
|
||||
} catch (IOException e) {
|
||||
throw new CatalogException(String.format("Creating database %s exception.", databaseName), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void dropDatabase(String databaseName, boolean ignoreIfNotExists, boolean cascade)
|
||||
throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException {
|
||||
if (!databaseExists(databaseName)) {
|
||||
if (ignoreIfNotExists) {
|
||||
return;
|
||||
} else {
|
||||
throw new DatabaseNotExistException(getName(), databaseName);
|
||||
}
|
||||
}
|
||||
|
||||
List<String> tables = listTables(databaseName);
|
||||
if (!tables.isEmpty() && !cascade) {
|
||||
throw new DatabaseNotEmptyException(getName(), databaseName);
|
||||
}
|
||||
|
||||
if (databaseName.equals(getDefaultDatabase())) {
|
||||
throw new IllegalArgumentException(
|
||||
"Hudi catalog doesn't support to drop the default database.");
|
||||
}
|
||||
|
||||
Path dbPath = new Path(catalogPath, databaseName);
|
||||
try {
|
||||
fs.delete(dbPath, true);
|
||||
} catch (IOException e) {
|
||||
throw new CatalogException(String.format("Dropping database %s exception.", databaseName), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void alterDatabase(String databaseName, CatalogDatabase catalogDatabase, boolean ignoreIfNotExists)
|
||||
throws DatabaseNotExistException, CatalogException {
|
||||
throw new UnsupportedOperationException("Altering database is not implemented.");
|
||||
}
|
||||
|
||||
// ------ tables ------
|
||||
|
||||
@Override
|
||||
public List<String> listTables(String databaseName) throws DatabaseNotExistException, CatalogException {
|
||||
if (!databaseExists(databaseName)) {
|
||||
throw new DatabaseNotExistException(getName(), databaseName);
|
||||
}
|
||||
|
||||
Path dbPath = new Path(catalogPath, databaseName);
|
||||
try {
|
||||
return Arrays.stream(fs.listStatus(dbPath))
|
||||
.filter(FileStatus::isDirectory)
|
||||
.map(fileStatus -> fileStatus.getPath().getName())
|
||||
.collect(Collectors.toList());
|
||||
} catch (IOException e) {
|
||||
throw new CatalogException(String.format("Listing table in database %s exception.", dbPath), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistException, CatalogException {
|
||||
if (!tableExists(tablePath)) {
|
||||
throw new TableNotExistException(getName(), tablePath);
|
||||
}
|
||||
|
||||
final String path = inferTablePath(catalogPathStr, tablePath);
|
||||
Map<String, String> options = TableOptionProperties.loadFromProperties(path, hadoopConf);
|
||||
final Schema latestSchema = getLatestTableSchema(path);
|
||||
if (latestSchema != null) {
|
||||
org.apache.flink.table.api.Schema.Builder builder = org.apache.flink.table.api.Schema.newBuilder()
|
||||
.fromRowDataType(AvroSchemaConverter.convertToDataType(latestSchema));
|
||||
final String pkConstraintName = TableOptionProperties.getPkConstraintName(options);
|
||||
if (pkConstraintName != null) {
|
||||
builder.primaryKeyNamed(pkConstraintName, TableOptionProperties.getPkColumns(options));
|
||||
}
|
||||
final org.apache.flink.table.api.Schema schema = builder.build();
|
||||
return CatalogTable.of(
|
||||
schema,
|
||||
TableOptionProperties.getComment(options),
|
||||
TableOptionProperties.getPartitionColumns(options),
|
||||
TableOptionProperties.getTableOptions(options));
|
||||
} else {
|
||||
throw new TableNotExistException(getName(), tablePath);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void createTable(ObjectPath tablePath, CatalogBaseTable catalogTable, boolean ignoreIfExists)
|
||||
throws TableAlreadyExistException, DatabaseNotExistException, CatalogException {
|
||||
if (!databaseExists(tablePath.getDatabaseName())) {
|
||||
throw new DatabaseNotExistException(getName(), tablePath.getDatabaseName());
|
||||
}
|
||||
if (tableExists(tablePath)) {
|
||||
if (ignoreIfExists) {
|
||||
return;
|
||||
} else {
|
||||
throw new TableAlreadyExistException(getName(), tablePath);
|
||||
}
|
||||
}
|
||||
|
||||
if (catalogTable instanceof CatalogView) {
|
||||
throw new UnsupportedOperationException(
|
||||
"Hudi catalog doesn't support to CREATE VIEW.");
|
||||
}
|
||||
|
||||
ResolvedCatalogTable resolvedTable = (ResolvedCatalogTable) catalogTable;
|
||||
final String tablePathStr = inferTablePath(catalogPathStr, tablePath);
|
||||
Map<String, String> options = applyOptionsHook(tablePathStr, catalogTable.getOptions());
|
||||
Configuration conf = Configuration.fromMap(options);
|
||||
conf.setString(FlinkOptions.PATH, tablePathStr);
|
||||
ResolvedSchema resolvedSchema = resolvedTable.getResolvedSchema();
|
||||
if (!resolvedSchema.getPrimaryKey().isPresent()) {
|
||||
throw new CatalogException("Primary key definition is missing");
|
||||
}
|
||||
final String avroSchema = AvroSchemaConverter.convertToSchema(resolvedSchema.toPhysicalRowDataType().getLogicalType()).toString();
|
||||
conf.setString(FlinkOptions.SOURCE_AVRO_SCHEMA, avroSchema);
|
||||
|
||||
// stores two copies of options:
|
||||
// - partition keys
|
||||
// - primary keys
|
||||
// because the HoodieTableMetaClient is a heavy impl, we try to avoid initializing it
|
||||
// when calling #getTable.
|
||||
|
||||
final String pkColumns = String.join(",", resolvedSchema.getPrimaryKey().get().getColumns());
|
||||
conf.setString(FlinkOptions.RECORD_KEY_FIELD, pkColumns);
|
||||
options.put(TableOptionProperties.PK_CONSTRAINT_NAME, resolvedSchema.getPrimaryKey().get().getName());
|
||||
options.put(TableOptionProperties.PK_COLUMNS, pkColumns);
|
||||
|
||||
if (resolvedTable.isPartitioned()) {
|
||||
final String partitions = String.join(",", resolvedTable.getPartitionKeys());
|
||||
conf.setString(FlinkOptions.PARTITION_PATH_FIELD, partitions);
|
||||
options.put(TableOptionProperties.PARTITION_COLUMNS, partitions);
|
||||
}
|
||||
conf.setString(FlinkOptions.TABLE_NAME, tablePath.getObjectName());
|
||||
try {
|
||||
StreamerUtil.initTableIfNotExists(conf);
|
||||
// prepare the non-table-options properties
|
||||
if (!StringUtils.isNullOrWhitespaceOnly(resolvedTable.getComment())) {
|
||||
options.put(TableOptionProperties.COMMENT, resolvedTable.getComment());
|
||||
}
|
||||
TableOptionProperties.createProperties(tablePathStr, hadoopConf, options);
|
||||
} catch (IOException e) {
|
||||
throw new CatalogException(String.format("Initialize table path %s exception.", tablePathStr), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean tableExists(ObjectPath tablePath) throws CatalogException {
|
||||
return StreamerUtil.tableExists(inferTablePath(catalogPathStr, tablePath), hadoopConf);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists)
|
||||
throws TableNotExistException, CatalogException {
|
||||
if (!tableExists(tablePath)) {
|
||||
if (ignoreIfNotExists) {
|
||||
return;
|
||||
} else {
|
||||
throw new TableNotExistException(getName(), tablePath);
|
||||
}
|
||||
}
|
||||
|
||||
Path path = new Path(inferTablePath(catalogPathStr, tablePath));
|
||||
try {
|
||||
this.fs.delete(path, true);
|
||||
} catch (IOException e) {
|
||||
throw new CatalogException(String.format("Dropping table %s exception.", tablePath), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists)
|
||||
throws TableNotExistException, TableAlreadyExistException, CatalogException {
|
||||
throw new UnsupportedOperationException("renameTable is not implemented.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void alterTable(ObjectPath tablePath, CatalogBaseTable catalogBaseTable, boolean ignoreIfNotExists)
|
||||
throws TableNotExistException, CatalogException {
|
||||
throw new UnsupportedOperationException("alterTable is not implemented.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> listViews(String databaseName) throws DatabaseNotExistException, CatalogException {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<CatalogPartitionSpec> listPartitions(ObjectPath tablePath)
|
||||
throws TableNotExistException, TableNotPartitionedException, CatalogException {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<CatalogPartitionSpec> listPartitions(ObjectPath tablePath, CatalogPartitionSpec catalogPartitionSpec)
|
||||
throws TableNotExistException, TableNotPartitionedException, PartitionSpecInvalidException, CatalogException {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<CatalogPartitionSpec> listPartitionsByFilter(ObjectPath tablePath, List<Expression> filters)
|
||||
throws TableNotExistException, TableNotPartitionedException, CatalogException {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CatalogPartition getPartition(ObjectPath tablePath, CatalogPartitionSpec catalogPartitionSpec)
|
||||
throws PartitionNotExistException, CatalogException {
|
||||
throw new PartitionNotExistException(getName(), tablePath, catalogPartitionSpec);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean partitionExists(ObjectPath tablePath, CatalogPartitionSpec catalogPartitionSpec) throws CatalogException {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void createPartition(ObjectPath tablePath, CatalogPartitionSpec catalogPartitionSpec, CatalogPartition catalogPartition, boolean ignoreIfExists)
|
||||
throws TableNotExistException, TableNotPartitionedException, PartitionSpecInvalidException, PartitionAlreadyExistsException, CatalogException {
|
||||
throw new UnsupportedOperationException("createPartition is not implemented.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void dropPartition(ObjectPath tablePath, CatalogPartitionSpec catalogPartitionSpec, boolean ignoreIfNotExists)
|
||||
throws PartitionNotExistException, CatalogException {
|
||||
throw new UnsupportedOperationException("dropPartition is not implemented.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void alterPartition(ObjectPath tablePath, CatalogPartitionSpec catalogPartitionSpec, CatalogPartition catalogPartition, boolean ignoreIfNotExists)
|
||||
throws PartitionNotExistException, CatalogException {
|
||||
throw new UnsupportedOperationException("alterPartition is not implemented.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> listFunctions(String databaseName) throws DatabaseNotExistException, CatalogException {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CatalogFunction getFunction(ObjectPath functionPath) throws FunctionNotExistException, CatalogException {
|
||||
throw new FunctionNotExistException(getName(), functionPath);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean functionExists(ObjectPath functionPath) throws CatalogException {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void createFunction(ObjectPath functionPath, CatalogFunction catalogFunction, boolean ignoreIfExists)
|
||||
throws FunctionAlreadyExistException, DatabaseNotExistException, CatalogException {
|
||||
throw new UnsupportedOperationException("createFunction is not implemented.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void alterFunction(ObjectPath functionPath, CatalogFunction catalogFunction, boolean ignoreIfNotExists)
|
||||
throws FunctionNotExistException, CatalogException {
|
||||
throw new UnsupportedOperationException("alterFunction is not implemented.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void dropFunction(ObjectPath functionPath, boolean ignoreIfNotExists)
|
||||
throws FunctionNotExistException, CatalogException {
|
||||
throw new UnsupportedOperationException("dropFunction is not implemented.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public CatalogTableStatistics getTableStatistics(ObjectPath tablePath)
|
||||
throws TableNotExistException, CatalogException {
|
||||
return CatalogTableStatistics.UNKNOWN;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CatalogColumnStatistics getTableColumnStatistics(ObjectPath tablePath)
|
||||
throws TableNotExistException, CatalogException {
|
||||
return CatalogColumnStatistics.UNKNOWN;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CatalogTableStatistics getPartitionStatistics(ObjectPath tablePath, CatalogPartitionSpec catalogPartitionSpec)
|
||||
throws PartitionNotExistException, CatalogException {
|
||||
return CatalogTableStatistics.UNKNOWN;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CatalogColumnStatistics getPartitionColumnStatistics(ObjectPath tablePath, CatalogPartitionSpec catalogPartitionSpec)
|
||||
throws PartitionNotExistException, CatalogException {
|
||||
return CatalogColumnStatistics.UNKNOWN;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void alterTableStatistics(ObjectPath tablePath, CatalogTableStatistics catalogTableStatistics, boolean ignoreIfNotExists)
|
||||
throws TableNotExistException, CatalogException {
|
||||
throw new UnsupportedOperationException("alterTableStatistics is not implemented.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void alterTableColumnStatistics(ObjectPath tablePath, CatalogColumnStatistics catalogColumnStatistics, boolean ignoreIfNotExists)
|
||||
throws TableNotExistException, CatalogException, TablePartitionedException {
|
||||
throw new UnsupportedOperationException("alterTableColumnStatistics is not implemented.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void alterPartitionStatistics(ObjectPath tablePath, CatalogPartitionSpec catalogPartitionSpec, CatalogTableStatistics catalogTableStatistics, boolean ignoreIfNotExists)
|
||||
throws PartitionNotExistException, CatalogException {
|
||||
throw new UnsupportedOperationException("alterPartitionStatistics is not implemented.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void alterPartitionColumnStatistics(ObjectPath tablePath, CatalogPartitionSpec catalogPartitionSpec, CatalogColumnStatistics catalogColumnStatistics, boolean ignoreIfNotExists)
|
||||
throws PartitionNotExistException, CatalogException {
|
||||
throw new UnsupportedOperationException("alterPartitionColumnStatistics is not implemented.");
|
||||
}
|
||||
|
||||
private @Nullable Schema getLatestTableSchema(String path) {
|
||||
if (path != null && StreamerUtil.tableExists(path, hadoopConf)) {
|
||||
try {
|
||||
HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(path, hadoopConf);
|
||||
return new TableSchemaResolver(metaClient).getTableAvroSchema(false); // change log mode is not supported now
|
||||
} catch (Throwable throwable) {
|
||||
LOG.warn("Error while resolving the latest table schema.", throwable);
|
||||
// ignored
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private Map<String, String> applyOptionsHook(String tablePath, Map<String, String> options) {
|
||||
Map<String, String> newOptions = new HashMap<>(options);
|
||||
newOptions.put("connector", "hudi");
|
||||
newOptions.computeIfAbsent(FlinkOptions.PATH.key(), k -> tablePath);
|
||||
tableCommonOptions.forEach(newOptions::putIfAbsent);
|
||||
return newOptions;
|
||||
}
|
||||
|
||||
private String inferTablePath(String catalogPath, ObjectPath tablePath) {
|
||||
return String.format("%s/%s/%s", catalogPath, tablePath.getDatabaseName(), tablePath.getObjectName());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,72 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.catalog;
|
||||
|
||||
import org.apache.flink.configuration.ConfigOption;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.table.catalog.Catalog;
|
||||
import org.apache.flink.table.factories.CatalogFactory;
|
||||
import org.apache.flink.table.factories.FactoryUtil;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.apache.hudi.table.catalog.CatalogOptions.CATALOG_PATH;
|
||||
import static org.apache.hudi.table.catalog.CatalogOptions.DEFAULT_DATABASE;
|
||||
|
||||
/**
|
||||
* A catalog factory impl that creates {@link HoodieCatalog}.
|
||||
*/
|
||||
public class HoodieCatalogFactory implements CatalogFactory {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(HoodieCatalogFactory.class);
|
||||
|
||||
public static final String IDENTIFIER = "hudi";
|
||||
|
||||
@Override
|
||||
public String factoryIdentifier() {
|
||||
return IDENTIFIER;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Catalog createCatalog(Context context) {
|
||||
final FactoryUtil.CatalogFactoryHelper helper =
|
||||
FactoryUtil.createCatalogFactoryHelper(this, context);
|
||||
helper.validate();
|
||||
|
||||
return new HoodieCatalog(
|
||||
context.getName(),
|
||||
(Configuration) helper.getOptions());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<ConfigOption<?>> requiredOptions() {
|
||||
Set<ConfigOption<?>> options = new HashSet<>();
|
||||
options.add(CATALOG_PATH);
|
||||
options.add(DEFAULT_DATABASE);
|
||||
return options;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<ConfigOption<?>> optionalOptions() {
|
||||
return Collections.emptySet();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,131 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.catalog;
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.common.table.HoodieTableMetaClient.AUXILIARYFOLDER_NAME;
|
||||
|
||||
/**
|
||||
* Helper class to read/write flink table options as a map.
|
||||
*/
|
||||
public class TableOptionProperties {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(TableOptionProperties.class);
|
||||
|
||||
private static final String FILE_NAME = "table_option.properties";
|
||||
|
||||
public static final String PK_CONSTRAINT_NAME = "pk.constraint.name";
|
||||
public static final String PK_COLUMNS = "pk.columns";
|
||||
public static final String COMMENT = "comment";
|
||||
public static final String PARTITION_COLUMNS = "partition.columns";
|
||||
|
||||
public static final List<String> NON_OPTION_KEYS = Arrays.asList(PK_CONSTRAINT_NAME, PK_COLUMNS, COMMENT, PARTITION_COLUMNS);
|
||||
|
||||
/**
|
||||
* Initialize the {@link #FILE_NAME} meta file.
|
||||
*/
|
||||
public static void createProperties(String basePath,
|
||||
Configuration hadoopConf,
|
||||
Map<String, String> options) throws IOException {
|
||||
Path propertiesFilePath = getPropertiesFilePath(basePath);
|
||||
FileSystem fs = FSUtils.getFs(basePath, hadoopConf);
|
||||
try (FSDataOutputStream outputStream = fs.create(propertiesFilePath)) {
|
||||
Properties properties = new Properties();
|
||||
properties.putAll(options);
|
||||
properties.store(outputStream,
|
||||
"Table option properties saved on " + new Date(System.currentTimeMillis()));
|
||||
}
|
||||
LOG.info(String.format("Create file %s success.", propertiesFilePath));
|
||||
}
|
||||
|
||||
/**
|
||||
* Read table options map from the given table base path.
|
||||
*/
|
||||
public static Map<String, String> loadFromProperties(String basePath, Configuration hadoopConf) {
|
||||
Path propertiesFilePath = getPropertiesFilePath(basePath);
|
||||
Map<String, String> options = new HashMap<>();
|
||||
Properties props = new Properties();
|
||||
|
||||
FileSystem fs = FSUtils.getFs(basePath, hadoopConf);
|
||||
try (FSDataInputStream inputStream = fs.open(propertiesFilePath)) {
|
||||
props.load(inputStream);
|
||||
for (final String name : props.stringPropertyNames()) {
|
||||
options.put(name, props.getProperty(name));
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(String.format("Could not load table option properties from %s", propertiesFilePath), e);
|
||||
}
|
||||
LOG.info(String.format("Loading table option properties from %s success.", propertiesFilePath));
|
||||
return options;
|
||||
}
|
||||
|
||||
private static Path getPropertiesFilePath(String basePath) {
|
||||
String auxPath = basePath + Path.SEPARATOR + AUXILIARYFOLDER_NAME;
|
||||
return new Path(auxPath, FILE_NAME);
|
||||
}
|
||||
|
||||
public static String getPkConstraintName(Map<String, String> options) {
|
||||
return options.get(PK_CONSTRAINT_NAME);
|
||||
}
|
||||
|
||||
public static List<String> getPkColumns(Map<String, String> options) {
|
||||
if (options.containsKey(PK_COLUMNS)) {
|
||||
return Arrays.stream(options.get(PK_COLUMNS).split(",")).collect(Collectors.toList());
|
||||
} else {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
}
|
||||
|
||||
public static List<String> getPartitionColumns(Map<String, String> options) {
|
||||
if (options.containsKey(PARTITION_COLUMNS)) {
|
||||
return Arrays.stream(options.get(PARTITION_COLUMNS).split(",")).collect(Collectors.toList());
|
||||
} else {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
}
|
||||
|
||||
public static String getComment(Map<String, String> options) {
|
||||
return options.get(COMMENT);
|
||||
}
|
||||
|
||||
public static Map<String, String> getTableOptions(Map<String, String> options) {
|
||||
Map<String, String> copied = new HashMap<>(options);
|
||||
NON_OPTION_KEYS.forEach(copied::remove);
|
||||
return copied;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,428 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.format;
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
|
||||
import org.apache.flink.api.java.tuple.Tuple2;
|
||||
import org.apache.flink.table.api.TableException;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.BitSet;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Reference the Flink {@link org.apache.flink.table.utils.PartitionPathUtils}
|
||||
* but supports simple partition path besides the Hive style.
|
||||
*/
|
||||
public class FilePathUtils {
|
||||
|
||||
private static final Pattern HIVE_PARTITION_NAME_PATTERN = Pattern.compile("([^/]+)=([^/]+)");
|
||||
|
||||
private static final BitSet CHAR_TO_ESCAPE = new BitSet(128);
|
||||
|
||||
static {
|
||||
for (char c = 0; c < ' '; c++) {
|
||||
CHAR_TO_ESCAPE.set(c);
|
||||
}
|
||||
|
||||
/*
|
||||
* ASCII 01-1F are HTTP control characters that need to be escaped.
|
||||
* \u000A and \u000D are \n and \r, respectively.
|
||||
*/
|
||||
char[] clist = new char[] {'\u0001', '\u0002', '\u0003', '\u0004',
|
||||
'\u0005', '\u0006', '\u0007', '\u0008', '\u0009', '\n', '\u000B',
|
||||
'\u000C', '\r', '\u000E', '\u000F', '\u0010', '\u0011', '\u0012',
|
||||
'\u0013', '\u0014', '\u0015', '\u0016', '\u0017', '\u0018', '\u0019',
|
||||
'\u001A', '\u001B', '\u001C', '\u001D', '\u001E', '\u001F',
|
||||
'"', '#', '%', '\'', '*', '/', ':', '=', '?', '\\', '\u007F', '{',
|
||||
'[', ']', '^'};
|
||||
|
||||
for (char c : clist) {
|
||||
CHAR_TO_ESCAPE.set(c);
|
||||
}
|
||||
}
|
||||
|
||||
private static boolean needsEscaping(char c) {
|
||||
return c < CHAR_TO_ESCAPE.size() && CHAR_TO_ESCAPE.get(c);
|
||||
}
|
||||
|
||||
/**
|
||||
* Make partition path from partition spec.
|
||||
*
|
||||
* @param partitionKVs The partition key value mapping
|
||||
* @param hivePartition Whether the partition path is with Hive style,
|
||||
* e.g. {partition key} = {partition value}
|
||||
* @param sepSuffix Whether to append the path separator as suffix
|
||||
* @return an escaped, valid partition name
|
||||
*/
|
||||
public static String generatePartitionPath(
|
||||
LinkedHashMap<String, String> partitionKVs,
|
||||
boolean hivePartition,
|
||||
boolean sepSuffix) {
|
||||
if (partitionKVs.isEmpty()) {
|
||||
return "";
|
||||
}
|
||||
StringBuilder suffixBuf = new StringBuilder();
|
||||
int i = 0;
|
||||
for (Map.Entry<String, String> e : partitionKVs.entrySet()) {
|
||||
if (i > 0) {
|
||||
suffixBuf.append(Path.SEPARATOR);
|
||||
}
|
||||
if (hivePartition) {
|
||||
suffixBuf.append(escapePathName(e.getKey()));
|
||||
suffixBuf.append('=');
|
||||
}
|
||||
suffixBuf.append(escapePathName(e.getValue()));
|
||||
i++;
|
||||
}
|
||||
if (sepSuffix) {
|
||||
suffixBuf.append(Path.SEPARATOR);
|
||||
}
|
||||
return suffixBuf.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Escapes a path name.
|
||||
*
|
||||
* @param path The path to escape.
|
||||
* @return An escaped path name.
|
||||
*/
|
||||
private static String escapePathName(String path) {
|
||||
if (path == null || path.length() == 0) {
|
||||
throw new TableException("Path should not be null or empty: " + path);
|
||||
}
|
||||
|
||||
StringBuilder sb = new StringBuilder();
|
||||
for (int i = 0; i < path.length(); i++) {
|
||||
char c = path.charAt(i);
|
||||
if (needsEscaping(c)) {
|
||||
sb.append('%');
|
||||
sb.append(String.format("%1$02X", (int) c));
|
||||
} else {
|
||||
sb.append(c);
|
||||
}
|
||||
}
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates partition key value mapping from path.
|
||||
*
|
||||
* @param currPath Partition file path
|
||||
* @param hivePartition Whether the partition path is with Hive style
|
||||
* @param partitionKeys Partition keys
|
||||
* @return Sequential partition specs.
|
||||
*/
|
||||
public static LinkedHashMap<String, String> extractPartitionKeyValues(
|
||||
Path currPath,
|
||||
boolean hivePartition,
|
||||
String[] partitionKeys) {
|
||||
LinkedHashMap<String, String> fullPartSpec = new LinkedHashMap<>();
|
||||
if (partitionKeys.length == 0) {
|
||||
return fullPartSpec;
|
||||
}
|
||||
List<String[]> kvs = new ArrayList<>();
|
||||
int curDepth = 0;
|
||||
do {
|
||||
String component = currPath.getName();
|
||||
final String[] kv = new String[2];
|
||||
if (hivePartition) {
|
||||
Matcher m = HIVE_PARTITION_NAME_PATTERN.matcher(component);
|
||||
if (m.matches()) {
|
||||
String k = unescapePathName(m.group(1));
|
||||
String v = unescapePathName(m.group(2));
|
||||
kv[0] = k;
|
||||
kv[1] = v;
|
||||
}
|
||||
} else {
|
||||
kv[0] = partitionKeys[partitionKeys.length - 1 - curDepth];
|
||||
kv[1] = unescapePathName(component);
|
||||
}
|
||||
kvs.add(kv);
|
||||
currPath = currPath.getParent();
|
||||
curDepth++;
|
||||
} while (currPath != null && !currPath.getName().isEmpty() && curDepth < partitionKeys.length);
|
||||
|
||||
// reverse the list since we checked the part from leaf dir to table's base dir
|
||||
for (int i = kvs.size(); i > 0; i--) {
|
||||
fullPartSpec.put(kvs.get(i - 1)[0], kvs.get(i - 1)[1]);
|
||||
}
|
||||
|
||||
return fullPartSpec;
|
||||
}
|
||||
|
||||
public static String unescapePathName(String path) {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
for (int i = 0; i < path.length(); i++) {
|
||||
char c = path.charAt(i);
|
||||
if (c == '%' && i + 2 < path.length()) {
|
||||
int code = -1;
|
||||
try {
|
||||
code = Integer.parseInt(path.substring(i + 1, i + 3), 16);
|
||||
} catch (Exception ignored) {
|
||||
// do nothing
|
||||
}
|
||||
if (code >= 0) {
|
||||
sb.append((char) code);
|
||||
i += 2;
|
||||
continue;
|
||||
}
|
||||
}
|
||||
sb.append(c);
|
||||
}
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Search all partitions in this path.
|
||||
*
|
||||
* @param fs File system
|
||||
* @param path Search path
|
||||
* @param hivePartition Whether the partition path is with Hive style
|
||||
* @param partitionKeys Partition keys
|
||||
* @return all partition key value mapping in sequence of the given path
|
||||
*/
|
||||
public static List<Tuple2<LinkedHashMap<String, String>, Path>> searchPartKeyValueAndPaths(
|
||||
FileSystem fs,
|
||||
Path path,
|
||||
boolean hivePartition,
|
||||
String[] partitionKeys) {
|
||||
// expectLevel start from 0, E.G. base_path/level0/level1/level2
|
||||
FileStatus[] generatedParts = getFileStatusRecursively(path, partitionKeys.length, fs);
|
||||
List<Tuple2<LinkedHashMap<String, String>, Path>> ret = new ArrayList<>();
|
||||
for (FileStatus part : generatedParts) {
|
||||
ret.add(
|
||||
new Tuple2<>(
|
||||
extractPartitionKeyValues(part.getPath(), hivePartition, partitionKeys),
|
||||
part.getPath()));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
public static FileStatus[] getFileStatusRecursively(Path path, int expectLevel, Configuration conf) {
|
||||
return getFileStatusRecursively(path, expectLevel, FSUtils.getFs(path.toString(), conf));
|
||||
}
|
||||
|
||||
public static FileStatus[] getFileStatusRecursively(Path path, int expectLevel, FileSystem fs) {
|
||||
ArrayList<FileStatus> result = new ArrayList<>();
|
||||
|
||||
try {
|
||||
FileStatus fileStatus = fs.getFileStatus(path);
|
||||
listStatusRecursively(fs, fileStatus, 0, expectLevel, result);
|
||||
} catch (IOException ignore) {
|
||||
return new FileStatus[0];
|
||||
}
|
||||
|
||||
return result.toArray(new FileStatus[0]);
|
||||
}
|
||||
|
||||
private static void listStatusRecursively(
|
||||
FileSystem fs,
|
||||
FileStatus fileStatus,
|
||||
int level,
|
||||
int expectLevel,
|
||||
List<FileStatus> results) throws IOException {
|
||||
if (expectLevel == level && !isHiddenFile(fileStatus)) {
|
||||
results.add(fileStatus);
|
||||
return;
|
||||
}
|
||||
|
||||
if (fileStatus.isDirectory() && !isHiddenFile(fileStatus)) {
|
||||
for (FileStatus stat : fs.listStatus(fileStatus.getPath())) {
|
||||
listStatusRecursively(fs, stat, level + 1, expectLevel, results);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static boolean isHiddenFile(FileStatus fileStatus) {
|
||||
String name = fileStatus.getPath().getName();
|
||||
// the log files is hidden file
|
||||
return name.startsWith("_") || (name.startsWith(".") && !name.contains(".log."));
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the partition path key and values as a list of map, each map item in the list
|
||||
* is a mapping of the partition key name to its actual partition value. For example, say
|
||||
* there is a file path with partition keys [key1, key2, key3]:
|
||||
*
|
||||
* <p><pre>
|
||||
* -- file:/// ... key1=val1/key2=val2/key3=val3
|
||||
* -- file:/// ... key1=val4/key2=val5/key3=val6
|
||||
* </pre>
|
||||
*
|
||||
* <p>The return list should be [{key1:val1, key2:val2, key3:val3}, {key1:val4, key2:val5, key3:val6}].
|
||||
*
|
||||
* @param path The base path
|
||||
* @param hadoopConf The hadoop configuration
|
||||
* @param partitionKeys The partition key list
|
||||
* @param defaultParName The default partition name for nulls
|
||||
* @param hivePartition Whether the partition path is in Hive style
|
||||
*/
|
||||
public static List<Map<String, String>> getPartitions(
|
||||
Path path,
|
||||
Configuration hadoopConf,
|
||||
List<String> partitionKeys,
|
||||
String defaultParName,
|
||||
boolean hivePartition) {
|
||||
try {
|
||||
return FilePathUtils
|
||||
.searchPartKeyValueAndPaths(
|
||||
FSUtils.getFs(path.toString(), hadoopConf),
|
||||
path,
|
||||
hivePartition,
|
||||
partitionKeys.toArray(new String[0]))
|
||||
.stream()
|
||||
.map(tuple2 -> tuple2.f0)
|
||||
.map(spec -> {
|
||||
LinkedHashMap<String, String> ret = new LinkedHashMap<>();
|
||||
spec.forEach((k, v) -> ret.put(k, defaultParName.equals(v) ? null : v));
|
||||
return ret;
|
||||
})
|
||||
.collect(Collectors.toList());
|
||||
} catch (Exception e) {
|
||||
throw new TableException("Fetch partitions fail.", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Reorder the partition key value mapping based on the given partition keys sequence.
|
||||
*
|
||||
* @param partitionKVs The partition key and value mapping
|
||||
* @param partitionKeys The partition key list
|
||||
*/
|
||||
public static LinkedHashMap<String, String> validateAndReorderPartitions(
|
||||
Map<String, String> partitionKVs,
|
||||
List<String> partitionKeys) {
|
||||
LinkedHashMap<String, String> map = new LinkedHashMap<>();
|
||||
for (String k : partitionKeys) {
|
||||
if (!partitionKVs.containsKey(k)) {
|
||||
throw new TableException("Partition keys are: " + partitionKeys
|
||||
+ ", incomplete partition spec: " + partitionKVs);
|
||||
}
|
||||
map.put(k, partitionKVs.get(k));
|
||||
}
|
||||
return map;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all the file paths that is the parents of the data files.
|
||||
*
|
||||
* @param path The base path
|
||||
* @param conf The Flink configuration
|
||||
* @param hadoopConf The hadoop configuration
|
||||
* @param partitionKeys The partition key list
|
||||
*/
|
||||
public static Path[] getReadPaths(
|
||||
Path path,
|
||||
org.apache.flink.configuration.Configuration conf,
|
||||
Configuration hadoopConf,
|
||||
List<String> partitionKeys) {
|
||||
if (partitionKeys.isEmpty()) {
|
||||
return new Path[] {path};
|
||||
} else {
|
||||
final String defaultParName = conf.getString(FlinkOptions.PARTITION_DEFAULT_NAME);
|
||||
final boolean hivePartition = conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING);
|
||||
List<Map<String, String>> partitionPaths =
|
||||
getPartitions(path, hadoopConf, partitionKeys, defaultParName, hivePartition);
|
||||
return partitionPath2ReadPath(path, partitionKeys, partitionPaths, hivePartition);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Transforms the given partition key value mapping to read paths.
|
||||
*
|
||||
* @param path The base path
|
||||
* @param partitionKeys The partition key list
|
||||
* @param partitionPaths The partition key value mapping
|
||||
* @param hivePartition Whether the partition path is in Hive style
|
||||
* @see #getReadPaths
|
||||
*/
|
||||
public static Path[] partitionPath2ReadPath(
|
||||
Path path,
|
||||
List<String> partitionKeys,
|
||||
List<Map<String, String>> partitionPaths,
|
||||
boolean hivePartition) {
|
||||
return partitionPaths.stream()
|
||||
.map(m -> validateAndReorderPartitions(m, partitionKeys))
|
||||
.map(kvs -> FilePathUtils.generatePartitionPath(kvs, hivePartition, true))
|
||||
.map(n -> new Path(path, n))
|
||||
.toArray(Path[]::new);
|
||||
}
|
||||
|
||||
/**
|
||||
* Transforms the given partition key value mapping to relative partition paths.
|
||||
*
|
||||
* @param partitionKeys The partition key list
|
||||
* @param partitionPaths The partition key value mapping
|
||||
* @param hivePartition Whether the partition path is in Hive style
|
||||
* @see #getReadPaths
|
||||
*/
|
||||
public static Set<String> toRelativePartitionPaths(
|
||||
List<String> partitionKeys,
|
||||
List<Map<String, String>> partitionPaths,
|
||||
boolean hivePartition) {
|
||||
return partitionPaths.stream()
|
||||
.map(m -> validateAndReorderPartitions(m, partitionKeys))
|
||||
.map(kvs -> FilePathUtils.generatePartitionPath(kvs, hivePartition, false))
|
||||
.collect(Collectors.toSet());
|
||||
}
|
||||
|
||||
/**
|
||||
* Transforms the array of Hadoop paths to Flink paths.
|
||||
*/
|
||||
public static org.apache.flink.core.fs.Path[] toFlinkPaths(Path[] paths) {
|
||||
return Arrays.stream(paths)
|
||||
.map(FilePathUtils::toFlinkPath)
|
||||
.toArray(org.apache.flink.core.fs.Path[]::new);
|
||||
}
|
||||
|
||||
/**
|
||||
* Transforms the Hadoop path to Flink path.
|
||||
*/
|
||||
public static org.apache.flink.core.fs.Path toFlinkPath(Path path) {
|
||||
return new org.apache.flink.core.fs.Path(path.toUri());
|
||||
}
|
||||
|
||||
/**
|
||||
* Extracts the partition keys with given configuration.
|
||||
*
|
||||
* @param conf The flink configuration
|
||||
* @return array of the partition fields
|
||||
*/
|
||||
public static String[] extractPartitionKeys(org.apache.flink.configuration.Configuration conf) {
|
||||
if (FlinkOptions.isDefaultValueDefined(conf, FlinkOptions.PARTITION_PATH_FIELD)) {
|
||||
return new String[0];
|
||||
}
|
||||
return conf.getString(FlinkOptions.PARTITION_PATH_FIELD).split(",");
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,266 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.format;
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieOperation;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
|
||||
import org.apache.hudi.common.table.log.HoodieUnMergedLogRecordScanner;
|
||||
import org.apache.hudi.common.util.DefaultSizeEstimator;
|
||||
import org.apache.hudi.common.util.Functions;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueProducer;
|
||||
import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.hadoop.config.HoodieRealtimeConfig;
|
||||
import org.apache.hudi.table.format.mor.MergeOnReadInputSplit;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.GenericRecordBuilder;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.types.RowKind;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
* Utilities for format.
|
||||
*/
|
||||
public class FormatUtils {
|
||||
private FormatUtils() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets up the row kind to the row data {@code rowData} from the resolved operation.
|
||||
*/
|
||||
public static void setRowKind(RowData rowData, IndexedRecord record, int index) {
|
||||
if (index == -1) {
|
||||
return;
|
||||
}
|
||||
rowData.setRowKind(getRowKind(record, index));
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the RowKind of the given record, never null.
|
||||
* Returns RowKind.INSERT when the given field value not found.
|
||||
*/
|
||||
private static RowKind getRowKind(IndexedRecord record, int index) {
|
||||
Object val = record.get(index);
|
||||
if (val == null) {
|
||||
return RowKind.INSERT;
|
||||
}
|
||||
final HoodieOperation operation = HoodieOperation.fromName(val.toString());
|
||||
if (HoodieOperation.isInsert(operation)) {
|
||||
return RowKind.INSERT;
|
||||
} else if (HoodieOperation.isUpdateBefore(operation)) {
|
||||
return RowKind.UPDATE_BEFORE;
|
||||
} else if (HoodieOperation.isUpdateAfter(operation)) {
|
||||
return RowKind.UPDATE_AFTER;
|
||||
} else if (HoodieOperation.isDelete(operation)) {
|
||||
return RowKind.DELETE;
|
||||
} else {
|
||||
throw new AssertionError();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the RowKind of the given record, never null.
|
||||
* Returns RowKind.INSERT when the given field value not found.
|
||||
*/
|
||||
public static RowKind getRowKindSafely(IndexedRecord record, int index) {
|
||||
if (index == -1) {
|
||||
return RowKind.INSERT;
|
||||
}
|
||||
return getRowKind(record, index);
|
||||
}
|
||||
|
||||
public static GenericRecord buildAvroRecordBySchema(
|
||||
IndexedRecord record,
|
||||
Schema requiredSchema,
|
||||
int[] requiredPos,
|
||||
GenericRecordBuilder recordBuilder) {
|
||||
List<Schema.Field> requiredFields = requiredSchema.getFields();
|
||||
assert (requiredFields.size() == requiredPos.length);
|
||||
Iterator<Integer> positionIterator = Arrays.stream(requiredPos).iterator();
|
||||
requiredFields.forEach(f -> recordBuilder.set(f, getVal(record, positionIterator.next())));
|
||||
return recordBuilder.build();
|
||||
}
|
||||
|
||||
private static Object getVal(IndexedRecord record, int pos) {
|
||||
return pos == -1 ? null : record.get(pos);
|
||||
}
|
||||
|
||||
public static HoodieMergedLogRecordScanner logScanner(
|
||||
MergeOnReadInputSplit split,
|
||||
Schema logSchema,
|
||||
Configuration config,
|
||||
boolean withOperationField) {
|
||||
FileSystem fs = FSUtils.getFs(split.getTablePath(), config);
|
||||
return HoodieMergedLogRecordScanner.newBuilder()
|
||||
.withFileSystem(fs)
|
||||
.withBasePath(split.getTablePath())
|
||||
.withLogFilePaths(split.getLogPaths().get())
|
||||
.withReaderSchema(logSchema)
|
||||
.withLatestInstantTime(split.getLatestCommit())
|
||||
.withReadBlocksLazily(
|
||||
string2Boolean(
|
||||
config.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP,
|
||||
HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)))
|
||||
.withReverseReader(false)
|
||||
.withBufferSize(
|
||||
config.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP,
|
||||
HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE))
|
||||
.withMaxMemorySizeInBytes(split.getMaxCompactionMemoryInBytes())
|
||||
.withSpillableMapBasePath(
|
||||
config.get(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP,
|
||||
HoodieRealtimeConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH))
|
||||
.withInstantRange(split.getInstantRange())
|
||||
.withOperationField(withOperationField)
|
||||
.build();
|
||||
}
|
||||
|
||||
private static HoodieUnMergedLogRecordScanner unMergedLogScanner(
|
||||
MergeOnReadInputSplit split,
|
||||
Schema logSchema,
|
||||
Configuration config,
|
||||
HoodieUnMergedLogRecordScanner.LogRecordScannerCallback callback) {
|
||||
FileSystem fs = FSUtils.getFs(split.getTablePath(), config);
|
||||
return HoodieUnMergedLogRecordScanner.newBuilder()
|
||||
.withFileSystem(fs)
|
||||
.withBasePath(split.getTablePath())
|
||||
.withLogFilePaths(split.getLogPaths().get())
|
||||
.withReaderSchema(logSchema)
|
||||
.withLatestInstantTime(split.getLatestCommit())
|
||||
.withReadBlocksLazily(
|
||||
string2Boolean(
|
||||
config.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP,
|
||||
HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)))
|
||||
.withReverseReader(false)
|
||||
.withBufferSize(
|
||||
config.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP,
|
||||
HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE))
|
||||
.withInstantRange(split.getInstantRange())
|
||||
.withLogRecordScannerCallback(callback)
|
||||
.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility to read and buffer the records in the unMerged log record scanner.
|
||||
*/
|
||||
public static class BoundedMemoryRecords {
|
||||
// Log Record unmerged scanner
|
||||
private final HoodieUnMergedLogRecordScanner scanner;
|
||||
|
||||
// Executor that runs the above producers in parallel
|
||||
private final BoundedInMemoryExecutor<HoodieRecord<?>, HoodieRecord<?>, ?> executor;
|
||||
|
||||
// Iterator for the buffer consumer
|
||||
private final Iterator<HoodieRecord<?>> iterator;
|
||||
|
||||
public BoundedMemoryRecords(
|
||||
MergeOnReadInputSplit split,
|
||||
Schema logSchema,
|
||||
Configuration hadoopConf,
|
||||
org.apache.flink.configuration.Configuration flinkConf) {
|
||||
this.executor = new BoundedInMemoryExecutor<>(
|
||||
StreamerUtil.getMaxCompactionMemoryInBytes(flinkConf),
|
||||
getParallelProducers(),
|
||||
Option.empty(),
|
||||
Function.identity(),
|
||||
new DefaultSizeEstimator<>(),
|
||||
Functions.noop());
|
||||
// Consumer of this record reader
|
||||
this.iterator = this.executor.getQueue().iterator();
|
||||
this.scanner = FormatUtils.unMergedLogScanner(split, logSchema, hadoopConf,
|
||||
record -> executor.getQueue().insertRecord(record));
|
||||
// Start reading and buffering
|
||||
this.executor.startProducers();
|
||||
}
|
||||
|
||||
public Iterator<HoodieRecord<?>> getRecordsIterator() {
|
||||
return this.iterator;
|
||||
}
|
||||
|
||||
/**
|
||||
* Setup log and parquet reading in parallel. Both write to central buffer.
|
||||
*/
|
||||
private List<BoundedInMemoryQueueProducer<HoodieRecord<?>>> getParallelProducers() {
|
||||
List<BoundedInMemoryQueueProducer<HoodieRecord<?>>> producers = new ArrayList<>();
|
||||
producers.add(new FunctionBasedQueueProducer<>(buffer -> {
|
||||
scanner.scan();
|
||||
return null;
|
||||
}));
|
||||
return producers;
|
||||
}
|
||||
|
||||
public void close() {
|
||||
this.executor.shutdownNow();
|
||||
}
|
||||
}
|
||||
|
||||
public static HoodieMergedLogRecordScanner logScanner(
|
||||
List<String> logPaths,
|
||||
Schema logSchema,
|
||||
String latestInstantTime,
|
||||
HoodieWriteConfig writeConfig,
|
||||
Configuration hadoopConf) {
|
||||
String basePath = writeConfig.getBasePath();
|
||||
return HoodieMergedLogRecordScanner.newBuilder()
|
||||
.withFileSystem(FSUtils.getFs(basePath, hadoopConf))
|
||||
.withBasePath(basePath)
|
||||
.withLogFilePaths(logPaths)
|
||||
.withReaderSchema(logSchema)
|
||||
.withLatestInstantTime(latestInstantTime)
|
||||
.withReadBlocksLazily(writeConfig.getCompactionLazyBlockReadEnabled())
|
||||
.withReverseReader(false)
|
||||
.withBufferSize(writeConfig.getMaxDFSStreamBufferSize())
|
||||
.withMaxMemorySizeInBytes(writeConfig.getMaxMemoryPerPartitionMerge())
|
||||
.withSpillableMapBasePath(writeConfig.getSpillableMapBasePath())
|
||||
.withDiskMapType(writeConfig.getCommonConfig().getSpillableDiskMapType())
|
||||
.withBitCaskDiskMapCompressionEnabled(writeConfig.getCommonConfig().isBitCaskDiskMapCompressionEnabled())
|
||||
.build();
|
||||
}
|
||||
|
||||
private static Boolean string2Boolean(String s) {
|
||||
return "true".equals(s.toLowerCase(Locale.ROOT));
|
||||
}
|
||||
|
||||
public static org.apache.hadoop.conf.Configuration getParquetConf(
|
||||
org.apache.flink.configuration.Configuration options,
|
||||
org.apache.hadoop.conf.Configuration hadoopConf) {
|
||||
final String prefix = "parquet.";
|
||||
org.apache.hadoop.conf.Configuration copy = new org.apache.hadoop.conf.Configuration(hadoopConf);
|
||||
Map<String, String> parquetOptions = FlinkOptions.getHoodiePropertiesWithPrefix(options.toMap(), prefix);
|
||||
parquetOptions.forEach((k, v) -> copy.set(prefix + k, v));
|
||||
return copy;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,393 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.format.cow;
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.table.format.cow.vector.reader.ParquetColumnarRowSplitReader;
|
||||
|
||||
import org.apache.flink.api.common.io.FileInputFormat;
|
||||
import org.apache.flink.api.common.io.FilePathFilter;
|
||||
import org.apache.flink.api.common.io.GlobFilePathFilter;
|
||||
import org.apache.flink.api.common.io.compression.InflaterInputStreamFactory;
|
||||
import org.apache.flink.core.fs.FileInputSplit;
|
||||
import org.apache.flink.core.fs.Path;
|
||||
import org.apache.flink.formats.parquet.utils.SerializableConfiguration;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.types.DataType;
|
||||
import org.apache.flink.table.utils.PartitionPathUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.BlockLocation;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.apache.flink.table.data.vector.VectorizedColumnBatch.DEFAULT_SIZE;
|
||||
import static org.apache.flink.table.filesystem.RowPartitionComputer.restorePartValueFromType;
|
||||
|
||||
/**
|
||||
* An implementation of {@link FileInputFormat} to read {@link RowData} records
|
||||
* from Parquet files.
|
||||
*
|
||||
* <p>Note: Reference Flink release 1.11.2
|
||||
* {@code org.apache.flink.formats.parquet.ParquetFileSystemFormatFactory.ParquetInputFormat}
|
||||
* to support TIMESTAMP_MILLIS.
|
||||
*
|
||||
* <p>Note: Override the {@link #createInputSplits} method from parent to rewrite the logic creating the FileSystem,
|
||||
* use {@link FSUtils#getFs} to get a plugin filesystem.
|
||||
*
|
||||
* @see ParquetSplitReaderUtil
|
||||
*/
|
||||
public class CopyOnWriteInputFormat extends FileInputFormat<RowData> {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(CopyOnWriteInputFormat.class);
|
||||
|
||||
private final String[] fullFieldNames;
|
||||
private final DataType[] fullFieldTypes;
|
||||
private final int[] selectedFields;
|
||||
private final String partDefaultName;
|
||||
private final boolean utcTimestamp;
|
||||
private final SerializableConfiguration conf;
|
||||
private final long limit;
|
||||
|
||||
private transient ParquetColumnarRowSplitReader reader;
|
||||
private transient long currentReadCount;
|
||||
|
||||
/**
|
||||
* Files filter for determining what files/directories should be included.
|
||||
*/
|
||||
private FilePathFilter localFilesFilter = new GlobFilePathFilter();
|
||||
|
||||
public CopyOnWriteInputFormat(
|
||||
Path[] paths,
|
||||
String[] fullFieldNames,
|
||||
DataType[] fullFieldTypes,
|
||||
int[] selectedFields,
|
||||
String partDefaultName,
|
||||
long limit,
|
||||
Configuration conf,
|
||||
boolean utcTimestamp) {
|
||||
super.setFilePaths(paths);
|
||||
this.limit = limit;
|
||||
this.partDefaultName = partDefaultName;
|
||||
this.fullFieldNames = fullFieldNames;
|
||||
this.fullFieldTypes = fullFieldTypes;
|
||||
this.selectedFields = selectedFields;
|
||||
this.conf = new SerializableConfiguration(conf);
|
||||
this.utcTimestamp = utcTimestamp;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(FileInputSplit fileSplit) throws IOException {
|
||||
// generate partition specs.
|
||||
List<String> fieldNameList = Arrays.asList(fullFieldNames);
|
||||
LinkedHashMap<String, String> partSpec = PartitionPathUtils.extractPartitionSpecFromPath(
|
||||
fileSplit.getPath());
|
||||
LinkedHashMap<String, Object> partObjects = new LinkedHashMap<>();
|
||||
partSpec.forEach((k, v) -> partObjects.put(k, restorePartValueFromType(
|
||||
partDefaultName.equals(v) ? null : v,
|
||||
fullFieldTypes[fieldNameList.indexOf(k)])));
|
||||
|
||||
this.reader = ParquetSplitReaderUtil.genPartColumnarRowReader(
|
||||
utcTimestamp,
|
||||
true,
|
||||
conf.conf(),
|
||||
fullFieldNames,
|
||||
fullFieldTypes,
|
||||
partObjects,
|
||||
selectedFields,
|
||||
DEFAULT_SIZE,
|
||||
fileSplit.getPath(),
|
||||
fileSplit.getStart(),
|
||||
fileSplit.getLength());
|
||||
this.currentReadCount = 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileInputSplit[] createInputSplits(int minNumSplits) throws IOException {
|
||||
if (minNumSplits < 1) {
|
||||
throw new IllegalArgumentException("Number of input splits has to be at least 1.");
|
||||
}
|
||||
|
||||
// take the desired number of splits into account
|
||||
minNumSplits = Math.max(minNumSplits, this.numSplits);
|
||||
|
||||
final List<FileInputSplit> inputSplits = new ArrayList<>(minNumSplits);
|
||||
|
||||
// get all the files that are involved in the splits
|
||||
List<FileStatus> files = new ArrayList<>();
|
||||
long totalLength = 0;
|
||||
|
||||
for (Path path : getFilePaths()) {
|
||||
final org.apache.hadoop.fs.Path hadoopPath = new org.apache.hadoop.fs.Path(path.toUri());
|
||||
final FileSystem fs = FSUtils.getFs(hadoopPath.toString(), this.conf.conf());
|
||||
final FileStatus pathFile = fs.getFileStatus(hadoopPath);
|
||||
|
||||
if (pathFile.isDirectory()) {
|
||||
totalLength += addFilesInDir(hadoopPath, files, true);
|
||||
} else {
|
||||
testForUnsplittable(pathFile);
|
||||
|
||||
files.add(pathFile);
|
||||
totalLength += pathFile.getLen();
|
||||
}
|
||||
}
|
||||
|
||||
// returns if unsplittable
|
||||
if (unsplittable) {
|
||||
int splitNum = 0;
|
||||
for (final FileStatus file : files) {
|
||||
final FileSystem fs = FSUtils.getFs(file.getPath().toString(), this.conf.conf());
|
||||
final BlockLocation[] blocks = fs.getFileBlockLocations(file, 0, file.getLen());
|
||||
Set<String> hosts = new HashSet<>();
|
||||
for (BlockLocation block : blocks) {
|
||||
hosts.addAll(Arrays.asList(block.getHosts()));
|
||||
}
|
||||
long len = file.getLen();
|
||||
if (testForUnsplittable(file)) {
|
||||
len = READ_WHOLE_SPLIT_FLAG;
|
||||
}
|
||||
FileInputSplit fis = new FileInputSplit(splitNum++, new Path(file.getPath().toUri()), 0, len,
|
||||
hosts.toArray(new String[0]));
|
||||
inputSplits.add(fis);
|
||||
}
|
||||
return inputSplits.toArray(new FileInputSplit[0]);
|
||||
}
|
||||
|
||||
|
||||
final long maxSplitSize = totalLength / minNumSplits + (totalLength % minNumSplits == 0 ? 0 : 1);
|
||||
|
||||
// now that we have the files, generate the splits
|
||||
int splitNum = 0;
|
||||
for (final FileStatus file : files) {
|
||||
|
||||
final FileSystem fs = FSUtils.getFs(file.getPath().toString(), this.conf.conf());
|
||||
final long len = file.getLen();
|
||||
final long blockSize = file.getBlockSize();
|
||||
|
||||
final long minSplitSize;
|
||||
if (this.minSplitSize <= blockSize) {
|
||||
minSplitSize = this.minSplitSize;
|
||||
} else {
|
||||
if (LOG.isWarnEnabled()) {
|
||||
LOG.warn("Minimal split size of " + this.minSplitSize + " is larger than the block size of "
|
||||
+ blockSize + ". Decreasing minimal split size to block size.");
|
||||
}
|
||||
minSplitSize = blockSize;
|
||||
}
|
||||
|
||||
final long splitSize = Math.max(minSplitSize, Math.min(maxSplitSize, blockSize));
|
||||
final long halfSplit = splitSize >>> 1;
|
||||
|
||||
final long maxBytesForLastSplit = (long) (splitSize * 1.1f);
|
||||
|
||||
if (len > 0) {
|
||||
|
||||
// get the block locations and make sure they are in order with respect to their offset
|
||||
final BlockLocation[] blocks = fs.getFileBlockLocations(file, 0, len);
|
||||
Arrays.sort(blocks, new Comparator<BlockLocation>() {
|
||||
@Override
|
||||
public int compare(BlockLocation o1, BlockLocation o2) {
|
||||
long diff = o1.getLength() - o2.getOffset();
|
||||
return Long.compare(diff, 0L);
|
||||
}
|
||||
});
|
||||
|
||||
long bytesUnassigned = len;
|
||||
long position = 0;
|
||||
|
||||
int blockIndex = 0;
|
||||
|
||||
while (bytesUnassigned > maxBytesForLastSplit) {
|
||||
// get the block containing the majority of the data
|
||||
blockIndex = getBlockIndexForPosition(blocks, position, halfSplit, blockIndex);
|
||||
// create a new split
|
||||
FileInputSplit fis = new FileInputSplit(splitNum++, new Path(file.getPath().toUri()), position, splitSize,
|
||||
blocks[blockIndex].getHosts());
|
||||
inputSplits.add(fis);
|
||||
|
||||
// adjust the positions
|
||||
position += splitSize;
|
||||
bytesUnassigned -= splitSize;
|
||||
}
|
||||
|
||||
// assign the last split
|
||||
if (bytesUnassigned > 0) {
|
||||
blockIndex = getBlockIndexForPosition(blocks, position, halfSplit, blockIndex);
|
||||
final FileInputSplit fis = new FileInputSplit(splitNum++, new Path(file.getPath().toUri()), position,
|
||||
bytesUnassigned, blocks[blockIndex].getHosts());
|
||||
inputSplits.add(fis);
|
||||
}
|
||||
} else {
|
||||
// special case with a file of zero bytes size
|
||||
final BlockLocation[] blocks = fs.getFileBlockLocations(file, 0, 0);
|
||||
String[] hosts;
|
||||
if (blocks.length > 0) {
|
||||
hosts = blocks[0].getHosts();
|
||||
} else {
|
||||
hosts = new String[0];
|
||||
}
|
||||
final FileInputSplit fis = new FileInputSplit(splitNum++, new Path(file.getPath().toUri()), 0, 0, hosts);
|
||||
inputSplits.add(fis);
|
||||
}
|
||||
}
|
||||
|
||||
return inputSplits.toArray(new FileInputSplit[0]);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean supportsMultiPaths() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean reachedEnd() throws IOException {
|
||||
if (currentReadCount >= limit) {
|
||||
return true;
|
||||
} else {
|
||||
return reader.reachedEnd();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowData nextRecord(RowData reuse) {
|
||||
currentReadCount++;
|
||||
return reader.nextRecord();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (reader != null) {
|
||||
this.reader.close();
|
||||
}
|
||||
this.reader = null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Enumerate all files in the directory and recursive if enumerateNestedFiles is true.
|
||||
*
|
||||
* @return the total length of accepted files.
|
||||
*/
|
||||
private long addFilesInDir(org.apache.hadoop.fs.Path path, List<FileStatus> files, boolean logExcludedFiles)
|
||||
throws IOException {
|
||||
final org.apache.hadoop.fs.Path hadoopPath = new org.apache.hadoop.fs.Path(path.toUri());
|
||||
final FileSystem fs = FSUtils.getFs(hadoopPath.toString(), this.conf.conf());
|
||||
|
||||
long length = 0;
|
||||
|
||||
for (FileStatus dir : fs.listStatus(hadoopPath)) {
|
||||
if (dir.isDirectory()) {
|
||||
if (acceptFile(dir) && enumerateNestedFiles) {
|
||||
length += addFilesInDir(dir.getPath(), files, logExcludedFiles);
|
||||
} else {
|
||||
if (logExcludedFiles && LOG.isDebugEnabled()) {
|
||||
LOG.debug("Directory " + dir.getPath().toString() + " did not pass the file-filter and is excluded.");
|
||||
}
|
||||
}
|
||||
} else {
|
||||
if (acceptFile(dir)) {
|
||||
files.add(dir);
|
||||
length += dir.getLen();
|
||||
testForUnsplittable(dir);
|
||||
} else {
|
||||
if (logExcludedFiles && LOG.isDebugEnabled()) {
|
||||
LOG.debug("Directory " + dir.getPath().toString() + " did not pass the file-filter and is excluded.");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setFilesFilter(FilePathFilter filesFilter) {
|
||||
this.localFilesFilter = filesFilter;
|
||||
super.setFilesFilter(filesFilter);
|
||||
}
|
||||
|
||||
/**
|
||||
* A simple hook to filter files and directories from the input.
|
||||
* The method may be overridden. Hadoop's FileInputFormat has a similar mechanism and applies the
|
||||
* same filters by default.
|
||||
*
|
||||
* @param fileStatus The file status to check.
|
||||
* @return true, if the given file or directory is accepted
|
||||
*/
|
||||
public boolean acceptFile(FileStatus fileStatus) {
|
||||
final String name = fileStatus.getPath().getName();
|
||||
return !name.startsWith("_")
|
||||
&& !name.startsWith(".")
|
||||
&& !localFilesFilter.filterPath(new Path(fileStatus.getPath().toUri()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieves the index of the <tt>BlockLocation</tt> that contains the part of the file described by the given
|
||||
* offset.
|
||||
*
|
||||
* @param blocks The different blocks of the file. Must be ordered by their offset.
|
||||
* @param offset The offset of the position in the file.
|
||||
* @param startIndex The earliest index to look at.
|
||||
* @return The index of the block containing the given position.
|
||||
*/
|
||||
private int getBlockIndexForPosition(BlockLocation[] blocks, long offset, long halfSplitSize, int startIndex) {
|
||||
// go over all indexes after the startIndex
|
||||
for (int i = startIndex; i < blocks.length; i++) {
|
||||
long blockStart = blocks[i].getOffset();
|
||||
long blockEnd = blockStart + blocks[i].getLength();
|
||||
|
||||
if (offset >= blockStart && offset < blockEnd) {
|
||||
// got the block where the split starts
|
||||
// check if the next block contains more than this one does
|
||||
if (i < blocks.length - 1 && blockEnd - offset < halfSplitSize) {
|
||||
return i + 1;
|
||||
} else {
|
||||
return i;
|
||||
}
|
||||
}
|
||||
}
|
||||
throw new IllegalArgumentException("The given offset is not contained in the any block.");
|
||||
}
|
||||
|
||||
private boolean testForUnsplittable(FileStatus pathFile) {
|
||||
if (getInflaterInputStreamFactory(pathFile.getPath()) != null) {
|
||||
unsplittable = true;
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private InflaterInputStreamFactory<?> getInflaterInputStreamFactory(org.apache.hadoop.fs.Path path) {
|
||||
String fileExtension = extractFileExtension(path.getName());
|
||||
if (fileExtension != null) {
|
||||
return getInflaterInputStreamFactory(fileExtension);
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,523 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.format.cow;
|
||||
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.table.format.cow.vector.HeapArrayVector;
|
||||
import org.apache.hudi.table.format.cow.vector.HeapMapColumnVector;
|
||||
import org.apache.hudi.table.format.cow.vector.HeapRowColumnVector;
|
||||
import org.apache.hudi.table.format.cow.vector.ParquetDecimalVector;
|
||||
import org.apache.hudi.table.format.cow.vector.reader.ArrayColumnReader;
|
||||
import org.apache.hudi.table.format.cow.vector.reader.FixedLenBytesColumnReader;
|
||||
import org.apache.hudi.table.format.cow.vector.reader.Int64TimestampColumnReader;
|
||||
import org.apache.hudi.table.format.cow.vector.reader.MapColumnReader;
|
||||
import org.apache.hudi.table.format.cow.vector.reader.ParquetColumnarRowSplitReader;
|
||||
import org.apache.hudi.table.format.cow.vector.reader.RowColumnReader;
|
||||
|
||||
import org.apache.flink.core.fs.Path;
|
||||
import org.apache.flink.formats.parquet.vector.reader.BooleanColumnReader;
|
||||
import org.apache.flink.formats.parquet.vector.reader.ByteColumnReader;
|
||||
import org.apache.flink.formats.parquet.vector.reader.BytesColumnReader;
|
||||
import org.apache.flink.formats.parquet.vector.reader.ColumnReader;
|
||||
import org.apache.flink.formats.parquet.vector.reader.DoubleColumnReader;
|
||||
import org.apache.flink.formats.parquet.vector.reader.FloatColumnReader;
|
||||
import org.apache.flink.formats.parquet.vector.reader.IntColumnReader;
|
||||
import org.apache.flink.formats.parquet.vector.reader.LongColumnReader;
|
||||
import org.apache.flink.formats.parquet.vector.reader.ShortColumnReader;
|
||||
import org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader;
|
||||
import org.apache.flink.table.data.DecimalData;
|
||||
import org.apache.flink.table.data.TimestampData;
|
||||
import org.apache.flink.table.data.vector.ColumnVector;
|
||||
import org.apache.flink.table.data.vector.VectorizedColumnBatch;
|
||||
import org.apache.flink.table.data.vector.heap.HeapBooleanVector;
|
||||
import org.apache.flink.table.data.vector.heap.HeapByteVector;
|
||||
import org.apache.flink.table.data.vector.heap.HeapBytesVector;
|
||||
import org.apache.flink.table.data.vector.heap.HeapDoubleVector;
|
||||
import org.apache.flink.table.data.vector.heap.HeapFloatVector;
|
||||
import org.apache.flink.table.data.vector.heap.HeapIntVector;
|
||||
import org.apache.flink.table.data.vector.heap.HeapLongVector;
|
||||
import org.apache.flink.table.data.vector.heap.HeapShortVector;
|
||||
import org.apache.flink.table.data.vector.heap.HeapTimestampVector;
|
||||
import org.apache.flink.table.data.vector.writable.WritableColumnVector;
|
||||
import org.apache.flink.table.types.DataType;
|
||||
import org.apache.flink.table.types.logical.ArrayType;
|
||||
import org.apache.flink.table.types.logical.DecimalType;
|
||||
import org.apache.flink.table.types.logical.IntType;
|
||||
import org.apache.flink.table.types.logical.LogicalType;
|
||||
import org.apache.flink.table.types.logical.MapType;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
import org.apache.flink.table.types.logical.TimestampType;
|
||||
import org.apache.flink.table.types.logical.VarBinaryType;
|
||||
import org.apache.flink.util.Preconditions;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.parquet.ParquetRuntimeException;
|
||||
import org.apache.parquet.column.ColumnDescriptor;
|
||||
import org.apache.parquet.column.page.PageReadStore;
|
||||
import org.apache.parquet.column.page.PageReader;
|
||||
import org.apache.parquet.schema.GroupType;
|
||||
import org.apache.parquet.schema.InvalidSchemaException;
|
||||
import org.apache.parquet.schema.OriginalType;
|
||||
import org.apache.parquet.schema.PrimitiveType;
|
||||
import org.apache.parquet.schema.Type;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.math.BigDecimal;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.sql.Date;
|
||||
import java.time.LocalDate;
|
||||
import java.time.LocalDateTime;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.flink.table.runtime.functions.SqlDateTimeUtils.dateToInternal;
|
||||
import static org.apache.parquet.Preconditions.checkArgument;
|
||||
|
||||
/**
|
||||
* Util for generating {@link ParquetColumnarRowSplitReader}.
|
||||
*
|
||||
* <p>NOTE: reference from Flink release 1.11.2 {@code ParquetSplitReaderUtil}, modify to support INT64
|
||||
* based TIMESTAMP_MILLIS as ConvertedType, should remove when Flink supports that.
|
||||
*/
|
||||
public class ParquetSplitReaderUtil {
|
||||
|
||||
/**
|
||||
* Util for generating partitioned {@link ParquetColumnarRowSplitReader}.
|
||||
*/
|
||||
public static ParquetColumnarRowSplitReader genPartColumnarRowReader(
|
||||
boolean utcTimestamp,
|
||||
boolean caseSensitive,
|
||||
Configuration conf,
|
||||
String[] fullFieldNames,
|
||||
DataType[] fullFieldTypes,
|
||||
Map<String, Object> partitionSpec,
|
||||
int[] selectedFields,
|
||||
int batchSize,
|
||||
Path path,
|
||||
long splitStart,
|
||||
long splitLength) throws IOException {
|
||||
List<String> selNonPartNames = Arrays.stream(selectedFields)
|
||||
.mapToObj(i -> fullFieldNames[i])
|
||||
.filter(n -> !partitionSpec.containsKey(n))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
int[] selParquetFields = Arrays.stream(selectedFields)
|
||||
.filter(i -> !partitionSpec.containsKey(fullFieldNames[i]))
|
||||
.toArray();
|
||||
|
||||
ParquetColumnarRowSplitReader.ColumnBatchGenerator gen = readVectors -> {
|
||||
// create and initialize the row batch
|
||||
ColumnVector[] vectors = new ColumnVector[selectedFields.length];
|
||||
for (int i = 0; i < vectors.length; i++) {
|
||||
String name = fullFieldNames[selectedFields[i]];
|
||||
LogicalType type = fullFieldTypes[selectedFields[i]].getLogicalType();
|
||||
vectors[i] = createVector(readVectors, selNonPartNames, name, type, partitionSpec, batchSize);
|
||||
}
|
||||
return new VectorizedColumnBatch(vectors);
|
||||
};
|
||||
|
||||
return new ParquetColumnarRowSplitReader(
|
||||
utcTimestamp,
|
||||
caseSensitive,
|
||||
conf,
|
||||
Arrays.stream(selParquetFields)
|
||||
.mapToObj(i -> fullFieldTypes[i].getLogicalType())
|
||||
.toArray(LogicalType[]::new),
|
||||
selNonPartNames.toArray(new String[0]),
|
||||
gen,
|
||||
batchSize,
|
||||
new org.apache.hadoop.fs.Path(path.toUri()),
|
||||
splitStart,
|
||||
splitLength);
|
||||
}
|
||||
|
||||
private static ColumnVector createVector(
|
||||
ColumnVector[] readVectors,
|
||||
List<String> selNonPartNames,
|
||||
String name,
|
||||
LogicalType type,
|
||||
Map<String, Object> partitionSpec,
|
||||
int batchSize) {
|
||||
if (partitionSpec.containsKey(name)) {
|
||||
return createVectorFromConstant(type, partitionSpec.get(name), batchSize);
|
||||
}
|
||||
ColumnVector readVector = readVectors[selNonPartNames.indexOf(name)];
|
||||
if (readVector == null) {
|
||||
// when the read vector is null, use a constant null vector instead
|
||||
readVector = createVectorFromConstant(type, null, batchSize);
|
||||
}
|
||||
return readVector;
|
||||
}
|
||||
|
||||
private static ColumnVector createVectorFromConstant(
|
||||
LogicalType type,
|
||||
Object value,
|
||||
int batchSize) {
|
||||
switch (type.getTypeRoot()) {
|
||||
case CHAR:
|
||||
case VARCHAR:
|
||||
case BINARY:
|
||||
case VARBINARY:
|
||||
HeapBytesVector bsv = new HeapBytesVector(batchSize);
|
||||
if (value == null) {
|
||||
bsv.fillWithNulls();
|
||||
} else {
|
||||
bsv.fill(value instanceof byte[]
|
||||
? (byte[]) value
|
||||
: value.toString().getBytes(StandardCharsets.UTF_8));
|
||||
}
|
||||
return bsv;
|
||||
case BOOLEAN:
|
||||
HeapBooleanVector bv = new HeapBooleanVector(batchSize);
|
||||
if (value == null) {
|
||||
bv.fillWithNulls();
|
||||
} else {
|
||||
bv.fill((boolean) value);
|
||||
}
|
||||
return bv;
|
||||
case TINYINT:
|
||||
HeapByteVector byteVector = new HeapByteVector(batchSize);
|
||||
if (value == null) {
|
||||
byteVector.fillWithNulls();
|
||||
} else {
|
||||
byteVector.fill(((Number) value).byteValue());
|
||||
}
|
||||
return byteVector;
|
||||
case SMALLINT:
|
||||
HeapShortVector sv = new HeapShortVector(batchSize);
|
||||
if (value == null) {
|
||||
sv.fillWithNulls();
|
||||
} else {
|
||||
sv.fill(((Number) value).shortValue());
|
||||
}
|
||||
return sv;
|
||||
case INTEGER:
|
||||
HeapIntVector iv = new HeapIntVector(batchSize);
|
||||
if (value == null) {
|
||||
iv.fillWithNulls();
|
||||
} else {
|
||||
iv.fill(((Number) value).intValue());
|
||||
}
|
||||
return iv;
|
||||
case BIGINT:
|
||||
HeapLongVector lv = new HeapLongVector(batchSize);
|
||||
if (value == null) {
|
||||
lv.fillWithNulls();
|
||||
} else {
|
||||
lv.fill(((Number) value).longValue());
|
||||
}
|
||||
return lv;
|
||||
case DECIMAL:
|
||||
DecimalType decimalType = (DecimalType) type;
|
||||
int precision = decimalType.getPrecision();
|
||||
int scale = decimalType.getScale();
|
||||
DecimalData decimal = value == null
|
||||
? null
|
||||
: Preconditions.checkNotNull(DecimalData.fromBigDecimal((BigDecimal) value, precision, scale));
|
||||
ColumnVector internalVector = createVectorFromConstant(
|
||||
new VarBinaryType(),
|
||||
decimal == null ? null : decimal.toUnscaledBytes(),
|
||||
batchSize);
|
||||
return new ParquetDecimalVector(internalVector);
|
||||
case FLOAT:
|
||||
HeapFloatVector fv = new HeapFloatVector(batchSize);
|
||||
if (value == null) {
|
||||
fv.fillWithNulls();
|
||||
} else {
|
||||
fv.fill(((Number) value).floatValue());
|
||||
}
|
||||
return fv;
|
||||
case DOUBLE:
|
||||
HeapDoubleVector dv = new HeapDoubleVector(batchSize);
|
||||
if (value == null) {
|
||||
dv.fillWithNulls();
|
||||
} else {
|
||||
dv.fill(((Number) value).doubleValue());
|
||||
}
|
||||
return dv;
|
||||
case DATE:
|
||||
if (value instanceof LocalDate) {
|
||||
value = Date.valueOf((LocalDate) value);
|
||||
}
|
||||
return createVectorFromConstant(
|
||||
new IntType(),
|
||||
value == null ? null : dateToInternal((Date) value),
|
||||
batchSize);
|
||||
case TIMESTAMP_WITHOUT_TIME_ZONE:
|
||||
HeapTimestampVector tv = new HeapTimestampVector(batchSize);
|
||||
if (value == null) {
|
||||
tv.fillWithNulls();
|
||||
} else {
|
||||
tv.fill(TimestampData.fromLocalDateTime((LocalDateTime) value));
|
||||
}
|
||||
return tv;
|
||||
default:
|
||||
throw new UnsupportedOperationException("Unsupported type: " + type);
|
||||
}
|
||||
}
|
||||
|
||||
private static List<ColumnDescriptor> filterDescriptors(int depth, Type type, List<ColumnDescriptor> columns) throws ParquetRuntimeException {
|
||||
List<ColumnDescriptor> filtered = new ArrayList<>();
|
||||
for (ColumnDescriptor descriptor : columns) {
|
||||
if (depth >= descriptor.getPath().length) {
|
||||
throw new InvalidSchemaException("Expect depth " + depth + " for schema: " + descriptor);
|
||||
}
|
||||
if (type.getName().equals(descriptor.getPath()[depth])) {
|
||||
filtered.add(descriptor);
|
||||
}
|
||||
}
|
||||
ValidationUtils.checkState(filtered.size() > 0, "Corrupted Parquet schema");
|
||||
return filtered;
|
||||
}
|
||||
|
||||
public static ColumnReader createColumnReader(
|
||||
boolean utcTimestamp,
|
||||
LogicalType fieldType,
|
||||
Type physicalType,
|
||||
List<ColumnDescriptor> descriptors,
|
||||
PageReadStore pages) throws IOException {
|
||||
return createColumnReader(utcTimestamp, fieldType, physicalType, descriptors,
|
||||
pages, 0);
|
||||
}
|
||||
|
||||
private static ColumnReader createColumnReader(
|
||||
boolean utcTimestamp,
|
||||
LogicalType fieldType,
|
||||
Type physicalType,
|
||||
List<ColumnDescriptor> columns,
|
||||
PageReadStore pages,
|
||||
int depth) throws IOException {
|
||||
List<ColumnDescriptor> descriptors = filterDescriptors(depth, physicalType, columns);
|
||||
ColumnDescriptor descriptor = descriptors.get(0);
|
||||
PageReader pageReader = pages.getPageReader(descriptor);
|
||||
switch (fieldType.getTypeRoot()) {
|
||||
case BOOLEAN:
|
||||
return new BooleanColumnReader(descriptor, pageReader);
|
||||
case TINYINT:
|
||||
return new ByteColumnReader(descriptor, pageReader);
|
||||
case DOUBLE:
|
||||
return new DoubleColumnReader(descriptor, pageReader);
|
||||
case FLOAT:
|
||||
return new FloatColumnReader(descriptor, pageReader);
|
||||
case INTEGER:
|
||||
case DATE:
|
||||
case TIME_WITHOUT_TIME_ZONE:
|
||||
return new IntColumnReader(descriptor, pageReader);
|
||||
case BIGINT:
|
||||
return new LongColumnReader(descriptor, pageReader);
|
||||
case SMALLINT:
|
||||
return new ShortColumnReader(descriptor, pageReader);
|
||||
case CHAR:
|
||||
case VARCHAR:
|
||||
case BINARY:
|
||||
case VARBINARY:
|
||||
return new BytesColumnReader(descriptor, pageReader);
|
||||
case TIMESTAMP_WITHOUT_TIME_ZONE:
|
||||
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
|
||||
switch (descriptor.getPrimitiveType().getPrimitiveTypeName()) {
|
||||
case INT64:
|
||||
return new Int64TimestampColumnReader(utcTimestamp, descriptor, pageReader, ((TimestampType) fieldType).getPrecision());
|
||||
case INT96:
|
||||
return new TimestampColumnReader(utcTimestamp, descriptor, pageReader);
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
case DECIMAL:
|
||||
switch (descriptor.getPrimitiveType().getPrimitiveTypeName()) {
|
||||
case INT32:
|
||||
return new IntColumnReader(descriptor, pageReader);
|
||||
case INT64:
|
||||
return new LongColumnReader(descriptor, pageReader);
|
||||
case BINARY:
|
||||
return new BytesColumnReader(descriptor, pageReader);
|
||||
case FIXED_LEN_BYTE_ARRAY:
|
||||
return new FixedLenBytesColumnReader(
|
||||
descriptor, pageReader, ((DecimalType) fieldType).getPrecision());
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
case ARRAY:
|
||||
return new ArrayColumnReader(
|
||||
descriptor,
|
||||
pageReader,
|
||||
utcTimestamp,
|
||||
descriptor.getPrimitiveType(),
|
||||
fieldType);
|
||||
case MAP:
|
||||
MapType mapType = (MapType) fieldType;
|
||||
ArrayColumnReader keyReader =
|
||||
new ArrayColumnReader(
|
||||
descriptor,
|
||||
pageReader,
|
||||
utcTimestamp,
|
||||
descriptor.getPrimitiveType(),
|
||||
new ArrayType(mapType.getKeyType()));
|
||||
ArrayColumnReader valueReader =
|
||||
new ArrayColumnReader(
|
||||
descriptors.get(1),
|
||||
pages.getPageReader(descriptors.get(1)),
|
||||
utcTimestamp,
|
||||
descriptors.get(1).getPrimitiveType(),
|
||||
new ArrayType(mapType.getValueType()));
|
||||
return new MapColumnReader(keyReader, valueReader, fieldType);
|
||||
case ROW:
|
||||
RowType rowType = (RowType) fieldType;
|
||||
GroupType groupType = physicalType.asGroupType();
|
||||
List<ColumnReader> fieldReaders = new ArrayList<>();
|
||||
for (int i = 0; i < rowType.getFieldCount(); i++) {
|
||||
fieldReaders.add(
|
||||
createColumnReader(
|
||||
utcTimestamp,
|
||||
rowType.getTypeAt(i),
|
||||
groupType.getType(i),
|
||||
descriptors,
|
||||
pages,
|
||||
depth + 1));
|
||||
}
|
||||
return new RowColumnReader(fieldReaders);
|
||||
default:
|
||||
throw new UnsupportedOperationException(fieldType + " is not supported now.");
|
||||
}
|
||||
}
|
||||
|
||||
public static WritableColumnVector createWritableColumnVector(
|
||||
int batchSize,
|
||||
LogicalType fieldType,
|
||||
Type physicalType,
|
||||
List<ColumnDescriptor> descriptors) {
|
||||
return createWritableColumnVector(batchSize, fieldType, physicalType, descriptors, 0);
|
||||
}
|
||||
|
||||
private static WritableColumnVector createWritableColumnVector(
|
||||
int batchSize,
|
||||
LogicalType fieldType,
|
||||
Type physicalType,
|
||||
List<ColumnDescriptor> columns,
|
||||
int depth) {
|
||||
List<ColumnDescriptor> descriptors = filterDescriptors(depth, physicalType, columns);
|
||||
PrimitiveType primitiveType = descriptors.get(0).getPrimitiveType();
|
||||
PrimitiveType.PrimitiveTypeName typeName = primitiveType.getPrimitiveTypeName();
|
||||
switch (fieldType.getTypeRoot()) {
|
||||
case BOOLEAN:
|
||||
checkArgument(
|
||||
typeName == PrimitiveType.PrimitiveTypeName.BOOLEAN,
|
||||
"Unexpected type: %s", typeName);
|
||||
return new HeapBooleanVector(batchSize);
|
||||
case TINYINT:
|
||||
checkArgument(
|
||||
typeName == PrimitiveType.PrimitiveTypeName.INT32,
|
||||
"Unexpected type: %s", typeName);
|
||||
return new HeapByteVector(batchSize);
|
||||
case DOUBLE:
|
||||
checkArgument(
|
||||
typeName == PrimitiveType.PrimitiveTypeName.DOUBLE,
|
||||
"Unexpected type: %s", typeName);
|
||||
return new HeapDoubleVector(batchSize);
|
||||
case FLOAT:
|
||||
checkArgument(
|
||||
typeName == PrimitiveType.PrimitiveTypeName.FLOAT,
|
||||
"Unexpected type: %s", typeName);
|
||||
return new HeapFloatVector(batchSize);
|
||||
case INTEGER:
|
||||
case DATE:
|
||||
case TIME_WITHOUT_TIME_ZONE:
|
||||
checkArgument(
|
||||
typeName == PrimitiveType.PrimitiveTypeName.INT32,
|
||||
"Unexpected type: %s", typeName);
|
||||
return new HeapIntVector(batchSize);
|
||||
case BIGINT:
|
||||
checkArgument(
|
||||
typeName == PrimitiveType.PrimitiveTypeName.INT64,
|
||||
"Unexpected type: %s", typeName);
|
||||
return new HeapLongVector(batchSize);
|
||||
case SMALLINT:
|
||||
checkArgument(
|
||||
typeName == PrimitiveType.PrimitiveTypeName.INT32,
|
||||
"Unexpected type: %s", typeName);
|
||||
return new HeapShortVector(batchSize);
|
||||
case CHAR:
|
||||
case VARCHAR:
|
||||
case BINARY:
|
||||
case VARBINARY:
|
||||
checkArgument(
|
||||
typeName == PrimitiveType.PrimitiveTypeName.BINARY,
|
||||
"Unexpected type: %s", typeName);
|
||||
return new HeapBytesVector(batchSize);
|
||||
case TIMESTAMP_WITHOUT_TIME_ZONE:
|
||||
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
|
||||
checkArgument(primitiveType.getOriginalType() != OriginalType.TIME_MICROS,
|
||||
"TIME_MICROS original type is not ");
|
||||
return new HeapTimestampVector(batchSize);
|
||||
case DECIMAL:
|
||||
checkArgument(
|
||||
(typeName == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY
|
||||
|| typeName == PrimitiveType.PrimitiveTypeName.BINARY)
|
||||
&& primitiveType.getOriginalType() == OriginalType.DECIMAL,
|
||||
"Unexpected type: %s", typeName);
|
||||
return new HeapBytesVector(batchSize);
|
||||
case ARRAY:
|
||||
ArrayType arrayType = (ArrayType) fieldType;
|
||||
return new HeapArrayVector(
|
||||
batchSize,
|
||||
createWritableColumnVector(
|
||||
batchSize,
|
||||
arrayType.getElementType(),
|
||||
physicalType,
|
||||
descriptors,
|
||||
depth));
|
||||
case MAP:
|
||||
MapType mapType = (MapType) fieldType;
|
||||
GroupType repeatedType = physicalType.asGroupType().getType(0).asGroupType();
|
||||
// the map column has three level paths.
|
||||
return new HeapMapColumnVector(
|
||||
batchSize,
|
||||
createWritableColumnVector(
|
||||
batchSize,
|
||||
mapType.getKeyType(),
|
||||
repeatedType.getType(0),
|
||||
descriptors,
|
||||
depth + 2),
|
||||
createWritableColumnVector(
|
||||
batchSize,
|
||||
mapType.getValueType(),
|
||||
repeatedType.getType(1),
|
||||
descriptors,
|
||||
depth + 2));
|
||||
case ROW:
|
||||
RowType rowType = (RowType) fieldType;
|
||||
GroupType groupType = physicalType.asGroupType();
|
||||
WritableColumnVector[] columnVectors =
|
||||
new WritableColumnVector[rowType.getFieldCount()];
|
||||
for (int i = 0; i < columnVectors.length; i++) {
|
||||
columnVectors[i] =
|
||||
createWritableColumnVector(
|
||||
batchSize,
|
||||
rowType.getTypeAt(i),
|
||||
groupType.getType(i),
|
||||
descriptors,
|
||||
depth + 1);
|
||||
}
|
||||
return new HeapRowColumnVector(batchSize, columnVectors);
|
||||
default:
|
||||
throw new UnsupportedOperationException(fieldType + " is not supported now.");
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,70 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.format.cow.vector;
|
||||
|
||||
import org.apache.flink.table.data.ArrayData;
|
||||
import org.apache.flink.table.data.ColumnarArrayData;
|
||||
import org.apache.flink.table.data.vector.ArrayColumnVector;
|
||||
import org.apache.flink.table.data.vector.ColumnVector;
|
||||
import org.apache.flink.table.data.vector.heap.AbstractHeapVector;
|
||||
import org.apache.flink.table.data.vector.writable.WritableColumnVector;
|
||||
|
||||
/**
|
||||
* This class represents a nullable heap array column vector.
|
||||
*/
|
||||
public class HeapArrayVector extends AbstractHeapVector
|
||||
implements WritableColumnVector, ArrayColumnVector {
|
||||
|
||||
public long[] offsets;
|
||||
public long[] lengths;
|
||||
public ColumnVector child;
|
||||
private int size;
|
||||
|
||||
public HeapArrayVector(int len) {
|
||||
super(len);
|
||||
offsets = new long[len];
|
||||
lengths = new long[len];
|
||||
}
|
||||
|
||||
public HeapArrayVector(int len, ColumnVector vector) {
|
||||
super(len);
|
||||
offsets = new long[len];
|
||||
lengths = new long[len];
|
||||
this.child = vector;
|
||||
}
|
||||
|
||||
public int getSize() {
|
||||
return size;
|
||||
}
|
||||
|
||||
public void setSize(int size) {
|
||||
this.size = size;
|
||||
}
|
||||
|
||||
public int getLen() {
|
||||
return this.isNull.length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ArrayData getArray(int i) {
|
||||
long offset = offsets[i];
|
||||
long length = lengths[i];
|
||||
return new ColumnarArrayData(child, (int) offset, (int) length);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,79 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.format.cow.vector;
|
||||
|
||||
import org.apache.flink.table.data.ColumnarMapData;
|
||||
import org.apache.flink.table.data.MapData;
|
||||
import org.apache.flink.table.data.vector.ColumnVector;
|
||||
import org.apache.flink.table.data.vector.MapColumnVector;
|
||||
import org.apache.flink.table.data.vector.heap.AbstractHeapVector;
|
||||
import org.apache.flink.table.data.vector.writable.WritableColumnVector;
|
||||
|
||||
/**
|
||||
* This class represents a nullable heap map column vector.
|
||||
*/
|
||||
public class HeapMapColumnVector extends AbstractHeapVector
|
||||
implements WritableColumnVector, MapColumnVector {
|
||||
|
||||
private long[] offsets;
|
||||
private long[] lengths;
|
||||
private int size;
|
||||
private ColumnVector keys;
|
||||
private ColumnVector values;
|
||||
|
||||
public HeapMapColumnVector(int len, ColumnVector keys, ColumnVector values) {
|
||||
super(len);
|
||||
size = 0;
|
||||
offsets = new long[len];
|
||||
lengths = new long[len];
|
||||
this.keys = keys;
|
||||
this.values = values;
|
||||
}
|
||||
|
||||
public void setOffsets(long[] offsets) {
|
||||
this.offsets = offsets;
|
||||
}
|
||||
|
||||
public void setLengths(long[] lengths) {
|
||||
this.lengths = lengths;
|
||||
}
|
||||
|
||||
public void setKeys(ColumnVector keys) {
|
||||
this.keys = keys;
|
||||
}
|
||||
|
||||
public void setValues(ColumnVector values) {
|
||||
this.values = values;
|
||||
}
|
||||
|
||||
public int getSize() {
|
||||
return size;
|
||||
}
|
||||
|
||||
public void setSize(int size) {
|
||||
this.size = size;
|
||||
}
|
||||
|
||||
@Override
|
||||
public MapData getMap(int i) {
|
||||
long offset = offsets[i];
|
||||
long length = lengths[i];
|
||||
return new ColumnarMapData(keys, values, (int) offset, (int) length);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,46 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.format.cow.vector;
|
||||
|
||||
import org.apache.flink.table.data.ColumnarRowData;
|
||||
import org.apache.flink.table.data.vector.RowColumnVector;
|
||||
import org.apache.flink.table.data.vector.VectorizedColumnBatch;
|
||||
import org.apache.flink.table.data.vector.heap.AbstractHeapVector;
|
||||
import org.apache.flink.table.data.vector.writable.WritableColumnVector;
|
||||
|
||||
/**
|
||||
* This class represents a nullable heap row column vector.
|
||||
*/
|
||||
public class HeapRowColumnVector extends AbstractHeapVector
|
||||
implements WritableColumnVector, RowColumnVector {
|
||||
|
||||
public WritableColumnVector[] vectors;
|
||||
|
||||
public HeapRowColumnVector(int len, WritableColumnVector... vectors) {
|
||||
super(len);
|
||||
this.vectors = vectors;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnarRowData getRow(int i) {
|
||||
ColumnarRowData columnarRowData = new ColumnarRowData(new VectorizedColumnBatch(vectors));
|
||||
columnarRowData.setRowId(i);
|
||||
return columnarRowData;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,54 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.format.cow.vector;
|
||||
|
||||
import org.apache.flink.table.data.DecimalData;
|
||||
import org.apache.flink.table.data.vector.BytesColumnVector;
|
||||
import org.apache.flink.table.data.vector.ColumnVector;
|
||||
import org.apache.flink.table.data.vector.DecimalColumnVector;
|
||||
|
||||
/**
|
||||
* Parquet write decimal as int32 and int64 and binary, this class wrap the real vector to
|
||||
* provide {@link DecimalColumnVector} interface.
|
||||
*
|
||||
* <p>Reference Flink release 1.11.2 {@link org.apache.flink.formats.parquet.vector.ParquetDecimalVector}
|
||||
* because it is not public.
|
||||
*/
|
||||
public class ParquetDecimalVector implements DecimalColumnVector {
|
||||
|
||||
public final ColumnVector vector;
|
||||
|
||||
public ParquetDecimalVector(ColumnVector vector) {
|
||||
this.vector = vector;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DecimalData getDecimal(int i, int precision, int scale) {
|
||||
return DecimalData.fromUnscaledBytes(
|
||||
((BytesColumnVector) vector).getBytes(i).getBytes(),
|
||||
precision,
|
||||
scale);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNullAt(int i) {
|
||||
return vector.isNullAt(i);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,325 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.format.cow.vector.reader;
|
||||
|
||||
import org.apache.flink.formats.parquet.vector.ParquetDictionary;
|
||||
import org.apache.flink.formats.parquet.vector.reader.ColumnReader;
|
||||
import org.apache.flink.table.data.vector.writable.WritableColumnVector;
|
||||
import org.apache.flink.table.data.vector.writable.WritableIntVector;
|
||||
import org.apache.parquet.Preconditions;
|
||||
import org.apache.parquet.bytes.ByteBufferInputStream;
|
||||
import org.apache.parquet.bytes.BytesInput;
|
||||
import org.apache.parquet.bytes.BytesUtils;
|
||||
import org.apache.parquet.column.ColumnDescriptor;
|
||||
import org.apache.parquet.column.Dictionary;
|
||||
import org.apache.parquet.column.Encoding;
|
||||
import org.apache.parquet.column.page.DataPage;
|
||||
import org.apache.parquet.column.page.DataPageV1;
|
||||
import org.apache.parquet.column.page.DataPageV2;
|
||||
import org.apache.parquet.column.page.DictionaryPage;
|
||||
import org.apache.parquet.column.page.PageReader;
|
||||
import org.apache.parquet.column.values.ValuesReader;
|
||||
import org.apache.parquet.io.ParquetDecodingException;
|
||||
import org.apache.parquet.schema.PrimitiveType;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
|
||||
import static org.apache.parquet.column.ValuesType.REPETITION_LEVEL;
|
||||
|
||||
/**
|
||||
* Abstract {@link ColumnReader}.
|
||||
* See {@link org.apache.parquet.column.impl.ColumnReaderImpl},
|
||||
* part of the code is referred from Apache Spark and Apache Parquet.
|
||||
*
|
||||
* <p>Note: Reference Flink release 1.11.2 {@link org.apache.flink.formats.parquet.vector.reader.AbstractColumnReader}
|
||||
* because some of the package scope methods.
|
||||
*/
|
||||
public abstract class AbstractColumnReader<V extends WritableColumnVector>
|
||||
implements ColumnReader<V> {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(org.apache.flink.formats.parquet.vector.reader.AbstractColumnReader.class);
|
||||
|
||||
private final PageReader pageReader;
|
||||
|
||||
/**
|
||||
* The dictionary, if this column has dictionary encoding.
|
||||
*/
|
||||
protected final Dictionary dictionary;
|
||||
|
||||
/**
|
||||
* Maximum definition level for this column.
|
||||
*/
|
||||
protected final int maxDefLevel;
|
||||
|
||||
protected final ColumnDescriptor descriptor;
|
||||
|
||||
/**
|
||||
* Total number of values read.
|
||||
*/
|
||||
private long valuesRead;
|
||||
|
||||
/**
|
||||
* value that indicates the end of the current page. That is, if valuesRead ==
|
||||
* endOfPageValueCount, we are at the end of the page.
|
||||
*/
|
||||
private long endOfPageValueCount;
|
||||
|
||||
/**
|
||||
* If true, the current page is dictionary encoded.
|
||||
*/
|
||||
private boolean isCurrentPageDictionaryEncoded;
|
||||
|
||||
/**
|
||||
* Total values in the current page.
|
||||
*/
|
||||
private int pageValueCount;
|
||||
|
||||
/*
|
||||
* Input streams:
|
||||
* 1.Run length encoder to encode every data, so we have run length stream to get
|
||||
* run length information.
|
||||
* 2.Data maybe is real data, maybe is dictionary ids which need be decode to real
|
||||
* data from Dictionary.
|
||||
*
|
||||
* Run length stream ------> Data stream
|
||||
* |
|
||||
* ------> Dictionary ids stream
|
||||
*/
|
||||
|
||||
/**
|
||||
* Run length decoder for data and dictionary.
|
||||
*/
|
||||
protected RunLengthDecoder runLenDecoder;
|
||||
|
||||
/**
|
||||
* Data input stream.
|
||||
*/
|
||||
ByteBufferInputStream dataInputStream;
|
||||
|
||||
/**
|
||||
* Dictionary decoder to wrap dictionary ids input stream.
|
||||
*/
|
||||
private RunLengthDecoder dictionaryIdsDecoder;
|
||||
|
||||
public AbstractColumnReader(
|
||||
ColumnDescriptor descriptor,
|
||||
PageReader pageReader) throws IOException {
|
||||
this.descriptor = descriptor;
|
||||
this.pageReader = pageReader;
|
||||
this.maxDefLevel = descriptor.getMaxDefinitionLevel();
|
||||
|
||||
DictionaryPage dictionaryPage = pageReader.readDictionaryPage();
|
||||
if (dictionaryPage != null) {
|
||||
try {
|
||||
this.dictionary = dictionaryPage.getEncoding().initDictionary(descriptor, dictionaryPage);
|
||||
this.isCurrentPageDictionaryEncoded = true;
|
||||
} catch (IOException e) {
|
||||
throw new IOException("could not decode the dictionary for " + descriptor, e);
|
||||
}
|
||||
} else {
|
||||
this.dictionary = null;
|
||||
this.isCurrentPageDictionaryEncoded = false;
|
||||
}
|
||||
/*
|
||||
* Total number of values in this column (in this row group).
|
||||
*/
|
||||
long totalValueCount = pageReader.getTotalValueCount();
|
||||
if (totalValueCount == 0) {
|
||||
throw new IOException("totalValueCount == 0");
|
||||
}
|
||||
}
|
||||
|
||||
protected void checkTypeName(PrimitiveType.PrimitiveTypeName expectedName) {
|
||||
PrimitiveType.PrimitiveTypeName actualName = descriptor.getPrimitiveType().getPrimitiveTypeName();
|
||||
Preconditions.checkArgument(
|
||||
actualName == expectedName,
|
||||
"Expected type name: %s, actual type name: %s",
|
||||
expectedName,
|
||||
actualName);
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads `total` values from this columnReader into column.
|
||||
*/
|
||||
@Override
|
||||
public final void readToVector(int readNumber, V vector) throws IOException {
|
||||
int rowId = 0;
|
||||
WritableIntVector dictionaryIds = null;
|
||||
if (dictionary != null) {
|
||||
dictionaryIds = vector.reserveDictionaryIds(readNumber);
|
||||
}
|
||||
while (readNumber > 0) {
|
||||
// Compute the number of values we want to read in this page.
|
||||
int leftInPage = (int) (endOfPageValueCount - valuesRead);
|
||||
if (leftInPage == 0) {
|
||||
DataPage page = pageReader.readPage();
|
||||
if (page instanceof DataPageV1) {
|
||||
readPageV1((DataPageV1) page);
|
||||
} else if (page instanceof DataPageV2) {
|
||||
readPageV2((DataPageV2) page);
|
||||
} else {
|
||||
throw new RuntimeException("Unsupported page type: " + page.getClass());
|
||||
}
|
||||
leftInPage = (int) (endOfPageValueCount - valuesRead);
|
||||
}
|
||||
int num = Math.min(readNumber, leftInPage);
|
||||
if (isCurrentPageDictionaryEncoded) {
|
||||
// Read and decode dictionary ids.
|
||||
runLenDecoder.readDictionaryIds(
|
||||
num, dictionaryIds, vector, rowId, maxDefLevel, this.dictionaryIdsDecoder);
|
||||
|
||||
if (vector.hasDictionary() || (rowId == 0 && supportLazyDecode())) {
|
||||
// Column vector supports lazy decoding of dictionary values so just set the dictionary.
|
||||
// We can't do this if rowId != 0 AND the column doesn't have a dictionary (i.e. some
|
||||
// non-dictionary encoded values have already been added).
|
||||
vector.setDictionary(new ParquetDictionary(dictionary));
|
||||
} else {
|
||||
readBatchFromDictionaryIds(rowId, num, vector, dictionaryIds);
|
||||
}
|
||||
} else {
|
||||
if (vector.hasDictionary() && rowId != 0) {
|
||||
// This batch already has dictionary encoded values but this new page is not. The batch
|
||||
// does not support a mix of dictionary and not so we will decode the dictionary.
|
||||
readBatchFromDictionaryIds(0, rowId, vector, vector.getDictionaryIds());
|
||||
}
|
||||
vector.setDictionary(null);
|
||||
readBatch(rowId, num, vector);
|
||||
}
|
||||
|
||||
valuesRead += num;
|
||||
rowId += num;
|
||||
readNumber -= num;
|
||||
}
|
||||
}
|
||||
|
||||
private void readPageV1(DataPageV1 page) throws IOException {
|
||||
this.pageValueCount = page.getValueCount();
|
||||
ValuesReader rlReader = page.getRlEncoding().getValuesReader(descriptor, REPETITION_LEVEL);
|
||||
|
||||
// Initialize the decoders.
|
||||
if (page.getDlEncoding() != Encoding.RLE && descriptor.getMaxDefinitionLevel() != 0) {
|
||||
throw new UnsupportedOperationException("Unsupported encoding: " + page.getDlEncoding());
|
||||
}
|
||||
int bitWidth = BytesUtils.getWidthFromMaxInt(descriptor.getMaxDefinitionLevel());
|
||||
this.runLenDecoder = new RunLengthDecoder(bitWidth);
|
||||
try {
|
||||
BytesInput bytes = page.getBytes();
|
||||
ByteBufferInputStream in = bytes.toInputStream();
|
||||
rlReader.initFromPage(pageValueCount, in);
|
||||
this.runLenDecoder.initFromStream(pageValueCount, in);
|
||||
prepareNewPage(page.getValueEncoding(), in);
|
||||
} catch (IOException e) {
|
||||
throw new IOException("could not read page " + page + " in col " + descriptor, e);
|
||||
}
|
||||
}
|
||||
|
||||
private void readPageV2(DataPageV2 page) throws IOException {
|
||||
this.pageValueCount = page.getValueCount();
|
||||
|
||||
int bitWidth = BytesUtils.getWidthFromMaxInt(descriptor.getMaxDefinitionLevel());
|
||||
// do not read the length from the stream. v2 pages handle dividing the page bytes.
|
||||
this.runLenDecoder = new RunLengthDecoder(bitWidth, false);
|
||||
this.runLenDecoder.initFromStream(
|
||||
this.pageValueCount, page.getDefinitionLevels().toInputStream());
|
||||
try {
|
||||
prepareNewPage(page.getDataEncoding(), page.getData().toInputStream());
|
||||
} catch (IOException e) {
|
||||
throw new IOException("could not read page " + page + " in col " + descriptor, e);
|
||||
}
|
||||
}
|
||||
|
||||
private void prepareNewPage(
|
||||
Encoding dataEncoding,
|
||||
ByteBufferInputStream in) throws IOException {
|
||||
this.endOfPageValueCount = valuesRead + pageValueCount;
|
||||
if (dataEncoding.usesDictionary()) {
|
||||
if (dictionary == null) {
|
||||
throw new IOException("Could not read page in col "
|
||||
+ descriptor
|
||||
+ " as the dictionary was missing for encoding "
|
||||
+ dataEncoding);
|
||||
}
|
||||
@SuppressWarnings("deprecation")
|
||||
Encoding plainDict = Encoding.PLAIN_DICTIONARY; // var to allow warning suppression
|
||||
if (dataEncoding != plainDict && dataEncoding != Encoding.RLE_DICTIONARY) {
|
||||
throw new UnsupportedOperationException("Unsupported encoding: " + dataEncoding);
|
||||
}
|
||||
this.dataInputStream = null;
|
||||
this.dictionaryIdsDecoder = new RunLengthDecoder();
|
||||
try {
|
||||
this.dictionaryIdsDecoder.initFromStream(pageValueCount, in);
|
||||
} catch (IOException e) {
|
||||
throw new IOException("could not read dictionary in col " + descriptor, e);
|
||||
}
|
||||
this.isCurrentPageDictionaryEncoded = true;
|
||||
} else {
|
||||
if (dataEncoding != Encoding.PLAIN) {
|
||||
throw new UnsupportedOperationException("Unsupported encoding: " + dataEncoding);
|
||||
}
|
||||
this.dictionaryIdsDecoder = null;
|
||||
LOG.debug("init from page at offset {} for length {}", in.position(), in.available());
|
||||
this.dataInputStream = in.remainingStream();
|
||||
this.isCurrentPageDictionaryEncoded = false;
|
||||
}
|
||||
|
||||
afterReadPage();
|
||||
}
|
||||
|
||||
final ByteBuffer readDataBuffer(int length) {
|
||||
try {
|
||||
return dataInputStream.slice(length).order(ByteOrder.LITTLE_ENDIAN);
|
||||
} catch (IOException e) {
|
||||
throw new ParquetDecodingException("Failed to read " + length + " bytes", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* After read a page, we may need some initialization.
|
||||
*/
|
||||
protected void afterReadPage() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Support lazy dictionary ids decode. See more in {@link ParquetDictionary}.
|
||||
* If return false, we will decode all the data first.
|
||||
*/
|
||||
protected boolean supportLazyDecode() {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Read batch from {@link #runLenDecoder} and {@link #dataInputStream}.
|
||||
*/
|
||||
protected abstract void readBatch(int rowId, int num, V column);
|
||||
|
||||
/**
|
||||
* Decode dictionary ids to data.
|
||||
* From {@link #runLenDecoder} and {@link #dictionaryIdsDecoder}.
|
||||
*/
|
||||
protected abstract void readBatchFromDictionaryIds(
|
||||
int rowId,
|
||||
int num,
|
||||
V column,
|
||||
WritableIntVector dictionaryIds);
|
||||
}
|
||||
|
||||
@@ -0,0 +1,473 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.format.cow.vector.reader;
|
||||
|
||||
import org.apache.hudi.table.format.cow.vector.HeapArrayVector;
|
||||
import org.apache.hudi.table.format.cow.vector.ParquetDecimalVector;
|
||||
|
||||
import org.apache.flink.formats.parquet.vector.reader.ColumnReader;
|
||||
import org.apache.flink.table.data.TimestampData;
|
||||
import org.apache.flink.table.data.vector.VectorizedColumnBatch;
|
||||
import org.apache.flink.table.data.vector.heap.HeapBooleanVector;
|
||||
import org.apache.flink.table.data.vector.heap.HeapByteVector;
|
||||
import org.apache.flink.table.data.vector.heap.HeapBytesVector;
|
||||
import org.apache.flink.table.data.vector.heap.HeapDoubleVector;
|
||||
import org.apache.flink.table.data.vector.heap.HeapFloatVector;
|
||||
import org.apache.flink.table.data.vector.heap.HeapIntVector;
|
||||
import org.apache.flink.table.data.vector.heap.HeapLongVector;
|
||||
import org.apache.flink.table.data.vector.heap.HeapShortVector;
|
||||
import org.apache.flink.table.data.vector.heap.HeapTimestampVector;
|
||||
import org.apache.flink.table.data.vector.writable.WritableColumnVector;
|
||||
import org.apache.flink.table.types.logical.ArrayType;
|
||||
import org.apache.flink.table.types.logical.LogicalType;
|
||||
import org.apache.parquet.column.ColumnDescriptor;
|
||||
import org.apache.parquet.column.page.PageReader;
|
||||
import org.apache.parquet.schema.PrimitiveType;
|
||||
import org.apache.parquet.schema.Type;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Array {@link ColumnReader}.
|
||||
*/
|
||||
public class ArrayColumnReader extends BaseVectorizedColumnReader {
|
||||
|
||||
// The value read in last time
|
||||
private Object lastValue;
|
||||
|
||||
// flag to indicate if there is no data in parquet data page
|
||||
private boolean eof = false;
|
||||
|
||||
// flag to indicate if it's the first time to read parquet data page with this instance
|
||||
boolean isFirstRow = true;
|
||||
|
||||
public ArrayColumnReader(
|
||||
ColumnDescriptor descriptor,
|
||||
PageReader pageReader,
|
||||
boolean isUtcTimestamp,
|
||||
Type type,
|
||||
LogicalType logicalType)
|
||||
throws IOException {
|
||||
super(descriptor, pageReader, isUtcTimestamp, type, logicalType);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readToVector(int readNumber, WritableColumnVector vector) throws IOException {
|
||||
HeapArrayVector lcv = (HeapArrayVector) vector;
|
||||
// before readBatch, initial the size of offsets & lengths as the default value,
|
||||
// the actual size will be assigned in setChildrenInfo() after reading complete.
|
||||
lcv.offsets = new long[VectorizedColumnBatch.DEFAULT_SIZE];
|
||||
lcv.lengths = new long[VectorizedColumnBatch.DEFAULT_SIZE];
|
||||
// Because the length of ListColumnVector.child can't be known now,
|
||||
// the valueList will save all data for ListColumnVector temporary.
|
||||
List<Object> valueList = new ArrayList<>();
|
||||
|
||||
LogicalType category = ((ArrayType) logicalType).getElementType();
|
||||
|
||||
// read the first row in parquet data page, this will be only happened once for this
|
||||
// instance
|
||||
if (isFirstRow) {
|
||||
if (!fetchNextValue(category)) {
|
||||
return;
|
||||
}
|
||||
isFirstRow = false;
|
||||
}
|
||||
|
||||
int index = collectDataFromParquetPage(readNumber, lcv, valueList, category);
|
||||
|
||||
// Convert valueList to array for the ListColumnVector.child
|
||||
fillColumnVector(category, lcv, valueList, index);
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads a single value from parquet page, puts it into lastValue. Returns a boolean indicating
|
||||
* if there is more values to read (true).
|
||||
*
|
||||
* @param category
|
||||
* @return boolean
|
||||
* @throws IOException
|
||||
*/
|
||||
private boolean fetchNextValue(LogicalType category) throws IOException {
|
||||
int left = readPageIfNeed();
|
||||
if (left > 0) {
|
||||
// get the values of repetition and definitionLevel
|
||||
readRepetitionAndDefinitionLevels();
|
||||
// read the data if it isn't null
|
||||
if (definitionLevel == maxDefLevel) {
|
||||
if (isCurrentPageDictionaryEncoded) {
|
||||
lastValue = dataColumn.readValueDictionaryId();
|
||||
} else {
|
||||
lastValue = readPrimitiveTypedRow(category);
|
||||
}
|
||||
} else {
|
||||
lastValue = null;
|
||||
}
|
||||
return true;
|
||||
} else {
|
||||
eof = true;
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
private int readPageIfNeed() throws IOException {
|
||||
// Compute the number of values we want to read in this page.
|
||||
int leftInPage = (int) (endOfPageValueCount - valuesRead);
|
||||
if (leftInPage == 0) {
|
||||
// no data left in current page, load data from new page
|
||||
readPage();
|
||||
leftInPage = (int) (endOfPageValueCount - valuesRead);
|
||||
}
|
||||
return leftInPage;
|
||||
}
|
||||
|
||||
// Need to be in consistent with that VectorizedPrimitiveColumnReader#readBatchHelper
|
||||
// TODO Reduce the duplicated code
|
||||
private Object readPrimitiveTypedRow(LogicalType category) {
|
||||
switch (category.getTypeRoot()) {
|
||||
case CHAR:
|
||||
case VARCHAR:
|
||||
case BINARY:
|
||||
case VARBINARY:
|
||||
return dataColumn.readString();
|
||||
case BOOLEAN:
|
||||
return dataColumn.readBoolean();
|
||||
case TIME_WITHOUT_TIME_ZONE:
|
||||
case DATE:
|
||||
case INTEGER:
|
||||
return dataColumn.readInteger();
|
||||
case TINYINT:
|
||||
return dataColumn.readTinyInt();
|
||||
case SMALLINT:
|
||||
return dataColumn.readSmallInt();
|
||||
case BIGINT:
|
||||
return dataColumn.readLong();
|
||||
case FLOAT:
|
||||
return dataColumn.readFloat();
|
||||
case DOUBLE:
|
||||
return dataColumn.readDouble();
|
||||
case DECIMAL:
|
||||
switch (descriptor.getPrimitiveType().getPrimitiveTypeName()) {
|
||||
case INT32:
|
||||
return dataColumn.readInteger();
|
||||
case INT64:
|
||||
return dataColumn.readLong();
|
||||
case BINARY:
|
||||
case FIXED_LEN_BYTE_ARRAY:
|
||||
return dataColumn.readString();
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
case TIMESTAMP_WITHOUT_TIME_ZONE:
|
||||
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
|
||||
return dataColumn.readTimestamp();
|
||||
default:
|
||||
throw new RuntimeException("Unsupported type in the list: " + type);
|
||||
}
|
||||
}
|
||||
|
||||
private Object dictionaryDecodeValue(LogicalType category, Integer dictionaryValue) {
|
||||
if (dictionaryValue == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
switch (category.getTypeRoot()) {
|
||||
case CHAR:
|
||||
case VARCHAR:
|
||||
case BINARY:
|
||||
case VARBINARY:
|
||||
return dictionary.readString(dictionaryValue);
|
||||
case DATE:
|
||||
case TIME_WITHOUT_TIME_ZONE:
|
||||
case INTEGER:
|
||||
return dictionary.readInteger(dictionaryValue);
|
||||
case BOOLEAN:
|
||||
return dictionary.readBoolean(dictionaryValue) ? 1 : 0;
|
||||
case DOUBLE:
|
||||
return dictionary.readDouble(dictionaryValue);
|
||||
case FLOAT:
|
||||
return dictionary.readFloat(dictionaryValue);
|
||||
case TINYINT:
|
||||
return dictionary.readTinyInt(dictionaryValue);
|
||||
case SMALLINT:
|
||||
return dictionary.readSmallInt(dictionaryValue);
|
||||
case BIGINT:
|
||||
return dictionary.readLong(dictionaryValue);
|
||||
case DECIMAL:
|
||||
switch (descriptor.getPrimitiveType().getPrimitiveTypeName()) {
|
||||
case INT32:
|
||||
return dictionary.readInteger(dictionaryValue);
|
||||
case INT64:
|
||||
return dictionary.readLong(dictionaryValue);
|
||||
case FIXED_LEN_BYTE_ARRAY:
|
||||
case BINARY:
|
||||
return dictionary.readString(dictionaryValue);
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
case TIMESTAMP_WITHOUT_TIME_ZONE:
|
||||
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
|
||||
return dictionary.readTimestamp(dictionaryValue);
|
||||
default:
|
||||
throw new RuntimeException("Unsupported type in the list: " + type);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Collects data from a parquet page and returns the final row index where it stopped. The
|
||||
* returned index can be equal to or less than total.
|
||||
*
|
||||
* @param total maximum number of rows to collect
|
||||
* @param lcv column vector to do initial setup in data collection time
|
||||
* @param valueList collection of values that will be fed into the vector later
|
||||
* @param category
|
||||
* @return int
|
||||
* @throws IOException
|
||||
*/
|
||||
private int collectDataFromParquetPage(
|
||||
int total, HeapArrayVector lcv, List<Object> valueList, LogicalType category)
|
||||
throws IOException {
|
||||
int index = 0;
|
||||
/*
|
||||
* Here is a nested loop for collecting all values from a parquet page.
|
||||
* A column of array type can be considered as a list of lists, so the two loops are as below:
|
||||
* 1. The outer loop iterates on rows (index is a row index, so points to a row in the batch), e.g.:
|
||||
* [0, 2, 3] <- index: 0
|
||||
* [NULL, 3, 4] <- index: 1
|
||||
*
|
||||
* 2. The inner loop iterates on values within a row (sets all data from parquet data page
|
||||
* for an element in ListColumnVector), so fetchNextValue returns values one-by-one:
|
||||
* 0, 2, 3, NULL, 3, 4
|
||||
*
|
||||
* As described below, the repetition level (repetitionLevel != 0)
|
||||
* can be used to decide when we'll start to read values for the next list.
|
||||
*/
|
||||
while (!eof && index < total) {
|
||||
// add element to ListColumnVector one by one
|
||||
lcv.offsets[index] = valueList.size();
|
||||
/*
|
||||
* Let's collect all values for a single list.
|
||||
* Repetition level = 0 means that a new list started there in the parquet page,
|
||||
* in that case, let's exit from the loop, and start to collect value for a new list.
|
||||
*/
|
||||
do {
|
||||
/*
|
||||
* Definition level = 0 when a NULL value was returned instead of a list
|
||||
* (this is not the same as a NULL value in of a list).
|
||||
*/
|
||||
if (definitionLevel == 0) {
|
||||
lcv.setNullAt(index);
|
||||
}
|
||||
valueList.add(
|
||||
isCurrentPageDictionaryEncoded
|
||||
? dictionaryDecodeValue(category, (Integer) lastValue)
|
||||
: lastValue);
|
||||
} while (fetchNextValue(category) && (repetitionLevel != 0));
|
||||
|
||||
lcv.lengths[index] = valueList.size() - lcv.offsets[index];
|
||||
index++;
|
||||
}
|
||||
return index;
|
||||
}
|
||||
|
||||
/**
|
||||
* The lengths & offsets will be initialized as default size (1024), it should be set to the
|
||||
* actual size according to the element number.
|
||||
*/
|
||||
private void setChildrenInfo(HeapArrayVector lcv, int itemNum, int elementNum) {
|
||||
lcv.setSize(itemNum);
|
||||
long[] lcvLength = new long[elementNum];
|
||||
long[] lcvOffset = new long[elementNum];
|
||||
System.arraycopy(lcv.lengths, 0, lcvLength, 0, elementNum);
|
||||
System.arraycopy(lcv.offsets, 0, lcvOffset, 0, elementNum);
|
||||
lcv.lengths = lcvLength;
|
||||
lcv.offsets = lcvOffset;
|
||||
}
|
||||
|
||||
private void fillColumnVector(
|
||||
LogicalType category, HeapArrayVector lcv, List valueList, int elementNum) {
|
||||
int total = valueList.size();
|
||||
setChildrenInfo(lcv, total, elementNum);
|
||||
switch (category.getTypeRoot()) {
|
||||
case CHAR:
|
||||
case VARCHAR:
|
||||
case BINARY:
|
||||
case VARBINARY:
|
||||
lcv.child = new HeapBytesVector(total);
|
||||
((HeapBytesVector) lcv.child).reset();
|
||||
for (int i = 0; i < valueList.size(); i++) {
|
||||
byte[] src = ((List<byte[]>) valueList).get(i);
|
||||
if (src == null) {
|
||||
((HeapBytesVector) lcv.child).setNullAt(i);
|
||||
} else {
|
||||
((HeapBytesVector) lcv.child).appendBytes(i, src, 0, src.length);
|
||||
}
|
||||
}
|
||||
break;
|
||||
case BOOLEAN:
|
||||
lcv.child = new HeapBooleanVector(total);
|
||||
((HeapBooleanVector) lcv.child).reset();
|
||||
for (int i = 0; i < valueList.size(); i++) {
|
||||
if (valueList.get(i) == null) {
|
||||
((HeapBooleanVector) lcv.child).setNullAt(i);
|
||||
} else {
|
||||
((HeapBooleanVector) lcv.child).vector[i] =
|
||||
((List<Boolean>) valueList).get(i);
|
||||
}
|
||||
}
|
||||
break;
|
||||
case TINYINT:
|
||||
lcv.child = new HeapByteVector(total);
|
||||
((HeapByteVector) lcv.child).reset();
|
||||
for (int i = 0; i < valueList.size(); i++) {
|
||||
if (valueList.get(i) == null) {
|
||||
((HeapByteVector) lcv.child).setNullAt(i);
|
||||
} else {
|
||||
((HeapByteVector) lcv.child).vector[i] =
|
||||
(byte) ((List<Integer>) valueList).get(i).intValue();
|
||||
}
|
||||
}
|
||||
break;
|
||||
case SMALLINT:
|
||||
lcv.child = new HeapShortVector(total);
|
||||
((HeapShortVector) lcv.child).reset();
|
||||
for (int i = 0; i < valueList.size(); i++) {
|
||||
if (valueList.get(i) == null) {
|
||||
((HeapShortVector) lcv.child).setNullAt(i);
|
||||
} else {
|
||||
((HeapShortVector) lcv.child).vector[i] =
|
||||
(short) ((List<Integer>) valueList).get(i).intValue();
|
||||
}
|
||||
}
|
||||
break;
|
||||
case INTEGER:
|
||||
case DATE:
|
||||
case TIME_WITHOUT_TIME_ZONE:
|
||||
lcv.child = new HeapIntVector(total);
|
||||
((HeapIntVector) lcv.child).reset();
|
||||
for (int i = 0; i < valueList.size(); i++) {
|
||||
if (valueList.get(i) == null) {
|
||||
((HeapIntVector) lcv.child).setNullAt(i);
|
||||
} else {
|
||||
((HeapIntVector) lcv.child).vector[i] = ((List<Integer>) valueList).get(i);
|
||||
}
|
||||
}
|
||||
break;
|
||||
case FLOAT:
|
||||
lcv.child = new HeapFloatVector(total);
|
||||
((HeapFloatVector) lcv.child).reset();
|
||||
for (int i = 0; i < valueList.size(); i++) {
|
||||
if (valueList.get(i) == null) {
|
||||
((HeapFloatVector) lcv.child).setNullAt(i);
|
||||
} else {
|
||||
((HeapFloatVector) lcv.child).vector[i] = ((List<Float>) valueList).get(i);
|
||||
}
|
||||
}
|
||||
break;
|
||||
case BIGINT:
|
||||
lcv.child = new HeapLongVector(total);
|
||||
((HeapLongVector) lcv.child).reset();
|
||||
for (int i = 0; i < valueList.size(); i++) {
|
||||
if (valueList.get(i) == null) {
|
||||
((HeapLongVector) lcv.child).setNullAt(i);
|
||||
} else {
|
||||
((HeapLongVector) lcv.child).vector[i] = ((List<Long>) valueList).get(i);
|
||||
}
|
||||
}
|
||||
break;
|
||||
case DOUBLE:
|
||||
lcv.child = new HeapDoubleVector(total);
|
||||
((HeapDoubleVector) lcv.child).reset();
|
||||
for (int i = 0; i < valueList.size(); i++) {
|
||||
if (valueList.get(i) == null) {
|
||||
((HeapDoubleVector) lcv.child).setNullAt(i);
|
||||
} else {
|
||||
((HeapDoubleVector) lcv.child).vector[i] =
|
||||
((List<Double>) valueList).get(i);
|
||||
}
|
||||
}
|
||||
break;
|
||||
case TIMESTAMP_WITHOUT_TIME_ZONE:
|
||||
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
|
||||
lcv.child = new HeapTimestampVector(total);
|
||||
((HeapTimestampVector) lcv.child).reset();
|
||||
for (int i = 0; i < valueList.size(); i++) {
|
||||
if (valueList.get(i) == null) {
|
||||
((HeapTimestampVector) lcv.child).setNullAt(i);
|
||||
} else {
|
||||
((HeapTimestampVector) lcv.child)
|
||||
.setTimestamp(i, ((List<TimestampData>) valueList).get(i));
|
||||
}
|
||||
}
|
||||
break;
|
||||
case DECIMAL:
|
||||
PrimitiveType.PrimitiveTypeName primitiveTypeName =
|
||||
descriptor.getPrimitiveType().getPrimitiveTypeName();
|
||||
switch (primitiveTypeName) {
|
||||
case INT32:
|
||||
lcv.child = new ParquetDecimalVector(new HeapIntVector(total));
|
||||
((HeapIntVector) ((ParquetDecimalVector) lcv.child).vector).reset();
|
||||
for (int i = 0; i < valueList.size(); i++) {
|
||||
if (valueList.get(i) == null) {
|
||||
((HeapIntVector) ((ParquetDecimalVector) lcv.child).vector)
|
||||
.setNullAt(i);
|
||||
} else {
|
||||
((HeapIntVector) ((ParquetDecimalVector) lcv.child).vector)
|
||||
.vector[i] =
|
||||
((List<Integer>) valueList).get(i);
|
||||
}
|
||||
}
|
||||
break;
|
||||
case INT64:
|
||||
lcv.child = new ParquetDecimalVector(new HeapLongVector(total));
|
||||
((HeapLongVector) ((ParquetDecimalVector) lcv.child).vector).reset();
|
||||
for (int i = 0; i < valueList.size(); i++) {
|
||||
if (valueList.get(i) == null) {
|
||||
((HeapLongVector) ((ParquetDecimalVector) lcv.child).vector)
|
||||
.setNullAt(i);
|
||||
} else {
|
||||
((HeapLongVector) ((ParquetDecimalVector) lcv.child).vector)
|
||||
.vector[i] =
|
||||
((List<Long>) valueList).get(i);
|
||||
}
|
||||
}
|
||||
break;
|
||||
default:
|
||||
lcv.child = new ParquetDecimalVector(new HeapBytesVector(total));
|
||||
((HeapBytesVector) ((ParquetDecimalVector) lcv.child).vector).reset();
|
||||
for (int i = 0; i < valueList.size(); i++) {
|
||||
byte[] src = ((List<byte[]>) valueList).get(i);
|
||||
if (valueList.get(i) == null) {
|
||||
((HeapBytesVector) ((ParquetDecimalVector) lcv.child).vector)
|
||||
.setNullAt(i);
|
||||
} else {
|
||||
((HeapBytesVector) ((ParquetDecimalVector) lcv.child).vector)
|
||||
.appendBytes(i, src, 0, src.length);
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
break;
|
||||
default:
|
||||
throw new RuntimeException("Unsupported type in the list: " + type);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,313 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.format.cow.vector.reader;
|
||||
|
||||
import org.apache.flink.formats.parquet.vector.reader.ColumnReader;
|
||||
import org.apache.flink.table.data.vector.writable.WritableColumnVector;
|
||||
import org.apache.flink.table.types.logical.LogicalType;
|
||||
import org.apache.parquet.bytes.ByteBufferInputStream;
|
||||
import org.apache.parquet.bytes.BytesInput;
|
||||
import org.apache.parquet.bytes.BytesUtils;
|
||||
import org.apache.parquet.column.ColumnDescriptor;
|
||||
import org.apache.parquet.column.Encoding;
|
||||
import org.apache.parquet.column.page.DataPage;
|
||||
import org.apache.parquet.column.page.DataPageV1;
|
||||
import org.apache.parquet.column.page.DataPageV2;
|
||||
import org.apache.parquet.column.page.DictionaryPage;
|
||||
import org.apache.parquet.column.page.PageReader;
|
||||
import org.apache.parquet.column.values.ValuesReader;
|
||||
import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder;
|
||||
import org.apache.parquet.io.ParquetDecodingException;
|
||||
import org.apache.parquet.schema.Type;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.apache.parquet.column.ValuesType.DEFINITION_LEVEL;
|
||||
import static org.apache.parquet.column.ValuesType.REPETITION_LEVEL;
|
||||
import static org.apache.parquet.column.ValuesType.VALUES;
|
||||
|
||||
/**
|
||||
* Abstract {@link ColumnReader}. part of the code is referred from Apache Hive and Apache Parquet.
|
||||
*/
|
||||
public abstract class BaseVectorizedColumnReader implements ColumnReader<WritableColumnVector> {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(BaseVectorizedColumnReader.class);
|
||||
|
||||
protected boolean isUtcTimestamp;
|
||||
|
||||
/**
|
||||
* Total number of values read.
|
||||
*/
|
||||
protected long valuesRead;
|
||||
|
||||
/**
|
||||
* value that indicates the end of the current page. That is, if valuesRead ==
|
||||
* endOfPageValueCount, we are at the end of the page.
|
||||
*/
|
||||
protected long endOfPageValueCount;
|
||||
|
||||
/**
|
||||
* The dictionary, if this column has dictionary encoding.
|
||||
*/
|
||||
protected final ParquetDataColumnReader dictionary;
|
||||
|
||||
/**
|
||||
* If true, the current page is dictionary encoded.
|
||||
*/
|
||||
protected boolean isCurrentPageDictionaryEncoded;
|
||||
|
||||
/**
|
||||
* Maximum definition level for this column.
|
||||
*/
|
||||
protected final int maxDefLevel;
|
||||
|
||||
protected int definitionLevel;
|
||||
protected int repetitionLevel;
|
||||
|
||||
/**
|
||||
* Repetition/Definition/Value readers.
|
||||
*/
|
||||
protected IntIterator repetitionLevelColumn;
|
||||
|
||||
protected IntIterator definitionLevelColumn;
|
||||
protected ParquetDataColumnReader dataColumn;
|
||||
|
||||
/**
|
||||
* Total values in the current page.
|
||||
*/
|
||||
protected int pageValueCount;
|
||||
|
||||
protected final PageReader pageReader;
|
||||
protected final ColumnDescriptor descriptor;
|
||||
protected final Type type;
|
||||
protected final LogicalType logicalType;
|
||||
|
||||
public BaseVectorizedColumnReader(
|
||||
ColumnDescriptor descriptor,
|
||||
PageReader pageReader,
|
||||
boolean isUtcTimestamp,
|
||||
Type parquetType,
|
||||
LogicalType logicalType)
|
||||
throws IOException {
|
||||
this.descriptor = descriptor;
|
||||
this.type = parquetType;
|
||||
this.pageReader = pageReader;
|
||||
this.maxDefLevel = descriptor.getMaxDefinitionLevel();
|
||||
this.isUtcTimestamp = isUtcTimestamp;
|
||||
this.logicalType = logicalType;
|
||||
|
||||
DictionaryPage dictionaryPage = pageReader.readDictionaryPage();
|
||||
if (dictionaryPage != null) {
|
||||
try {
|
||||
this.dictionary =
|
||||
ParquetDataColumnReaderFactory.getDataColumnReaderByTypeOnDictionary(
|
||||
parquetType.asPrimitiveType(),
|
||||
dictionaryPage
|
||||
.getEncoding()
|
||||
.initDictionary(descriptor, dictionaryPage),
|
||||
isUtcTimestamp);
|
||||
this.isCurrentPageDictionaryEncoded = true;
|
||||
} catch (IOException e) {
|
||||
throw new IOException("could not decode the dictionary for " + descriptor, e);
|
||||
}
|
||||
} else {
|
||||
this.dictionary = null;
|
||||
this.isCurrentPageDictionaryEncoded = false;
|
||||
}
|
||||
}
|
||||
|
||||
protected void readRepetitionAndDefinitionLevels() {
|
||||
repetitionLevel = repetitionLevelColumn.nextInt();
|
||||
definitionLevel = definitionLevelColumn.nextInt();
|
||||
valuesRead++;
|
||||
}
|
||||
|
||||
protected void readPage() throws IOException {
|
||||
DataPage page = pageReader.readPage();
|
||||
|
||||
if (page == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
page.accept(
|
||||
new DataPage.Visitor<Void>() {
|
||||
@Override
|
||||
public Void visit(DataPageV1 dataPageV1) {
|
||||
readPageV1(dataPageV1);
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Void visit(DataPageV2 dataPageV2) {
|
||||
readPageV2(dataPageV2);
|
||||
return null;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private void initDataReader(Encoding dataEncoding, ByteBufferInputStream in, int valueCount)
|
||||
throws IOException {
|
||||
this.pageValueCount = valueCount;
|
||||
this.endOfPageValueCount = valuesRead + pageValueCount;
|
||||
if (dataEncoding.usesDictionary()) {
|
||||
this.dataColumn = null;
|
||||
if (dictionary == null) {
|
||||
throw new IOException(
|
||||
"could not read page in col "
|
||||
+ descriptor
|
||||
+ " as the dictionary was missing for encoding "
|
||||
+ dataEncoding);
|
||||
}
|
||||
dataColumn =
|
||||
ParquetDataColumnReaderFactory.getDataColumnReaderByType(
|
||||
type.asPrimitiveType(),
|
||||
dataEncoding.getDictionaryBasedValuesReader(
|
||||
descriptor, VALUES, dictionary.getDictionary()),
|
||||
isUtcTimestamp);
|
||||
this.isCurrentPageDictionaryEncoded = true;
|
||||
} else {
|
||||
dataColumn =
|
||||
ParquetDataColumnReaderFactory.getDataColumnReaderByType(
|
||||
type.asPrimitiveType(),
|
||||
dataEncoding.getValuesReader(descriptor, VALUES),
|
||||
isUtcTimestamp);
|
||||
this.isCurrentPageDictionaryEncoded = false;
|
||||
}
|
||||
|
||||
try {
|
||||
dataColumn.initFromPage(pageValueCount, in);
|
||||
} catch (IOException e) {
|
||||
throw new IOException("could not read page in col " + descriptor, e);
|
||||
}
|
||||
}
|
||||
|
||||
private void readPageV1(DataPageV1 page) {
|
||||
ValuesReader rlReader = page.getRlEncoding().getValuesReader(descriptor, REPETITION_LEVEL);
|
||||
ValuesReader dlReader = page.getDlEncoding().getValuesReader(descriptor, DEFINITION_LEVEL);
|
||||
this.repetitionLevelColumn = new ValuesReaderIntIterator(rlReader);
|
||||
this.definitionLevelColumn = new ValuesReaderIntIterator(dlReader);
|
||||
try {
|
||||
BytesInput bytes = page.getBytes();
|
||||
LOG.debug("page size " + bytes.size() + " bytes and " + pageValueCount + " records");
|
||||
ByteBufferInputStream in = bytes.toInputStream();
|
||||
LOG.debug("reading repetition levels at " + in.position());
|
||||
rlReader.initFromPage(pageValueCount, in);
|
||||
LOG.debug("reading definition levels at " + in.position());
|
||||
dlReader.initFromPage(pageValueCount, in);
|
||||
LOG.debug("reading data at " + in.position());
|
||||
initDataReader(page.getValueEncoding(), in, page.getValueCount());
|
||||
} catch (IOException e) {
|
||||
throw new ParquetDecodingException(
|
||||
"could not read page " + page + " in col " + descriptor, e);
|
||||
}
|
||||
}
|
||||
|
||||
private void readPageV2(DataPageV2 page) {
|
||||
this.pageValueCount = page.getValueCount();
|
||||
this.repetitionLevelColumn =
|
||||
newRLEIterator(descriptor.getMaxRepetitionLevel(), page.getRepetitionLevels());
|
||||
this.definitionLevelColumn =
|
||||
newRLEIterator(descriptor.getMaxDefinitionLevel(), page.getDefinitionLevels());
|
||||
try {
|
||||
LOG.debug(
|
||||
"page data size "
|
||||
+ page.getData().size()
|
||||
+ " bytes and "
|
||||
+ pageValueCount
|
||||
+ " records");
|
||||
initDataReader(
|
||||
page.getDataEncoding(), page.getData().toInputStream(), page.getValueCount());
|
||||
} catch (IOException e) {
|
||||
throw new ParquetDecodingException(
|
||||
"could not read page " + page + " in col " + descriptor, e);
|
||||
}
|
||||
}
|
||||
|
||||
private IntIterator newRLEIterator(int maxLevel, BytesInput bytes) {
|
||||
try {
|
||||
if (maxLevel == 0) {
|
||||
return new NullIntIterator();
|
||||
}
|
||||
return new RLEIntIterator(
|
||||
new RunLengthBitPackingHybridDecoder(
|
||||
BytesUtils.getWidthFromMaxInt(maxLevel),
|
||||
new ByteArrayInputStream(bytes.toByteArray())));
|
||||
} catch (IOException e) {
|
||||
throw new ParquetDecodingException(
|
||||
"could not read levels in page for col " + descriptor, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility classes to abstract over different way to read ints with different encodings.
|
||||
*/
|
||||
abstract static class IntIterator {
|
||||
abstract int nextInt();
|
||||
}
|
||||
|
||||
/**
|
||||
* read ints from {@link ValuesReader}.
|
||||
*/
|
||||
protected static final class ValuesReaderIntIterator extends IntIterator {
|
||||
ValuesReader delegate;
|
||||
|
||||
public ValuesReaderIntIterator(ValuesReader delegate) {
|
||||
this.delegate = delegate;
|
||||
}
|
||||
|
||||
@Override
|
||||
int nextInt() {
|
||||
return delegate.readInteger();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* read ints from {@link RunLengthBitPackingHybridDecoder}.
|
||||
*/
|
||||
protected static final class RLEIntIterator extends IntIterator {
|
||||
RunLengthBitPackingHybridDecoder delegate;
|
||||
|
||||
public RLEIntIterator(RunLengthBitPackingHybridDecoder delegate) {
|
||||
this.delegate = delegate;
|
||||
}
|
||||
|
||||
@Override
|
||||
int nextInt() {
|
||||
try {
|
||||
return delegate.readInt();
|
||||
} catch (IOException e) {
|
||||
throw new ParquetDecodingException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* return zero.
|
||||
*/
|
||||
protected static final class NullIntIterator extends IntIterator {
|
||||
@Override
|
||||
int nextInt() {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,84 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.format.cow.vector.reader;
|
||||
|
||||
import org.apache.flink.table.data.vector.writable.WritableBytesVector;
|
||||
import org.apache.flink.table.data.vector.writable.WritableColumnVector;
|
||||
import org.apache.flink.table.data.vector.writable.WritableIntVector;
|
||||
import org.apache.parquet.column.ColumnDescriptor;
|
||||
import org.apache.parquet.column.page.PageReader;
|
||||
import org.apache.parquet.io.api.Binary;
|
||||
import org.apache.parquet.schema.PrimitiveType;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
* Fixed length bytes {@code ColumnReader}, just for decimal.
|
||||
*
|
||||
* <p>Note: Reference Flink release 1.13.2
|
||||
* {@code org.apache.flink.formats.parquet.vector.reader.FixedLenBytesColumnReader}
|
||||
* to always write as legacy decimal format.
|
||||
*/
|
||||
public class FixedLenBytesColumnReader<V extends WritableColumnVector>
|
||||
extends AbstractColumnReader<V> {
|
||||
|
||||
public FixedLenBytesColumnReader(
|
||||
ColumnDescriptor descriptor, PageReader pageReader, int precision) throws IOException {
|
||||
super(descriptor, pageReader);
|
||||
checkTypeName(PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void readBatch(int rowId, int num, V column) {
|
||||
int bytesLen = descriptor.getPrimitiveType().getTypeLength();
|
||||
WritableBytesVector bytesVector = (WritableBytesVector) column;
|
||||
for (int i = 0; i < num; i++) {
|
||||
if (runLenDecoder.readInteger() == maxDefLevel) {
|
||||
byte[] bytes = readDataBinary(bytesLen).getBytes();
|
||||
bytesVector.appendBytes(rowId + i, bytes, 0, bytes.length);
|
||||
} else {
|
||||
bytesVector.setNullAt(rowId + i);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void readBatchFromDictionaryIds(
|
||||
int rowId, int num, V column, WritableIntVector dictionaryIds) {
|
||||
WritableBytesVector bytesVector = (WritableBytesVector) column;
|
||||
for (int i = rowId; i < rowId + num; ++i) {
|
||||
if (!bytesVector.isNullAt(i)) {
|
||||
byte[] v = dictionary.decodeToBinary(dictionaryIds.getInt(i)).getBytes();
|
||||
bytesVector.appendBytes(i, v, 0, v.length);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private Binary readDataBinary(int len) {
|
||||
ByteBuffer buffer = readDataBuffer(len);
|
||||
if (buffer.hasArray()) {
|
||||
return Binary.fromConstantByteArray(
|
||||
buffer.array(), buffer.arrayOffset() + buffer.position(), len);
|
||||
} else {
|
||||
byte[] bytes = new byte[len];
|
||||
buffer.get(bytes);
|
||||
return Binary.fromConstantByteArray(bytes);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,119 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.format.cow.vector.reader;
|
||||
|
||||
import org.apache.flink.table.data.TimestampData;
|
||||
import org.apache.flink.table.data.vector.writable.WritableIntVector;
|
||||
import org.apache.flink.table.data.vector.writable.WritableTimestampVector;
|
||||
import org.apache.parquet.column.ColumnDescriptor;
|
||||
import org.apache.parquet.column.page.PageReader;
|
||||
import org.apache.parquet.schema.PrimitiveType;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.sql.Timestamp;
|
||||
import java.time.Instant;
|
||||
import java.time.temporal.ChronoUnit;
|
||||
|
||||
/**
|
||||
* Timestamp {@link org.apache.flink.formats.parquet.vector.reader.ColumnReader} that supports INT64 8 bytes,
|
||||
* TIMESTAMP_MILLIS is the deprecated ConvertedType counterpart of a TIMESTAMP logical type
|
||||
* that is UTC normalized and has MILLIS precision.
|
||||
*
|
||||
* <p>See https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#timestamp
|
||||
* TIMESTAMP_MILLIS and TIMESTAMP_MICROS are the deprecated ConvertedType.
|
||||
*/
|
||||
public class Int64TimestampColumnReader extends AbstractColumnReader<WritableTimestampVector> {
|
||||
|
||||
private final boolean utcTimestamp;
|
||||
|
||||
private final ChronoUnit chronoUnit;
|
||||
|
||||
public Int64TimestampColumnReader(
|
||||
boolean utcTimestamp,
|
||||
ColumnDescriptor descriptor,
|
||||
PageReader pageReader,
|
||||
int precision) throws IOException {
|
||||
super(descriptor, pageReader);
|
||||
this.utcTimestamp = utcTimestamp;
|
||||
if (precision <= 3) {
|
||||
this.chronoUnit = ChronoUnit.MILLIS;
|
||||
} else if (precision <= 6) {
|
||||
this.chronoUnit = ChronoUnit.MICROS;
|
||||
} else {
|
||||
throw new IllegalArgumentException(
|
||||
"Avro does not support TIMESTAMP type with precision: "
|
||||
+ precision
|
||||
+ ", it only supports precision less than 6.");
|
||||
}
|
||||
checkTypeName(PrimitiveType.PrimitiveTypeName.INT64);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean supportLazyDecode() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void readBatch(int rowId, int num, WritableTimestampVector column) {
|
||||
for (int i = 0; i < num; i++) {
|
||||
if (runLenDecoder.readInteger() == maxDefLevel) {
|
||||
ByteBuffer buffer = readDataBuffer(8);
|
||||
column.setTimestamp(rowId + i, int64ToTimestamp(utcTimestamp, buffer.getLong(), chronoUnit));
|
||||
} else {
|
||||
column.setNullAt(rowId + i);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void readBatchFromDictionaryIds(
|
||||
int rowId,
|
||||
int num,
|
||||
WritableTimestampVector column,
|
||||
WritableIntVector dictionaryIds) {
|
||||
for (int i = rowId; i < rowId + num; ++i) {
|
||||
if (!column.isNullAt(i)) {
|
||||
column.setTimestamp(i, decodeInt64ToTimestamp(
|
||||
utcTimestamp, dictionary, dictionaryIds.getInt(i), chronoUnit));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static TimestampData decodeInt64ToTimestamp(
|
||||
boolean utcTimestamp,
|
||||
org.apache.parquet.column.Dictionary dictionary,
|
||||
int id,
|
||||
ChronoUnit unit) {
|
||||
long value = dictionary.decodeToLong(id);
|
||||
return int64ToTimestamp(utcTimestamp, value, unit);
|
||||
}
|
||||
|
||||
private static TimestampData int64ToTimestamp(
|
||||
boolean utcTimestamp,
|
||||
long interval,
|
||||
ChronoUnit unit) {
|
||||
final Instant instant = Instant.EPOCH.plus(interval, unit);
|
||||
if (utcTimestamp) {
|
||||
return TimestampData.fromInstant(instant);
|
||||
} else {
|
||||
// this applies the local timezone
|
||||
return TimestampData.fromTimestamp(Timestamp.from(instant));
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,76 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.format.cow.vector.reader;
|
||||
|
||||
import org.apache.hudi.table.format.cow.vector.HeapArrayVector;
|
||||
import org.apache.hudi.table.format.cow.vector.HeapMapColumnVector;
|
||||
|
||||
import org.apache.flink.formats.parquet.vector.reader.ColumnReader;
|
||||
import org.apache.flink.table.data.vector.ColumnVector;
|
||||
import org.apache.flink.table.data.vector.writable.WritableColumnVector;
|
||||
import org.apache.flink.table.types.logical.LogicalType;
|
||||
import org.apache.flink.table.types.logical.MapType;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Map {@link ColumnReader}.
|
||||
*/
|
||||
public class MapColumnReader implements ColumnReader<WritableColumnVector> {
|
||||
|
||||
private final LogicalType logicalType;
|
||||
private final ArrayColumnReader keyReader;
|
||||
private final ArrayColumnReader valueReader;
|
||||
|
||||
public MapColumnReader(
|
||||
ArrayColumnReader keyReader, ArrayColumnReader valueReader, LogicalType logicalType) {
|
||||
this.keyReader = keyReader;
|
||||
this.valueReader = valueReader;
|
||||
this.logicalType = logicalType;
|
||||
}
|
||||
|
||||
public void readBatch(int total, ColumnVector column) throws IOException {
|
||||
HeapMapColumnVector mapColumnVector = (HeapMapColumnVector) column;
|
||||
MapType mapType = (MapType) logicalType;
|
||||
// initialize 2 ListColumnVector for keys and values
|
||||
HeapArrayVector keyArrayColumnVector = new HeapArrayVector(total);
|
||||
HeapArrayVector valueArrayColumnVector = new HeapArrayVector(total);
|
||||
// read the keys and values
|
||||
keyReader.readToVector(total, keyArrayColumnVector);
|
||||
valueReader.readToVector(total, valueArrayColumnVector);
|
||||
|
||||
// set the related attributes according to the keys and values
|
||||
mapColumnVector.setKeys(keyArrayColumnVector.child);
|
||||
mapColumnVector.setValues(valueArrayColumnVector.child);
|
||||
mapColumnVector.setOffsets(keyArrayColumnVector.offsets);
|
||||
mapColumnVector.setLengths(keyArrayColumnVector.lengths);
|
||||
mapColumnVector.setSize(keyArrayColumnVector.getSize());
|
||||
for (int i = 0; i < keyArrayColumnVector.getLen(); i++) {
|
||||
if (keyArrayColumnVector.isNullAt(i)) {
|
||||
mapColumnVector.setNullAt(i);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readToVector(int readNumber, WritableColumnVector vector) throws IOException {
|
||||
readBatch(readNumber, vector);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,385 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.format.cow.vector.reader;
|
||||
|
||||
import org.apache.hudi.table.format.cow.vector.ParquetDecimalVector;
|
||||
|
||||
import org.apache.flink.formats.parquet.vector.reader.ColumnReader;
|
||||
import org.apache.flink.table.data.ColumnarRowData;
|
||||
import org.apache.flink.table.data.vector.ColumnVector;
|
||||
import org.apache.flink.table.data.vector.VectorizedColumnBatch;
|
||||
import org.apache.flink.table.data.vector.writable.WritableColumnVector;
|
||||
import org.apache.flink.table.types.logical.LogicalType;
|
||||
import org.apache.flink.table.types.logical.LogicalTypeRoot;
|
||||
import org.apache.flink.util.FlinkRuntimeException;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.parquet.column.ColumnDescriptor;
|
||||
import org.apache.parquet.column.page.PageReadStore;
|
||||
import org.apache.parquet.filter2.compat.FilterCompat;
|
||||
import org.apache.parquet.hadoop.ParquetFileReader;
|
||||
import org.apache.parquet.hadoop.metadata.BlockMetaData;
|
||||
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
|
||||
import org.apache.parquet.schema.GroupType;
|
||||
import org.apache.parquet.schema.MessageType;
|
||||
import org.apache.parquet.schema.Type;
|
||||
import org.apache.parquet.schema.Types;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
import static org.apache.hudi.table.format.cow.ParquetSplitReaderUtil.createColumnReader;
|
||||
import static org.apache.hudi.table.format.cow.ParquetSplitReaderUtil.createWritableColumnVector;
|
||||
import static org.apache.parquet.filter2.compat.RowGroupFilter.filterRowGroups;
|
||||
import static org.apache.parquet.format.converter.ParquetMetadataConverter.range;
|
||||
import static org.apache.parquet.hadoop.ParquetFileReader.readFooter;
|
||||
import static org.apache.parquet.hadoop.ParquetInputFormat.getFilter;
|
||||
|
||||
/**
|
||||
* This reader is used to read a {@link VectorizedColumnBatch} from input split.
|
||||
*
|
||||
* <p>Note: Reference Flink release 1.11.2
|
||||
* {@code org.apache.flink.formats.parquet.vector.ParquetColumnarRowSplitReader}
|
||||
* because it is package scope.
|
||||
*/
|
||||
public class ParquetColumnarRowSplitReader implements Closeable {
|
||||
|
||||
private final boolean utcTimestamp;
|
||||
|
||||
private final MessageType fileSchema;
|
||||
|
||||
private final LogicalType[] requestedTypes;
|
||||
|
||||
private final MessageType requestedSchema;
|
||||
|
||||
/**
|
||||
* The total number of rows this RecordReader will eventually read. The sum of the rows of all
|
||||
* the row groups.
|
||||
*/
|
||||
private final long totalRowCount;
|
||||
|
||||
private final WritableColumnVector[] writableVectors;
|
||||
|
||||
private final VectorizedColumnBatch columnarBatch;
|
||||
|
||||
private final ColumnarRowData row;
|
||||
|
||||
private final int batchSize;
|
||||
|
||||
private ParquetFileReader reader;
|
||||
|
||||
/**
|
||||
* For each request column, the reader to read this column. This is NULL if this column is
|
||||
* missing from the file, in which case we populate the attribute with NULL.
|
||||
*/
|
||||
private ColumnReader[] columnReaders;
|
||||
|
||||
/**
|
||||
* The number of rows that have been returned.
|
||||
*/
|
||||
private long rowsReturned;
|
||||
|
||||
/**
|
||||
* The number of rows that have been reading, including the current in flight row group.
|
||||
*/
|
||||
private long totalCountLoadedSoFar;
|
||||
|
||||
// the index of the next row to return
|
||||
private int nextRow;
|
||||
|
||||
// the number of rows in the current batch
|
||||
private int rowsInBatch;
|
||||
|
||||
public ParquetColumnarRowSplitReader(
|
||||
boolean utcTimestamp,
|
||||
boolean caseSensitive,
|
||||
Configuration conf,
|
||||
LogicalType[] selectedTypes,
|
||||
String[] selectedFieldNames,
|
||||
ColumnBatchGenerator generator,
|
||||
int batchSize,
|
||||
Path path,
|
||||
long splitStart,
|
||||
long splitLength) throws IOException {
|
||||
this.utcTimestamp = utcTimestamp;
|
||||
this.batchSize = batchSize;
|
||||
// then we need to apply the predicate push down filter
|
||||
ParquetMetadata footer = readFooter(conf, path, range(splitStart, splitStart + splitLength));
|
||||
MessageType fileSchema = footer.getFileMetaData().getSchema();
|
||||
FilterCompat.Filter filter = getFilter(conf);
|
||||
List<BlockMetaData> blocks = filterRowGroups(filter, footer.getBlocks(), fileSchema);
|
||||
|
||||
this.fileSchema = footer.getFileMetaData().getSchema();
|
||||
|
||||
Type[] types = clipParquetSchema(fileSchema, selectedFieldNames, caseSensitive);
|
||||
int[] requestedIndices = IntStream.range(0, types.length).filter(i -> types[i] != null).toArray();
|
||||
Type[] readTypes = Arrays.stream(requestedIndices).mapToObj(i -> types[i]).toArray(Type[]::new);
|
||||
|
||||
this.requestedTypes = Arrays.stream(requestedIndices).mapToObj(i -> selectedTypes[i]).toArray(LogicalType[]::new);
|
||||
this.requestedSchema = Types.buildMessage().addFields(readTypes).named("flink-parquet");
|
||||
this.reader = new ParquetFileReader(
|
||||
conf, footer.getFileMetaData(), path, blocks, requestedSchema.getColumns());
|
||||
|
||||
long totalRowCount = 0;
|
||||
for (BlockMetaData block : blocks) {
|
||||
totalRowCount += block.getRowCount();
|
||||
}
|
||||
this.totalRowCount = totalRowCount;
|
||||
this.nextRow = 0;
|
||||
this.rowsInBatch = 0;
|
||||
this.rowsReturned = 0;
|
||||
|
||||
checkSchema();
|
||||
|
||||
this.writableVectors = createWritableVectors();
|
||||
ColumnVector[] columnVectors = patchedVector(selectedFieldNames.length, createReadableVectors(), requestedIndices);
|
||||
this.columnarBatch = generator.generate(columnVectors);
|
||||
this.row = new ColumnarRowData(columnarBatch);
|
||||
}
|
||||
|
||||
/**
|
||||
* Patches the given vectors with nulls.
|
||||
* The vector position that is not requested (or read from file) is patched as null.
|
||||
*
|
||||
* @param fields The total selected fields number
|
||||
* @param vectors The readable vectors
|
||||
* @param indices The requested indices from the selected fields
|
||||
*/
|
||||
private static ColumnVector[] patchedVector(int fields, ColumnVector[] vectors, int[] indices) {
|
||||
ColumnVector[] patched = new ColumnVector[fields];
|
||||
for (int i = 0; i < indices.length; i++) {
|
||||
patched[indices[i]] = vectors[i];
|
||||
}
|
||||
return patched;
|
||||
}
|
||||
|
||||
/**
|
||||
* Clips `parquetSchema` according to `fieldNames`.
|
||||
*/
|
||||
private static Type[] clipParquetSchema(
|
||||
GroupType parquetSchema, String[] fieldNames, boolean caseSensitive) {
|
||||
Type[] types = new Type[fieldNames.length];
|
||||
if (caseSensitive) {
|
||||
for (int i = 0; i < fieldNames.length; ++i) {
|
||||
String fieldName = fieldNames[i];
|
||||
types[i] = parquetSchema.containsField(fieldName) ? parquetSchema.getType(fieldName) : null;
|
||||
}
|
||||
} else {
|
||||
Map<String, Type> caseInsensitiveFieldMap = new HashMap<>();
|
||||
for (Type type : parquetSchema.getFields()) {
|
||||
caseInsensitiveFieldMap.compute(type.getName().toLowerCase(Locale.ROOT),
|
||||
(key, previousType) -> {
|
||||
if (previousType != null) {
|
||||
throw new FlinkRuntimeException(
|
||||
"Parquet with case insensitive mode should have no duplicate key: " + key);
|
||||
}
|
||||
return type;
|
||||
});
|
||||
}
|
||||
for (int i = 0; i < fieldNames.length; ++i) {
|
||||
Type type = caseInsensitiveFieldMap.get(fieldNames[i].toLowerCase(Locale.ROOT));
|
||||
// TODO clip for array,map,row types.
|
||||
types[i] = type;
|
||||
}
|
||||
}
|
||||
|
||||
return types;
|
||||
}
|
||||
|
||||
private WritableColumnVector[] createWritableVectors() {
|
||||
WritableColumnVector[] columns = new WritableColumnVector[requestedTypes.length];
|
||||
List<Type> types = requestedSchema.getFields();
|
||||
List<ColumnDescriptor> descriptors = requestedSchema.getColumns();
|
||||
for (int i = 0; i < requestedTypes.length; i++) {
|
||||
columns[i] = createWritableColumnVector(
|
||||
batchSize,
|
||||
requestedTypes[i],
|
||||
types.get(i),
|
||||
descriptors);
|
||||
}
|
||||
return columns;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create readable vectors from writable vectors.
|
||||
* Especially for decimal, see {@link org.apache.flink.formats.parquet.vector.ParquetDecimalVector}.
|
||||
*/
|
||||
private ColumnVector[] createReadableVectors() {
|
||||
ColumnVector[] vectors = new ColumnVector[writableVectors.length];
|
||||
for (int i = 0; i < writableVectors.length; i++) {
|
||||
vectors[i] = requestedTypes[i].getTypeRoot() == LogicalTypeRoot.DECIMAL
|
||||
? new ParquetDecimalVector(writableVectors[i])
|
||||
: writableVectors[i];
|
||||
}
|
||||
return vectors;
|
||||
}
|
||||
|
||||
private void checkSchema() throws IOException, UnsupportedOperationException {
|
||||
/*
|
||||
* Check that the requested schema is supported.
|
||||
*/
|
||||
for (int i = 0; i < requestedSchema.getFieldCount(); ++i) {
|
||||
String[] colPath = requestedSchema.getPaths().get(i);
|
||||
if (fileSchema.containsPath(colPath)) {
|
||||
ColumnDescriptor fd = fileSchema.getColumnDescription(colPath);
|
||||
if (!fd.equals(requestedSchema.getColumns().get(i))) {
|
||||
throw new UnsupportedOperationException("Schema evolution not supported.");
|
||||
}
|
||||
} else {
|
||||
if (requestedSchema.getColumns().get(i).getMaxDefinitionLevel() == 0) {
|
||||
// Column is missing in data but the required data is non-nullable. This file is invalid.
|
||||
throw new IOException("Required column is missing in data file. Col: " + Arrays.toString(colPath));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Method used to check if the end of the input is reached.
|
||||
*
|
||||
* @return True if the end is reached, otherwise false.
|
||||
* @throws IOException Thrown, if an I/O error occurred.
|
||||
*/
|
||||
public boolean reachedEnd() throws IOException {
|
||||
return !ensureBatch();
|
||||
}
|
||||
|
||||
public ColumnarRowData nextRecord() {
|
||||
// return the next row
|
||||
row.setRowId(this.nextRow++);
|
||||
return row;
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if there is at least one row left in the batch to return. If no more row are
|
||||
* available, it reads another batch of rows.
|
||||
*
|
||||
* @return Returns true if there is one more row to return, false otherwise.
|
||||
* @throws IOException throw if an exception happens while reading a batch.
|
||||
*/
|
||||
private boolean ensureBatch() throws IOException {
|
||||
if (nextRow >= rowsInBatch) {
|
||||
// No more rows available in the Rows array.
|
||||
nextRow = 0;
|
||||
// Try to read the next batch if rows from the file.
|
||||
return nextBatch();
|
||||
}
|
||||
// there is at least one Row left in the Rows array.
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Advances to the next batch of rows. Returns false if there are no more.
|
||||
*/
|
||||
private boolean nextBatch() throws IOException {
|
||||
for (WritableColumnVector v : writableVectors) {
|
||||
v.reset();
|
||||
}
|
||||
columnarBatch.setNumRows(0);
|
||||
if (rowsReturned >= totalRowCount) {
|
||||
return false;
|
||||
}
|
||||
if (rowsReturned == totalCountLoadedSoFar) {
|
||||
readNextRowGroup();
|
||||
}
|
||||
|
||||
int num = (int) Math.min(batchSize, totalCountLoadedSoFar - rowsReturned);
|
||||
for (int i = 0; i < columnReaders.length; ++i) {
|
||||
//noinspection unchecked
|
||||
columnReaders[i].readToVector(num, writableVectors[i]);
|
||||
}
|
||||
rowsReturned += num;
|
||||
columnarBatch.setNumRows(num);
|
||||
rowsInBatch = num;
|
||||
return true;
|
||||
}
|
||||
|
||||
private void readNextRowGroup() throws IOException {
|
||||
PageReadStore pages = reader.readNextRowGroup();
|
||||
if (pages == null) {
|
||||
throw new IOException("expecting more rows but reached last block. Read "
|
||||
+ rowsReturned + " out of " + totalRowCount);
|
||||
}
|
||||
List<Type> types = requestedSchema.getFields();
|
||||
List<ColumnDescriptor> columns = requestedSchema.getColumns();
|
||||
columnReaders = new ColumnReader[types.size()];
|
||||
for (int i = 0; i < types.size(); ++i) {
|
||||
columnReaders[i] = createColumnReader(
|
||||
utcTimestamp,
|
||||
requestedTypes[i],
|
||||
types.get(i),
|
||||
columns,
|
||||
pages);
|
||||
}
|
||||
totalCountLoadedSoFar += pages.getRowCount();
|
||||
}
|
||||
|
||||
/**
|
||||
* Seek to a particular row number.
|
||||
*/
|
||||
public void seekToRow(long rowCount) throws IOException {
|
||||
if (totalCountLoadedSoFar != 0) {
|
||||
throw new UnsupportedOperationException("Only support seek at first.");
|
||||
}
|
||||
|
||||
List<BlockMetaData> blockMetaData = reader.getRowGroups();
|
||||
|
||||
for (BlockMetaData metaData : blockMetaData) {
|
||||
if (metaData.getRowCount() > rowCount) {
|
||||
break;
|
||||
} else {
|
||||
reader.skipNextRowGroup();
|
||||
rowsReturned += metaData.getRowCount();
|
||||
totalCountLoadedSoFar += metaData.getRowCount();
|
||||
rowsInBatch = (int) metaData.getRowCount();
|
||||
nextRow = (int) metaData.getRowCount();
|
||||
rowCount -= metaData.getRowCount();
|
||||
}
|
||||
}
|
||||
for (int i = 0; i < rowCount; i++) {
|
||||
boolean end = reachedEnd();
|
||||
if (end) {
|
||||
throw new RuntimeException("Seek to many rows.");
|
||||
}
|
||||
nextRecord();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (reader != null) {
|
||||
reader.close();
|
||||
reader = null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Interface to gen {@link VectorizedColumnBatch}.
|
||||
*/
|
||||
public interface ColumnBatchGenerator {
|
||||
VectorizedColumnBatch generate(ColumnVector[] readVectors);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,199 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.format.cow.vector.reader;
|
||||
|
||||
import org.apache.flink.table.data.TimestampData;
|
||||
import org.apache.parquet.bytes.ByteBufferInputStream;
|
||||
import org.apache.parquet.column.Dictionary;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* The interface to wrap the underlying Parquet dictionary and non dictionary encoded page reader.
|
||||
*/
|
||||
public interface ParquetDataColumnReader {
|
||||
|
||||
/**
|
||||
* Initialize the reader by page data.
|
||||
*
|
||||
* @param valueCount value count
|
||||
* @param in page data
|
||||
* @throws IOException
|
||||
*/
|
||||
void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException;
|
||||
|
||||
/**
|
||||
* @return the next Dictionary ID from the page
|
||||
*/
|
||||
int readValueDictionaryId();
|
||||
|
||||
/**
|
||||
* @return the next Long from the page
|
||||
*/
|
||||
long readLong();
|
||||
|
||||
/**
|
||||
* @return the next Integer from the page
|
||||
*/
|
||||
int readInteger();
|
||||
|
||||
/**
|
||||
* @return the next SmallInt from the page
|
||||
*/
|
||||
int readSmallInt();
|
||||
|
||||
/**
|
||||
* @return the next TinyInt from the page
|
||||
*/
|
||||
int readTinyInt();
|
||||
|
||||
/**
|
||||
* @return the next Float from the page
|
||||
*/
|
||||
float readFloat();
|
||||
|
||||
/**
|
||||
* @return the next Boolean from the page
|
||||
*/
|
||||
boolean readBoolean();
|
||||
|
||||
/**
|
||||
* @return the next String from the page
|
||||
*/
|
||||
byte[] readString();
|
||||
|
||||
/**
|
||||
* @return the next Varchar from the page
|
||||
*/
|
||||
byte[] readVarchar();
|
||||
|
||||
/**
|
||||
* @return the next Char from the page
|
||||
*/
|
||||
byte[] readChar();
|
||||
|
||||
/**
|
||||
* @return the next Bytes from the page
|
||||
*/
|
||||
byte[] readBytes();
|
||||
|
||||
/**
|
||||
* @return the next Decimal from the page
|
||||
*/
|
||||
byte[] readDecimal();
|
||||
|
||||
/**
|
||||
* @return the next Double from the page
|
||||
*/
|
||||
double readDouble();
|
||||
|
||||
/**
|
||||
* @return the next TimestampData from the page
|
||||
*/
|
||||
TimestampData readTimestamp();
|
||||
|
||||
/**
|
||||
* @return is data valid
|
||||
*/
|
||||
boolean isValid();
|
||||
|
||||
/**
|
||||
* @return the underlying dictionary if current reader is dictionary encoded
|
||||
*/
|
||||
Dictionary getDictionary();
|
||||
|
||||
/**
|
||||
* @param id in dictionary
|
||||
* @return the Bytes from the dictionary by id
|
||||
*/
|
||||
byte[] readBytes(int id);
|
||||
|
||||
/**
|
||||
* @param id in dictionary
|
||||
* @return the Float from the dictionary by id
|
||||
*/
|
||||
float readFloat(int id);
|
||||
|
||||
/**
|
||||
* @param id in dictionary
|
||||
* @return the Double from the dictionary by id
|
||||
*/
|
||||
double readDouble(int id);
|
||||
|
||||
/**
|
||||
* @param id in dictionary
|
||||
* @return the Integer from the dictionary by id
|
||||
*/
|
||||
int readInteger(int id);
|
||||
|
||||
/**
|
||||
* @param id in dictionary
|
||||
* @return the Long from the dictionary by id
|
||||
*/
|
||||
long readLong(int id);
|
||||
|
||||
/**
|
||||
* @param id in dictionary
|
||||
* @return the Small Int from the dictionary by id
|
||||
*/
|
||||
int readSmallInt(int id);
|
||||
|
||||
/**
|
||||
* @param id in dictionary
|
||||
* @return the tiny int from the dictionary by id
|
||||
*/
|
||||
int readTinyInt(int id);
|
||||
|
||||
/**
|
||||
* @param id in dictionary
|
||||
* @return the Boolean from the dictionary by id
|
||||
*/
|
||||
boolean readBoolean(int id);
|
||||
|
||||
/**
|
||||
* @param id in dictionary
|
||||
* @return the Decimal from the dictionary by id
|
||||
*/
|
||||
byte[] readDecimal(int id);
|
||||
|
||||
/**
|
||||
* @param id in dictionary
|
||||
* @return the TimestampData from the dictionary by id
|
||||
*/
|
||||
TimestampData readTimestamp(int id);
|
||||
|
||||
/**
|
||||
* @param id in dictionary
|
||||
* @return the String from the dictionary by id
|
||||
*/
|
||||
byte[] readString(int id);
|
||||
|
||||
/**
|
||||
* @param id in dictionary
|
||||
* @return the Varchar from the dictionary by id
|
||||
*/
|
||||
byte[] readVarchar(int id);
|
||||
|
||||
/**
|
||||
* @param id in dictionary
|
||||
* @return the Char from the dictionary by id
|
||||
*/
|
||||
byte[] readChar(int id);
|
||||
}
|
||||
|
||||
@@ -0,0 +1,304 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.format.cow.vector.reader;
|
||||
|
||||
import org.apache.flink.table.data.TimestampData;
|
||||
import org.apache.parquet.bytes.ByteBufferInputStream;
|
||||
import org.apache.parquet.column.Dictionary;
|
||||
import org.apache.parquet.column.values.ValuesReader;
|
||||
import org.apache.parquet.io.api.Binary;
|
||||
import org.apache.parquet.schema.PrimitiveType;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
import java.sql.Timestamp;
|
||||
|
||||
import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.JULIAN_EPOCH_OFFSET_DAYS;
|
||||
import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.MILLIS_IN_DAY;
|
||||
import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.NANOS_PER_MILLISECOND;
|
||||
import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.NANOS_PER_SECOND;
|
||||
|
||||
/**
|
||||
* Parquet file has self-describing schema which may differ from the user required schema (e.g.
|
||||
* schema evolution). This factory is used to retrieve user required typed data via corresponding
|
||||
* reader which reads the underlying data.
|
||||
*/
|
||||
public final class ParquetDataColumnReaderFactory {
|
||||
|
||||
private ParquetDataColumnReaderFactory() {
|
||||
}
|
||||
|
||||
/**
|
||||
* default reader for {@link ParquetDataColumnReader}.
|
||||
*/
|
||||
public static class DefaultParquetDataColumnReader implements ParquetDataColumnReader {
|
||||
protected ValuesReader valuesReader;
|
||||
protected Dictionary dict;
|
||||
|
||||
// After the data is read in the parquet type, isValid will be set to true if the data can
|
||||
// be returned in the type defined in HMS. Otherwise isValid is set to false.
|
||||
boolean isValid = true;
|
||||
|
||||
public DefaultParquetDataColumnReader(ValuesReader valuesReader) {
|
||||
this.valuesReader = valuesReader;
|
||||
}
|
||||
|
||||
public DefaultParquetDataColumnReader(Dictionary dict) {
|
||||
this.dict = dict;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initFromPage(int i, ByteBufferInputStream in) throws IOException {
|
||||
valuesReader.initFromPage(i, in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean readBoolean() {
|
||||
return valuesReader.readBoolean();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean readBoolean(int id) {
|
||||
return dict.decodeToBoolean(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] readString(int id) {
|
||||
return dict.decodeToBinary(id).getBytesUnsafe();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] readString() {
|
||||
return valuesReader.readBytes().getBytesUnsafe();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] readVarchar() {
|
||||
// we need to enforce the size here even the types are the same
|
||||
return valuesReader.readBytes().getBytesUnsafe();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] readVarchar(int id) {
|
||||
return dict.decodeToBinary(id).getBytesUnsafe();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] readChar() {
|
||||
return valuesReader.readBytes().getBytesUnsafe();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] readChar(int id) {
|
||||
return dict.decodeToBinary(id).getBytesUnsafe();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] readBytes() {
|
||||
return valuesReader.readBytes().getBytesUnsafe();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] readBytes(int id) {
|
||||
return dict.decodeToBinary(id).getBytesUnsafe();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] readDecimal() {
|
||||
return valuesReader.readBytes().getBytesUnsafe();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] readDecimal(int id) {
|
||||
return dict.decodeToBinary(id).getBytesUnsafe();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float readFloat() {
|
||||
return valuesReader.readFloat();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float readFloat(int id) {
|
||||
return dict.decodeToFloat(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public double readDouble() {
|
||||
return valuesReader.readDouble();
|
||||
}
|
||||
|
||||
@Override
|
||||
public double readDouble(int id) {
|
||||
return dict.decodeToDouble(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TimestampData readTimestamp() {
|
||||
throw new RuntimeException("Unsupported operation");
|
||||
}
|
||||
|
||||
@Override
|
||||
public TimestampData readTimestamp(int id) {
|
||||
throw new RuntimeException("Unsupported operation");
|
||||
}
|
||||
|
||||
@Override
|
||||
public int readInteger() {
|
||||
return valuesReader.readInteger();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int readInteger(int id) {
|
||||
return dict.decodeToInt(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isValid() {
|
||||
return isValid;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long readLong(int id) {
|
||||
return dict.decodeToLong(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long readLong() {
|
||||
return valuesReader.readLong();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int readSmallInt() {
|
||||
return valuesReader.readInteger();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int readSmallInt(int id) {
|
||||
return dict.decodeToInt(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int readTinyInt() {
|
||||
return valuesReader.readInteger();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int readTinyInt(int id) {
|
||||
return dict.decodeToInt(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int readValueDictionaryId() {
|
||||
return valuesReader.readValueDictionaryId();
|
||||
}
|
||||
|
||||
public void skip() {
|
||||
valuesReader.skip();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Dictionary getDictionary() {
|
||||
return dict;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The reader who reads from the underlying Timestamp value value.
|
||||
*/
|
||||
public static class TypesFromInt96PageReader extends DefaultParquetDataColumnReader {
|
||||
private final boolean isUtcTimestamp;
|
||||
|
||||
public TypesFromInt96PageReader(ValuesReader realReader, boolean isUtcTimestamp) {
|
||||
super(realReader);
|
||||
this.isUtcTimestamp = isUtcTimestamp;
|
||||
}
|
||||
|
||||
public TypesFromInt96PageReader(Dictionary dict, boolean isUtcTimestamp) {
|
||||
super(dict);
|
||||
this.isUtcTimestamp = isUtcTimestamp;
|
||||
}
|
||||
|
||||
private TimestampData convert(Binary binary) {
|
||||
ByteBuffer buf = binary.toByteBuffer();
|
||||
buf.order(ByteOrder.LITTLE_ENDIAN);
|
||||
long timeOfDayNanos = buf.getLong();
|
||||
int julianDay = buf.getInt();
|
||||
return int96ToTimestamp(isUtcTimestamp, timeOfDayNanos, julianDay);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TimestampData readTimestamp(int id) {
|
||||
return convert(dict.decodeToBinary(id));
|
||||
}
|
||||
|
||||
@Override
|
||||
public TimestampData readTimestamp() {
|
||||
return convert(valuesReader.readBytes());
|
||||
}
|
||||
}
|
||||
|
||||
private static ParquetDataColumnReader getDataColumnReaderByTypeHelper(
|
||||
boolean isDictionary,
|
||||
PrimitiveType parquetType,
|
||||
Dictionary dictionary,
|
||||
ValuesReader valuesReader,
|
||||
boolean isUtcTimestamp) {
|
||||
if (parquetType.getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.INT96) {
|
||||
return isDictionary
|
||||
? new TypesFromInt96PageReader(dictionary, isUtcTimestamp)
|
||||
: new TypesFromInt96PageReader(valuesReader, isUtcTimestamp);
|
||||
} else {
|
||||
return isDictionary
|
||||
? new DefaultParquetDataColumnReader(dictionary)
|
||||
: new DefaultParquetDataColumnReader(valuesReader);
|
||||
}
|
||||
}
|
||||
|
||||
public static ParquetDataColumnReader getDataColumnReaderByTypeOnDictionary(
|
||||
PrimitiveType parquetType, Dictionary realReader, boolean isUtcTimestamp) {
|
||||
return getDataColumnReaderByTypeHelper(true, parquetType, realReader, null, isUtcTimestamp);
|
||||
}
|
||||
|
||||
public static ParquetDataColumnReader getDataColumnReaderByType(
|
||||
PrimitiveType parquetType, ValuesReader realReader, boolean isUtcTimestamp) {
|
||||
return getDataColumnReaderByTypeHelper(
|
||||
false, parquetType, null, realReader, isUtcTimestamp);
|
||||
}
|
||||
|
||||
private static TimestampData int96ToTimestamp(
|
||||
boolean utcTimestamp, long nanosOfDay, int julianDay) {
|
||||
long millisecond = julianDayToMillis(julianDay) + (nanosOfDay / NANOS_PER_MILLISECOND);
|
||||
|
||||
if (utcTimestamp) {
|
||||
int nanoOfMillisecond = (int) (nanosOfDay % NANOS_PER_MILLISECOND);
|
||||
return TimestampData.fromEpochMillis(millisecond, nanoOfMillisecond);
|
||||
} else {
|
||||
Timestamp timestamp = new Timestamp(millisecond);
|
||||
timestamp.setNanos((int) (nanosOfDay % NANOS_PER_SECOND));
|
||||
return TimestampData.fromTimestamp(timestamp);
|
||||
}
|
||||
}
|
||||
|
||||
private static long julianDayToMillis(int julianDay) {
|
||||
return (julianDay - JULIAN_EPOCH_OFFSET_DAYS) * MILLIS_IN_DAY;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,57 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.format.cow.vector.reader;
|
||||
|
||||
import org.apache.hudi.table.format.cow.vector.HeapRowColumnVector;
|
||||
|
||||
import org.apache.flink.formats.parquet.vector.reader.ColumnReader;
|
||||
import org.apache.flink.table.data.vector.writable.WritableColumnVector;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Row {@link ColumnReader}.
|
||||
*/
|
||||
public class RowColumnReader implements ColumnReader<WritableColumnVector> {
|
||||
|
||||
private final List<ColumnReader> fieldReaders;
|
||||
|
||||
public RowColumnReader(List<ColumnReader> fieldReaders) {
|
||||
this.fieldReaders = fieldReaders;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readToVector(int readNumber, WritableColumnVector vector) throws IOException {
|
||||
HeapRowColumnVector rowColumnVector = (HeapRowColumnVector) vector;
|
||||
WritableColumnVector[] vectors = rowColumnVector.vectors;
|
||||
for (int i = 0; i < vectors.length; i++) {
|
||||
fieldReaders.get(i).readToVector(readNumber, vectors[i]);
|
||||
|
||||
for (int j = 0; j < readNumber; j++) {
|
||||
boolean isNull = (i == 0)
|
||||
? vectors[i].isNullAt(j)
|
||||
: rowColumnVector.isNullAt(j) && vectors[i].isNullAt(j);
|
||||
if (isNull) {
|
||||
rowColumnVector.setNullAt(j);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,304 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.format.cow.vector.reader;
|
||||
|
||||
import org.apache.flink.table.data.vector.writable.WritableColumnVector;
|
||||
import org.apache.flink.table.data.vector.writable.WritableIntVector;
|
||||
import org.apache.parquet.Preconditions;
|
||||
import org.apache.parquet.bytes.ByteBufferInputStream;
|
||||
import org.apache.parquet.bytes.BytesUtils;
|
||||
import org.apache.parquet.column.values.bitpacking.BytePacker;
|
||||
import org.apache.parquet.column.values.bitpacking.Packer;
|
||||
import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder;
|
||||
import org.apache.parquet.io.ParquetDecodingException;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
* Run length decoder for data and dictionary ids.
|
||||
* See https://github.com/apache/parquet-format/blob/master/Encodings.md
|
||||
* See {@link RunLengthBitPackingHybridDecoder}.
|
||||
*
|
||||
* <p>Note: Reference Flink release 1.11.2
|
||||
* {@code org.apache.flink.formats.parquet.vector.reader.RunLengthDecoder}
|
||||
* because it is package scope.
|
||||
*/
|
||||
final class RunLengthDecoder {
|
||||
|
||||
/**
|
||||
* If true, the bit width is fixed. This decoder is used in different places and this also
|
||||
* controls if we need to read the bitwidth from the beginning of the data stream.
|
||||
*/
|
||||
private final boolean fixedWidth;
|
||||
private final boolean readLength;
|
||||
|
||||
// Encoded data.
|
||||
private ByteBufferInputStream in;
|
||||
|
||||
// bit/byte width of decoded data and utility to batch unpack them.
|
||||
private int bitWidth;
|
||||
private int bytesWidth;
|
||||
private BytePacker packer;
|
||||
|
||||
// Current decoding mode and values
|
||||
RunLengthDecoder.MODE mode;
|
||||
int currentCount;
|
||||
int currentValue;
|
||||
|
||||
// Buffer of decoded values if the values are PACKED.
|
||||
int[] currentBuffer = new int[16];
|
||||
int currentBufferIdx = 0;
|
||||
|
||||
RunLengthDecoder() {
|
||||
this.fixedWidth = false;
|
||||
this.readLength = false;
|
||||
}
|
||||
|
||||
RunLengthDecoder(int bitWidth) {
|
||||
this.fixedWidth = true;
|
||||
this.readLength = bitWidth != 0;
|
||||
initWidthAndPacker(bitWidth);
|
||||
}
|
||||
|
||||
RunLengthDecoder(int bitWidth, boolean readLength) {
|
||||
this.fixedWidth = true;
|
||||
this.readLength = readLength;
|
||||
initWidthAndPacker(bitWidth);
|
||||
}
|
||||
|
||||
/**
|
||||
* Init from input stream.
|
||||
*/
|
||||
void initFromStream(int valueCount, ByteBufferInputStream in) throws IOException {
|
||||
this.in = in;
|
||||
if (fixedWidth) {
|
||||
// initialize for repetition and definition levels
|
||||
if (readLength) {
|
||||
int length = readIntLittleEndian();
|
||||
this.in = in.sliceStream(length);
|
||||
}
|
||||
} else {
|
||||
// initialize for values
|
||||
if (in.available() > 0) {
|
||||
initWidthAndPacker(in.read());
|
||||
}
|
||||
}
|
||||
if (bitWidth == 0) {
|
||||
// 0 bit width, treat this as an RLE run of valueCount number of 0's.
|
||||
this.mode = RunLengthDecoder.MODE.RLE;
|
||||
this.currentCount = valueCount;
|
||||
this.currentValue = 0;
|
||||
} else {
|
||||
this.currentCount = 0;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Initializes the internal state for decoding ints of `bitWidth`.
|
||||
*/
|
||||
private void initWidthAndPacker(int bitWidth) {
|
||||
Preconditions.checkArgument(bitWidth >= 0 && bitWidth <= 32, "bitWidth must be >= 0 and <= 32");
|
||||
this.bitWidth = bitWidth;
|
||||
this.bytesWidth = BytesUtils.paddedByteCountFromBits(bitWidth);
|
||||
this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
|
||||
}
|
||||
|
||||
int readInteger() {
|
||||
if (this.currentCount == 0) {
|
||||
this.readNextGroup();
|
||||
}
|
||||
|
||||
this.currentCount--;
|
||||
switch (mode) {
|
||||
case RLE:
|
||||
return this.currentValue;
|
||||
case PACKED:
|
||||
return this.currentBuffer[currentBufferIdx++];
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Decoding for dictionary ids. The IDs are populated into `values` and the nullability is
|
||||
* populated into `nulls`.
|
||||
*/
|
||||
void readDictionaryIds(
|
||||
int total,
|
||||
WritableIntVector values,
|
||||
WritableColumnVector nulls,
|
||||
int rowId,
|
||||
int level,
|
||||
RunLengthDecoder data) {
|
||||
int left = total;
|
||||
while (left > 0) {
|
||||
if (this.currentCount == 0) {
|
||||
this.readNextGroup();
|
||||
}
|
||||
int n = Math.min(left, this.currentCount);
|
||||
switch (mode) {
|
||||
case RLE:
|
||||
if (currentValue == level) {
|
||||
data.readDictionaryIdData(n, values, rowId);
|
||||
} else {
|
||||
nulls.setNulls(rowId, n);
|
||||
}
|
||||
break;
|
||||
case PACKED:
|
||||
for (int i = 0; i < n; ++i) {
|
||||
if (currentBuffer[currentBufferIdx++] == level) {
|
||||
values.setInt(rowId + i, data.readInteger());
|
||||
} else {
|
||||
nulls.setNullAt(rowId + i);
|
||||
}
|
||||
}
|
||||
break;
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
rowId += n;
|
||||
left -= n;
|
||||
currentCount -= n;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* It is used to decode dictionary IDs.
|
||||
*/
|
||||
private void readDictionaryIdData(int total, WritableIntVector c, int rowId) {
|
||||
int left = total;
|
||||
while (left > 0) {
|
||||
if (this.currentCount == 0) {
|
||||
this.readNextGroup();
|
||||
}
|
||||
int n = Math.min(left, this.currentCount);
|
||||
switch (mode) {
|
||||
case RLE:
|
||||
c.setInts(rowId, n, currentValue);
|
||||
break;
|
||||
case PACKED:
|
||||
c.setInts(rowId, n, currentBuffer, currentBufferIdx);
|
||||
currentBufferIdx += n;
|
||||
break;
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
rowId += n;
|
||||
left -= n;
|
||||
currentCount -= n;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads the next varint encoded int.
|
||||
*/
|
||||
private int readUnsignedVarInt() throws IOException {
|
||||
int value = 0;
|
||||
int shift = 0;
|
||||
int b;
|
||||
do {
|
||||
b = in.read();
|
||||
value |= (b & 0x7F) << shift;
|
||||
shift += 7;
|
||||
} while ((b & 0x80) != 0);
|
||||
return value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads the next 4 byte little endian int.
|
||||
*/
|
||||
private int readIntLittleEndian() throws IOException {
|
||||
int ch4 = in.read();
|
||||
int ch3 = in.read();
|
||||
int ch2 = in.read();
|
||||
int ch1 = in.read();
|
||||
return ((ch1 << 24) + (ch2 << 16) + (ch3 << 8) + ch4);
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads the next byteWidth little endian int.
|
||||
*/
|
||||
private int readIntLittleEndianPaddedOnBitWidth() throws IOException {
|
||||
switch (bytesWidth) {
|
||||
case 0:
|
||||
return 0;
|
||||
case 1:
|
||||
return in.read();
|
||||
case 2: {
|
||||
int ch2 = in.read();
|
||||
int ch1 = in.read();
|
||||
return (ch1 << 8) + ch2;
|
||||
}
|
||||
case 3: {
|
||||
int ch3 = in.read();
|
||||
int ch2 = in.read();
|
||||
int ch1 = in.read();
|
||||
return (ch1 << 16) + (ch2 << 8) + ch3;
|
||||
}
|
||||
case 4: {
|
||||
return readIntLittleEndian();
|
||||
}
|
||||
default:
|
||||
throw new RuntimeException("Unreachable");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads the next group.
|
||||
*/
|
||||
void readNextGroup() {
|
||||
try {
|
||||
int header = readUnsignedVarInt();
|
||||
this.mode = (header & 1) == 0 ? RunLengthDecoder.MODE.RLE : RunLengthDecoder.MODE.PACKED;
|
||||
switch (mode) {
|
||||
case RLE:
|
||||
this.currentCount = header >>> 1;
|
||||
this.currentValue = readIntLittleEndianPaddedOnBitWidth();
|
||||
return;
|
||||
case PACKED:
|
||||
int numGroups = header >>> 1;
|
||||
this.currentCount = numGroups * 8;
|
||||
|
||||
if (this.currentBuffer.length < this.currentCount) {
|
||||
this.currentBuffer = new int[this.currentCount];
|
||||
}
|
||||
currentBufferIdx = 0;
|
||||
int valueIndex = 0;
|
||||
while (valueIndex < this.currentCount) {
|
||||
// values are bit packed 8 at a time, so reading bitWidth will always work
|
||||
ByteBuffer buffer = in.slice(bitWidth);
|
||||
this.packer.unpack8Values(buffer, buffer.position(), this.currentBuffer, valueIndex);
|
||||
valueIndex += 8;
|
||||
}
|
||||
return;
|
||||
default:
|
||||
throw new ParquetDecodingException("not a valid mode " + this.mode);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new ParquetDecodingException("Failed to read from input stream", e);
|
||||
}
|
||||
}
|
||||
|
||||
enum MODE {
|
||||
RLE,
|
||||
PACKED
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,822 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.format.mor;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieOperation;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
|
||||
import org.apache.hudi.common.table.log.InstantRange;
|
||||
import org.apache.hudi.common.util.ClosableIterator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.configuration.OptionsResolver;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.keygen.KeyGenUtils;
|
||||
import org.apache.hudi.table.format.FilePathUtils;
|
||||
import org.apache.hudi.table.format.FormatUtils;
|
||||
import org.apache.hudi.table.format.cow.ParquetSplitReaderUtil;
|
||||
import org.apache.hudi.table.format.cow.vector.reader.ParquetColumnarRowSplitReader;
|
||||
import org.apache.hudi.util.AvroToRowDataConverters;
|
||||
import org.apache.hudi.util.RowDataProjection;
|
||||
import org.apache.hudi.util.RowDataToAvroConverters;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.util.StringToRowDataConverter;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.GenericRecordBuilder;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.flink.annotation.VisibleForTesting;
|
||||
import org.apache.flink.api.common.io.DefaultInputSplitAssigner;
|
||||
import org.apache.flink.api.common.io.RichInputFormat;
|
||||
import org.apache.flink.api.common.io.statistics.BaseStatistics;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.core.io.InputSplitAssigner;
|
||||
import org.apache.flink.table.data.GenericRowData;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.types.DataType;
|
||||
import org.apache.flink.table.types.logical.LogicalType;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
import org.apache.flink.types.RowKind;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
import static org.apache.flink.table.data.vector.VectorizedColumnBatch.DEFAULT_SIZE;
|
||||
import static org.apache.flink.table.filesystem.RowPartitionComputer.restorePartValueFromType;
|
||||
import static org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.HOODIE_COMMIT_TIME_COL_POS;
|
||||
import static org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS;
|
||||
import static org.apache.hudi.table.format.FormatUtils.buildAvroRecordBySchema;
|
||||
|
||||
/**
|
||||
* The base InputFormat class to read from Hoodie data + log files.
|
||||
*
|
||||
* <P>Use {@link org.apache.flink.formats.parquet.utils.ParquetRecordReader}
|
||||
* to read files instead of {@link org.apache.flink.core.fs.FSDataInputStream},
|
||||
* overrides {@link #createInputSplits(int)} and {@link #close()} to change the behaviors.
|
||||
*/
|
||||
public class MergeOnReadInputFormat
|
||||
extends RichInputFormat<RowData, MergeOnReadInputSplit> {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
private final Configuration conf;
|
||||
|
||||
private transient org.apache.hadoop.conf.Configuration hadoopConf;
|
||||
|
||||
private final MergeOnReadTableState tableState;
|
||||
|
||||
/**
|
||||
* Uniform iterator view for the underneath records.
|
||||
*/
|
||||
private transient RecordIterator iterator;
|
||||
|
||||
// for project push down
|
||||
/**
|
||||
* Full table names.
|
||||
*/
|
||||
private final List<String> fieldNames;
|
||||
|
||||
/**
|
||||
* Full field data types.
|
||||
*/
|
||||
private final List<DataType> fieldTypes;
|
||||
|
||||
/**
|
||||
* Default partition name when the field value is null.
|
||||
*/
|
||||
private final String defaultPartName;
|
||||
|
||||
/**
|
||||
* Required field positions.
|
||||
*/
|
||||
private final int[] requiredPos;
|
||||
|
||||
// for limit push down
|
||||
/**
|
||||
* Limit for the reader, -1 when the reading is not limited.
|
||||
*/
|
||||
private final long limit;
|
||||
|
||||
/**
|
||||
* Recording the current read count for limit check.
|
||||
*/
|
||||
private long currentReadCount = 0;
|
||||
|
||||
/**
|
||||
* Flag saying whether to emit the deletes. In streaming read mode, downstream
|
||||
* operators need the DELETE messages to retract the legacy accumulator.
|
||||
*/
|
||||
private boolean emitDelete;
|
||||
|
||||
/**
|
||||
* Flag saying whether the input format has been closed.
|
||||
*/
|
||||
private boolean closed = true;
|
||||
|
||||
private MergeOnReadInputFormat(
|
||||
Configuration conf,
|
||||
MergeOnReadTableState tableState,
|
||||
List<DataType> fieldTypes,
|
||||
String defaultPartName,
|
||||
long limit,
|
||||
boolean emitDelete) {
|
||||
this.conf = conf;
|
||||
this.tableState = tableState;
|
||||
this.fieldNames = tableState.getRowType().getFieldNames();
|
||||
this.fieldTypes = fieldTypes;
|
||||
this.defaultPartName = defaultPartName;
|
||||
// Needs improvement: this requiredPos is only suitable for parquet reader,
|
||||
// because we need to
|
||||
this.requiredPos = tableState.getRequiredPositions();
|
||||
this.limit = limit;
|
||||
this.emitDelete = emitDelete;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the builder for {@link MergeOnReadInputFormat}.
|
||||
*/
|
||||
public static Builder builder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(MergeOnReadInputSplit split) throws IOException {
|
||||
this.currentReadCount = 0L;
|
||||
this.closed = false;
|
||||
this.hadoopConf = StreamerUtil.getHadoopConf();
|
||||
if (!(split.getLogPaths().isPresent() && split.getLogPaths().get().size() > 0)) {
|
||||
if (split.getInstantRange() != null) {
|
||||
// base file only with commit time filtering
|
||||
this.iterator = new BaseFileOnlyFilteringIterator(
|
||||
split.getInstantRange(),
|
||||
this.tableState.getRequiredRowType(),
|
||||
getReader(split.getBasePath().get(), getRequiredPosWithCommitTime(this.requiredPos)));
|
||||
} else {
|
||||
// base file only
|
||||
this.iterator = new BaseFileOnlyIterator(getRequiredSchemaReader(split.getBasePath().get()));
|
||||
}
|
||||
} else if (!split.getBasePath().isPresent()) {
|
||||
// log files only
|
||||
if (OptionsResolver.emitChangelog(conf)) {
|
||||
this.iterator = new LogFileOnlyIterator(getUnMergedLogFileIterator(split));
|
||||
} else {
|
||||
this.iterator = new LogFileOnlyIterator(getLogFileIterator(split));
|
||||
}
|
||||
} else if (split.getMergeType().equals(FlinkOptions.REALTIME_SKIP_MERGE)) {
|
||||
this.iterator = new SkipMergeIterator(
|
||||
getRequiredSchemaReader(split.getBasePath().get()),
|
||||
getLogFileIterator(split));
|
||||
} else if (split.getMergeType().equals(FlinkOptions.REALTIME_PAYLOAD_COMBINE)) {
|
||||
this.iterator = new MergeIterator(
|
||||
hadoopConf,
|
||||
split,
|
||||
this.tableState.getRowType(),
|
||||
this.tableState.getRequiredRowType(),
|
||||
new Schema.Parser().parse(this.tableState.getAvroSchema()),
|
||||
new Schema.Parser().parse(this.tableState.getRequiredAvroSchema()),
|
||||
this.requiredPos,
|
||||
this.emitDelete,
|
||||
this.conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED),
|
||||
this.tableState.getOperationPos(),
|
||||
getFullSchemaReader(split.getBasePath().get()));
|
||||
} else {
|
||||
throw new HoodieException("Unable to select an Iterator to read the Hoodie MOR File Split for "
|
||||
+ "file path: " + split.getBasePath()
|
||||
+ "log paths: " + split.getLogPaths()
|
||||
+ "hoodie table path: " + split.getTablePath()
|
||||
+ "spark partition Index: " + split.getSplitNumber()
|
||||
+ "merge type: " + split.getMergeType());
|
||||
}
|
||||
mayShiftInputSplit(split);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void configure(Configuration configuration) {
|
||||
// no operation
|
||||
// may support nested files in the future.
|
||||
}
|
||||
|
||||
@Override
|
||||
public BaseStatistics getStatistics(BaseStatistics baseStatistics) {
|
||||
// statistics not supported yet.
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public MergeOnReadInputSplit[] createInputSplits(int minNumSplits) {
|
||||
return this.tableState.getInputSplits().toArray(new MergeOnReadInputSplit[0]);
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputSplitAssigner getInputSplitAssigner(MergeOnReadInputSplit[] mergeOnReadInputSplits) {
|
||||
return new DefaultInputSplitAssigner(mergeOnReadInputSplits);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean reachedEnd() throws IOException {
|
||||
if (limit > 0 && currentReadCount >= limit) {
|
||||
return true;
|
||||
} else {
|
||||
// log file reaches end ?
|
||||
return this.iterator.reachedEnd();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowData nextRecord(RowData o) {
|
||||
currentReadCount++;
|
||||
return this.iterator.nextRecord();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (this.iterator != null) {
|
||||
this.iterator.close();
|
||||
}
|
||||
this.iterator = null;
|
||||
this.closed = true;
|
||||
}
|
||||
|
||||
public boolean isClosed() {
|
||||
return this.closed;
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Utilities
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
/**
|
||||
* Shifts the input split by its consumed records number.
|
||||
*
|
||||
* <p>Note: This action is time-consuming.
|
||||
*/
|
||||
private void mayShiftInputSplit(MergeOnReadInputSplit split) throws IOException {
|
||||
if (split.isConsumed()) {
|
||||
// if the input split has been consumed before,
|
||||
// shift the input split with consumed num of records first
|
||||
for (long i = 0; i < split.getConsumed() && !reachedEnd(); i++) {
|
||||
nextRecord(null);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private ParquetColumnarRowSplitReader getFullSchemaReader(String path) throws IOException {
|
||||
return getReader(path, IntStream.range(0, this.tableState.getRowType().getFieldCount()).toArray());
|
||||
}
|
||||
|
||||
private ParquetColumnarRowSplitReader getRequiredSchemaReader(String path) throws IOException {
|
||||
return getReader(path, this.requiredPos);
|
||||
}
|
||||
|
||||
private ParquetColumnarRowSplitReader getReader(String path, int[] requiredPos) throws IOException {
|
||||
// generate partition specs.
|
||||
LinkedHashMap<String, String> partSpec = FilePathUtils.extractPartitionKeyValues(
|
||||
new org.apache.hadoop.fs.Path(path).getParent(),
|
||||
this.conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING),
|
||||
FilePathUtils.extractPartitionKeys(this.conf));
|
||||
LinkedHashMap<String, Object> partObjects = new LinkedHashMap<>();
|
||||
partSpec.forEach((k, v) -> partObjects.put(k, restorePartValueFromType(
|
||||
defaultPartName.equals(v) ? null : v,
|
||||
fieldTypes.get(fieldNames.indexOf(k)))));
|
||||
|
||||
return ParquetSplitReaderUtil.genPartColumnarRowReader(
|
||||
this.conf.getBoolean(FlinkOptions.UTC_TIMEZONE),
|
||||
true,
|
||||
FormatUtils.getParquetConf(this.conf, hadoopConf),
|
||||
fieldNames.toArray(new String[0]),
|
||||
fieldTypes.toArray(new DataType[0]),
|
||||
partObjects,
|
||||
requiredPos,
|
||||
DEFAULT_SIZE,
|
||||
new org.apache.flink.core.fs.Path(path),
|
||||
0,
|
||||
Long.MAX_VALUE); // read the whole file
|
||||
}
|
||||
|
||||
private ClosableIterator<RowData> getLogFileIterator(MergeOnReadInputSplit split) {
|
||||
final Schema tableSchema = new Schema.Parser().parse(tableState.getAvroSchema());
|
||||
final Schema requiredSchema = new Schema.Parser().parse(tableState.getRequiredAvroSchema());
|
||||
final GenericRecordBuilder recordBuilder = new GenericRecordBuilder(requiredSchema);
|
||||
final AvroToRowDataConverters.AvroToRowDataConverter avroToRowDataConverter =
|
||||
AvroToRowDataConverters.createRowConverter(tableState.getRequiredRowType());
|
||||
final HoodieMergedLogRecordScanner scanner = FormatUtils.logScanner(split, tableSchema, hadoopConf, conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED));
|
||||
final Iterator<String> logRecordsKeyIterator = scanner.getRecords().keySet().iterator();
|
||||
final int[] pkOffset = tableState.getPkOffsetsInRequired();
|
||||
// flag saying whether the pk semantics has been dropped by user specified
|
||||
// projections. For e.g, if the pk fields are [a, b] but user only select a,
|
||||
// then the pk semantics is lost.
|
||||
final boolean pkSemanticLost = Arrays.stream(pkOffset).anyMatch(offset -> offset == -1);
|
||||
final LogicalType[] pkTypes = pkSemanticLost ? null : tableState.getPkTypes(pkOffset);
|
||||
final StringToRowDataConverter converter = pkSemanticLost ? null : new StringToRowDataConverter(pkTypes);
|
||||
|
||||
return new ClosableIterator<RowData>() {
|
||||
private RowData currentRecord;
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
while (logRecordsKeyIterator.hasNext()) {
|
||||
String curAvroKey = logRecordsKeyIterator.next();
|
||||
Option<IndexedRecord> curAvroRecord = null;
|
||||
final HoodieAvroRecord<?> hoodieRecord = (HoodieAvroRecord) scanner.getRecords().get(curAvroKey);
|
||||
try {
|
||||
curAvroRecord = hoodieRecord.getData().getInsertValue(tableSchema);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Get avro insert value error for key: " + curAvroKey, e);
|
||||
}
|
||||
if (!curAvroRecord.isPresent()) {
|
||||
// delete record found
|
||||
if (emitDelete && !pkSemanticLost) {
|
||||
GenericRowData delete = new GenericRowData(tableState.getRequiredRowType().getFieldCount());
|
||||
|
||||
final String recordKey = hoodieRecord.getRecordKey();
|
||||
final String[] pkFields = KeyGenUtils.extractRecordKeys(recordKey);
|
||||
final Object[] converted = converter.convert(pkFields);
|
||||
for (int i = 0; i < pkOffset.length; i++) {
|
||||
delete.setField(pkOffset[i], converted[i]);
|
||||
}
|
||||
delete.setRowKind(RowKind.DELETE);
|
||||
|
||||
this.currentRecord = delete;
|
||||
return true;
|
||||
}
|
||||
// skipping if the condition is unsatisfied
|
||||
// continue;
|
||||
} else {
|
||||
final IndexedRecord avroRecord = curAvroRecord.get();
|
||||
final RowKind rowKind = FormatUtils.getRowKindSafely(avroRecord, tableState.getOperationPos());
|
||||
if (rowKind == RowKind.DELETE && !emitDelete) {
|
||||
// skip the delete record
|
||||
continue;
|
||||
}
|
||||
GenericRecord requiredAvroRecord = buildAvroRecordBySchema(
|
||||
avroRecord,
|
||||
requiredSchema,
|
||||
requiredPos,
|
||||
recordBuilder);
|
||||
currentRecord = (RowData) avroToRowDataConverter.convert(requiredAvroRecord);
|
||||
currentRecord.setRowKind(rowKind);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowData next() {
|
||||
return currentRecord;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
scanner.close();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private ClosableIterator<RowData> getUnMergedLogFileIterator(MergeOnReadInputSplit split) {
|
||||
final Schema tableSchema = new Schema.Parser().parse(tableState.getAvroSchema());
|
||||
final Schema requiredSchema = new Schema.Parser().parse(tableState.getRequiredAvroSchema());
|
||||
final GenericRecordBuilder recordBuilder = new GenericRecordBuilder(requiredSchema);
|
||||
final AvroToRowDataConverters.AvroToRowDataConverter avroToRowDataConverter =
|
||||
AvroToRowDataConverters.createRowConverter(tableState.getRequiredRowType());
|
||||
final FormatUtils.BoundedMemoryRecords records = new FormatUtils.BoundedMemoryRecords(split, tableSchema, hadoopConf, conf);
|
||||
final Iterator<HoodieRecord<?>> recordsIterator = records.getRecordsIterator();
|
||||
|
||||
return new ClosableIterator<RowData>() {
|
||||
private RowData currentRecord;
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
while (recordsIterator.hasNext()) {
|
||||
Option<IndexedRecord> curAvroRecord = null;
|
||||
final HoodieAvroRecord<?> hoodieRecord = (HoodieAvroRecord) recordsIterator.next();
|
||||
try {
|
||||
curAvroRecord = hoodieRecord.getData().getInsertValue(tableSchema);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Get avro insert value error for key: " + hoodieRecord.getRecordKey(), e);
|
||||
}
|
||||
if (curAvroRecord.isPresent()) {
|
||||
final IndexedRecord avroRecord = curAvroRecord.get();
|
||||
GenericRecord requiredAvroRecord = buildAvroRecordBySchema(
|
||||
avroRecord,
|
||||
requiredSchema,
|
||||
requiredPos,
|
||||
recordBuilder);
|
||||
currentRecord = (RowData) avroToRowDataConverter.convert(requiredAvroRecord);
|
||||
FormatUtils.setRowKind(currentRecord, avroRecord, tableState.getOperationPos());
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowData next() {
|
||||
return currentRecord;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
records.close();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Inner Class
|
||||
// -------------------------------------------------------------------------
|
||||
private interface RecordIterator {
|
||||
boolean reachedEnd() throws IOException;
|
||||
|
||||
RowData nextRecord();
|
||||
|
||||
void close() throws IOException;
|
||||
}
|
||||
|
||||
static class BaseFileOnlyIterator implements RecordIterator {
|
||||
// base file reader
|
||||
private final ParquetColumnarRowSplitReader reader;
|
||||
|
||||
BaseFileOnlyIterator(ParquetColumnarRowSplitReader reader) {
|
||||
this.reader = reader;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean reachedEnd() throws IOException {
|
||||
return this.reader.reachedEnd();
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowData nextRecord() {
|
||||
return this.reader.nextRecord();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (this.reader != null) {
|
||||
this.reader.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Similar with {@link BaseFileOnlyIterator} but with instant time filtering.
|
||||
*/
|
||||
static class BaseFileOnlyFilteringIterator implements RecordIterator {
|
||||
// base file reader
|
||||
private final ParquetColumnarRowSplitReader reader;
|
||||
private final InstantRange instantRange;
|
||||
private final RowDataProjection projection;
|
||||
|
||||
private RowData currentRecord;
|
||||
|
||||
BaseFileOnlyFilteringIterator(
|
||||
Option<InstantRange> instantRange,
|
||||
RowType requiredRowType,
|
||||
ParquetColumnarRowSplitReader reader) {
|
||||
this.reader = reader;
|
||||
this.instantRange = instantRange.orElse(null);
|
||||
int[] positions = IntStream.range(1, 1 + requiredRowType.getFieldCount()).toArray();
|
||||
projection = RowDataProjection.instance(requiredRowType, positions);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean reachedEnd() throws IOException {
|
||||
while (!this.reader.reachedEnd()) {
|
||||
currentRecord = this.reader.nextRecord();
|
||||
if (instantRange != null) {
|
||||
boolean isInRange = instantRange.isInRange(currentRecord.getString(HOODIE_COMMIT_TIME_COL_POS).toString());
|
||||
if (isInRange) {
|
||||
return false;
|
||||
}
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowData nextRecord() {
|
||||
// can promote: no need to project with null instant range
|
||||
return projection.project(currentRecord);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (this.reader != null) {
|
||||
this.reader.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static class LogFileOnlyIterator implements RecordIterator {
|
||||
// iterator for log files
|
||||
private final ClosableIterator<RowData> iterator;
|
||||
|
||||
LogFileOnlyIterator(ClosableIterator<RowData> iterator) {
|
||||
this.iterator = iterator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean reachedEnd() {
|
||||
return !this.iterator.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowData nextRecord() {
|
||||
return this.iterator.next();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
if (this.iterator != null) {
|
||||
this.iterator.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static class SkipMergeIterator implements RecordIterator {
|
||||
// base file reader
|
||||
private final ParquetColumnarRowSplitReader reader;
|
||||
// iterator for log files
|
||||
private final ClosableIterator<RowData> iterator;
|
||||
|
||||
// add the flag because the flink ParquetColumnarRowSplitReader is buggy:
|
||||
// method #reachedEnd() returns false after it returns true.
|
||||
// refactor it out once FLINK-22370 is resolved.
|
||||
private boolean readLogs = false;
|
||||
|
||||
private RowData currentRecord;
|
||||
|
||||
SkipMergeIterator(ParquetColumnarRowSplitReader reader, ClosableIterator<RowData> iterator) {
|
||||
this.reader = reader;
|
||||
this.iterator = iterator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean reachedEnd() throws IOException {
|
||||
if (!readLogs && !this.reader.reachedEnd()) {
|
||||
currentRecord = this.reader.nextRecord();
|
||||
return false;
|
||||
}
|
||||
readLogs = true;
|
||||
if (this.iterator.hasNext()) {
|
||||
currentRecord = this.iterator.next();
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowData nextRecord() {
|
||||
return currentRecord;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (this.reader != null) {
|
||||
this.reader.close();
|
||||
}
|
||||
if (this.iterator != null) {
|
||||
this.iterator.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static class MergeIterator implements RecordIterator {
|
||||
// base file reader
|
||||
private final ParquetColumnarRowSplitReader reader;
|
||||
// log keys used for merging
|
||||
private final Iterator<String> logKeysIterator;
|
||||
// scanner
|
||||
private final HoodieMergedLogRecordScanner scanner;
|
||||
|
||||
private final Schema tableSchema;
|
||||
private final Schema requiredSchema;
|
||||
private final int[] requiredPos;
|
||||
private final boolean emitDelete;
|
||||
private final int operationPos;
|
||||
private final RowDataToAvroConverters.RowDataToAvroConverter rowDataToAvroConverter;
|
||||
private final AvroToRowDataConverters.AvroToRowDataConverter avroToRowDataConverter;
|
||||
private final GenericRecordBuilder recordBuilder;
|
||||
|
||||
private final RowDataProjection projection;
|
||||
|
||||
private final InstantRange instantRange;
|
||||
|
||||
// add the flag because the flink ParquetColumnarRowSplitReader is buggy:
|
||||
// method #reachedEnd() returns false after it returns true.
|
||||
// refactor it out once FLINK-22370 is resolved.
|
||||
private boolean readLogs = false;
|
||||
|
||||
private final Set<String> keyToSkip = new HashSet<>();
|
||||
|
||||
private RowData currentRecord;
|
||||
|
||||
MergeIterator(
|
||||
org.apache.hadoop.conf.Configuration hadoopConf,
|
||||
MergeOnReadInputSplit split,
|
||||
RowType tableRowType,
|
||||
RowType requiredRowType,
|
||||
Schema tableSchema,
|
||||
Schema requiredSchema,
|
||||
int[] requiredPos,
|
||||
boolean emitDelete,
|
||||
boolean withOperationField,
|
||||
int operationPos,
|
||||
ParquetColumnarRowSplitReader reader) { // the reader should be with full schema
|
||||
this.tableSchema = tableSchema;
|
||||
this.reader = reader;
|
||||
this.scanner = FormatUtils.logScanner(split, tableSchema, hadoopConf, withOperationField);
|
||||
this.logKeysIterator = scanner.getRecords().keySet().iterator();
|
||||
this.requiredSchema = requiredSchema;
|
||||
this.requiredPos = requiredPos;
|
||||
this.emitDelete = emitDelete;
|
||||
this.operationPos = operationPos;
|
||||
this.recordBuilder = new GenericRecordBuilder(requiredSchema);
|
||||
this.rowDataToAvroConverter = RowDataToAvroConverters.createConverter(tableRowType);
|
||||
this.avroToRowDataConverter = AvroToRowDataConverters.createRowConverter(requiredRowType);
|
||||
this.projection = RowDataProjection.instance(requiredRowType, requiredPos);
|
||||
this.instantRange = split.getInstantRange().orElse(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean reachedEnd() throws IOException {
|
||||
while (!readLogs && !this.reader.reachedEnd()) {
|
||||
currentRecord = this.reader.nextRecord();
|
||||
if (instantRange != null) {
|
||||
boolean isInRange = instantRange.isInRange(currentRecord.getString(HOODIE_COMMIT_TIME_COL_POS).toString());
|
||||
if (!isInRange) {
|
||||
// filter base file by instant range
|
||||
continue;
|
||||
}
|
||||
}
|
||||
final String curKey = currentRecord.getString(HOODIE_RECORD_KEY_COL_POS).toString();
|
||||
if (scanner.getRecords().containsKey(curKey)) {
|
||||
keyToSkip.add(curKey);
|
||||
Option<IndexedRecord> mergedAvroRecord = mergeRowWithLog(currentRecord, curKey);
|
||||
if (!mergedAvroRecord.isPresent()) {
|
||||
// deleted
|
||||
continue;
|
||||
} else {
|
||||
final RowKind rowKind = FormatUtils.getRowKindSafely(mergedAvroRecord.get(), this.operationPos);
|
||||
if (!emitDelete && rowKind == RowKind.DELETE) {
|
||||
// deleted
|
||||
continue;
|
||||
}
|
||||
GenericRecord avroRecord = buildAvroRecordBySchema(
|
||||
mergedAvroRecord.get(),
|
||||
requiredSchema,
|
||||
requiredPos,
|
||||
recordBuilder);
|
||||
this.currentRecord = (RowData) avroToRowDataConverter.convert(avroRecord);
|
||||
this.currentRecord.setRowKind(rowKind);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
// project the full record in base with required positions
|
||||
currentRecord = projection.project(currentRecord);
|
||||
return false;
|
||||
}
|
||||
// read the logs
|
||||
readLogs = true;
|
||||
while (logKeysIterator.hasNext()) {
|
||||
final String curKey = logKeysIterator.next();
|
||||
if (!keyToSkip.contains(curKey)) {
|
||||
Option<IndexedRecord> insertAvroRecord = getInsertValue(curKey);
|
||||
if (insertAvroRecord.isPresent()) {
|
||||
// the record is a DELETE if insertAvroRecord not present, skipping
|
||||
GenericRecord avroRecord = buildAvroRecordBySchema(
|
||||
insertAvroRecord.get(),
|
||||
requiredSchema,
|
||||
requiredPos,
|
||||
recordBuilder);
|
||||
this.currentRecord = (RowData) avroToRowDataConverter.convert(avroRecord);
|
||||
FormatUtils.setRowKind(this.currentRecord, insertAvroRecord.get(), this.operationPos);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
private Option<IndexedRecord> getInsertValue(String curKey) throws IOException {
|
||||
final HoodieAvroRecord<?> record = (HoodieAvroRecord) scanner.getRecords().get(curKey);
|
||||
if (!emitDelete && HoodieOperation.isDelete(record.getOperation())) {
|
||||
return Option.empty();
|
||||
}
|
||||
return record.getData().getInsertValue(tableSchema);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowData nextRecord() {
|
||||
return currentRecord;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (this.reader != null) {
|
||||
this.reader.close();
|
||||
}
|
||||
if (this.scanner != null) {
|
||||
this.scanner.close();
|
||||
}
|
||||
}
|
||||
|
||||
private Option<IndexedRecord> mergeRowWithLog(
|
||||
RowData curRow,
|
||||
String curKey) throws IOException {
|
||||
final HoodieAvroRecord<?> record = (HoodieAvroRecord) scanner.getRecords().get(curKey);
|
||||
GenericRecord historyAvroRecord = (GenericRecord) rowDataToAvroConverter.convert(tableSchema, curRow);
|
||||
return record.getData().combineAndGetUpdateValue(historyAvroRecord, tableSchema);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Builder for {@link MergeOnReadInputFormat}.
|
||||
*/
|
||||
public static class Builder {
|
||||
private Configuration conf;
|
||||
private MergeOnReadTableState tableState;
|
||||
private List<DataType> fieldTypes;
|
||||
private String defaultPartName;
|
||||
private long limit = -1;
|
||||
private boolean emitDelete = false;
|
||||
|
||||
public Builder config(Configuration conf) {
|
||||
this.conf = conf;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder tableState(MergeOnReadTableState tableState) {
|
||||
this.tableState = tableState;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder fieldTypes(List<DataType> fieldTypes) {
|
||||
this.fieldTypes = fieldTypes;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder defaultPartName(String defaultPartName) {
|
||||
this.defaultPartName = defaultPartName;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder limit(long limit) {
|
||||
this.limit = limit;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder emitDelete(boolean emitDelete) {
|
||||
this.emitDelete = emitDelete;
|
||||
return this;
|
||||
}
|
||||
|
||||
public MergeOnReadInputFormat build() {
|
||||
return new MergeOnReadInputFormat(conf, tableState, fieldTypes,
|
||||
defaultPartName, limit, emitDelete);
|
||||
}
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Utilities
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
private static int[] getRequiredPosWithCommitTime(int[] requiredPos) {
|
||||
int[] requiredPos2 = new int[requiredPos.length + 1];
|
||||
requiredPos2[0] = HOODIE_COMMIT_TIME_COL_POS;
|
||||
System.arraycopy(requiredPos, 0, requiredPos2, 1, requiredPos.length);
|
||||
return requiredPos2;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public void isEmitDelete(boolean emitDelete) {
|
||||
this.emitDelete = emitDelete;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,128 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.format.mor;
|
||||
|
||||
import org.apache.hudi.common.table.log.InstantRange;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import org.apache.flink.core.io.InputSplit;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Represents an input split of source, actually a data bucket.
|
||||
*/
|
||||
public class MergeOnReadInputSplit implements InputSplit {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
private static final long NUM_NO_CONSUMPTION = 0L;
|
||||
|
||||
private final int splitNum;
|
||||
private final Option<String> basePath;
|
||||
private final Option<List<String>> logPaths;
|
||||
private final String latestCommit;
|
||||
private final String tablePath;
|
||||
private final long maxCompactionMemoryInBytes;
|
||||
private final String mergeType;
|
||||
private final Option<InstantRange> instantRange;
|
||||
|
||||
// for streaming reader to record the consumed offset,
|
||||
// which is the start of next round reading.
|
||||
private long consumed = NUM_NO_CONSUMPTION;
|
||||
|
||||
public MergeOnReadInputSplit(
|
||||
int splitNum,
|
||||
@Nullable String basePath,
|
||||
Option<List<String>> logPaths,
|
||||
String latestCommit,
|
||||
String tablePath,
|
||||
long maxCompactionMemoryInBytes,
|
||||
String mergeType,
|
||||
@Nullable InstantRange instantRange) {
|
||||
this.splitNum = splitNum;
|
||||
this.basePath = Option.ofNullable(basePath);
|
||||
this.logPaths = logPaths;
|
||||
this.latestCommit = latestCommit;
|
||||
this.tablePath = tablePath;
|
||||
this.maxCompactionMemoryInBytes = maxCompactionMemoryInBytes;
|
||||
this.mergeType = mergeType;
|
||||
this.instantRange = Option.ofNullable(instantRange);
|
||||
}
|
||||
|
||||
public Option<String> getBasePath() {
|
||||
return basePath;
|
||||
}
|
||||
|
||||
public Option<List<String>> getLogPaths() {
|
||||
return logPaths;
|
||||
}
|
||||
|
||||
public String getLatestCommit() {
|
||||
return latestCommit;
|
||||
}
|
||||
|
||||
public String getTablePath() {
|
||||
return tablePath;
|
||||
}
|
||||
|
||||
public long getMaxCompactionMemoryInBytes() {
|
||||
return maxCompactionMemoryInBytes;
|
||||
}
|
||||
|
||||
public String getMergeType() {
|
||||
return mergeType;
|
||||
}
|
||||
|
||||
public Option<InstantRange> getInstantRange() {
|
||||
return this.instantRange;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getSplitNumber() {
|
||||
return this.splitNum;
|
||||
}
|
||||
|
||||
public void consume() {
|
||||
this.consumed += 1L;
|
||||
}
|
||||
|
||||
public long getConsumed() {
|
||||
return consumed;
|
||||
}
|
||||
|
||||
public boolean isConsumed() {
|
||||
return this.consumed != NUM_NO_CONSUMPTION;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "MergeOnReadInputSplit{"
|
||||
+ "splitNum=" + splitNum
|
||||
+ ", basePath=" + basePath
|
||||
+ ", logPaths=" + logPaths
|
||||
+ ", latestCommit='" + latestCommit + '\''
|
||||
+ ", tablePath='" + tablePath + '\''
|
||||
+ ", maxCompactionMemoryInBytes=" + maxCompactionMemoryInBytes
|
||||
+ ", mergeType='" + mergeType + '\''
|
||||
+ ", instantRange=" + instantRange
|
||||
+ '}';
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,117 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.format.mor;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
|
||||
import org.apache.flink.table.types.logical.LogicalType;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Statistics for merge on read table source.
|
||||
*/
|
||||
public class MergeOnReadTableState implements Serializable {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
private final RowType rowType;
|
||||
private final RowType requiredRowType;
|
||||
private final String avroSchema;
|
||||
private final String requiredAvroSchema;
|
||||
private final List<MergeOnReadInputSplit> inputSplits;
|
||||
private final String[] pkFields;
|
||||
private final int operationPos;
|
||||
|
||||
public MergeOnReadTableState(
|
||||
RowType rowType,
|
||||
RowType requiredRowType,
|
||||
String avroSchema,
|
||||
String requiredAvroSchema,
|
||||
List<MergeOnReadInputSplit> inputSplits,
|
||||
String[] pkFields) {
|
||||
this.rowType = rowType;
|
||||
this.requiredRowType = requiredRowType;
|
||||
this.avroSchema = avroSchema;
|
||||
this.requiredAvroSchema = requiredAvroSchema;
|
||||
this.inputSplits = inputSplits;
|
||||
this.pkFields = pkFields;
|
||||
this.operationPos = rowType.getFieldIndex(HoodieRecord.OPERATION_METADATA_FIELD);
|
||||
}
|
||||
|
||||
public RowType getRowType() {
|
||||
return rowType;
|
||||
}
|
||||
|
||||
public RowType getRequiredRowType() {
|
||||
return requiredRowType;
|
||||
}
|
||||
|
||||
public String getAvroSchema() {
|
||||
return avroSchema;
|
||||
}
|
||||
|
||||
public String getRequiredAvroSchema() {
|
||||
return requiredAvroSchema;
|
||||
}
|
||||
|
||||
public List<MergeOnReadInputSplit> getInputSplits() {
|
||||
return inputSplits;
|
||||
}
|
||||
|
||||
public int getOperationPos() {
|
||||
return operationPos;
|
||||
}
|
||||
|
||||
public int[] getRequiredPositions() {
|
||||
final List<String> fieldNames = rowType.getFieldNames();
|
||||
return requiredRowType.getFieldNames().stream()
|
||||
.map(fieldNames::indexOf)
|
||||
.mapToInt(i -> i)
|
||||
.toArray();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the primary key positions in required row type.
|
||||
*/
|
||||
public int[] getPkOffsetsInRequired() {
|
||||
final List<String> fieldNames = requiredRowType.getFieldNames();
|
||||
return Arrays.stream(pkFields)
|
||||
.map(fieldNames::indexOf)
|
||||
.mapToInt(i -> i)
|
||||
.toArray();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the primary key fields logical type with given offsets.
|
||||
*
|
||||
* @param pkOffsets the pk offsets in required row type
|
||||
* @return pk field logical types
|
||||
* @see #getPkOffsetsInRequired()
|
||||
*/
|
||||
public LogicalType[] getPkTypes(int[] pkOffsets) {
|
||||
final LogicalType[] requiredTypes = requiredRowType.getFields().stream()
|
||||
.map(RowType.RowField::getType).toArray(LogicalType[]::new);
|
||||
return Arrays.stream(pkOffsets).mapToObj(offset -> requiredTypes[offset])
|
||||
.toArray(LogicalType[]::new);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,328 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.util;
|
||||
|
||||
import org.apache.avro.LogicalTypes;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.SchemaBuilder;
|
||||
import org.apache.flink.api.common.typeinfo.Types;
|
||||
import org.apache.flink.table.api.DataTypes;
|
||||
import org.apache.flink.table.types.AtomicDataType;
|
||||
import org.apache.flink.table.types.DataType;
|
||||
import org.apache.flink.table.types.logical.ArrayType;
|
||||
import org.apache.flink.table.types.logical.DecimalType;
|
||||
import org.apache.flink.table.types.logical.IntType;
|
||||
import org.apache.flink.table.types.logical.LogicalType;
|
||||
import org.apache.flink.table.types.logical.LogicalTypeFamily;
|
||||
import org.apache.flink.table.types.logical.MapType;
|
||||
import org.apache.flink.table.types.logical.MultisetType;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
import org.apache.flink.table.types.logical.TimeType;
|
||||
import org.apache.flink.table.types.logical.TimestampType;
|
||||
import org.apache.flink.table.types.logical.TypeInformationRawType;
|
||||
import org.apache.flink.table.types.logical.utils.LogicalTypeChecks;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Converts an Avro schema into Flink's type information. It uses {@link org.apache.flink.api.java.typeutils.RowTypeInfo} for
|
||||
* representing objects and converts Avro types into types that are compatible with Flink's Table &
|
||||
* SQL API.
|
||||
*
|
||||
* <p>Note: Changes in this class need to be kept in sync with the corresponding runtime classes
|
||||
* {@code org.apache.flink.formats.avro.AvroRowDeserializationSchema} and {@code org.apache.flink.formats.avro.AvroRowSerializationSchema}.
|
||||
*
|
||||
* <p>NOTE: reference from Flink release 1.12.0, should remove when Flink version upgrade to that.
|
||||
*/
|
||||
public class AvroSchemaConverter {
|
||||
|
||||
/**
|
||||
* Converts an Avro schema {@code schema} into a nested row structure with deterministic field order and
|
||||
* data types that are compatible with Flink's Table & SQL API.
|
||||
*
|
||||
* @param schema Avro schema definition
|
||||
* @return data type matching the schema
|
||||
*/
|
||||
public static DataType convertToDataType(Schema schema) {
|
||||
switch (schema.getType()) {
|
||||
case RECORD:
|
||||
final List<Schema.Field> schemaFields = schema.getFields();
|
||||
|
||||
final DataTypes.Field[] fields = new DataTypes.Field[schemaFields.size()];
|
||||
for (int i = 0; i < schemaFields.size(); i++) {
|
||||
final Schema.Field field = schemaFields.get(i);
|
||||
fields[i] = DataTypes.FIELD(field.name(), convertToDataType(field.schema()));
|
||||
}
|
||||
return DataTypes.ROW(fields).notNull();
|
||||
case ENUM:
|
||||
return DataTypes.STRING().notNull();
|
||||
case ARRAY:
|
||||
return DataTypes.ARRAY(convertToDataType(schema.getElementType())).notNull();
|
||||
case MAP:
|
||||
return DataTypes.MAP(
|
||||
DataTypes.STRING().notNull(),
|
||||
convertToDataType(schema.getValueType()))
|
||||
.notNull();
|
||||
case UNION:
|
||||
final Schema actualSchema;
|
||||
final boolean nullable;
|
||||
if (schema.getTypes().size() == 2
|
||||
&& schema.getTypes().get(0).getType() == Schema.Type.NULL) {
|
||||
actualSchema = schema.getTypes().get(1);
|
||||
nullable = true;
|
||||
} else if (schema.getTypes().size() == 2
|
||||
&& schema.getTypes().get(1).getType() == Schema.Type.NULL) {
|
||||
actualSchema = schema.getTypes().get(0);
|
||||
nullable = true;
|
||||
} else if (schema.getTypes().size() == 1) {
|
||||
actualSchema = schema.getTypes().get(0);
|
||||
nullable = false;
|
||||
} else {
|
||||
// use Kryo for serialization
|
||||
return new AtomicDataType(
|
||||
new TypeInformationRawType<>(false, Types.GENERIC(Object.class)));
|
||||
}
|
||||
DataType converted = convertToDataType(actualSchema);
|
||||
return nullable ? converted.nullable() : converted;
|
||||
case FIXED:
|
||||
// logical decimal type
|
||||
if (schema.getLogicalType() instanceof LogicalTypes.Decimal) {
|
||||
final LogicalTypes.Decimal decimalType =
|
||||
(LogicalTypes.Decimal) schema.getLogicalType();
|
||||
return DataTypes.DECIMAL(decimalType.getPrecision(), decimalType.getScale())
|
||||
.notNull();
|
||||
}
|
||||
// convert fixed size binary data to primitive byte arrays
|
||||
return DataTypes.VARBINARY(schema.getFixedSize()).notNull();
|
||||
case STRING:
|
||||
// convert Avro's Utf8/CharSequence to String
|
||||
return DataTypes.STRING().notNull();
|
||||
case BYTES:
|
||||
// logical decimal type
|
||||
if (schema.getLogicalType() instanceof LogicalTypes.Decimal) {
|
||||
final LogicalTypes.Decimal decimalType =
|
||||
(LogicalTypes.Decimal) schema.getLogicalType();
|
||||
return DataTypes.DECIMAL(decimalType.getPrecision(), decimalType.getScale())
|
||||
.notNull();
|
||||
}
|
||||
return DataTypes.BYTES().notNull();
|
||||
case INT:
|
||||
// logical date and time type
|
||||
final org.apache.avro.LogicalType logicalType = schema.getLogicalType();
|
||||
if (logicalType == LogicalTypes.date()) {
|
||||
return DataTypes.DATE().notNull();
|
||||
} else if (logicalType == LogicalTypes.timeMillis()) {
|
||||
return DataTypes.TIME(3).notNull();
|
||||
}
|
||||
return DataTypes.INT().notNull();
|
||||
case LONG:
|
||||
// logical timestamp type
|
||||
if (schema.getLogicalType() == LogicalTypes.timestampMillis()) {
|
||||
return DataTypes.TIMESTAMP(3).notNull();
|
||||
} else if (schema.getLogicalType() == LogicalTypes.timestampMicros()) {
|
||||
return DataTypes.TIMESTAMP(6).notNull();
|
||||
} else if (schema.getLogicalType() == LogicalTypes.timeMillis()) {
|
||||
return DataTypes.TIME(3).notNull();
|
||||
} else if (schema.getLogicalType() == LogicalTypes.timeMicros()) {
|
||||
return DataTypes.TIME(6).notNull();
|
||||
}
|
||||
return DataTypes.BIGINT().notNull();
|
||||
case FLOAT:
|
||||
return DataTypes.FLOAT().notNull();
|
||||
case DOUBLE:
|
||||
return DataTypes.DOUBLE().notNull();
|
||||
case BOOLEAN:
|
||||
return DataTypes.BOOLEAN().notNull();
|
||||
case NULL:
|
||||
return DataTypes.NULL();
|
||||
default:
|
||||
throw new IllegalArgumentException("Unsupported Avro type '" + schema.getType() + "'.");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts Flink SQL {@link LogicalType} (can be nested) into an Avro schema.
|
||||
*
|
||||
* <p>Use "record" as the type name.
|
||||
*
|
||||
* @param schema the schema type, usually it should be the top level record type, e.g. not a
|
||||
* nested type
|
||||
* @return Avro's {@link Schema} matching this logical type.
|
||||
*/
|
||||
public static Schema convertToSchema(LogicalType schema) {
|
||||
return convertToSchema(schema, "record");
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts Flink SQL {@link LogicalType} (can be nested) into an Avro schema.
|
||||
*
|
||||
* <p>The "{rowName}_" is used as the nested row type name prefix in order to generate the right
|
||||
* schema. Nested record type that only differs with type name is still compatible.
|
||||
*
|
||||
* @param logicalType logical type
|
||||
* @param rowName the record name
|
||||
* @return Avro's {@link Schema} matching this logical type.
|
||||
*/
|
||||
public static Schema convertToSchema(LogicalType logicalType, String rowName) {
|
||||
int precision;
|
||||
boolean nullable = logicalType.isNullable();
|
||||
switch (logicalType.getTypeRoot()) {
|
||||
case NULL:
|
||||
return SchemaBuilder.builder().nullType();
|
||||
case BOOLEAN:
|
||||
Schema bool = SchemaBuilder.builder().booleanType();
|
||||
return nullable ? nullableSchema(bool) : bool;
|
||||
case TINYINT:
|
||||
case SMALLINT:
|
||||
case INTEGER:
|
||||
Schema integer = SchemaBuilder.builder().intType();
|
||||
return nullable ? nullableSchema(integer) : integer;
|
||||
case BIGINT:
|
||||
Schema bigint = SchemaBuilder.builder().longType();
|
||||
return nullable ? nullableSchema(bigint) : bigint;
|
||||
case FLOAT:
|
||||
Schema f = SchemaBuilder.builder().floatType();
|
||||
return nullable ? nullableSchema(f) : f;
|
||||
case DOUBLE:
|
||||
Schema d = SchemaBuilder.builder().doubleType();
|
||||
return nullable ? nullableSchema(d) : d;
|
||||
case CHAR:
|
||||
case VARCHAR:
|
||||
Schema str = SchemaBuilder.builder().stringType();
|
||||
return nullable ? nullableSchema(str) : str;
|
||||
case BINARY:
|
||||
case VARBINARY:
|
||||
Schema binary = SchemaBuilder.builder().bytesType();
|
||||
return nullable ? nullableSchema(binary) : binary;
|
||||
case TIMESTAMP_WITHOUT_TIME_ZONE:
|
||||
// use long to represents Timestamp
|
||||
final TimestampType timestampType = (TimestampType) logicalType;
|
||||
precision = timestampType.getPrecision();
|
||||
org.apache.avro.LogicalType avroLogicalType;
|
||||
if (precision <= 3) {
|
||||
avroLogicalType = LogicalTypes.timestampMillis();
|
||||
} else if (precision <= 6) {
|
||||
avroLogicalType = LogicalTypes.timestampMicros();
|
||||
} else {
|
||||
throw new IllegalArgumentException(
|
||||
"Avro does not support TIMESTAMP type with precision: "
|
||||
+ precision
|
||||
+ ", it only supports precision less than 6.");
|
||||
}
|
||||
Schema timestamp = avroLogicalType.addToSchema(SchemaBuilder.builder().longType());
|
||||
return nullable ? nullableSchema(timestamp) : timestamp;
|
||||
case DATE:
|
||||
// use int to represents Date
|
||||
Schema date = LogicalTypes.date().addToSchema(SchemaBuilder.builder().intType());
|
||||
return nullable ? nullableSchema(date) : date;
|
||||
case TIME_WITHOUT_TIME_ZONE:
|
||||
precision = ((TimeType) logicalType).getPrecision();
|
||||
if (precision > 3) {
|
||||
throw new IllegalArgumentException(
|
||||
"Avro does not support TIME type with precision: "
|
||||
+ precision
|
||||
+ ", it only supports precision less than 3.");
|
||||
}
|
||||
// use int to represents Time, we only support millisecond when deserialization
|
||||
Schema time =
|
||||
LogicalTypes.timeMillis().addToSchema(SchemaBuilder.builder().intType());
|
||||
return nullable ? nullableSchema(time) : time;
|
||||
case DECIMAL:
|
||||
DecimalType decimalType = (DecimalType) logicalType;
|
||||
// store BigDecimal as byte[]
|
||||
Schema decimal =
|
||||
LogicalTypes.decimal(decimalType.getPrecision(), decimalType.getScale())
|
||||
.addToSchema(SchemaBuilder.builder().bytesType());
|
||||
return nullable ? nullableSchema(decimal) : decimal;
|
||||
case ROW:
|
||||
RowType rowType = (RowType) logicalType;
|
||||
List<String> fieldNames = rowType.getFieldNames();
|
||||
// we have to make sure the record name is different in a Schema
|
||||
SchemaBuilder.FieldAssembler<Schema> builder =
|
||||
SchemaBuilder.builder().record(rowName).fields();
|
||||
for (int i = 0; i < rowType.getFieldCount(); i++) {
|
||||
String fieldName = fieldNames.get(i);
|
||||
LogicalType fieldType = rowType.getTypeAt(i);
|
||||
SchemaBuilder.GenericDefault<Schema> fieldBuilder =
|
||||
builder.name(fieldName)
|
||||
.type(convertToSchema(fieldType, rowName + "_" + fieldName));
|
||||
|
||||
if (fieldType.isNullable()) {
|
||||
builder = fieldBuilder.withDefault(null);
|
||||
} else {
|
||||
builder = fieldBuilder.noDefault();
|
||||
}
|
||||
}
|
||||
Schema record = builder.endRecord();
|
||||
return nullable ? nullableSchema(record) : record;
|
||||
case MULTISET:
|
||||
case MAP:
|
||||
Schema map =
|
||||
SchemaBuilder.builder()
|
||||
.map()
|
||||
.values(
|
||||
convertToSchema(
|
||||
extractValueTypeToAvroMap(logicalType), rowName));
|
||||
return nullable ? nullableSchema(map) : map;
|
||||
case ARRAY:
|
||||
ArrayType arrayType = (ArrayType) logicalType;
|
||||
Schema array =
|
||||
SchemaBuilder.builder()
|
||||
.array()
|
||||
.items(convertToSchema(arrayType.getElementType(), rowName));
|
||||
return nullable ? nullableSchema(array) : array;
|
||||
case RAW:
|
||||
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
|
||||
default:
|
||||
throw new UnsupportedOperationException(
|
||||
"Unsupported to derive Schema for type: " + logicalType);
|
||||
}
|
||||
}
|
||||
|
||||
public static LogicalType extractValueTypeToAvroMap(LogicalType type) {
|
||||
LogicalType keyType;
|
||||
LogicalType valueType;
|
||||
if (type instanceof MapType) {
|
||||
MapType mapType = (MapType) type;
|
||||
keyType = mapType.getKeyType();
|
||||
valueType = mapType.getValueType();
|
||||
} else {
|
||||
MultisetType multisetType = (MultisetType) type;
|
||||
keyType = multisetType.getElementType();
|
||||
valueType = new IntType();
|
||||
}
|
||||
if (!LogicalTypeChecks.hasFamily(keyType, LogicalTypeFamily.CHARACTER_STRING)) {
|
||||
throw new UnsupportedOperationException(
|
||||
"Avro format doesn't support non-string as key type of map. "
|
||||
+ "The key type is: "
|
||||
+ keyType.asSummaryString());
|
||||
}
|
||||
return valueType;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns schema with nullable true.
|
||||
*/
|
||||
private static Schema nullableSchema(Schema schema) {
|
||||
return schema.isNullable()
|
||||
? schema
|
||||
: Schema.createUnion(SchemaBuilder.builder().nullType(), schema);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,327 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.util;
|
||||
|
||||
import org.apache.avro.generic.GenericFixed;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.flink.annotation.Internal;
|
||||
import org.apache.flink.table.api.DataTypes;
|
||||
import org.apache.flink.table.data.DecimalData;
|
||||
import org.apache.flink.table.data.GenericArrayData;
|
||||
import org.apache.flink.table.data.GenericMapData;
|
||||
import org.apache.flink.table.data.GenericRowData;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.data.StringData;
|
||||
import org.apache.flink.table.data.TimestampData;
|
||||
import org.apache.flink.table.types.logical.ArrayType;
|
||||
import org.apache.flink.table.types.logical.DecimalType;
|
||||
import org.apache.flink.table.types.logical.LogicalType;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
import org.apache.flink.table.types.logical.TimestampType;
|
||||
import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeFieldType;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.lang.reflect.Array;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.time.Instant;
|
||||
import java.time.LocalDate;
|
||||
import java.time.LocalTime;
|
||||
import java.time.temporal.ChronoField;
|
||||
import java.time.temporal.ChronoUnit;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Tool class used to convert from Avro {@link GenericRecord} to {@link RowData}.
|
||||
*
|
||||
* <p>NOTE: reference from Flink release 1.12.0, should remove when Flink version upgrade to that.
|
||||
*/
|
||||
@Internal
|
||||
public class AvroToRowDataConverters {
|
||||
|
||||
/**
|
||||
* Runtime converter that converts Avro data structures into objects of Flink Table & SQL
|
||||
* internal data structures.
|
||||
*/
|
||||
@FunctionalInterface
|
||||
public interface AvroToRowDataConverter extends Serializable {
|
||||
Object convert(Object object);
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------------------
|
||||
// Runtime Converters
|
||||
// -------------------------------------------------------------------------------------
|
||||
|
||||
public static AvroToRowDataConverter createRowConverter(RowType rowType) {
|
||||
final AvroToRowDataConverter[] fieldConverters =
|
||||
rowType.getFields().stream()
|
||||
.map(RowType.RowField::getType)
|
||||
.map(AvroToRowDataConverters::createNullableConverter)
|
||||
.toArray(AvroToRowDataConverter[]::new);
|
||||
final int arity = rowType.getFieldCount();
|
||||
|
||||
return avroObject -> {
|
||||
IndexedRecord record = (IndexedRecord) avroObject;
|
||||
GenericRowData row = new GenericRowData(arity);
|
||||
for (int i = 0; i < arity; ++i) {
|
||||
row.setField(i, fieldConverters[i].convert(record.get(i)));
|
||||
}
|
||||
return row;
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a runtime converter which is null safe.
|
||||
*/
|
||||
private static AvroToRowDataConverter createNullableConverter(LogicalType type) {
|
||||
final AvroToRowDataConverter converter = createConverter(type);
|
||||
return avroObject -> {
|
||||
if (avroObject == null) {
|
||||
return null;
|
||||
}
|
||||
return converter.convert(avroObject);
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a runtime converter which assuming input object is not null.
|
||||
*/
|
||||
private static AvroToRowDataConverter createConverter(LogicalType type) {
|
||||
switch (type.getTypeRoot()) {
|
||||
case NULL:
|
||||
return avroObject -> null;
|
||||
case TINYINT:
|
||||
return avroObject -> ((Integer) avroObject).byteValue();
|
||||
case SMALLINT:
|
||||
return avroObject -> ((Integer) avroObject).shortValue();
|
||||
case BOOLEAN: // boolean
|
||||
case INTEGER: // int
|
||||
case INTERVAL_YEAR_MONTH: // long
|
||||
case BIGINT: // long
|
||||
case INTERVAL_DAY_TIME: // long
|
||||
case FLOAT: // float
|
||||
case DOUBLE: // double
|
||||
return avroObject -> avroObject;
|
||||
case DATE:
|
||||
return AvroToRowDataConverters::convertToDate;
|
||||
case TIME_WITHOUT_TIME_ZONE:
|
||||
return AvroToRowDataConverters::convertToTime;
|
||||
case TIMESTAMP_WITHOUT_TIME_ZONE:
|
||||
return createTimestampConverter(((TimestampType) type).getPrecision());
|
||||
case CHAR:
|
||||
case VARCHAR:
|
||||
return avroObject -> StringData.fromString(avroObject.toString());
|
||||
case BINARY:
|
||||
case VARBINARY:
|
||||
return AvroToRowDataConverters::convertToBytes;
|
||||
case DECIMAL:
|
||||
return createDecimalConverter((DecimalType) type);
|
||||
case ARRAY:
|
||||
return createArrayConverter((ArrayType) type);
|
||||
case ROW:
|
||||
return createRowConverter((RowType) type);
|
||||
case MAP:
|
||||
case MULTISET:
|
||||
return createMapConverter(type);
|
||||
case RAW:
|
||||
default:
|
||||
throw new UnsupportedOperationException("Unsupported type: " + type);
|
||||
}
|
||||
}
|
||||
|
||||
private static AvroToRowDataConverter createDecimalConverter(DecimalType decimalType) {
|
||||
final int precision = decimalType.getPrecision();
|
||||
final int scale = decimalType.getScale();
|
||||
return avroObject -> {
|
||||
final byte[] bytes;
|
||||
if (avroObject instanceof GenericFixed) {
|
||||
bytes = ((GenericFixed) avroObject).bytes();
|
||||
} else if (avroObject instanceof ByteBuffer) {
|
||||
ByteBuffer byteBuffer = (ByteBuffer) avroObject;
|
||||
bytes = new byte[byteBuffer.remaining()];
|
||||
byteBuffer.get(bytes);
|
||||
} else {
|
||||
bytes = (byte[]) avroObject;
|
||||
}
|
||||
return DecimalData.fromUnscaledBytes(bytes, precision, scale);
|
||||
};
|
||||
}
|
||||
|
||||
private static AvroToRowDataConverter createArrayConverter(ArrayType arrayType) {
|
||||
final AvroToRowDataConverter elementConverter =
|
||||
createNullableConverter(arrayType.getElementType());
|
||||
final Class<?> elementClass =
|
||||
LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType());
|
||||
|
||||
return avroObject -> {
|
||||
final List<?> list = (List<?>) avroObject;
|
||||
final int length = list.size();
|
||||
final Object[] array = (Object[]) Array.newInstance(elementClass, length);
|
||||
for (int i = 0; i < length; ++i) {
|
||||
array[i] = elementConverter.convert(list.get(i));
|
||||
}
|
||||
return new GenericArrayData(array);
|
||||
};
|
||||
}
|
||||
|
||||
private static AvroToRowDataConverter createMapConverter(LogicalType type) {
|
||||
final AvroToRowDataConverter keyConverter =
|
||||
createConverter(DataTypes.STRING().getLogicalType());
|
||||
final AvroToRowDataConverter valueConverter =
|
||||
createNullableConverter(AvroSchemaConverter.extractValueTypeToAvroMap(type));
|
||||
|
||||
return avroObject -> {
|
||||
final Map<?, ?> map = (Map<?, ?>) avroObject;
|
||||
Map<Object, Object> result = new HashMap<>();
|
||||
for (Map.Entry<?, ?> entry : map.entrySet()) {
|
||||
Object key = keyConverter.convert(entry.getKey());
|
||||
Object value = valueConverter.convert(entry.getValue());
|
||||
result.put(key, value);
|
||||
}
|
||||
return new GenericMapData(result);
|
||||
};
|
||||
}
|
||||
|
||||
private static AvroToRowDataConverter createTimestampConverter(int precision) {
|
||||
final ChronoUnit chronoUnit;
|
||||
if (precision <= 3) {
|
||||
chronoUnit = ChronoUnit.MILLIS;
|
||||
} else if (precision <= 6) {
|
||||
chronoUnit = ChronoUnit.MICROS;
|
||||
} else {
|
||||
throw new IllegalArgumentException(
|
||||
"Avro does not support TIMESTAMP type with precision: "
|
||||
+ precision
|
||||
+ ", it only supports precision less than 6.");
|
||||
}
|
||||
return avroObject -> {
|
||||
final Instant instant;
|
||||
if (avroObject instanceof Long) {
|
||||
instant = Instant.EPOCH.plus((Long) avroObject, chronoUnit);
|
||||
} else if (avroObject instanceof Instant) {
|
||||
instant = (Instant) avroObject;
|
||||
} else {
|
||||
JodaConverter jodaConverter = JodaConverter.getConverter();
|
||||
if (jodaConverter != null) {
|
||||
// joda time has only millisecond precision
|
||||
instant = Instant.ofEpochMilli(jodaConverter.convertTimestamp(avroObject));
|
||||
} else {
|
||||
throw new IllegalArgumentException(
|
||||
"Unexpected object type for TIMESTAMP logical type. Received: " + avroObject);
|
||||
}
|
||||
}
|
||||
return TimestampData.fromInstant(instant);
|
||||
};
|
||||
}
|
||||
|
||||
private static int convertToDate(Object object) {
|
||||
if (object instanceof Integer) {
|
||||
return (Integer) object;
|
||||
} else if (object instanceof LocalDate) {
|
||||
return (int) ((LocalDate) object).toEpochDay();
|
||||
} else {
|
||||
JodaConverter jodaConverter = JodaConverter.getConverter();
|
||||
if (jodaConverter != null) {
|
||||
return (int) jodaConverter.convertDate(object);
|
||||
} else {
|
||||
throw new IllegalArgumentException(
|
||||
"Unexpected object type for DATE logical type. Received: " + object);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static int convertToTime(Object object) {
|
||||
final int millis;
|
||||
if (object instanceof Integer) {
|
||||
millis = (Integer) object;
|
||||
} else if (object instanceof LocalTime) {
|
||||
millis = ((LocalTime) object).get(ChronoField.MILLI_OF_DAY);
|
||||
} else {
|
||||
JodaConverter jodaConverter = JodaConverter.getConverter();
|
||||
if (jodaConverter != null) {
|
||||
millis = jodaConverter.convertTime(object);
|
||||
} else {
|
||||
throw new IllegalArgumentException(
|
||||
"Unexpected object type for TIME logical type. Received: " + object);
|
||||
}
|
||||
}
|
||||
return millis;
|
||||
}
|
||||
|
||||
private static byte[] convertToBytes(Object object) {
|
||||
if (object instanceof GenericFixed) {
|
||||
return ((GenericFixed) object).bytes();
|
||||
} else if (object instanceof ByteBuffer) {
|
||||
ByteBuffer byteBuffer = (ByteBuffer) object;
|
||||
byte[] bytes = new byte[byteBuffer.remaining()];
|
||||
byteBuffer.get(bytes);
|
||||
return bytes;
|
||||
} else {
|
||||
return (byte[]) object;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Encapsulates joda optional dependency. Instantiates this class only if joda is available on the
|
||||
* classpath.
|
||||
*/
|
||||
static class JodaConverter {
|
||||
|
||||
private static JodaConverter instance;
|
||||
|
||||
public static JodaConverter getConverter() {
|
||||
if (instance != null) {
|
||||
return instance;
|
||||
}
|
||||
|
||||
try {
|
||||
Class.forName(
|
||||
"org.joda.time.DateTime",
|
||||
false,
|
||||
Thread.currentThread().getContextClassLoader());
|
||||
instance = new JodaConverter();
|
||||
} catch (ClassNotFoundException e) {
|
||||
instance = null;
|
||||
}
|
||||
return instance;
|
||||
}
|
||||
|
||||
public long convertDate(Object object) {
|
||||
final org.joda.time.LocalDate value = (org.joda.time.LocalDate) object;
|
||||
return value.toDate().getTime();
|
||||
}
|
||||
|
||||
public int convertTime(Object object) {
|
||||
final org.joda.time.LocalTime value = (org.joda.time.LocalTime) object;
|
||||
return value.get(DateTimeFieldType.millisOfDay());
|
||||
}
|
||||
|
||||
public long convertTimestamp(Object object) {
|
||||
final DateTime value = (DateTime) object;
|
||||
return value.toDate().getTime();
|
||||
}
|
||||
|
||||
private JodaConverter() {
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,46 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.util;
|
||||
|
||||
import org.apache.flink.table.connector.ChangelogMode;
|
||||
import org.apache.flink.types.RowKind;
|
||||
|
||||
/**
|
||||
* Utilities for all kinds of common {@link org.apache.flink.table.connector.ChangelogMode}s.
|
||||
*/
|
||||
public class ChangelogModes {
|
||||
public static final ChangelogMode FULL = ChangelogMode.newBuilder()
|
||||
.addContainedKind(RowKind.INSERT)
|
||||
.addContainedKind(RowKind.UPDATE_BEFORE)
|
||||
.addContainedKind(RowKind.UPDATE_AFTER)
|
||||
.addContainedKind(RowKind.DELETE)
|
||||
.build();
|
||||
|
||||
/**
|
||||
* Change log mode that ignores UPDATE_BEFORE, e.g UPSERT.
|
||||
*/
|
||||
public static final ChangelogMode UPSERT = ChangelogMode.newBuilder()
|
||||
.addContainedKind(RowKind.INSERT)
|
||||
.addContainedKind(RowKind.UPDATE_AFTER)
|
||||
.addContainedKind(RowKind.DELETE)
|
||||
.build();
|
||||
|
||||
private ChangelogModes() {
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,211 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.util;
|
||||
|
||||
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.TableSchemaResolver;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.sink.compact.FlinkCompactionConfig;
|
||||
import org.apache.hudi.table.HoodieFlinkTable;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Locale;
|
||||
|
||||
/**
|
||||
* Utilities for flink hudi compaction.
|
||||
*/
|
||||
public class CompactionUtil {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(CompactionUtil.class);
|
||||
|
||||
/**
|
||||
* Schedules a new compaction instant.
|
||||
*
|
||||
* @param metaClient The metadata client
|
||||
* @param writeClient The write client
|
||||
* @param deltaTimeCompaction Whether the compaction is trigger by elapsed delta time
|
||||
* @param committed Whether the last instant was committed successfully
|
||||
*/
|
||||
public static void scheduleCompaction(
|
||||
HoodieTableMetaClient metaClient,
|
||||
HoodieFlinkWriteClient<?> writeClient,
|
||||
boolean deltaTimeCompaction,
|
||||
boolean committed) {
|
||||
if (committed) {
|
||||
writeClient.scheduleCompaction(Option.empty());
|
||||
} else if (deltaTimeCompaction) {
|
||||
// if there are no new commits and the compaction trigger strategy is based on elapsed delta time,
|
||||
// schedules the compaction anyway.
|
||||
metaClient.reloadActiveTimeline();
|
||||
Option<String> compactionInstantTime = CompactionUtil.getCompactionInstantTime(metaClient);
|
||||
if (compactionInstantTime.isPresent()) {
|
||||
writeClient.scheduleCompactionAtInstant(compactionInstantTime.get(), Option.empty());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets compaction Instant time.
|
||||
*/
|
||||
public static Option<String> getCompactionInstantTime(HoodieTableMetaClient metaClient) {
|
||||
Option<HoodieInstant> firstPendingInstant = metaClient.getCommitsTimeline()
|
||||
.filterPendingExcludingCompaction().firstInstant();
|
||||
Option<HoodieInstant> lastCompleteInstant = metaClient.getActiveTimeline().getWriteTimeline()
|
||||
.filterCompletedAndCompactionInstants().lastInstant();
|
||||
if (firstPendingInstant.isPresent() && lastCompleteInstant.isPresent()) {
|
||||
String firstPendingTimestamp = firstPendingInstant.get().getTimestamp();
|
||||
String lastCompleteTimestamp = lastCompleteInstant.get().getTimestamp();
|
||||
// Committed and pending compaction instants should have strictly lower timestamps
|
||||
return StreamerUtil.medianInstantTime(firstPendingTimestamp, lastCompleteTimestamp);
|
||||
} else if (!lastCompleteInstant.isPresent()) {
|
||||
LOG.info("No instants to schedule the compaction plan");
|
||||
return Option.empty();
|
||||
} else {
|
||||
return Option.of(HoodieActiveTimeline.createNewInstantTime());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets up the avro schema string into the give configuration {@code conf}
|
||||
* through reading from the hoodie table metadata.
|
||||
*
|
||||
* @param conf The configuration
|
||||
*/
|
||||
public static void setAvroSchema(Configuration conf, HoodieTableMetaClient metaClient) throws Exception {
|
||||
TableSchemaResolver tableSchemaResolver = new TableSchemaResolver(metaClient);
|
||||
Schema tableAvroSchema = tableSchemaResolver.getTableAvroSchema(false);
|
||||
conf.setString(FlinkOptions.SOURCE_AVRO_SCHEMA, tableAvroSchema.toString());
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets up the avro schema string into the HoodieWriteConfig {@code HoodieWriteConfig}
|
||||
* through reading from the hoodie table metadata.
|
||||
*
|
||||
* @param writeConfig The HoodieWriteConfig
|
||||
*/
|
||||
public static void setAvroSchema(HoodieWriteConfig writeConfig, HoodieTableMetaClient metaClient) throws Exception {
|
||||
TableSchemaResolver tableSchemaResolver = new TableSchemaResolver(metaClient);
|
||||
Schema tableAvroSchema = tableSchemaResolver.getTableAvroSchema(false);
|
||||
writeConfig.setSchema(tableAvroSchema.toString());
|
||||
}
|
||||
|
||||
/**
|
||||
* Infers the changelog mode based on the data file schema(including metadata fields).
|
||||
*
|
||||
* <p>We can improve the code if the changelog mode is set up as table config.
|
||||
*
|
||||
* @param conf The configuration
|
||||
*/
|
||||
public static void inferChangelogMode(Configuration conf, HoodieTableMetaClient metaClient) throws Exception {
|
||||
TableSchemaResolver tableSchemaResolver = new TableSchemaResolver(metaClient);
|
||||
Schema tableAvroSchema = tableSchemaResolver.getTableAvroSchemaFromDataFile();
|
||||
if (tableAvroSchema.getField(HoodieRecord.OPERATION_METADATA_FIELD) != null) {
|
||||
conf.setBoolean(FlinkOptions.CHANGELOG_ENABLED, true);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Cleans the metadata file for given instant {@code instant}.
|
||||
*/
|
||||
public static void cleanInstant(HoodieTableMetaClient metaClient, HoodieInstant instant) {
|
||||
Path commitFilePath = new Path(metaClient.getMetaAuxiliaryPath(), instant.getFileName());
|
||||
try {
|
||||
if (metaClient.getFs().exists(commitFilePath)) {
|
||||
boolean deleted = metaClient.getFs().delete(commitFilePath, false);
|
||||
if (deleted) {
|
||||
LOG.info("Removed instant " + instant);
|
||||
} else {
|
||||
throw new HoodieIOException("Could not delete instant " + instant);
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not remove requested commit " + commitFilePath, e);
|
||||
}
|
||||
}
|
||||
|
||||
public static void rollbackCompaction(HoodieFlinkTable<?> table, String instantTime) {
|
||||
HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(instantTime);
|
||||
if (table.getMetaClient().reloadActiveTimeline().filterPendingCompactionTimeline().containsInstant(inflightInstant)) {
|
||||
LOG.warn("Rollback failed compaction instant: [" + instantTime + "]");
|
||||
table.rollbackInflightCompaction(inflightInstant);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Force rolls back all the inflight compaction instants, especially for job failover restart.
|
||||
*
|
||||
* @param table The hoodie table
|
||||
*/
|
||||
public static void rollbackCompaction(HoodieFlinkTable<?> table) {
|
||||
HoodieTimeline inflightCompactionTimeline = table.getActiveTimeline()
|
||||
.filterPendingCompactionTimeline()
|
||||
.filter(instant ->
|
||||
instant.getState() == HoodieInstant.State.INFLIGHT);
|
||||
inflightCompactionTimeline.getInstants().forEach(inflightInstant -> {
|
||||
LOG.info("Rollback the inflight compaction instant: " + inflightInstant + " for failover");
|
||||
table.rollbackInflightCompaction(inflightInstant);
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Rolls back the earliest compaction if there exists.
|
||||
*
|
||||
* <p>Makes the strategy not that radical: firstly check whether there exists inflight compaction instants,
|
||||
* rolls back the first inflight instant only if it has timed out. That means, if there are
|
||||
* multiple timed out instants on the timeline, we only roll back the first one at a time.
|
||||
*/
|
||||
public static void rollbackEarliestCompaction(HoodieFlinkTable<?> table, Configuration conf) {
|
||||
Option<HoodieInstant> earliestInflight = table.getActiveTimeline()
|
||||
.filterPendingCompactionTimeline()
|
||||
.filter(instant ->
|
||||
instant.getState() == HoodieInstant.State.INFLIGHT).firstInstant();
|
||||
if (earliestInflight.isPresent()) {
|
||||
HoodieInstant instant = earliestInflight.get();
|
||||
String currentTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
int timeout = conf.getInteger(FlinkOptions.COMPACTION_TIMEOUT_SECONDS);
|
||||
if (StreamerUtil.instantTimeDiffSeconds(currentTime, instant.getTimestamp()) >= timeout) {
|
||||
LOG.info("Rollback the inflight compaction instant: " + instant + " for timeout(" + timeout + "s)");
|
||||
table.rollbackInflightCompaction(instant);
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether the execution sequence is LIFO.
|
||||
*/
|
||||
public static boolean isLIFO(String seq) {
|
||||
return seq.toUpperCase(Locale.ROOT).equals(FlinkCompactionConfig.SEQ_LIFO);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,61 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.util;
|
||||
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
|
||||
import org.apache.flink.table.types.DataType;
|
||||
import org.apache.flink.table.types.logical.LogicalType;
|
||||
import org.apache.flink.table.types.logical.LogicalTypeRoot;
|
||||
import org.apache.flink.table.types.logical.TimestampType;
|
||||
|
||||
/**
|
||||
* Utilities for {@link org.apache.flink.table.types.DataType}.
|
||||
*/
|
||||
public class DataTypeUtils {
|
||||
/**
|
||||
* Returns whether the given type is TIMESTAMP type.
|
||||
*/
|
||||
public static boolean isTimestampType(DataType type) {
|
||||
return type.getLogicalType().getTypeRoot() == LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the precision of the given TIMESTAMP type.
|
||||
*/
|
||||
public static int precision(LogicalType logicalType) {
|
||||
ValidationUtils.checkArgument(logicalType instanceof TimestampType);
|
||||
TimestampType timestampType = (TimestampType) logicalType;
|
||||
return timestampType.getPrecision();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether the given type is DATE type.
|
||||
*/
|
||||
public static boolean isDateType(DataType type) {
|
||||
return type.getLogicalType().getTypeRoot() == LogicalTypeRoot.DATE;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether the given type is DATETIME type.
|
||||
*/
|
||||
public static boolean isDatetimeType(DataType type) {
|
||||
return isTimestampType(type) || isDateType(type);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,41 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.util;
|
||||
|
||||
import com.beust.jcommander.IStringConverter;
|
||||
import com.beust.jcommander.ParameterException;
|
||||
import org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend;
|
||||
import org.apache.flink.runtime.state.StateBackend;
|
||||
import org.apache.flink.runtime.state.hashmap.HashMapStateBackend;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
/**
|
||||
* Converter that converts a string into Flink StateBackend.
|
||||
*/
|
||||
public class FlinkStateBackendConverter implements IStringConverter<StateBackend> {
|
||||
@Override
|
||||
public StateBackend convert(String value) throws ParameterException {
|
||||
switch (value) {
|
||||
case "hashmap" : return new HashMapStateBackend();
|
||||
case "rocksdb" : return new EmbeddedRocksDBStateBackend();
|
||||
default:
|
||||
throw new HoodieException(String.format("Unknown flink state backend %s.", value));
|
||||
}
|
||||
}
|
||||
}
|
||||
Some files were not shown because too many files have changed in this diff Show More
Reference in New Issue
Block a user