Skip to content

Commit

Permalink
[pulsar-io] Add a Pulsar IO connector for Redis sink (apache#3700)
Browse files Browse the repository at this point in the history
### Motivation

This PR provides a built-in Redis sink Connector, in order to cache messages in Redis [key-value] pairs. This will effectively make Redis a caching system, which other applications can access to get the latest value.

### Modifications

Add a new sub-module in the `pulsar-io` module.

### Verifying this change

This change can be verified as follows:

* deploy the redis sink connector with configuration file containing the following fields:

```
configs:
    redisHosts: "localhost:6379"
    redisPassword: "redis@123"
    redisDatabase: "1"
    clientMode: "Standalone"
    operationTimeout: "3000"
    batchSize: "100"
```
* start a redis instance with auth
* send messages with `NotNull` key/value in the topic declared when deploying the connector
* check in Redis if the message's key-value pairs have been stored in above database

### Documentation
```
# Submit a Redis Sink
$ bin/pulsar-admin sink create --tenant public --namespace default --name redis-test-sink --sink-type redis --sink-config-file examples/redis-sink.yaml --inputs test_redis
    
# List Sink
$ bin/pulsar-admin sink list --tenant public --namespace default
    
# Get Sink Info
$ bin/pulsar-admin sink get --tenant public --namespace default --name redis-test-sink
    
# Get Sink Status
$ bin/pulsar-admin sink status --tenant public --namespace default --name redis-test-sink
    
# Delete the Redis Sink
$ bin/pulsar-admin sink delete --tenant public --namespace default --name redis-test-sink
```
  • Loading branch information
murong00 authored and sijie committed Mar 12, 2019
1 parent ff4db8d commit c5e3baa
Show file tree
Hide file tree
Showing 11 changed files with 1,002 additions and 0 deletions.
1 change: 1 addition & 0 deletions pulsar-io/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,7 @@
<module>hbase</module>
<module>mongo</module>
<module>flume</module>
<module>redis</module>
</modules>

</project>
100 changes: 100 additions & 0 deletions pulsar-io/redis/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,100 @@
<!--
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.
-->
<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>
<parent>
<artifactId>pulsar-io</artifactId>
<groupId>org.apache.pulsar</groupId>
<version>2.4.0-SNAPSHOT</version>
</parent>

<artifactId>pulsar-io-redis</artifactId>
<name>Pulsar IO :: Redis</name>

<dependencies>
<dependency>
<groupId>${project.parent.groupId}</groupId>
<artifactId>pulsar-io-core</artifactId>
<version>${project.parent.version}</version>
</dependency>
<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>pulsar-functions-instance</artifactId>
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>pulsar-client-original</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>io.lettuce</groupId>
<artifactId>lettuce-core</artifactId>
<version>5.0.2.RELEASE</version>
</dependency>
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.dataformat</groupId>
<artifactId>jackson-dataformat-yaml</artifactId>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId>
<version>3.4</version>
</dependency>
<dependency>
<groupId>commons-collections</groupId>
<artifactId>commons-collections</artifactId>
<version>3.2.2</version>
</dependency>
<dependency>
<groupId>${project.parent.groupId}</groupId>
<artifactId>buildtools</artifactId>
<version>${project.parent.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.github.kstyrc</groupId>
<artifactId>embedded-redis</artifactId>
<version>0.6</version>
<scope>test</scope>
</dependency>
</dependencies>

<build>
<plugins>
<plugin>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-nar-maven-plugin</artifactId>
</plugin>
</plugins>
</build>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,124 @@
/**
* 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.pulsar.io.redis;

import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.net.HostAndPort;
import lombok.Data;
import lombok.EqualsAndHashCode;
import lombok.Getter;
import lombok.Setter;
import lombok.ToString;
import lombok.experimental.Accessors;
import org.apache.commons.lang3.StringUtils;
import org.apache.pulsar.io.core.annotations.FieldDoc;

import java.io.Serializable;
import java.util.List;

/**
* Configuration object for all Redis Sink components.
*/
@Data
@Setter
@Getter
@EqualsAndHashCode
@ToString
@Accessors(chain = true)
public class RedisAbstractConfig implements Serializable {

private static final long serialVersionUID = -7860917032537872317L;

@FieldDoc(
required = true,
defaultValue = "",
help = "A comma separated list of Redis hosts to connect to")
private String redisHosts;

@FieldDoc(
required = false,
defaultValue = "",
help = "The password used to connect to Redis")
private String redisPassword;

@FieldDoc(
required = true,
defaultValue = "0",
help = "The Redis database to connect to")
private int redisDatabase = 0;

@FieldDoc(
required = false,
defaultValue = "Standalone",
help = "The client mode to use when interacting with the Redis cluster. Possible values [Standalone, Cluster]")
private String clientMode = "Standalone";

@FieldDoc(
required = false,
defaultValue = "true",
help = "Flag to determine if the Redis client should automatically reconnect")
private boolean autoReconnect = true;

@FieldDoc(
required = false,
defaultValue = "2147483647",
help = "The maximum number of queued requests to Redis")
private int requestQueue = 2147483647;

@FieldDoc(
required = false,
defaultValue = "false",
help = "Flag to enable TCP no delay should be used")
private boolean tcpNoDelay = false;

@FieldDoc(
required = false,
defaultValue = "false",
help = "Flag to enable a keepalive to Redis")
private boolean keepAlive = false;

@FieldDoc(
required = false,
defaultValue = "10000L",
help = "The amount of time in milliseconds to wait before timing out when connecting")
private long connectTimeout = 10000L;

public void validate() {
Preconditions.checkNotNull(redisHosts, "redisHosts property not set.");
Preconditions.checkNotNull(redisDatabase, "redisDatabase property not set.");
Preconditions.checkNotNull(clientMode, "clientMode property not set.");
}

public enum ClientMode {
STANDALONE,
CLUSTER
}

public List<HostAndPort> getHostAndPorts() {
List<HostAndPort> hostAndPorts = Lists.newArrayList();;
Preconditions.checkNotNull(redisHosts, "redisHosts property not set.");
String[] hosts = StringUtils.split(redisHosts, ",");
for (String host : hosts) {
HostAndPort hostAndPort = HostAndPort.fromString(host);
hostAndPorts.add(hostAndPort);
}
return hostAndPorts;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,140 @@
/**
* 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.pulsar.io.redis;

import com.google.common.collect.Lists;
import com.google.common.net.HostAndPort;
import io.lettuce.core.AbstractRedisClient;
import io.lettuce.core.ClientOptions;
import io.lettuce.core.RedisClient;
import io.lettuce.core.RedisURI;
import io.lettuce.core.SocketOptions;
import io.lettuce.core.api.StatefulConnection;
import io.lettuce.core.api.StatefulRedisConnection;
import io.lettuce.core.cluster.ClusterClientOptions;
import io.lettuce.core.cluster.RedisClusterClient;
import io.lettuce.core.cluster.api.StatefulRedisClusterConnection;
import io.lettuce.core.cluster.api.async.RedisClusterAsyncCommands;
import io.lettuce.core.codec.ByteArrayCodec;
import io.lettuce.core.codec.RedisCodec;
import org.apache.commons.lang3.StringUtils;
import org.apache.pulsar.io.redis.RedisAbstractConfig.ClientMode;
import org.apache.pulsar.io.redis.sink.RedisSinkConfig;

import java.time.Duration;
import java.util.Arrays;
import java.util.List;

public class RedisSession {

private final AbstractRedisClient client;
private final StatefulConnection connection;
private final RedisClusterAsyncCommands<byte[], byte[]> asyncCommands;

public RedisSession(AbstractRedisClient client, StatefulConnection connection, RedisClusterAsyncCommands<byte[], byte[]> asyncCommands) {
this.client = client;
this.connection = connection;
this.asyncCommands = asyncCommands;
}

public AbstractRedisClient client() {
return this.client;
}

public StatefulConnection connection() {
return this.connection;
}

public RedisClusterAsyncCommands<byte[], byte[]> asyncCommands() {
return this.asyncCommands;
}

public void close() throws Exception {
if (null != this.connection) {
this.connection.close();
}
if (null != this.client) {
this.client.shutdown();
}
}

public static RedisSession create(RedisSinkConfig config) {
RedisSession redisSession;
final RedisCodec<byte[], byte[]> codec = new ByteArrayCodec();

final SocketOptions socketOptions = SocketOptions.builder()
.tcpNoDelay(config.isTcpNoDelay())
.connectTimeout(Duration.ofMillis(config.getConnectTimeout()))
.keepAlive(config.isKeepAlive())
.build();

final ClientMode clientMode;
try {
clientMode = ClientMode.valueOf(config.getClientMode().toUpperCase());
} catch (IllegalArgumentException e) {
throw new IllegalArgumentException("Illegal Redis client mode, valid values are: "
+ Arrays.asList(ClientMode.values()));
}

List<RedisURI> redisURIs = redisURIs(config.getHostAndPorts(), config);

if (clientMode == ClientMode.STANDALONE) {
ClientOptions.Builder clientOptions = ClientOptions.builder()
.socketOptions(socketOptions)
.requestQueueSize(config.getRequestQueue())
.autoReconnect(config.isAutoReconnect());

final RedisClient client = RedisClient.create(redisURIs.get(0));
client.setOptions(clientOptions.build());
final StatefulRedisConnection<byte[], byte[]> connection = client.connect(codec);
redisSession = new RedisSession(client, connection, connection.async());
} else if (clientMode == ClientMode.CLUSTER) {
ClusterClientOptions.Builder clientOptions = ClusterClientOptions.builder()
.requestQueueSize(config.getRequestQueue())
.autoReconnect(config.isAutoReconnect());

final RedisClusterClient client = RedisClusterClient.create(redisURIs);
client.setOptions(clientOptions.build());

final StatefulRedisClusterConnection<byte[], byte[]> connection = client.connect(codec);
redisSession = new RedisSession(client, connection, connection.async());
} else {
throw new UnsupportedOperationException(
String.format("%s is not supported", config.getClientMode())
);
}

return redisSession;
}

private static List<RedisURI> redisURIs(List<HostAndPort> hostAndPorts, RedisSinkConfig config) {
List<RedisURI> redisURIs = Lists.newArrayList();
for (HostAndPort hostAndPort : hostAndPorts) {
RedisURI.Builder builder = RedisURI.builder();
builder.withHost(hostAndPort.getHost());
builder.withPort(hostAndPort.getPort());
builder.withDatabase(config.getRedisDatabase());
if (!StringUtils.isBlank(config.getRedisPassword())) {
builder.withPassword(config.getRedisPassword());
}
redisURIs.add(builder.build());
}
return redisURIs;
}
}
Loading

0 comments on commit c5e3baa

Please sign in to comment.