forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-35347][table] Introduce embedded scheduler to support full ref…
…resh mode for materialized table
- Loading branch information
Showing
26 changed files
with
2,061 additions
and
7 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
98 changes: 98 additions & 0 deletions
98
...eway/rest/handler/materializedtable/scheduler/CreateEmbeddedSchedulerWorkflowHandler.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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); | ||
} | ||
} | ||
} |
75 changes: 75 additions & 0 deletions
75
...eway/rest/handler/materializedtable/scheduler/DeleteEmbeddedSchedulerWorkflowHandler.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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); | ||
} | ||
} | ||
} |
Oops, something went wrong.