-
Notifications
You must be signed in to change notification settings - Fork 1.8k
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
[Feature][Connector-V2] Socket Connector Sink #2549
Merged
Merged
Changes from 3 commits
Commits
Show all changes
4 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,89 @@ | ||
# Socket | ||
|
||
> Socket sink connector | ||
|
||
## Description | ||
|
||
Used to send data to Socket Server. Both support streaming and batch mode. | ||
> For example, if the data from upstream is [`age: 12, name: jared`], the content send to socket server is the following: `{"name":"jared","age":17}` | ||
|
||
|
||
## Options | ||
|
||
| name | type | required | default value | | ||
| --- |--------|----------|---------------| | ||
| host | String | Yes | - | | ||
| port | Integer | yes | - | | ||
| max_retries | Integer | No | 3 | | ||
|
||
### host [string] | ||
socket server host | ||
|
||
### port [integer] | ||
|
||
socket server port | ||
|
||
### max_retries [integer] | ||
|
||
The number of retries to send record failed | ||
|
||
## Example | ||
|
||
simple: | ||
|
||
```hocon | ||
Socket { | ||
host = "localhost" | ||
port = 9999 | ||
} | ||
``` | ||
|
||
test: | ||
|
||
* Configuring the SeaTunnel config file | ||
|
||
```hocon | ||
env { | ||
execution.parallelism = 1 | ||
job.mode = "STREAMING" | ||
} | ||
|
||
source { | ||
FakeSource { | ||
result_table_name = "fake" | ||
schema = { | ||
fields { | ||
name = "string" | ||
age = "int" | ||
} | ||
} | ||
} | ||
} | ||
|
||
transform { | ||
sql = "select name, age from fake" | ||
} | ||
|
||
sink { | ||
Socket { | ||
host = "localhost" | ||
port = 9999 | ||
} | ||
} | ||
|
||
``` | ||
|
||
* Start a port listening | ||
|
||
```shell | ||
nc -l -v 9999 | ||
``` | ||
|
||
* Start a SeaTunnel task | ||
|
||
|
||
* Socket Server Console print data | ||
|
||
```text | ||
{"name":"jared","age":17} | ||
``` |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
43 changes: 43 additions & 0 deletions
43
...ket/src/main/java/org/apache/seatunnel/connectors/seatunnel/socket/config/SinkConfig.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,43 @@ | ||
/* | ||
* 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.seatunnel.connectors.seatunnel.socket.config; | ||
|
||
import org.apache.seatunnel.shade.com.typesafe.config.Config; | ||
|
||
import lombok.Data; | ||
|
||
import java.io.Serializable; | ||
|
||
@Data | ||
public class SinkConfig implements Serializable { | ||
public static final String HOST = "host"; | ||
public static final String PORT = "port"; | ||
private static final String MAX_RETRIES = "max_retries"; | ||
private static final int DEFAULT_MAX_RETRIES = 3; | ||
private String host; | ||
private int port; | ||
private Integer maxNumRetries = DEFAULT_MAX_RETRIES; | ||
|
||
public SinkConfig(Config config) { | ||
this.host = config.getString(HOST); | ||
this.port = config.getInt(PORT); | ||
if (config.hasPath(MAX_RETRIES)) { | ||
this.maxNumRetries = config.getInt(MAX_RETRIES); | ||
} | ||
} | ||
} |
178 changes: 178 additions & 0 deletions
178
...ket/src/main/java/org/apache/seatunnel/connectors/seatunnel/socket/sink/SocketClient.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,178 @@ | ||
/* | ||
* 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.seatunnel.connectors.seatunnel.socket.sink; | ||
|
||
import org.apache.seatunnel.api.serialization.SerializationSchema; | ||
import org.apache.seatunnel.api.table.type.SeaTunnelRow; | ||
import org.apache.seatunnel.connectors.seatunnel.socket.config.SinkConfig; | ||
|
||
import lombok.extern.slf4j.Slf4j; | ||
|
||
import java.io.IOException; | ||
import java.io.OutputStream; | ||
import java.net.Socket; | ||
|
||
@Slf4j | ||
public class SocketClient { | ||
|
||
private final String hostName; | ||
private final int port; | ||
private int retries; | ||
private final int maxNumRetries; | ||
private transient Socket client; | ||
private transient OutputStream outputStream; | ||
private final SerializationSchema serializationSchema; | ||
private volatile boolean isRunning = Boolean.TRUE; | ||
private static final int CONNECTION_RETRY_DELAY = 500; | ||
|
||
public SocketClient(SinkConfig config, SerializationSchema serializationSchema) { | ||
this.hostName = config.getHost(); | ||
this.port = config.getPort(); | ||
this.serializationSchema = serializationSchema; | ||
retries = config.getMaxNumRetries(); | ||
maxNumRetries = config.getMaxNumRetries(); | ||
} | ||
|
||
private void createConnection() throws IOException { | ||
client = new Socket(hostName, port); | ||
client.setKeepAlive(true); | ||
client.setTcpNoDelay(true); | ||
|
||
outputStream = client.getOutputStream(); | ||
} | ||
|
||
public void open() throws IOException { | ||
try { | ||
synchronized (SocketClient.class) { | ||
createConnection(); | ||
} | ||
} catch (IOException e) { | ||
throw new IOException("Cannot connect to socket server at " + hostName + ":" + port, e); | ||
} | ||
} | ||
|
||
public void wirte(SeaTunnelRow row) throws IOException { | ||
byte[] msg = serializationSchema.serialize(row); | ||
try { | ||
outputStream.write(msg); | ||
outputStream.flush(); | ||
|
||
} catch (IOException e) { | ||
// if no re-tries are enable, fail immediately | ||
if (maxNumRetries == 0) { | ||
throw new IOException( | ||
"Failed to send message '" | ||
+ row | ||
+ "' to socket server at " | ||
+ hostName | ||
+ ":" | ||
+ port | ||
+ ". Connection re-tries are not enabled.", | ||
e); | ||
} | ||
|
||
log.error( | ||
"Failed to send message '" | ||
+ row | ||
+ "' to socket server at " | ||
+ hostName | ||
+ ":" | ||
+ port | ||
+ ". Trying to reconnect...", | ||
e); | ||
|
||
synchronized (SocketClient.class) { | ||
IOException lastException = null; | ||
retries = 0; | ||
|
||
while (isRunning && (maxNumRetries < 0 || retries < maxNumRetries)) { | ||
|
||
// first, clean up the old resources | ||
try { | ||
if (outputStream != null) { | ||
outputStream.close(); | ||
} | ||
} catch (IOException ee) { | ||
log.error("Could not close output stream from failed write attempt", ee); | ||
} | ||
try { | ||
if (client != null) { | ||
client.close(); | ||
} | ||
} catch (IOException ee) { | ||
log.error("Could not close socket from failed write attempt", ee); | ||
} | ||
|
||
// try again | ||
retries++; | ||
|
||
try { | ||
// initialize a new connection | ||
createConnection(); | ||
outputStream.write(msg); | ||
return; | ||
} catch (IOException ee) { | ||
lastException = ee; | ||
log.error( | ||
"Re-connect to socket server and send message failed. Retry time(s): " | ||
+ retries, | ||
ee); | ||
} | ||
try { | ||
this.wait(CONNECTION_RETRY_DELAY); | ||
} | ||
catch (InterruptedException ex) { | ||
Thread.currentThread().interrupt(); | ||
throw new IOException( | ||
"unable to write; interrupted while doing another attempt", e); | ||
} | ||
} | ||
|
||
if (isRunning) { | ||
throw new IOException( | ||
"Failed to send message '" | ||
+ row | ||
+ "' to socket server at " | ||
+ hostName | ||
+ ":" | ||
+ port | ||
+ ". Failed after " | ||
+ retries | ||
+ " retries.", | ||
lastException); | ||
} | ||
} | ||
} | ||
} | ||
|
||
public void close() throws IOException { | ||
isRunning = false; | ||
synchronized (this) { | ||
this.notifyAll(); | ||
try { | ||
if (outputStream != null) { | ||
outputStream.close(); | ||
} | ||
} finally { | ||
if (client != null) { | ||
client.close(); | ||
} | ||
} | ||
} | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The same as
port
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thank you very much for your advice. I have fixed the problems mentioned above