hackergin commented on code in PR #24767: URL: https://github.com/apache/flink/pull/24767#discussion_r1600850246
########## flink-table/flink-table-common/src/main/java/org/apache/flink/table/workflow/WorkflowScheduler.java: ########## @@ -0,0 +1,91 @@ +/* + * 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.workflow; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.refresh.RefreshHandler; +import org.apache.flink.table.refresh.RefreshHandlerSerializer; + +/** + * This interface is used to interact with specific workflow scheduler services that support + * creating, modifying, and deleting refreshed workflow of Materialized Table. + * + * @param <T> The type of {@link RefreshHandler} used by specific {@link WorkflowScheduler} to + * locate the refresh workflow in scheduler service. + */ +@PublicEvolving +public interface WorkflowScheduler<T extends RefreshHandler> { + + /** + * Open this workflow scheduler instance. Used for any required preparation in initialization + * phase. + * + * @throws WorkflowException if initializing workflow scheduler occur exception + */ + void open() throws WorkflowException; + + /** + * Close this workflow scheduler when it is no longer needed and release any resource that it + * might be holding. + * + * @throws WorkflowException if close the related resources of workflow scheduler failed + */ + void close() throws WorkflowException; + + /** + * Return a {@link RefreshHandlerSerializer} instance to serialize and deserialize {@link + * RefreshHandler} created by specific workflow scheduler service. + */ + RefreshHandlerSerializer<T> getRefreshHandlerSerializer(); + + /** + * Create a refresh workflow in specific scheduler service for the materialized table, return a + * {@link RefreshHandler} instance which can locate the refresh workflow detail information. + * + * <p>This method supports creating workflow for periodic refresh, as well as workflow for a + * one-time refresh only. + * + * @param createRefreshWorkflow The detail info for create refresh workflow of materialized + * table. + * @return The meta info which points to the refresh workflow in scheduler service. + * @throws WorkflowException if create refresh workflow failed Review Comment: ```suggestion * @throws WorkflowException if creating refresh workflow failed ``` ########## flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/WorkflowSchedulerFactoryUtil.java: ########## @@ -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.flink.table.factories; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.DelegatingConfiguration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.workflow.WorkflowScheduler; +import org.apache.flink.util.StringUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.flink.table.factories.FactoryUtil.PROPERTY_VERSION; +import static org.apache.flink.table.factories.FactoryUtil.WORKFLOW_SCHEDULER_TYPE; +import static org.apache.flink.table.factories.FactoryUtil.stringifyOption; + +/** Utility for working with {@link WorkflowScheduler}. */ +@PublicEvolving +public class WorkflowSchedulerFactoryUtil { + + private static final Logger LOG = LoggerFactory.getLogger(WorkflowSchedulerFactoryUtil.class); + + public static final String WORKFLOW_SCHEDULER_PREFIX = "workflow-scheduler"; + + private WorkflowSchedulerFactoryUtil() { + // no instantiation + } + + /** + * Attempts to discover the appropriate workflow scheduler factory and creates the instance of + * the scheduler. Return null directly if doesn't specify the workflow scheduler in config + * because it is optional for materialized table. + */ + public static @Nullable WorkflowScheduler<?> createWorkflowScheduler( + Configuration configuration, ClassLoader classLoader) { Review Comment: Is it possible to pass in a ReadableConfig here? Such as TableConfig? If so, we can directly use the following method to get all options. ``` ConfigOption<Map<String, String>> workflowSchedulerOptions = new ConfigOptions.key(WORKFLOW_SCHEDULER_PREFIX+ "." + identifier) .mapType() .defaultValue(new HashMap<>()); Map<String, String> options = configuration.get(workflowSchedulerOptions); ``` ########## flink-table/flink-table-common/src/main/java/org/apache/flink/table/workflow/WorkflowScheduler.java: ########## @@ -0,0 +1,91 @@ +/* + * 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.workflow; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.refresh.RefreshHandler; +import org.apache.flink.table.refresh.RefreshHandlerSerializer; + +/** + * This interface is used to interact with specific workflow scheduler services that support + * creating, modifying, and deleting refreshed workflow of Materialized Table. + * + * @param <T> The type of {@link RefreshHandler} used by specific {@link WorkflowScheduler} to + * locate the refresh workflow in scheduler service. + */ +@PublicEvolving +public interface WorkflowScheduler<T extends RefreshHandler> { + + /** + * Open this workflow scheduler instance. Used for any required preparation in initialization + * phase. + * + * @throws WorkflowException if initializing workflow scheduler occur exception + */ + void open() throws WorkflowException; + + /** + * Close this workflow scheduler when it is no longer needed and release any resource that it + * might be holding. + * + * @throws WorkflowException if close the related resources of workflow scheduler failed Review Comment: ```suggestion * @throws WorkflowException if closing the related resources of workflow scheduler failed ``` ########## flink-table/flink-table-common/src/main/java/org/apache/flink/table/workflow/WorkflowScheduler.java: ########## @@ -0,0 +1,91 @@ +/* + * 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.workflow; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.refresh.RefreshHandler; +import org.apache.flink.table.refresh.RefreshHandlerSerializer; + +/** + * This interface is used to interact with specific workflow scheduler services that support + * creating, modifying, and deleting refreshed workflow of Materialized Table. + * + * @param <T> The type of {@link RefreshHandler} used by specific {@link WorkflowScheduler} to + * locate the refresh workflow in scheduler service. + */ +@PublicEvolving +public interface WorkflowScheduler<T extends RefreshHandler> { + + /** + * Open this workflow scheduler instance. Used for any required preparation in initialization + * phase. + * + * @throws WorkflowException if initializing workflow scheduler occur exception + */ + void open() throws WorkflowException; + + /** + * Close this workflow scheduler when it is no longer needed and release any resource that it + * might be holding. + * + * @throws WorkflowException if close the related resources of workflow scheduler failed + */ + void close() throws WorkflowException; + + /** + * Return a {@link RefreshHandlerSerializer} instance to serialize and deserialize {@link + * RefreshHandler} created by specific workflow scheduler service. + */ + RefreshHandlerSerializer<T> getRefreshHandlerSerializer(); + + /** + * Create a refresh workflow in specific scheduler service for the materialized table, return a + * {@link RefreshHandler} instance which can locate the refresh workflow detail information. + * + * <p>This method supports creating workflow for periodic refresh, as well as workflow for a + * one-time refresh only. + * + * @param createRefreshWorkflow The detail info for create refresh workflow of materialized + * table. + * @return The meta info which points to the refresh workflow in scheduler service. + * @throws WorkflowException if create refresh workflow failed Review Comment: As I understand it, there may be two possible implementations for the current CreateRefreshWorkflow in the future, corn and one-time. If there is a WorkFlowScheduler that supports only one of these two types, should we throw an unsupported exception at this point? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
