Skip to content

[FLINK-15571][WIP] Add Redis Stream and String sinks #5

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

Open
wants to merge 2 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
169 changes: 169 additions & 0 deletions pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,169 @@
<?xml version="1.0" encoding="UTF-8"?>
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">

<modelVersion>4.0.0</modelVersion>

<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-redis</artifactId>
<version>1.16-SNAPSHOT</version>
<packaging>jar</packaging>
<name>Flink : Connectors : Redis</name>

<licenses>
<license>
<name>The Apache Software License, Version 2.0</name>
<url>https://www.apache.org/licenses/LICENSE-2.0.txt</url>
<distribution>repo</distribution>
</license>
</licenses>

<properties>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>

<!--<flink.version>1.16-SNAPSHOT</flink.version>-->
<flink.version>1.15.1</flink.version>
<!--<flink.shaded.version>15.0</flink.shaded.version>-->
<target.java.version>1.8</target.java.version>
<maven.compiler.source>${target.java.version}</maven.compiler.source>
<maven.compiler.target>${target.java.version}</maven.compiler.target>
<!-- Default scala versions, must be overwritten by build profiles, so we set something
invalid here -->
<!--<scala.version>2.12.7</scala.version>-->
<!--<scala.binary.version>2.12</scala.binary.version>-->
<jedis.version>4.2.3</jedis.version>
<slf4j.version>1.7.36</slf4j.version>
<testcontainers.version>1.17.3</testcontainers.version>
<!--<junit.version>4.13.2</junit.version>-->
<!--<hamcrest.version>1.3</hamcrest.version>-->
<junit.jupiter.version>5.8.2</junit.jupiter.version>
</properties>

<dependencies>

<!-- Redis dependencies -->

<dependency>
<groupId>redis.clients</groupId>
<artifactId>jedis</artifactId>
<version>${jedis.version}</version>
<scope>compile</scope>
<exclusions>
<exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j.api</artifactId>
</exclusion>
</exclusions>
</dependency>

<!-- Logging dependencies -->

<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
<version>${slf4j.version}</version>
</dependency>

<!-- Flink dependencies -->

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-base</artifactId>
<version>${flink.version}</version>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java</artifactId>
<version>${flink.version}</version>
<scope>provided</scope>
</dependency>

<!-- Table ecosystem -->

<!-- Projects depending on this project won't depend on flink-table-*. -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-api-java-bridge</artifactId>
<version>${flink.version}</version>
<scope>provided</scope>
<optional>true</optional>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
<exclusions>
<exclusion>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
<exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
</exclusion>
</exclusions>
</dependency>

<!-- Test dependencies -->

<!--<dependency>
<groupId>org.testcontainers</groupId>
<artifactId>testcontainers</artifactId>
<version>${testcontainers.version}</version>
<scope>test</scope>
</dependency>-->

<!--<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<version>${junit.version}</version>
<type>jar</type>
<scope>test</scope>
</dependency>-->

