Class IcebergSink.Builder
- Enclosing class:
IcebergSink
-
Method Summary
Modifier and TypeMethodDescriptionorg.apache.flink.streaming.api.datastream.DataStreamSink<org.apache.flink.table.data.RowData> append()Append the iceberg sink operators to write records to iceberg table.Configure the writeDistributionModethat the IcebergSink will use.equalityFieldColumns(List<String> columns) Configuring the equality field columns for iceberg table that accept CDC or UPSERT events.flinkConf(org.apache.flink.configuration.ReadableConfig config) overwrite(boolean newOverwrite) rangeDistributionSortKeyBaseWeight(double weight) If sort order contains partition columns, each sort key would map to one partition and data file.Range distribution needs to collect statistics about data distribution to properly shuffle the records in relatively balanced way.resolvedSchema(org.apache.flink.table.catalog.ResolvedSchema newResolvedSchema) Set the write properties for IcebergSink.Set the write properties for IcebergSink.setSnapshotProperty(String property, String value) snapshotProperties(Map<String, String> properties) This icebergSerializableTableinstance is used for initializingIcebergStreamWriterwhich will write all the records intoDataFiles and emit them to downstream operator.tableLoader(TableLoader newTableLoader) The table loader is used for loading tables inIcebergCommitterlazily, we need this loader becauseTableis not serializable and could not just use the loaded table from Builder#table in the remote task manager.tableSchema(org.apache.flink.table.legacy.api.TableSchema newTableSchema) Set the uid suffix for IcebergSink operators.upsert(boolean enabled) All INSERT/UPDATE_AFTER events from input stream will be transformed to UPSERT events, which means it will DELETE the old records and then INSERT the new records.writeParallelism(int newWriteParallelism) Configuring the write parallel number for iceberg stream writer.
-
Method Details
-
table
This icebergSerializableTableinstance is used for initializingIcebergStreamWriterwhich will write all the records intoDataFiles and emit them to downstream operator. Providing a table would avoid so many table loading from each separate task.- Parameters:
newTable- the loaded iceberg table instance.- Returns:
IcebergSink.Builderto connect the iceberg table.
-
tableLoader
The table loader is used for loading tables inIcebergCommitterlazily, we need this loader becauseTableis not serializable and could not just use the loaded table from Builder#table in the remote task manager.- Parameters:
newTableLoader- to load iceberg table inside tasks.- Returns:
IcebergSink.Builderto connect the iceberg table.
-
set
Set the write properties for IcebergSink. View the supported properties inFlinkWriteOptions -
setAll
Set the write properties for IcebergSink. View the supported properties inFlinkWriteOptions -
tableSchema
public IcebergSink.Builder tableSchema(org.apache.flink.table.legacy.api.TableSchema newTableSchema) -
resolvedSchema
public IcebergSink.Builder resolvedSchema(org.apache.flink.table.catalog.ResolvedSchema newResolvedSchema) -
overwrite
-
flinkConf
-
distributionMode
Configure the writeDistributionModethat the IcebergSink will use. Currently, flink supportDistributionMode.NONEandDistributionMode.HASHandDistributionMode.RANGE- Parameters:
mode- to specify the write distribution mode.- Returns:
IcebergSink.Builderto connect the iceberg table.
-
rangeDistributionStatisticsType
Range distribution needs to collect statistics about data distribution to properly shuffle the records in relatively balanced way. In general, low cardinality should useStatisticsType.Mapand high cardinality should useStatisticsType.SketchRefer toStatisticsTypeJavadoc for more details.Default is
StatisticsType.Autowhere initially Map statistics is used. But if cardinality is higher than the threshold (currently 10K) as defined inSketchUtil#OPERATOR_SKETCH_SWITCH_THRESHOLD, statistics collection automatically switches to the sketch reservoir sampling.Explicit set the statistics type if the default behavior doesn't work.
- Parameters:
type- to specify the statistics type for range distribution.- Returns:
IcebergSink.Builderto connect the iceberg table.
-
rangeDistributionSortKeyBaseWeight
If sort order contains partition columns, each sort key would map to one partition and data file. This relative weight can avoid placing too many small files for sort keys with low traffic. It is a double value that defines the minimal weight for each sort key. `0.02` means each key has a base weight of `2%` of the targeted traffic weight per writer task.E.g. the sink Iceberg table is partitioned daily by event time. Assume the data stream contains events from now up to 180 days ago. With event time, traffic weight distribution across different days typically has a long tail pattern. Current day contains the most traffic. The older days (long tail) contain less and less traffic. Assume writer parallelism is `10`. The total weight across all 180 days is `10,000`. Target traffic weight per writer task would be `1,000`. Assume the weight sum for the oldest 150 days is `1,000`. Normally, the range partitioner would put all the oldest 150 days in one writer task. That writer task would write to 150 small files (one per day). Keeping 150 open files can potentially consume large amount of memory. Flushing and uploading 150 files (however small) at checkpoint time can also be potentially slow. If this config is set to `0.02`. It means every sort key has a base weight of `2%` of targeted weight of `1,000` for every write task. It would essentially avoid placing more than `50` data files (one per day) on one writer task no matter how small they are.
This is only applicable to
StatisticsType.Mapfor low-cardinality scenario. ForStatisticsType.Sketchhigh-cardinality sort columns, they are usually not used as partition columns. Otherwise, too many partitions and small files may be generated during write. Sketch range partitioner simply splits high-cardinality keys into ordered ranges.Default is
0.0%. -
writeParallelism
Configuring the write parallel number for iceberg stream writer.- Parameters:
newWriteParallelism- the number of parallel iceberg stream writer.- Returns:
IcebergSink.Builderto connect the iceberg table.
-
upsert
All INSERT/UPDATE_AFTER events from input stream will be transformed to UPSERT events, which means it will DELETE the old records and then INSERT the new records. In partitioned table, the partition fields should be a subset of equality fields, otherwise the old row that located in partition-A could not be deleted by the new row that located in partition-B.- Parameters:
enabled- indicate whether it should transform all INSERT/UPDATE_AFTER events to UPSERT.- Returns:
IcebergSink.Builderto connect the iceberg table.
-
equalityFieldColumns
Configuring the equality field columns for iceberg table that accept CDC or UPSERT events.- Parameters:
columns- defines the iceberg table's key.- Returns:
IcebergSink.Builderto connect the iceberg table.
-
uidSuffix
Set the uid suffix for IcebergSink operators. Note that IcebergSink internally consists of multiple operators (like writer, committer, aggregator). Actual operator uid will be appended with a suffix like "Sink Committer: $uidSuffix".Flink auto generates operator uid if not set explicitly. It is a recommended best-practice to set uid for all operators before deploying to production. Flink has an option to
pipeline.auto-generate-uid=falseto disable auto-generation and force explicit setting of all operator uid.Be careful with setting this for an existing job, because now we are changing the operator uid from an auto-generated one to this new value. When deploying the change with a checkpoint, Flink won't be able to restore the previous IcebergSink operator state (more specifically the committer operator state). You need to use
--allowNonRestoredStateto ignore the previous sink state. During restore IcebergSink state is used to check if last commit was actually successful or not.--allowNonRestoredStatecan lead to data loss if the Iceberg commit failed in the last completed checkpoint.- Parameters:
newSuffix- suffix for Flink sink operator uid and name- Returns:
IcebergSink.Builderto connect the iceberg table.
-
snapshotProperties
-
setSnapshotProperty
-
toBranch
-
append
public org.apache.flink.streaming.api.datastream.DataStreamSink<org.apache.flink.table.data.RowData> append()Append the iceberg sink operators to write records to iceberg table.- Returns:
DataStreamSinkfor sink.
-