-
Notifications
You must be signed in to change notification settings - Fork 13.7k
[FLINK-38257][SQL] Add module dynamic function loading possibility #26933
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,61 +19,34 @@ | |
package org.apache.flink.state.table.module; | ||
|
||
import org.apache.flink.annotation.Experimental; | ||
import org.apache.flink.annotation.VisibleForTesting; | ||
import org.apache.flink.state.table.SavepointMetadataTableFunction; | ||
import org.apache.flink.table.api.DataTypes; | ||
import org.apache.flink.table.functions.BuiltInFunctionDefinition; | ||
import org.apache.flink.table.functions.DynamicBuiltInFunctionDefinitionFactory; | ||
import org.apache.flink.table.functions.FunctionDefinition; | ||
import org.apache.flink.table.module.Module; | ||
import org.apache.flink.table.types.inference.TypeStrategies; | ||
|
||
import java.util.Collections; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import java.util.ArrayList; | ||
import java.util.HashSet; | ||
import java.util.List; | ||
import java.util.Locale; | ||
import java.util.Map; | ||
import java.util.Optional; | ||
import java.util.ServiceLoader; | ||
import java.util.Set; | ||
import java.util.function.Function; | ||
import java.util.stream.Collectors; | ||
|
||
import static org.apache.flink.table.functions.FunctionKind.TABLE; | ||
|
||
/** Module of state in Flink. */ | ||
@Experimental | ||
public class StateModule implements Module { | ||
|
||
public static final String IDENTIFIER = "state"; | ||
private static final Logger LOG = LoggerFactory.getLogger(StateModule.class); | ||
|
||
public static final BuiltInFunctionDefinition SAVEPOINT_METADATA = | ||
BuiltInFunctionDefinition.newBuilder() | ||
.name("savepoint_metadata") | ||
.kind(TABLE) | ||
.runtimeClass(SavepointMetadataTableFunction.class.getName()) | ||
.outputTypeStrategy( | ||
TypeStrategies.explicit( | ||
DataTypes.ROW( | ||
DataTypes.FIELD( | ||
"checkpoint-id", DataTypes.BIGINT().notNull()), | ||
DataTypes.FIELD("operator-name", DataTypes.STRING()), | ||
DataTypes.FIELD("operator-uid", DataTypes.STRING()), | ||
DataTypes.FIELD( | ||
"operator-uid-hash", | ||
DataTypes.STRING().notNull()), | ||
DataTypes.FIELD( | ||
"operator-parallelism", | ||
DataTypes.INT().notNull()), | ||
DataTypes.FIELD( | ||
"operator-max-parallelism", | ||
DataTypes.INT().notNull()), | ||
DataTypes.FIELD( | ||
"operator-subtask-state-count", | ||
DataTypes.INT().notNull()), | ||
DataTypes.FIELD( | ||
"operator-coordinator-state-size-in-bytes", | ||
DataTypes.BIGINT().notNull()), | ||
DataTypes.FIELD( | ||
"operator-total-size-in-bytes", | ||
DataTypes.BIGINT().notNull())))) | ||
.build(); | ||
public static final String IDENTIFIER = "state"; | ||
|
||
public static final StateModule INSTANCE = new StateModule(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: this is not part of this pr, but it seems like the constructor does classpath scanning, and that is fine if the module is loaded once, but worth a note: if someone repeatedly constructs modules in tests, scanning happens each time There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think testing the module as it's going to behave in PROD is something which is desired even if it requires some time. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yeah, totally fair |
||
|
||
|
@@ -82,8 +55,19 @@ public class StateModule implements Module { | |
private final Set<String> functionNamesWithoutInternal; | ||
|
||
private StateModule() { | ||
final List<BuiltInFunctionDefinition> definitions = | ||
Collections.singletonList(SAVEPOINT_METADATA); | ||
final List<BuiltInFunctionDefinition> definitions = new ArrayList<>(); | ||
|
||
definitions.add(SavepointMetadataTableFunction.SAVEPOINT_METADATA); | ||
gaborgsomogyi marked this conversation as resolved.
Show resolved
Hide resolved
|
||
ServiceLoader.load(DynamicBuiltInFunctionDefinitionFactory.class) | ||
.iterator() | ||
.forEachRemaining( | ||
f -> { | ||
if (f.factoryIdentifier().startsWith(IDENTIFIER + ".")) { | ||
definitions.addAll(f.getBuiltInFunctionDefinitions()); | ||
} | ||
}); | ||
checkDuplicatedFunctions(definitions); | ||
|
||
this.normalizedFunctions = | ||
definitions.stream() | ||
.collect( | ||
|
@@ -101,6 +85,25 @@ private StateModule() { | |
.collect(Collectors.toSet()); | ||
} | ||
|
||
@VisibleForTesting | ||
static void checkDuplicatedFunctions(List<BuiltInFunctionDefinition> definitions) { | ||
Set<String> seen = new HashSet<>(); | ||
Set<String> duplicates = new HashSet<>(); | ||
|
||
for (BuiltInFunctionDefinition definition : definitions) { | ||
String name = definition.getName(); | ||
if (!seen.add(name)) { | ||
duplicates.add(name); | ||
} | ||
} | ||
|
||
if (!duplicates.isEmpty()) { | ||
String error = "Duplicate function names found: " + String.join(",", duplicates); | ||
LOG.error(error); | ||
throw new IllegalStateException(error); | ||
} | ||
} | ||
|
||
@Override | ||
public Set<String> listFunctions() { | ||
return listFunctions(false); | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,37 @@ | ||
/* | ||
* 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.flink.state.table.module; | ||
|
||
import org.apache.flink.table.functions.BuiltInFunctionDefinition; | ||
import org.apache.flink.table.functions.DynamicBuiltInFunctionDefinitionFactory; | ||
|
||
import java.util.List; | ||
|
||
public class ExampleDynamicBuiltInFunctionDefinitionFactory | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. api looks minimal and clear. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This API is now fully experimental and later subject to change. If the time comes and There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. makes sense, thanks for clarifying bottom line: +1 to keep the api minimal as proposed |
||
implements DynamicBuiltInFunctionDefinitionFactory { | ||
@Override | ||
public String factoryIdentifier() { | ||
return StateModule.IDENTIFIER + ".example_function_factory"; | ||
} | ||
|
||
@Override | ||
public List<BuiltInFunctionDefinition> getBuiltInFunctionDefinitions() { | ||
return List.of(ExampleDynamicTableFunction.FUNCTION_DEFINITION); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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.flink.state.table.module; | ||
|
||
import org.apache.flink.annotation.Internal; | ||
import org.apache.flink.table.annotation.DataTypeHint; | ||
import org.apache.flink.table.annotation.FunctionHint; | ||
import org.apache.flink.table.api.DataTypes; | ||
import org.apache.flink.table.functions.BuiltInFunctionDefinition; | ||
import org.apache.flink.table.functions.SpecializedFunction; | ||
import org.apache.flink.table.functions.TableFunction; | ||
import org.apache.flink.table.types.inference.TypeStrategies; | ||
import org.apache.flink.types.Row; | ||
|
||
import static org.apache.flink.table.functions.FunctionKind.TABLE; | ||
|
||
@Internal | ||
@FunctionHint(output = @DataTypeHint("ROW<my-column STRING NOT NULL>")) | ||
public class ExampleDynamicTableFunction extends TableFunction<Row> { | ||
|
||
public static final BuiltInFunctionDefinition FUNCTION_DEFINITION = | ||
BuiltInFunctionDefinition.newBuilder() | ||
.name("example_dynamic_table_function") | ||
.kind(TABLE) | ||
.runtimeClass(ExampleDynamicTableFunction.class.getName()) | ||
.outputTypeStrategy( | ||
TypeStrategies.explicit( | ||
DataTypes.ROW( | ||
DataTypes.FIELD( | ||
"my-column", DataTypes.STRING().notNull())))) | ||
.build(); | ||
|
||
public ExampleDynamicTableFunction(SpecializedFunction.SpecializedContext context) {} | ||
|
||
public void eval() { | ||
Row row = Row.withNames(); | ||
row.setField("my-column", "my-value"); | ||
collect(row); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,66 @@ | ||
/* | ||
* 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.flink.state.table.module; | ||
|
||
import org.apache.flink.api.common.RuntimeExecutionMode; | ||
import org.apache.flink.configuration.Configuration; | ||
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; | ||
import org.apache.flink.table.api.Table; | ||
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; | ||
import org.apache.flink.types.Row; | ||
|
||
import org.junit.jupiter.api.Test; | ||
|
||
import java.util.Iterator; | ||
import java.util.List; | ||
|
||
import static org.apache.flink.configuration.ExecutionOptions.RUNTIME_MODE; | ||
import static org.assertj.core.api.Assertions.assertThat; | ||
import static org.assertj.core.api.Assertions.assertThatThrownBy; | ||
|
||
/** Unit tests for the savepoint SQL reader. */ | ||
public class StateModuleTest { | ||
@Test | ||
public void testDynamicBuiltinFunctionShouldBeLoaded() throws Exception { | ||
Configuration config = new Configuration(); | ||
config.set(RUNTIME_MODE, RuntimeExecutionMode.BATCH); | ||
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(config); | ||
StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); | ||
|
||
tEnv.executeSql("LOAD MODULE state"); | ||
Table table = tEnv.sqlQuery("SELECT * FROM example_dynamic_table_function()"); | ||
List<Row> result = tEnv.toDataStream(table).executeAndCollect(100); | ||
|
||
assertThat(result.size()).isEqualTo(1); | ||
Iterator<Row> it = result.iterator(); | ||
assertThat(it.next().toString()).isEqualTo("+I[my-value]"); | ||
} | ||
|
||
@Test | ||
public void testMultipleFunctionsWithSameNameShouldThrow() { | ||
assertThatThrownBy( | ||
() -> | ||
StateModule.checkDuplicatedFunctions( | ||
List.of( | ||
ExampleDynamicTableFunction.FUNCTION_DEFINITION, | ||
ExampleDynamicTableFunction.FUNCTION_DEFINITION))) | ||
.isInstanceOf(IllegalStateException.class) | ||
.hasMessage("Duplicate function names found: example_dynamic_table_function"); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,16 @@ | ||
# 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. | ||
|
||
org.apache.flink.state.table.module.ExampleDynamicBuiltInFunctionDefinitionFactory |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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.flink.table.functions; | ||
|
||
import org.apache.flink.annotation.Experimental; | ||
|
||
import java.util.List; | ||
|
||
/** | ||
* {@link BuiltInFunctionDefinition} factory for dynamic function registration. This could be useful | ||
* when functions wanted to be automatically loaded from SQL modules. A good example usage is that | ||
* state processor API SQL module is providing extra table functions from Kafka connector. | ||
*/ | ||
@Experimental | ||
public interface DynamicBuiltInFunctionDefinitionFactory { | ||
/** | ||
* Returns the unique identifier of the factory. The suggested pattern is the following: | ||
* [module-name].[factory-name]. Such case modules can load all [module-name] prefixed functions | ||
* which belong to them. | ||
*/ | ||
String factoryIdentifier(); | ||
|
||
/** Returns list of {@link BuiltInFunctionDefinition} which can be registered dynamically. */ | ||
List<BuiltInFunctionDefinition> getBuiltInFunctionDefinitions(); | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nice to keep the definition near the code. do you think we should also mark that the function should not be pre-calculated by the optimizer? i mean to make it clear that flink must always run it at query time, not try to be clever and “pre-compute” it
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This one is not yet clear🤷🏻♂️
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
maybe i misunderstood what happens here with
savepoint_metadata
, but flink's planner can constant-fold and cache function results at plan time if it believes a function is deterministic. that is correct for pure math/string functions, but could be problematic for something like thissavepoint_metadata
, which read external, changing state (fs/checkpoint files). and between planning and exec, or between two exec, the metadata could change. so i just thought whether we should tell the planner this function is non-deterministic, so it will not pre-evaluate or cache it, and will always run it at exec time. idk whether theBuiltInFunctionDefinition.newBuilder()
can supports to call.deterministic(false)
, or use and override the UDFisDeterministic()
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
That's fair point. I'm not fan of fixing something which is unrelated but this is so tiny that adding
notDeterministic()
is fine.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Though it worth the discussion separate whether the planner caches it with input params or not. I would guess yes, otherwise it just doesn't make sense from usage point of view. If the params are stored together with the result then adding
notDeterministic()
doesn't hurt but also not helping.