Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-19508][DataStream] Add collect() operation on DataStream #13752

Closed
wants to merge 2 commits into from
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.DataStreamUtils;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.operators.collect.ClientAndIterator;
import org.apache.flink.test.util.MiniClusterWithClientResource;
import org.apache.flink.util.TestLogger;

Expand Down Expand Up @@ -114,7 +115,7 @@ public void testContinuousTextFileSource() throws Exception {
WatermarkStrategy.noWatermarks(),
"file-source");

final DataStreamUtils.ClientAndIterator<String> client =
final ClientAndIterator<String> client =
DataStreamUtils.collectWithClient(stream, "Continuous TextFiles Monitoring Test");

// write one file, execute, and wait for its result
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@
import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.api.java.Utils;
import org.apache.flink.api.java.functions.KeySelector;
Expand All @@ -48,6 +49,7 @@
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.core.execution.JobClient;
import org.apache.flink.core.fs.FileSystem.WriteMode;
import org.apache.flink.core.fs.Path;
import org.apache.flink.streaming.api.TimeCharacteristic;
Expand All @@ -68,6 +70,11 @@
import org.apache.flink.streaming.api.operators.StreamMap;
import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
import org.apache.flink.streaming.api.operators.StreamSink;
import org.apache.flink.streaming.api.operators.collect.ClientAndIterator;
import org.apache.flink.streaming.api.operators.collect.CollectResultIterator;
import org.apache.flink.streaming.api.operators.collect.CollectSinkOperator;
import org.apache.flink.streaming.api.operators.collect.CollectSinkOperatorFactory;
import org.apache.flink.streaming.api.operators.collect.CollectStreamSink;
import org.apache.flink.streaming.api.transformations.OneInputTransformation;
import org.apache.flink.streaming.api.transformations.PartitionTransformation;
import org.apache.flink.streaming.api.transformations.UnionTransformation;
Expand Down Expand Up @@ -96,11 +103,13 @@
import org.apache.flink.streaming.runtime.partitioner.ShufflePartitioner;
import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
import org.apache.flink.streaming.util.keys.KeySelectorUtil;
import org.apache.flink.util.CloseableIterator;
import org.apache.flink.util.OutputTag;
import org.apache.flink.util.Preconditions;

import java.util.ArrayList;
import java.util.List;
import java.util.UUID;

/**
* A DataStream represents a stream of elements of the same type. A DataStream
Expand Down Expand Up @@ -1294,6 +1303,92 @@ public DataStreamSink<T> addSink(SinkFunction<T> sinkFunction) {
return sink;
}

/**
* Triggers the distributed execution of the streaming dataflow and returns an iterator over the elements
* of the given DataStream.
*
* <p>The DataStream application is executed in the regular distributed manner on the target environment,
* and the events from the stream are polled back to this application process and thread through
* Flink's REST API.
*
*<p><b>IMPORTANT</b> The returned iterator must be closed to free all cluster resources.
*/
public CloseableIterator<T> executeAndCollect() throws Exception {
return executeAndCollect("DataStream Collect");
}

/**
* Triggers the distributed execution of the streaming dataflow and returns an iterator over the elements
* of the given DataStream.
*
* <p>The DataStream application is executed in the regular distributed manner on the target environment,
* and the events from the stream are polled back to this application process and thread through
* Flink's REST API.
*
*<p><b>IMPORTANT</b> The returned iterator must be closed to free all cluster resources.
*/
public CloseableIterator<T> executeAndCollect(String jobExecutionName) throws Exception {
return executeAndCollectWithClient(jobExecutionName).iterator;
}

/**
* Triggers the distributed execution of the streaming dataflow and returns an iterator over the elements
* of the given DataStream.
*
* <p>The DataStream application is executed in the regular distributed manner on the target environment,
* and the events from the stream are polled back to this application process and thread through
* Flink's REST API.
*/
public List<T> executeAndCollect(int limit) throws Exception {
return executeAndCollect("DataStream Collect", limit);
}

/**
* Triggers the distributed execution of the streaming dataflow and returns an iterator over the elements
* of the given DataStream.
*
* <p>The DataStream application is executed in the regular distributed manner on the target environment,
* and the events from the stream are polled back to this application process and thread through
* Flink's REST API.
*/
public List<T> executeAndCollect(String jobExecutionName, int limit) throws Exception {
Preconditions.checkState(limit > 0, "Limit must be greater than 0");

ClientAndIterator<T> clientAndIterator = executeAndCollectWithClient(jobExecutionName);

try {
List<T> results = new ArrayList<>(limit);
while (clientAndIterator.iterator.hasNext() && limit > 0) {
results.add(clientAndIterator.iterator.next());
limit--;
}

return results;
} finally {
clientAndIterator.iterator.close();
aljoscha marked this conversation as resolved.
Show resolved Hide resolved
clientAndIterator.client.cancel();
}
}

