Skip to content
This repository was archived by the owner on Feb 16, 2024. It is now read-only.
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
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,7 @@ public List<String> supportedProperties() {
properties.add(REDIS_CLUSTER_PASSWORD);
properties.add(REDIS_SENTINEL);
properties.add(REDIS_KEY_TTL);
properties.add(REDIS_KEY_ADDITIONAL);
// schema
properties.add(SCHEMA + ".#." + SCHEMA_DATA_TYPE);
properties.add(SCHEMA + ".#." + SCHEMA_NAME);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import java.lang.reflect.Constructor;
import java.util.Map;

import static org.apache.flink.streaming.connectors.redis.descriptor.RedisValidator.REDIS_KEY_ADDITIONAL;
import static org.apache.flink.streaming.connectors.redis.descriptor.RedisValidator.REDIS_KEY_TTL;

/**
Expand All @@ -40,14 +41,21 @@ public interface RedisMapperHandler extends RedisHandler {
*/
default RedisMapper createRedisMapper(Map<String, String> properties) {
String ttl = properties.get(REDIS_KEY_TTL);
String additionalKey = properties.get(REDIS_KEY_ADDITIONAL);
try {
Class redisMapper = Class.forName(this.getClass().getCanonicalName());

if (ttl == null) {
return (RedisMapper) redisMapper.newInstance();
if (ttl != null) {
Constructor c = redisMapper.getConstructor(Integer.class);
return (RedisMapper) c.newInstance(Integer.parseInt(ttl));
}
Constructor c = redisMapper.getConstructor(Integer.class);
return (RedisMapper) c.newInstance(Integer.parseInt(ttl));

if (additionalKey != null) {
Constructor c = redisMapper.getConstructor(String.class);
return (RedisMapper) c.newInstance(additionalKey);
}

return (RedisMapper) redisMapper.newInstance();
} catch (Exception e) {
LOGGER.error("create redis mapper failed", e);
throw new RuntimeException(e);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,8 @@ public abstract class RowRedisMapper implements RedisMapper<Tuple2<Boolean, Row>

private Integer ttl;

private String additionalKey;

private RedisCommand redisCommand;

public Integer getTtl() {
Expand All @@ -67,6 +69,11 @@ public RowRedisMapper(int ttl, RedisCommand redisCommand) {
this.redisCommand = redisCommand;
}

public RowRedisMapper(String additionalKey, RedisCommand redisCommand) {
this.additionalKey = additionalKey;
this.redisCommand = redisCommand;
}

public RowRedisMapper(RedisCommand redisCommand) {
this.redisCommand = redisCommand;
}
Expand All @@ -76,6 +83,9 @@ public RedisCommandDescription getCommandDescription() {
if (ttl != null) {
return new RedisCommandDescription(redisCommand, ttl);
}
if (additionalKey != null) {
return new RedisCommandDescription(redisCommand, additionalKey);
}
return new RedisCommandDescription(redisCommand);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,4 +28,8 @@ public class ZAddMapper extends RowRedisMapper {
public ZAddMapper() {
super(RedisCommand.ZADD);
}

public ZAddMapper(String additionalKey) {
super(additionalKey, RedisCommand.ZADD);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,4 +28,7 @@ public class ZIncrByMapper extends RowRedisMapper {
public ZIncrByMapper() {
super(RedisCommand.ZINCRBY);
}
public ZIncrByMapper(String additionalKey) {
super(additionalKey, RedisCommand.ZINCRBY);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -32,5 +32,6 @@ public class RedisValidator {
public static final String SENTINELS_PASSWORD = "sentinels.password";
public static final String REDIS_CLUSTER_PASSWORD = "cluster.password";
public static final String REDIS_KEY_TTL = "key.ttl";
public static final String REDIS_KEY_ADDITIONAL = "key.additional";

}
Original file line number Diff line number Diff line change
Expand Up @@ -25,5 +25,6 @@ org.apache.flink.streaming.connectors.redis.common.mapper.row.RPushMapper
org.apache.flink.streaming.connectors.redis.common.mapper.row.SAddMapper
org.apache.flink.streaming.connectors.redis.common.mapper.row.SetMapper
org.apache.flink.streaming.connectors.redis.common.mapper.row.ZAddMapper
org.apache.flink.streaming.connectors.redis.common.mapper.row.ZIncrByMapper
org.apache.flink.streaming.connectors.redis.common.config.handler.FlinkJedisClusterConfigHandler
org.apache.flink.streaming.connectors.redis.common.config.handler.FlinkJedisSentinelConfigHandler
Original file line number Diff line number Diff line change
@@ -0,0 +1,87 @@
/*
* 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.common.mapper.row;

import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.typeutils.RowTypeInfo;
import org.apache.flink.streaming.api.datastream.DataStreamSource;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.streaming.connectors.redis.RedisITCaseBase;
import org.apache.flink.table.api.EnvironmentSettings;
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
import org.apache.flink.types.Row;
import org.junit.Before;
import org.junit.Test;

public class RedisTableSinkZIncrByTest extends RedisITCaseBase {
private static final String REDIS_KEY = "TEST_KEY";

StreamExecutionEnvironment env;

@Before
public void setUp(){
env = StreamExecutionEnvironment.getExecutionEnvironment();
}

@Test
public void testRedisDescriptor() throws Exception {
DataStreamSource<Row> source = (DataStreamSource<Row>) env.addSource(new RedisTableSinkZIncrByTest.TestSourceFunctionString())
.returns(new RowTypeInfo(TypeInformation.of(String.class), TypeInformation.of(Long.class)));

EnvironmentSettings settings = EnvironmentSettings
.newInstance()
.inStreamingMode()
.build();
StreamTableEnvironment tableEnvironment = StreamTableEnvironment.create(env, settings);
tableEnvironment.registerDataStream("t1", source, "k, v");

tableEnvironment.executeSql("create table redis " +
"(k string, " +
"v bigint) " +
"with (" +
"'connector.type'='redis'," +
"'redis-mode'='cluster'," +
"'cluster-nodes'='"+String.format("%s:%s",REDIS_HOST, REDIS_PORT)+"'," +
"'command'='ZINCRBY'," +
"'key.additional'='ADDITIONAL')");

tableEnvironment.executeSql("insert into redis select k, v from t1");
}

private static class TestSourceFunctionString implements SourceFunction<Row> {
private static final long serialVersionUID = 1L;

private volatile boolean running = true;

@Override
public void run(SourceContext<Row> ctx) throws Exception {
while (running) {
Row row = new Row(2);
row.setField(0, REDIS_KEY);
row.setField(1, 2L);
ctx.collect(row);
Thread.sleep(2000L);
}
}

@Override
public void cancel() {
running = false;
}
}
}