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

[FLINK-11421][Table API & SQL] Providing more compilation options for code-generated o… #7680

Closed
wants to merge 1 commit into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
@@ -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://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.api.common;

/**
* The option for the compiler used for compiling generated Java code.
*/
public enum CompilationOption {

/**
* Compiling Java code by Janino.
* The compilation is fast, but the generated binary code is of low quality.
*/
FAST,

/**
* Compiling Java code by Java Compiler API (JCA)
* The compilation is slow, but the generated binary code is of high quality.
*/
SLOW;

public static boolean inTest = false;

public static CompilationOption currentOption = FAST;
}
Expand Up @@ -162,6 +162,9 @@ public class ExecutionConfig implements Serializable, Archiveable<ArchivedExecut
/** The default input dependency constraint to schedule tasks. */
private InputDependencyConstraint defaultInputDependencyConstraint = InputDependencyConstraint.ANY;

/** The option for compiling generated Java code. */
private CompilationOption compilationOption = CompilationOption.FAST;

// ------------------------------- User code values --------------------------------------------

private GlobalJobParameters globalJobParameters;
Expand Down Expand Up @@ -521,6 +524,22 @@ public ExecutionMode getExecutionMode() {
return executionMode;
}

/**
* Sets the option for compiling generated Java code.
* @param compilationOption
*/
public void setCompilationOption(CompilationOption compilationOption) {
this.compilationOption = compilationOption;
}

/**
* Gets the option for compiling generated Java code.
* @return
*/
public CompilationOption getCompileOption() {
return this.compilationOption;
}

