Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Flink: table supplier in writer operator for optional table refresh #8555

Merged
merged 45 commits into from
Sep 21, 2023
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
45 commits
Select commit Hold shift + click to select a range
c9bbee3
Flink: provide table supplier for optional table reload
bryanck Sep 12, 2023
d5f884d
add log for reload
bryanck Sep 13, 2023
41a27f4
open table loader if needed
bryanck Sep 13, 2023
de536d5
reload table at commit in committer
bryanck Sep 13, 2023
41dfe7e
store initial table props in manifest util
bryanck Sep 13, 2023
08825a5
PR feedback
bryanck Sep 13, 2023
90934eb
comment grammar
bryanck Sep 13, 2023
3b101ae
use junit5 for new test
bryanck Sep 13, 2023
dd6f799
spacing
bryanck Sep 13, 2023
77fd879
explicit refresh of table
bryanck Sep 14, 2023
389e1c3
reload test fix
bryanck Sep 14, 2023
b5dee48
PR cleanup feedback
bryanck Sep 14, 2023
83c6fb0
Merge branch 'master' into flink-table-supplier
bryanck Sep 14, 2023
b1e0ed7
remove jitter, rename vars
bryanck Sep 14, 2023
74754f5
update log statement
bryanck Sep 14, 2023
3b93157
javadoc fix
bryanck Sep 14, 2023
5d20cc6
test assertion fix
bryanck Sep 14, 2023
8af3916
log warning on reload failure
bryanck Sep 14, 2023
501ccce
refresh table in writer at startup also
bryanck Sep 15, 2023
86bbc69
specify table supplier in sink builder
bryanck Sep 15, 2023
f40330b
add comments
bryanck Sep 15, 2023
e0f12f1
comment typo
bryanck Sep 15, 2023
a50f545
update method name
bryanck Sep 15, 2023
1deaae5
use table loader interface instead of table supplier
bryanck Sep 17, 2023
d4a7116
check if table loader is already open
bryanck Sep 17, 2023
8190115
PR feedback
bryanck Sep 19, 2023
a1d5e60
checkstyle
bryanck Sep 19, 2023
48ef055
package private method
bryanck Sep 19, 2023
dcd6630
grammar/typo
bryanck Sep 19, 2023
f417bfd
revert back to table supplier
bryanck Sep 19, 2023
626312d
test fix
bryanck Sep 19, 2023
4e84f76
update test name
bryanck Sep 20, 2023
9b874c1
change param to SerializableTable
bryanck Sep 20, 2023
256859d
Merge branch 'master' into flink-table-supplier
bryanck Sep 20, 2023
bab1429
update test assertion
bryanck Sep 20, 2023
4a2370d
move refresh interval config to Flink config
bryanck Sep 20, 2023
ca740ca
javadoc
bryanck Sep 20, 2023
1290e20
duration config
bryanck Sep 20, 2023
d8c472e
test fix
bryanck Sep 20, 2023
e0346bc
store last load time instead of next
bryanck Sep 20, 2023
fe46157
add duration type support for config
bryanck Sep 20, 2023
967edcb
refresh table on get in supplier
bryanck Sep 21, 2023
77836e7
fix comment
bryanck Sep 21, 2023
d87a246
ensure initial table is used for schema for now
bryanck Sep 21, 2023
3a121bc
refresh before creating task writer
bryanck Sep 21, 2023
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,9 @@
*/
package org.apache.iceberg.flink;

import java.time.Duration;
import java.util.Map;
import org.apache.calcite.linq4j.function.Experimental;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.iceberg.DistributionMode;
import org.apache.iceberg.FileFormat;
Expand Down Expand Up @@ -184,4 +186,27 @@ public String branch() {
public Integer writeParallelism() {
return confParser.intConf().option(FlinkWriteOptions.WRITE_PARALLELISM.key()).parseOptional();
}

/**
* NOTE: This may be removed or changed in a future release. This value specifies the interval for
* refreshing the table instance in {@link IcebergStreamWriter}. If not specified then the default
* behavior is to not refresh the table.
*
* @return the interval for refreshing the table in writer subtasks
*/
@Experimental
public Duration tableRefreshInterval() {
Long intervalMs =
confParser
.longConf()
.option(FlinkWriteOptions.TABLE_REFRSH_MS.key())
.flinkConfig(FlinkWriteOptions.TABLE_REFRSH_MS)
.parseOptional();

if (intervalMs == null) {
return null;
}

return Duration.ofMillis(intervalMs);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
*/
package org.apache.iceberg.flink;

import org.apache.calcite.linq4j.function.Experimental;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ConfigOptions;
import org.apache.iceberg.SnapshotRef;
Expand Down Expand Up @@ -64,4 +65,8 @@ private FlinkWriteOptions() {}

public static final ConfigOption<Integer> WRITE_PARALLELISM =
ConfigOptions.key("write-parallelism").intType().noDefaultValue();

@Experimental
public static final ConfigOption<Long> TABLE_REFRSH_MS =
ConfigOptions.key("table-refresh-ms").longType().noDefaultValue();
stevenzwu marked this conversation as resolved.
Show resolved Hide resolved
}
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@
import java.util.Map;
import java.util.Set;
import java.util.function.Function;
import org.apache.flink.annotation.Experimental;
import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeinfo.Types;
Expand Down Expand Up @@ -133,7 +132,6 @@ public static Builder forRowData(DataStream<RowData> input) {
public static class Builder {
private Function<String, DataStream<RowData>> inputCreator = null;
private TableLoader tableLoader;
private Duration tableRefreshInterval;
private Table table;
private TableSchema tableSchema;
private List<String> equalityFieldColumns = null;
Expand Down Expand Up @@ -270,20 +268,6 @@ public Builder upsert(boolean enabled) {
return this;
}

/**
* Sets the interval for refreshing the table instance in {@link IcebergStreamWriter}. If not
* specified then the default behavior is to not refresh the table, and the initial table
* instance initialized is used for the lifetime of the job.
*
* @param refreshInterval the interval for refreshing the table in writer subtasks
* @return {@link Builder} to connect the iceberg table.
*/
@Experimental
public Builder tableRefreshInterval(Duration refreshInterval) {
this.tableRefreshInterval = refreshInterval;
return this;
}

/**
* Configuring the equality field columns for iceberg table that accept CDC or UPSERT events.
*
Expand Down Expand Up @@ -484,6 +468,8 @@ private SingleOutputStreamOperator<WriteResult> appendWriter(
}

SerializableTable serializableTable = (SerializableTable) SerializableTable.copyOf(table);
Duration tableRefreshInterval = flinkWriteConf.tableRefreshInterval();

SerializableSupplier<Table> tableSupplier;
if (tableRefreshInterval != null) {
tableSupplier =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,10 @@
package org.apache.iceberg.flink.sink;

import java.io.IOException;
import java.time.Duration;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.table.api.TableSchema;
Expand Down Expand Up @@ -378,10 +378,13 @@ public void testWriteRowWithTableRefreshInterval() throws Exception {
env.addSource(createBoundedSource(rows), ROW_TYPE_INFO)
.map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE));

Configuration flinkConf = new Configuration();
flinkConf.setLong(FlinkWriteOptions.TABLE_REFRSH_MS.key(), 100L);

FlinkSink.forRowData(dataStream)
.table(table)
.tableLoader(tableLoader)
.tableRefreshInterval(Duration.ofMillis(100))
.flinkConf(flinkConf)
.writeParallelism(parallelism)
.append();

Expand Down
Loading