Skip to content
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 @@ -29,6 +29,8 @@
import org.apache.flink.table.api.Table;
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;

import javax.annotation.Nullable;

import java.lang.reflect.Method;
import java.lang.reflect.Modifier;
import java.util.HashMap;
Expand Down Expand Up @@ -63,10 +65,11 @@ protected AgentsExecutionEnvironment() {
*
* @param env Optional StreamExecutionEnvironment for remote execution. If null, a local
* execution environment will be created.
* @param tEnv Optional StreamTableEnvironment for table-to-stream conversion.
* @return AgentsExecutionEnvironment appropriate for the execution context.
*/
public static AgentsExecutionEnvironment getExecutionEnvironment(
StreamExecutionEnvironment env) {
StreamExecutionEnvironment env, @Nullable StreamTableEnvironment tEnv) {
if (env == null) {
// Return local execution environment for testing/development
try {
Expand All @@ -86,14 +89,31 @@ public static AgentsExecutionEnvironment getExecutionEnvironment(
"org.apache.flink.agents.runtime.env.RemoteExecutionEnvironment");
return (AgentsExecutionEnvironment)
remoteEnvClass
.getDeclaredConstructor(StreamExecutionEnvironment.class)
.newInstance(env);
.getDeclaredConstructor(
StreamExecutionEnvironment.class,
StreamTableEnvironment.class)
.newInstance(env, tEnv);
} catch (Exception e) {
throw new RuntimeException("Failed to create RemoteExecutionEnvironment", e);
}
}
}

/**
* Convenience method to get execution environment without Flink StreamTableEnvironment. If
* StreamTableEnvironment is needed during execution, the environment will auto crate using
* StreamExecutionEnvironment.
*
* <p>* @param env Optional StreamExecutionEnvironment for remote execution. If null, a local
* execution environment will be created.
*
* @return Remote execution environment for testing and development.
*/
public static AgentsExecutionEnvironment getExecutionEnvironment(
StreamExecutionEnvironment env) {
return getExecutionEnvironment(env, null);
}

/**
* Convenience method to get execution environment without Flink integration.
*
Expand Down Expand Up @@ -155,23 +175,20 @@ public <T> AgentBuilder fromDataStream(DataStream<T> input) {
* and processing it through agents.
*
* @param input Table to be processed by agents.
* @param tableEnv StreamTableEnvironment for table-to-stream conversion.
* @param keySelector Optional KeySelector for extracting keys from table rows.
* @param <K> Type of the key extracted by the KeySelector.
* @return AgentBuilder for configuring the agent pipeline.
*/
public abstract <K> AgentBuilder fromTable(
Table input, StreamTableEnvironment tableEnv, KeySelector<Object, K> keySelector);
public abstract <K> AgentBuilder fromTable(Table input, KeySelector<Object, K> keySelector);

/**
* Set input for agents from a Table without keying.
*
* @param input Table to be processed by agents.
* @param tableEnv StreamTableEnvironment for table-to-stream conversion.
* @return AgentBuilder for configuring the agent pipeline.
*/
public AgentBuilder fromTable(Table input, StreamTableEnvironment tableEnv) {
return fromTable(input, tableEnv, null);
public AgentBuilder fromTable(Table input) {
return fromTable(input, null);
}

/**
Expand Down
69 changes: 69 additions & 0 deletions e2e-test/integration-test/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,69 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.flink</groupId>
<artifactId>flink-agents-e2e-tests</artifactId>
<version>0.1-SNAPSHOT</version>
</parent>

<artifactId>flink-agents-integration-tests</artifactId>
<name>Flink Agents : E2E Tests: Integration</name>

<dependencies>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-agents-api</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-agents-runtime</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java</artifactId>
<version>${flink.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-api-java-bridge</artifactId>
<version>${flink.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-planner_2.12</artifactId>
<version>${flink.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-clients</artifactId>
<version>${flink.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-agents-integrations-chat-models-ollama</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>

</project>
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* limitations under the License.
*/

package org.apache.flink.agents.examples;
package org.apache.flink.agents.integration.test;

import org.apache.flink.agents.api.Agent;
import org.apache.flink.agents.api.InputEvent;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* limitations under the License.
*/

package org.apache.flink.agents.examples;
package org.apache.flink.agents.integration.test;

import org.apache.flink.agents.api.AgentsExecutionEnvironment;
import org.apache.flink.api.java.functions.KeySelector;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* limitations under the License.
*/

package org.apache.flink.agents.examples;
package org.apache.flink.agents.integration.test;

import org.apache.flink.agents.api.Agent;
import org.apache.flink.agents.api.InputEvent;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.agents.examples;
package org.apache.flink.agents.integration.test;

import org.apache.flink.agents.api.AgentsExecutionEnvironment;
import org.apache.flink.api.java.functions.KeySelector;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.agents.examples;
package org.apache.flink.agents.integration.test;

import org.apache.flink.agents.api.Agent;
import org.apache.flink.agents.api.Event;
Expand All @@ -25,7 +25,7 @@
import org.apache.flink.agents.api.context.MemoryObject;
import org.apache.flink.agents.api.context.MemoryRef;
import org.apache.flink.agents.api.context.RunnerContext;
import org.apache.flink.agents.examples.DataStreamIntegrationExample.ItemData;
import org.apache.flink.agents.integration.test.DataStreamIntegrationExample.ItemData;

/**
* A simple example agent used for explaining integrating agents with DataStream.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* limitations under the License.
*/

package org.apache.flink.agents.examples;
package org.apache.flink.agents.integration.test;

import org.apache.flink.agents.api.AgentsExecutionEnvironment;
import org.apache.flink.api.java.functions.KeySelector;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,93 @@
/*
* 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.agents.integration.test;

import org.apache.flink.agents.api.AgentsExecutionEnvironment;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.streaming.api.datastream.DataStream;
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;

public class DataStreamTableIntegrationExample {
/** Simple data class for the example. */
public static class ItemData {
public final int id;
public final String name;
public final double value;
public int visit_count;

public ItemData(int id, String name, double value) {
this.id = id;
this.name = name;
this.value = value;
this.visit_count = 0;
}

@Override
public String toString() {
return String.format(
"ItemData{id=%d, name='%s', value=%.2f,visit_count=%d}",
id, name, value, visit_count);
}
}

/** Key selector for extracting keys from ItemData. */
public static class ItemKeySelector
implements KeySelector<DataStreamIntegrationExample.ItemData, Integer> {
@Override
public Integer getKey(DataStreamIntegrationExample.ItemData item) {
return item.id;
}
}

public static void main(String[] args) throws Exception {
// Create the execution environment
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setParallelism(1);

// Create input DataStream
DataStream<DataStreamIntegrationExample.ItemData> inputStream =
env.fromElements(
new DataStreamIntegrationExample.ItemData(1, "item1", 10.5),
new DataStreamIntegrationExample.ItemData(2, "item2", 20.0),
new DataStreamIntegrationExample.ItemData(3, "item3", 15.7),
new DataStreamIntegrationExample.ItemData(1, "item1_updated", 12.3),
new DataStreamIntegrationExample.ItemData(2, "item2_updated", 22.1),
new DataStreamIntegrationExample.ItemData(1, "item1_updated_again", 15.3));

// Create agents execution environment
AgentsExecutionEnvironment agentsEnv =
AgentsExecutionEnvironment.getExecutionEnvironment(env);

// Define output schema
Schema outputSchema = Schema.newBuilder().column("f0", DataTypes.STRING()).build();

// Apply agent to the Table
Table outputTable =
agentsEnv
.fromDataStream(
inputStream, new DataStreamIntegrationExample.ItemKeySelector())
.apply(new DataStreamAgent())
.toTable(outputSchema);

outputTable.execute().print();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.agents.examples;
package org.apache.flink.agents.integration.test;

import org.apache.flink.agents.api.Agent;
import org.apache.flink.agents.api.Event;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.agents.examples;
package org.apache.flink.agents.integration.test;

import org.apache.flink.agents.api.AgentsExecutionEnvironment;
import org.apache.flink.api.java.functions.KeySelector;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* limitations under the License.
*/

package org.apache.flink.agents.examples;
package org.apache.flink.agents.integration.test;

import org.apache.flink.agents.api.Agent;
import org.apache.flink.agents.api.AgentsExecutionEnvironment;
Expand Down Expand Up @@ -66,7 +66,7 @@ public static void main(String[] args) throws Exception {

// Create agents execution environment
AgentsExecutionEnvironment agentsEnv =
AgentsExecutionEnvironment.getExecutionEnvironment(env);
AgentsExecutionEnvironment.getExecutionEnvironment(env, tableEnv);

// register resource to agents execution environment.
agentsEnv
Expand Down Expand Up @@ -102,7 +102,6 @@ public static void main(String[] args) throws Exception {
agentsEnv
.fromTable(
inputTable,
tableEnv,
(KeySelector<Object, Double>)
value -> (Double) ((Row) value).getField("a"))
.apply(agent)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* limitations under the License.
*/

package org.apache.flink.agents.examples;
package org.apache.flink.agents.integration.test;

import org.apache.flink.agents.api.Agent;
import org.apache.flink.agents.api.Event;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.agents.examples;
package org.apache.flink.agents.integration.test;

import org.apache.flink.agents.api.Agent;
import org.apache.flink.agents.api.Event;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* limitations under the License.
*/

package org.apache.flink.agents.examples;
package org.apache.flink.agents.integration.test;

import org.apache.flink.agents.api.AgentsExecutionEnvironment;
import org.apache.flink.api.java.functions.KeySelector;
Expand Down Expand Up @@ -78,15 +78,15 @@ public static void main(String[] args) throws Exception {

// Create agents execution environment
AgentsExecutionEnvironment agentsEnv =
AgentsExecutionEnvironment.getExecutionEnvironment(env);
AgentsExecutionEnvironment.getExecutionEnvironment(env, tableEnv);

// Define output schema
Schema outputSchema = Schema.newBuilder().column("f0", DataTypes.STRING()).build();

// Apply agent to the Table
Table outputTable =
agentsEnv
.fromTable(inputTable, tableEnv, new RowKeySelector())
.fromTable(inputTable, new RowKeySelector())
.apply(new TableAgent())
.toTable(outputSchema);

Expand Down
Loading