private ClientAndIterator<T> executeAndCollectWithClient(String jobExecutionName) throws Exception {
aljoscha marked this conversation as resolved.
Show resolved Hide resolved
TypeSerializer<T> serializer = getType().createSerializer(getExecutionEnvironment().getConfig());
String accumulatorName = "dataStreamCollect_" + UUID.randomUUID().toString();

StreamExecutionEnvironment env = getExecutionEnvironment();
CollectSinkOperatorFactory<T> factory = new CollectSinkOperatorFactory<>(serializer, accumulatorName);
CollectSinkOperator<T> operator = (CollectSinkOperator<T>) factory.getOperator();
CollectResultIterator<T> iterator = new CollectResultIterator<>(
operator.getOperatorIdFuture(), serializer, accumulatorName, env.getCheckpointConfig());
CollectStreamSink<T> sink = new CollectStreamSink<>(this, factory);
sink.name("Data stream collect sink");
env.addOperator(sink.getTransformation());

final JobClient jobClient = env.executeAsync(jobExecutionName);
iterator.setJobClient(jobClient);

return new ClientAndIterator<>(jobClient, iterator);
}

/**
* Returns the {@link Transformation} that represents the operation that logically creates
* this {@link DataStream}.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.core.execution.JobClient;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.operators.collect.ClientAndIterator;
import org.apache.flink.streaming.api.operators.collect.CollectResultIterator;
import org.apache.flink.streaming.api.operators.collect.CollectSinkOperator;
import org.apache.flink.streaming.api.operators.collect.CollectSinkOperatorFactory;
Expand Down Expand Up @@ -52,7 +53,10 @@ public final class DataStreamUtils {
* <p>The DataStream application is executed in the regular distributed manner on the target environment,
* and the events from the stream are polled back to this application process and thread through
* Flink's REST API.
*
* @deprecated Please use {@link DataStream#executeAndCollect()}.
*/
@Deprecated
public static <OUT> Iterator<OUT> collect(DataStream<OUT> stream) {
return collect(stream, "Data Stream Collect");
}
Expand All @@ -64,7 +68,9 @@ public static <OUT> Iterator<OUT> collect(DataStream<OUT> stream) {
* <p>The DataStream application is executed in the regular distributed manner on the target environment,
* and the events from the stream are polled back to this application process and thread through
* Flink's REST API.
* @deprecated Please use {@link DataStream#executeAndCollect()}.
*/
@Deprecated
public static <OUT> Iterator<OUT> collect(DataStream<OUT> stream, String executionJobName) {
try {
return collectWithClient(stream, executionJobName).iterator;
Expand All @@ -78,7 +84,9 @@ public static <OUT> Iterator<OUT> collect(DataStream<OUT> stream, String executi
/**
* Starts the execution of the program and returns an iterator to read the result of the
* given data stream, plus a {@link JobClient} to interact with the application execution.
* @deprecated Please use {@link DataStream#executeAndCollect()}.
*/
@Deprecated
public static <OUT> ClientAndIterator<OUT> collectWithClient(
DataStream<OUT> stream,
String jobExecutionName) throws Exception {
Expand Down Expand Up @@ -112,7 +120,9 @@ public static <OUT> ClientAndIterator<OUT> collectWithClient(
* Out-of-Memory Error because it attempts to collect an infinite stream into a list.
*
* @throws Exception Exceptions that occur during the execution are forwarded.
* @deprecated Please use {@link DataStream#executeAndCollect()}.
*/
@Deprecated
public static <E> List<E> collectBoundedStream(DataStream<E> stream, String jobName) throws Exception {
final ArrayList<E> list = new ArrayList<>();
final Iterator<E> iter = collectWithClient(stream, jobName).iterator;
Expand All @@ -126,7 +136,9 @@ public static <E> List<E> collectBoundedStream(DataStream<E> stream, String jobN
/**
* Triggers execution of the DataStream application and collects the given number of records from the stream.
* After the records are received, the execution is canceled.
* @deprecated Please use {@link DataStream#executeAndCollect()}.
*/
@Deprecated
public static <E> List<E> collectUnboundedStream(DataStream<E> stream, int numElements, String jobName) throws Exception {
final ClientAndIterator<E> clientAndIterator = collectWithClient(stream, jobName);
final List<E> result = collectRecordsFromUnboundedStream(clientAndIterator, numElements);
Expand All @@ -137,6 +149,10 @@ public static <E> List<E> collectUnboundedStream(DataStream<E> stream, int numEl
return result;
}

/**
* @deprecated Please use {@link DataStream#executeAndCollect()}.
*/
@Deprecated
public static <E> List<E> collectRecordsFromUnboundedStream(
final ClientAndIterator<E> client,
final int numElements) {
Expand Down Expand Up @@ -228,18 +244,4 @@ private DataStreamUtils() {}

// ------------------------------------------------------------------------

/**
* A pair of an {@link Iterator} to receive results from a streaming application and a
* {@link JobClient} to interact with the program.
*/
public static final class ClientAndIterator<E> {

public final JobClient client;
public final Iterator<E> iterator;

ClientAndIterator(JobClient client, Iterator<E> iterator) {
this.client = checkNotNull(client);
this.iterator = checkNotNull(iterator);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
/*
* 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.collect;

import org.apache.flink.annotation.Internal;
import org.apache.flink.core.execution.JobClient;
import org.apache.flink.util.CloseableIterator;

import java.util.Iterator;

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

/**
* A pair of an {@link Iterator} to receive results from a streaming application and a
* {@link JobClient} to interact with the program.
*/
@Internal
public final class ClientAndIterator<E> {

public final JobClient client;
public final CloseableIterator<E> iterator;

public ClientAndIterator(JobClient client, CloseableIterator<E> iterator) {
this.client = checkNotNull(client);
this.iterator = checkNotNull(iterator);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
/*
* 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.scala

import org.apache.flink.util.{CloseableIterator => JCloseableIterator}

/**
* This interface represents an [[Iterator]] that is also [[AutoCloseable]]. A typical use-case
* for this interface are iterators that are based on native-resources such as files, network, or
* database connections. Clients must call close after using the iterator.
*/
trait CloseableIterator[T] extends Iterator[T] with AutoCloseable {

}

object CloseableIterator {

def fromJava[T](iterator: JCloseableIterator[T]): CloseableIterator[T] =
new CloseableIterator[T] {
override def hasNext: Boolean = iterator.hasNext

override def next(): T = iterator.next

override def close(): Unit = iterator.close()
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ import org.apache.flink.streaming.api.operators.OneInputStreamOperator
import org.apache.flink.streaming.api.windowing.assigners._
import org.apache.flink.streaming.api.windowing.time.Time
import org.apache.flink.streaming.api.windowing.windows.{GlobalWindow, TimeWindow, Window}
import org.apache.flink.util.Collector
import org.apache.flink.util.{CloseableIterator, Collector}

import scala.collection.JavaConverters._

Expand Down Expand Up @@ -1125,6 +1125,54 @@ class DataStream[T](stream: JavaStream[T]) {
this.addSink(sinkFunction)
}

/**
* Triggers the distributed execution of the streaming dataflow and returns an iterator over the
* elements of the given DataStream.
*
* <p>The DataStream application is executed in the regular distributed manner on the target
* environment, and the events from the stream are polled back to this application process and
* thread through Flink's REST API.
*
* <p><b>IMPORTANT</b> The returned iterator must be closed to free all cluster resources.
*/
def executeAndCollect(): CloseableIterator[T] =
CloseableIterator.fromJava(stream.executeAndCollect())

/**
* Triggers the distributed execution of the streaming dataflow and returns an iterator over the
* elements of the given DataStream.
*
* <p>The DataStream application is executed in the regular distributed manner on the target
* environment, and the events from the stream are polled back to this application process and
* thread through Flink's REST API.
*
* <p><b>IMPORTANT</b> The returned iterator must be closed to free all cluster resources.
*/
def executeAndCollect(jobExecutionName: String): CloseableIterator[T] =
CloseableIterator.fromJava(stream.executeAndCollect(jobExecutionName))

/**
* Triggers the distributed execution of the streaming dataflow and returns an iterator over the
* elements of the given DataStream.
*
* <p>The DataStream application is executed in the regular distributed manner on the target
* environment, and the events from the stream are polled back to this application process and
* thread through Flink's REST API.
*/
def executeAndCollect(limit: Int): List[T] =
stream.executeAndCollect(limit).asScala.toList

/**
* Triggers the distributed execution of the streaming dataflow and returns an iterator over the
* elements of the given DataStream.
*
* <p>The DataStream application is executed in the regular distributed manner on the target
* environment, and the events from the stream are polled back to this application process and
* thread through Flink's REST API.
*/
def executeAndCollect(jobExecutionName: String, limit: Int): List[T] =
stream.executeAndCollect(jobExecutionName, limit).asScala.toList

/**
* Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
* is not disabled in the [[org.apache.flink.api.common.ExecutionConfig]].
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,8 @@ class DataStreamUtils[T: TypeInformation : ClassTag](val self: DataStream[T]) {
/**
* Returns a scala iterator to iterate over the elements of the DataStream.
* @return The iterator
*
* @deprecated Replaced with [[DataStream#executeAndCollect]].
*/
def collect() : Iterator[T] = {
JavaStreamUtils.collect(self.javaStream).asScala
Expand Down