lindong28 commented on code in PR #175: URL: https://github.com/apache/flink-ml/pull/175#discussion_r1024912755
########## flink-ml-lib/src/test/java/org/apache/flink/ml/feature/SQLTransformerTest.java: ########## @@ -0,0 +1,241 @@ +/* + * 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.ml.feature; + +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.ml.feature.sqltransformer.SQLTransformer; +import org.apache.flink.ml.util.TestUtils; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.types.Row; +import org.apache.flink.types.RowKind; + +import org.apache.commons.collections.IteratorUtils; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +/** Tests {@link SQLTransformer}. */ +public class SQLTransformerTest extends AbstractTestBase { + private static final List<Row> INPUT_DATA = + Arrays.asList(Row.of(0, 1.0, 3.0), Row.of(2, 2.0, 3.0)); + + private static final List<Row> EXPECTED_NUMERIC_DATA_OUTPUT = + Arrays.asList(Row.of(0, 1.0, 3.0, 4.0, 3.0), Row.of(2, 2.0, 3.0, 5.0, 6.0)); + + private static final List<Row> EXPECTED_BUILT_IN_FUNCTION_OUTPUT = + Arrays.asList(Row.of(0, 1.0, 3.0, 1.0), Row.of(2, 2.0, 3.0, Math.sqrt(2.0))); + + private static final List<Row> EXPECTED_BATCH_AGGREGATION_OUTPUT = + Collections.singletonList(Row.of(3.0)); + + private static final List<Row> EXPECTED_STREAMING_AGGREGATION_OUTPUT = + Arrays.asList( + Row.of(1.0), + Row.ofKind(RowKind.UPDATE_BEFORE, 1.0), + Row.ofKind(RowKind.UPDATE_AFTER, 3.0)); + + private StreamTableEnvironment tEnv; + private StreamExecutionEnvironment env; + private Table inputTable; + + @Before + public void before() { + Configuration config = new Configuration(); + config.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, true); + env = StreamExecutionEnvironment.getExecutionEnvironment(config); + env.setParallelism(4); + env.enableCheckpointing(100); + env.setRestartStrategy(RestartStrategies.noRestart()); + tEnv = StreamTableEnvironment.create(env); + DataStream<Row> inputStream = + env.fromCollection( + INPUT_DATA, new RowTypeInfo(Types.INT, Types.DOUBLE, Types.DOUBLE)); + inputTable = tEnv.fromDataStream(inputStream).as("id", "v1", "v2"); + } + + @Test + public void testParam() { + SQLTransformer sqlTransformer = new SQLTransformer(); + sqlTransformer.setStatement("SELECT * FROM __THIS__"); + assertEquals("SELECT * FROM __THIS__", sqlTransformer.getStatement()); + } + + @Test + public void testInvalidSQLStatement() { + SQLTransformer sqlTransformer = new SQLTransformer(); + + try { + sqlTransformer.setStatement("SELECT * FROM __THAT__"); + fail(); + } catch (Exception e) { + assertEquals( + "Parameter statement is given an invalid value SELECT * FROM __THAT__", + e.getMessage()); + } + } + + @Test + public void testOutputSchema() { + SQLTransformer sqlTransformer = + new SQLTransformer() + .setStatement("SELECT *, (v1 + v2) AS v3, (v1 * v2) AS v4 FROM __THIS__"); + + Table outputTable = sqlTransformer.transform(inputTable)[0]; + + assertEquals( + Arrays.asList("id", "v1", "v2", "v3", "v4"), + outputTable.getResolvedSchema().getColumnNames()); + } + + @Test + public void testTransformNumericData() { + SQLTransformer sqlTransformer = + new SQLTransformer() + .setStatement("SELECT *, (v1 + v2) AS v3, (v1 * v2) AS v4 FROM __THIS__"); + + Table outputTable = sqlTransformer.transform(inputTable)[0]; + + verifyOutputResult(outputTable, EXPECTED_NUMERIC_DATA_OUTPUT); + } + + @Test + public void testBuiltInFunction() { + SQLTransformer sqlTransformer = + new SQLTransformer().setStatement("SELECT *, SQRT(v1) AS v3 FROM __THIS__"); + + Table outputTable = sqlTransformer.transform(inputTable)[0]; + + verifyOutputResult(outputTable, EXPECTED_BUILT_IN_FUNCTION_OUTPUT); + } + + @Test + public void testBatchAggregation() { + env.setRuntimeMode(RuntimeExecutionMode.BATCH); + tEnv = StreamTableEnvironment.create(env); + DataStream<Row> inputStream = + env.fromCollection( + INPUT_DATA, new RowTypeInfo(Types.INT, Types.DOUBLE, Types.DOUBLE)); + inputTable = tEnv.fromDataStream(inputStream).as("id", "v1", "v2"); + + SQLTransformer sqlTransformer = + new SQLTransformer().setStatement("SELECT SUM(v1) AS v3 FROM __THIS__ GROUP BY v2"); + + Table outputTable = sqlTransformer.transform(inputTable)[0]; + + verifyOutputResult(outputTable, EXPECTED_BATCH_AGGREGATION_OUTPUT, false); + } + + @Test + public void testStreamingAggregation() { + env.setRuntimeMode(RuntimeExecutionMode.STREAMING); + tEnv = StreamTableEnvironment.create(env); + DataStream<Row> inputStream = + env.fromCollection( + INPUT_DATA, new RowTypeInfo(Types.INT, Types.DOUBLE, Types.DOUBLE)); + inputTable = tEnv.fromDataStream(inputStream).as("id", "v1", "v2"); + + SQLTransformer sqlTransformer = + new SQLTransformer().setStatement("SELECT SUM(v1) AS v3 FROM __THIS__ GROUP BY v2"); + + Table outputTable = sqlTransformer.transform(inputTable)[0]; + + verifyOutputResult(outputTable, EXPECTED_STREAMING_AGGREGATION_OUTPUT, false); Review Comment: Is there any operator in Flink ML that can handle upsert events? If no, it seems better to have this query also output EXPECTED_BATCH_AGGREGATION_OUTPUT when runtimeMode=Batch. One benefit of this approach is to have all Flink ML operators be able to generate reason output (that can be used in the machine learning context) in streaming mode. ########## flink-ml-lib/src/test/java/org/apache/flink/ml/feature/SQLTransformerTest.java: ########## @@ -0,0 +1,241 @@ +/* + * 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.ml.feature; + +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.ml.feature.sqltransformer.SQLTransformer; +import org.apache.flink.ml.util.TestUtils; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.types.Row; +import org.apache.flink.types.RowKind; + +import org.apache.commons.collections.IteratorUtils; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +/** Tests {@link SQLTransformer}. */ +public class SQLTransformerTest extends AbstractTestBase { + private static final List<Row> INPUT_DATA = + Arrays.asList(Row.of(0, 1.0, 3.0), Row.of(2, 2.0, 3.0)); + + private static final List<Row> EXPECTED_NUMERIC_DATA_OUTPUT = + Arrays.asList(Row.of(0, 1.0, 3.0, 4.0, 3.0), Row.of(2, 2.0, 3.0, 5.0, 6.0)); + + private static final List<Row> EXPECTED_BUILT_IN_FUNCTION_OUTPUT = + Arrays.asList(Row.of(0, 1.0, 3.0, 1.0), Row.of(2, 2.0, 3.0, Math.sqrt(2.0))); + + private static final List<Row> EXPECTED_BATCH_AGGREGATION_OUTPUT = + Collections.singletonList(Row.of(3.0)); + + private static final List<Row> EXPECTED_STREAMING_AGGREGATION_OUTPUT = + Arrays.asList( + Row.of(1.0), + Row.ofKind(RowKind.UPDATE_BEFORE, 1.0), + Row.ofKind(RowKind.UPDATE_AFTER, 3.0)); + + private StreamTableEnvironment tEnv; + private StreamExecutionEnvironment env; + private Table inputTable; + + @Before + public void before() { + Configuration config = new Configuration(); + config.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, true); + env = StreamExecutionEnvironment.getExecutionEnvironment(config); + env.setParallelism(4); + env.enableCheckpointing(100); + env.setRestartStrategy(RestartStrategies.noRestart()); + tEnv = StreamTableEnvironment.create(env); + DataStream<Row> inputStream = + env.fromCollection( + INPUT_DATA, new RowTypeInfo(Types.INT, Types.DOUBLE, Types.DOUBLE)); + inputTable = tEnv.fromDataStream(inputStream).as("id", "v1", "v2"); + } + + @Test + public void testParam() { + SQLTransformer sqlTransformer = new SQLTransformer(); + sqlTransformer.setStatement("SELECT * FROM __THIS__"); + assertEquals("SELECT * FROM __THIS__", sqlTransformer.getStatement()); + } + + @Test + public void testInvalidSQLStatement() { + SQLTransformer sqlTransformer = new SQLTransformer(); + + try { + sqlTransformer.setStatement("SELECT * FROM __THAT__"); + fail(); + } catch (Exception e) { + assertEquals( + "Parameter statement is given an invalid value SELECT * FROM __THAT__", + e.getMessage()); + } + } + + @Test + public void testOutputSchema() { + SQLTransformer sqlTransformer = + new SQLTransformer() + .setStatement("SELECT *, (v1 + v2) AS v3, (v1 * v2) AS v4 FROM __THIS__"); + + Table outputTable = sqlTransformer.transform(inputTable)[0]; + + assertEquals( + Arrays.asList("id", "v1", "v2", "v3", "v4"), + outputTable.getResolvedSchema().getColumnNames()); + } + + @Test + public void testTransformNumericData() { + SQLTransformer sqlTransformer = + new SQLTransformer() + .setStatement("SELECT *, (v1 + v2) AS v3, (v1 * v2) AS v4 FROM __THIS__"); + + Table outputTable = sqlTransformer.transform(inputTable)[0]; + + verifyOutputResult(outputTable, EXPECTED_NUMERIC_DATA_OUTPUT); + } + + @Test + public void testBuiltInFunction() { + SQLTransformer sqlTransformer = + new SQLTransformer().setStatement("SELECT *, SQRT(v1) AS v3 FROM __THIS__"); + + Table outputTable = sqlTransformer.transform(inputTable)[0]; + + verifyOutputResult(outputTable, EXPECTED_BUILT_IN_FUNCTION_OUTPUT); + } + + @Test + public void testBatchAggregation() { + env.setRuntimeMode(RuntimeExecutionMode.BATCH); + tEnv = StreamTableEnvironment.create(env); + DataStream<Row> inputStream = + env.fromCollection( + INPUT_DATA, new RowTypeInfo(Types.INT, Types.DOUBLE, Types.DOUBLE)); + inputTable = tEnv.fromDataStream(inputStream).as("id", "v1", "v2"); + + SQLTransformer sqlTransformer = + new SQLTransformer().setStatement("SELECT SUM(v1) AS v3 FROM __THIS__ GROUP BY v2"); + + Table outputTable = sqlTransformer.transform(inputTable)[0]; + + verifyOutputResult(outputTable, EXPECTED_BATCH_AGGREGATION_OUTPUT, false); + } + + @Test + public void testStreamingAggregation() { + env.setRuntimeMode(RuntimeExecutionMode.STREAMING); Review Comment: Runtime mode is streaming by default. It seems better to remove this line for consistency with existing tests. ########## flink-ml-lib/src/test/java/org/apache/flink/ml/feature/SQLTransformerTest.java: ########## @@ -0,0 +1,241 @@ +/* + * 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.ml.feature; + +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.ml.feature.sqltransformer.SQLTransformer; +import org.apache.flink.ml.util.TestUtils; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.types.Row; +import org.apache.flink.types.RowKind; + +import org.apache.commons.collections.IteratorUtils; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +/** Tests {@link SQLTransformer}. */ +public class SQLTransformerTest extends AbstractTestBase { + private static final List<Row> INPUT_DATA = + Arrays.asList(Row.of(0, 1.0, 3.0), Row.of(2, 2.0, 3.0)); + + private static final List<Row> EXPECTED_NUMERIC_DATA_OUTPUT = + Arrays.asList(Row.of(0, 1.0, 3.0, 4.0, 3.0), Row.of(2, 2.0, 3.0, 5.0, 6.0)); + + private static final List<Row> EXPECTED_BUILT_IN_FUNCTION_OUTPUT = + Arrays.asList(Row.of(0, 1.0, 3.0, 1.0), Row.of(2, 2.0, 3.0, Math.sqrt(2.0))); + + private static final List<Row> EXPECTED_BATCH_AGGREGATION_OUTPUT = + Collections.singletonList(Row.of(3.0)); + + private static final List<Row> EXPECTED_STREAMING_AGGREGATION_OUTPUT = + Arrays.asList( + Row.of(1.0), + Row.ofKind(RowKind.UPDATE_BEFORE, 1.0), + Row.ofKind(RowKind.UPDATE_AFTER, 3.0)); + + private StreamTableEnvironment tEnv; + private StreamExecutionEnvironment env; + private Table inputTable; + + @Before + public void before() { + Configuration config = new Configuration(); + config.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, true); + env = StreamExecutionEnvironment.getExecutionEnvironment(config); + env.setParallelism(4); + env.enableCheckpointing(100); + env.setRestartStrategy(RestartStrategies.noRestart()); + tEnv = StreamTableEnvironment.create(env); + DataStream<Row> inputStream = + env.fromCollection( + INPUT_DATA, new RowTypeInfo(Types.INT, Types.DOUBLE, Types.DOUBLE)); + inputTable = tEnv.fromDataStream(inputStream).as("id", "v1", "v2"); + } + + @Test + public void testParam() { + SQLTransformer sqlTransformer = new SQLTransformer(); + sqlTransformer.setStatement("SELECT * FROM __THIS__"); + assertEquals("SELECT * FROM __THIS__", sqlTransformer.getStatement()); + } + + @Test + public void testInvalidSQLStatement() { + SQLTransformer sqlTransformer = new SQLTransformer(); + + try { + sqlTransformer.setStatement("SELECT * FROM __THAT__"); + fail(); + } catch (Exception e) { + assertEquals( + "Parameter statement is given an invalid value SELECT * FROM __THAT__", + e.getMessage()); + } + } + + @Test + public void testOutputSchema() { + SQLTransformer sqlTransformer = + new SQLTransformer() + .setStatement("SELECT *, (v1 + v2) AS v3, (v1 * v2) AS v4 FROM __THIS__"); + + Table outputTable = sqlTransformer.transform(inputTable)[0]; + + assertEquals( + Arrays.asList("id", "v1", "v2", "v3", "v4"), + outputTable.getResolvedSchema().getColumnNames()); + } + + @Test + public void testTransformNumericData() { + SQLTransformer sqlTransformer = + new SQLTransformer() + .setStatement("SELECT *, (v1 + v2) AS v3, (v1 * v2) AS v4 FROM __THIS__"); + + Table outputTable = sqlTransformer.transform(inputTable)[0]; + + verifyOutputResult(outputTable, EXPECTED_NUMERIC_DATA_OUTPUT); + } + + @Test + public void testBuiltInFunction() { + SQLTransformer sqlTransformer = + new SQLTransformer().setStatement("SELECT *, SQRT(v1) AS v3 FROM __THIS__"); + + Table outputTable = sqlTransformer.transform(inputTable)[0]; + + verifyOutputResult(outputTable, EXPECTED_BUILT_IN_FUNCTION_OUTPUT); + } + + @Test + public void testBatchAggregation() { + env.setRuntimeMode(RuntimeExecutionMode.BATCH); + tEnv = StreamTableEnvironment.create(env); + DataStream<Row> inputStream = + env.fromCollection( + INPUT_DATA, new RowTypeInfo(Types.INT, Types.DOUBLE, Types.DOUBLE)); + inputTable = tEnv.fromDataStream(inputStream).as("id", "v1", "v2"); + + SQLTransformer sqlTransformer = + new SQLTransformer().setStatement("SELECT SUM(v1) AS v3 FROM __THIS__ GROUP BY v2"); + + Table outputTable = sqlTransformer.transform(inputTable)[0]; + + verifyOutputResult(outputTable, EXPECTED_BATCH_AGGREGATION_OUTPUT, false); + } + + @Test + public void testStreamingAggregation() { + env.setRuntimeMode(RuntimeExecutionMode.STREAMING); + tEnv = StreamTableEnvironment.create(env); + DataStream<Row> inputStream = + env.fromCollection( + INPUT_DATA, new RowTypeInfo(Types.INT, Types.DOUBLE, Types.DOUBLE)); + inputTable = tEnv.fromDataStream(inputStream).as("id", "v1", "v2"); + + SQLTransformer sqlTransformer = + new SQLTransformer().setStatement("SELECT SUM(v1) AS v3 FROM __THIS__ GROUP BY v2"); + + Table outputTable = sqlTransformer.transform(inputTable)[0]; + + verifyOutputResult(outputTable, EXPECTED_STREAMING_AGGREGATION_OUTPUT, false); + } + + @Test + public void testWindow() { + env.setRuntimeMode(RuntimeExecutionMode.STREAMING); + tEnv = StreamTableEnvironment.create(env); + + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("v1", DataTypes.DOUBLE()) + .column("v2", DataTypes.DOUBLE()) + .columnByExpression("time_ltz", "TO_TIMESTAMP_LTZ(id * 1000, 3)") + .watermark("time_ltz", "time_ltz - INTERVAL '5' SECOND") + .build(); + + DataStream<Row> inputStream = + env.fromCollection( + INPUT_DATA, + new RowTypeInfo( + new TypeInformation[] {Types.INT, Types.DOUBLE, Types.DOUBLE}, + new String[] {"id", "v1", "v2"})); + inputTable = tEnv.fromDataStream(inputStream, schema); + + String statement = + "SELECT SUM(v1) AS v3 " + + "FROM TABLE(TUMBLE(TABLE __THIS__, DESCRIPTOR(time_ltz), INTERVAL '10' MINUTES)) " + + "GROUP BY window_start, window_end"; + + SQLTransformer sqlTransformer = new SQLTransformer().setStatement(statement); + + Table outputTable = sqlTransformer.transform(inputTable)[0]; + + verifyOutputResult(outputTable, EXPECTED_BATCH_AGGREGATION_OUTPUT, false); Review Comment: It seems a bit confusing to verify that the output of this query is `EXPECTED_BATCH_AGGREGATION_OUTPUT` when this query is actually executed with streaming mode. ########## flink-ml-lib/src/main/java/org/apache/flink/ml/feature/sqltransformer/SQLTransformer.java: ########## @@ -0,0 +1,76 @@ +/* + * 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.ml.feature.sqltransformer; + +import org.apache.flink.ml.api.Transformer; +import org.apache.flink.ml.param.Param; +import org.apache.flink.ml.util.ParamUtils; +import org.apache.flink.ml.util.ReadWriteUtils; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.internal.TableImpl; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +/** + * SQLTransformer implements the transformations that are defined by SQL statement. + * + * <p>Currently we only support SQL syntax like 'SELECT ... FROM __THIS__ ...' where "__THIS__" Review Comment: Can we also support `GROUP BY`? When the statement includes `group by`, this operator can emit results at the end of stream, similar to the behavior of many existing operators in Flink ML. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
