Skip to content

Commit

Permalink
[FLINK-35347][table] Introduce embedded scheduler to support full ref…
Browse files Browse the repository at this point in the history
…resh mode for materialized table
  • Loading branch information
lsyldliu committed May 24, 2024
1 parent 0ec6302 commit 62b8fee
Show file tree
Hide file tree
Showing 26 changed files with 2,061 additions and 7 deletions.
26 changes: 26 additions & 0 deletions flink-table/flink-sql-gateway/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,13 @@
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>org.quartz-scheduler</groupId>
<artifactId>quartz</artifactId>
<version>${quartz.version}</version>
<optional>${flink.markBundledAsOptional}</optional>
</dependency>

<!-- Test dependencies -->
<dependency>
<groupId>org.apache.flink</groupId>
Expand Down Expand Up @@ -153,8 +160,27 @@
<artifactSet>
<includes combine.children="append">
<include>org.apache.flink:flink-sql-gateway-api</include>
<include>org.quartz-scheduler:quartz</include>
</includes>
</artifactSet>
<filters>
<filter>
<artifact>org.quartz-scheduler:quartz</artifact>
<excludes>
<exclude>**/checkstyle.xml</exclude>
</excludes>
</filter>
</filters>
<relocations>
<relocation>
<pattern>org.quartz</pattern>
<shadedPattern>org.apache.flink.table.shaded.org.quartz</shadedPattern>
</relocation>
<relocation>
<pattern>org.terracotta.quartz</pattern>
<shadedPattern>org.apache.flink.table.shaded.org.terracotta.quartz</shadedPattern>
</relocation>
</relocations>
</configuration>
</execution>
</executions>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,10 @@
import org.apache.flink.runtime.rest.handler.RestHandlerSpecification;
import org.apache.flink.table.gateway.api.SqlGatewayService;
import org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpoint;
import org.apache.flink.table.gateway.rest.handler.materializedtable.scheduler.CreateEmbeddedSchedulerWorkflowHandler;
import org.apache.flink.table.gateway.rest.handler.materializedtable.scheduler.DeleteEmbeddedSchedulerWorkflowHandler;
import org.apache.flink.table.gateway.rest.handler.materializedtable.scheduler.ResumeEmbeddedSchedulerWorkflowHandler;
import org.apache.flink.table.gateway.rest.handler.materializedtable.scheduler.SuspendEmbeddedSchedulerWorkflowHandler;
import org.apache.flink.table.gateway.rest.handler.operation.CancelOperationHandler;
import org.apache.flink.table.gateway.rest.handler.operation.CloseOperationHandler;
import org.apache.flink.table.gateway.rest.handler.operation.GetOperationStatusHandler;
Expand All @@ -37,6 +41,10 @@
import org.apache.flink.table.gateway.rest.handler.statement.FetchResultsHandler;
import org.apache.flink.table.gateway.rest.handler.util.GetApiVersionHandler;
import org.apache.flink.table.gateway.rest.handler.util.GetInfoHandler;
import org.apache.flink.table.gateway.rest.header.materializedtable.scheduler.CreateEmbeddedSchedulerWorkflowHeaders;
import org.apache.flink.table.gateway.rest.header.materializedtable.scheduler.DeleteEmbeddedSchedulerWorkflowHeaders;
import org.apache.flink.table.gateway.rest.header.materializedtable.scheduler.ResumeEmbeddedSchedulerWorkflowHeaders;
import org.apache.flink.table.gateway.rest.header.materializedtable.scheduler.SuspendEmbeddedSchedulerWorkflowHeaders;
import org.apache.flink.table.gateway.rest.header.operation.CancelOperationHeaders;
import org.apache.flink.table.gateway.rest.header.operation.CloseOperationHeaders;
import org.apache.flink.table.gateway.rest.header.operation.GetOperationStatusHeaders;
Expand All @@ -50,6 +58,7 @@
import org.apache.flink.table.gateway.rest.header.statement.FetchResultsHeaders;
import org.apache.flink.table.gateway.rest.header.util.GetApiVersionHeaders;
import org.apache.flink.table.gateway.rest.header.util.GetInfoHeaders;
import org.apache.flink.table.gateway.workflow.scheduler.EmbeddedQuartzScheduler;
import org.apache.flink.util.ConfigurationException;

