Skip to content
Open
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
10 changes: 10 additions & 0 deletions wayang-platforms/wayang-flink/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -116,6 +116,16 @@
<artifactId>flink-hadoop-compatibility_2.12</artifactId>
<version>${flink.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-files</artifactId>
<version>${flink.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java</artifactId>
<version>${flink.version}</version>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-math3</artifactId>
Expand Down
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.wayang.flink.channels;

import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.wayang.core.optimizer.OptimizationContext;
import org.apache.wayang.core.plan.executionplan.Channel;
import org.apache.wayang.core.plan.wayangplan.OutputSlot;
import org.apache.wayang.core.platform.AbstractChannelInstance;
import org.apache.wayang.core.platform.ChannelDescriptor;
import org.apache.wayang.core.platform.ChannelInstance;
import org.apache.wayang.core.platform.Executor;
import org.apache.wayang.flink.execution.FlinkExecutor;

import java.util.OptionalLong;

public class DataStreamChannel extends Channel {

/**
* {@link ChannelInstance} implementation for {@link DataStream}s.
*/
public class Instance extends AbstractChannelInstance {

private DataStream<?> dataStream;

// TODO: this.size is currently always 0
private long size;
Copy link
Contributor

Choose a reason for hiding this comment

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

Does this have any effects?

Copy link
Contributor Author

@mspruc mspruc Jan 29, 2026

Choose a reason for hiding this comment

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

Leftovers from the old DataSetChannel implementation the user could theoretically extend the DataStreamChannel and provide their own, but its such a niche situation I'm fine with removing it.


public Instance(final FlinkExecutor executor,
final OptimizationContext.OperatorContext producerOperatorContext,
final int producerOutputIndex) {
super(executor, producerOperatorContext, producerOutputIndex);
}

public void accept(final DataStream<?> dataStream) {
this.dataStream = dataStream;
}

@SuppressWarnings("unchecked")
public <T> DataStream<T> provideDataStream() {
return (DataStream<T>) this.dataStream;
}

@Override
public OptionalLong getMeasuredCardinality() {
return this.size == 0 ? super.getMeasuredCardinality() : OptionalLong.of(this.size);
}

@Override
public DataStreamChannel getChannel() {
return DataStreamChannel.this;
}

@Override
protected void doDispose() {
this.dataStream = null;
}
}

public static final ChannelDescriptor DESCRIPTOR = new ChannelDescriptor(
DataStreamChannel.class, true, false);

public static final ChannelDescriptor DESCRIPTOR_MANY = new ChannelDescriptor(
DataStreamChannel.class, true, false);

public DataStreamChannel(final ChannelDescriptor descriptor, final OutputSlot<?> outputSlot) {
super(descriptor, outputSlot);
assert descriptor == DESCRIPTOR || descriptor == DESCRIPTOR_MANY;
this.markForInstrumentation();
}

private DataStreamChannel(final DataStreamChannel parent) {
super(parent);
}

@Override
public Channel copy() {
return new DataStreamChannel(this);
}

@Override
public Instance createInstance(final Executor executor,
final OptimizationContext.OperatorContext producerOperatorContext,
final int producerOutputIndex) {
return new Instance((FlinkExecutor) executor, producerOperatorContext, producerOutputIndex);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ public class FunctionCompiler {
* @param <O> output type of the transformation
* @return a compiled function
*/
public <I, O> MapFunction<I, O> compile(TransformationDescriptor<I, O> descriptor) {
public static <I, O> MapFunction<I, O> compile(TransformationDescriptor<I, O> descriptor) {
// This is a dummy method but shows the intention of having something compilable in the descriptors.
Function<I, O> function = descriptor.getJavaImplementation();
return (MapFunction<I, O>) i -> function.apply(i);
Expand All @@ -72,7 +72,7 @@ public <I, O> MapFunction<I, O> compile(TransformationDescriptor<I, O> descripto
* @param <O> output type of the transformation
* @return a compiled function
*/
public <I, O> FlatMapFunction<I, O> compile(FunctionDescriptor.SerializableFunction<I, Iterable<O>> flatMapDescriptor) {
public static <I, O> FlatMapFunction<I, O> compile(FunctionDescriptor.SerializableFunction<I, Iterable<O>> flatMapDescriptor) {
return (t, collector) -> flatMapDescriptor.apply(t).forEach(collector::collect);
}

Expand All @@ -83,7 +83,7 @@ public <I, O> FlatMapFunction<I, O> compile(FunctionDescriptor.SerializableFunct
* @param <T> input/output type of the transformation
* @return a compiled function
*/
public <T> ReduceFunction<T> compile(ReduceDescriptor<T> descriptor) {
public static <T> ReduceFunction<T> compile(ReduceDescriptor<T> descriptor) {
// This is a dummy method but shows the intention of having something compilable in the descriptors.
BiFunction<T, T, T> reduce_function = descriptor.getJavaImplementation();
return new ReduceFunction<T>() {
Expand All @@ -94,26 +94,26 @@ public T reduce(T t, T t1) throws Exception {
};
}

public <T> FilterFunction<T> compile(PredicateDescriptor.SerializablePredicate<T> predicateDescriptor) {
return t -> predicateDescriptor.test(t);
public static <T> FilterFunction<T> compile(PredicateDescriptor.SerializablePredicate<T> predicateDescriptor) {
return predicateDescriptor::test;
}


public <T> OutputFormat<T> compile(ConsumerDescriptor.SerializableConsumer<T> consumerDescriptor) {
public static <T> OutputFormat<T> compile(ConsumerDescriptor.SerializableConsumer<T> consumerDescriptor) {
return new OutputFormatConsumer<T>(consumerDescriptor);
}


public <T, K> KeySelector<T, K> compileKeySelector(TransformationDescriptor<T, K> descriptor){
public static <T, K> KeySelector<T, K> compileKeySelector(TransformationDescriptor<T, K> descriptor){
return new KeySelectorFunction<T, K>(descriptor);
}

public <T0, T1, O> CoGroupFunction<T0, T1, O> compileCoGroup(){
public static <T0, T1, O> CoGroupFunction<T0, T1, O> compileCoGroup(){
return new FlinkCoGroupFunction<T0, T1, O>();
}


public <T> TextOutputFormat.TextFormatter<T> compileOutput(TransformationDescriptor<T, String> formattingDescriptor) {
public static <T> TextOutputFormat.TextFormatter<T> compileOutput(TransformationDescriptor<T, String> formattingDescriptor) {
Function<T, String> format = formattingDescriptor.getJavaImplementation();
return new TextOutputFormat.TextFormatter<T>(){

Expand All @@ -132,7 +132,7 @@ public String format(T value) {
* @param <O> output type of the transformation
* @return a compiled function
*/
public <I, O> MapPartitionFunction<I, O> compile(MapPartitionsDescriptor<I, O> descriptor){
public static <I, O> MapPartitionFunction<I, O> compile(MapPartitionsDescriptor<I, O> descriptor){
Function<Iterable<I>, Iterable<O>> function = descriptor.getJavaImplementation();
return new MapPartitionFunction<I, O>() {
@Override
Expand All @@ -146,13 +146,12 @@ public void mapPartition(Iterable<I> iterable, Collector<O> collector) throws Ex
};
}

public <T> WayangConvergenceCriterion compile(PredicateDescriptor<Collection<T>> descriptor){
FunctionDescriptor.SerializablePredicate<Collection<T>> predicate = descriptor.getJavaImplementation();
return new WayangConvergenceCriterion(predicate);
public static <T> WayangConvergenceCriterion<T> compile(PredicateDescriptor<Collection<T>> descriptor){
return new WayangConvergenceCriterion<T>(descriptor.getJavaImplementation());
}


public <I, O> RichFlatMapFunction<I, O> compile(FunctionDescriptor.ExtendedSerializableFunction<I, Iterable<O>> flatMapDescriptor, FlinkExecutionContext exe) {
public static <I, O> RichFlatMapFunction<I, O> compile(FunctionDescriptor.ExtendedSerializableFunction<I, Iterable<O>> flatMapDescriptor, FlinkExecutionContext exe) {

return new RichFlatMapFunction<I, O>() {
@Override
Expand All @@ -168,7 +167,7 @@ public void flatMap(I value, Collector<O> out) throws Exception {
}


public <I, O> RichMapFunction<I, O> compile(TransformationDescriptor<I, O> mapDescriptor, FlinkExecutionContext fex ) {
public static <I, O> RichMapFunction<I, O> compile(TransformationDescriptor<I, O> mapDescriptor, FlinkExecutionContext fex ) {

FunctionDescriptor.ExtendedSerializableFunction<I, O> map = (FunctionDescriptor.ExtendedSerializableFunction) mapDescriptor.getJavaImplementation();
return new RichMapFunction<I, O>() {
Expand All @@ -186,7 +185,7 @@ public void open(Configuration parameters) throws Exception {



public <I, O> RichMapPartitionFunction<I, O> compile(MapPartitionsDescriptor<I, O> descriptor, FlinkExecutionContext fex){
public static <I, O> RichMapPartitionFunction<I, O> compile(MapPartitionsDescriptor<I, O> descriptor, FlinkExecutionContext fex){
FunctionDescriptor.ExtendedSerializableFunction<Iterable<I>, Iterable<O>> function =
(FunctionDescriptor.ExtendedSerializableFunction<Iterable<I>, Iterable<O>>)
descriptor.getJavaImplementation();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.wayang.flink.execution;

import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.wayang.core.api.Job;
import org.apache.wayang.core.api.exception.WayangException;
import org.apache.wayang.core.optimizer.OptimizationContext;
Expand All @@ -35,7 +36,6 @@
import org.apache.wayang.flink.compiler.FunctionCompiler;
import org.apache.wayang.flink.operators.FlinkExecutionOperator;
import org.apache.wayang.flink.platform.FlinkPlatform;
import com.esotericsoftware.kryo.serializers.DefaultSerializers;

import java.util.Arrays;
import java.util.Collection;
Expand All @@ -56,6 +56,12 @@ public class FlinkExecutor extends PushExecutorTemplate {
*/
public ExecutionEnvironment fee;
Copy link
Contributor

Choose a reason for hiding this comment

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

I think ideally we just remove the old Env and also DataSet with it, so that the end user of wayang doesn't even notice this change (other than a few operators missing maybe)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Per discussion in the dev-list I've delegated this to the configuration.



/**
* {@link StreamExecutionEnvironment} for bounded and continuous streams.
*/
public StreamExecutionEnvironment sEnv;

/**
* Compiler to create flink UDFs.
*/
Expand All @@ -76,6 +82,7 @@ public FlinkExecutor(FlinkPlatform flinkPlatform, Job job) {
super(job);
this.platform = flinkPlatform;
this.flinkContextReference = this.platform.getFlinkContext(job);
this.sEnv = flinkPlatform.streamExecutionEnvironment;
this.fee = this.flinkContextReference.get();
this.numDefaultPartitions = (int) this.getConfiguration().getLongProperty("wayang.flink.parallelism");
this.fee.setParallelism(this.numDefaultPartitions);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
/*
* 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.wayang.flink.mapping;

import java.util.Collection;
import java.util.Collections;

import org.apache.wayang.basic.operators.TextFileSource;
import org.apache.wayang.core.mapping.Mapping;
import org.apache.wayang.core.mapping.OperatorPattern;
import org.apache.wayang.core.mapping.PlanTransformation;
import org.apache.wayang.core.mapping.ReplacementSubplanFactory;
import org.apache.wayang.core.mapping.SubplanPattern;
import org.apache.wayang.flink.operators.FlinkBoundedTextFileSource;
import org.apache.wayang.flink.platform.FlinkPlatform;

public class BoundedTextFileSourceMapping implements Mapping {
Copy link
Contributor

Choose a reason for hiding this comment

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

As everything is bounded as of now in Wayang, I would just call this TextFileSource and replace TextFileSource from Flink with it.

We can later add a StreamedTextFileSource.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

See comment above.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Nevertheless since our discussion on continuous streams hasn't concluded either I still propose we keep the bounded semantic as a way to leave the door open for continuous sources later.

@Override
public Collection<PlanTransformation> getTransformations() {
return Collections.singleton(new PlanTransformation(
this.createSubplanPattern(),
this.createReplacementSubplanFactory(),
FlinkPlatform.getInstance()
));
}

private SubplanPattern createSubplanPattern() {
final OperatorPattern<?> operatorPattern = new OperatorPattern<>(
"source", new TextFileSource("", null), false
);
return SubplanPattern.createSingleton(operatorPattern);
}

private ReplacementSubplanFactory createReplacementSubplanFactory() {
return new ReplacementSubplanFactory.OfSingleOperators<TextFileSource>(
(matchedOperator, epoch) -> new FlinkBoundedTextFileSource(matchedOperator).at(epoch)
);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,16 +19,21 @@
package org.apache.wayang.flink.mapping;

import org.apache.wayang.core.mapping.Mapping;
import org.apache.wayang.flink.plugin.FlinkBasicPlugin;

import java.util.Arrays;
import java.util.Collection;

/**
* Register for {@link Mapping}s for this platform.
* Register for {@link Mapping}s for {@link FlinkBasicPlugin}.
*/
public class Mappings {

public static Collection<Mapping> BASIC_MAPPINGS = Arrays.asList(
/**
* Mappings using Flink's DataSets
* @deprecated DataSet API in Flink has been deprecated move over to bounded streams for a 1-to-1 replacement {@link #BOUNDED_STREAM_MAPPINGS}.
*/
public static final Collection<Mapping> BASIC_MAPPINGS = Arrays.asList(
new CartesianMapping(),
new CoGroupMapping(),
new CollectionSourceMapping(),
Expand Down Expand Up @@ -60,6 +65,12 @@ public class Mappings {
new ZipWithIdMapping()
);

public static final Collection<Mapping> BOUNDED_STREAM_MAPPINGS = Arrays.asList(
new BoundedTextFileSourceMapping(),
new StreamedMapMapping(),
new StreamedJoinMapping(),
new StreamedLocalCallbackSinkMapping()
);
}


Loading
Loading