<dependency>
<groupId>org.junit.jupiter</groupId>
<artifactId>junit-jupiter-api</artifactId>
<version>${junit.jupiter.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.junit.jupiter</groupId>
<artifactId>junit-jupiter</artifactId>
<version>${junit.jupiter.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.testcontainers</groupId>
<artifactId>testcontainers</artifactId>
<version>${testcontainers.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.testcontainers</groupId>
<artifactId>junit-jupiter</artifactId>
<version>${testcontainers.version}</version>
<scope>test</scope>
</dependency>
</dependencies>

<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<includes>
<include>%regex[.*ITCase.*]</include>
</includes>
</configuration>
</plugin>
</plugins>
</build>

</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
/*
* 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.connector.redis;

import redis.clients.jedis.Protocol;

/** */
public class JedisConfigConstants {

public static final String REDIS_HOST = "redis.host";

public static final String REDIS_PORT = "redis.port";

public static final String DEFAULT_REDIS_HOST = Protocol.DEFAULT_HOST;

public static final int DEFAULT_REDIS_PORT = Protocol.DEFAULT_PORT;
}
58 changes: 58 additions & 0 deletions src/main/java/org/apache/flink/connector/redis/JedisUtils.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
/*
* 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.connector.redis;

import redis.clients.jedis.Connection;
import redis.clients.jedis.HostAndPort;
import redis.clients.jedis.UnifiedJedis;

import java.util.Properties;

/** */
public class JedisUtils {

public static UnifiedJedis createJedis(Properties configProps) {

String host =
configProps.getProperty(
JedisConfigConstants.REDIS_HOST, JedisConfigConstants.DEFAULT_REDIS_HOST);

int port =
Integer.parseInt(
configProps.getProperty(
JedisConfigConstants.REDIS_PORT,
Integer.toString(JedisConfigConstants.DEFAULT_REDIS_PORT)));

return new UnifiedJedis(new HostAndPort(host, port));
}

public static Connection createConnection(Properties configProps) {

String host =
configProps.getProperty(
JedisConfigConstants.REDIS_HOST, JedisConfigConstants.DEFAULT_REDIS_HOST);

int port =
Integer.parseInt(
configProps.getProperty(
JedisConfigConstants.REDIS_PORT,
Integer.toString(JedisConfigConstants.DEFAULT_REDIS_PORT)));

return new Connection(host, port);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
package org.apache.flink.connector.redis.sink2;

import org.apache.flink.api.connector.sink2.Sink;
import org.apache.flink.connector.base.sink.writer.AsyncSinkWriter;
import redis.clients.jedis.UnifiedJedis;
import redis.clients.jedis.exceptions.JedisDataException;

import java.util.ArrayList;
import java.util.List;
import java.util.function.Consumer;

public class RedisAsyncWriter<IN> extends AsyncSinkWriter<IN, RedisWriteRequest> {

private final UnifiedJedis jedis;

public RedisAsyncWriter(
UnifiedJedis jedis,
RedisConverter<IN> converter,
RedisSinkConfig sinkConfig,
Sink.InitContext context) {
super(
converter,
context,
sinkConfig.maxBatchSize,
sinkConfig.maxInFlightRequests,
sinkConfig.maxBufferedRequests,
sinkConfig.maxBatchSizeInBytes,
sinkConfig.maxTimeInBufferMS,
sinkConfig.maxRecordSizeInBytes);
this.jedis = jedis;
}

@Override
protected void submitRequestEntries(
List<RedisWriteRequest> requests, Consumer<List<RedisWriteRequest>> consumer) {
List<RedisWriteRequest> toRetry = new ArrayList<>();
for (RedisWriteRequest request : requests) {
try {
request.write(jedis);
} catch (JedisDataException de) {
// not worth retrying; will fail again
// TODO ?
} catch (Exception e) {
toRetry.add(request);
}
}
consumer.accept(toRetry);
}

@Override
protected long getSizeInBytes(RedisWriteRequest request) {
return request.getSizeInBytes();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,18 @@
package org.apache.flink.connector.redis.sink2;

import org.apache.flink.api.connector.sink2.SinkWriter;
import org.apache.flink.connector.base.sink.writer.ElementConverter;

public class RedisConverter<T> implements ElementConverter<T, RedisWriteRequest> {

private final RedisSerializer<T> serializer;

public RedisConverter(RedisSerializer<T> serializer) {
this.serializer = serializer;
}

@Override
public RedisWriteRequest apply(T input, SinkWriter.Context context) {
return serializer.serialize(input);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,10 @@
package org.apache.flink.connector.redis.sink2;

import org.apache.flink.api.common.functions.Function;

import java.io.Serializable;

public interface RedisSerializer<T> extends Function, Serializable {

RedisWriteRequest serialize(T input);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
package org.apache.flink.connector.redis.sink2;

import org.apache.flink.api.connector.sink2.Sink;
import org.apache.flink.api.connector.sink2.SinkWriter;
import org.apache.flink.connector.redis.JedisUtils;

import java.io.IOException;
import java.util.Properties;

public class RedisSink<T> implements Sink<T> {

private final Properties config;
private final RedisSerializer<T> serializer;

private final RedisSinkConfig sinkConfig;

public RedisSink(Properties config, RedisSinkConfig sinkConfig, RedisSerializer<T> serializer) {
this.config = config;
this.sinkConfig = sinkConfig;
this.serializer = serializer;
}

@Override
public SinkWriter<T> createWriter(Sink.InitContext initContext) throws IOException {
// return new RedisAsyncWriter<>(
// JedisUtils.createJedis(config),
// new RedisConverter<>(serializer),
// sinkConfig,
// initContext);
return new RedisSyncWriter<T>(JedisUtils.createConnection(config), serializer);
}
}
Loading