提交 58865ff3 编写于 作者: A Aljoscha Krettek

[FLINK-1977] Rework Stream Operators to always be push based

上级 3cd36a51
......@@ -299,20 +299,45 @@ The user is expected to connect to the outside world through the source and the
#### Sources
The user can create(connect to) data streams by the different implementations of `SourceFunction` interface using `StreamExecutionEnvironment.addSource(sourceFunction)`. By default, sources run with parallelism of 1.
To create parallel sources the users source function needs to implement `ParallelSourceFunction` or extend `RichParallelSourceFunction` in which cases the source will have the parallelism of the environment. The parallelism for ParallelSourceFunctions can be changed afterwards using `source.setParallelism(parallelism)`.
Sources can by created by using `StreamExecutionEnvironment.addSource(sourceFunction)`.
Either use one of the source functions that come with Flink or write a custom source
by implementing the `SourceFunction` interface. By default, sources run with
parallelism of 1. To create parallel sources the users source function needs to implement
`ParallelSourceFunction` or extend `RichParallelSourceFunction` in which cases the source will have
the parallelism of the environment. The parallelism for ParallelSourceFunctions can be changed
after creation by using `source.setParallelism(parallelism)`.
The `SourceFunction` interface has two methods: `reachedEnd()` and `next()`. The former is used
by the system to determine whether more input data is available. This method can block if there
is no data available right now but there might come more data in the future. The `next()` method
is called to get next data element. This method will only be called if `reachedEnd()` returns
false. This method can also block if no data is currently available but more will arrive in the
future.
The methods must react to thread interrupt calls and break out of blocking calls with
`InterruptedException`. The method may ignore interrupt calls and/or swallow InterruptedExceptions,
if it is guaranteed that the method returns quasi immediately irrespectively of the input.
This is true for example for file streams, where the call is guaranteed to return after a very
short I/O delay in the order of milliseconds.
In addition to the bounded data sources (with similar method signatures as the
[batch API](programming_guide.html#data-sources)) there are several predefined stream sources
accessible from the `StreamExecutionEnvironment`:
* *Socket text stream*: Creates a new `DataStream` that contains the strings received
from the given socket. Strings are decoded by the system's default character set. The user
can optionally set the delimiters or the number of connection retries in case of errors.
Usage: `env.socketTextStream(hostname, port,…)`
The `SourceFunction` interface contains only two methods: `run(Collector<T> out)` and `cancel()`.
The `run` method will be called only once when the program starts, and should encapsulate the logic for generating the `DataStream`. Any object collected to the collector will be part of the `DataStream`. The `cancel` method will be called whenever the topology has failed for some reason.
* *Text file stream*: Creates a new `DataStream` that contains the lines of the files created
(or modified) in a given directory. The system continuously monitors the given path, and processes
any new files or modifications based on the settings. The file will be read with the system's
default character set.
Usage: `env.readFileStream(String path, long checkFrequencyMillis, WatchType watchType)`
In addition to the static data sources (with similar method signatures as the [batch API](programming_guide.html#data-sources)) there are several predefined stream sources accessible from the `StreamExecutionEnvironment`:
* *Message queue connectors*: There are pre-implemented connectors for a number of popular message
queue services, please refer to the section on [connectors](#stream-connectors) for more details.
* *Socket text stream*: Creates a new `DataStream` that contains the strings received infinitely from the given socket. Strings are decoded by the system's default character set. The user can optionally set the delimiters or the number of connection retries in case of errors.
Usage: `env.socketTextStream(hostname, port,…)`
* *Text file stream*: Creates a new `DataStream` that contains the lines of the files created (or modified) in a given directory. The system continuously monitors the given path, and processes any new files or modifications based on the settings. The file will be read with the system's default character set.
Usage: `env.readFileStream(String path, long checkFrequencyMillis, WatchType watchType)`
* *Message queue connectors*: There are pre-implemented connectors for a number of the most popular message queue services, please refer to the section on [connectors](#stream-connectors) for more detail.
* *Custom source*: Creates a new `DataStream` by using a user defined `SourceFunction` implementation.
Usage: `env.addSource(sourceFunction)`
......
......@@ -146,7 +146,7 @@ public abstract class AbstractInvokable {
return executionConfig;
}
catch (Exception e) {
LOG.warn("Could not load ExecutionConfig from Environment, returning default ExecutionConfig: {}", e);
LOG.warn("Could not load ExecutionConfig from Environment, returning default ExecutionConfig", e);
return new ExecutionConfig();
}
}
......
/*
* 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.streaming.connectors.flume;
import java.util.List;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.connectors.ConnectorSource;
import org.apache.flink.streaming.util.serialization.DeserializationSchema;
import org.apache.flink.util.Collector;
import org.apache.flume.Context;
import org.apache.flume.channel.ChannelProcessor;
import org.apache.flume.source.AvroSource;
import org.apache.flume.source.avro.AvroFlumeEvent;
import org.apache.flume.source.avro.Status;
public class FlumeSource<OUT> extends ConnectorSource<OUT> {
private static final long serialVersionUID = 1L;
String host;
String port;
volatile boolean finished = false;
private volatile boolean isRunning = false;
FlumeSource(String host, int port, DeserializationSchema<OUT> deserializationSchema) {
super(deserializationSchema);
this.host = host;
this.port = Integer.toString(port);
}
public class MyAvroSource extends AvroSource {
Collector<OUT> collector;
/**
* Sends the AvroFlumeEvent from it's argument list to the Apache Flink
* {@link DataStream}.
*
* @param avroEvent
* The event that should be sent to the dataStream
* @return A {@link Status}.OK message if sending the event was
* successful.
*/
@Override
public Status append(AvroFlumeEvent avroEvent) {
collect(avroEvent);
return Status.OK;
}
/**
* Sends the AvroFlumeEvents from it's argument list to the Apache Flink
* {@link DataStream}.
*
* @param events
* The events that is sent to the dataStream
* @return A Status.OK message if sending the events was successful.
*/
@Override
public Status appendBatch(List<AvroFlumeEvent> events) {
for (AvroFlumeEvent avroEvent : events) {
collect(avroEvent);
}
return Status.OK;
}
/**
* Deserializes the AvroFlumeEvent before sending it to the Apache Flink
* {@link DataStream}.
*
* @param avroEvent
* The event that is sent to the dataStream
*/
private void collect(AvroFlumeEvent avroEvent) {
byte[] b = avroEvent.getBody().array();
OUT out = FlumeSource.this.schema.deserialize(b);
if (schema.isEndOfStream(out)) {
FlumeSource.this.finished = true;
this.stop();
FlumeSource.this.notifyAll();
} else {
collector.collect(out);
}
}
}
MyAvroSource avroSource;
/**
* Configures the AvroSource. Also sets the collector so the application can
* use it from outside of the invoke function.
*
* @param collector
* The collector used in the invoke function
*/
public void configureAvroSource(Collector<OUT> collector) {
avroSource = new MyAvroSource();
avroSource.collector = collector;
Context context = new Context();
context.put("port", port);
context.put("bind", host);
avroSource.configure(context);
// An instance of a ChannelProcessor is required for configuring the
// avroSource although it will not be used in this case.
ChannelProcessor cp = new ChannelProcessor(null);
avroSource.setChannelProcessor(cp);
}
/**
* Configures the AvroSource and runs until the user calls a close function.
*
* @param collector
* The Collector for sending data to the datastream
*/
@Override
public void run(Collector<OUT> collector) throws Exception {
isRunning = true;
configureAvroSource(collector);
avroSource.start();
while (!finished && isRunning) {
this.wait();
}
}
@Override
public void cancel() {
isRunning = false;
}
}
///*
// * 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.streaming.connectors.flume;
//
//import java.util.List;
//
//import org.apache.flink.streaming.api.datastream.DataStream;
//import org.apache.flink.streaming.connectors.ConnectorSource;
//import org.apache.flink.streaming.util.serialization.DeserializationSchema;
//import org.apache.flink.util.Collector;
//import org.apache.flume.Context;
//import org.apache.flume.channel.ChannelProcessor;
//import org.apache.flume.source.AvroSource;
//import org.apache.flume.source.avro.AvroFlumeEvent;
//import org.apache.flume.source.avro.Status;
//
//public class FlumeSource<OUT> extends ConnectorSource<OUT> {
// private static final long serialVersionUID = 1L;
//
// String host;
// String port;
// volatile boolean finished = false;
//
// private volatile boolean isRunning = false;
//
// FlumeSource(String host, int port, DeserializationSchema<OUT> deserializationSchema) {
// super(deserializationSchema);
// this.host = host;
// this.port = Integer.toString(port);
// }
//
// public class MyAvroSource extends AvroSource {
// Collector<OUT> output;
//
// /**
// * Sends the AvroFlumeEvent from it's argument list to the Apache Flink
// * {@link DataStream}.
// *
// * @param avroEvent
// * The event that should be sent to the dataStream
// * @return A {@link Status}.OK message if sending the event was
// * successful.
// */
// @Override
// public Status append(AvroFlumeEvent avroEvent) {
// collect(avroEvent);
// return Status.OK;
// }
//
// /**
// * Sends the AvroFlumeEvents from it's argument list to the Apache Flink
// * {@link DataStream}.
// *
// * @param events
// * The events that is sent to the dataStream
// * @return A Status.OK message if sending the events was successful.
// */
// @Override
// public Status appendBatch(List<AvroFlumeEvent> events) {
// for (AvroFlumeEvent avroEvent : events) {
// collect(avroEvent);
// }
//
// return Status.OK;
// }
//
// /**
// * Deserializes the AvroFlumeEvent before sending it to the Apache Flink
// * {@link DataStream}.
// *
// * @param avroEvent
// * The event that is sent to the dataStream
// */
// private void collect(AvroFlumeEvent avroEvent) {
// byte[] b = avroEvent.getBody().array();
// OUT out = FlumeSource.this.schema.deserialize(b);
//
// if (schema.isEndOfStream(out)) {
// FlumeSource.this.finished = true;
// this.stop();
// FlumeSource.this.notifyAll();
// } else {
// output.collect(out);
// }
//
// }
//
// }
//
// MyAvroSource avroSource;
//
// /**
// * Configures the AvroSource. Also sets the output so the application can
// * use it from outside of the invoke function.
// *
// * @param output
// * The output used in the invoke function
// */
// public void configureAvroSource(Collector<OUT> output) {
//
// avroSource = new MyAvroSource();
// avroSource.output = output;
// Context context = new Context();
// context.put("port", port);
// context.put("bind", host);
// avroSource.configure(context);
// // An instance of a ChannelProcessor is required for configuring the
// // avroSource although it will not be used in this case.
// ChannelProcessor cp = new ChannelProcessor(null);
// avroSource.setChannelProcessor(cp);
// }
//
// /**
// * Configures the AvroSource and runs until the user calls a close function.
// *
// * @param output
// * The Collector for sending data to the datastream
// */
// @Override
// public void run(Collector<OUT> output) throws Exception {
// isRunning = true;
// configureAvroSource(output);
// avroSource.start();
// while (!finished && isRunning) {
// this.wait();
// }
// }
//
// @Override
// public void cancel() {
// isRunning = false;
// }
//
//}
/*
* 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.streaming.connectors.flume;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.util.serialization.SerializationSchema;
import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
public class FlumeTopology {
public static void main(String[] args) throws Exception {
StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
@SuppressWarnings("unused")
DataStream<String> dataStream1 = env.addSource(
new FlumeSource<String>("localhost", 41414, new SimpleStringSchema())).addSink(
new FlumeSink<String>("localhost", 42424, new StringToByteSerializer()));
env.execute();
}
public static class StringToByteSerializer implements SerializationSchema<String, byte[]> {
private static final long serialVersionUID = 1L;
@Override
public byte[] serialize(String element) {
return element.getBytes();
}
}
}
///*
// * 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.streaming.connectors.flume;
//
//import org.apache.flink.streaming.api.datastream.DataStream;
//import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
//import org.apache.flink.streaming.util.serialization.SerializationSchema;
//import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
//
//public class FlumeTopology {
//
// public static void main(String[] args) throws Exception {
//
// StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
//
// @SuppressWarnings("unused")
// DataStream<String> dataStream1 = env.addSource(
// new FlumeSource<String>("localhost", 41414, new SimpleStringSchema())).addSink(
// new FlumeSink<String>("localhost", 42424, new StringToByteSerializer()));
//
// env.execute();
// }
//
// public static class StringToByteSerializer implements SerializationSchema<String, byte[]> {
//
// private static final long serialVersionUID = 1L;
//
// @Override
// public byte[] serialize(String element) {
// return element.getBytes();
// }
// }
//
//}
......@@ -22,7 +22,6 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.streaming.connectors.kafka.api.KafkaSink;
import org.apache.flink.streaming.util.serialization.JavaDefaultStringSchema;
import org.apache.flink.util.Collector;
public class KafkaProducerExample {
......@@ -40,21 +39,25 @@ public class KafkaProducerExample {
@SuppressWarnings({ "unused", "serial" })
DataStream<String> stream1 = env.addSource(new SourceFunction<String>() {
@Override
public void run(Collector<String> collector) throws Exception {
for (int i = 0; i < 20; i++) {
collector.collect("message #" + i);
Thread.sleep(100L);
}
collector.collect(new String("q"));
private int index = 0;
@Override
public boolean reachedEnd() throws Exception {
return index >= 20;
}
@Override
public void cancel() {
public String next() throws Exception {
if (index < 20) {
String result = "message #" + index;
index++;
return result;
}
return "q";
}
}).addSink(
new KafkaSink<String>(host + ":" + port, topic, new JavaDefaultStringSchema())
)
......
......@@ -30,10 +30,8 @@ import kafka.consumer.KafkaStream;
import kafka.javaapi.consumer.ConsumerConnector;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.connectors.ConnectorSource;
import org.apache.flink.streaming.util.serialization.DeserializationSchema;
import org.apache.flink.util.Collector;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
......@@ -61,7 +59,9 @@ public class KafkaSource<OUT> extends ConnectorSource<OUT> {
private static final long ZOOKEEPER_DEFAULT_SYNC_TIME = 200;
private static final String DEFAULT_GROUP_ID = "flink-group";
private volatile boolean isRunning = false;
// We must read this in reachedEnd() to check for the end. We keep it to return it in
// next()
private OUT nextElement;
/**
* Creates a KafkaSource that consumes a topic.
......@@ -177,38 +177,43 @@ public class KafkaSource<OUT> extends ConnectorSource<OUT> {
consumerIterator = stream.iterator();
}
/**
* Called to forward the data from the source to the {@link DataStream}.
*
* @param collector
* The Collector for sending data to the dataStream
*/
@Override
public void run(Collector<OUT> collector) throws Exception {
isRunning = true;
try {
while (isRunning && consumerIterator.hasNext()) {
OUT out = schema.deserialize(consumerIterator.next().message());
if (schema.isEndOfStream(out)) {
break;
}
collector.collect(out);
}
} finally {
public void open(Configuration config) throws Exception {
super.open(config);
initializeConnection();
}
@Override
public void close() throws Exception {
super.close();
if (consumer != null) {
consumer.shutdown();
}
}
@Override
public void open(Configuration config) throws Exception {
initializeConnection();
public boolean reachedEnd() throws Exception {
if (nextElement != null) {
return false;
} else if (consumerIterator.hasNext()) {
OUT out = schema.deserialize(consumerIterator.next().message());
if (schema.isEndOfStream(out)) {
return true;
}
nextElement = out;
}
return false;
}
@Override
public void cancel() {
isRunning = false;
if (consumer != null) {
consumer.shutdown();
public OUT next() throws Exception {
if (!reachedEnd()) {
OUT result = nextElement;
nextElement = null;
return result;
} else {
throw new RuntimeException("Source exhausted");
}
}
}
......@@ -37,7 +37,6 @@ import org.apache.flink.streaming.api.checkpoint.CheckpointCommitter;
import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously;
import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
import org.apache.flink.streaming.util.serialization.DeserializationSchema;
import org.apache.flink.util.Collector;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
......@@ -80,6 +79,8 @@ public class PersistentKafkaSource<OUT> extends RichParallelSourceFunction<OUT>
private transient ZkClient zkClient;
private transient long[] commitedOffsets; // maintain committed offsets, to avoid committing the same over and over again.
// We set this in reachedEnd to carry it over to next()
private OUT nextElement = null;
/**
*
......@@ -143,58 +144,49 @@ public class PersistentKafkaSource<OUT> extends RichParallelSourceFunction<OUT>
this.commitedOffsets = new long[numPartitions];
Arrays.fill(this.lastOffsets, -1);
Arrays.fill(this.commitedOffsets, 0); // just to make it clear
}
nextElement = null;
}
@Override
public void run(Collector<OUT> collector) throws Exception {
if(iteratorToRead == null) {
throw new RuntimeException("Stream to read not initialized properly. Has open() been called");
public boolean reachedEnd() throws Exception {
if (nextElement != null) {
return false;
}
try {
while (iteratorToRead.hasNext()) {
if (!running) {
LOG.info("Source got stopped");
break;
}
MessageAndMetadata<byte[], byte[]> message = iteratorToRead.next();
if(lastOffsets[message.partition()] >= message.offset()) {
LOG.info("Skipping message with offset {} from partition {}", message.offset(), message.partition());
continue;
}
OUT out = deserializationSchema.deserialize(message.message());
if (LOG.isTraceEnabled()) {
LOG.trace("Processed record with offset {} from partition {}", message.offset(), message.partition());
}
lastOffsets[message.partition()] = message.offset();
if (deserializationSchema.isEndOfStream(out)) {
LOG.info("DeserializationSchema signaled end of stream for this source");
break;
}
while (iteratorToRead.hasNext()) {
MessageAndMetadata<byte[], byte[]> message = iteratorToRead.next();
if(lastOffsets[message.partition()] >= message.offset()) {
LOG.info("Skipping message with offset {} from partition {}", message.offset(), message.partition());
continue;
}
lastOffsets[message.partition()] = message.offset();
collector.collect(out);
OUT out = deserializationSchema.deserialize(message.message());
if (deserializationSchema.isEndOfStream(out)) {
LOG.info("DeserializationSchema signaled end of stream for this source");
break;
}
} catch(Exception ie) {
// this exception is coming from Scala code.
if(ie instanceof InterruptedException) {
if(running) {
throw new RuntimeException("Error while reading kafka consumer", ie);
} else {
LOG.debug("Kafka consumer got interrupted because it has been cancelled. This is expected", ie);
}
} else {
throw ie;
nextElement = out;
if (LOG.isTraceEnabled()) {
LOG.trace("Processed record with offset {} from partition {}", message.offset(), message.partition());
}
break;
}
LOG.info("Source has finished reading data from the KafkaStream");
return nextElement == null;
}
@Override
public void cancel() {
LOG.info("Instructing source to stop reading data from Kafka");
running = false;
public OUT next() throws Exception {
if (!reachedEnd()) {
OUT result = nextElement;
nextElement = null;
return result;
} else {
throw new RuntimeException("Source exhausted");
}
}
@Override
......
......@@ -20,10 +20,8 @@ package org.apache.flink.streaming.connectors.rabbitmq;
import java.io.IOException;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.connectors.ConnectorSource;
import org.apache.flink.streaming.util.serialization.DeserializationSchema;
import org.apache.flink.util.Collector;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
......@@ -75,47 +73,14 @@ public class RMQSource<OUT> extends ConnectorSource<OUT> {
}
}
/**
* Called to forward the data from the source to the {@link DataStream}.
*
* @param collector
* The Collector for sending data to the dataStream
*/
@Override
public void run(Collector<OUT> collector) throws Exception {
isRunning = true;
try {
while (isRunning) {
try {
delivery = consumer.nextDelivery();
} catch (Exception e) {
if (LOG.isErrorEnabled()) {
LOG.error("Cannot recieve RMQ message {} at {}", QUEUE_NAME, HOST_NAME);
}
}
out = schema.deserialize(delivery.getBody());
if (schema.isEndOfStream(out)) {
break;
} else {
collector.collect(out);
}
}
} finally {
connection.close();
}
}
@Override
public void open(Configuration config) throws Exception {
initializeConnection();
}
@Override
public void cancel() {
isRunning = false;
public void close() throws Exception {
super.close();
try {
connection.close();
} catch (IOException e) {
......@@ -124,4 +89,50 @@ public class RMQSource<OUT> extends ConnectorSource<OUT> {
}
}
@Override
public boolean reachedEnd() throws Exception {
if (out != null) {
return true;
}
try {
delivery = consumer.nextDelivery();
} catch (Exception e) {
if (LOG.isErrorEnabled()) {
LOG.error("Cannot recieve RMQ message {} at {}", QUEUE_NAME, HOST_NAME);
}
}
out = schema.deserialize(delivery.getBody());
if (schema.isEndOfStream(out)) {
out = null;
return false;
}
return true;
}
@Override
public OUT next() throws Exception {
if (out != null) {
OUT result = out;
out = null;
return result;
}
try {
delivery = consumer.nextDelivery();
} catch (Exception e) {
if (LOG.isErrorEnabled()) {
LOG.error("Cannot recieve RMQ message {} at {}", QUEUE_NAME, HOST_NAME);
}
}
out = schema.deserialize(delivery.getBody());
if (schema.isEndOfStream(out)) {
throw new RuntimeException("RMQ source is at end.");
}
OUT result = out;
out = null;
return result;
}
}
......@@ -54,8 +54,10 @@ public class TwitterSource extends RichParallelSourceFunction<String> {
private transient BasicClient client;
private int waitSec = 5;
private boolean streaming;
private int numberOfTweets;
private int maxNumberOfTweets;
private int currentNumberOfTweets;
private String nextElement = null;
private volatile boolean isRunning = false;
......@@ -68,7 +70,7 @@ public class TwitterSource extends RichParallelSourceFunction<String> {
*/
public TwitterSource(String authPath) {
this.authPath = authPath;
streaming = true;
maxNumberOfTweets = -1;
}
/**
......@@ -82,28 +84,13 @@ public class TwitterSource extends RichParallelSourceFunction<String> {
*/
public TwitterSource(String authPath, int numberOfTweets) {
this.authPath = authPath;
streaming = false;
this.numberOfTweets = numberOfTweets;
this.maxNumberOfTweets = numberOfTweets;
}
@Override
public void open(Configuration parameters) throws Exception {
initializeConnection();
}
@Override
public void run(Collector<String> collector) throws Exception {
isRunning = true;
try {
if (streaming) {
collectMessages(collector);
} else {
collectFiniteMessages(collector);
}
} finally {
closeConnection();
isRunning = false;
}
currentNumberOfTweets = 0;
}
/**
......@@ -166,7 +153,7 @@ public class TwitterSource extends RichParallelSourceFunction<String> {
}
/**
* Put tweets into collector
* Put tweets into output
*
* @param collector
* Collector in which the tweets are collected.
......@@ -177,7 +164,7 @@ public class TwitterSource extends RichParallelSourceFunction<String> {
LOG.info("Collecting tweets");
}
for (int i = 0; i < numberOfTweets; i++) {
for (int i = 0; i < maxNumberOfTweets; i++) {
collectOneMessage(collector);
}
......@@ -187,7 +174,7 @@ public class TwitterSource extends RichParallelSourceFunction<String> {
}
/**
* Put tweets into collector
* Put tweets into output
*
* @param collector
* Collector in which the tweets are collected.
......@@ -204,7 +191,7 @@ public class TwitterSource extends RichParallelSourceFunction<String> {
}
/**
* Put one tweet into the collector.
* Put one tweet into the output.
*
* @param collector
* Collector in which the tweets are collected.
......@@ -285,8 +272,51 @@ public class TwitterSource extends RichParallelSourceFunction<String> {
}
@Override
public void cancel() {
isRunning = false;
closeConnection();
public boolean reachedEnd() throws Exception {
if (currentNumberOfTweets >= maxNumberOfTweets) {
return false;
}
if (nextElement != null) {
return true;
}
if (client.isDone()) {
if (LOG.isErrorEnabled()) {
LOG.error("Client connection closed unexpectedly: {}", client.getExitEvent()
.getMessage());
}
return false;
}
try {
String msg = queue.poll(waitSec, TimeUnit.SECONDS);
if (msg != null) {
nextElement = msg;
return true;
} else {
if (LOG.isInfoEnabled()) {
LOG.info("Did not receive a message in {} seconds", waitSec);
}
}
} catch (InterruptedException e) {
throw new RuntimeException("'Waiting for tweet' thread is interrupted", e);
}
return false;
}
@Override
public String next() throws Exception {
if (nextElement != null) {
String result = nextElement;
nextElement = null;
return result;
}
if (reachedEnd()) {
throw new RuntimeException("Twitter stream end reached.");
} else {
String result = nextElement;
nextElement = null;
return result;
}
}
}
\ No newline at end of file
}
......@@ -312,32 +312,33 @@ public class KafkaITCase {
private void writeSequence(StreamExecutionEnvironment env, String topicName, final int from, final int to) throws Exception {
LOG.info("Writing sequence from {} to {} to topic {}", from, to, topicName);
DataStream<Tuple2<Integer, Integer>> stream = env.addSource(new RichParallelSourceFunction<Tuple2<Integer, Integer>>() {
private static final long serialVersionUID = 1L;
boolean running = true;
int cnt = from;
int partition;
@Override
public void run(Collector<Tuple2<Integer, Integer>> collector) throws Exception {
LOG.info("Starting source.");
int cnt = from;
int partition = getRuntimeContext().getIndexOfThisSubtask();
while (running) {
LOG.info("Writing " + cnt + " to partition " + partition);
collector.collect(new Tuple2<Integer, Integer>(getRuntimeContext().getIndexOfThisSubtask(), cnt));
if (cnt == to) {
LOG.info("Writer reached end.");
return;
}
cnt++;
}
public void open(Configuration parameters) throws Exception {
super.open(parameters);
partition = getRuntimeContext().getIndexOfThisSubtask();
}
@Override
public boolean reachedEnd() throws Exception {
return cnt > to;
}
@Override
public void cancel() {
LOG.info("Source got cancel()");
running = false;
public Tuple2<Integer, Integer> next() throws Exception {
LOG.info("Writing " + cnt + " to partition " + partition);
Tuple2<Integer, Integer> result = new Tuple2<Integer, Integer>(getRuntimeContext().getIndexOfThisSubtask(), cnt);
cnt++;
return result;
}
}).setParallelism(3);
stream.addSink(new KafkaSink<Tuple2<Integer, Integer>>(brokerConnectionStrings,
topicName,
new Utils.TypeInformationSerializationSchema<Tuple2<Integer, Integer>>(new Tuple2<Integer, Integer>(1, 1), env.getConfig()),
......@@ -403,25 +404,17 @@ public class KafkaITCase {
// add producing topology
DataStream<Tuple2<Long, String>> stream = env.addSource(new SourceFunction<Tuple2<Long, String>>() {
private static final long serialVersionUID = 1L;
boolean running = true;
int cnt = 0;
@Override
public void run(Collector<Tuple2<Long, String>> collector) throws Exception {
LOG.info("Starting source.");
int cnt = 0;
while (running) {
collector.collect(new Tuple2<Long, String>(1000L + cnt, "kafka-" + cnt++));
try {
Thread.sleep(100);
} catch (InterruptedException ignored) {
}
}
public boolean reachedEnd() throws Exception {
return false;
}
@Override
public void cancel() {
LOG.info("Source got cancel()");
running = false;
public Tuple2<Long, String> next() throws Exception {
Thread.sleep(100);
return new Tuple2<Long, String>(1000L + cnt, "kafka-" + cnt++);
}
});
stream.addSink(new KafkaSink<Tuple2<Long, String>>(brokerConnectionStrings, topic, new Utils.TypeInformationSerializationSchema<Tuple2<Long, String>>(new Tuple2<Long, String>(1L, ""), env.getConfig())));
......@@ -485,27 +478,17 @@ public class KafkaITCase {
// add producing topology
DataStream<Tuple2<Long, String>> stream = env.addSource(new SourceFunction<Tuple2<Long, String>>() {
private static final long serialVersionUID = 1L;
boolean running = true;
int cnt = 0;
@Override
public void run(Collector<Tuple2<Long, String>> collector) throws Exception {
LOG.info("Starting source.");
int cnt = 0;
while (running) {
collector.collect(new Tuple2<Long, String>(1000L + cnt, "kafka-" + cnt++));
LOG.info("Produced " + cnt);
try {
Thread.sleep(100);
} catch (InterruptedException ignored) {
}
}
public boolean reachedEnd() throws Exception {
return false;
}
@Override
public void cancel() {
LOG.info("Source got cancel()");
running = false;
public Tuple2<Long, String> next() throws Exception {
Thread.sleep(100);
return new Tuple2<Long, String>(1000L + cnt, "kafka-" + cnt++);
}
});
stream.addSink(new KafkaSink<Tuple2<Long, String>>(brokerConnectionStrings, topic, new Utils.TypeInformationSerializationSchema<Tuple2<Long, String>>(new Tuple2<Long, String>(1L, ""), env.getConfig())));
......@@ -570,40 +553,39 @@ public class KafkaITCase {
DataStream<Tuple2<Long, byte[]>> stream = env.addSource(new RichSourceFunction<Tuple2<Long, byte[]>>() {
private static final long serialVersionUID = 1L;
boolean running = true;
long cnt;
transient Random rnd;
@Override
public void open(Configuration parameters) throws Exception {
super.open(parameters);
cnt = 0;
rnd = new Random(1337);
}
@Override
public void run(Collector<Tuple2<Long, byte[]>> collector) throws Exception {
LOG.info("Starting source.");
long cnt = 0;
Random rnd = new Random(1337);
while (running) {
//
byte[] wl = new byte[Math.abs(rnd.nextInt(1024 * 1024 * 30))];
collector.collect(new Tuple2<Long, byte[]>(cnt++, wl));
LOG.info("Emitted cnt=" + (cnt - 1) + " with byte.length = " + wl.length);
try {
Thread.sleep(100);
} catch (InterruptedException ignored) {
}
if(cnt == 10) {
LOG.info("Send end signal");
// signal end
collector.collect(new Tuple2<Long, byte[]>(-1L, new byte[]{1}));
running = false;
}
}
public boolean reachedEnd() throws Exception {
return cnt > 10;
}
@Override
public void cancel() {
LOG.info("Source got cancel()");
running = false;
public Tuple2<Long, byte[]> next() throws Exception {
Thread.sleep(100);
if (cnt < 10) {
byte[] wl = new byte[Math.abs(rnd.nextInt(1024 * 1024 * 30))];
Tuple2<Long, byte[]> result = new Tuple2<Long, byte[]>(cnt++, wl);
LOG.info("Emitted cnt=" + (cnt - 1) + " with byte.length = " + wl.length);
return result;
} else if (cnt == 10) {
Tuple2<Long, byte[]> result = new Tuple2<Long, byte[]>(-1L, new byte[]{1});
cnt++;
return result;
} else {
throw new RuntimeException("Source is exhausted.");
}
}
});
......@@ -680,25 +662,17 @@ public class KafkaITCase {
// add producing topology
DataStream<Tuple2<Long, String>> stream = env.addSource(new SourceFunction<Tuple2<Long, String>>() {
private static final long serialVersionUID = 1L;
boolean running = true;
int cnt = 0;
@Override
public void run(Collector<Tuple2<Long, String>> collector) throws Exception {
LOG.info("Starting source.");
int cnt = 0;
while (running) {
collector.collect(new Tuple2<Long, String>(1000L + cnt, "kafka-" + cnt++));
try {
Thread.sleep(100);
} catch (InterruptedException ignored) {
}
}
public boolean reachedEnd() throws Exception {
return false;
}
@Override
public void cancel() {
LOG.info("Source got cancel()");
running = false;
public Tuple2<Long, String> next() throws Exception {
Thread.sleep(100);
return new Tuple2<Long, String>(1000L + cnt, "kafka-" + cnt++);
}
});
stream.addSink(new KafkaSink<Tuple2<Long, String>>(brokerConnectionStrings, topic, new Utils.TypeInformationSerializationSchema<Tuple2<Long, String>>(new Tuple2<Long, String>(1L, ""), env.getConfig()), new CustomPartitioner()));
......@@ -771,24 +745,17 @@ public class KafkaITCase {
DataStream<String> stream = env.addSource(new SourceFunction<String>() {
private static final long serialVersionUID = 1L;
boolean running = true;
int cnt = 0;
@Override
public void run(Collector<String> collector) throws Exception {
LOG.info("Starting source.");
int cnt = 0;
while (running) {
collector.collect("kafka-" + cnt++);
try {
Thread.sleep(100);
} catch (InterruptedException ignored) {
}
}
public boolean reachedEnd() throws Exception {
return false;
}
@Override
public void cancel() {
LOG.info("Source got cancel()");
running = false;
public String next() throws Exception {
Thread.sleep(100);
return "kafka-" + cnt++;
}
});
stream.addSink(new KafkaSink<String>(brokerConnectionStrings, topic, new JavaDefaultStringSchema()));
......@@ -810,33 +777,26 @@ public class KafkaITCase {
StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
DataStream<String> stream = env.addSource(new SourceFunction<String>() {
boolean running = true;
@Override
public void run(Collector<String> collector) throws Exception {
LOG.info("Starting source.");
int cnt = 0;
while (running) {
String msg = "kafka-" + cnt++;
collector.collect(msg);
LOG.info("sending message = "+msg);
if ((cnt - 1) % 20 == 0) {
LOG.debug("Sending message #{}", cnt - 1);
}
if(cnt == 200) {
LOG.info("Stopping to produce after 200 msgs");
break;
}
private int cnt = 0;
}
@Override
public boolean reachedEnd() throws Exception {
return cnt == 200;
}
@Override
public void cancel() {
LOG.info("Source got chancel()");
running = false;
public String next() throws Exception {
String msg = "kafka-" + cnt++;
LOG.info("sending message = "+msg);
if ((cnt - 1) % 20 == 0) {
LOG.debug("Sending message #{}", cnt - 1);
}
return msg;
}
});
stream.addSink(new KafkaSink<String>(brokerConnectionStrings, topic, new JavaDefaultStringSchema()))
.setParallelism(1);
......
......@@ -19,9 +19,10 @@ package org.apache.flink.streaming.api.collector;
import org.apache.flink.streaming.api.collector.selector.OutputSelectorWrapper;
import org.apache.flink.streaming.api.graph.StreamEdge;
import org.apache.flink.streaming.api.operators.Output;
import org.apache.flink.util.Collector;
public class CollectorWrapper<OUT> implements Collector<OUT> {
public class CollectorWrapper<OUT> implements Output<OUT> {
private OutputSelectorWrapper<OUT> outputSelectorWrapper;
......
......@@ -37,8 +37,7 @@ public class StreamOutput<OUT> implements Collector<OUT> {
private SerializationDelegate<StreamRecord<OUT>> serializationDelegate;
private StreamRecord<OUT> streamRecord;
public StreamOutput(RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output,
int channelID, SerializationDelegate<StreamRecord<OUT>> serializationDelegate) {
public StreamOutput(RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output, SerializationDelegate<StreamRecord<OUT>> serializationDelegate) {
this.serializationDelegate = serializationDelegate;
......
......@@ -31,10 +31,10 @@ import org.apache.flink.streaming.api.functions.co.CoWindowFunction;
import org.apache.flink.streaming.api.functions.co.RichCoMapFunction;
import org.apache.flink.streaming.api.functions.co.RichCoReduceFunction;
import org.apache.flink.streaming.api.graph.StreamGraph;
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
import org.apache.flink.streaming.api.operators.co.CoStreamFlatMap;
import org.apache.flink.streaming.api.operators.co.CoStreamGroupedReduce;
import org.apache.flink.streaming.api.operators.co.CoStreamMap;
import org.apache.flink.streaming.api.operators.co.CoStreamOperator;
import org.apache.flink.streaming.api.operators.co.CoStreamReduce;
import org.apache.flink.streaming.api.operators.co.CoStreamWindow;
import org.apache.flink.streaming.api.windowing.helper.SystemTimestamp;
......@@ -373,7 +373,7 @@ public class ConnectedDataStream<IN1, IN2> {
}
protected <OUT> CoStreamOperator<IN1, IN2, OUT> getReduceOperator(
protected <OUT> TwoInputStreamOperator<IN1, IN2, OUT> getReduceOperator(
CoReduceFunction<IN1, IN2, OUT> coReducer) {
CoStreamReduce<IN1, IN2, OUT> operator;
if (isGrouped) {
......@@ -403,7 +403,7 @@ public class ConnectedDataStream<IN1, IN2> {
}
public <OUT> SingleOutputStreamOperator<OUT, ?> transform(String functionName,
TypeInformation<OUT> outTypeInfo, CoStreamOperator<IN1, IN2, OUT> operator) {
TypeInformation<OUT> outTypeInfo, TwoInputStreamOperator<IN1, IN2, OUT> operator) {
@SuppressWarnings({ "unchecked", "rawtypes" })
SingleOutputStreamOperator<OUT, ?> returnStream = new SingleOutputStreamOperator(
......
......@@ -62,12 +62,12 @@ import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
import org.apache.flink.streaming.api.functions.sink.SinkFunction;
import org.apache.flink.streaming.api.functions.sink.SocketClientSink;
import org.apache.flink.streaming.api.graph.StreamGraph;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.StreamCounter;
import org.apache.flink.streaming.api.operators.StreamFilter;
import org.apache.flink.streaming.api.operators.StreamFlatMap;
import org.apache.flink.streaming.api.operators.StreamFold;
import org.apache.flink.streaming.api.operators.StreamMap;
import org.apache.flink.streaming.api.operators.StreamOperator;
import org.apache.flink.streaming.api.operators.StreamReduce;
import org.apache.flink.streaming.api.operators.StreamSink;
import org.apache.flink.streaming.api.windowing.helper.Count;
......@@ -1214,7 +1214,7 @@ public class DataStream<OUT> {
* @return the data stream constructed
*/
public <R> SingleOutputStreamOperator<R, ?> transform(String operatorName,
TypeInformation<R> outTypeInfo, StreamOperator<OUT, R> operator) {
TypeInformation<R> outTypeInfo, OneInputStreamOperator<OUT, R> operator) {
DataStream<OUT> inputStream = this.copy();
@SuppressWarnings({ "unchecked", "rawtypes" })
SingleOutputStreamOperator<R, ?> returnStream = new SingleOutputStreamOperator(environment,
......@@ -1277,7 +1277,7 @@ public class DataStream<OUT> {
*/
public DataStreamSink<OUT> addSink(SinkFunction<OUT> sinkFunction) {
StreamOperator<OUT, OUT> sinkOperator = new StreamSink<OUT>(clean(sinkFunction));
OneInputStreamOperator<OUT, Object> sinkOperator = new StreamSink<OUT>(clean(sinkFunction));
DataStreamSink<OUT> returnStream = new DataStreamSink<OUT>(environment, "sink", getType(),
sinkOperator);
......
......@@ -19,7 +19,7 @@ package org.apache.flink.streaming.api.datastream;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.operators.StreamOperator;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
/**
* Represents the end of a DataStream.
......@@ -30,7 +30,7 @@ import org.apache.flink.streaming.api.operators.StreamOperator;
public class DataStreamSink<IN> extends SingleOutputStreamOperator<IN, DataStreamSink<IN>> {
protected DataStreamSink(StreamExecutionEnvironment environment, String operatorType,
TypeInformation<IN> outTypeInfo, StreamOperator<?,?> operator) {
TypeInformation<IN> outTypeInfo, OneInputStreamOperator<IN, ?> operator) {
super(environment, operatorType, outTypeInfo, operator);
}
......
......@@ -32,7 +32,7 @@ public class DataStreamSource<OUT> extends SingleOutputStreamOperator<OUT, DataS
boolean isParallel;
public DataStreamSource(StreamExecutionEnvironment environment, String operatorType,
TypeInformation<OUT> outTypeInfo, StreamOperator<?, OUT> operator,
TypeInformation<OUT> outTypeInfo, StreamOperator<OUT> operator,
boolean isParallel, String sourceName) {
super(environment, operatorType, outTypeInfo, operator);
......
......@@ -31,9 +31,9 @@ import org.apache.flink.api.java.typeutils.TupleTypeInfo;
import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.streaming.api.functions.WindowMapFunction;
import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.StreamFilter;
import org.apache.flink.streaming.api.operators.StreamFlatMap;
import org.apache.flink.streaming.api.operators.StreamOperator;
import org.apache.flink.streaming.api.operators.co.CoStreamFlatMap;
import org.apache.flink.streaming.api.operators.windowing.EmptyWindowFilter;
import org.apache.flink.streaming.api.operators.windowing.ParallelGroupedMerge;
......@@ -192,7 +192,7 @@ public class DiscretizedStream<OUT> extends WindowedDataStream<OUT> {
private <R> DiscretizedStream<R> transform(WindowTransformation transformation,
String operatorName, TypeInformation<R> retType,
StreamOperator<StreamWindow<OUT>, StreamWindow<R>> operator) {
OneInputStreamOperator<StreamWindow<OUT>, StreamWindow<R>> operator) {
return wrap(discretizedStream.transform(operatorName, new StreamWindowTypeInfo<R>(retType),
operator), transformation);
......@@ -200,8 +200,7 @@ public class DiscretizedStream<OUT> extends WindowedDataStream<OUT> {
private DiscretizedStream<OUT> filterEmpty(DiscretizedStream<OUT> input) {
return wrap(input.discretizedStream.transform("Filter", input.discretizedStream.getType(),
new StreamFilter<StreamWindow<OUT>>(new EmptyWindowFilter<OUT>())
.withoutInputCopy()), input.isPartitioned);
new StreamFilter<StreamWindow<OUT>>(new EmptyWindowFilter<OUT>())), input.isPartitioned);
}
@SuppressWarnings({ "unchecked", "rawtypes" })
......@@ -209,7 +208,7 @@ public class DiscretizedStream<OUT> extends WindowedDataStream<OUT> {
return input.discretizedStream.transform("ExtractParts", new TupleTypeInfo(Tuple2.class,
BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO),
new StreamFlatMap<StreamWindow<OUT>, Tuple2<Integer, Integer>>(
new WindowPartExtractor<OUT>()).withoutInputCopy());
new WindowPartExtractor<OUT>()));
}
private DiscretizedStream<OUT> partition(WindowTransformation transformation) {
......
......@@ -221,8 +221,7 @@ public class GroupedDataStream<OUT> extends DataStream<OUT> {
@Override
protected SingleOutputStreamOperator<OUT, ?> aggregate(AggregationFunction<OUT> aggregate) {
StreamGroupedReduce<OUT> operator = new StreamGroupedReduce<OUT>(clean(aggregate),
keySelector);
StreamGroupedReduce<OUT> operator = new StreamGroupedReduce<OUT>(clean(aggregate), keySelector);
SingleOutputStreamOperator<OUT, ?> returnStream = transform("Grouped Aggregation",
getType(), operator);
......
......@@ -18,7 +18,7 @@
package org.apache.flink.streaming.api.datastream;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.streaming.api.operators.StreamOperator;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
/**
* The iterative data stream represents the start of an iteration in a
......@@ -78,7 +78,7 @@ public class IterativeDataStream<IN> extends
@Override
public <R> SingleOutputStreamOperator<R, ?> transform(String operatorName,
TypeInformation<R> outTypeInfo, StreamOperator<IN, R> operator) {
TypeInformation<R> outTypeInfo, OneInputStreamOperator<IN, R> operator) {
// We call the superclass tranform method
SingleOutputStreamOperator<R, ?> returnStream = super.transform(operatorName, outTypeInfo,
......
......@@ -23,6 +23,7 @@ import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.api.java.typeutils.TypeInfoParser;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.graph.StreamGraph.ResourceStrategy;
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.StreamOperator;
import org.apache.flink.streaming.api.operators.StreamOperator.ChainingStrategy;
......@@ -39,7 +40,7 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
DataStream<OUT> {
protected boolean isSplit;
protected StreamOperator<?, ?> operator;
protected StreamOperator<?> operator;
/**
* Gets the name of the current data stream. This name is
......@@ -63,7 +64,7 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
}
protected SingleOutputStreamOperator(StreamExecutionEnvironment environment,
String operatorType, TypeInformation<OUT> outTypeInfo, StreamOperator<?, ?> operator) {
String operatorType, TypeInformation<OUT> outTypeInfo, StreamOperator<?> operator) {
super(environment, operatorType, outTypeInfo);
this.isSplit = false;
this.operator = operator;
......@@ -162,7 +163,7 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
* @return The operator with chaining disabled
*/
public SingleOutputStreamOperator<OUT, O> disableChaining() {
return setChainingStrategy(ChainingStrategy.NEVER);
return setChainingStrategy(AbstractStreamOperator.ChainingStrategy.NEVER);
}
/**
......@@ -173,7 +174,7 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
* @return The operator with chaining set.
*/
public SingleOutputStreamOperator<OUT, O> startNewChain() {
return setChainingStrategy(ChainingStrategy.HEAD);
return setChainingStrategy(AbstractStreamOperator.ChainingStrategy.HEAD);
}
/**
......
......@@ -37,7 +37,7 @@ import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction;
import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType;
import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator;
import org.apache.flink.streaming.api.functions.aggregation.SumAggregator;
import org.apache.flink.streaming.api.operators.StreamOperator;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.windowing.GroupedActiveDiscretizer;
import org.apache.flink.streaming.api.operators.windowing.GroupedStreamDiscretizer;
import org.apache.flink.streaming.api.operators.windowing.GroupedWindowBuffer;
......@@ -383,9 +383,9 @@ public class WindowedDataStream<OUT> {
private DiscretizedStream<OUT> discretize(WindowTransformation transformation,
WindowBuffer<OUT> windowBuffer) {
StreamOperator<OUT, WindowEvent<OUT>> discretizer = getDiscretizer();
OneInputStreamOperator<OUT, WindowEvent<OUT>> discretizer = getDiscretizer();
StreamOperator<WindowEvent<OUT>, StreamWindow<OUT>> bufferOperator = getBufferOperator(windowBuffer);
OneInputStreamOperator<WindowEvent<OUT>, StreamWindow<OUT>> bufferOperator = getBufferOperator(windowBuffer);
@SuppressWarnings({ "unchecked", "rawtypes" })
TypeInformation<WindowEvent<OUT>> bufferEventType = new TupleTypeInfo(WindowEvent.class,
......@@ -463,7 +463,7 @@ public class WindowedDataStream<OUT> {
/**
* Based on the defined policies, returns the stream discretizer to be used
*/
private StreamOperator<OUT, WindowEvent<OUT>> getDiscretizer() {
private OneInputStreamOperator<OUT, WindowEvent<OUT>> getDiscretizer() {
if (discretizerKey == null) {
return new StreamDiscretizer<OUT>(getTrigger(), getEviction());
} else if (getTrigger() instanceof CentralActiveTrigger) {
......@@ -478,7 +478,7 @@ public class WindowedDataStream<OUT> {
}
private StreamOperator<WindowEvent<OUT>, StreamWindow<OUT>> getBufferOperator(
private OneInputStreamOperator<WindowEvent<OUT>, StreamWindow<OUT>> getBufferOperator(
WindowBuffer<OUT> windowBuffer) {
if (discretizerKey == null) {
return new StreamWindowBuffer<OUT>(windowBuffer);
......
......@@ -31,7 +31,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.ClosureCleaner;
import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.io.TextInputFormat;
import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.api.java.typeutils.MissingTypeInfo;
import org.apache.flink.api.java.typeutils.PojoTypeInfo;
import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
......@@ -43,9 +42,7 @@ import org.apache.flink.client.program.PackagedProgram.PreviewPlanEnvironment;
import org.apache.flink.core.fs.Path;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.DataStreamSource;
import org.apache.flink.streaming.api.functions.source.FileMonitoringFunction;
import org.apache.flink.streaming.api.functions.source.FileMonitoringFunction.WatchType;
import org.apache.flink.streaming.api.functions.source.FileReadFunction;
import org.apache.flink.streaming.api.functions.source.FileSourceFunction;
import org.apache.flink.streaming.api.functions.source.FromElementsFunction;
import org.apache.flink.streaming.api.functions.source.GenSequenceFunction;
......@@ -435,10 +432,10 @@ public abstract class StreamExecutionEnvironment {
*/
public DataStream<String> readFileStream(String filePath, long intervalMillis,
WatchType watchType) {
DataStream<Tuple3<String, Long, Long>> source = addSource(new FileMonitoringFunction(
filePath, intervalMillis, watchType), "File Stream");
return source.flatMap(new FileReadFunction());
// DataStream<Tuple3<String, Long, Long>> source = addSource(new FileMonitoringFunction(
// filePath, intervalMillis, watchType), "File Stream");
// return source.flatMap(new FileReadFunction());
return null;
}
/**
......@@ -611,8 +608,8 @@ public abstract class StreamExecutionEnvironment {
* Ads a data source with a custom type information thus opening a
* {@link DataStream}. Only in very special cases does the user need to
* support type information. Otherwise use
* {@link #addSource(SourceFunction)}
*
* {@link #addSource(org.apache.flink.streaming.api.functions.source.SourceFunction)}
*
* @param function
* the user defined function
* @param sourceName
......@@ -630,7 +627,8 @@ public abstract class StreamExecutionEnvironment {
outTypeInfo = ((ResultTypeQueryable<OUT>) function).getProducedType();
} else {
try {
outTypeInfo = TypeExtractor.createTypeInfo(SourceFunction.class,
outTypeInfo = TypeExtractor.createTypeInfo(
SourceFunction.class,
function.getClass(), 0, null, null);
} catch (InvalidTypesException e) {
outTypeInfo = (TypeInformation<OUT>) new MissingTypeInfo("Custom source", e);
......@@ -640,7 +638,7 @@ public abstract class StreamExecutionEnvironment {
boolean isParallel = function instanceof ParallelSourceFunction;
ClosureCleaner.clean(function, true);
StreamOperator<OUT, OUT> sourceOperator = new StreamSource<OUT>(function);
StreamOperator<OUT> sourceOperator = new StreamSource<OUT>(function);
return new DataStreamSource<OUT>(this, sourceName, outTypeInfo, sourceOperator, isParallel,
sourceName);
......
......@@ -74,6 +74,9 @@ public abstract class FileSinkFunction<IN> extends RichSinkFunction<IN> {
try {
format.close();
} catch (Exception ex) {
if (LOG.isErrorEnabled()) {
LOG.error("Error while writing element.", ex);
}
try {
if (!cleanupCalled && format instanceof CleanupWhenUnsuccessful) {
cleanupCalled = true;
......@@ -92,6 +95,9 @@ public abstract class FileSinkFunction<IN> extends RichSinkFunction<IN> {
}
} catch (Exception ex) {
try {
if (LOG.isErrorEnabled()) {
LOG.error("Error while writing element.", ex);
}
if (!cleanupCalled && format instanceof CleanupWhenUnsuccessful) {
cleanupCalled = true;
((CleanupWhenUnsuccessful) format).tryCleanupOnError();
......
......@@ -21,18 +21,20 @@ import java.io.IOException;
import java.net.URI;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.fs.FileStatus;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.Path;
import org.apache.flink.util.Collector;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class FileMonitoringFunction implements SourceFunction<Tuple3<String, Long, Long>> {
public class FileMonitoringFunction extends RichSourceFunction<Tuple3<String, Long, Long>> {
private static final long serialVersionUID = 1L;
private static final Logger LOG = LoggerFactory.getLogger(FileMonitoringFunction.class);
......@@ -55,42 +57,21 @@ public class FileMonitoringFunction implements SourceFunction<Tuple3<String, Lon
private volatile boolean isRunning = false;
private Queue<Tuple3<String, Long, Long>> pendingFiles;
public FileMonitoringFunction(String path, long interval, WatchType watchType) {
this.path = path;
this.interval = interval;
this.watchType = watchType;
this.modificationTimes = new HashMap<String, Long>();
this.offsetOfFiles = new HashMap<String, Long>();
}
@Override
public void run(Collector<Tuple3<String, Long, Long>> collector) throws Exception {
isRunning = true;
public void open(Configuration parameters) throws Exception {
super.open(parameters);
this.modificationTimes = new HashMap<String, Long>();
this.offsetOfFiles = new HashMap<String, Long>();
this.pendingFiles = new LinkedList<Tuple3<String, Long, Long>>();
fileSystem = FileSystem.get(new URI(path));
while (isRunning) {
List<String> files = listNewFiles();
for (String filePath : files) {
if (watchType == WatchType.ONLY_NEW_FILES
|| watchType == WatchType.REPROCESS_WITH_APPENDED) {
collector.collect(new Tuple3<String, Long, Long>(filePath, 0L, -1L));
offsetOfFiles.put(filePath, -1L);
} else if (watchType == WatchType.PROCESS_ONLY_APPENDED) {
long offset = 0;
long fileSize = fileSystem.getFileStatus(new Path(filePath)).getLen();
if (offsetOfFiles.containsKey(filePath)) {
offset = offsetOfFiles.get(filePath);
}
collector.collect(new Tuple3<String, Long, Long>(filePath, offset, fileSize));
offsetOfFiles.put(filePath, fileSize);
LOG.info("File processed: {}, {}, {}", filePath, offset, fileSize);
}
}
Thread.sleep(interval);
}
}
private List<String> listNewFiles() throws IOException {
......@@ -126,8 +107,44 @@ public class FileMonitoringFunction implements SourceFunction<Tuple3<String, Lon
}
}
@Override
public boolean reachedEnd() throws Exception {
return false;
}
@Override
public void cancel() {
isRunning = false;
public Tuple3<String, Long, Long> next() throws Exception {
if (pendingFiles.size() > 0) {
return pendingFiles.poll();
} else {
while (true) {
List<String> files = listNewFiles();
for (String filePath : files) {
if (watchType == WatchType.ONLY_NEW_FILES
|| watchType == WatchType.REPROCESS_WITH_APPENDED) {
pendingFiles.add(new Tuple3<String, Long, Long>(filePath, 0L, -1L));
offsetOfFiles.put(filePath, -1L);
} else if (watchType == WatchType.PROCESS_ONLY_APPENDED) {
long offset = 0;
long fileSize = fileSystem.getFileStatus(new Path(filePath)).getLen();
if (offsetOfFiles.containsKey(filePath)) {
offset = offsetOfFiles.get(filePath);
}
pendingFiles.add(new Tuple3<String, Long, Long>(filePath, offset, fileSize));
offsetOfFiles.put(filePath, fileSize);
LOG.info("File added to queue: {}, {}, {}", filePath, offset, fileSize);
}
}
if (files.size() > 0) {
break;
}
Thread.sleep(interval);
}
}
return pendingFiles.poll();
}
}
......@@ -27,7 +27,6 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.io.InputSplit;
import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
import org.apache.flink.util.Collector;
public class FileSourceFunction extends RichParallelSourceFunction<String> {
private static final long serialVersionUID = 1L;
......@@ -37,8 +36,13 @@ public class FileSourceFunction extends RichParallelSourceFunction<String> {
private InputFormat<String, ?> inputFormat;
private TypeInformation<String> typeInfo;
private transient TypeSerializer<String> serializer;
private volatile boolean isRunning;
private InputFormat<String, InputSplit> format;
private Iterator<InputSplit> splitIterator;
private transient String nextElement;
public FileSourceFunction(InputFormat<String, ?> format, TypeInformation<String> typeInfo) {
this.inputFormat = format;
......@@ -46,39 +50,25 @@ public class FileSourceFunction extends RichParallelSourceFunction<String> {
}
@Override
@SuppressWarnings("unchecked")
public void open(Configuration parameters) throws Exception {
StreamingRuntimeContext context = (StreamingRuntimeContext) getRuntimeContext();
this.provider = context.getInputSplitProvider();
inputFormat.configure(context.getTaskStubParameters());
serializer = typeInfo.createSerializer(getRuntimeContext().getExecutionConfig());
format = (InputFormat<String, InputSplit>) this.inputFormat;
splitIterator = getInputSplits();
if (splitIterator.hasNext()) {
format.open(splitIterator.next());
}
}
@Override
public void run(Collector<String> collector) throws Exception {
isRunning = true;
final TypeSerializer<String> serializer = typeInfo.createSerializer(getRuntimeContext()
.getExecutionConfig());
final Iterator<InputSplit> splitIterator = getInputSplits();
@SuppressWarnings("unchecked")
final InputFormat<String, InputSplit> format = (InputFormat<String, InputSplit>) this.inputFormat;
try {
while (isRunning && splitIterator.hasNext()) {
final InputSplit split = splitIterator.next();
String record = serializer.createInstance();
format.open(split);
while (isRunning && !format.reachedEnd()) {
if ((record = format.nextRecord(record)) != null) {
collector.collect(record);
}
}
}
collector.close();
} finally {
format.close();
}
isRunning = false;
public void close() throws Exception {
super.close();
format.close();
}
private Iterator<InputSplit> getInputSplits() {
......@@ -129,7 +119,30 @@ public class FileSourceFunction extends RichParallelSourceFunction<String> {
}
@Override
public void cancel() {
isRunning = false;
public boolean reachedEnd() throws Exception {
if (nextElement != null) {
return false;
}
nextElement = serializer.createInstance();
nextElement = format.nextRecord(nextElement);
if (nextElement == null && splitIterator.hasNext()) {
format.open(splitIterator.next());
return reachedEnd();
} else if (nextElement == null) {
return true;
}
return false;
}
@Override
public String next() throws Exception {
if (reachedEnd()) {
throw new RuntimeException("End of FileSource reached.");
}
String result = nextElement;
nextElement = null;
return result;
}
}
......@@ -19,15 +19,16 @@ package org.apache.flink.streaming.api.functions.source;
import java.util.Arrays;
import java.util.Collection;
import java.util.Iterator;
import org.apache.flink.util.Collector;
import org.apache.flink.configuration.Configuration;
public class FromElementsFunction<T> implements SourceFunction<T> {
public class FromElementsFunction<T> extends RichSourceFunction<T> {
private static final long serialVersionUID = 1L;
Iterable<T> iterable;
private transient Iterator<T> iterator;
private volatile boolean isRunning;
private Iterable<T> iterable;
public FromElementsFunction(T... elements) {
this.iterable = Arrays.asList(elements);
......@@ -42,20 +43,19 @@ public class FromElementsFunction<T> implements SourceFunction<T> {
}
@Override
public void run(Collector<T> collector) throws Exception {
isRunning = true;
for (T element : iterable) {
if (isRunning) {
collector.collect(element);
} else {
break;
}
}
public void open(Configuration parameters) throws Exception {
super.open(parameters);
iterator = iterable.iterator();
}
@Override
public void cancel() {
isRunning = false;
public boolean reachedEnd() throws Exception {
return !iterator.hasNext();
}
@Override
public T next() throws Exception {
return iterator.next();
}
}
......@@ -18,7 +18,6 @@
package org.apache.flink.streaming.api.functions.source;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.util.Collector;
import org.apache.flink.util.NumberSequenceIterator;
/**
......@@ -32,20 +31,10 @@ public class GenSequenceFunction extends RichParallelSourceFunction<Long> {
private NumberSequenceIterator fullIterator;
private NumberSequenceIterator splitIterator;
private volatile boolean isRunning;
public GenSequenceFunction(long from, long to) {
fullIterator = new NumberSequenceIterator(from, to);
}
@Override
public void run(Collector<Long> collector) throws Exception {
isRunning = true;
while (splitIterator.hasNext() && isRunning) {
collector.collect(splitIterator.next());
}
}
@Override
public void open(Configuration config) {
int splitNumber = getRuntimeContext().getIndexOfThisSubtask();
......@@ -54,8 +43,13 @@ public class GenSequenceFunction extends RichParallelSourceFunction<Long> {
}
@Override
public void cancel() {
isRunning = false;
public boolean reachedEnd() throws Exception {
return !splitIterator.hasNext();
}
@Override
public Long next() throws Exception {
return splitIterator.next();
}
}
......@@ -26,7 +26,6 @@ import java.net.Socket;
import java.net.SocketException;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.util.Collector;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
......@@ -45,7 +44,11 @@ public class SocketTextStreamFunction extends RichSourceFunction<String> {
private static final int CONNECTION_TIMEOUT_TIME = 0;
private static final int CONNECTION_RETRY_SLEEP = 1000;
private volatile boolean isRunning = false;
private transient StringBuffer buffer;
private transient BufferedReader reader;
private boolean socketClosed;
private transient String nextElement;
public SocketTextStreamFunction(String hostname, int port, char delimiter, long maxRetry) {
this.hostname = hostname;
......@@ -60,89 +63,102 @@ public class SocketTextStreamFunction extends RichSourceFunction<String> {
super.open(parameters);
socket = new Socket();
socket.connect(new InetSocketAddress(hostname, port), CONNECTION_TIMEOUT_TIME);
buffer = new StringBuffer();
reader = new BufferedReader(new InputStreamReader(
socket.getInputStream()));
socketClosed = false;
}
@Override
public void run(Collector<String> collector) throws Exception {
streamFromSocket(collector, socket);
public void close() throws Exception {
super.close();
if (reader != null) {
reader.close();
}
if (socket != null && !socket.isClosed()) {
try {
socket.close();
} catch (IOException e) {
if (LOG.isErrorEnabled()) {
LOG.error("Could not close open socket");
}
}
}
}
public void streamFromSocket(Collector<String> collector, Socket socket) throws Exception {
isRunning = true;
try {
StringBuffer buffer = new StringBuffer();
BufferedReader reader = new BufferedReader(new InputStreamReader(
socket.getInputStream()));
while (isRunning) {
int data;
try {
data = reader.read();
} catch (SocketException e) {
if (!isRunning) {
break;
} else {
throw e;
}
}
public String blockingRead(Socket socket) throws Exception {
if (data == -1) {
socket.close();
long retry = 0;
boolean success = false;
while (retry < maxRetry && !success) {
if (!retryForever) {
retry++;
}
LOG.warn("Lost connection to server socket. Retrying in "
+ (CONNECTION_RETRY_SLEEP / 1000) + " seconds...");
try {
socket = new Socket();
socket.connect(new InetSocketAddress(hostname, port),
CONNECTION_TIMEOUT_TIME);
success = true;
} catch (ConnectException ce) {
Thread.sleep(CONNECTION_RETRY_SLEEP);
}
}
while (true) {
int data;
try {
data = reader.read();
} catch (SocketException e) {
socketClosed = true;
break;
}
if (success) {
LOG.info("Server socket is reconnected.");
} else {
LOG.error("Could not reconnect to server socket.");
break;
if (data == -1) {
socket.close();
long retry = 0;
boolean success = false;
while (retry < maxRetry && !success) {
if (!retryForever) {
retry++;
}
LOG.warn("Lost connection to server socket. Retrying in "
+ (CONNECTION_RETRY_SLEEP / 1000) + " seconds...");
try {
socket = new Socket();
socket.connect(new InetSocketAddress(hostname, port),
CONNECTION_TIMEOUT_TIME);
success = true;
} catch (ConnectException ce) {
Thread.sleep(CONNECTION_RETRY_SLEEP);
}
reader = new BufferedReader(new InputStreamReader(socket.getInputStream()));
continue;
}
if (data == delimiter) {
collector.collect(buffer.toString());
buffer = new StringBuffer();
} else if (data != '\r') { // ignore carriage return
buffer.append((char) data);
if (success) {
LOG.info("Server socket is reconnected.");
} else {
LOG.error("Could not reconnect to server socket.");
break;
}
reader = new BufferedReader(new InputStreamReader(socket.getInputStream()));
continue;
}
if (buffer.length() > 0) {
collector.collect(buffer.toString());
if (data == delimiter) {
String result = buffer.toString();
buffer = new StringBuffer();
return result;
} else if (data != '\r') { // ignore carriage return
buffer.append((char) data);
}
} finally {
socket.close();
}
return null;
}
@Override
public void cancel() {
isRunning = false;
if (socket != null && !socket.isClosed()) {
try {
socket.close();
} catch (IOException e) {
if (LOG.isErrorEnabled()) {
LOG.error("Could not close open socket");
}
}
public boolean reachedEnd() throws Exception {
if (socketClosed) {
return false;
}
nextElement = blockingRead(socket);
return nextElement == null;
}
@Override
public String next() throws Exception {
if (nextElement == null) {
reachedEnd();
}
return nextElement;
}
}
/*
* 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.streaming.api.functions.source;
import java.io.Serializable;
import org.apache.flink.api.common.functions.Function;
import org.apache.flink.util.Collector;
/**
* Interface for a stream data source.
*
* <p>Sources implementing this specific interface are executed with
* parallelism 1. To execute your sources in parallel
* see {@link ParallelSourceFunction}.</p>
*
* @param <OUT> The type of the records produced by this source.
*/
public interface SourceFunction<OUT> extends Function, Serializable {
/**
* Main work method of the source. This function is invoked at the beginning of the
* source's life and is expected to produce its data py "pushing" the records into
* the given collector.
*
* @param collector The collector that forwards records to the source's consumers.
*
* @throws Exception Throwing any type of exception will cause the source to be considered
* failed. When fault tolerance is enabled, recovery will be triggered,
* which may create a new instance of this source.
*/
public void run(Collector<OUT> collector) throws Exception;
/**
* This method signals the source function to cancel its operation
* The method is called by the framework if the task is to be aborted prematurely.
* This happens when the user cancels the job, or when the task is canceled as
* part of a program failure and cleanup.
*/
public void cancel();
}
/*
* 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.streaming.api.functions.source;
import org.apache.flink.api.common.functions.Function;
import java.io.Serializable;
/**
* Base interface for all stream data sources in Flink. The contract of a stream source
* is similar to an iterator - it is consumed as in the following pseudo code:
*
* <pre>{@code
* StreamSource<T> source = ...;
* Collector<T> out = ...;
* while (!source.reachedEnd()) {
* out.collect(source.next());
* }
* }
* </pre>
*
* <b>Note about blocking behavior</b>
* <p>This implementations of the methods in the stream sources must have certain guarantees about
* blocking behavior. One of the two characteristics must be fulfilled.</p>
* <ul>
* <li>The methods must react to thread interrupt calls and break out of blocking calls with
* an {@link InterruptedException}.</li>
* <li>The method may ignore interrupt calls and/or swallow InterruptedExceptions, if it is guaranteed
* that the method returns quasi immediately irrespectively of the input. This is true for example
* for file streams, where the call is guaranteed to return after a very short I/O delay in
* the order of milliseconds.</li>
* </ul>
*
* @param <T> The type of the records produced by this source.
*/
public interface SourceFunction<T> extends Function, Serializable {
/**
* Checks whether the stream has reached its end.
*
* <p>This method must obey the contract about blocking behavior declared in the
* description of this class.</p>
*
* @return True, if the end of the stream has been reached, false if more data is available.
*
* @throws InterruptedException The calling thread may be interrupted to pull the function out of this
* method during checkpoints.
* @throws Exception Any other exception that is thrown causes the source to fail and results in failure of
* the streaming program, or triggers recovery, depending on the program setup.
*/
boolean reachedEnd() throws Exception;
/**
* Produces the next record.
*
* <p>This method must obey the contract about blocking behavior declared in the
* description of this class.</p>
*
* @return The next record produced by this stream source.
*
* @throws InterruptedException The calling thread may be interrupted to pull the function out of this
* method during checkpoints.
* @throws Exception Any other exception that is thrown causes the source to fail and results in failure of
* the streaming program, or triggers recovery, depending on the program setup.
*/
T next() throws Exception;
}
......@@ -165,16 +165,11 @@ public class JSONGenerator {
node.put(PACT, "Data Stream");
}
StreamOperator<?, ?> operator = streamGraph.getStreamNode(vertexID).getOperator();
StreamOperator<?> operator = streamGraph.getStreamNode(vertexID).getOperator();
if (operator != null && operator.getUserFunction() != null) {
node.put(CONTENTS, vertex.getOperatorName() + " at "
+ operator.getUserFunction().getClass().getSimpleName());
} else {
node.put(CONTENTS, vertex.getOperatorName());
}
node.put(CONTENTS, vertex.getOperatorName());
node.put(PARALLELISM, streamGraph.getStreamNode(vertexID).getParallelism());
}
}
\ No newline at end of file
}
......@@ -162,7 +162,7 @@ public class StreamConfig implements Serializable {
return config.getLong(BUFFER_TIMEOUT, DEFAULT_TIMEOUT);
}
public void setStreamOperator(StreamOperator<?, ?> operator) {
public void setStreamOperator(StreamOperator<?> operator) {
if (operator != null) {
config.setClass(USER_FUNCTION, operator.getClass());
......
......@@ -41,13 +41,15 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
import org.apache.flink.streaming.api.collector.selector.OutputSelector;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.operators.StreamOperator;
import org.apache.flink.streaming.api.operators.co.CoStreamOperator;
import org.apache.flink.streaming.api.operators.StreamSource;
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer;
import org.apache.flink.streaming.runtime.tasks.CoStreamTask;
import org.apache.flink.streaming.runtime.tasks.TwoInputStreamTask;
import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask;
import org.apache.flink.streaming.runtime.tasks.SourceStreamTask;
import org.apache.flink.streaming.runtime.tasks.StreamIterationHead;
import org.apache.flink.streaming.runtime.tasks.StreamIterationTail;
import org.apache.flink.streaming.runtime.tasks.StreamTask;
import org.apache.sling.commons.json.JSONException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
......@@ -130,16 +132,20 @@ public class StreamGraph extends StreamingPlan {
return !streamLoops.isEmpty();
}
public <IN, OUT> void addSource(Integer vertexID, StreamOperator<IN, OUT> operatorObject,
public <IN, OUT> void addSource(Integer vertexID, StreamOperator<OUT> operatorObject,
TypeInformation<IN> inTypeInfo, TypeInformation<OUT> outTypeInfo, String operatorName) {
addOperator(vertexID, operatorObject, inTypeInfo, outTypeInfo, operatorName);
sources.add(vertexID);
}
public <IN, OUT> void addOperator(Integer vertexID, StreamOperator<IN, OUT> operatorObject,
public <IN, OUT> void addOperator(Integer vertexID, StreamOperator<OUT> operatorObject,
TypeInformation<IN> inTypeInfo, TypeInformation<OUT> outTypeInfo, String operatorName) {
addNode(vertexID, StreamTask.class, operatorObject, operatorName);
if (operatorObject instanceof StreamSource) {
addNode(vertexID, SourceStreamTask.class, operatorObject, operatorName);
} else {
addNode(vertexID, OneInputStreamTask.class, operatorObject, operatorName);
}
StreamRecordSerializer<IN> inSerializer = inTypeInfo != null ? new StreamRecordSerializer<IN>(
inTypeInfo, executionConfig) : null;
......@@ -156,10 +162,10 @@ public class StreamGraph extends StreamingPlan {
}
public <IN1, IN2, OUT> void addCoOperator(Integer vertexID,
CoStreamOperator<IN1, IN2, OUT> taskoperatorObject, TypeInformation<IN1> in1TypeInfo,
TwoInputStreamOperator<IN1, IN2, OUT> taskoperatorObject, TypeInformation<IN1> in1TypeInfo,
TypeInformation<IN2> in2TypeInfo, TypeInformation<OUT> outTypeInfo, String operatorName) {
addNode(vertexID, CoStreamTask.class, taskoperatorObject, operatorName);
addNode(vertexID, TwoInputStreamTask.class, taskoperatorObject, operatorName);
StreamRecordSerializer<OUT> outSerializer = (outTypeInfo != null)
&& !(outTypeInfo instanceof MissingTypeInfo) ? new StreamRecordSerializer<OUT>(
......@@ -228,7 +234,7 @@ public class StreamGraph extends StreamingPlan {
}
protected StreamNode addNode(Integer vertexID, Class<? extends AbstractInvokable> vertexClass,
StreamOperator<?, ?> operatorObject, String operatorName) {
StreamOperator<?> operatorObject, String operatorName) {
StreamNode vertex = new StreamNode(environemnt, vertexID, operatorObject, operatorName,
new ArrayList<OutputSelector<?>>(), vertexClass);
......@@ -286,7 +292,7 @@ public class StreamGraph extends StreamingPlan {
getStreamNode(vertexID).setSerializerOut(serializer);
}
public <IN, OUT> void setOperator(Integer vertexID, StreamOperator<IN, OUT> operatorObject) {
public <IN, OUT> void setOperator(Integer vertexID, StreamOperator<OUT> operatorObject) {
getStreamNode(vertexID).setOperator(operatorObject);
}
......@@ -337,10 +343,10 @@ public class StreamGraph extends StreamingPlan {
return streamNodes.values();
}
public Set<Tuple2<Integer, StreamOperator<?, ?>>> getOperators() {
Set<Tuple2<Integer, StreamOperator<?, ?>>> operatorSet = new HashSet<Tuple2<Integer, StreamOperator<?, ?>>>();
public Set<Tuple2<Integer, StreamOperator<?>>> getOperators() {
Set<Tuple2<Integer, StreamOperator<?>>> operatorSet = new HashSet<Tuple2<Integer, StreamOperator<?>>>();
for (StreamNode vertex : streamNodes.values()) {
operatorSet.add(new Tuple2<Integer, StreamOperator<?, ?>>(vertex.getID(), vertex
operatorSet.add(new Tuple2<Integer, StreamOperator<?>>(vertex.getID(), vertex
.getOperator()));
}
return operatorSet;
......
......@@ -49,7 +49,7 @@ public class StreamNode implements Serializable {
private Integer slotSharingID;
private boolean isolatedSlot = false;
private transient StreamOperator<?, ?> operator;
private transient StreamOperator<?> operator;
private List<OutputSelector<?>> outputSelectors;
private StreamRecordSerializer<?> typeSerializerIn1;
private StreamRecordSerializer<?> typeSerializerIn2;
......@@ -62,7 +62,7 @@ public class StreamNode implements Serializable {
private InputFormat<String, ?> inputFormat;
public StreamNode(StreamExecutionEnvironment env, Integer ID, StreamOperator<?, ?> operator,
public StreamNode(StreamExecutionEnvironment env, Integer ID, StreamOperator<?> operator,
String operatorName, List<OutputSelector<?>> outputSelector,
Class<? extends AbstractInvokable> jobVertexClass) {
this.env = env;
......@@ -138,11 +138,11 @@ public class StreamNode implements Serializable {
this.bufferTimeout = bufferTimeout;
}
public StreamOperator<?, ?> getOperator() {
public StreamOperator<?> getOperator() {
return operator;
}
public void setOperator(StreamOperator<?, ?> operator) {
public void setOperator(StreamOperator<?> operator) {
this.operator = operator;
}
......
......@@ -308,8 +308,8 @@ public class StreamingJobGraphGenerator {
StreamNode upStreamVertex = edge.getSourceVertex();
StreamNode downStreamVertex = edge.getTargetVertex();
StreamOperator<?, ?> headOperator = upStreamVertex.getOperator();
StreamOperator<?, ?> outOperator = downStreamVertex.getOperator();
StreamOperator<?> headOperator = upStreamVertex.getOperator();
StreamOperator<?> outOperator = downStreamVertex.getOperator();
return downStreamVertex.getInEdges().size() == 1
&& outOperator != null
......
......@@ -43,10 +43,10 @@ public class WindowingOptimizer {
@SuppressWarnings("rawtypes")
private static void removeMergeBeforeFlatten(StreamGraph streamGraph) {
Set<Tuple2<Integer, StreamOperator<?, ?>>> operators = streamGraph.getOperators();
Set<Tuple2<Integer, StreamOperator<?>>> operators = streamGraph.getOperators();
List<Integer> flatteners = new ArrayList<Integer>();
for (Tuple2<Integer, StreamOperator<?, ?>> entry : operators) {
for (Tuple2<Integer, StreamOperator<?>> entry : operators) {
if (entry.f1 instanceof WindowFlattener) {
flatteners.add(entry.f0);
}
......
/*
* 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.streaming.api.operators;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.functions.RuntimeContext;
import org.apache.flink.configuration.Configuration;
/**
* Base class for operators that do not contain a user-defined function.
*
* @param <OUT> The output type of the operator
*/
public abstract class AbstractStreamOperator<OUT> implements StreamOperator<OUT> {
private static final long serialVersionUID = 1L;
protected RuntimeContext runtimeContext;
protected ExecutionConfig executionConfig;
public Output<OUT> output;
// A sane default for most operators
protected ChainingStrategy chainingStrategy = ChainingStrategy.HEAD;
@Override
public final void setup(Output<OUT> output, RuntimeContext runtimeContext) {
this.output = output;
this.executionConfig = runtimeContext.getExecutionConfig();
this.runtimeContext = runtimeContext;
}
@Override
public void open(Configuration parameters) throws Exception {
}
@Override
public void close() throws Exception {
}
@Override
public final void setChainingStrategy(ChainingStrategy strategy) {
this.chainingStrategy = strategy;
}
@Override
public final ChainingStrategy getChainingStrategy() {
return chainingStrategy;
}
}
/*
* 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.streaming.api.operators;
import org.apache.flink.api.common.functions.Function;
import org.apache.flink.api.common.functions.util.FunctionUtils;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.checkpoint.CheckpointCommitter;
import org.apache.flink.streaming.api.checkpoint.Checkpointed;
import java.io.Serializable;
/**
* This is used as the base class for operators that have a user-defined function.
*
* @param <OUT> The output type of the operator
* @param <F> The type of the user function
*/
public abstract class AbstractUdfStreamOperator<OUT, F extends Function> extends AbstractStreamOperator<OUT> implements StatefulStreamOperator<OUT> {
protected final F userFunction;
public AbstractUdfStreamOperator(F userFunction) {
this.userFunction = userFunction;
}
@Override
public void open(Configuration parameters) throws Exception {
super.open(parameters);
FunctionUtils.setFunctionRuntimeContext(userFunction, runtimeContext);
FunctionUtils.openFunction(userFunction, parameters);
}
@Override
public void close() throws Exception{
super.close();
FunctionUtils.closeFunction(userFunction);
}
public void restoreInitialState(Serializable state) throws Exception {
if (userFunction instanceof Checkpointed) {
setStateOnFunction(state, userFunction);
}
else {
throw new IllegalStateException("Trying to restore state of a non-checkpointed function");
}
}
public Serializable getStateSnapshotFromFunction(long checkpointId, long timestamp) throws Exception {
if (userFunction instanceof Checkpointed) {
return ((Checkpointed<?>) userFunction).snapshotState(checkpointId, timestamp);
}
else {
return null;
}
}
public void confirmCheckpointCompleted(long checkpointId, long timestamp) throws Exception {
if (userFunction instanceof CheckpointCommitter) {
try {
((CheckpointCommitter) userFunction).commitCheckpoint(checkpointId);
}
catch (Exception e) {
throw new Exception("Error while confirming checkpoint " + checkpointId + " to the stream function", e);
}
}
}
private static <T extends Serializable> void setStateOnFunction(Serializable state, Function function) {
@SuppressWarnings("unchecked")
T typedState = (T) state;
@SuppressWarnings("unchecked")
Checkpointed<T> typedFunction = (Checkpointed<T>) function;
typedFunction.restoreState(typedState);
}
}
/*
* 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.streaming.api.operators;
/**
* Interface for stream operators with one input. Use
* {@link org.apache.flink.streaming.api.operators.AbstractStreamOperator} as a base class if
* you want to implement a custom operator.
*
* @param <IN> The input type of the operator
* @param <OUT> The output type of the operator
*/
public interface OneInputStreamOperator<IN, OUT> extends StreamOperator<OUT> {
public void processElement(IN element) throws Exception;
}
......@@ -15,32 +15,18 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.streaming.api.operators;
package org.apache.flink.streaming.runtime.tasks;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.runtime.io.CoReaderIterator;
import org.apache.flink.streaming.runtime.io.IndexedReaderIterator;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer;
import org.apache.flink.util.Collector;
import org.apache.flink.util.MutableObjectIterator;
public interface StreamTaskContext<OUT> {
StreamConfig getConfig();
ClassLoader getUserCodeClassLoader();
<X> MutableObjectIterator<X> getInput(int index);
<X> IndexedReaderIterator<X> getIndexedInput(int index);
<X> StreamRecordSerializer<X> getInputSerializer(int index);
Collector<OUT> getOutputCollector();
<X, Y> CoReaderIterator<X, Y> getCoReader();
ExecutionConfig getExecutionConfig();
/**
* A {@link org.apache.flink.streaming.api.operators.StreamOperator} is supplied with an object
* of this interface that can be used to emit elements and other messages, such as barriers
* and low watermarks, from an operator.
*
* @param <T> The type of the elments that can be emitted.
*/
public interface Output<T> extends Collector<T> {
// NOTE: This does not yet have methods for barriers/low watermarks, this needs to be
// extended when this functionality arrives.
}
......@@ -17,41 +17,19 @@
package org.apache.flink.streaming.api.operators;
import org.apache.flink.api.common.functions.Function;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer;
import org.apache.flink.util.Collector;
import java.io.Serializable;
public abstract class ChainableStreamOperator<IN, OUT> extends StreamOperator<IN, OUT> implements
Collector<IN> {
private static final long serialVersionUID = 1L;
private boolean copyInput = true;
public ChainableStreamOperator(Function userFunction) {
super(userFunction);
setChainingStrategy(ChainingStrategy.ALWAYS);
}
public void setup(Collector<OUT> collector, StreamRecordSerializer<IN> inSerializer) {
this.collector = collector;
this.inSerializer = inSerializer;
this.objectSerializer = inSerializer.getObjectSerializer();
}
/**
* Interface for Stream operators that can have state. This interface is used for checkpointing
* and restoring that state.
*
* @param <OUT> The output type of the operator
*/
public interface StatefulStreamOperator<OUT> extends StreamOperator<OUT> {
public ChainableStreamOperator<IN, OUT> withoutInputCopy() {
copyInput = false;
return this;
}
void restoreInitialState(Serializable state) throws Exception;
protected IN copyInput(IN input) {
return copyInput ? copy(input) : input;
}
Serializable getStateSnapshotFromFunction(long checkpointId, long timestamp) throws Exception;
@Override
public void collect(IN record) {
if (isRunning) {
nextObject = copyInput(record);
callUserFunctionAndLogException();
}
}
void confirmCheckpointCompleted(long checkpointId, long timestamp) throws Exception;
}
......@@ -17,28 +17,16 @@
package org.apache.flink.streaming.api.operators;
public class StreamCounter<IN> extends AbstractStreamOperator<Long> implements OneInputStreamOperator<IN, Long> {
public class StreamCounter<IN> extends ChainableStreamOperator<IN, Long> {
private static final long serialVersionUID = 1L;
Long count = 0L;
private Long count = 0L;
public StreamCounter() {
super(null);
chainingStrategy = ChainingStrategy.ALWAYS;
}
@Override
public void run() throws Exception {
while (isRunning && readNext() != null) {
collector.collect(++count);
}
public void processElement(IN element) {
output.collect(++count);
}
@Override
public void collect(IN record) {
if (isRunning) {
collector.collect(++count);
}
}
}
......@@ -19,29 +19,17 @@ package org.apache.flink.streaming.api.operators;
import org.apache.flink.api.common.functions.FilterFunction;
public class StreamFilter<IN> extends ChainableStreamOperator<IN, IN> {
private static final long serialVersionUID = 1L;
private boolean collect;
public class StreamFilter<IN> extends AbstractUdfStreamOperator<IN, FilterFunction<IN>> implements OneInputStreamOperator<IN, IN> {
public StreamFilter(FilterFunction<IN> filterFunction) {
super(filterFunction);
chainingStrategy = ChainingStrategy.ALWAYS;
}
@Override
public void run() throws Exception {
while (isRunning && readNext() != null) {
callUserFunctionAndLogException();
}
}
@Override
@SuppressWarnings("unchecked")
protected void callUserFunction() throws Exception {
collect = ((FilterFunction<IN>) userFunction).filter(nextObject);
if (collect) {
collector.collect(nextObject);
public void processElement(IN element) throws Exception {
if (userFunction.filter(element)) {
output.collect(element);
}
}
}
......@@ -19,24 +19,17 @@ package org.apache.flink.streaming.api.operators;
import org.apache.flink.api.common.functions.FlatMapFunction;
public class StreamFlatMap<IN, OUT> extends ChainableStreamOperator<IN, OUT> {
private static final long serialVersionUID = 1L;
public class StreamFlatMap<IN, OUT>
extends AbstractUdfStreamOperator<OUT, FlatMapFunction<IN, OUT>>
implements OneInputStreamOperator<IN, OUT> {
public StreamFlatMap(FlatMapFunction<IN, OUT> flatMapper) {
super(flatMapper);
chainingStrategy = ChainingStrategy.ALWAYS;
}
@Override
public void run() throws Exception {
while (isRunning && readNext() != null) {
callUserFunctionAndLogException();
}
public void processElement(IN element) throws Exception {
userFunction.flatMap(element, output);
}
@Override
@SuppressWarnings("unchecked")
protected void callUserFunction() throws Exception {
((FlatMapFunction<IN, OUT>) userFunction).flatMap(nextObject, collector);
}
}
......@@ -20,31 +20,32 @@ package org.apache.flink.streaming.api.operators;
import org.apache.flink.api.common.functions.FoldFunction;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.configuration.Configuration;
public class StreamFold<IN, OUT> extends ChainableStreamOperator<IN, OUT> {
private static final long serialVersionUID = 1L;
public class StreamFold<IN, OUT>
extends AbstractUdfStreamOperator<OUT, FoldFunction<IN, OUT>>
implements OneInputStreamOperator<IN, OUT> {
private OUT accumulator;
protected TypeSerializer<OUT> outTypeSerializer;
protected TypeInformation<OUT> outTypeInformation;
public StreamFold(FoldFunction<IN, OUT> folder, OUT initialValue,
TypeInformation<OUT> outTypeInformation) {
public StreamFold(FoldFunction<IN, OUT> folder, OUT initialValue, TypeInformation<OUT> outTypeInformation) {
super(folder);
this.accumulator = initialValue;
this.outTypeSerializer = outTypeInformation.createSerializer(executionConfig);
this.outTypeInformation = outTypeInformation;
this.chainingStrategy = ChainingStrategy.ALWAYS;
}
@Override
public void run() throws Exception {
while (isRunning && readNext() != null) {
callUserFunctionAndLogException();
}
public void processElement(IN element) throws Exception {
accumulator = userFunction.fold(outTypeSerializer.copy(accumulator), element);
output.collect(accumulator);
}
@Override
@SuppressWarnings("unchecked")
protected void callUserFunction() throws Exception {
accumulator = ((FoldFunction<IN, OUT>) userFunction).fold(outTypeSerializer.copy(accumulator), nextObject);
collector.collect(accumulator);
public void open(Configuration config) throws Exception {
super.open(config);
this.outTypeSerializer = outTypeInformation.createSerializer(executionConfig);
}
}
......@@ -25,7 +25,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.functions.KeySelector;
public class StreamGroupedFold<IN, OUT> extends StreamFold<IN, OUT> {
private static final long serialVersionUID = 1L;
private KeySelector<IN, ?> keySelector;
private Map<Object, OUT> values;
......@@ -40,20 +39,19 @@ public class StreamGroupedFold<IN, OUT> extends StreamFold<IN, OUT> {
}
@Override
@SuppressWarnings("unchecked")
protected void callUserFunction() throws Exception {
Object key = nextRecord.getKey(keySelector);
public void processElement(IN element) throws Exception {
Object key = keySelector.getKey(element);
OUT accumulator = values.get(key);
FoldFunction<IN, OUT> folder = ((FoldFunction<IN, OUT>) userFunction);
if (accumulator != null) {
OUT folded = folder.fold(outTypeSerializer.copy(accumulator), nextObject);
OUT folded = folder.fold(outTypeSerializer.copy(accumulator), element);
values.put(key, folded);
collector.collect(folded);
output.collect(folded);
} else {
OUT first = folder.fold(outTypeSerializer.copy(initialValue), nextObject);
OUT first = folder.fold(outTypeSerializer.copy(initialValue), element);
values.put(key, first);
collector.collect(first);
output.collect(first);
}
}
......
......@@ -24,7 +24,6 @@ import org.apache.flink.api.common.functions.ReduceFunction;
import org.apache.flink.api.java.functions.KeySelector;
public class StreamGroupedReduce<IN> extends StreamReduce<IN> {
private static final long serialVersionUID = 1L;
private KeySelector<IN, ?> keySelector;
private Map<Object, IN> values;
......@@ -36,17 +35,17 @@ public class StreamGroupedReduce<IN> extends StreamReduce<IN> {
}
@Override
@SuppressWarnings("unchecked")
protected void callUserFunction() throws Exception {
Object key = keySelector.getKey(nextObject);
public void processElement(IN element) throws Exception {
Object key = keySelector.getKey(element);
IN currentValue = values.get(key);
if (currentValue != null) {
IN reduced = ((ReduceFunction<IN>) userFunction).reduce(copy(currentValue), nextObject);
// TODO: find a way to let operators copy elements (maybe)
IN reduced = userFunction.reduce(currentValue, element);
values.put(key, reduced);
collector.collect(reduced);
output.collect(reduced);
} else {
values.put(key, nextObject);
collector.collect(nextObject);
values.put(key, element);
output.collect(element);
}
}
......
......@@ -19,24 +19,17 @@ package org.apache.flink.streaming.api.operators;
import org.apache.flink.api.common.functions.MapFunction;
public class StreamMap<IN, OUT> extends ChainableStreamOperator<IN, OUT> {
private static final long serialVersionUID = 1L;
public class StreamMap<IN, OUT>
extends AbstractUdfStreamOperator<OUT, MapFunction<IN, OUT>>
implements OneInputStreamOperator<IN, OUT> {
public StreamMap(MapFunction<IN, OUT> mapper) {
super(mapper);
chainingStrategy = ChainingStrategy.ALWAYS;
}
@Override
public void run() throws Exception {
while (isRunning && readNext() != null) {
callUserFunctionAndLogException();
}
public void processElement(IN element) throws Exception {
output.collect(userFunction.map(element));
}
@Override
@SuppressWarnings("unchecked")
protected void callUserFunction() throws Exception {
collector.collect(((MapFunction<IN, OUT>) userFunction).map(nextObject));
}
}
......@@ -17,183 +17,41 @@
package org.apache.flink.streaming.api.operators;
import java.io.IOException;
import java.io.Serializable;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.functions.Function;
import org.apache.flink.api.common.functions.RuntimeContext;
import org.apache.flink.api.common.functions.util.FunctionUtils;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.checkpoint.CheckpointCommitter;
import org.apache.flink.streaming.api.checkpoint.Checkpointed;
import org.apache.flink.streaming.runtime.io.IndexedReaderIterator;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer;
import org.apache.flink.streaming.runtime.tasks.StreamTaskContext;
import org.apache.flink.util.Collector;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.Serializable;
/**
* The StreamOperator represents the base class for all operators in the
* streaming topology.
* Basic interface for stream operators. Implementers would implement one of
* {@link org.apache.flink.streaming.api.operators.OneInputStreamOperator} or
* {@link org.apache.flink.streaming.api.operators.TwoInputStreamOperator} to create operators
* that process elements. You can use
* {@link org.apache.flink.streaming.api.operators.AbstractStreamOperator} as a base class for
* custom operators.
*
* @param <OUT>
* The output type of the operator
* @param <OUT> The output type of the operator
*/
public abstract class StreamOperator<IN, OUT> implements Serializable {
private static final long serialVersionUID = 1L;
private static final Logger LOG = LoggerFactory.getLogger(StreamOperator.class);
protected StreamTaskContext<OUT> taskContext;
protected ExecutionConfig executionConfig = null;
protected IndexedReaderIterator<StreamRecord<IN>> recordIterator;
protected StreamRecordSerializer<IN> inSerializer;
protected TypeSerializer<IN> objectSerializer;
protected StreamRecord<IN> nextRecord;
protected IN nextObject;
public Collector<OUT> collector;
protected Function userFunction;
protected volatile boolean isRunning;
private ChainingStrategy chainingStrategy = ChainingStrategy.HEAD;
public StreamOperator(Function userFunction) {
this.userFunction = userFunction;
}
/**
* Initializes the {@link StreamOperator} for input and output handling
*
* @param taskContext
* StreamTaskContext representing the vertex
*/
public void setup(StreamTaskContext<OUT> taskContext) {
this.collector = taskContext.getOutputCollector();
this.recordIterator = taskContext.getIndexedInput(0);
this.inSerializer = taskContext.getInputSerializer(0);
if (this.inSerializer != null) {
this.nextRecord = inSerializer.createInstance();
this.objectSerializer = inSerializer.getObjectSerializer();
}
this.taskContext = taskContext;
this.executionConfig = taskContext.getExecutionConfig();
}
/**
* Method that will be called when the operator starts, should encode the
* processing logic
*/
public abstract void run() throws Exception;
/*
* Reads the next record from the reader iterator and stores it in the
* nextRecord variable
*/
protected StreamRecord<IN> readNext() throws IOException {
this.nextRecord = inSerializer.createInstance();
try {
nextRecord = recordIterator.next(nextRecord);
try {
nextObject = nextRecord.getObject();
} catch (NullPointerException e) {
// end of stream
}
return nextRecord;
} catch (IOException e) {
if (isRunning) {
throw new RuntimeException("Could not read next record", e);
} else {
// Task already cancelled do nothing
return null;
}
} catch (IllegalStateException e) {
if (isRunning) {
throw new RuntimeException("Could not read next record", e);
} else {
// Task already cancelled do nothing
return null;
}
}
}
public interface StreamOperator<OUT> extends Serializable {
/**
* The call of the user implemented function should be implemented here
* Initializes the {@link StreamOperator} for input and output handling.
*/
protected void callUserFunction() throws Exception {
}
/**
* Method for logging exceptions thrown during the user function call
*/
protected void callUserFunctionAndLogException() {
try {
callUserFunction();
} catch (Exception e) {
if (LOG.isErrorEnabled()) {
LOG.error("Calling user function failed", e);
}
throw new RuntimeException(e);
}
}
public void setup(Output<OUT> output, RuntimeContext runtimeContext);
/**
* Open method to be used if the user defined function extends the
* RichFunction class
*
* @param parameters
* The configuration parameters for the operator
* This method is called before any elements are processed.
*/
public void open(Configuration parameters) throws Exception {
isRunning = true;
FunctionUtils.openFunction(userFunction, parameters);
}
public void open(Configuration config) throws Exception;
/**
* Close method to be used if the user defined function extends the
* RichFunction class
*
* This method is called after no more elements for can arrive for processing.
*/
public void close() {
isRunning = false;
collector.close();
try {
FunctionUtils.closeFunction(userFunction);
} catch (Exception e) {
throw new RuntimeException("Error when closing the function", e);
}
}
public void close() throws Exception;
public void cancel() {
isRunning = false;
}
public void setChainingStrategy(ChainingStrategy strategy);
public void setRuntimeContext(RuntimeContext t) {
FunctionUtils.setFunctionRuntimeContext(userFunction, t);
}
protected IN copy(IN record) {
return objectSerializer.copy(record);
}
public void setChainingStrategy(ChainingStrategy strategy) {
if (strategy == ChainingStrategy.ALWAYS) {
if (!(this instanceof ChainableStreamOperator)) {
throw new RuntimeException("Operator needs to extend ChainableOperator to be chained");
}
}
this.chainingStrategy = strategy;
}
public ChainingStrategy getChainingStrategy() {
return chainingStrategy;
}
public ChainingStrategy getChainingStrategy();
/**
* Defines the chaining scheme for the operator. By default <b>ALWAYS</b> is used,
......@@ -208,53 +66,4 @@ public abstract class StreamOperator<IN, OUT> implements Serializable {
public static enum ChainingStrategy {
ALWAYS, NEVER, HEAD
}
public Function getUserFunction() {
return userFunction;
}
// ------------------------------------------------------------------------
// Checkpoints and Checkpoint Confirmations
// ------------------------------------------------------------------------
// NOTE - ALL OF THIS CODE WORKS ONLY FOR THE FIRST OPERATOR IN THE CHAIN
// IT NEEDS TO BE EXTENDED TO SUPPORT CHAINS
public void restoreInitialState(Serializable state) throws Exception {
if (userFunction instanceof Checkpointed) {
setStateOnFunction(state, userFunction);
}
else {
throw new IllegalStateException("Trying to restore state of a non-checkpointed function");
}
}
public Serializable getStateSnapshotFromFunction(long checkpointId, long timestamp) throws Exception {
if (userFunction instanceof Checkpointed) {
return ((Checkpointed<?>) userFunction).snapshotState(checkpointId, timestamp);
}
else {
return null;
}
}
public void confirmCheckpointCompleted(long checkpointId, long timestamp) throws Exception {
if (userFunction instanceof CheckpointCommitter) {
try {
((CheckpointCommitter) userFunction).commitCheckpoint(checkpointId);
}
catch (Exception e) {
throw new Exception("Error while confirming checkpoint " + checkpointId + " to the stream function", e);
}
}
}
private static <T extends Serializable> void setStateOnFunction(Serializable state, Function function) {
@SuppressWarnings("unchecked")
T typedState = (T) state;
@SuppressWarnings("unchecked")
Checkpointed<T> typedFunction = (Checkpointed<T>) function;
typedFunction.restoreState(typedState);
}
}
......@@ -22,8 +22,9 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.configuration.Configuration;
public class StreamProject<IN, OUT extends Tuple> extends ChainableStreamOperator<IN, OUT> {
private static final long serialVersionUID = 1L;
public class StreamProject<IN, OUT extends Tuple>
extends AbstractStreamOperator<OUT>
implements OneInputStreamOperator<IN, OUT> {
transient OUT outTuple;
TypeSerializer<OUT> outTypeSerializer;
......@@ -32,25 +33,20 @@ public class StreamProject<IN, OUT extends Tuple> extends ChainableStreamOperato
int numFields;
public StreamProject(int[] fields, TypeInformation<OUT> outTypeInformation) {
super(null);
this.fields = fields;
this.numFields = this.fields.length;
this.outTypeInformation = outTypeInformation;
}
@Override
public void run() throws Exception {
while (isRunning && readNext() != null) {
callUserFunctionAndLogException();
}
chainingStrategy = ChainingStrategy.ALWAYS;
}
@Override
protected void callUserFunction() throws Exception {
public void processElement(IN element) throws Exception {
for (int i = 0; i < this.numFields; i++) {
outTuple.setField(((Tuple)nextObject).getField(fields[i]), i);
outTuple.setField(((Tuple) element).getField(fields[i]), i);
}
collector.collect(outTuple);
output.collect(outTuple);
}
@Override
......@@ -59,5 +55,4 @@ public class StreamProject<IN, OUT extends Tuple> extends ChainableStreamOperato
this.outTypeSerializer = outTypeInformation.createSerializer(executionConfig);
outTuple = outTypeSerializer.createInstance();
}
}
......@@ -19,35 +19,28 @@ package org.apache.flink.streaming.api.operators;
import org.apache.flink.api.common.functions.ReduceFunction;
public class StreamReduce<IN> extends ChainableStreamOperator<IN, IN> {
private static final long serialVersionUID = 1L;
public class StreamReduce<IN> extends AbstractUdfStreamOperator<IN, ReduceFunction<IN>>
implements OneInputStreamOperator<IN, IN> {
private IN currentValue;
public StreamReduce(ReduceFunction<IN> reducer) {
super(reducer);
currentValue = null;
}
@Override
public void run() throws Exception {
while (isRunning && readNext() != null) {
callUserFunctionAndLogException();
}
chainingStrategy = ChainingStrategy.ALWAYS;
}
@Override
@SuppressWarnings("unchecked")
protected void callUserFunction() throws Exception {
public void processElement(IN element) throws Exception {
if (currentValue != null) {
currentValue = ((ReduceFunction<IN>) userFunction).reduce(copy(currentValue), nextObject);
// TODO: give operator a way to specify that elements should be copied
currentValue = userFunction.reduce(currentValue, element);
} else {
currentValue = nextObject;
currentValue = element;
}
collector.collect(currentValue);
output.collect(currentValue);
}
}
......@@ -19,23 +19,17 @@ package org.apache.flink.streaming.api.operators;
import org.apache.flink.streaming.api.functions.sink.SinkFunction;
public class StreamSink<IN> extends ChainableStreamOperator<IN, IN> {
private static final long serialVersionUID = 1L;
public class StreamSink<IN> extends AbstractUdfStreamOperator<Object, SinkFunction<IN>>
implements OneInputStreamOperator<IN, Object> {
public StreamSink(SinkFunction<IN> sinkFunction) {
super(sinkFunction);
}
@Override
public void run() throws Exception {
while (isRunning && readNext() != null) {
callUserFunctionAndLogException();
}
chainingStrategy = ChainingStrategy.ALWAYS;
}
@Override
@SuppressWarnings("unchecked")
protected void callUserFunction() throws Exception {
((SinkFunction<IN>) userFunction).invoke(nextObject);
public void processElement(IN element) throws Exception {
userFunction.invoke(element);
}
}
......@@ -17,33 +17,27 @@
package org.apache.flink.streaming.api.operators;
import java.io.Serializable;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
public class StreamSource<OUT> extends StreamOperator<OUT, OUT> implements Serializable {
private static final long serialVersionUID = 1L;
public class StreamSource<OUT> extends AbstractUdfStreamOperator<OUT, SourceFunction<OUT>> implements StreamOperator<OUT> {
public StreamSource(SourceFunction<OUT> sourceFunction) {
super(sourceFunction);
}
@Override
public void run() {
callUserFunctionAndLogException();
}
public void run() throws Exception {
while (true) {
@Override
@SuppressWarnings("unchecked")
protected void callUserFunction() throws Exception {
((SourceFunction<OUT>) userFunction).run(collector);
}
synchronized (userFunction) {
if (userFunction.reachedEnd()) {
break;
}
OUT result = userFunction.next();
@Override
@SuppressWarnings("unchecked")
public void cancel() {
super.cancel();
((SourceFunction<OUT>) userFunction).cancel();
output.collect(result);
}
Thread.yield();
}
}
}
/*
* 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.streaming.api.operators;
/**
* Interface for stream operators with two inputs. Use
* {@link org.apache.flink.streaming.api.operators.AbstractStreamOperator} as a base class if
* you want to implement a custom operator.
*
* @param <IN1> The input type of the operator
* @param <IN2> The input type of the operator
* @param <OUT> The output type of the operator
*/
public interface TwoInputStreamOperator<IN1, IN2, OUT> extends StreamOperator<OUT> {
public void processElement1(IN1 element) throws Exception;
public void processElement2(IN2 element) throws Exception;
}
......@@ -18,36 +18,25 @@
package org.apache.flink.streaming.api.operators.co;
import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction;
import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
public class CoStreamFlatMap<IN1, IN2, OUT> extends CoStreamOperator<IN1, IN2, OUT> {
private static final long serialVersionUID = 1L;
public class CoStreamFlatMap<IN1, IN2, OUT>
extends AbstractUdfStreamOperator<OUT, CoFlatMapFunction<IN1, IN2, OUT>>
implements TwoInputStreamOperator<IN1, IN2, OUT> {
public CoStreamFlatMap(CoFlatMapFunction<IN1, IN2, OUT> flatMapper) {
super(flatMapper);
}
@Override
public void handleStream1() throws Exception {
callUserFunctionAndLogException1();
}
@Override
public void handleStream2() throws Exception {
callUserFunctionAndLogException2();
}
@Override
@SuppressWarnings("unchecked")
protected void callUserFunction1() throws Exception {
((CoFlatMapFunction<IN1, IN2, OUT>) userFunction).flatMap1(reuse1.getObject(), collector);
public void processElement1(IN1 element) throws Exception {
userFunction.flatMap1(element, output);
}
@Override
@SuppressWarnings("unchecked")
protected void callUserFunction2() throws Exception {
((CoFlatMapFunction<IN1, IN2, OUT>) userFunction).flatMap2(reuse2.getObject(), collector);
public void processElement2(IN2 element) throws Exception {
userFunction.flatMap2(element, output);
}
}
......@@ -43,51 +43,30 @@ public class CoStreamGroupedReduce<IN1, IN2, OUT> extends CoStreamReduce<IN1, IN
}
@Override
@SuppressWarnings("unchecked")
public void handleStream1() throws Exception {
CoReduceFunction<IN1, IN2, OUT> coReducer = (CoReduceFunction<IN1, IN2, OUT>) userFunction;
Object key = reuse1.getKey(keySelector1);
public void processElement1(IN1 element) throws Exception {
Object key = keySelector1.getKey(element);
currentValue1 = values1.get(key);
nextValue1 = reuse1.getObject();
if (currentValue1 != null) {
callUserFunctionAndLogException1();
reduced1 = userFunction.reduce1(currentValue1, element);
values1.put(key, reduced1);
collector.collect(coReducer.map1(reduced1));
output.collect(userFunction.map1(reduced1));
} else {
values1.put(key, nextValue1);
collector.collect(coReducer.map1(nextValue1));
values1.put(key, element);
output.collect(userFunction.map1(element));
}
}
@Override
@SuppressWarnings("unchecked")
public void handleStream2() throws Exception {
CoReduceFunction<IN1, IN2, OUT> coReducer = (CoReduceFunction<IN1, IN2, OUT>) userFunction;
Object key = reuse2.getKey(keySelector2);
public void processElement2(IN2 element) throws Exception {
Object key = keySelector2.getKey(element);
currentValue2 = values2.get(key);
nextValue2 = reuse2.getObject();
if (currentValue2 != null) {
callUserFunctionAndLogException2();
reduced2 = userFunction.reduce2(currentValue2, element);
values2.put(key, reduced2);
collector.collect(coReducer.map2(reduced2));
output.collect(userFunction.map2(reduced2));
} else {
values2.put(key, nextValue2);
collector.collect(coReducer.map2(nextValue2));
values2.put(key, element);
output.collect(userFunction.map2(element));
}
}
@Override
@SuppressWarnings("unchecked")
protected void callUserFunction1() throws Exception {
reduced1 = ((CoReduceFunction<IN1, IN2, OUT>) userFunction).reduce1(currentValue1, nextValue1);
}
@Override
@SuppressWarnings("unchecked")
protected void callUserFunction2() throws Exception {
reduced2 = ((CoReduceFunction<IN1, IN2, OUT>) userFunction).reduce2(currentValue2, nextValue2);
}
}
......@@ -18,37 +18,26 @@
package org.apache.flink.streaming.api.operators.co;
import org.apache.flink.streaming.api.functions.co.CoMapFunction;
import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
public class CoStreamMap<IN1, IN2, OUT> extends CoStreamOperator<IN1, IN2, OUT> {
private static final long serialVersionUID = 1L;
private CoMapFunction<IN1, IN2, OUT> mapper;
public class CoStreamMap<IN1, IN2, OUT>
extends AbstractUdfStreamOperator<OUT, CoMapFunction<IN1, IN2, OUT>>
implements TwoInputStreamOperator<IN1, IN2, OUT> {
public CoStreamMap(CoMapFunction<IN1, IN2, OUT> mapper) {
super(mapper);
this.mapper = mapper;
}
@Override
public void handleStream1() throws Exception {
callUserFunctionAndLogException1();
}
@Override
public void handleStream2() throws Exception {
callUserFunctionAndLogException2();
}
@Override
protected void callUserFunction1() throws Exception {
collector.collect(mapper.map1(reuse1.getObject()));
public void processElement1(IN1 element) throws Exception {
output.collect(userFunction.map1(element));
}
@Override
protected void callUserFunction2() throws Exception {
collector.collect(mapper.map2(reuse2.getObject()));
public void processElement2(IN2 element) throws Exception {
output.collect(userFunction.map2(element));
}
}
/*
* 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.streaming.api.operators.co;
import java.io.IOException;
import org.apache.flink.api.common.functions.Function;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.streaming.api.operators.StreamOperator;
import org.apache.flink.streaming.runtime.io.CoReaderIterator;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer;
import org.apache.flink.streaming.runtime.tasks.StreamTaskContext;
import org.apache.flink.util.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public abstract class CoStreamOperator<IN1, IN2, OUT> extends StreamOperator<IN1, OUT> {
public CoStreamOperator(Function userFunction) {
super(userFunction);
}
private static final long serialVersionUID = 1L;
private static final Logger LOG = LoggerFactory.getLogger(CoStreamOperator.class);
protected CoReaderIterator<StreamRecord<IN1>, StreamRecord<IN2>> recordIterator;
protected StreamRecord<IN1> reuse1;
protected StreamRecord<IN2> reuse2;
protected StreamRecordSerializer<IN1> srSerializer1;
protected StreamRecordSerializer<IN2> srSerializer2;
protected TypeSerializer<IN1> serializer1;
protected TypeSerializer<IN2> serializer2;
@Override
public void setup(StreamTaskContext<OUT> taskContext) {
this.collector = taskContext.getOutputCollector();
this.recordIterator = taskContext.getCoReader();
this.srSerializer1 = taskContext.getInputSerializer(0);
this.srSerializer2 = taskContext.getInputSerializer(1);
this.reuse1 = srSerializer1.createInstance();
this.reuse2 = srSerializer2.createInstance();
this.serializer1 = srSerializer1.getObjectSerializer();
this.serializer2 = srSerializer2.getObjectSerializer();
}
protected void resetReuseAll() {
this.reuse1 = srSerializer1.createInstance();
this.reuse2 = srSerializer2.createInstance();
}
protected void resetReuse1() {
this.reuse1 = srSerializer1.createInstance();
}
protected void resetReuse2() {
this.reuse2 = srSerializer2.createInstance();
}
@Override
public void run() throws Exception {
while (isRunning) {
int next;
try {
next = recordIterator.next(reuse1, reuse2);
} catch (IOException e) {
if (isRunning) {
throw new RuntimeException("Could not read next record.", e);
} else {
// Task already cancelled do nothing
next = 0;
}
} catch (IllegalStateException e) {
if (isRunning) {
throw new RuntimeException("Could not read next record.", e);
} else {
// Task already cancelled do nothing
next = 0;
}
}
if (next == 0) {
break;
} else if (next == 1) {
initialize1();
handleStream1();
resetReuse1();
} else {
initialize2();
handleStream2();
resetReuse2();
}
}
}
protected abstract void handleStream1() throws Exception;
protected abstract void handleStream2() throws Exception;
protected abstract void callUserFunction1() throws Exception;
protected abstract void callUserFunction2() throws Exception;
protected void initialize1() {
}
protected void initialize2() {
}
protected void callUserFunctionAndLogException1() {
try {
callUserFunction1();
} catch (Exception e) {
if (LOG.isErrorEnabled()) {
LOG.error("Calling user function failed due to: {}",
StringUtils.stringifyException(e));
}
throw new RuntimeException(e);
}
}
protected void callUserFunctionAndLogException2() {
try {
callUserFunction2();
} catch (Exception e) {
if (LOG.isErrorEnabled()) {
LOG.error("Calling user function failed due to: {}",
StringUtils.stringifyException(e));
}
throw new RuntimeException(e);
}
}
}
......@@ -18,14 +18,15 @@
package org.apache.flink.streaming.api.operators.co;
import org.apache.flink.streaming.api.functions.co.CoReduceFunction;
import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
public class CoStreamReduce<IN1, IN2, OUT> extends CoStreamOperator<IN1, IN2, OUT> {
private static final long serialVersionUID = 1L;
public class CoStreamReduce<IN1, IN2, OUT>
extends AbstractUdfStreamOperator<OUT, CoReduceFunction<IN1, IN2, OUT>>
implements TwoInputStreamOperator<IN1, IN2, OUT> {
protected IN1 currentValue1 = null;
protected IN2 currentValue2 = null;
protected IN1 nextValue1 = null;
protected IN2 nextValue2 = null;
public CoStreamReduce(CoReduceFunction<IN1, IN2, OUT> coReducer) {
super(coReducer);
......@@ -34,39 +35,23 @@ public class CoStreamReduce<IN1, IN2, OUT> extends CoStreamOperator<IN1, IN2, OU
}
@Override
public void handleStream1() throws Exception {
nextValue1 = reuse1.getObject();
callUserFunctionAndLogException1();
}
@Override
public void handleStream2() throws Exception {
nextValue2 = reuse2.getObject();
callUserFunctionAndLogException2();
}
@Override
@SuppressWarnings("unchecked")
protected void callUserFunction1() throws Exception {
CoReduceFunction<IN1, IN2, OUT> coReducer = (CoReduceFunction<IN1, IN2, OUT>) userFunction;
public void processElement1(IN1 element) throws Exception {
if (currentValue1 != null) {
currentValue1 = coReducer.reduce1(currentValue1, nextValue1);
currentValue1 = userFunction.reduce1(currentValue1, element);
} else {
currentValue1 = nextValue1;
currentValue1 = element;
}
collector.collect(coReducer.map1(currentValue1));
output.collect(userFunction.map1(currentValue1));
}
@Override
@SuppressWarnings("unchecked")
protected void callUserFunction2() throws Exception {
CoReduceFunction<IN1, IN2, OUT> coReducer = (CoReduceFunction<IN1, IN2, OUT>) userFunction;
public void processElement2(IN2 element) throws Exception {
if (currentValue2 != null) {
currentValue2 = coReducer.reduce2(currentValue2, nextValue2);
currentValue2 = userFunction.reduce2(currentValue2, element);
} else {
currentValue2 = nextValue2;
currentValue2 = element;
}
collector.collect(coReducer.map2(currentValue2));
output.collect(userFunction.map2(currentValue2));
}
}
......@@ -23,12 +23,15 @@ import java.util.List;
import org.apache.commons.math.util.MathUtils;
import org.apache.flink.streaming.api.functions.co.CoWindowFunction;
import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
import org.apache.flink.streaming.api.state.CircularFifoList;
import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
public class CoStreamWindow<IN1, IN2, OUT> extends CoStreamOperator<IN1, IN2, OUT> {
private static final long serialVersionUID = 1L;
public class CoStreamWindow<IN1, IN2, OUT>
extends AbstractUdfStreamOperator<OUT, CoWindowFunction<IN1, IN2, OUT>>
implements TwoInputStreamOperator<IN1, IN2, OUT> {
protected long windowSize;
protected long slideSize;
......@@ -57,31 +60,32 @@ public class CoStreamWindow<IN1, IN2, OUT> extends CoStreamOperator<IN1, IN2, OU
}
@Override
protected void handleStream1() throws Exception {
window.addToBuffer1(reuse1.getObject());
public void processElement1(IN1 element) throws Exception {
window.addToBuffer1(element);
}
@Override
protected void handleStream2() throws Exception {
window.addToBuffer2(reuse2.getObject());
public void processElement2(IN2 element) throws Exception {
window.addToBuffer2(element);
}
@Override
@SuppressWarnings("unchecked")
protected void callUserFunction() throws Exception {
List<IN1> first = new ArrayList<IN1>();
List<IN2> second = new ArrayList<IN2>();
// TODO: Give operators a way to copy elements
for (IN1 element : window.circularList1.getElements()) {
first.add(serializer1.copy(element));
first.add(element);
}
for (IN2 element : window.circularList2.getElements()) {
second.add(serializer2.copy(element));
second.add(element);
}
if (!window.circularList1.isEmpty() || !window.circularList2.isEmpty()) {
((CoWindowFunction<IN1, IN2, OUT>) userFunction).coWindow(first, second, collector);
userFunction.coWindow(first, second, output);
}
}
......@@ -120,7 +124,7 @@ public class CoStreamWindow<IN1, IN2, OUT> extends CoStreamOperator<IN1, IN2, OU
}
}
protected synchronized void checkWindowEnd(long timeStamp) {
protected synchronized void checkWindowEnd(long timeStamp) throws Exception{
nextRecordTime = timeStamp;
while (miniBatchEnd()) {
......@@ -128,7 +132,7 @@ public class CoStreamWindow<IN1, IN2, OUT> extends CoStreamOperator<IN1, IN2, OU
circularList2.newSlide();
minibatchCounter++;
if (windowEnd()) {
callUserFunctionAndLogException();
callUserFunction();
circularList1.shiftWindow(batchPerSlide);
circularList2.shiftWindow(batchPerSlide);
}
......@@ -152,9 +156,9 @@ public class CoStreamWindow<IN1, IN2, OUT> extends CoStreamOperator<IN1, IN2, OU
return false;
}
public void reduceLastBatch() {
public void reduceLastBatch() throws Exception{
if (!miniBatchEnd()) {
callUserFunctionAndLogException();
callUserFunction();
}
}
......@@ -174,21 +178,17 @@ public class CoStreamWindow<IN1, IN2, OUT> extends CoStreamOperator<IN1, IN2, OU
}
@Override
public void close() {
public void close() throws Exception {
if (!window.miniBatchEnd()) {
callUserFunctionAndLogException();
try {
callUserFunction();
} catch (Exception e) {
throw new RuntimeException("Could not call user function in CoStreamWindow.close()", e);
}
}
super.close();
}
@Override
protected void callUserFunction1() throws Exception {
}
@Override
protected void callUserFunction2() throws Exception {
}
public void setSlideSize(long slideSize) {
this.slideSize = slideSize;
}
......
......@@ -20,13 +20,19 @@ package org.apache.flink.streaming.api.operators.windowing;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.streaming.api.windowing.policy.CentralActiveTrigger;
import org.apache.flink.streaming.api.windowing.policy.CloneableEvictionPolicy;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class GroupedActiveDiscretizer<IN> extends GroupedStreamDiscretizer<IN> {
private static final Logger LOG = LoggerFactory.getLogger(GroupedActiveDiscretizer.class);
private static final long serialVersionUID = -3469545957144404137L;
private volatile IN last;
private Thread centralThread;
private CentralCheck centralCheck;
public GroupedActiveDiscretizer(KeySelector<IN, ?> keySelector,
CentralActiveTrigger<IN> triggerPolicy, CloneableEvictionPolicy<IN> evictionPolicy) {
......@@ -39,18 +45,18 @@ public class GroupedActiveDiscretizer<IN> extends GroupedStreamDiscretizer<IN> {
StreamDiscretizer<IN> groupDiscretizer = new StreamDiscretizer<IN>(triggerPolicy.clone(),
evictionPolicy.clone());
groupDiscretizer.collector = taskContext.getOutputCollector();
groupDiscretizer.setup(this.output, this.runtimeContext);
// We omit the groupDiscretizer.open(...) call here to avoid starting
// new active threads
return groupDiscretizer;
}
@Override
public void run() throws Exception {
public void processElement(IN element) throws Exception {
while (isRunning && readNext() != null) {
last = copy(nextObject);
Object key = keySelector.getKey(nextObject);
// last = copy(element);
last = element;
Object key = keySelector.getKey(element);
synchronized (groupedDiscretizers) {
StreamDiscretizer<IN> groupDiscretizer = groupedDiscretizers.get(key);
......@@ -60,29 +66,46 @@ public class GroupedActiveDiscretizer<IN> extends GroupedStreamDiscretizer<IN> {
groupedDiscretizers.put(key, groupDiscretizer);
}
groupDiscretizer.processRealElement(nextObject);
groupDiscretizer.processRealElement(element);
}
}
for (StreamDiscretizer<IN> group : groupedDiscretizers.values()) {
group.emitWindow();
}
}
@Override
public void open(org.apache.flink.configuration.Configuration parameters) throws Exception {
super.open(parameters);
centralThread = new Thread(new CentralCheck());
centralCheck = new CentralCheck();
centralThread = new Thread(centralCheck);
centralThread.start();
}
@Override
public void close() throws Exception {
super.close();
for (StreamDiscretizer<IN> group : groupedDiscretizers.values()) {
group.emitWindow();
}
try {
centralCheck.running = false;
centralThread.interrupt();
centralThread.join();
} catch (InterruptedException e) {
e.printStackTrace();
LOG.info("GroupedActiveDiscretizer got interruped while joining with central thread: {}", e);
}
}
private class CentralCheck implements Runnable {
volatile boolean running = true;
@Override
public void run() {
while (isRunning) {
while (running) {
// wait for the specified granularity
try {
Thread.sleep(2000);
......
......@@ -59,11 +59,18 @@ public class GroupedStreamDiscretizer<IN> extends StreamDiscretizer<IN> {
}
@Override
public void run() throws Exception {
public void close() throws Exception {
super.close();
for (StreamDiscretizer<IN> group : groupedDiscretizers.values()) {
group.emitWindow();
}
}
@Override
public void processElement(IN element) throws Exception {
while (isRunning && readNext() != null) {
Object key = keySelector.getKey(nextObject);
Object key = keySelector.getKey(element);
StreamDiscretizer<IN> groupDiscretizer = groupedDiscretizers.get(key);
......@@ -72,12 +79,7 @@ public class GroupedStreamDiscretizer<IN> extends StreamDiscretizer<IN> {
groupedDiscretizers.put(key, groupDiscretizer);
}
groupDiscretizer.processRealElement(nextObject);
}
for (StreamDiscretizer<IN> group : groupedDiscretizers.values()) {
group.emitWindow();
}
groupDiscretizer.processRealElement(element);
}
......@@ -95,9 +97,12 @@ public class GroupedStreamDiscretizer<IN> extends StreamDiscretizer<IN> {
StreamDiscretizer<IN> groupDiscretizer = new StreamDiscretizer<IN>(triggerPolicy.clone(),
evictionPolicy.clone());
groupDiscretizer.collector = taskContext.getOutputCollector();
// groupDiscretizer.output = taskContext.getOutputCollector();
// TODO: this seems very hacky, maybe we can get around this
groupDiscretizer.setup(this.output, this.runtimeContext);
groupDiscretizer.open(this.parameters);
return groupDiscretizer;
}
......
......@@ -31,7 +31,6 @@ import org.apache.flink.streaming.api.windowing.windowbuffer.WindowBuffer;
*/
public class GroupedWindowBuffer<T> extends StreamWindowBuffer<T> {
private static final long serialVersionUID = 1L;
private Map<Object, WindowBuffer<T>> windowMap = new HashMap<Object, WindowBuffer<T>>();
private KeySelector<T, ?> keySelector;
......@@ -41,16 +40,9 @@ public class GroupedWindowBuffer<T> extends StreamWindowBuffer<T> {
}
@Override
public void run() throws Exception {
while (isRunning && readNext() != null) {
callUserFunctionAndLogException();
}
}
@Override
protected void callUserFunction() throws Exception {
if (nextObject.getElement() != null) {
Object key = keySelector.getKey(nextObject.getElement());
public void processElement(WindowEvent<T> event) throws Exception {
if (event.getElement() != null) {
Object key = keySelector.getKey(event.getElement());
WindowBuffer<T> currentWindow = windowMap.get(key);
if (currentWindow == null) {
......@@ -58,15 +50,7 @@ public class GroupedWindowBuffer<T> extends StreamWindowBuffer<T> {
windowMap.put(key, currentWindow);
}
handleWindowEvent(nextObject, currentWindow);
}
}
@Override
public void collect(WindowEvent<T> record) {
if (isRunning) {
nextObject = record;
callUserFunctionAndLogException();
handleWindowEvent(event, currentWindow);
}
}
......
......@@ -17,7 +17,8 @@
package org.apache.flink.streaming.api.operators.windowing;
import org.apache.flink.streaming.api.operators.StreamOperator;
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.windowing.StreamWindow;
import org.apache.flink.streaming.api.windowing.WindowEvent;
import org.apache.flink.streaming.api.windowing.policy.ActiveEvictionPolicy;
......@@ -31,7 +32,9 @@ import org.apache.flink.streaming.api.windowing.policy.TriggerPolicy;
* The user supplied eviction and trigger policies are applied to create the
* {@link StreamWindow} that will be further transformed in the next stages.
*/
public class StreamDiscretizer<IN> extends StreamOperator<IN, WindowEvent<IN>> {
public class StreamDiscretizer<IN>
extends AbstractStreamOperator<WindowEvent<IN>>
implements OneInputStreamOperator<IN, WindowEvent<IN>> {
/**
* Auto-generated serial version UID
......@@ -48,8 +51,6 @@ public class StreamDiscretizer<IN> extends StreamOperator<IN, WindowEvent<IN>> {
protected WindowEvent<IN> windowEvent = new WindowEvent<IN>();
public StreamDiscretizer(TriggerPolicy<IN> triggerPolicy, EvictionPolicy<IN> evictionPolicy) {
super(null);
this.triggerPolicy = triggerPolicy;
this.evictionPolicy = evictionPolicy;
......@@ -66,19 +67,8 @@ public class StreamDiscretizer<IN> extends StreamOperator<IN, WindowEvent<IN>> {
}
@Override
public void run() throws Exception {
// Continuously run
while (isRunning && readNext() != null) {
processRealElement(nextObject);
}
if (activePolicyThread != null) {
activePolicyThread.interrupt();
}
emitWindow();
public void processElement(IN element) throws Exception {
processRealElement(element);
}
/**
......@@ -111,7 +101,7 @@ public class StreamDiscretizer<IN> extends StreamOperator<IN, WindowEvent<IN>> {
evict(input, isTriggered);
collector.collect(windowEvent.setElement(input));
output.collect(windowEvent.setElement(input));
bufferSize++;
}
......@@ -140,7 +130,7 @@ public class StreamDiscretizer<IN> extends StreamOperator<IN, WindowEvent<IN>> {
* if not empty
*/
protected void emitWindow() {
collector.collect(windowEvent.setTrigger());
output.collect(windowEvent.setTrigger());
}
private void activeEvict(Object input) {
......@@ -152,7 +142,7 @@ public class StreamDiscretizer<IN> extends StreamOperator<IN, WindowEvent<IN>> {
}
if (numToEvict > 0) {
collector.collect(windowEvent.setEviction(numToEvict));
output.collect(windowEvent.setEviction(numToEvict));
bufferSize -= numToEvict;
bufferSize = bufferSize >= 0 ? bufferSize : 0;
}
......@@ -162,7 +152,7 @@ public class StreamDiscretizer<IN> extends StreamOperator<IN, WindowEvent<IN>> {
int numToEvict = evictionPolicy.notifyEviction(input, isTriggered, bufferSize);
if (numToEvict > 0) {
collector.collect(windowEvent.setEviction(numToEvict));
output.collect(windowEvent.setEviction(numToEvict));
bufferSize -= numToEvict;
bufferSize = bufferSize >= 0 ? bufferSize : 0;
}
......@@ -183,6 +173,16 @@ public class StreamDiscretizer<IN> extends StreamOperator<IN, WindowEvent<IN>> {
}
}
@Override
public void close() throws Exception {
super.close();
if (activePolicyThread != null) {
activePolicyThread.interrupt();
}
emitWindow();
}
/**
* This class allows the active trigger thread to call back and push fake
* elements at any time.
......
......@@ -17,7 +17,8 @@
package org.apache.flink.streaming.api.operators.windowing;
import org.apache.flink.streaming.api.operators.ChainableStreamOperator;
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.windowing.StreamWindow;
import org.apache.flink.streaming.api.windowing.WindowEvent;
import org.apache.flink.streaming.api.windowing.windowbuffer.WindowBuffer;
......@@ -25,28 +26,20 @@ import org.apache.flink.streaming.api.windowing.windowbuffer.WindowBuffer;
/**
* This operator manages the window buffers attached to the discretizers.
*/
public class StreamWindowBuffer<T> extends ChainableStreamOperator<WindowEvent<T>, StreamWindow<T>> {
public class StreamWindowBuffer<T>
extends AbstractStreamOperator<StreamWindow<T>>
implements OneInputStreamOperator<WindowEvent<T>, StreamWindow<T>> {
protected WindowBuffer<T> buffer;
public StreamWindowBuffer(WindowBuffer<T> buffer) {
super(null);
this.buffer = buffer;
withoutInputCopy();
}
private static final long serialVersionUID = 1L;
@Override
public void run() throws Exception {
while (isRunning && readNext() != null) {
callUserFunctionAndLogException();
}
setChainingStrategy(ChainingStrategy.ALWAYS);
}
@Override
protected void callUserFunction() throws Exception {
handleWindowEvent(nextObject);
public void processElement(WindowEvent<T> windowEvent) throws Exception {
handleWindowEvent(windowEvent);
}
protected void handleWindowEvent(WindowEvent<T> windowEvent, WindowBuffer<T> buffer)
......@@ -56,7 +49,7 @@ public class StreamWindowBuffer<T> extends ChainableStreamOperator<WindowEvent<T
} else if (windowEvent.isEviction()) {
buffer.evict(windowEvent.getEviction());
} else if (windowEvent.isTrigger()) {
buffer.emitWindow(collector);
buffer.emitWindow(output);
}
}
......
......@@ -17,34 +17,25 @@
package org.apache.flink.streaming.api.operators.windowing;
import org.apache.flink.streaming.api.operators.ChainableStreamOperator;
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.windowing.StreamWindow;
/**
* This operator flattens the results of the window transformations by
* outputing the elements of the {@link StreamWindow} one-by-one
*/
public class WindowFlattener<T> extends ChainableStreamOperator<StreamWindow<T>, T> {
public class WindowFlattener<T> extends AbstractStreamOperator<T>
implements OneInputStreamOperator<StreamWindow<T>, T> {
public WindowFlattener() {
super(null);
withoutInputCopy();
}
private static final long serialVersionUID = 1L;
@Override
public void run() throws Exception {
while (isRunning && readNext() != null) {
callUserFunctionAndLogException();
}
chainingStrategy = ChainingStrategy.ALWAYS;
}
@Override
protected void callUserFunction() throws Exception {
for (T element : nextObject) {
collector.collect(element);
public void processElement(StreamWindow<T> window) throws Exception {
for (T element : window) {
output.collect(element);
}
}
}
......@@ -38,7 +38,6 @@ public class WindowFolder<IN, OUT> extends StreamMap<StreamWindow<IN>, StreamWin
public WindowFolder(FoldFunction<IN, OUT> folder, OUT initialValue) {
super(new WindowFoldFunction<IN, OUT>(folder, initialValue));
this.folder = folder;
withoutInputCopy();
}
private static class WindowFoldFunction<IN, OUT> extends AbstractRichFunction implements
......
......@@ -39,7 +39,6 @@ public class WindowMapper<IN, OUT> extends StreamMap<StreamWindow<IN>, StreamWin
public WindowMapper(WindowMapFunction<IN, OUT> mapper) {
super(new WindowMap<IN, OUT>(mapper));
this.mapper = mapper;
withoutInputCopy();
}
private static class WindowMap<T, R> extends AbstractRichFunction
......@@ -70,4 +69,4 @@ public class WindowMapper<IN, OUT> extends StreamMap<StreamWindow<IN>, StreamWin
}
}
\ No newline at end of file
}
......@@ -20,7 +20,8 @@ package org.apache.flink.streaming.api.operators.windowing;
import java.util.HashMap;
import java.util.Map;
import org.apache.flink.streaming.api.operators.ChainableStreamOperator;
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.windowing.StreamWindow;
/**
......@@ -28,29 +29,19 @@ import org.apache.flink.streaming.api.windowing.StreamWindow;
* {@link StreamWindow}s used to merge the results of parallel transformations
* that belong in the same window.
*/
public class WindowMerger<T> extends ChainableStreamOperator<StreamWindow<T>, StreamWindow<T>> {
public class WindowMerger<T> extends AbstractStreamOperator<StreamWindow<T>>
implements OneInputStreamOperator<StreamWindow<T>, StreamWindow<T>> {
private Map<Integer, StreamWindow<T>> windows;
public WindowMerger() {
super(null);
this.windows = new HashMap<Integer, StreamWindow<T>>();
withoutInputCopy();
}
private static final long serialVersionUID = 1L;
@Override
public void run() throws Exception {
while (isRunning && readNext() != null) {
callUserFunctionAndLogException();
}
chainingStrategy = ChainingStrategy.ALWAYS;
}
@Override
@SuppressWarnings("unchecked")
protected void callUserFunction() throws Exception {
StreamWindow<T> nextWindow = nextObject;
public void processElement(StreamWindow<T> nextWindow) throws Exception {
StreamWindow<T> current = windows.get(nextWindow.windowID);
......@@ -61,7 +52,7 @@ public class WindowMerger<T> extends ChainableStreamOperator<StreamWindow<T>, St
}
if (current.numberOfParts == 1) {
collector.collect(current);
output.collect(current);
windows.remove(nextWindow.windowID);
} else {
windows.put(nextWindow.windowID, current);
......
......@@ -18,55 +18,48 @@
package org.apache.flink.streaming.api.operators.windowing;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.streaming.api.operators.ChainableStreamOperator;
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.windowing.StreamWindow;
/**
* This operator applies either split or key partitioning depending on the
* transformation.
*/
public class WindowPartitioner<T> extends ChainableStreamOperator<StreamWindow<T>, StreamWindow<T>> {
public class WindowPartitioner<T> extends AbstractStreamOperator<StreamWindow<T>>
implements OneInputStreamOperator<StreamWindow<T>, StreamWindow<T>> {
private KeySelector<T, ?> keySelector;
private int numberOfSplits;
public WindowPartitioner(KeySelector<T, ?> keySelector) {
super(null);
this.keySelector = keySelector;
withoutInputCopy();
chainingStrategy = ChainingStrategy.ALWAYS;
}
public WindowPartitioner(int numberOfSplits) {
super(null);
this.numberOfSplits = numberOfSplits;
withoutInputCopy();
}
private static final long serialVersionUID = 1L;
@Override
public void run() throws Exception {
while (isRunning && readNext() != null) {
callUserFunctionAndLogException();
}
chainingStrategy = ChainingStrategy.ALWAYS;
}
@Override
protected void callUserFunction() throws Exception {
StreamWindow<T> currentWindow = nextObject;
public void processElement(StreamWindow<T> currentWindow) throws Exception {
if (keySelector == null) {
if (numberOfSplits <= 1) {
collector.collect(currentWindow);
output.collect(currentWindow);
} else {
for (StreamWindow<T> window : StreamWindow.split(currentWindow, numberOfSplits)) {
collector.collect(window);
output.collect(window);
}
}
} else {
for (StreamWindow<T> window : StreamWindow
.partitionBy(currentWindow, keySelector, true)) {
collector.collect(window);
output.collect(window);
}
}
......
......@@ -39,7 +39,6 @@ public class WindowReducer<IN> extends StreamMap<StreamWindow<IN>, StreamWindow<
public WindowReducer(ReduceFunction<IN> reducer) {
super(new WindowReduceFunction<IN>(reducer));
this.reducer = reducer;
withoutInputCopy();
}
private static class WindowReduceFunction<T> extends AbstractRichFunction implements
......
/*
* 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
* 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
* 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,
......@@ -17,78 +18,119 @@
package org.apache.flink.streaming.runtime.tasks;
import java.io.IOException;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.runtime.io.network.api.reader.MutableReader;
import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
import org.apache.flink.runtime.plugable.DeserializationDelegate;
import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.runtime.io.IndexedMutableReader;
import org.apache.flink.streaming.runtime.io.IndexedReaderIterator;
import org.apache.flink.streaming.runtime.io.InputGateFactory;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class InputHandler<IN> {
private StreamRecordSerializer<IN> inputSerializer = null;
private IndexedReaderIterator<StreamRecord<IN>> inputIter;
private IndexedMutableReader<DeserializationDelegate<StreamRecord<IN>>> inputs;
import java.io.IOException;
private StreamTask<IN, ?> streamVertex;
private StreamConfig configuration;
public class OneInputStreamTask<IN, OUT> extends StreamTask<OUT, OneInputStreamOperator<IN, OUT>> {
public InputHandler(StreamTask<IN, ?> streamComponent) {
this.streamVertex = streamComponent;
this.configuration = new StreamConfig(streamComponent.getTaskConfiguration());
try {
setConfigInputs();
} catch (Exception e) {
throw new StreamTaskException("Cannot register inputs for "
+ getClass().getSimpleName(), e);
}
private static final Logger LOG = LoggerFactory.getLogger(OneInputStreamTask.class);
protected StreamRecordSerializer<IN> inSerializer;
private IndexedMutableReader<DeserializationDelegate<StreamRecord<IN>>> inputs;
protected IndexedReaderIterator<StreamRecord<IN>> recordIterator;
}
protected void setConfigInputs() throws StreamTaskException {
inputSerializer = configuration.getTypeSerializerIn1(streamVertex.userClassLoader);
@Override
public void registerInputOutput() {
super.registerInputOutput();
inSerializer = configuration.getTypeSerializerIn1(getUserCodeClassLoader());
int numberOfInputs = configuration.getNumberOfInputs();
if (numberOfInputs > 0) {
InputGate inputGate = InputGateFactory.createInputGate(streamVertex.getEnvironment().getAllInputGates());
InputGate inputGate = InputGateFactory.createInputGate(getEnvironment().getAllInputGates());
inputs = new IndexedMutableReader<DeserializationDelegate<StreamRecord<IN>>>(inputGate);
inputs.registerTaskEventListener(streamVertex.getSuperstepListener(),
StreamingSuperstep.class);
inputs.registerTaskEventListener(getSuperstepListener(), StreamingSuperstep.class);
inputIter = new IndexedReaderIterator<StreamRecord<IN>>(inputs, inputSerializer);
recordIterator = new IndexedReaderIterator<StreamRecord<IN>>(inputs, inSerializer);
}
}
protected static <T> IndexedReaderIterator<StreamRecord<T>> staticCreateInputIterator(
MutableReader<?> inputReader, TypeSerializer<StreamRecord<T>> serializer) {
// generic data type serialization
@SuppressWarnings("unchecked")
IndexedMutableReader<DeserializationDelegate<StreamRecord<T>>> reader = (IndexedMutableReader<DeserializationDelegate<StreamRecord<T>>>) inputReader;
final IndexedReaderIterator<StreamRecord<T>> iter = new IndexedReaderIterator<StreamRecord<T>>(
reader, serializer);
return iter;
/*
* Reads the next record from the reader iterator and stores it in the
* nextRecord variable
*/
protected StreamRecord<IN> readNext() throws IOException {
StreamRecord<IN> nextRecord = inSerializer.createInstance();
try {
return recordIterator.next(nextRecord);
} catch (IOException e) {
if (isRunning) {
throw new RuntimeException("Could not read next record.", e);
} else {
// Task already cancelled do nothing
return null;
}
} catch (IllegalStateException e) {
if (isRunning) {
throw new RuntimeException("Could not read next record.", e);
} else {
// Task already cancelled do nothing
return null;
}
}
}
public StreamRecordSerializer<IN> getInputSerializer() {
return inputSerializer;
}
@Override
public void invoke() throws Exception {
this.isRunning = true;
public IndexedReaderIterator<StreamRecord<IN>> getInputIter() {
return inputIter;
}
boolean operatorOpen = false;
if (LOG.isDebugEnabled()) {
LOG.debug("Task {} invoked", getName());
}
try {
openOperator();
operatorOpen = true;
public void clearReaders() throws IOException {
if (inputs != null) {
StreamRecord<IN> nextRecord;
while (isRunning && (nextRecord = readNext()) != null) {
streamOperator.processElement(nextRecord.getObject());
}
closeOperator();
operatorOpen = false;
if (LOG.isDebugEnabled()) {
LOG.debug("Task {} invocation finished", getName());
}
} catch (Exception e) {
if (operatorOpen) {
try {
closeOperator();
} catch (Throwable t) {
LOG.info("Caught exception while closing operator.", e);
}
}
if (LOG.isErrorEnabled()) {
LOG.error("StreamOperator failed.", e);
}
throw e;
} finally {
this.isRunning = false;
// Cleanup
inputs.clearBuffers();
inputs.cleanup();
outputHandler.flushOutputs();
clearBuffers();
}
}
}
......@@ -32,7 +32,8 @@ import org.apache.flink.streaming.api.collector.StreamOutput;
import org.apache.flink.streaming.api.collector.selector.OutputSelectorWrapper;
import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.api.graph.StreamEdge;
import org.apache.flink.streaming.api.operators.ChainableStreamOperator;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.Output;
import org.apache.flink.streaming.runtime.io.RecordWriterFactory;
import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
......@@ -44,24 +45,24 @@ import org.slf4j.LoggerFactory;
public class OutputHandler<OUT> {
private static final Logger LOG = LoggerFactory.getLogger(OutputHandler.class);
private StreamTask<?, OUT> vertex;
private StreamTask<OUT, ?> vertex;
private StreamConfig configuration;
private ClassLoader cl;
private Collector<OUT> outerCollector;
private Output<OUT> outerOutput;
private List<ChainableStreamOperator<?, ?>> chainedOperators;
public List<OneInputStreamOperator<?, ?>> chainedOperators;
private Map<StreamEdge, StreamOutput<?>> outputMap;
private Map<Integer, StreamConfig> chainedConfigs;
private List<StreamEdge> outEdgesInOrder;
public OutputHandler(StreamTask<?, OUT> vertex) {
public OutputHandler(StreamTask<OUT, ?> vertex) {
// Initialize some fields
this.vertex = vertex;
this.configuration = new StreamConfig(vertex.getTaskConfiguration());
this.chainedOperators = new ArrayList<ChainableStreamOperator<?, ?>>();
this.chainedOperators = new ArrayList<OneInputStreamOperator<?, ?>>();
this.outputMap = new HashMap<StreamEdge, StreamOutput<?>>();
this.cl = vertex.getUserCodeClassLoader();
......@@ -83,9 +84,9 @@ public class OutputHandler<OUT> {
outputMap.put(outEdge, streamOutput);
}
// We create the outer collector that will be passed to the first task
// We create the outer output that will be passed to the first task
// in the chain
this.outerCollector = createChainedCollector(configuration);
this.outerOutput = createChainedCollector(configuration);
}
public void broadcastBarrier(long id, long timestamp) throws IOException, InterruptedException {
......@@ -99,30 +100,30 @@ public class OutputHandler<OUT> {
return outputMap.values();
}
public List<ChainableStreamOperator<?, ?>> getChainedOperators(){
public List<OneInputStreamOperator<?, ?>> getChainedOperators(){
return chainedOperators;
}
/**
* This method builds up a nested collector which encapsulates all the
* This method builds up a nested output which encapsulates all the
* chained operators and their network output. The result of this recursive
* call will be passed as collector to the first operator in the chain.
* call will be passed as output to the first operator in the chain.
*
* @param chainedTaskConfig
* The configuration of the starting operator of the chain, we
* use this paramater to recursively build the whole chain
* @return Returns the collector for the chain starting from the given
* @return Returns the output for the chain starting from the given
* config
*/
@SuppressWarnings({"unchecked", "rawtypes"})
private Collector<OUT> createChainedCollector(StreamConfig chainedTaskConfig) {
private <X> Output<X> createChainedCollector(StreamConfig chainedTaskConfig) {
// We create a wrapper that will encapsulate the chained operators and
// network outputs
OutputSelectorWrapper<OUT> outputSelectorWrapper = chainedTaskConfig.getOutputSelectorWrapper(cl);
CollectorWrapper<OUT> wrapper = new CollectorWrapper<OUT>(outputSelectorWrapper);
OutputSelectorWrapper<?> outputSelectorWrapper = chainedTaskConfig.getOutputSelectorWrapper(cl);
CollectorWrapper wrapper = new CollectorWrapper(outputSelectorWrapper);
// Create collectors for the network outputs
for (StreamEdge outputEdge : chainedTaskConfig.getNonChainedOutputs(cl)) {
......@@ -143,23 +144,22 @@ public class OutputHandler<OUT> {
if (chainedTaskConfig.isChainStart()) {
// The current task is the first chained task at this vertex so we
// return the wrapper
return wrapper;
return (Output<X>) wrapper;
} else {
// The current task is a part of the chain so we get the chainable
// operator which will be returned and set it up using the wrapper
ChainableStreamOperator chainableOperator = chainedTaskConfig.getStreamOperator(vertex
.getUserCodeClassLoader());
chainableOperator.setup(wrapper,
chainedTaskConfig.getTypeSerializerIn1(vertex.getUserCodeClassLoader()));
OneInputStreamOperator chainableOperator =
chainedTaskConfig.getStreamOperator(vertex.getUserCodeClassLoader());
chainableOperator.setup(wrapper, vertex.context);
chainedOperators.add(chainableOperator);
return chainableOperator;
return new OperatorCollector<X>(chainableOperator);
}
}
public Collector<OUT> getCollector() {
return outerCollector;
public Output<OUT> getOutput() {
return outerOutput;
}
/**
......@@ -192,8 +192,7 @@ public class OutputHandler<OUT> {
RecordWriter<SerializationDelegate<StreamRecord<T>>> output =
RecordWriterFactory.createRecordWriter(bufferWriter, outputPartitioner, upStreamConfig.getBufferTimeout());
StreamOutput<T> streamOutput = new StreamOutput<T>(output, vertex.instanceID,
outSerializationDelegate);
StreamOutput<T> streamOutput = new StreamOutput<T>(output, outSerializationDelegate);
if (LOG.isTraceEnabled()) {
LOG.trace("Partitioner set: {} with {} outputs for {}", outputPartitioner.getClass()
......@@ -214,4 +213,36 @@ public class OutputHandler<OUT> {
output.clearBuffers();
}
}
private static class OperatorCollector<T> implements Output<T> {
private OneInputStreamOperator operator;
public OperatorCollector(OneInputStreamOperator<?, T> operator) {
this.operator = operator;
}
@Override
public void collect(T record) {
try {
operator.processElement(record);
} catch (Exception e) {
if (LOG.isErrorEnabled()) {
LOG.error("Could not forward element to operator.", e);
}
throw new RuntimeException(e);
}
}
@Override
public void close() {
try {
operator.close();
} catch (Exception e) {
if (LOG.isErrorEnabled()) {
LOG.error("Could not forward close call to operator.", e);
}
}
}
}
}
/*
* 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.streaming.runtime.tasks;
import org.apache.flink.streaming.api.operators.StreamSource;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class SourceStreamTask<OUT> extends StreamTask<OUT, StreamSource<OUT>> {
private static final Logger LOG = LoggerFactory.getLogger(SourceStreamTask.class);
@Override
public void invoke() throws Exception {
this.isRunning = true;
boolean operatorOpen = false;
if (LOG.isDebugEnabled()) {
LOG.debug("Task {} invoked", getName());
}
try {
openOperator();
operatorOpen = true;
streamOperator.run();
closeOperator();
operatorOpen = false;
if (LOG.isDebugEnabled()) {
LOG.debug("Task {} invocation finished", getName());
}
} catch (Exception e) {
if (operatorOpen) {
try {
closeOperator();
} catch (Throwable t) {
LOG.info("Caught exception while closing operator.", e);
}
}
if (LOG.isErrorEnabled()) {
LOG.error("StreamOperator failed.", e);
}
throw e;
} finally {
this.isRunning = false;
// Cleanup
outputHandler.flushOutputs();
clearBuffers();
}
}
}
......@@ -29,14 +29,11 @@ import org.apache.flink.util.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class StreamIterationHead<OUT> extends StreamTask<OUT, OUT> {
public class StreamIterationHead<OUT> extends OneInputStreamTask<OUT, OUT> {
private static final Logger LOG = LoggerFactory.getLogger(StreamIterationHead.class);
private Collection<StreamOutput<?>> outputs;
private static int numSources;
private Integer iterationId;
@SuppressWarnings("rawtypes")
private BlockingQueue<StreamRecord> dataChannel;
private long iterationWaitTime;
......@@ -44,17 +41,15 @@ public class StreamIterationHead<OUT> extends StreamTask<OUT, OUT> {
@SuppressWarnings("rawtypes")
public StreamIterationHead() {
numSources = newTask();
instanceID = numSources;
dataChannel = new ArrayBlockingQueue<StreamRecord>(1);
}
@Override
public void setInputsOutputs() {
public void registerInputOutput() {
super.registerInputOutput();
outputHandler = new OutputHandler<OUT>(this);
outputs = outputHandler.getOutputs();
iterationId = configuration.getIterationId();
Integer iterationId = configuration.getIterationId();
iterationWaitTime = configuration.getIterationWaitTime();
shouldWait = iterationWaitTime > 0;
......@@ -71,9 +66,11 @@ public class StreamIterationHead<OUT> extends StreamTask<OUT, OUT> {
@Override
public void invoke() throws Exception {
if (LOG.isDebugEnabled()) {
LOG.debug("Iteration source {} invoked with instance id {}", getName(), getInstanceID());
LOG.debug("Iteration source {} invoked", getName());
}
Collection<StreamOutput<?>> outputs = outputHandler.getOutputs();
try {
StreamRecord<OUT> nextRecord;
......@@ -103,8 +100,4 @@ public class StreamIterationHead<OUT> extends StreamTask<OUT, OUT> {
}
}
@Override
protected void setOperator() {
}
}
......@@ -26,13 +26,12 @@ import org.apache.flink.util.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class StreamIterationTail<IN> extends StreamTask<IN, IN> {
public class StreamIterationTail<IN> extends OneInputStreamTask<IN, IN> {
private static final Logger LOG = LoggerFactory.getLogger(StreamIterationTail.class);
private InputHandler<IN> inputHandler;
private Integer iterationId;
@SuppressWarnings("rawtypes")
private BlockingQueue<StreamRecord> dataChannel;
private long iterationWaitTime;
......@@ -42,10 +41,9 @@ public class StreamIterationTail<IN> extends StreamTask<IN, IN> {
}
@Override
public void setInputsOutputs() {
public void registerInputOutput() {
super.registerInputOutput();
try {
inputHandler = new InputHandler<IN>(this);
iterationId = configuration.getIterationId();
iterationWaitTime = configuration.getIterationWaitTime();
shouldWait = iterationWaitTime > 0;
......@@ -81,12 +79,12 @@ public class StreamIterationTail<IN> extends StreamTask<IN, IN> {
}
protected void forwardRecords() throws Exception {
StreamRecord<IN> reuse = inputHandler.getInputSerializer().createInstance();
while ((reuse = inputHandler.getInputIter().next(reuse)) != null) {
StreamRecord<IN> reuse = inSerializer.createInstance();
while ((reuse = recordIterator.next(reuse)) != null) {
if (!pushToQueue(reuse)) {
break;
}
reuse = inputHandler.getInputSerializer().createInstance();
reuse = inSerializer.createInstance();
}
}
......@@ -107,8 +105,4 @@ public class StreamIterationTail<IN> extends StreamTask<IN, IN> {
return false;
}
}
@Override
protected void setOperator() {
}
}
......@@ -19,24 +19,18 @@ package org.apache.flink.streaming.api;
import static org.junit.Assert.assertEquals;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.ByteArrayInputStream;
import java.net.Socket;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import org.apache.flink.streaming.api.functions.source.FromElementsFunction;
import org.apache.flink.streaming.api.functions.source.SocketTextStreamFunction;
import org.apache.flink.streaming.util.MockCollector;
import org.apache.flink.streaming.util.MockSource;
import org.junit.Test;
public class SourceTest {
public class SourceFunctionTest {
@Test
public void fromElementsTest() {
public void fromElementsTest() throws Exception {
List<Integer> expectedList = Arrays.asList(1, 2, 3);
List<Integer> actualList = MockSource.createAndExecute(new FromElementsFunction<Integer>(1,
2, 3));
......@@ -44,7 +38,7 @@ public class SourceTest {
}
@Test
public void fromCollectionTest() {
public void fromCollectionTest() throws Exception {
List<Integer> expectedList = Arrays.asList(1, 2, 3);
List<Integer> actualList = MockSource.createAndExecute(new FromElementsFunction<Integer>(
Arrays.asList(1, 2, 3)));
......@@ -53,18 +47,22 @@ public class SourceTest {
@Test
public void socketTextStreamTest() throws Exception {
List<String> expectedList = Arrays.asList("a", "b", "c");
List<String> actualList = new ArrayList<String>();
byte[] data = { 'a', '\n', 'b', '\n', 'c', '\n' };
Socket socket = mock(Socket.class);
when(socket.getInputStream()).thenReturn(new ByteArrayInputStream(data));
when(socket.isClosed()).thenReturn(false);
when(socket.isConnected()).thenReturn(true);
new SocketTextStreamFunction("", 0, '\n', 0).streamFromSocket(new MockCollector<String>(
actualList), socket);
assertEquals(expectedList, actualList);
// TODO: does not work because we cannot set the internal socket anymore
// List<String> expectedList = Arrays.asList("a", "b", "c");
// List<String> actualList = new ArrayList<String>();
//
// byte[] data = { 'a', '\n', 'b', '\n', 'c', '\n' };
//
// Socket socket = mock(Socket.class);
// when(socket.getInputStream()).thenReturn(new ByteArrayInputStream(data));
// when(socket.isClosed()).thenReturn(false);
// when(socket.isConnected()).thenReturn(true);
//
// SocketTextStreamFunction source = new SocketTextStreamFunction("", 0, '\n', 0);
// source.open(new Configuration());
// while (!source.reachedEnd()) {
// actualList.add(source.next());
// }
// assertEquals(expectedList, actualList);
}
}
......@@ -116,13 +116,15 @@ public class TypeFillTest {
private class TestSource<T> implements SourceFunction<T> {
@Override
public void run(Collector<T> collector) throws Exception {
@Override
public boolean reachedEnd() throws Exception {
return false;
}
@Override
public void cancel() {
public T next() throws Exception {
return null;
}
}
......
......@@ -36,7 +36,7 @@ public class StreamCollectorTest {
null);
sd.setInstance(new StreamRecord<Tuple1<Integer>>().setObject(new Tuple1<Integer>()));
Collector<Tuple1<Integer>> collector = new StreamOutput<Tuple1<Integer>>(recWriter, 2, sd);
Collector<Tuple1<Integer>> collector = new StreamOutput<Tuple1<Integer>>(recWriter, sd);
collector.collect(new Tuple1<Integer>(3));
collector.collect(new Tuple1<Integer>(4));
collector.collect(new Tuple1<Integer>(5));
......
......@@ -24,7 +24,6 @@ import java.util.List;
import org.apache.flink.api.common.functions.ReduceFunction;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.streaming.api.operators.StreamGroupedReduce;
import org.apache.flink.streaming.util.MockContext;
import org.junit.Test;
......
......@@ -22,8 +22,7 @@ import static org.junit.Assert.assertEquals;
import java.util.ArrayList;
import java.util.List;
import org.apache.flink.streaming.api.operators.StreamOperator;
import org.apache.flink.streaming.api.operators.windowing.WindowFlattener;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.windowing.StreamWindow;
import org.apache.flink.streaming.util.MockContext;
import org.junit.Test;
......@@ -32,7 +31,7 @@ public class WindowFlattenerTest {
@Test
public void test() {
StreamOperator<StreamWindow<Integer>, Integer> flattener = new WindowFlattener<Integer>();
OneInputStreamOperator<StreamWindow<Integer>, Integer> flattener = new WindowFlattener<Integer>();
StreamWindow<Integer> w1 = StreamWindow.fromElements(1, 2, 3);
StreamWindow<Integer> w2 = new StreamWindow<Integer>();
......
......@@ -23,8 +23,7 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.flink.api.common.functions.FoldFunction;
import org.apache.flink.streaming.api.operators.StreamOperator;
import org.apache.flink.streaming.api.operators.windowing.WindowFolder;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.windowing.StreamWindow;
import org.apache.flink.streaming.util.MockContext;
import org.junit.Test;
......@@ -33,7 +32,7 @@ public class WindowFolderTest {
@Test
public void test() {
StreamOperator<StreamWindow<Integer>, StreamWindow<String>> windowReducer = new WindowFolder<Integer,String>(
OneInputStreamOperator<StreamWindow<Integer>, StreamWindow<String>> windowReducer = new WindowFolder<Integer,String>(
new FoldFunction<Integer, String>() {
private static final long serialVersionUID = 1L;
......
......@@ -23,8 +23,7 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.flink.streaming.api.functions.WindowMapFunction;
import org.apache.flink.streaming.api.operators.StreamOperator;
import org.apache.flink.streaming.api.operators.windowing.WindowMapper;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.windowing.StreamWindow;
import org.apache.flink.streaming.util.MockContext;
import org.apache.flink.util.Collector;
......@@ -34,7 +33,7 @@ public class WindowMapperTest {
@Test
public void test() {
StreamOperator<StreamWindow<Integer>, StreamWindow<Integer>> windowMapper = new WindowMapper<Integer, Integer>(
OneInputStreamOperator<StreamWindow<Integer>, StreamWindow<Integer>> windowMapper = new WindowMapper<Integer, Integer>(
new WindowMapFunction<Integer, Integer>() {
private static final long serialVersionUID = 1L;
......
......@@ -21,12 +21,13 @@ import java.io.Serializable;
import java.util.Collection;
import org.apache.commons.lang3.SerializationUtils;
import org.apache.flink.streaming.api.operators.Output;
import org.apache.flink.util.Collector;
public class MockCollector<T> implements Collector<T> {
public class MockOutput<T> implements Output<T> {
private Collection<T> outputs;
public MockCollector(Collection<T> outputs) {
public MockOutput(Collection<T> outputs) {
this.outputs = outputs;
}
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册