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

This file was deleted.

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

import com.google.common.base.Preconditions;
import org.apache.storm.redis.common.mapper.RedisDataTypeDescription;
import org.apache.storm.redis.common.mapper.RedisLookupMapper;
import org.apache.storm.task.OutputCollector;
import org.apache.storm.tuple.Tuple;
import org.apache.storm.tuple.Values;
import redis.clients.jedis.JedisCommands;

import java.util.List;

public abstract class BaseLookupBolt extends BaseRedisBolt {
protected OutputCollector collector;

protected final RedisLookupMapper mapper;
protected final RedisDataTypeDescription.RedisDataType dataType;
protected final String additionalKey;

public BaseLookupBolt(RedisLookupMapper lookupMapper) {
this.mapper = lookupMapper;
Preconditions.checkNotNull(this.mapper, "Mapper is Null");

RedisDataTypeDescription dataTypeDescription = lookupMapper.getDataTypeDescription();
this.dataType = dataTypeDescription.getDataType();
this.additionalKey = dataTypeDescription.getAdditionalKey();
}

@Override
public void execute(Tuple input) {
String key = mapper.getKeyFromTuple(input);
Object lookupValue;

JedisCommands jedisCommand = getInstance();
try {
switch (dataType) {
case STRING:
lookupValue = jedisCommand.get(key);
break;

case LIST:
lookupValue = jedisCommand.lpop(key);
break;

case HASH:
lookupValue = jedisCommand.hget(additionalKey, key);
break;

case SET:
lookupValue = jedisCommand.scard(key);
break;

case SORTED_SET:
lookupValue = jedisCommand.zscore(additionalKey, key);
break;

case HYPER_LOG_LOG:
lookupValue = jedisCommand.pfcount(key);
break;

case GEO:
lookupValue = jedisCommand.geopos(additionalKey, key);
break;

default:
throw new IllegalArgumentException("Cannot process such data type: " + dataType);
}

List<Values> values = mapper.toTuple(input, lookupValue);
for (Values value : values) {
collector.emit(input, value);
}

collector.ack(input);
} catch (Exception e) {
this.collector.reportError(e);
this.collector.fail(input);
} finally {
returnInstance(jedisCommand);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
/**
* 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.storm.redis.bolt;

import org.apache.storm.redis.common.container.JedisCommandsInstanceContainer;
import org.apache.storm.topology.base.BaseRichBolt;
import redis.clients.jedis.JedisCommands;

public abstract class BaseRedisBolt extends BaseRichBolt {
protected transient JedisCommandsInstanceContainer container;

protected JedisCommands getInstance() {
return this.container.getInstance();
}

protected void returnInstance(JedisCommands instance) {
this.container.returnInstance(instance);
}
}
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.
*/
package org.apache.storm.redis.bolt;

import com.google.common.base.Preconditions;
import org.apache.storm.redis.common.mapper.RedisDataTypeDescription;
import org.apache.storm.redis.common.mapper.RedisStoreMapper;
import org.apache.storm.task.OutputCollector;
import org.apache.storm.tuple.Tuple;
import redis.clients.jedis.JedisCommands;

public abstract class BaseStoreBolt extends BaseRedisBolt {
protected OutputCollector collector;

protected final RedisStoreMapper mapper;
protected final RedisDataTypeDescription.RedisDataType dataType;
protected final String additionalKey;

public BaseStoreBolt(RedisStoreMapper mapper) {
this.mapper = mapper;
Preconditions.checkNotNull(this.mapper, "Mapper is Null");

RedisDataTypeDescription dataTypeDescription = mapper.getDataTypeDescription();
this.dataType = dataTypeDescription.getDataType();
this.additionalKey = dataTypeDescription.getAdditionalKey();
}

@Override
public void execute(Tuple input) {
String key = mapper.getKeyFromTuple(input);
String value = mapper.getValueFromTuple(input);

JedisCommands jedisCommand = null;
try {
jedisCommand = getInstance();

switch (dataType) {
case STRING:
jedisCommand.set(key, value);
break;

case LIST:
jedisCommand.rpush(key, value);
break;

case HASH:
jedisCommand.hset(additionalKey, key, value);
break;

case SET:
jedisCommand.sadd(key, value);
break;

case SORTED_SET:
jedisCommand.zadd(additionalKey, Double.valueOf(value), key);
break;

case HYPER_LOG_LOG:
jedisCommand.pfadd(key, value);
break;

case GEO:
String[] array = value.split(":");
if (array.length != 2) {
throw new IllegalArgumentException("value structure should be longitude:latitude");
}

double longitude = Double.valueOf(array[0]);
double latitude = Double.valueOf(array[1]);
jedisCommand.geoadd(additionalKey, longitude, latitude, key);
break;

default:
throw new IllegalArgumentException("Cannot process such data type: " + dataType);
}

collector.ack(input);
} catch (Exception e) {
this.collector.reportError(e);
this.collector.fail(input);
} finally {
returnInstance(jedisCommand);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
/**
* 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.storm.redis.bolt;

import org.apache.storm.redis.common.config.JedisClusterConfig;
import org.apache.storm.redis.common.container.JedisCommandsContainerBuilder;
import org.apache.storm.redis.common.mapper.RedisLookupMapper;
import org.apache.storm.task.OutputCollector;
import org.apache.storm.task.TopologyContext;
import org.apache.storm.topology.OutputFieldsDeclarer;

import java.util.Map;

public class RedisClusterLookupBolt extends BaseLookupBolt {

private final JedisClusterConfig config;

public RedisClusterLookupBolt(JedisClusterConfig config, RedisLookupMapper lookupMapper) {
super(lookupMapper);
this.config = config;
}

@Override
public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
this.collector = collector;

if (this.config != null) {
this.container = JedisCommandsContainerBuilder.buildClusterContainer(config);
} else {
throw new IllegalArgumentException("Jedis configuration not found");
}
}

@Override
public void declareOutputFields(OutputFieldsDeclarer declarer) {
mapper.declareOutputFields(declarer);
}
}
Loading