Skip to content

Commit fdc3d79

Browse files
committed
[FLINK-15571] Change Redis Sink to AsyncSink implementation
1 parent d272e01 commit fdc3d79

File tree

9 files changed

+372
-107
lines changed

9 files changed

+372
-107
lines changed
Lines changed: 35 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -14,19 +14,24 @@
1414
* See the License for the specific language governing permissions and
1515
* limitations under the License.
1616
*/
17-
package org.apache.flink.connector.redis.streams.sink.command;
1817

18+
package org.apache.flink.connector.redis.streams.sink;
1919

2020
import org.apache.flink.connector.redis.streams.sink.connection.JedisConnector;
21+
2122
import redis.clients.jedis.StreamEntryID;
2223

24+
import java.io.Serializable;
2325
import java.util.Map;
2426

25-
public class StreamRedisCommand implements RedisCommand {
27+
/** A Redis Streams Command. */
28+
public class RedisStreamsCommand implements Serializable {
29+
30+
private transient StreamEntryID streamId = null;
2631
public final String key;
2732
public final Map<String, String> value;
2833

29-
private StreamRedisCommand(String key, Map<String, String> value) {
34+
private RedisStreamsCommand(String key, Map<String, String> value) {
3035
this.key = key;
3136
this.value = value;
3237
}
@@ -35,11 +40,33 @@ public static Builder builder() {
3540
return new Builder();
3641
}
3742

38-
@Override
3943
public void send(JedisConnector connector) {
40-
connector.getJedisCommands().xadd(key, StreamEntryID.NEW_ENTRY, value);
44+
this.streamId =
45+
connector
46+
.getJedisCommands()
47+
.xadd(
48+
key,
49+
(this.streamId != null) ? this.streamId : StreamEntryID.NEW_ENTRY,
50+
value);
51+
}
52+
53+
public boolean sendCorrectly() {
54+
return true;
55+
}
56+
57+
public boolean sendIncorrectly() {
58+
return !sendCorrectly();
59+
}
60+
61+
public long getMessageSize() {
62+
return this.key.length()
63+
+ this.value.entrySet().stream()
64+
.map(k -> k.getKey().length() + k.getValue().length())
65+
.reduce(Integer::sum)
66+
.orElse(0);
4167
}
4268

69+
/** The builder for {@link RedisStreamsCommand}. */
4370
public static class Builder {
4471
private String key;
4572
private Map<String, String> value;
@@ -54,8 +81,8 @@ public Builder withValue(Map<String, String> value) {
5481
return this;
5582
}
5683

57-
public StreamRedisCommand build() {
58-
return new StreamRedisCommand(key, value);
84+
public RedisStreamsCommand build() {
85+
return new RedisStreamsCommand(key, value);
5986
}
6087
}
61-
}
88+
}

flink-connector-redis-streams/src/main/java/org/apache/flink/connector/redis/streams/sink/RedisSerializer.java renamed to flink-connector-redis-streams/src/main/java/org/apache/flink/connector/redis/streams/sink/RedisStreamsCommandSerializer.java

Lines changed: 4 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -14,19 +14,15 @@
1414
* See the License for the specific language governing permissions and
1515
* limitations under the License.
1616
*/
17-
package org.apache.flink.connector.redis.streams.sink;
1817

19-
import org.apache.flink.api.common.functions.Function;
20-
import org.apache.flink.connector.redis.streams.sink.command.RedisCommand;
18+
package org.apache.flink.connector.redis.streams.sink;
2119

22-
import java.io.Serializable;
20+
import org.apache.flink.connector.base.sink.writer.ElementConverter;
2321

2422
/**
2523
* Function that creates the description how the input data should be mapped to redis type.
2624
*
2725
* @param <T> The type of the element handled by this {@code RedisSerializer}
2826
*/
29-
public interface RedisSerializer<T> extends Function, Serializable {
30-
31-
RedisCommand getMessage(T input);
32-
}
27+
public interface RedisStreamsCommandSerializer<T>
28+
extends ElementConverter<T, RedisStreamsCommand> {}

flink-connector-redis-streams/src/main/java/org/apache/flink/connector/redis/streams/sink/RedisStreamsSink.java

Lines changed: 50 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -14,30 +14,72 @@
1414
* See the License for the specific language governing permissions and
1515
* limitations under the License.
1616
*/
17+
1718
package org.apache.flink.connector.redis.streams.sink;
1819

19-
import org.apache.flink.api.connector.sink2.Sink;
20-
import org.apache.flink.api.connector.sink2.SinkWriter;
20+
import org.apache.flink.connector.base.sink.AsyncSinkBase;
21+
import org.apache.flink.connector.base.sink.writer.BufferedRequestState;
22+
import org.apache.flink.connector.base.sink.writer.config.AsyncSinkWriterConfiguration;
2123
import org.apache.flink.connector.redis.streams.sink.config.JedisConfig;
2224
import org.apache.flink.connector.redis.streams.sink.connection.JedisConnector;
2325
import org.apache.flink.connector.redis.streams.sink.connection.JedisConnectorBuilder;
26+
import org.apache.flink.core.io.SimpleVersionedSerializer;
2427

2528
import java.io.IOException;
29+
import java.util.Collection;
30+
import java.util.Collections;
2631

27-
public class RedisStreamsSink<T> implements Sink<T> {
32+
/**
33+
* A sink for publishing data into Redis.
34+
*
35+
* @param <T>
36+
*/
37+
public class RedisStreamsSink<T> extends AsyncSinkBase<T, RedisStreamsCommand> {
2838

2939
private final JedisConfig jedisConfig;
30-
private final RedisSerializer<T> serializer;
3140

32-
public RedisStreamsSink(JedisConfig jedisConfig, RedisSerializer<T> serializer) {
41+
public RedisStreamsSink(
42+
JedisConfig jedisConfig,
43+
RedisStreamsCommandSerializer<T> converter,
44+
AsyncSinkWriterConfiguration asyncConfig) {
45+
super(
46+
converter,
47+
asyncConfig.getMaxBatchSize(),
48+
asyncConfig.getMaxInFlightRequests(),
49+
asyncConfig.getMaxBufferedRequests(),
50+
asyncConfig.getMaxBatchSizeInBytes(),
51+
asyncConfig.getMaxTimeInBufferMS(),
52+
asyncConfig.getMaxRecordSizeInBytes());
3353
this.jedisConfig = jedisConfig;
34-
this.serializer = serializer;
3554
}
3655

3756
@Override
38-
public SinkWriter<T> createWriter(InitContext initContext) throws IOException {
57+
public RedisStreamsWriter<T> createWriter(InitContext initContext) throws IOException {
58+
return restoreWriter(initContext, Collections.emptyList());
59+
}
60+
61+
@Override
62+
public RedisStreamsWriter<T> restoreWriter(
63+
InitContext initContext,
64+
Collection<BufferedRequestState<RedisStreamsCommand>> recoveredState)
65+
throws IOException {
66+
AsyncSinkWriterConfiguration asyncConfig =
67+
AsyncSinkWriterConfiguration.builder()
68+
.setMaxBatchSize(getMaxBatchSize())
69+
.setMaxBatchSizeInBytes(getMaxBatchSizeInBytes())
70+
.setMaxInFlightRequests(getMaxInFlightRequests())
71+
.setMaxBufferedRequests(getMaxBufferedRequests())
72+
.setMaxTimeInBufferMS(getMaxTimeInBufferMS())
73+
.setMaxRecordSizeInBytes(getMaxRecordSizeInBytes())
74+
.build();
3975
JedisConnector connection = JedisConnectorBuilder.build(jedisConfig);
40-
return new RedisStreamsWriter<>(connection, this.serializer);
76+
return new RedisStreamsWriter<>(
77+
connection, getElementConverter(), asyncConfig, initContext, recoveredState);
4178
}
4279

80+
@Override
81+
public SimpleVersionedSerializer<BufferedRequestState<RedisStreamsCommand>>
82+
getWriterStateSerializer() {
83+
return new RedisStreamsStateSerializer();
84+
}
4385
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,117 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.flink.connector.redis.streams.sink;
19+
20+
import org.apache.flink.connector.base.sink.writer.BufferedRequestState;
21+
import org.apache.flink.connector.base.sink.writer.RequestEntryWrapper;
22+
import org.apache.flink.core.io.SimpleVersionedSerializer;
23+
24+
import java.io.ByteArrayInputStream;
25+
import java.io.ByteArrayOutputStream;
26+
import java.io.DataInputStream;
27+
import java.io.DataOutputStream;
28+
import java.io.IOException;
29+
import java.util.ArrayList;
30+
import java.util.Collection;
31+
import java.util.HashMap;
32+
import java.util.List;
33+
import java.util.Map;
34+
35+
/** The Redis implementation for {@link SimpleVersionedSerializer}. */
36+
public class RedisStreamsStateSerializer
37+
implements SimpleVersionedSerializer<BufferedRequestState<RedisStreamsCommand>> {
38+
39+
@Override
40+
public int getVersion() {
41+
return 1;
42+
}
43+
44+
@Override
45+
public byte[] serialize(BufferedRequestState<RedisStreamsCommand> obj) throws IOException {
46+
Collection<RequestEntryWrapper<RedisStreamsCommand>> bufferState =
47+
obj.getBufferedRequestEntries();
48+
49+
try (final ByteArrayOutputStream baos = new ByteArrayOutputStream();
50+
final DataOutputStream out = new DataOutputStream(baos)) {
51+
52+
out.writeInt(getVersion());
53+
out.writeInt(bufferState.size());
54+
55+
for (RequestEntryWrapper<RedisStreamsCommand> wrapper : bufferState) {
56+
RedisStreamsCommand command = wrapper.getRequestEntry();
57+
writeString(out, command.key);
58+
out.writeInt(command.value.size());
59+
for (Map.Entry<String, String> entry : command.value.entrySet()) {
60+
writeString(out, entry.getKey());
61+
writeString(out, entry.getValue());
62+
}
63+
}
64+
65+
out.flush();
66+
return baos.toByteArray();
67+
}
68+
}
69+
70+
@Override
71+
public BufferedRequestState<RedisStreamsCommand> deserialize(int version, byte[] serialized)
72+
throws IOException {
73+
try (final ByteArrayInputStream bais = new ByteArrayInputStream(serialized);
74+
final DataInputStream in = new DataInputStream(bais)) {
75+
76+
int byteVersion = in.readInt();
77+
78+
int bufferSize = in.readInt();
79+
List<RequestEntryWrapper<RedisStreamsCommand>> state = new ArrayList<>();
80+
for (int bs = 0; bs < bufferSize; bs++) {
81+
String key = readString(in);
82+
83+
int valueSize = in.readInt();
84+
Map<String, String> values = new HashMap<>();
85+
for (int i = 0; i < valueSize; i++) {
86+
String eKey = readString(in);
87+
String eValue = readString(in);
88+
values.put(eKey, eValue);
89+
}
90+
91+
RedisStreamsCommand command =
92+
RedisStreamsCommand.builder().withKey(key).withValue(values).build();
93+
94+
state.add(new RequestEntryWrapper<>(command, command.getMessageSize()));
95+
}
96+
return new BufferedRequestState<>(state);
97+
}
98+
}
99+
100+
private void writeString(final DataOutputStream out, String value) throws IOException {
101+
out.writeInt(value.length());
102+
out.writeBytes(value);
103+
}
104+
105+
private String readString(final DataInputStream in) throws IOException {
106+
int sizeToRead = in.readInt();
107+
byte[] bytesRead = new byte[sizeToRead];
108+
int sizeRead = in.read(bytesRead);
109+
110+
if (sizeToRead != sizeRead) {
111+
throw new IOException(
112+
String.format("Expected to read %s but read %s", sizeToRead, sizeRead));
113+
}
114+
115+
return new String(bytesRead);
116+
}
117+
}

flink-connector-redis-streams/src/main/java/org/apache/flink/connector/redis/streams/sink/RedisStreamsWriter.java

Lines changed: 30 additions & 28 deletions
Original file line numberDiff line numberDiff line change
@@ -14,48 +14,50 @@
1414
* See the License for the specific language governing permissions and
1515
* limitations under the License.
1616
*/
17+
1718
package org.apache.flink.connector.redis.streams.sink;
1819

19-
import org.apache.flink.api.connector.sink2.SinkWriter;
20-
import org.apache.flink.connector.redis.streams.sink.command.RedisCommand;
20+
import org.apache.flink.api.connector.sink2.Sink;
21+
import org.apache.flink.connector.base.sink.writer.AsyncSinkWriter;
22+
import org.apache.flink.connector.base.sink.writer.BufferedRequestState;
23+
import org.apache.flink.connector.base.sink.writer.ElementConverter;
24+
import org.apache.flink.connector.base.sink.writer.config.AsyncSinkWriterConfiguration;
2125
import org.apache.flink.connector.redis.streams.sink.connection.JedisConnector;
2226

23-
import java.io.IOException;
24-
import java.util.ArrayDeque;
25-
import java.util.Queue;
27+
import java.util.Collection;
28+
import java.util.List;
29+
import java.util.function.Consumer;
30+
import java.util.stream.Collectors;
2631

27-
public class RedisStreamsWriter<T> implements SinkWriter<T> {
32+
class RedisStreamsWriter<T> extends AsyncSinkWriter<T, RedisStreamsCommand> {
2833

2934
private final JedisConnector jedisConnector;
30-
private final RedisSerializer<T> serializer;
31-
private final Queue<RedisCommand> queue = new ArrayDeque<>();
3235

33-
public RedisStreamsWriter(JedisConnector jedisConnector, RedisSerializer<T> serializer) {
36+
public RedisStreamsWriter(
37+
JedisConnector jedisConnector,
38+
ElementConverter<T, RedisStreamsCommand> elementConverter,
39+
AsyncSinkWriterConfiguration asyncConfig,
40+
Sink.InitContext initContext,
41+
Collection<BufferedRequestState<RedisStreamsCommand>> recoveredState) {
42+
super(elementConverter, initContext, asyncConfig, recoveredState);
3443
this.jedisConnector = jedisConnector;
35-
this.serializer = serializer;
3644
}
3745

38-
3946
@Override
40-
public void write(T input, Context context) throws IOException, InterruptedException {
41-
RedisCommand message = serializer.getMessage(input);
42-
queue.add(message);
43-
}
44-
45-
@Override
46-
public void flush(boolean endOfInput) throws IOException, InterruptedException {
47-
flush();
48-
}
49-
50-
private void flush() {
51-
while(!this.queue.isEmpty()) {
52-
RedisCommand element = this.queue.remove();
53-
element.send(this.jedisConnector);
54-
}
47+
protected void submitRequestEntries(
48+
List<RedisStreamsCommand> requestEntries,
49+
Consumer<List<RedisStreamsCommand>> requestResult) {
50+
List<RedisStreamsCommand> errors =
51+
requestEntries.stream()
52+
.peek(command -> command.send(this.jedisConnector))
53+
.filter(RedisStreamsCommand::sendIncorrectly)
54+
.collect(Collectors.toList());
55+
56+
requestResult.accept(errors);
5557
}
5658

5759
@Override
58-
public void close() throws Exception {
59-
jedisConnector.close();
60+
protected long getSizeInBytes(RedisStreamsCommand command) {
61+
return command.getMessageSize();
6062
}
6163
}

0 commit comments

Comments
 (0)