Skip to content
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.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
38 changes: 38 additions & 0 deletions flink-python/pyflink/table/tests/test_udaf.py
Original file line number Diff line number Diff line change
Expand Up @@ -785,6 +785,44 @@ def test_session_group_window_over_time(self):
"+I[1, 2018-03-11 03:10:00.0, 2018-03-11 04:10:00.0, 2]",
"+I[1, 2018-03-11 04:20:00.0, 2018-03-11 04:50:00.0, 1]"])

def test_execute_group_aggregate_from_json_plan(self):
# create source file path
tmp_dir = self.tempdir
data = ['1,1', '3,2', '1,3']
source_path = tmp_dir + '/test_execute_group_aggregate_from_json_plan.csv'
with open(source_path, 'w') as fd:
for ele in data:
fd.write(ele + '\n')

source_table = """
CREATE TABLE source_table (
a BIGINT,
b BIGINT
) WITH (
'connector' = 'filesystem',
'path' = '%s',
'format' = 'csv'
)
""" % source_path
self.t_env.execute_sql(source_table)

self.t_env.execute_sql("""
CREATE TABLE sink_table (
a BIGINT,
b BIGINT
) WITH (
'connector' = 'blackhole'
)
""")

self.t_env.create_temporary_function("my_sum", SumAggregateFunction())

json_plan = self.t_env._j_tenv.getJsonPlan("INSERT INTO sink_table "
"SELECT a, my_sum(b) FROM source_table "
"GROUP BY a")
from py4j.java_gateway import get_method
get_method(self.t_env._j_tenv.executeJsonPlan(json_plan), "await")()

def test_execute_group_window_aggregate_from_json_plan(self):
# create source file path
tmp_dir = self.tempdir
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,9 +31,7 @@
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.utils.CommonPythonUtil;
import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
import org.apache.flink.table.planner.plan.utils.AggregateUtil;
Expand All @@ -44,26 +42,41 @@
import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.table.types.logical.RowType;

import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;

import org.apache.calcite.rel.core.AggregateCall;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.lang.reflect.Constructor;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;

import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;

/** Stream {@link ExecNode} for Python unbounded group aggregate. */
public class StreamExecPythonGroupAggregate extends ExecNodeBase<RowData>
implements StreamExecNode<RowData>, SingleTransformationTranslator<RowData> {
public class StreamExecPythonGroupAggregate extends StreamExecAggregateBase {

private static final Logger LOG = LoggerFactory.getLogger(StreamExecPythonGroupAggregate.class);
private static final String PYTHON_STREAM_AGGREAGTE_OPERATOR_NAME =
"org.apache.flink.table.runtime.operators.python.aggregate.PythonStreamGroupAggregateOperator";

@JsonProperty(FIELD_NAME_GROUPING)
private final int[] grouping;

@JsonProperty(FIELD_NAME_AGG_CALLS)
private final AggregateCall[] aggCalls;

@JsonProperty(FIELD_NAME_AGG_CALL_NEED_RETRACTIONS)
private final boolean[] aggCallNeedRetractions;

@JsonProperty(FIELD_NAME_GENERATE_UPDATE_BEFORE)
private final boolean generateUpdateBefore;

@JsonProperty(FIELD_NAME_NEED_RETRACTION)
private final boolean needRetraction;

public StreamExecPythonGroupAggregate(
Expand All @@ -75,10 +88,34 @@ public StreamExecPythonGroupAggregate(
InputProperty inputProperty,
RowType outputType,
String description) {
super(Collections.singletonList(inputProperty), outputType, description);
this.grouping = grouping;
this.aggCalls = aggCalls;
this.aggCallNeedRetractions = aggCallNeedRetractions;
this(
grouping,
aggCalls,
aggCallNeedRetractions,
generateUpdateBefore,
needRetraction,
getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
}

@JsonCreator
public StreamExecPythonGroupAggregate(
@JsonProperty(FIELD_NAME_GROUPING) int[] grouping,
@JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls,
@JsonProperty(FIELD_NAME_AGG_CALL_NEED_RETRACTIONS) boolean[] aggCallNeedRetractions,
@JsonProperty(FIELD_NAME_GENERATE_UPDATE_BEFORE) boolean generateUpdateBefore,
@JsonProperty(FIELD_NAME_NEED_RETRACTION) boolean needRetraction,
@JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
super(id, inputProperties, outputType, description);
this.grouping = checkNotNull(grouping);
this.aggCalls = checkNotNull(aggCalls);
this.aggCallNeedRetractions = checkNotNull(aggCallNeedRetractions);
checkArgument(aggCalls.length == aggCallNeedRetractions.length);
this.generateUpdateBefore = generateUpdateBefore;
this.needRetraction = needRetraction;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,6 @@ public class JsonSerdeCoverageTest {
"StreamExecDataStreamScan",
"StreamExecLegacyTableSourceScan",
"StreamExecLegacySink",
"StreamExecPythonGroupAggregate",
"StreamExecWindowTableFunction",
"StreamExecGroupTableAggregate",
"StreamExecPythonGroupTableAggregate",
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,70 @@
/*
* 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.table.planner.plan.nodes.exec.stream;

import org.apache.flink.table.api.TableConfig;
import org.apache.flink.table.api.TableEnvironment;
import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedAggFunctions.TestPythonAggregateFunction;
import org.apache.flink.table.planner.utils.StreamTableTestUtil;
import org.apache.flink.table.planner.utils.TableTestBase;

import org.junit.Before;
import org.junit.Test;

/** Test json serialization/deserialization for group aggregate. */
public class PythonGroupAggregateJsonPlanTest extends TableTestBase {

private StreamTableTestUtil util;
private TableEnvironment tEnv;

@Before
public void setup() {
util = streamTestUtil(TableConfig.getDefault());
tEnv = util.getTableEnv();

String srcTableDdl =
"CREATE TABLE MyTable (\n"
+ " a int not null,\n"
+ " b int not null,\n"
+ " c int not null,\n"
+ " d bigint\n"
+ ") with (\n"
+ " 'connector' = 'values',\n"
+ " 'bounded' = 'false')";
tEnv.executeSql(srcTableDdl);
tEnv.createTemporarySystemFunction("pyFunc", new TestPythonAggregateFunction());
}

@Test
public void tesPythonAggCallsWithGroupBy() {
String sinkTableDdl =
"CREATE TABLE MySink (\n"
+ " a bigint,\n"
+ " b bigint\n"
+ ") with (\n"
+ " 'connector' = 'values',\n"
+ " 'sink-insert-only' = 'false',\n"
+ " 'table-sink-class' = 'DEFAULT')";
tEnv.executeSql(sinkTableDdl);
util.verifyJsonPlan(
"insert into MySink select b, "
+ "pyFunc(a, c) filter (where b > 1) "
+ "from MyTable group by b");
}
}
Loading