Skip to content

[FLINK-15571] Add Redis sink #3

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 3 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
58 changes: 58 additions & 0 deletions docs/content/docs/connectors/datastream/redis.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
---
title: Redis
weight: 5
type: docs
aliases:
---
<!--
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.
-->

# Redis Connector

This connector provides sinks that can request document actions to an
[Redis](https://redis.io/). To use this connector, add the following
dependencies to your project:

<table class="table table-bordered">
<thead>
<tr>
<th class="text-left">Redis version</th>
<th class="text-left">Maven Dependency</th>
</tr>
</thead>
<tbody>
<tr>
<td>7.x</td>
<td>{{< connector_artifact flink-connector-redis-streams 3.0.0 >}}</td>
</tr>
</tbody>
</table>

{{< py_download_link "redis" >}}

Note that the streaming connectors are currently not part of the binary
distribution. See [here]({{< ref "docs/dev/configuration/overview" >}}) for information
about how to package the program with the libraries for cluster execution.

## Installing Redis

Instructions for setting up a Redis cluster can be found
[here](https://redis.io/docs/getting-started/installation/).


36 changes: 36 additions & 0 deletions flink-connector-redis-streams/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
<?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">
<parent>
<artifactId>flink-connector-redis-parent</artifactId>
<groupId>org.apache.flink</groupId>
<version>3.0-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>

<artifactId>flink-connector-redis-streams</artifactId>
<name>Flink : Connectors : Redis : Streams</name>

<packaging>jar</packaging>

<dependencies>
<dependency>
<groupId>redis.clients</groupId>
<artifactId>jedis</artifactId>
</dependency>

<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>
</dependency>
</dependencies>


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

import org.apache.flink.connector.redis.streams.sink.connection.JedisConnector;

import redis.clients.jedis.StreamEntryID;

import java.io.Serializable;
import java.util.Map;

/** A Redis Streams Command. */
public class RedisStreamsCommand implements Serializable {

private transient StreamEntryID streamId = null;
public final String key;
public final Map<String, String> value;

private RedisStreamsCommand(String key, Map<String, String> value) {
this.key = key;
this.value = value;
}

public static Builder builder() {
return new Builder();
}

public void send(JedisConnector connector) {
this.streamId =
connector
.getJedisCommands()
.xadd(
key,
(this.streamId != null) ? this.streamId : StreamEntryID.NEW_ENTRY,
value);
}

public boolean sendCorrectly() {
return true;
}

public boolean sendIncorrectly() {
return !sendCorrectly();
}

public long getMessageSize() {
return this.key.length()
+ this.value.entrySet().stream()
.map(k -> k.getKey().length() + k.getValue().length())
.reduce(Integer::sum)
.orElse(0);
}

/** The builder for {@link RedisStreamsCommand}. */
public static class Builder {
private String key;
private Map<String, String> value;

public Builder withKey(String key) {
this.key = key;
return this;
}

public Builder withValue(Map<String, String> value) {
this.value = value;
return this;
}

public RedisStreamsCommand build() {
return new RedisStreamsCommand(key, value);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
/*
* 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.streams.sink;

import org.apache.flink.connector.base.sink.writer.ElementConverter;

/**
* Function that creates the description how the input data should be mapped to redis type.
*
* @param <T> The type of the element handled by this {@code RedisSerializer}
*/
public interface RedisStreamsCommandSerializer<T>
extends ElementConverter<T, RedisStreamsCommand> {}
Original file line number Diff line number Diff line change
@@ -0,0 +1,85 @@
/*
* 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.streams.sink;

import org.apache.flink.connector.base.sink.AsyncSinkBase;
import org.apache.flink.connector.base.sink.writer.BufferedRequestState;
import org.apache.flink.connector.base.sink.writer.config.AsyncSinkWriterConfiguration;
import org.apache.flink.connector.redis.streams.sink.config.JedisConfig;
import org.apache.flink.connector.redis.streams.sink.connection.JedisConnector;
import org.apache.flink.connector.redis.streams.sink.connection.JedisConnectorBuilder;
import org.apache.flink.core.io.SimpleVersionedSerializer;

import java.io.IOException;
import java.util.Collection;
import java.util.Collections;

/**
* A sink for publishing data into Redis.
*
* @param <T>
*/
public class RedisStreamsSink<T> extends AsyncSinkBase<T, RedisStreamsCommand> {

private final JedisConfig jedisConfig;

public RedisStreamsSink(
JedisConfig jedisConfig,
RedisStreamsCommandSerializer<T> converter,
AsyncSinkWriterConfiguration asyncConfig) {
super(
converter,
asyncConfig.getMaxBatchSize(),
asyncConfig.getMaxInFlightRequests(),
asyncConfig.getMaxBufferedRequests(),
asyncConfig.getMaxBatchSizeInBytes(),
asyncConfig.getMaxTimeInBufferMS(),
asyncConfig.getMaxRecordSizeInBytes());
this.jedisConfig = jedisConfig;
}

@Override
public RedisStreamsWriter<T> createWriter(InitContext initContext) throws IOException {
return restoreWriter(initContext, Collections.emptyList());
}

@Override
public RedisStreamsWriter<T> restoreWriter(
InitContext initContext,
Collection<BufferedRequestState<RedisStreamsCommand>> recoveredState)
throws IOException {
AsyncSinkWriterConfiguration asyncConfig =
AsyncSinkWriterConfiguration.builder()
.setMaxBatchSize(getMaxBatchSize())
.setMaxBatchSizeInBytes(getMaxBatchSizeInBytes())
.setMaxInFlightRequests(getMaxInFlightRequests())
.setMaxBufferedRequests(getMaxBufferedRequests())
.setMaxTimeInBufferMS(getMaxTimeInBufferMS())
.setMaxRecordSizeInBytes(getMaxRecordSizeInBytes())
.build();
JedisConnector connection = JedisConnectorBuilder.build(jedisConfig);
return new RedisStreamsWriter<>(
connection, getElementConverter(), asyncConfig, initContext, recoveredState);
}

@Override
public SimpleVersionedSerializer<BufferedRequestState<RedisStreamsCommand>>
getWriterStateSerializer() {
return new RedisStreamsStateSerializer();
}
}
Loading