import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandler;
Expand All @@ -63,11 +72,13 @@
public class SqlGatewayRestEndpoint extends RestServerEndpoint implements SqlGatewayEndpoint {

public final SqlGatewayService service;
private final EmbeddedQuartzScheduler quartzScheduler;

public SqlGatewayRestEndpoint(Configuration configuration, SqlGatewayService sqlGatewayService)
throws IOException, ConfigurationException {
super(configuration);
service = sqlGatewayService;
quartzScheduler = new EmbeddedQuartzScheduler();
}

@Override
Expand All @@ -78,6 +89,7 @@ protected List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initiali
addOperationRelatedHandlers(handlers);
addUtilRelatedHandlers(handlers);
addStatementRelatedHandlers(handlers);
addEmbeddedSchedulerRelatedHandlers(handlers);
return handlers;
}

Expand Down Expand Up @@ -181,11 +193,57 @@ private void addStatementRelatedHandlers(
service, responseHeaders, FetchResultsHeaders.getInstanceV1())));
}

private void addEmbeddedSchedulerRelatedHandlers(
List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> handlers) {
// create workflow
CreateEmbeddedSchedulerWorkflowHandler createHandler =
new CreateEmbeddedSchedulerWorkflowHandler(
service,
quartzScheduler,
responseHeaders,
CreateEmbeddedSchedulerWorkflowHeaders.getInstance());
handlers.add(
Tuple2.of(CreateEmbeddedSchedulerWorkflowHeaders.getInstance(), createHandler));

// suspend workflow
SuspendEmbeddedSchedulerWorkflowHandler suspendHandler =
new SuspendEmbeddedSchedulerWorkflowHandler(
service,
quartzScheduler,
responseHeaders,
SuspendEmbeddedSchedulerWorkflowHeaders.getInstance());
handlers.add(
Tuple2.of(SuspendEmbeddedSchedulerWorkflowHeaders.getInstance(), suspendHandler));

// resume workflow
ResumeEmbeddedSchedulerWorkflowHandler resumeHandler =
new ResumeEmbeddedSchedulerWorkflowHandler(
service,
quartzScheduler,
responseHeaders,
ResumeEmbeddedSchedulerWorkflowHeaders.getInstance());
handlers.add(
Tuple2.of(ResumeEmbeddedSchedulerWorkflowHeaders.getInstance(), resumeHandler));

// delete workflow
DeleteEmbeddedSchedulerWorkflowHandler deleteHandler =
new DeleteEmbeddedSchedulerWorkflowHandler(
service,
quartzScheduler,
responseHeaders,
DeleteEmbeddedSchedulerWorkflowHeaders.getInstance());
handlers.add(
Tuple2.of(DeleteEmbeddedSchedulerWorkflowHeaders.getInstance(), deleteHandler));
}

@Override
protected void startInternal() {}
protected void startInternal() {
quartzScheduler.start();
}

