forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 1
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-27931][sql-gateway] Introduce the SqlGateway to assemble all c…
…omponents This closes apache#19894
- Loading branch information
Showing
9 changed files
with
482 additions
and
16 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
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
174 changes: 174 additions & 0 deletions
174
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/SqlGateway.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,174 @@ | ||
/* | ||
* 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:https://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; | ||
|
||
import org.apache.flink.annotation.VisibleForTesting; | ||
import org.apache.flink.configuration.ConfigurationUtils; | ||
import org.apache.flink.runtime.util.EnvironmentInformation; | ||
import org.apache.flink.runtime.util.JvmShutdownSafeguard; | ||
import org.apache.flink.runtime.util.SignalHandler; | ||
import org.apache.flink.table.gateway.api.SqlGatewayService; | ||
import org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpoint; | ||
import org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpointFactoryUtils; | ||
import org.apache.flink.table.gateway.api.utils.SqlGatewayException; | ||
import org.apache.flink.table.gateway.cli.SqlGatewayOptions; | ||
import org.apache.flink.table.gateway.cli.SqlGatewayOptionsParser; | ||
import org.apache.flink.table.gateway.service.SqlGatewayServiceImpl; | ||
import org.apache.flink.table.gateway.service.context.DefaultContext; | ||
import org.apache.flink.table.gateway.service.session.SessionManager; | ||
|
||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import java.io.PrintStream; | ||
import java.util.ArrayList; | ||
import java.util.List; | ||
import java.util.Properties; | ||
import java.util.concurrent.CountDownLatch; | ||
|
||
/** Main entry point for the SQL Gateway. */ | ||
public class SqlGateway { | ||
|
||
private static final Logger LOG = LoggerFactory.getLogger(SqlGateway.class); | ||
|
||
private final List<SqlGatewayEndpoint> endpoints; | ||
private final Properties dynamicConfig; | ||
private final CountDownLatch latch; | ||
|
||
private SessionManager sessionManager; | ||
|
||
public SqlGateway(Properties dynamicConfig) { | ||
this.endpoints = new ArrayList<>(); | ||
this.dynamicConfig = dynamicConfig; | ||
this.latch = new CountDownLatch(1); | ||
} | ||
|
||
public void start() throws Exception { | ||
DefaultContext context = | ||
DefaultContext.load(ConfigurationUtils.createConfiguration(dynamicConfig)); | ||
sessionManager = new SessionManager(context); | ||
|
||
sessionManager.start(); | ||
SqlGatewayService sqlGatewayService = new SqlGatewayServiceImpl(sessionManager); | ||
|
||
endpoints.addAll( | ||
SqlGatewayEndpointFactoryUtils.createSqlGatewayEndpoint( | ||
sqlGatewayService, context.getFlinkConfig())); | ||
|
||
for (SqlGatewayEndpoint endpoint : endpoints) { | ||
try { | ||
endpoint.start(); | ||
} catch (Throwable t) { | ||
LOG.error("Failed to start the endpoint.", t); | ||
stop(); | ||
throw new SqlGatewayException("Failed to start the endpoint.", t); | ||
} | ||
} | ||
} | ||
|
||
public void stop() { | ||
for (SqlGatewayEndpoint endpoint : endpoints) { | ||
stopEndpointSilently(endpoint); | ||
} | ||
if (sessionManager != null) { | ||
sessionManager.stop(); | ||
} | ||
latch.countDown(); | ||
} | ||
|
||
public void waitUntilStop() throws Exception { | ||
latch.await(); | ||
} | ||
|
||
public static void main(String[] args) { | ||
// startup checks and logging | ||
EnvironmentInformation.logEnvironmentInfo(LOG, "SqlGateway", args); | ||
SignalHandler.register(LOG); | ||
JvmShutdownSafeguard.installAsShutdownHook(LOG); | ||
|
||
startSqlGateway(System.out, args); | ||
} | ||
|
||
@VisibleForTesting | ||
static void startSqlGateway(PrintStream stream, String[] args) { | ||
SqlGatewayOptions cliOptions = SqlGatewayOptionsParser.parseSqlGatewayOptions(args); | ||
|
||
if (cliOptions.isPrintHelp()) { | ||
SqlGatewayOptionsParser.printHelpSqlGateway(stream); | ||
return; | ||
} | ||
|
||
SqlGateway gateway = new SqlGateway(cliOptions.getDynamicConfigs()); | ||
try { | ||
Runtime.getRuntime().addShutdownHook(new ShutdownThread(gateway)); | ||
gateway.start(); | ||
gateway.waitUntilStop(); | ||
} catch (Throwable t) { | ||
// User uses ctrl + c to cancel the Gateway manually | ||
if (t instanceof InterruptedException) { | ||
LOG.info("Caught " + t.getClass().getSimpleName() + ". Shutting down."); | ||
return; | ||
} | ||
// make space in terminal | ||
stream.println(); | ||
stream.println(); | ||
LOG.error( | ||
"SqlGateway must stop. Unexpected exception. This is a bug. Please consider filing an issue.", | ||
t); | ||
throw new SqlGatewayException( | ||
"Unexpected exception. This is a bug. Please consider filing an issue.", t); | ||
} | ||
} | ||
|
||
private void stopEndpointSilently(SqlGatewayEndpoint endpoint) { | ||
try { | ||
endpoint.stop(); | ||
} catch (Exception e) { | ||
LOG.error("Failed to stop the endpoint. Ignore.", e); | ||
} | ||
} | ||
|
||
// -------------------------------------------------------------------------------------------- | ||
|
||
private static class ShutdownThread extends Thread { | ||
|
||
private final SqlGateway gateway; | ||
|
||
public ShutdownThread(SqlGateway gateway) { | ||
this.gateway = gateway; | ||
} | ||
|
||
@Override | ||
public void run() { | ||
// Shutdown the gateway | ||
System.out.println("\nShutting down the Flink SqlGateway..."); | ||
LOG.info("Shutting down the Flink SqlGateway..."); | ||
|
||
try { | ||
gateway.stop(); | ||
} catch (Exception e) { | ||
LOG.error("Failed to shut down the Flink SqlGateway: " + e.getMessage(), e); | ||
System.out.println("Failed to shut down the Flink SqlGateway: " + e.getMessage()); | ||
} | ||
|
||
LOG.info("Flink SqlGateway has been shutdown."); | ||
System.out.println("Flink SqlGateway has been shutdown."); | ||
} | ||
} | ||
} |
41 changes: 41 additions & 0 deletions
41
...flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/cli/SqlGatewayOptions.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,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:https://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.cli; | ||
|
||
import java.util.Properties; | ||
|
||
/** Options to configure the {@code SqlGateway}. */ | ||
public class SqlGatewayOptions { | ||
|
||
private final boolean isPrintHelp; | ||
private final Properties dynamicConfigs; | ||
|
||
public SqlGatewayOptions(boolean isPrintHelp, Properties dynamicConfigs) { | ||
this.isPrintHelp = isPrintHelp; | ||
this.dynamicConfigs = dynamicConfigs; | ||
} | ||
|
||
public boolean isPrintHelp() { | ||
return isPrintHelp; | ||
} | ||
|
||
public Properties getDynamicConfigs() { | ||
return dynamicConfigs; | ||
} | ||
} |
Oops, something went wrong.