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;

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 @@ -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;


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

/**
* Compiler to create flink UDFs.
*/
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,102 @@
/*
* 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.operators;

import org.apache.flink.api.common.eventtime.WatermarkStrategy;
import org.apache.flink.connector.file.src.FileSource;
import org.apache.flink.connector.file.src.reader.TextLineInputFormat;
import org.apache.flink.core.fs.Path;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;

import org.apache.wayang.basic.operators.TextFileSource;
import org.apache.wayang.core.optimizer.OptimizationContext.OperatorContext;
import org.apache.wayang.core.optimizer.costs.LoadProfileEstimators;
import org.apache.wayang.core.platform.ChannelDescriptor;
import org.apache.wayang.core.platform.ChannelInstance;
import org.apache.wayang.core.platform.lineage.ExecutionLineageNode;
import org.apache.wayang.core.util.Tuple;
import org.apache.wayang.flink.channels.DataStreamChannel;
import org.apache.wayang.flink.execution.FlinkExecutor;

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

/**
* Opens a Flink bounded {@link DataStream} on a {@link TextFileSource}.
*/
public class FlinkBoundedTextFileSource extends TextFileSource implements FlinkExecutionOperator {

public FlinkBoundedTextFileSource(final TextFileSource that) {
super(that);
}

public FlinkBoundedTextFileSource(final String inputUrl) {
super(inputUrl);
}

@Override
public List<ChannelDescriptor> getSupportedInputChannels(final int index) {
throw new UnsupportedOperationException(String.format("%s does not have input channels.", this));
}

@Override
public List<ChannelDescriptor> getSupportedOutputChannels(final int index) {
return Arrays.asList(DataStreamChannel.DESCRIPTOR, DataStreamChannel.DESCRIPTOR_MANY);
}

@Override
public Tuple<Collection<ExecutionLineageNode>, Collection<ChannelInstance>> evaluate(final ChannelInstance[] inputs,
final ChannelInstance[] outputs, final FlinkExecutor flinkExecutor, final OperatorContext operatorContext)
throws Exception {
assert inputs.length == this.getNumInputs();
assert outputs.length == this.getNumOutputs();

final DataStreamChannel.Instance output = (DataStreamChannel.Instance) outputs[0];

final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();

final FileSource<String> fs = FileSource
.forRecordStreamFormat(new TextLineInputFormat(), new Path(this.getInputUrl()))
.build();

final DataStream<String> dataStream = env.fromSource(fs, WatermarkStrategy.noWatermarks(),
"FlinkDataStreamFileSource[" + this.getInputUrl() + "]");

output.accept(dataStream);

final ExecutionLineageNode prepareLineageNode = new ExecutionLineageNode(operatorContext);
prepareLineageNode.add(LoadProfileEstimators.createFromSpecification(
"wayang.flink.textfilesource.load.prepare", flinkExecutor.getConfiguration()));

final ExecutionLineageNode mainLineageNode = new ExecutionLineageNode(operatorContext);
mainLineageNode.add(LoadProfileEstimators.createFromSpecification(
"wayang.flink.textfilesource.load.main", flinkExecutor.getConfiguration()));

output.getLineage().addPredecessor(mainLineageNode);

return prepareLineageNode.collectAndMark();
}

@Override
public boolean containsAction() {
return false;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,105 @@
/*
* 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.operators;

import org.apache.flink.api.common.eventtime.WatermarkStrategy;
import org.apache.flink.connector.file.src.FileSource;
import org.apache.flink.connector.file.src.reader.TextLineInputFormat;
import org.apache.flink.core.fs.Path;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;

import org.apache.wayang.basic.operators.TextFileSource;
import org.apache.wayang.core.optimizer.OptimizationContext.OperatorContext;
import org.apache.wayang.core.optimizer.costs.LoadProfileEstimators;
import org.apache.wayang.core.platform.ChannelDescriptor;
import org.apache.wayang.core.platform.ChannelInstance;
import org.apache.wayang.core.platform.lineage.ExecutionLineageNode;
import org.apache.wayang.core.util.Tuple;
import org.apache.wayang.flink.channels.DataStreamChannel;
import org.apache.wayang.flink.execution.FlinkExecutor;

import java.time.Duration;
import java.util.Collection;
import java.util.List;

/**
* Opens a Flink continuous {@link DataStream} that monitors a file directory.
*/
public class FlinkContinuousTextFileSource extends TextFileSource implements FlinkExecutionOperator {

public FlinkContinuousTextFileSource(final TextFileSource that) {
super(that);
}

public FlinkContinuousTextFileSource(final String inputUrl) {
super(inputUrl);
}

@Override
public List<ChannelDescriptor> getSupportedInputChannels(final int index) {
throw new UnsupportedOperationException("Unimplemented method 'getSupportedInputChannels'");
}

@Override
public List<ChannelDescriptor> getSupportedOutputChannels(final int index) {
throw new UnsupportedOperationException("Unimplemented method 'getSupportedOutputChannels'");
}

@Override
public Tuple<Collection<ExecutionLineageNode>, Collection<ChannelInstance>> evaluate(final ChannelInstance[] inputs,
final ChannelInstance[] outputs, final FlinkExecutor flinkExecutor, final OperatorContext operatorContext)
throws Exception {
assert inputs.length == this.getNumInputs();
assert outputs.length == this.getNumOutputs();

final DataStreamChannel.Instance output = (DataStreamChannel.Instance) outputs[0];

final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();

final FileSource<String> fs = FileSource
.forRecordStreamFormat(new TextLineInputFormat(), new Path(this.getInputUrl()))
//TODO: I set manually 1 here but should be in config.
.monitorContinuously(Duration.ofSeconds(1))
.build();

final DataStream<String> dataStream = env.fromSource(fs, WatermarkStrategy.noWatermarks(),
"FlinkDataStreamFileSource[" + this.getInputUrl() + "]");

output.accept(dataStream);

final ExecutionLineageNode prepareLineageNode = new ExecutionLineageNode(operatorContext);
prepareLineageNode.add(LoadProfileEstimators.createFromSpecification(
"wayang.flink.textfilesource.load.prepare", flinkExecutor.getConfiguration()));

final ExecutionLineageNode mainLineageNode = new ExecutionLineageNode(operatorContext);
mainLineageNode.add(LoadProfileEstimators.createFromSpecification(
"wayang.flink.textfilesource.load.main", flinkExecutor.getConfiguration()));

output.getLineage().addPredecessor(mainLineageNode);

return prepareLineageNode.collectAndMark();
}

@Override
public boolean containsAction() {
throw new UnsupportedOperationException("Unimplemented method 'containsAction'");
}

}
Loading
Loading