Skip to content
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

[FLINK-15571][connector][WIP] Redis Stream connector for Flink #2

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
138 changes: 138 additions & 0 deletions pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,138 @@
<?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.shaded.version>15.0</flink.shaded.version>
<target.java.version>1.8</target.java.version>
<slf4j.version>1.7.32</slf4j.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>3.8.0</jedis.version>
<slf4j.version>1.7.32</slf4j.version>
<testcontainers.version>1.16.2</testcontainers.version>
<junit.version>4.13.2</junit.version>
<hamcrest.version>1.3</hamcrest.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-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>

</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,127 @@
/*
* 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.streaming.connectors.redis;

import org.apache.flink.streaming.connectors.redis.config.StartupMode;
import redis.clients.jedis.Jedis;
import redis.clients.jedis.StreamEntry;
import redis.clients.jedis.StreamEntryID;

import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Properties;
import java.util.stream.Collectors;

/**
* @param <T>
*/
public abstract class AbstractRedisStreamConsumer<T> extends RedisConsumerBase<T> {

protected final Map<String, StreamEntryID> streamEntryIds;

public AbstractRedisStreamConsumer(
StartupMode startupMode, List<String> streamKeys, Properties configProps) {
super(streamKeys, configProps);
final StreamEntryID streamEntryID;
switch (startupMode) {
case EARLIEST:
streamEntryID = new StreamEntryID();
break;
case LATEST:
streamEntryID = StreamEntryID.LAST_ENTRY;
break;
case GROUP_OFFSETS:
streamEntryID = StreamEntryID.UNRECEIVED_ENTRY;
break;
case SPECIFIC_OFFSETS:
throw new RuntimeException(
"Use the constructor with 'StreamEntryID[] streamIds' as param");
case TIMESTAMP:
throw new RuntimeException("Use the constructor with 'Long[] timestamps' param");
default:
throw new IllegalStateException();
}
this.streamEntryIds = prepareStreamEntryIds(streamKeys, streamEntryID);
}

public AbstractRedisStreamConsumer(
List<String> streamKeys, List<StreamEntryID> streamIds, Properties configProps) {
this(prepareStreamEntryIds(streamKeys, streamIds), configProps);
}

private AbstractRedisStreamConsumer(
Map<String, StreamEntryID> streamIds, Properties configProps) {
super(null, configProps);
this.streamEntryIds = streamIds;
}

@Override
protected final boolean readAndCollect(
Jedis jedis, List<String> streamKeys, SourceContext<T> sourceContext) {
boolean anyEntry = false;
List<Entry<String, List<StreamEntry>>> response = read(jedis);
if (response != null) {
for (Entry<String, List<StreamEntry>> streamEntries : response) {
String streamKey = streamEntries.getKey();
for (StreamEntry entry : streamEntries.getValue()) {
anyEntry = true;
collect(sourceContext, streamKey, entry);
updateIdForKey(streamKey, entry.getID());
}
}
}
return anyEntry;
}

protected abstract List<Entry<String, List<StreamEntry>>> read(Jedis jedis);

protected abstract void collect(
SourceContext<T> sourceContext, String streamKey, StreamEntry streamEntry);

protected void updateIdForKey(String streamKey, StreamEntryID streamEntryID) {
if (this.streamEntryIds.get(streamKey).toString().equals(">")) {
// skip
} else {
this.streamEntryIds.put(streamKey, streamEntryID);
}
}

private static Map<String, StreamEntryID> prepareStreamEntryIds(
List<String> streamKeys, StreamEntryID streamId) {
Map<String, StreamEntryID> streams = new LinkedHashMap<>(streamKeys.size());
streamKeys.forEach(streamKey -> streams.put(streamKey, streamId));
return streams;
}

private static Map<String, StreamEntryID> prepareStreamEntryIds(
List<String> streamKeys, List<StreamEntryID> streamIds) {
Map<String, StreamEntryID> streams = new LinkedHashMap<>(streamKeys.size());
for (int i = 0; i < streamKeys.size(); i++) {
streams.put(streamKeys.get(i), streamIds.get(i));
}
return streams;
}

public static List<StreamEntryID> convertToStreamEntryIDs(List<Long> timestamps) {
return timestamps.stream()
.map(ts -> new StreamEntryID(ts, 0L))
.collect(Collectors.toList());
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
/*
* 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.streaming.connectors.redis;

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

/** @param <OUT> */
public interface DataConverter<OUT> extends Serializable {

OUT toData(Map<String, String> input);
}
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.streaming.connectors.redis;

import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.state.FunctionInitializationContext;
import org.apache.flink.runtime.state.FunctionSnapshotContext;
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
import org.apache.flink.streaming.connectors.redis.util.JedisUtils;

import redis.clients.jedis.Jedis;

import java.util.Properties;

import static org.apache.flink.util.Preconditions.checkNotNull;

/** @param <OUT> */
public abstract class FlinkRedisProducerBase<OUT> extends RichSinkFunction<OUT>
implements CheckpointedFunction {

private final Properties configProps;

private transient Jedis jedis;

private final String key;

public FlinkRedisProducerBase(String key, Properties configProps) {
checkNotNull(key, "key can not be null");
this.key = key;

checkNotNull(configProps, "configProps can not be null");
this.configProps = configProps;
}

/** Initializes the connection to Redis. */
@Override
public void open(Configuration configuration) throws Exception {
super.open(configuration);
// code
this.jedis = JedisUtils.createResource(this.configProps);
}

@Override
public void close() throws Exception {
super.close();
// code
if (jedis != null) {
jedis.close();
}
}

@Override
public void invoke(OUT value, Context context) throws Exception {
invoke(this.jedis, this.key, value, context);
}

protected abstract void invoke(Jedis resource, String key, OUT value, Context context)
throws Exception;

@Override
public void snapshotState(FunctionSnapshotContext context) {
// in synchronous mode, nothing to do
}

@Override
public void initializeState(FunctionInitializationContext context) {
// nothing to do
}
}
Loading