forked from apache/beam
-
Notifications
You must be signed in to change notification settings - Fork 5
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Let SQLPTransform support full SQL scripts
- Loading branch information
Becket Qin
committed
Feb 23, 2024
1 parent
cccac3a
commit b645f96
Showing
13 changed files
with
644 additions
and
38 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
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
109 changes: 109 additions & 0 deletions
109
...org/apache/beam/runners/flink/transform/sql/StatementOnlyFlinkSqlTransformTranslator.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,109 @@ | ||
/* | ||
* 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.beam.runners.flink.transform.sql; | ||
|
||
import com.google.auto.service.AutoService; | ||
import java.util.Map; | ||
import java.util.StringJoiner; | ||
import org.apache.beam.runners.core.construction.PTransformTranslation; | ||
import org.apache.beam.runners.core.construction.TransformPayloadTranslatorRegistrar; | ||
import org.apache.beam.runners.flink.FlinkCustomTransformTranslatorRegistrar; | ||
import org.apache.beam.runners.flink.FlinkStreamingPipelineTranslator; | ||
import org.apache.beam.runners.flink.FlinkStreamingTranslationContext; | ||
import org.apache.beam.sdk.transforms.PTransform; | ||
import org.apache.beam.sdk.values.PBegin; | ||
import org.apache.beam.sdk.values.PDone; | ||
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; | ||
import org.apache.flink.table.api.bridge.java.StreamStatementSet; | ||
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
|
||
public class StatementOnlyFlinkSqlTransformTranslator | ||
extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<PTransform<PBegin, PDone>>{ | ||
private static final Logger LOG = LoggerFactory.getLogger(StatementOnlyFlinkSqlTransformTranslator.class); | ||
public static final String FLINK_STATEMENT_ONLY_SQL_URN = "beam:transform:flink:sql-statements-only:v1"; | ||
private static final String INSERT_INTO = "INSERT INTO"; | ||
|
||
@Override | ||
public void translateNode(PTransform<PBegin, PDone> transform, FlinkStreamingTranslationContext context) { | ||
StatementOnlySqlTransform sqlTransform = (StatementOnlySqlTransform) transform; | ||
|
||
StreamTableEnvironment tEnv = StreamTableEnvironment.create(context.getExecutionEnvironment()); | ||
sqlTransform.getCatalogs().forEach(tEnv::registerCatalog); | ||
StringJoiner combinedStatements = new StringJoiner("\n\n"); | ||
StreamStatementSet ss = tEnv.createStatementSet(); | ||
for (String statement : sqlTransform.getStatements()) { | ||
combinedStatements.add(statement); | ||
try { | ||
if (isInsertIntoStatement(statement)) { | ||
ss.addInsertSql(statement); | ||
} else { | ||
// Not an insert into statement. Treat it as a DDL. | ||
tEnv.executeSql(statement); | ||
} | ||
} catch (Exception e) { | ||
LOG.error("Encountered exception when executing statement: {}", statement); | ||
throw new RuntimeException(e); | ||
} | ||
} | ||
// Now attach everything to StreamExecutionEnv. | ||
ss.attachAsDataStream(); | ||
LOG.info("Executing SQL statements:\n {}", combinedStatements); | ||
} | ||
|
||
/** Registers Flink SQL PTransform URN. */ | ||
@AutoService(TransformPayloadTranslatorRegistrar.class) | ||
@SuppressWarnings("rawtypes") | ||
public static class FlinkTransformsRegistrar implements TransformPayloadTranslatorRegistrar { | ||
@Override | ||
public Map< | ||
? extends Class<? extends PTransform>, | ||
? extends PTransformTranslation.TransformPayloadTranslator> | ||
getTransformPayloadTranslators() { | ||
return ImmutableMap | ||
.<Class<? extends PTransform>, PTransformTranslation.TransformPayloadTranslator>builder() | ||
.put( | ||
StatementOnlySqlTransform.class, | ||
PTransformTranslation.TransformPayloadTranslator.NotSerializable.forUrn( | ||
FLINK_STATEMENT_ONLY_SQL_URN)) | ||
.build(); | ||
} | ||
} | ||
|
||
/** Registers Flink SQL PTransform to the Flink runner. */ | ||
@AutoService(FlinkCustomTransformTranslatorRegistrar.class) | ||
public static class FlinkSqlTransformsRegistrar | ||
implements FlinkCustomTransformTranslatorRegistrar { | ||
@Override | ||
public Map<String, FlinkStreamingPipelineTranslator.StreamTransformTranslator<?>> | ||
getTransformTranslators() { | ||
return ImmutableMap | ||
.<String, FlinkStreamingPipelineTranslator.StreamTransformTranslator<?>>builder() | ||
.put(FLINK_STATEMENT_ONLY_SQL_URN, new StatementOnlyFlinkSqlTransformTranslator()) | ||
.build(); | ||
} | ||
} | ||
|
||
// ------------------- private helper methods ----------------- | ||
private static boolean isInsertIntoStatement(String statement) { | ||
return statement.substring(0, INSERT_INTO.length()).toUpperCase().startsWith(INSERT_INTO); | ||
} | ||
} |
103 changes: 103 additions & 0 deletions
103
.../src/main/java/org/apache/beam/runners/flink/transform/sql/StatementOnlySqlTransform.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,103 @@ | ||
/* | ||
* 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.beam.runners.flink.transform.sql; | ||
|
||
import java.util.ArrayList; | ||
import java.util.Collections; | ||
import java.util.HashMap; | ||
import java.util.List; | ||
import java.util.Map; | ||
import org.apache.beam.sdk.options.PipelineOptions; | ||
import org.apache.beam.sdk.transforms.PTransform; | ||
import org.apache.beam.sdk.values.PBegin; | ||
import org.apache.beam.sdk.values.PDone; | ||
import org.apache.flink.table.catalog.Catalog; | ||
import org.apache.flink.util.Preconditions; | ||
import org.checkerframework.checker.nullness.qual.Nullable; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
/** | ||
* A Beam PTransform that only take a complete SQL statements with INSERT INTO clause. | ||
*/ | ||
public class StatementOnlySqlTransform extends PTransform<PBegin, PDone> { | ||
private static final Logger LOG = LoggerFactory.getLogger(StatementOnlySqlTransform.class); | ||
|
||
private final List<String> statements; | ||
private final Map<String, SerializableCatalog> catalogs; | ||
|
||
StatementOnlySqlTransform() { | ||
this.statements = new ArrayList<>(); | ||
this.catalogs = new HashMap<>(); | ||
} | ||
|
||
@Override | ||
public PDone expand(PBegin input) { | ||
if (LOG.isDebugEnabled()) { | ||
LOG.debug("User statements:"); | ||
for (String statement : statements) { | ||
LOG.debug("{}\n", statement); | ||
} | ||
} | ||
return PDone.in(input.getPipeline()); | ||
} | ||
|
||
@Override | ||
public void validate(@Nullable PipelineOptions options) { | ||
Preconditions.checkArgument(!statements.isEmpty(), "No statement is provided for the SqlPtransform.."); | ||
} | ||
|
||
/** | ||
* Add any Flink SQL statement to this transform. Note that there must be a <code>INSERT INTO</code> | ||
* statement. Otherwise, an exception will be thrown. | ||
* | ||
* @param statement the statement to be added. | ||
* @return this {@link StatementOnlySqlTransform}. | ||
*/ | ||
public StatementOnlySqlTransform addStatement(String statement) { | ||
statements.add(cleanUp(statement)); | ||
return this; | ||
} | ||
|
||
/** | ||
* Define add a new {@link Catalog} to be used by the SQL query. | ||
* | ||
* @param name the name of the catalog. | ||
* @param catalog the catalog to use. | ||
* @return this {@link MultiOutputSqlTransformWithInput} itself. | ||
*/ | ||
public StatementOnlySqlTransform withCatalog(String name, SerializableCatalog catalog) { | ||
catalogs.put(name, catalog); | ||
return this; | ||
} | ||
|
||
// --------------------- package private getters ----------------- | ||
List<String> getStatements() { | ||
return Collections.unmodifiableList(statements); | ||
} | ||
|
||
Map<String, SerializableCatalog> getCatalogs() { | ||
return Collections.unmodifiableMap(catalogs); | ||
} | ||
|
||
// --------------------- private helpers ------------------------ | ||
private static String cleanUp(String s) { | ||
return s.trim().endsWith(";") ? s : s + ";"; | ||
} | ||
} |
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
Oops, something went wrong.