godfreyhe commented on a change in pull request #12202: URL: https://github.com/apache/flink/pull/12202#discussion_r426364305
########## File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java ########## @@ -219,6 +219,22 @@ "NOTE: MiniBatch only works for non-windowed aggregations currently. If " + TABLE_EXEC_MINIBATCH_ENABLED.key() + " is set true, its value must be positive."); + // ------------------------------------------------------------------------ + // Result Collect Options + // ------------------------------------------------------------------------ + + public static final ConfigOption<Integer> TABLE_EXEC_COLLECT_BATCH_SIZE = + key("table.exec.collect.batch.size") + .defaultValue(10000) + .withDescription("The maximum number of results transmitted from the sink function to the client each time. " + + "This option can be set to a larger value if both network bandwidth and task manager's memory are enough."); + + public static final ConfigOption<Integer> TABLE_EXEC_COLLECT_SOCKET_TIMEOUT = + key("table.exec.collect.socket.timeout") + .defaultValue(10000) Review comment: use `Duration` ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamUtils.java ########## @@ -47,44 +44,30 @@ * Returns an iterator to iterate over the elements of the DataStream. * @return The iterator */ - public static <OUT> Iterator<OUT> collect(DataStream<OUT> stream) throws IOException { - + public static <OUT> Iterator<OUT> collect(DataStream<OUT> stream) { TypeSerializer<OUT> serializer = stream.getType().createSerializer( - stream.getExecutionEnvironment().getConfig()); + stream.getExecutionEnvironment().getConfig()); + String id = UUID.randomUUID().toString(); + String accumulatorName = "dataStreamCollect_" + id; - SocketStreamIterator<OUT> iter = new SocketStreamIterator<OUT>(serializer); + CollectSinkOperatorFactory<OUT> factory = new CollectSinkOperatorFactory<>(serializer, accumulatorName); + CollectSinkOperator<OUT> operator = (CollectSinkOperator<OUT>) factory.getOperator(); + CollectResultIterator<OUT> iterator = new CollectResultIterator<>( + operator.getOperatorIdFuture(), serializer, accumulatorName); + CollectStreamSink<OUT> sink = new CollectStreamSink<>(stream, factory); + sink.name("Data stream collect sink"); - //Find out what IP of us should be given to CollectSink, that it will be able to connect to StreamExecutionEnvironment env = stream.getExecutionEnvironment(); - InetAddress clientAddress; - - if (env instanceof RemoteStreamEnvironment) { - String host = ((RemoteStreamEnvironment) env).getHost(); - int port = ((RemoteStreamEnvironment) env).getPort(); - try { - clientAddress = ConnectionUtils.findConnectingAddress(new InetSocketAddress(host, port), 2000, 400); - } - catch (Exception e) { - throw new IOException("Could not determine an suitable network address to " + - "receive back data from the streaming program.", e); - } - } else if (env instanceof LocalStreamEnvironment) { - clientAddress = InetAddress.getLoopbackAddress(); - } else { - try { - clientAddress = InetAddress.getLocalHost(); - } catch (UnknownHostException e) { - throw new IOException("Could not determine this machines own local address to " + - "receive back data from the streaming program.", e); - } - } - - DataStreamSink<OUT> sink = stream.addSink(new CollectSink<OUT>(clientAddress, iter.getPort(), serializer)); - sink.setParallelism(1); // It would not work if multiple instances would connect to the same port + env.addOperator(sink.getTransformation()); - (new CallExecute(env, iter)).start(); + try { + JobClient jobClient = env.executeAsync("DataStreamCollect_" + id); Review comment: id is `meaningless ` here, use "data stream collect" ? ########## File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/sinks/BatchSelectTableSink.java ########## @@ -49,19 +44,24 @@ * once FLINK-14807 is finished, the implementation should be changed. */ public class BatchSelectTableSink implements StreamTableSink<Row>, SelectTableSink { + private final TableSchema tableSchema; - private final String accumulatorName; - private final TypeSerializer<Row> typeSerializer; - private JobClient jobClient; + private final CollectSinkOperatorFactory<Row> factory; + private final CollectResultIterator<Row> iterator; @SuppressWarnings("unchecked") public BatchSelectTableSink(TableSchema tableSchema) { Review comment: we can extract the most code of `BatchSelectTableSink` and `StreamSelectTableSink ` into a base class (SelectTableSinkBase) ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectStreamSink.java ########## @@ -0,0 +1,84 @@ +/* + * 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.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.operators.ChainingStrategy; +import org.apache.flink.streaming.api.transformations.SinkTransformation; + +/** + * A {@link DataStreamSink} which is used to collect query results. Review comment: `CollectStreamSink ` is not only used for select query but also for DataStream collect, so make this comment more generic. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org