@Override
public void stop() throws Exception {
super.close();
quartzScheduler.stop();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,98 @@
/*
* 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.gateway.rest.handler.materializedtable.scheduler;

import org.apache.flink.runtime.rest.handler.HandlerRequest;
import org.apache.flink.runtime.rest.handler.RestHandlerException;
import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
import org.apache.flink.runtime.rest.messages.MessageHeaders;
import org.apache.flink.table.gateway.api.SqlGatewayService;
import org.apache.flink.table.gateway.rest.handler.AbstractSqlGatewayRestHandler;
import org.apache.flink.table.gateway.rest.message.materializedtable.scheduler.CreateEmbeddedSchedulerWorkflowRequestBody;
import org.apache.flink.table.gateway.rest.message.materializedtable.scheduler.CreateEmbeddedSchedulerWorkflowResponseBody;
import org.apache.flink.table.gateway.rest.util.SqlGatewayRestAPIVersion;
import org.apache.flink.table.gateway.workflow.WorkflowInfo;
import org.apache.flink.table.gateway.workflow.scheduler.EmbeddedQuartzScheduler;

import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;

import org.quartz.JobDetail;
import org.quartz.JobKey;

import javax.annotation.Nonnull;
import javax.annotation.Nullable;

import java.util.Collections;
import java.util.Map;
import java.util.concurrent.CompletableFuture;

/** Handler to create workflow in embedded scheduler. */
public class CreateEmbeddedSchedulerWorkflowHandler
extends AbstractSqlGatewayRestHandler<
CreateEmbeddedSchedulerWorkflowRequestBody,
CreateEmbeddedSchedulerWorkflowResponseBody,
EmptyMessageParameters> {

private final EmbeddedQuartzScheduler quartzScheduler;

public CreateEmbeddedSchedulerWorkflowHandler(
SqlGatewayService service,
EmbeddedQuartzScheduler quartzScheduler,
Map<String, String> responseHeaders,
MessageHeaders<
CreateEmbeddedSchedulerWorkflowRequestBody,
CreateEmbeddedSchedulerWorkflowResponseBody,
EmptyMessageParameters>
messageHeaders) {
super(service, responseHeaders, messageHeaders);
this.quartzScheduler = quartzScheduler;
}

@Override
protected CompletableFuture<CreateEmbeddedSchedulerWorkflowResponseBody> handleRequest(
@Nullable SqlGatewayRestAPIVersion version,
@Nonnull HandlerRequest<CreateEmbeddedSchedulerWorkflowRequestBody> request)
throws RestHandlerException {
String materializedTableIdentifier =
request.getRequestBody().getMaterializedTableIdentifier();
String cronExpression = request.getRequestBody().getCronExpression();
Map<String, String> dynamicOptions = request.getRequestBody().getDynamicOptions();
Map<String, String> executionConfig = request.getRequestBody().getExecutionConfig();
String customScheduleTime = request.getRequestBody().getCustomScheduleTime();
String restEndpointURL = request.getRequestBody().getRestEndpointUrl();
WorkflowInfo workflowInfo =
new WorkflowInfo(
materializedTableIdentifier,
dynamicOptions == null ? Collections.emptyMap() : dynamicOptions,
executionConfig == null ? Collections.emptyMap() : executionConfig,
customScheduleTime,
restEndpointURL);
try {
JobDetail jobDetail =
quartzScheduler.createScheduleWorkflow(workflowInfo, cronExpression);
JobKey jobKey = jobDetail.getKey();
return CompletableFuture.completedFuture(
new CreateEmbeddedSchedulerWorkflowResponseBody(
jobKey.getName(), jobKey.getGroup()));
} catch (Exception e) {
throw new RestHandlerException(
e.getMessage(), HttpResponseStatus.INTERNAL_SERVER_ERROR, e);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
/*
* 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.gateway.rest.handler.materializedtable.scheduler;

import org.apache.flink.runtime.rest.handler.HandlerRequest;
import org.apache.flink.runtime.rest.handler.RestHandlerException;
import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
import org.apache.flink.runtime.rest.messages.EmptyResponseBody;
import org.apache.flink.runtime.rest.messages.MessageHeaders;
import org.apache.flink.table.gateway.api.SqlGatewayService;
import org.apache.flink.table.gateway.rest.handler.AbstractSqlGatewayRestHandler;
import org.apache.flink.table.gateway.rest.message.materializedtable.scheduler.EmbeddedSchedulerWorkflowRequestBody;
import org.apache.flink.table.gateway.rest.util.SqlGatewayRestAPIVersion;
import org.apache.flink.table.gateway.workflow.scheduler.EmbeddedQuartzScheduler;

import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;

import javax.annotation.Nonnull;
import javax.annotation.Nullable;

import java.util.Map;
import java.util.concurrent.CompletableFuture;

/** Handler to delete workflow in embedded scheduler. */
public class DeleteEmbeddedSchedulerWorkflowHandler
extends AbstractSqlGatewayRestHandler<
EmbeddedSchedulerWorkflowRequestBody, EmptyResponseBody, EmptyMessageParameters> {

private final EmbeddedQuartzScheduler quartzScheduler;

public DeleteEmbeddedSchedulerWorkflowHandler(
SqlGatewayService service,
EmbeddedQuartzScheduler quartzScheduler,
Map<String, String> responseHeaders,
MessageHeaders<
EmbeddedSchedulerWorkflowRequestBody,
EmptyResponseBody,
EmptyMessageParameters>
messageHeaders) {
super(service, responseHeaders, messageHeaders);
this.quartzScheduler = quartzScheduler;
}

@Override
protected CompletableFuture<EmptyResponseBody> handleRequest(
@Nullable SqlGatewayRestAPIVersion version,
@Nonnull HandlerRequest<EmbeddedSchedulerWorkflowRequestBody> request)
throws RestHandlerException {
String workflowName = request.getRequestBody().getWorkflowName();
String workflowGroup = request.getRequestBody().getWorkflowGroup();
try {
quartzScheduler.deleteScheduleWorkflow(workflowName, workflowGroup);
return CompletableFuture.completedFuture(EmptyResponseBody.getInstance());
} catch (Exception e) {
throw new RestHandlerException(
e.getMessage(), HttpResponseStatus.INTERNAL_SERVER_ERROR, e);
}
}
}
Loading

0 comments on commit 62b8fee

Please sign in to comment.