You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/05/14 03:59:33 UTC

[GitHub] [flink] KurtYoung commented on a change in pull request #12069: [FLINK-14807][streaming] Add specialized collecting sink function

KurtYoung commented on a change in pull request #12069:
URL: https://github.com/apache/flink/pull/12069#discussion_r424837349



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java
##########
@@ -162,9 +174,28 @@ public RetryingRegistrationConfiguration getRetryingRegistrationConfiguration()
 	//  Static factory methods
 	// --------------------------------------------------------------------------------------------
 
+	@VisibleForTesting
 	public static TaskManagerConfiguration fromConfiguration(
 			Configuration configuration,
 			TaskExecutorResourceSpec taskExecutorResourceSpec) {
+		try {
+			TaskManagerServicesConfiguration servicesConfiguration =
+				TaskManagerServicesConfiguration.fromConfiguration(
+					configuration,
+					ResourceID.generate(),
+					InetAddress.getLoopbackAddress().getHostAddress(),

Review comment:
       As long as I can see, currently there is no magic happens during obtaining task manager's address. Why not simply using this `InetAddress.getLoopbackAddress().getHostAddress()` to get the address inside the collect sink function? 

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectSinkOperatorCoordinator.java
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequest;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequestHandler;
+import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.Collections;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * {@link OperatorCoordinator} for {@link CollectSinkFunction}.
+ *
+ * <p>This coordinator only forwards requests and responses from clients and sinks,
+ * it does not store any state in itself.
+ */
+public class CollectSinkOperatorCoordinator implements OperatorCoordinator, CoordinationRequestHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(CollectSinkOperatorCoordinator.class);
+
+	private InetSocketAddress address;
+	private Socket socket;
+
+	private ExecutorService executorService;
+
+	@Override
+	public void start() throws Exception {
+		this.executorService = Executors.newFixedThreadPool(1);
+	}
+
+	@Override
+	public void close() throws Exception {
+		closeCurrentConnection();
+		this.executorService.shutdown();

Review comment:
       shutdown service first

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectSinkOperatorCoordinator.java
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequest;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequestHandler;
+import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.Collections;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * {@link OperatorCoordinator} for {@link CollectSinkFunction}.
+ *
+ * <p>This coordinator only forwards requests and responses from clients and sinks,
+ * it does not store any state in itself.

Review comment:
       This comment conflicts with current implementation. 

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectCoordinationRequest.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequest;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+/**
+ * A {@link CoordinationRequest} from the client indicating that it wants a new batch of query results.
+ */
+public class CollectCoordinationRequest implements CoordinationRequest {
+
+	private static final long serialVersionUID = 1L;
+
+	private final byte[] bytes;
+
+	public CollectCoordinationRequest(String version, long token) {
+		ByteArrayOutputStream baos = new ByteArrayOutputStream();
+		DataOutputViewStreamWrapper wrapper = new DataOutputViewStreamWrapper(baos);
+		TypeSerializer<String> versionSerializer = new StringSerializer();
+		TypeSerializer<Long> tokenSerializer = new LongSerializer();
+		try {
+			versionSerializer.serialize(version, wrapper);
+			tokenSerializer.serialize(token, wrapper);
+		} catch (IOException e) {
+			throw new RuntimeException("Failed to serialize collect sink request", e);
+		}
+		this.bytes = baos.toByteArray();
+	}
+
+	public byte[] getBytes() {
+		return bytes;
+	}
+
+	public static DeserializedRequest deserialize(DataInputViewStreamWrapper wrapper) throws IOException {
+		TypeSerializer<String> versionSerializer = new StringSerializer();
+		TypeSerializer<Long> tokenSerializer = new LongSerializer();
+		String version = versionSerializer.deserialize(wrapper);
+		long token = tokenSerializer.deserialize(wrapper);
+		return new DeserializedRequest(version, token);
+	}
+
+	/**
+	 * Deserialized request containing version and token.
+	 */
+	public static class DeserializedRequest {

Review comment:
       This is unnecessary, you can directly put these two fields into CollectCoordinationRequest, and mark the `bytes` as `serializedBytes`.

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectSinkOperatorCoordinator.java
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequest;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequestHandler;
+import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.Collections;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * {@link OperatorCoordinator} for {@link CollectSinkFunction}.
+ *
+ * <p>This coordinator only forwards requests and responses from clients and sinks,
+ * it does not store any state in itself.
+ */
+public class CollectSinkOperatorCoordinator implements OperatorCoordinator, CoordinationRequestHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(CollectSinkOperatorCoordinator.class);
+
+	private InetSocketAddress address;

Review comment:
       This address needs lock protection. Event handling and coordination request handling will be executed by different threads.

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectSinkOperatorCoordinator.java
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequest;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequestHandler;
+import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.Collections;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * {@link OperatorCoordinator} for {@link CollectSinkFunction}.
+ *
+ * <p>This coordinator only forwards requests and responses from clients and sinks,
+ * it does not store any state in itself.
+ */
+public class CollectSinkOperatorCoordinator implements OperatorCoordinator, CoordinationRequestHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(CollectSinkOperatorCoordinator.class);
+
+	private InetSocketAddress address;
+	private Socket socket;
+
+	private ExecutorService executorService;
+
+	@Override
+	public void start() throws Exception {
+		this.executorService = Executors.newFixedThreadPool(1);
+	}
+
+	@Override
+	public void close() throws Exception {
+		closeCurrentConnection();
+		this.executorService.shutdown();
+	}
+
+	@Override
+	public void handleEventFromOperator(int subtask, OperatorEvent event) throws Exception {
+		Preconditions.checkArgument(
+			event instanceof CollectSinkAddressEvent, "Operator event must be a CollectSinkAddressEvent");
+		address = ((CollectSinkAddressEvent) event).getAddress();
+		LOG.debug("Received sink socket server address: " + address);
+
+		// this event is sent when the sink function starts, so we remove the old socket if it is present
+		socket = null;
+	}
+
+	@Override
+	public CompletableFuture<CoordinationResponse> handleCoordinationRequest(CoordinationRequest request) {
+		Preconditions.checkArgument(
+			request instanceof CollectCoordinationRequest,
+			"Coordination request must be a CollectCoordinationRequest");
+
+		CompletableFuture<CoordinationResponse> responseFuture = new CompletableFuture<>();
+		executorService.submit(() -> handleRequestImpl((CollectCoordinationRequest) request, responseFuture));
+		return responseFuture;
+	}
+
+	private void handleRequestImpl(
+			CollectCoordinationRequest request,
+			CompletableFuture<CoordinationResponse> responseFuture) {
+		// there is no thread safety issue in this method
+		// because the thread pool has only 1 thread
+		// and by design there shall be only 1 client reading from this coordinator
+		if (address == null) {
+			responseFuture.complete(new CollectCoordinationResponse(createEmptySerializedResult(request)));
+		}
+
+		try {
+			if (socket == null) {
+				socket = new Socket(address.getAddress(), address.getPort());
+				LOG.debug("Sink connection established");

Review comment:
       use info log

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectCoordinationResponse.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A {@link CoordinationResponse} from the coordinator containing the required batch or new results
+ * and other necessary information in serialized form.
+ */
+public class CollectCoordinationResponse implements CoordinationResponse {
+
+	private static final long serialVersionUID = 1L;
+
+	private final byte[] bytes;
+
+	public CollectCoordinationResponse(byte[] bytes) {
+		this.bytes = bytes;
+	}
+
+	public <T> DeserializedResponse<T> getDeserializedResponse(TypeSerializer<T> serializer) {
+		ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+		DataInputViewStreamWrapper wrapper = new DataInputViewStreamWrapper(bais);
+		TypeSerializer<String> versionSerializer = new StringSerializer();

Review comment:
       use `StringSerializer.INSTANCE` instead

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectSinkOperatorCoordinator.java
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequest;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequestHandler;
+import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.Collections;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * {@link OperatorCoordinator} for {@link CollectSinkFunction}.
+ *
+ * <p>This coordinator only forwards requests and responses from clients and sinks,
+ * it does not store any state in itself.
+ */
+public class CollectSinkOperatorCoordinator implements OperatorCoordinator, CoordinationRequestHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(CollectSinkOperatorCoordinator.class);
+
+	private InetSocketAddress address;
+	private Socket socket;
+
+	private ExecutorService executorService;
+
+	@Override
+	public void start() throws Exception {
+		this.executorService = Executors.newFixedThreadPool(1);
+	}
+
+	@Override
+	public void close() throws Exception {
+		closeCurrentConnection();
+		this.executorService.shutdown();
+	}
+
+	@Override
+	public void handleEventFromOperator(int subtask, OperatorEvent event) throws Exception {
+		Preconditions.checkArgument(
+			event instanceof CollectSinkAddressEvent, "Operator event must be a CollectSinkAddressEvent");
+		address = ((CollectSinkAddressEvent) event).getAddress();
+		LOG.debug("Received sink socket server address: " + address);
+
+		// this event is sent when the sink function starts, so we remove the old socket if it is present
+		socket = null;
+	}
+
+	@Override
+	public CompletableFuture<CoordinationResponse> handleCoordinationRequest(CoordinationRequest request) {
+		Preconditions.checkArgument(
+			request instanceof CollectCoordinationRequest,
+			"Coordination request must be a CollectCoordinationRequest");
+
+		CompletableFuture<CoordinationResponse> responseFuture = new CompletableFuture<>();
+		executorService.submit(() -> handleRequestImpl((CollectCoordinationRequest) request, responseFuture));
+		return responseFuture;
+	}
+
+	private void handleRequestImpl(
+			CollectCoordinationRequest request,
+			CompletableFuture<CoordinationResponse> responseFuture) {
+		// there is no thread safety issue in this method

Review comment:
       not really, event handling is executed by JM's rpc thread.

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectSinkOperatorCoordinator.java
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequest;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequestHandler;
+import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.Collections;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * {@link OperatorCoordinator} for {@link CollectSinkFunction}.
+ *
+ * <p>This coordinator only forwards requests and responses from clients and sinks,
+ * it does not store any state in itself.
+ */
+public class CollectSinkOperatorCoordinator implements OperatorCoordinator, CoordinationRequestHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(CollectSinkOperatorCoordinator.class);
+
+	private InetSocketAddress address;
+	private Socket socket;
+
+	private ExecutorService executorService;
+
+	@Override
+	public void start() throws Exception {
+		this.executorService = Executors.newFixedThreadPool(1);
+	}
+
+	@Override
+	public void close() throws Exception {
+		closeCurrentConnection();
+		this.executorService.shutdown();
+	}
+
+	@Override
+	public void handleEventFromOperator(int subtask, OperatorEvent event) throws Exception {
+		Preconditions.checkArgument(
+			event instanceof CollectSinkAddressEvent, "Operator event must be a CollectSinkAddressEvent");
+		address = ((CollectSinkAddressEvent) event).getAddress();
+		LOG.debug("Received sink socket server address: " + address);
+
+		// this event is sent when the sink function starts, so we remove the old socket if it is present
+		socket = null;

Review comment:
       call `closeCurrentConnection` instead?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectSinkOperatorCoordinator.java
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequest;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequestHandler;
+import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.Collections;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * {@link OperatorCoordinator} for {@link CollectSinkFunction}.
+ *
+ * <p>This coordinator only forwards requests and responses from clients and sinks,
+ * it does not store any state in itself.
+ */
+public class CollectSinkOperatorCoordinator implements OperatorCoordinator, CoordinationRequestHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(CollectSinkOperatorCoordinator.class);
+
+	private InetSocketAddress address;
+	private Socket socket;
+
+	private ExecutorService executorService;
+
+	@Override
+	public void start() throws Exception {
+		this.executorService = Executors.newFixedThreadPool(1);
+	}
+
+	@Override
+	public void close() throws Exception {
+		closeCurrentConnection();
+		this.executorService.shutdown();
+	}
+
+	@Override
+	public void handleEventFromOperator(int subtask, OperatorEvent event) throws Exception {
+		Preconditions.checkArgument(
+			event instanceof CollectSinkAddressEvent, "Operator event must be a CollectSinkAddressEvent");
+		address = ((CollectSinkAddressEvent) event).getAddress();
+		LOG.debug("Received sink socket server address: " + address);
+
+		// this event is sent when the sink function starts, so we remove the old socket if it is present
+		socket = null;
+	}
+
+	@Override
+	public CompletableFuture<CoordinationResponse> handleCoordinationRequest(CoordinationRequest request) {
+		Preconditions.checkArgument(
+			request instanceof CollectCoordinationRequest,
+			"Coordination request must be a CollectCoordinationRequest");
+
+		CompletableFuture<CoordinationResponse> responseFuture = new CompletableFuture<>();
+		executorService.submit(() -> handleRequestImpl((CollectCoordinationRequest) request, responseFuture));
+		return responseFuture;
+	}
+
+	private void handleRequestImpl(
+			CollectCoordinationRequest request,
+			CompletableFuture<CoordinationResponse> responseFuture) {
+		// there is no thread safety issue in this method
+		// because the thread pool has only 1 thread
+		// and by design there shall be only 1 client reading from this coordinator
+		if (address == null) {
+			responseFuture.complete(new CollectCoordinationResponse(createEmptySerializedResult(request)));
+		}
+
+		try {
+			if (socket == null) {
+				socket = new Socket(address.getAddress(), address.getPort());
+				LOG.debug("Sink connection established");
+			}
+
+			// send version and token to sink server
+			LOG.debug("Forwarding request to sink socket server");
+			DataOutputViewStreamWrapper outStream = new DataOutputViewStreamWrapper(socket.getOutputStream());

Review comment:
       after socket established, this `outStream` and `inStream` could be reused and don't have to create new instance each time

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectSinkOperatorCoordinator.java
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequest;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequestHandler;
+import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.Collections;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * {@link OperatorCoordinator} for {@link CollectSinkFunction}.
+ *
+ * <p>This coordinator only forwards requests and responses from clients and sinks,
+ * it does not store any state in itself.
+ */
+public class CollectSinkOperatorCoordinator implements OperatorCoordinator, CoordinationRequestHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(CollectSinkOperatorCoordinator.class);
+
+	private InetSocketAddress address;
+	private Socket socket;
+
+	private ExecutorService executorService;
+
+	@Override
+	public void start() throws Exception {
+		this.executorService = Executors.newFixedThreadPool(1);
+	}
+
+	@Override
+	public void close() throws Exception {
+		closeCurrentConnection();
+		this.executorService.shutdown();
+	}
+
+	@Override
+	public void handleEventFromOperator(int subtask, OperatorEvent event) throws Exception {
+		Preconditions.checkArgument(
+			event instanceof CollectSinkAddressEvent, "Operator event must be a CollectSinkAddressEvent");
+		address = ((CollectSinkAddressEvent) event).getAddress();
+		LOG.debug("Received sink socket server address: " + address);

Review comment:
       Use info log

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectSinkOperatorCoordinator.java
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequest;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequestHandler;
+import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.Collections;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * {@link OperatorCoordinator} for {@link CollectSinkFunction}.
+ *
+ * <p>This coordinator only forwards requests and responses from clients and sinks,
+ * it does not store any state in itself.
+ */
+public class CollectSinkOperatorCoordinator implements OperatorCoordinator, CoordinationRequestHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(CollectSinkOperatorCoordinator.class);
+
+	private InetSocketAddress address;
+	private Socket socket;
+
+	private ExecutorService executorService;
+
+	@Override
+	public void start() throws Exception {
+		this.executorService = Executors.newFixedThreadPool(1);
+	}
+
+	@Override
+	public void close() throws Exception {
+		closeCurrentConnection();
+		this.executorService.shutdown();
+	}
+
+	@Override
+	public void handleEventFromOperator(int subtask, OperatorEvent event) throws Exception {
+		Preconditions.checkArgument(
+			event instanceof CollectSinkAddressEvent, "Operator event must be a CollectSinkAddressEvent");
+		address = ((CollectSinkAddressEvent) event).getAddress();
+		LOG.debug("Received sink socket server address: " + address);
+
+		// this event is sent when the sink function starts, so we remove the old socket if it is present
+		socket = null;
+	}
+
+	@Override
+	public CompletableFuture<CoordinationResponse> handleCoordinationRequest(CoordinationRequest request) {
+		Preconditions.checkArgument(
+			request instanceof CollectCoordinationRequest,
+			"Coordination request must be a CollectCoordinationRequest");
+
+		CompletableFuture<CoordinationResponse> responseFuture = new CompletableFuture<>();
+		executorService.submit(() -> handleRequestImpl((CollectCoordinationRequest) request, responseFuture));
+		return responseFuture;
+	}
+
+	private void handleRequestImpl(
+			CollectCoordinationRequest request,
+			CompletableFuture<CoordinationResponse> responseFuture) {
+		// there is no thread safety issue in this method
+		// because the thread pool has only 1 thread
+		// and by design there shall be only 1 client reading from this coordinator
+		if (address == null) {
+			responseFuture.complete(new CollectCoordinationResponse(createEmptySerializedResult(request)));
+		}
+
+		try {
+			if (socket == null) {
+				socket = new Socket(address.getAddress(), address.getPort());

Review comment:
       also set 
   		socket.setKeepAlive(true);
   		socket.setTcpNoDelay(true);

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectCoordinationResponse.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A {@link CoordinationResponse} from the coordinator containing the required batch or new results
+ * and other necessary information in serialized form.
+ */
+public class CollectCoordinationResponse implements CoordinationResponse {
+
+	private static final long serialVersionUID = 1L;
+
+	private final byte[] bytes;
+
+	public CollectCoordinationResponse(byte[] bytes) {
+		this.bytes = bytes;
+	}
+
+	public <T> DeserializedResponse<T> getDeserializedResponse(TypeSerializer<T> serializer) {
+		ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+		DataInputViewStreamWrapper wrapper = new DataInputViewStreamWrapper(bais);
+		TypeSerializer<String> versionSerializer = new StringSerializer();
+		TypeSerializer<Long> tokenSerializer = new LongSerializer();
+		try {
+			String version = versionSerializer.deserialize(wrapper);
+			long token = tokenSerializer.deserialize(wrapper);
+			long checkpointedToken = tokenSerializer.deserialize(wrapper);
+			int length = wrapper.readInt();
+			List<T> results = new ArrayList<>(length);
+			for (int i = 0; i < length; i++) {
+				results.add(serializer.deserialize(wrapper));
+			}
+			return new DeserializedResponse<>(version, token, checkpointedToken, results);
+		} catch (IOException e) {
+			throw new RuntimeException("Failed to deserialize collect sink result", e);
+		}
+	}
+
+	public static <T> byte[] serialize(
+			String version,
+			long token,
+			long lastCheckpointId,
+			List<T> results,
+			TypeSerializer<T> serializer) {
+		ByteArrayOutputStream baos = new ByteArrayOutputStream();
+		DataOutputViewStreamWrapper wrapper = new DataOutputViewStreamWrapper(baos);
+		TypeSerializer<String> versionSerializer = new StringSerializer();
+		TypeSerializer<Long> tokenSerializer = new LongSerializer();

Review comment:
       ditto

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectCoordinationResponse.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A {@link CoordinationResponse} from the coordinator containing the required batch or new results
+ * and other necessary information in serialized form.
+ */
+public class CollectCoordinationResponse implements CoordinationResponse {
+
+	private static final long serialVersionUID = 1L;
+
+	private final byte[] bytes;
+
+	public CollectCoordinationResponse(byte[] bytes) {
+		this.bytes = bytes;
+	}
+
+	public <T> DeserializedResponse<T> getDeserializedResponse(TypeSerializer<T> serializer) {
+		ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+		DataInputViewStreamWrapper wrapper = new DataInputViewStreamWrapper(bais);
+		TypeSerializer<String> versionSerializer = new StringSerializer();
+		TypeSerializer<Long> tokenSerializer = new LongSerializer();

Review comment:
       use `LongSerializer.INSTANCE` instead

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectSinkOperatorCoordinator.java
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequest;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequestHandler;
+import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.Collections;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * {@link OperatorCoordinator} for {@link CollectSinkFunction}.
+ *
+ * <p>This coordinator only forwards requests and responses from clients and sinks,
+ * it does not store any state in itself.
+ */
+public class CollectSinkOperatorCoordinator implements OperatorCoordinator, CoordinationRequestHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(CollectSinkOperatorCoordinator.class);
+
+	private InetSocketAddress address;
+	private Socket socket;
+
+	private ExecutorService executorService;
+
+	@Override
+	public void start() throws Exception {
+		this.executorService = Executors.newFixedThreadPool(1);

Review comment:
       give the thread a name by using `Executors.newSingleThreadExecutor(new ExecutorThreadFactory("the name you want"));`

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectCoordinationResponse.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A {@link CoordinationResponse} from the coordinator containing the required batch or new results
+ * and other necessary information in serialized form.
+ */
+public class CollectCoordinationResponse implements CoordinationResponse {
+
+	private static final long serialVersionUID = 1L;
+
+	private final byte[] bytes;
+
+	public CollectCoordinationResponse(byte[] bytes) {
+		this.bytes = bytes;
+	}
+
+	public <T> DeserializedResponse<T> getDeserializedResponse(TypeSerializer<T> serializer) {
+		ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+		DataInputViewStreamWrapper wrapper = new DataInputViewStreamWrapper(bais);
+		TypeSerializer<String> versionSerializer = new StringSerializer();
+		TypeSerializer<Long> tokenSerializer = new LongSerializer();
+		try {
+			String version = versionSerializer.deserialize(wrapper);
+			long token = tokenSerializer.deserialize(wrapper);
+			long checkpointedToken = tokenSerializer.deserialize(wrapper);
+			int length = wrapper.readInt();
+			List<T> results = new ArrayList<>(length);
+			for (int i = 0; i < length; i++) {
+				results.add(serializer.deserialize(wrapper));
+			}
+			return new DeserializedResponse<>(version, token, checkpointedToken, results);
+		} catch (IOException e) {
+			throw new RuntimeException("Failed to deserialize collect sink result", e);
+		}
+	}
+
+	public static <T> byte[] serialize(
+			String version,
+			long token,
+			long lastCheckpointId,
+			List<T> results,
+			TypeSerializer<T> serializer) {
+		ByteArrayOutputStream baos = new ByteArrayOutputStream();
+		DataOutputViewStreamWrapper wrapper = new DataOutputViewStreamWrapper(baos);
+		TypeSerializer<String> versionSerializer = new StringSerializer();
+		TypeSerializer<Long> tokenSerializer = new LongSerializer();
+		try {
+			versionSerializer.serialize(version, wrapper);
+			tokenSerializer.serialize(token, wrapper);
+			tokenSerializer.serialize(lastCheckpointId, wrapper);
+			wrapper.writeInt(results.size());

Review comment:
       use `ListSerializer`

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectSinkFunction.java
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.api.common.accumulators.LongCounter;
+import org.apache.flink.api.common.accumulators.SerializedListAccumulator;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.operators.coordination.OperatorEventGateway;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A sink function that collects query results and sends them back to the client.
+ *
+ * <p>This sink works by limiting the number of results buffered in it (can be configured) so
+ * that when the buffer is full, it back-pressures the job until the client consumes some results.
+ *
+ * <p>NOTE: When using this sink, make sure that its parallelism is 1, and make sure that it is used
+ * in a {@link StreamTask}.
+ *
+ * @param <IN> type of results to be written into the sink.
+ */
+@Internal
+public class CollectSinkFunction<IN> extends RichSinkFunction<IN> implements CheckpointedFunction {
+
+	private static final Logger LOG = LoggerFactory.getLogger(CollectSinkFunction.class);
+
+	private final TypeSerializer<IN> serializer;
+	private final LinkedList<IN> bufferedResults;
+	private final int maxResultsPerBatch;
+	private final int maxResultsBuffered;
+	private final String finalResultListAccumulatorName;
+	private final String finalResultTokenAccumulatorName;
+
+	private final ReentrantLock bufferedResultsLock;
+	private final Condition bufferNotFullCondition;
+
+	private OperatorEventGateway eventGateway;
+
+	private String version;
+	private long firstBufferedResultToken;
+	private long lastCheckpointId;
+	private ServerThread serverThread;
+
+	private ListState<IN> bufferedResultsState;
+	private ListState<Long> firstBufferedResultTokenState;
+	private ListState<Long> lastCheckpointIdState;

Review comment:
       mark all `runtime generated` fields with `transient`, this would reduce the serialization cost when sending this function from client to server




----------------------------------------------------------------
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:
users@infra.apache.org