Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

SQL PTransform Update. #112

Merged
merged 3 commits into from
Feb 24, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -398,7 +398,7 @@ class BeamModulePlugin implements Plugin<Project> {

// Automatically use the official release version if we are performing a release
// otherwise append '-SNAPSHOT'
project.version = '2.45.18'
project.version = '2.45.19'
if (isLinkedin(project)) {
project.ext.mavenGroupId = 'com.linkedin.beam'
}
Expand Down
4 changes: 2 additions & 2 deletions gradle.properties
Original file line number Diff line number Diff line change
Expand Up @@ -30,8 +30,8 @@ signing.gnupg.useLegacyGpg=true
# buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy.
# To build a custom Beam version make sure you change it in both places, see
# https://github.com/apache/beam/issues/21302.
version=2.45.18
sdk_version=2.45.18
version=2.45.19
sdk_version=2.45.19

javaVersion=1.8

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,10 +58,6 @@ class FlinkSQLTransformTranslator<InputT, OutputT>
public void translateNode(
PTransform<PCollection<InputT>, PCollection<OutputT>> transform,
FlinkStreamingTranslationContext context) {
if (context.isStreaming()) {
throw new IllegalStateException(
"The current job is a streaming job. Flink SQL transform only support batch jobs.");
}
MultiOutputSqlTransformWithInput<InputT, OutputT> sqlTransform =
(MultiOutputSqlTransformWithInput) transform;
StreamTableEnvironment tEnv = StreamTableEnvironment.create(context.getExecutionEnvironment());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,10 +39,6 @@
* {@link PCollection PCollections}, either {@link SingleOutputSqlTransformWithInput} or {@link
* MultiOutputSqlTransformWithInput} is the way to go.
*
* <p>NOTE: <b>This {@link PTransform} only works with Flink Runner in batch mode.</b>
*
* <p>
*
* <p>
*
* <h1>Specify the input tables</h1>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,10 +44,6 @@
* PCollection PCollections}. The {@link MultiOutputSqlTransformWithInput} differs from {@link
* SingleOutputSqlTransformWithInput} that it supports multiple output {@link PCollection}s.
*
* <p>NOTE: <b>This {@link PTransform} only works with Flink Runner in batch mode.</b>
*
* <p>
*
* <p>
*
* <h1>Specify the input tables</h1>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,10 +37,6 @@
* want to apply a SQL Transform to existing {@link PCollection PCollections}, either {@link
* SingleOutputSqlTransformWithInput} or {@link MultiOutputSqlTransformWithInput} is the way to go.
*
* <p>NOTE: <b>This {@link PTransform} only works with Flink Runner in batch mode.</b>
*
* <p>
*
* <p>
*
* <h1>Specify the input tables</h1>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,10 +37,6 @@
* PCollection PCollections}. The {@link SingleOutputSqlTransformWithInput} differs from the {@link
* MultiOutputSqlTransformWithInput} that it only supports one output {@link PCollection}.
*
* <p>NOTE: <b>This {@link PTransform} only works with Flink Runner in batch mode.</b>
*
* <p>
*
* <p>
*
* <h1>Specify the input tables</h1>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -114,6 +114,16 @@ public static <T> SingleOutputSqlTransform<T> of(Class<T> outputClass) {
return new SingleOutputSqlTransform<>(of(Integer.class, outputClass));
}

/**
* Create a {@link StatementOnlySqlTransform} which takes a full script of SQL statements and
* execute them. The statements must have at least one <code>INSERT INTO</code> statement.
Copy link

@venkata91 venkata91 Mar 2, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@becketqin Sorry for commenting after the PR is merged. Couple of clarifying questions.

  1. What if the SQL statements had a INSERT OVERWRITE instead of INSERT INTO? Is that not a valid StatementOnlySqlTransform?
  2. Similarly, how about having a CREATE TABLE AS SELECT with out INSERT INTO, should we support that as well?

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That is a good point. We should support both. I'll have a follow up patch.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I created the PR. However, currenlty we cannot support CTAS because the OSS Flink StreamStatementSet.attachAsDataStream() does not support that yet. We will need to make change to the OSS Flink first.

*
* @return A {@link StatementOnlySqlTransform}.
*/
public static StatementOnlySqlTransform ofStatements() {
return new StatementOnlySqlTransform();
}

// --------------------- setters ----------------------------
/**
* Use DDL to define Tables. The DDL string can contain multiple {@code CREATE TABLE} / {@code
Expand Down
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);
}
}
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;

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just curious: we can support multiple catalogs here? Curious what's the use case would look like.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, sometimes the datasets may come from different external storage systems. For example, we may have a job reading from Hive, MySql and Kafka at the same time. In this case, there might be three catalogs each for one of the external system.


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());

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shall we log the statements and catalogs here? Seems better if we log them somewhere for debugging purpose.

Btw, there is an overridable "PTransform.validate(PipelineOptions)" method that you can use to do validations. I think we can double check things like statements are not empty and catalog is valid.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We are logging the full statements in the translator. I changed that logging to info and added a debug level logging here. Good point about validation. I added an empty statement check there.

}

@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 + ";";
}
}
Loading
Loading