/**
* Sets the default input dependency constraint for vertex scheduling. It indicates when a task
* should be scheduled considering its inputs status.
Expand Down
Expand Up @@ -105,6 +105,12 @@ public class CoreOptions {
" resolved through the parent ClassLoader first. A pattern is a simple prefix that is checked against" +
" the fully qualified class name. These patterns are appended to \"" + ALWAYS_PARENT_FIRST_LOADER_PATTERNS.key() + "\".");

public static final ConfigOption<String> CODEGEN_COMPILATION_OPTION = ConfigOptions
.key("codegen.compilation.option")
.defaultValue("fast")
.withDescription("A string indicating the option used for compiling generated code. 'fast' means compiling with Janino, " +
"whereas 'slow' means compiling with Java Compiler API (JCA).");

public static String[] getParentFirstLoaderPatterns(Configuration config) {
String base = config.getString(ALWAYS_PARENT_FIRST_LOADER_PATTERNS);
String append = config.getString(ALWAYS_PARENT_FIRST_LOADER_PATTERNS_ADDITIONAL);
Expand Down
Expand Up @@ -18,7 +18,9 @@

package org.apache.flink.table.codegen

import org.apache.flink.api.common.InvalidProgramException
import org.apache.flink.api.common.{CompilationOption, InvalidProgramException}
import org.apache.flink.api.common.functions.RuntimeContext
import org.apache.flink.streaming.util.JCACompiler
import org.codehaus.commons.compiler.CompileException
import org.codehaus.janino.SimpleCompiler

Expand All @@ -38,4 +40,13 @@ trait Compiler[T] {
}
compiler.getClassLoader.loadClass(name).asInstanceOf[Class[T]]
}

@throws(classOf[CompileException])
def compile(ctx: RuntimeContext, name: String, code: String): Class[T] = {
if (ctx.getExecutionConfig.getCompileOption == CompilationOption.FAST) {
compile(ctx.getUserCodeClassLoader, name, code)
} else {
JCACompiler.getInstance.getCodeClass(name, code).asInstanceOf[Class[T]]
}
}
}
Expand Up @@ -22,7 +22,7 @@ import org.apache.flink.api.common.functions.util.FunctionUtils
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.api.java.typeutils.ResultTypeQueryable
import org.apache.flink.configuration.Configuration
import org.apache.flink.streaming.api.functions.ProcessFunction
import org.apache.flink.streaming.api.functions.{CodeGenFunction, ProcessFunction}
import org.apache.flink.streaming.api.operators.TimestampedCollector
import org.apache.flink.table.codegen.Compiler
import org.apache.flink.table.runtime.types.CRow
Expand All @@ -40,14 +40,15 @@ class CRowOutputProcessRunner(
extends ProcessFunction[Any, CRow]
with ResultTypeQueryable[CRow]
with Compiler[ProcessFunction[Any, Row]]
with Logging {
with Logging
with CodeGenFunction{

private var function: ProcessFunction[Any, Row] = _
private var cRowWrapper: CRowWrappingCollector = _

override def open(parameters: Configuration): Unit = {
LOG.debug(s"Compiling ProcessFunction: $name \n\n Code:\n$code")
val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
val clazz = compile(getRuntimeContext, name, code)
LOG.debug("Instantiating ProcessFunction.")
function = clazz.newInstance()
FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext)
Expand Down Expand Up @@ -75,4 +76,8 @@ class CRowOutputProcessRunner(
override def close(): Unit = {
FunctionUtils.closeFunction(function)
}

override def getName: String = name

override def getCode: String = code
}
Expand Up @@ -22,7 +22,7 @@ import org.apache.flink.api.common.functions.util.FunctionUtils
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.api.java.typeutils.ResultTypeQueryable
import org.apache.flink.configuration.Configuration
import org.apache.flink.streaming.api.functions.ProcessFunction
import org.apache.flink.streaming.api.functions.{CodeGenFunction, ProcessFunction}
import org.apache.flink.table.codegen.Compiler
import org.apache.flink.table.runtime.types.CRow
import org.apache.flink.table.util.Logging
Expand All @@ -39,14 +39,15 @@ class CRowProcessRunner(
extends ProcessFunction[CRow, CRow]
with ResultTypeQueryable[CRow]
with Compiler[ProcessFunction[Row, Row]]
with Logging {
with Logging
with CodeGenFunction {

private var function: ProcessFunction[Row, Row] = _
private var cRowWrapper: CRowWrappingCollector = _

override def open(parameters: Configuration): Unit = {
LOG.debug(s"Compiling ProcessFunction: $name \n\n Code:\n$code")
val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
val clazz = compile(getRuntimeContext, name, code)
LOG.debug("Instantiating ProcessFunction.")
function = clazz.newInstance()
FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext)
Expand Down Expand Up @@ -74,6 +75,10 @@ class CRowProcessRunner(
override def close(): Unit = {
FunctionUtils.closeFunction(function)
}

override def getName: String = name

override def getCode: String = code
}


Expand Up @@ -23,6 +23,7 @@ import org.apache.flink.api.common.functions.{FlatMapFunction, RichFlatMapFuncti
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.api.java.typeutils.ResultTypeQueryable
import org.apache.flink.configuration.Configuration
import org.apache.flink.streaming.api.functions.CodeGenFunction
import org.apache.flink.table.codegen.Compiler
import org.apache.flink.table.util.Logging
import org.apache.flink.types.Row
Expand All @@ -35,7 +36,8 @@ class FlatMapRunner(
extends RichFlatMapFunction[Row, Row]
with ResultTypeQueryable[Row]
with Compiler[FlatMapFunction[Row, Row]]
with Logging {
with Logging
with CodeGenFunction {

private var function: FlatMapFunction[Row, Row] = _

Expand All @@ -56,4 +58,8 @@ class FlatMapRunner(
override def close(): Unit = {
FunctionUtils.closeFunction(function)
}

override def getName: String = name

override def getCode: String = code
}
Expand Up @@ -18,8 +18,10 @@

package org.apache.flink.table.runtime.stream.sql

import org.apache.flink.api.common.CompilationOption
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.api.scala._
import org.apache.flink.configuration.{ConfigOptions, CoreOptions}
import org.apache.flink.streaming.api.TimeCharacteristic
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction
import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
Expand All @@ -32,10 +34,25 @@ import org.apache.flink.table.utils.MemoryTableSourceSinkUtil
import org.apache.flink.types.Row
import org.junit.Assert._
import org.junit._
import org.junit.runner.RunWith
import org.junit.runners.Parameterized

import scala.collection.mutable

class SortITCase extends StreamingWithStateTestBase {
@RunWith(classOf[Parameterized])
class SortITCase (compilationOption: CompilationOption) extends StreamingWithStateTestBase {

@Before
def before(): Unit = {
CompilationOption.inTest = true
CompilationOption.currentOption = compilationOption
}

@After
def after(): Unit = {
CompilationOption.inTest = false
CompilationOption.currentOption = CompilationOption.FAST
}

@Test
def testEventTimeOrderBy(): Unit = {
Expand Down Expand Up @@ -72,27 +89,28 @@ class SortITCase extends StreamingWithStateTestBase {
Right(14000L),
Left((15000L, (8L, 8, "Hello World"))),
Right(17000L),
Left((20000L, (20L, 20, "Hello World"))),
Left((20000L, (20L, 20, "Hello World"))),
Right(19000L))

val env = StreamExecutionEnvironment.getExecutionEnvironment
env.setStateBackend(getStateBackend)
env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
val tEnv = TableEnvironment.getTableEnvironment(env)
StreamITCase.clear
SortITCase.clear

val t1 = env.addSource(new EventTimeSourceFunction[(Long, Int, String)](data))
.toTable(tEnv, 'a, 'b, 'c, 'rowtime.rowtime)

tEnv.registerTable("T1", t1)

val sqlQuery = "SELECT b FROM T1 ORDER BY rowtime, b ASC "


val result = tEnv.sqlQuery(sqlQuery).toAppendStream[Row]
result.addSink(new StringRowSelectorSink(0)).setParallelism(1)
env.execute()

val expected = mutable.MutableList(
"1", "15", "16",
"1", "2", "2", "3",
Expand All @@ -101,7 +119,7 @@ class SortITCase extends StreamingWithStateTestBase {
"5",
"-1", "6", "6", "65", "67",
"18", "7", "9",
"7", "17", "77",
"7", "17", "77",
"18",
"8",
"20")
Expand Down Expand Up @@ -150,4 +168,11 @@ object SortITCase {
}

var testResults: mutable.MutableList[String] = mutable.MutableList.empty[String]

def clear = testResults.clear()

@Parameterized.Parameters(name = "compilationOption = {0}")
def compilationConfig(): Array[CompilationOption] = {
Array(CompilationOption.FAST, CompilationOption.SLOW)
}
}
@@ -0,0 +1,37 @@
/*
* 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.streaming.api.functions;

/**
* A function based on generated Java code.
*/
public interface CodeGenFunction {

/**
* Gets the class name of generated code.
* @return
*/
String getName();

/**
* Gets the generated code.
* @return
*/
String getCode();
}
Expand Up @@ -18,6 +18,7 @@
package org.apache.flink.streaming.api.graph;

import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.CompilationOption;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.io.InputFormat;
Expand All @@ -28,6 +29,8 @@
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
import org.apache.flink.api.java.typeutils.MissingTypeInfo;
import org.apache.flink.configuration.CoreOptions;
import org.apache.flink.configuration.GlobalConfiguration;
import org.apache.flink.optimizer.plan.StreamingPlan;
import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
Expand Down Expand Up @@ -103,6 +106,11 @@ public StreamGraph(StreamExecutionEnvironment environment) {
this.executionConfig = environment.getConfig();
this.checkpointConfig = environment.getCheckpointConfig();

// get compilation option
CompilationOption compilationOption = CompilationOption.inTest ? CompilationOption.currentOption :
CompilationOption.valueOf(GlobalConfiguration.loadConfiguration().getString(CoreOptions.CODEGEN_COMPILATION_OPTION).toUpperCase());
this.executionConfig.setCompilationOption(compilationOption);

// create an empty new stream graph.
clear();
}
Expand Down