Skip to content

Flink: SQL: Make Dynamic sink options to be configurable in SQL#15780

Open
swapna267 wants to merge 2 commits intoapache:mainfrom
swapna267:dynamic_sink_sql_config
Open

Flink: SQL: Make Dynamic sink options to be configurable in SQL#15780
swapna267 wants to merge 2 commits intoapache:mainfrom
swapna267:dynamic_sink_sql_config

Conversation

@swapna267
Copy link
Copy Markdown
Contributor

Support following configs to be configurable from SQL for dynamic sink.

Fallback to writeproperties or Flink configuration if following are not set on DynamicRecord,

writeParallelism(int) → FlinkWriteOptions.WRITE_PARALLELISM
distributionMode -> FlinkWriteOptions.DISTRIBUTION_MODE
toBranch(String) → FlinkWriteOptions.BRANCH

Provide options to configure following behavior of Dynamic Sink in SQl

cacheMaxSize(int)
immediateTableUpdate(boolean)
dropUnusedColumns(boolean)
cacheRefreshMs(long)
inputSchemasPerTableCacheMaxSize(int)
caseSensitive(boolean)

More context here, #15471 (comment)

@github-actions github-actions bot added the flink label Mar 26, 2026
Copy link
Copy Markdown
Contributor

@mxm mxm left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @swapna267!

* specific language governing permissions and limitations
* under the License.
*/
package org.apache.iceberg.flink;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should probably be in the dynamic package. Or should we create a config package?

* specific language governing permissions and limitations
* under the License.
*/
package org.apache.iceberg.flink;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same here.

Comment on lines +96 to +97
FlinkDynamicSinkConf flinkDynamicSinkConf =
new FlinkDynamicSinkConf(writeProperties, flinkConfig);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we directly pass FlinkDynamicSinkConf to the constructor?

Comment on lines +303 to +305
writeOptions.put(
FlinkDynamicSinkOptions.IMMEDIATE_TABLE_UPDATE.key(),
Boolean.toString(newImmediateUpdate));
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not sure this should go into WriteOptions. I think it is better to have a separate config for DynamicSink options.

Copy link
Copy Markdown
Contributor Author

@swapna267 swapna267 Mar 27, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

With all of them written into WriteOptions, it is easier to pass these configs from SQL by using setAll(Map<String, String> properties) for DynamicIcebergSink initialization.

If we separate them, we either need to handle it in setAll or upstream users need to provide them separately.

As Dynamic sink configs are scoped with prefix dynamic-sink , should be ok to go in same map ?

Comment on lines +75 to +76
FlinkDynamicSinkConf flinkDynamicSinkConf =
new FlinkDynamicSinkConf(writeProperties, flinkConfig);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we create the config only once and pass it to the constructor?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Did and removed for consistency.

DynamicRecordProcessor needs FlinkDynamicSinkConf and also writeProperties/flinkConfig.

WriteProperties and FlinkConfig are required to create FlinkWriteConf in Open as it's not serializable.

May be, I can simply pass FlinkDynamicSinkConf also along with writeProperties/flinkConfig.

Comment on lines +61 to +63
if (super.writeParallelism() != Integer.MAX_VALUE) {
return super.writeParallelism();
}
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not sure about this logic. The default for writeParallelism is 0.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

https://github.com/apache/iceberg/blob/main/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java#L51-L71

DynamicRecord constructor has writeParallelism as primitive int. And we basically are using Integer.MAX_VALUE to fall back to other value like job parallelism.

   * @param writeParallelism The number of parallel writers. Can be set to any value {@literal > 0},
   *     but will always be automatically capped by the maximum write parallelism, which is the
   *     parallelism of the sink. Set to Integer.MAX_VALUE for always using the maximum available
   *     write parallelism.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we have similar issue with upsertMode as it's using primitive boolean.

* have fields set.
*/
@Internal
class DynamicRecordWithDefaults extends DynamicRecord {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
class DynamicRecordWithDefaults extends DynamicRecord {
class DynamicRecordWithConfig extends DynamicRecord {

Should we add a test to verify config handling?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants