Skip to content

Commit

Permalink
[hotfix] [contrib] Fix robustness of DataStreamUtils stream collecting
Browse files Browse the repository at this point in the history
  • Loading branch information
StephanEwen committed May 30, 2016
1 parent 77afe28 commit d3e3bd5
Show file tree
Hide file tree
Showing 6 changed files with 398 additions and 261 deletions.
Expand Up @@ -17,35 +17,30 @@

package org.apache.flink.contrib.streaming;

import java.io.DataOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.net.Socket;
import java.net.InetAddress;

import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
import org.apache.flink.streaming.api.functions.sink.SocketClientSink;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.DataOutputView;

/**
* A specialized data sink to be used by DataStreamUtils.collect.
*/
class CollectSink<IN> extends RichSinkFunction<IN> {

private static final long serialVersionUID = 1L;

private static final Logger LOG = LoggerFactory.getLogger(SocketClientSink.class);

private final InetAddress hostIp;
private final int port;
private final TypeSerializer<IN> serializer;

private transient Socket client;
private transient DataOutputStream dataOutputStream;
private StreamWriterDataOutputView streamWriter;
private transient OutputStream outputStream;
private transient DataOutputViewStreamWrapper streamWriter;

/**
* Creates a CollectSink that will send the data to the specified host.
Expand All @@ -60,57 +55,13 @@ public CollectSink(InetAddress hostIp, int port, TypeSerializer<IN> serializer)
this.serializer = serializer;
}

/**
* Initializes the connection to Socket.
*/
public void initializeConnection() {
OutputStream outputStream;
try {
client = new Socket(hostIp, port);
outputStream = client.getOutputStream();
streamWriter = new StreamWriterDataOutputView(outputStream);
} catch (IOException e) {
throw new RuntimeException(e);
}
dataOutputStream = new DataOutputStream(outputStream);
}

/**
* Called when new data arrives to the sink, and forwards it to Socket.
*
* @param value
* The incoming data
*/
@Override
public void invoke(IN value) {
public void invoke(IN value) throws Exception {
try {
serializer.serialize(value, streamWriter);
} catch (IOException e) {
if(LOG.isErrorEnabled()){
LOG.error("Cannot send message to socket server at " + hostIp.toString() + ":" + port, e);
}
}
}

/**
* Closes the connection of the Socket client.
*/
private void closeConnection(){
try {
dataOutputStream.flush();
client.close();
} catch (IOException e) {
throw new RuntimeException("Error while closing connection with socket server at "
+ hostIp.toString() + ":" + port, e);
} finally {
if (client != null) {
try {
client.close();
} catch (IOException e) {
LOG.error("Cannot close connection with socket server at "
+ hostIp.toString() + ":" + port, e);
}
}
catch (Exception e) {
throw new IOException("Error sending data back to client (" + hostIp.toString() + ":" + port + ')', e);
}
}

Expand All @@ -119,34 +70,47 @@ private void closeConnection(){
* @param parameters Configuration.
*/
@Override
public void open(Configuration parameters) {
initializeConnection();
public void open(Configuration parameters) throws Exception {
try {
client = new Socket(hostIp, port);
outputStream = client.getOutputStream();
streamWriter = new DataOutputViewStreamWrapper(outputStream);
}
catch (IOException e) {
throw new IOException("Cannot connect to the client to send back the stream", e);
}
}

/**
* Closes the connection with the Socket server.
*/
@Override
public void close() {
closeConnection();
}

private static class StreamWriterDataOutputView extends DataOutputStream implements DataOutputView {

public StreamWriterDataOutputView(OutputStream stream) {
super(stream);
}

public void skipBytesToWrite(int numBytes) throws IOException {
for (int i = 0; i < numBytes; i++) {
write(0);
public void close() throws Exception {
try {
if (outputStream != null) {
outputStream.flush();
outputStream.close();
}

// first regular attempt to cleanly close. Failing that will escalate
if (client != null) {
client.close();
}
}

public void write(DataInputView source, int numBytes) throws IOException {
byte[] data = new byte[numBytes];
source.readFully(data);
write(data);
catch (Exception e) {
throw new IOException("Error while closing connection that streams data back to client at "
+ hostIp.toString() + ":" + port, e);
}
finally {
// if we failed prior to closing the client, close it
if (client != null) {
try {
client.close();
}
catch (Throwable t) {
// best effort to close, we do not care about an exception here any more
}
}
}
}
}

This file was deleted.

0 comments on commit d3e3bd5

Please sign in to comment.