You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@storm.apache.org by HeartSaVioR <gi...@git.apache.org> on 2016/06/24 11:54:35 UTC

[GitHub] storm pull request #1517: STORM-1919 Introduce FilterBolt on storm-redis

GitHub user HeartSaVioR opened a pull request:

    https://github.com/apache/storm/pull/1517

    STORM-1919 Introduce FilterBolt on storm-redis

    * introduce RedisFilterBolt, and relevant class (RedisFilterMapper)
    * add example topology: WhitelistWordCount
    * update how to use to README.md
    * also correct some javadocs
    
    It should be easy to backport to 1.x-branch as well so I didn't create another pull request against 1.x-branch.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/HeartSaVioR/storm STORM-1919

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/storm/pull/1517.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1517
    
----
commit 5e72b7c90f6e157ad7e5a194697547dbf3b22fee
Author: Jungtaek Lim <ka...@gmail.com>
Date:   2016-06-24T11:50:49Z

    STORM-1919 Introduce FilterBolt on storm-redis
    
    * introduce RedisFilterBolt, and relevant class (RedisFilterMapper)
    * add example topology: WhitelistWordCount
    * update how to use to README.md
    * also correct some javadocs

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1517: STORM-1919 Introduce FilterBolt on storm-redis

Posted by abhishekagarwal87 <gi...@git.apache.org>.
Github user abhishekagarwal87 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1517#discussion_r68916823
  
    --- Diff: external/storm-redis/src/main/java/org/apache/storm/redis/bolt/RedisFilterBolt.java ---
    @@ -0,0 +1,146 @@
    +/**
    + * 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.config.JedisPoolConfig;
    +import org.apache.storm.redis.common.mapper.RedisDataTypeDescription;
    +import org.apache.storm.redis.common.mapper.RedisFilterMapper;
    +import org.apache.storm.topology.OutputFieldsDeclarer;
    +import org.apache.storm.tuple.Tuple;
    +import redis.clients.jedis.GeoCoordinate;
    +import redis.clients.jedis.JedisCommands;
    +
    +import java.util.List;
    +
    +/**
    + * Basic bolt for querying from Redis and filters out if key/field doesn't exist.
    + * If key/field exists on Redis, this bolt just forwards input tuple to default stream.
    + * <p/>
    + * Supported data types: STRING, HASH, SET, SORTED_SET, HYPER_LOG_LOG, GEO.
    + * <p/>
    + * Note: For STRING it checks such key exists on the key space.
    + * For HASH and SORTED_SET and GEO, it checks such field exists on that data structure.
    + * For SET and HYPER_LOG_LOG, it check such value exists on that data structure.
    + * (Note that it still refers key from tuple via RedisFilterMapper#getKeyFromTuple())
    + * In order to apply checking this to SET, you need to input additional key this case.
    + * <p/>
    + * Note2: If you want to just query about existence of key regardless of actual data type,
    + * specify STRING to data type of RedisFilterMapper.
    + */
    +public class RedisFilterBolt extends AbstractRedisBolt {
    +    private final RedisFilterMapper filterMapper;
    +    private final RedisDataTypeDescription.RedisDataType dataType;
    +    private final String additionalKey;
    +
    +    /**
    +     * Constructor for single Redis environment (JedisPool)
    +     * @param config configuration for initializing JedisPool
    +     * @param filterMapper mapper containing which datatype, query key that Bolt uses
    +     */
    +    public RedisFilterBolt(JedisPoolConfig config, RedisFilterMapper filterMapper) {
    +        super(config);
    +
    +        this.filterMapper = filterMapper;
    +
    +        RedisDataTypeDescription dataTypeDescription = filterMapper.getDataTypeDescription();
    +        this.dataType = dataTypeDescription.getDataType();
    +        this.additionalKey = dataTypeDescription.getAdditionalKey();
    +    }
    +
    +    /**
    +     * Constructor for Redis Cluster environment (JedisCluster)
    +     * @param config configuration for initializing JedisCluster
    +     * @param filterMapper mapper containing which datatype, query key that Bolt uses
    +     */
    +    public RedisFilterBolt(JedisClusterConfig config, RedisFilterMapper filterMapper) {
    +        super(config);
    +
    +        this.filterMapper = filterMapper;
    +
    +        RedisDataTypeDescription dataTypeDescription = filterMapper.getDataTypeDescription();
    +        this.dataType = dataTypeDescription.getDataType();
    +        this.additionalKey = dataTypeDescription.getAdditionalKey();
    +    }
    +
    +    /**
    +     * {@inheritDoc}
    +     */
    +    @Override
    +    public void execute(Tuple input) {
    +        String key = filterMapper.getKeyFromTuple(input);
    +
    +        boolean found;
    +        JedisCommands jedisCommand = null;
    +        try {
    +            jedisCommand = getInstance();
    +
    +            switch (dataType) {
    +                case STRING:
    +                    found = jedisCommand.exists(key);
    +                    break;
    +
    +                case SET:
    +                    if (additionalKey == null) {
    +                        throw new IllegalArgumentException("additionalKey should be defined");
    +                    }
    +                    found = jedisCommand.sismember(additionalKey, key);
    +                    break;
    +
    +                case HASH:
    +                    found = jedisCommand.hexists(additionalKey, key);
    +                    break;
    +
    +                case SORTED_SET:
    +                    found = jedisCommand.zrank(additionalKey, key) != null;
    +                    break;
    +
    +                case HYPER_LOG_LOG:
    +                    found = jedisCommand.pfcount(key) > 0;
    +                    break;
    +
    +                case GEO:
    +                    List<GeoCoordinate> geopos = jedisCommand.geopos(additionalKey, key);
    +                    found = geopos != null && geopos.size() > 0;
    --- End diff --
    
    minor. Better to wrap in parentheses for clarity
    ` found = (geopos != null && geopos.size() > 0)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1517: STORM-1919 Introduce FilterBolt on storm-redis

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1517#discussion_r68917641
  
    --- Diff: external/storm-redis/src/main/java/org/apache/storm/redis/bolt/RedisFilterBolt.java ---
    @@ -0,0 +1,146 @@
    +/**
    + * 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.config.JedisPoolConfig;
    +import org.apache.storm.redis.common.mapper.RedisDataTypeDescription;
    +import org.apache.storm.redis.common.mapper.RedisFilterMapper;
    +import org.apache.storm.topology.OutputFieldsDeclarer;
    +import org.apache.storm.tuple.Tuple;
    +import redis.clients.jedis.GeoCoordinate;
    +import redis.clients.jedis.JedisCommands;
    +
    +import java.util.List;
    +
    +/**
    + * Basic bolt for querying from Redis and filters out if key/field doesn't exist.
    + * If key/field exists on Redis, this bolt just forwards input tuple to default stream.
    + * <p/>
    + * Supported data types: STRING, HASH, SET, SORTED_SET, HYPER_LOG_LOG, GEO.
    + * <p/>
    + * Note: For STRING it checks such key exists on the key space.
    + * For HASH and SORTED_SET and GEO, it checks such field exists on that data structure.
    + * For SET and HYPER_LOG_LOG, it check such value exists on that data structure.
    + * (Note that it still refers key from tuple via RedisFilterMapper#getKeyFromTuple())
    + * In order to apply checking this to SET, you need to input additional key this case.
    + * <p/>
    + * Note2: If you want to just query about existence of key regardless of actual data type,
    + * specify STRING to data type of RedisFilterMapper.
    + */
    +public class RedisFilterBolt extends AbstractRedisBolt {
    +    private final RedisFilterMapper filterMapper;
    +    private final RedisDataTypeDescription.RedisDataType dataType;
    +    private final String additionalKey;
    +
    +    /**
    +     * Constructor for single Redis environment (JedisPool)
    +     * @param config configuration for initializing JedisPool
    +     * @param filterMapper mapper containing which datatype, query key that Bolt uses
    +     */
    +    public RedisFilterBolt(JedisPoolConfig config, RedisFilterMapper filterMapper) {
    +        super(config);
    +
    +        this.filterMapper = filterMapper;
    +
    +        RedisDataTypeDescription dataTypeDescription = filterMapper.getDataTypeDescription();
    +        this.dataType = dataTypeDescription.getDataType();
    +        this.additionalKey = dataTypeDescription.getAdditionalKey();
    +    }
    +
    +    /**
    +     * Constructor for Redis Cluster environment (JedisCluster)
    +     * @param config configuration for initializing JedisCluster
    +     * @param filterMapper mapper containing which datatype, query key that Bolt uses
    +     */
    +    public RedisFilterBolt(JedisClusterConfig config, RedisFilterMapper filterMapper) {
    +        super(config);
    +
    +        this.filterMapper = filterMapper;
    +
    +        RedisDataTypeDescription dataTypeDescription = filterMapper.getDataTypeDescription();
    +        this.dataType = dataTypeDescription.getDataType();
    +        this.additionalKey = dataTypeDescription.getAdditionalKey();
    +    }
    +
    +    /**
    +     * {@inheritDoc}
    +     */
    +    @Override
    +    public void execute(Tuple input) {
    +        String key = filterMapper.getKeyFromTuple(input);
    +
    +        boolean found;
    +        JedisCommands jedisCommand = null;
    +        try {
    +            jedisCommand = getInstance();
    +
    +            switch (dataType) {
    +                case STRING:
    +                    found = jedisCommand.exists(key);
    +                    break;
    +
    +                case SET:
    +                    if (additionalKey == null) {
    +                        throw new IllegalArgumentException("additionalKey should be defined");
    +                    }
    +                    found = jedisCommand.sismember(additionalKey, key);
    +                    break;
    +
    +                case HASH:
    +                    found = jedisCommand.hexists(additionalKey, key);
    +                    break;
    +
    +                case SORTED_SET:
    +                    found = jedisCommand.zrank(additionalKey, key) != null;
    +                    break;
    +
    +                case HYPER_LOG_LOG:
    +                    found = jedisCommand.pfcount(key) > 0;
    +                    break;
    +
    +                case GEO:
    +                    List<GeoCoordinate> geopos = jedisCommand.geopos(additionalKey, key);
    +                    found = geopos != null && geopos.size() > 0;
    --- End diff --
    
    Good point. Will fix.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1517: STORM-1919 Introduce FilterBolt on storm-redis

Posted by vesense <gi...@git.apache.org>.
Github user vesense commented on the issue:

    https://github.com/apache/storm/pull/1517
  
    LGTM


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1517: STORM-1919 Introduce FilterBolt on storm-redis

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1517#discussion_r68879755
  
    --- Diff: external/storm-redis/README.md ---
    @@ -21,13 +21,16 @@ use it as a maven dependency:
     
     ### For normal Bolt
     
    -Storm-redis provides basic Bolt implementations, ```RedisLookupBolt``` and ```RedisStoreBolt```.
    +Storm-redis provides basic Bolt implementations, ```RedisLookupBolt``` and ```RedisStoreBolt```, and ```RedisFilterBolt```.
     
    -As name represents its usage, ```RedisLookupBolt``` retrieves value from Redis using key, and ```RedisStoreBolt``` stores key / value to Redis. One tuple will be matched to one key / value pair, and you can define match pattern to ```TupleMapper```.
    +As name represents its usage, ```RedisLookupBolt``` retrieves value from Redis using key, and ```RedisStoreBolt``` stores key / value to Redis, and ```RedisFilterBolt``` filters out tuple which key or field doesn't exist on Redis.
     
    -You can also choose data type from ```RedisDataTypeDescription``` to use. Please refer ```RedisDataTypeDescription.RedisDataType``` to see what data types are supported. In some data types (hash and sorted set), it requires additional key and converted key from tuple becomes element.
    +One tuple will be matched to one key / value pair, and you can define match pattern to ```TupleMapper```.
     
    -These interfaces are combined with ```RedisLookupMapper``` and ```RedisStoreMapper``` which fit ```RedisLookupBolt``` and ```RedisStoreBolt``` respectively.
    +You can also choose data type from ```RedisDataTypeDescription``` to use. Please refer ```RedisDataTypeDescription.RedisDataType``` to see what data types are supported. In some data types (hash and sorted set, and set if only RedisFilterBolt), it requires additional key and converted key from tuple becomes element.
    +
    +These interfaces are combined with ```RedisLookupMapper``` and ```RedisStoreMapper``` and ```RedisFilterMapper``` which fit ```RedisLookupBolt``` and ```RedisStoreBolt```, and ```RedisFilterBolt``` respectively.
    +(When you want to implement RedisFilterMapper, be sure to set declareOutputFields() to declare same fields to input stream, since FilterBolt forwards input tuples when they exist on Redis.)   
    --- End diff --
    
    @darionyaphet 
    Yeah right. It can be arranged but we would want to resolve this with backward-compatible way. Maybe adding @Deprecated for 1.x / 1.0.x and removing for master.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1517: STORM-1919 Introduce FilterBolt on storm-redis

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1517#discussion_r69050854
  
    --- Diff: external/storm-redis/src/test/java/org/apache/storm/redis/topology/WhitelistWordCount.java ---
    @@ -0,0 +1,155 @@
    +/**
    + * 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.topology;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.LocalCluster;
    +import org.apache.storm.StormSubmitter;
    +import org.apache.storm.redis.bolt.RedisFilterBolt;
    +import org.apache.storm.redis.common.config.JedisPoolConfig;
    +import org.apache.storm.redis.common.mapper.RedisDataTypeDescription;
    +import org.apache.storm.redis.common.mapper.RedisFilterMapper;
    +import org.apache.storm.task.OutputCollector;
    +import org.apache.storm.task.TopologyContext;
    +import org.apache.storm.topology.OutputFieldsDeclarer;
    +import org.apache.storm.topology.TopologyBuilder;
    +import org.apache.storm.topology.base.BaseRichBolt;
    +import org.apache.storm.tuple.Fields;
    +import org.apache.storm.tuple.ITuple;
    +import org.apache.storm.tuple.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Map;
    +import java.util.Random;
    +
    +public class WhitelistWordCount {
    --- End diff --
    
    @abhishekagarwal87 
    I'm OK for that, but I'd prefer to get this out of scope for pull request.
    This needs discussion to have a generic rule across all external modules rather than fix for only this module.
    Could you initiate the discussion to dev@ or file an issue? Thanks in advance!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1517: STORM-1919 Introduce FilterBolt on storm-redis

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/1517
  
    @darionyaphet Could you check if this bolt covers your origin intention? Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1517: STORM-1919 Introduce FilterBolt on storm-redis

Posted by darionyaphet <gi...@git.apache.org>.
Github user darionyaphet commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1517#discussion_r68543199
  
    --- Diff: external/storm-redis/README.md ---
    @@ -21,13 +21,16 @@ use it as a maven dependency:
     
     ### For normal Bolt
     
    -Storm-redis provides basic Bolt implementations, ```RedisLookupBolt``` and ```RedisStoreBolt```.
    +Storm-redis provides basic Bolt implementations, ```RedisLookupBolt``` and ```RedisStoreBolt```, and ```RedisFilterBolt```.
     
    -As name represents its usage, ```RedisLookupBolt``` retrieves value from Redis using key, and ```RedisStoreBolt``` stores key / value to Redis. One tuple will be matched to one key / value pair, and you can define match pattern to ```TupleMapper```.
    +As name represents its usage, ```RedisLookupBolt``` retrieves value from Redis using key, and ```RedisStoreBolt``` stores key / value to Redis, and ```RedisFilterBolt``` filters out tuple which key or field doesn't exist on Redis.
     
    -You can also choose data type from ```RedisDataTypeDescription``` to use. Please refer ```RedisDataTypeDescription.RedisDataType``` to see what data types are supported. In some data types (hash and sorted set), it requires additional key and converted key from tuple becomes element.
    +One tuple will be matched to one key / value pair, and you can define match pattern to ```TupleMapper```.
     
    -These interfaces are combined with ```RedisLookupMapper``` and ```RedisStoreMapper``` which fit ```RedisLookupBolt``` and ```RedisStoreBolt``` respectively.
    +You can also choose data type from ```RedisDataTypeDescription``` to use. Please refer ```RedisDataTypeDescription.RedisDataType``` to see what data types are supported. In some data types (hash and sorted set, and set if only RedisFilterBolt), it requires additional key and converted key from tuple becomes element.
    +
    +These interfaces are combined with ```RedisLookupMapper``` and ```RedisStoreMapper``` and ```RedisFilterMapper``` which fit ```RedisLookupBolt``` and ```RedisStoreBolt```, and ```RedisFilterBolt``` respectively.
    +(When you want to implement RedisFilterMapper, be sure to set declareOutputFields() to declare same fields to input stream, since FilterBolt forwards input tuples when they exist on Redis.)   
    --- End diff --
    
    Actually  `RedisFilterMapper` and `RedisLookupMapper` are the same . It get a key from tuple then use to query from `Redis` . The difference is `RedisFilterBolt` use `RedisFilterMapper` and `RedisLookupBolt` use `RedisLookupMapper` . Should the tow mappers could extends from one basic abstract class ? 
    
    `RedisFilterMapper` and `RedisLookupBolt` will not use getValueFromTuple() , this method will always return `null`. So maybe I will open another ticket to discuss Mapper  Structure .


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1517: STORM-1919 Introduce FilterBolt on storm-redis

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1517#discussion_r68947165
  
    --- Diff: external/storm-redis/src/test/java/org/apache/storm/redis/topology/WhitelistWordCount.java ---
    @@ -0,0 +1,155 @@
    +/**
    + * 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.topology;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.LocalCluster;
    +import org.apache.storm.StormSubmitter;
    +import org.apache.storm.redis.bolt.RedisFilterBolt;
    +import org.apache.storm.redis.common.config.JedisPoolConfig;
    +import org.apache.storm.redis.common.mapper.RedisDataTypeDescription;
    +import org.apache.storm.redis.common.mapper.RedisFilterMapper;
    +import org.apache.storm.task.OutputCollector;
    +import org.apache.storm.task.TopologyContext;
    +import org.apache.storm.topology.OutputFieldsDeclarer;
    +import org.apache.storm.topology.TopologyBuilder;
    +import org.apache.storm.topology.base.BaseRichBolt;
    +import org.apache.storm.tuple.Fields;
    +import org.apache.storm.tuple.ITuple;
    +import org.apache.storm.tuple.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Map;
    +import java.util.Random;
    +
    +public class WhitelistWordCount {
    --- End diff --
    
    Not sure where to move. Do you have an idea?
    Maybe we would like to have a rule for this case, and apply all external modules since many modules already have example topologies in test package. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1517: STORM-1919 Introduce FilterBolt on storm-redis

Posted by abhishekagarwal87 <gi...@git.apache.org>.
Github user abhishekagarwal87 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1517#discussion_r69093854
  
    --- Diff: external/storm-redis/src/test/java/org/apache/storm/redis/topology/WhitelistWordCount.java ---
    @@ -0,0 +1,155 @@
    +/**
    + * 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.topology;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.LocalCluster;
    +import org.apache.storm.StormSubmitter;
    +import org.apache.storm.redis.bolt.RedisFilterBolt;
    +import org.apache.storm.redis.common.config.JedisPoolConfig;
    +import org.apache.storm.redis.common.mapper.RedisDataTypeDescription;
    +import org.apache.storm.redis.common.mapper.RedisFilterMapper;
    +import org.apache.storm.task.OutputCollector;
    +import org.apache.storm.task.TopologyContext;
    +import org.apache.storm.topology.OutputFieldsDeclarer;
    +import org.apache.storm.topology.TopologyBuilder;
    +import org.apache.storm.topology.base.BaseRichBolt;
    +import org.apache.storm.tuple.Fields;
    +import org.apache.storm.tuple.ITuple;
    +import org.apache.storm.tuple.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Map;
    +import java.util.Random;
    +
    +public class WhitelistWordCount {
    --- End diff --
    
    @HeartSaVioR That's fine with me. I have initiated the discussion. 
    +1 for the PR. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1517: STORM-1919 Introduce FilterBolt on storm-redis

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1517#discussion_r68947209
  
    --- Diff: external/storm-redis/src/main/java/org/apache/storm/redis/bolt/RedisFilterBolt.java ---
    @@ -0,0 +1,146 @@
    +/**
    + * 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.config.JedisPoolConfig;
    +import org.apache.storm.redis.common.mapper.RedisDataTypeDescription;
    +import org.apache.storm.redis.common.mapper.RedisFilterMapper;
    +import org.apache.storm.topology.OutputFieldsDeclarer;
    +import org.apache.storm.tuple.Tuple;
    +import redis.clients.jedis.GeoCoordinate;
    +import redis.clients.jedis.JedisCommands;
    +
    +import java.util.List;
    +
    +/**
    + * Basic bolt for querying from Redis and filters out if key/field doesn't exist.
    + * If key/field exists on Redis, this bolt just forwards input tuple to default stream.
    + * <p/>
    + * Supported data types: STRING, HASH, SET, SORTED_SET, HYPER_LOG_LOG, GEO.
    + * <p/>
    + * Note: For STRING it checks such key exists on the key space.
    + * For HASH and SORTED_SET and GEO, it checks such field exists on that data structure.
    + * For SET and HYPER_LOG_LOG, it check such value exists on that data structure.
    + * (Note that it still refers key from tuple via RedisFilterMapper#getKeyFromTuple())
    + * In order to apply checking this to SET, you need to input additional key this case.
    + * <p/>
    + * Note2: If you want to just query about existence of key regardless of actual data type,
    + * specify STRING to data type of RedisFilterMapper.
    + */
    +public class RedisFilterBolt extends AbstractRedisBolt {
    +    private final RedisFilterMapper filterMapper;
    +    private final RedisDataTypeDescription.RedisDataType dataType;
    +    private final String additionalKey;
    +
    +    /**
    +     * Constructor for single Redis environment (JedisPool)
    +     * @param config configuration for initializing JedisPool
    +     * @param filterMapper mapper containing which datatype, query key that Bolt uses
    +     */
    +    public RedisFilterBolt(JedisPoolConfig config, RedisFilterMapper filterMapper) {
    +        super(config);
    +
    +        this.filterMapper = filterMapper;
    +
    +        RedisDataTypeDescription dataTypeDescription = filterMapper.getDataTypeDescription();
    +        this.dataType = dataTypeDescription.getDataType();
    +        this.additionalKey = dataTypeDescription.getAdditionalKey();
    +    }
    +
    +    /**
    +     * Constructor for Redis Cluster environment (JedisCluster)
    +     * @param config configuration for initializing JedisCluster
    +     * @param filterMapper mapper containing which datatype, query key that Bolt uses
    +     */
    +    public RedisFilterBolt(JedisClusterConfig config, RedisFilterMapper filterMapper) {
    +        super(config);
    +
    +        this.filterMapper = filterMapper;
    +
    +        RedisDataTypeDescription dataTypeDescription = filterMapper.getDataTypeDescription();
    +        this.dataType = dataTypeDescription.getDataType();
    +        this.additionalKey = dataTypeDescription.getAdditionalKey();
    +    }
    +
    +    /**
    +     * {@inheritDoc}
    +     */
    +    @Override
    +    public void execute(Tuple input) {
    +        String key = filterMapper.getKeyFromTuple(input);
    +
    +        boolean found;
    +        JedisCommands jedisCommand = null;
    +        try {
    +            jedisCommand = getInstance();
    +
    +            switch (dataType) {
    +                case STRING:
    +                    found = jedisCommand.exists(key);
    +                    break;
    +
    +                case SET:
    +                    if (additionalKey == null) {
    +                        throw new IllegalArgumentException("additionalKey should be defined");
    +                    }
    +                    found = jedisCommand.sismember(additionalKey, key);
    +                    break;
    +
    +                case HASH:
    +                    found = jedisCommand.hexists(additionalKey, key);
    +                    break;
    +
    +                case SORTED_SET:
    +                    found = jedisCommand.zrank(additionalKey, key) != null;
    +                    break;
    +
    +                case HYPER_LOG_LOG:
    +                    found = jedisCommand.pfcount(key) > 0;
    +                    break;
    +
    +                case GEO:
    +                    List<GeoCoordinate> geopos = jedisCommand.geopos(additionalKey, key);
    +                    found = geopos != null && geopos.size() > 0;
    --- End diff --
    
    Addressed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1517: STORM-1919 Introduce FilterBolt on storm-redis

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/storm/pull/1517


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1517: STORM-1919 Introduce FilterBolt on storm-redis

Posted by darionyaphet <gi...@git.apache.org>.
Github user darionyaphet commented on the issue:

    https://github.com/apache/storm/pull/1517
  
    @HeartSaVioR Thank you for you Great Work . I have a small question about `Mapper` extention structure .


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1517: STORM-1919 Introduce FilterBolt on storm-redis

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1517#discussion_r68917610
  
    --- Diff: external/storm-redis/src/main/java/org/apache/storm/redis/bolt/RedisFilterBolt.java ---
    @@ -0,0 +1,146 @@
    +/**
    + * 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.config.JedisPoolConfig;
    +import org.apache.storm.redis.common.mapper.RedisDataTypeDescription;
    +import org.apache.storm.redis.common.mapper.RedisFilterMapper;
    +import org.apache.storm.topology.OutputFieldsDeclarer;
    +import org.apache.storm.tuple.Tuple;
    +import redis.clients.jedis.GeoCoordinate;
    +import redis.clients.jedis.JedisCommands;
    +
    +import java.util.List;
    +
    +/**
    + * Basic bolt for querying from Redis and filters out if key/field doesn't exist.
    + * If key/field exists on Redis, this bolt just forwards input tuple to default stream.
    + * <p/>
    + * Supported data types: STRING, HASH, SET, SORTED_SET, HYPER_LOG_LOG, GEO.
    + * <p/>
    + * Note: For STRING it checks such key exists on the key space.
    + * For HASH and SORTED_SET and GEO, it checks such field exists on that data structure.
    + * For SET and HYPER_LOG_LOG, it check such value exists on that data structure.
    + * (Note that it still refers key from tuple via RedisFilterMapper#getKeyFromTuple())
    + * In order to apply checking this to SET, you need to input additional key this case.
    + * <p/>
    + * Note2: If you want to just query about existence of key regardless of actual data type,
    + * specify STRING to data type of RedisFilterMapper.
    + */
    +public class RedisFilterBolt extends AbstractRedisBolt {
    +    private final RedisFilterMapper filterMapper;
    +    private final RedisDataTypeDescription.RedisDataType dataType;
    +    private final String additionalKey;
    +
    +    /**
    +     * Constructor for single Redis environment (JedisPool)
    +     * @param config configuration for initializing JedisPool
    +     * @param filterMapper mapper containing which datatype, query key that Bolt uses
    +     */
    +    public RedisFilterBolt(JedisPoolConfig config, RedisFilterMapper filterMapper) {
    +        super(config);
    +
    +        this.filterMapper = filterMapper;
    +
    +        RedisDataTypeDescription dataTypeDescription = filterMapper.getDataTypeDescription();
    +        this.dataType = dataTypeDescription.getDataType();
    +        this.additionalKey = dataTypeDescription.getAdditionalKey();
    +    }
    +
    +    /**
    +     * Constructor for Redis Cluster environment (JedisCluster)
    +     * @param config configuration for initializing JedisCluster
    +     * @param filterMapper mapper containing which datatype, query key that Bolt uses
    +     */
    +    public RedisFilterBolt(JedisClusterConfig config, RedisFilterMapper filterMapper) {
    +        super(config);
    +
    +        this.filterMapper = filterMapper;
    +
    +        RedisDataTypeDescription dataTypeDescription = filterMapper.getDataTypeDescription();
    +        this.dataType = dataTypeDescription.getDataType();
    +        this.additionalKey = dataTypeDescription.getAdditionalKey();
    +    }
    +
    +    /**
    +     * {@inheritDoc}
    +     */
    +    @Override
    +    public void execute(Tuple input) {
    +        String key = filterMapper.getKeyFromTuple(input);
    +
    +        boolean found;
    +        JedisCommands jedisCommand = null;
    +        try {
    +            jedisCommand = getInstance();
    +
    +            switch (dataType) {
    +                case STRING:
    +                    found = jedisCommand.exists(key);
    +                    break;
    +
    +                case SET:
    +                    if (additionalKey == null) {
    --- End diff --
    
    @abhishekagarwal87 Great point! Will fix.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1517: STORM-1919 Introduce FilterBolt on storm-redis

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1517#discussion_r68947207
  
    --- Diff: external/storm-redis/src/main/java/org/apache/storm/redis/bolt/RedisFilterBolt.java ---
    @@ -0,0 +1,146 @@
    +/**
    + * 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.config.JedisPoolConfig;
    +import org.apache.storm.redis.common.mapper.RedisDataTypeDescription;
    +import org.apache.storm.redis.common.mapper.RedisFilterMapper;
    +import org.apache.storm.topology.OutputFieldsDeclarer;
    +import org.apache.storm.tuple.Tuple;
    +import redis.clients.jedis.GeoCoordinate;
    +import redis.clients.jedis.JedisCommands;
    +
    +import java.util.List;
    +
    +/**
    + * Basic bolt for querying from Redis and filters out if key/field doesn't exist.
    + * If key/field exists on Redis, this bolt just forwards input tuple to default stream.
    + * <p/>
    + * Supported data types: STRING, HASH, SET, SORTED_SET, HYPER_LOG_LOG, GEO.
    + * <p/>
    + * Note: For STRING it checks such key exists on the key space.
    + * For HASH and SORTED_SET and GEO, it checks such field exists on that data structure.
    + * For SET and HYPER_LOG_LOG, it check such value exists on that data structure.
    + * (Note that it still refers key from tuple via RedisFilterMapper#getKeyFromTuple())
    + * In order to apply checking this to SET, you need to input additional key this case.
    + * <p/>
    + * Note2: If you want to just query about existence of key regardless of actual data type,
    + * specify STRING to data type of RedisFilterMapper.
    + */
    +public class RedisFilterBolt extends AbstractRedisBolt {
    +    private final RedisFilterMapper filterMapper;
    +    private final RedisDataTypeDescription.RedisDataType dataType;
    +    private final String additionalKey;
    +
    +    /**
    +     * Constructor for single Redis environment (JedisPool)
    +     * @param config configuration for initializing JedisPool
    +     * @param filterMapper mapper containing which datatype, query key that Bolt uses
    +     */
    +    public RedisFilterBolt(JedisPoolConfig config, RedisFilterMapper filterMapper) {
    +        super(config);
    +
    +        this.filterMapper = filterMapper;
    +
    +        RedisDataTypeDescription dataTypeDescription = filterMapper.getDataTypeDescription();
    +        this.dataType = dataTypeDescription.getDataType();
    +        this.additionalKey = dataTypeDescription.getAdditionalKey();
    +    }
    +
    +    /**
    +     * Constructor for Redis Cluster environment (JedisCluster)
    +     * @param config configuration for initializing JedisCluster
    +     * @param filterMapper mapper containing which datatype, query key that Bolt uses
    +     */
    +    public RedisFilterBolt(JedisClusterConfig config, RedisFilterMapper filterMapper) {
    +        super(config);
    +
    +        this.filterMapper = filterMapper;
    +
    +        RedisDataTypeDescription dataTypeDescription = filterMapper.getDataTypeDescription();
    +        this.dataType = dataTypeDescription.getDataType();
    +        this.additionalKey = dataTypeDescription.getAdditionalKey();
    +    }
    +
    +    /**
    +     * {@inheritDoc}
    +     */
    +    @Override
    +    public void execute(Tuple input) {
    +        String key = filterMapper.getKeyFromTuple(input);
    +
    +        boolean found;
    +        JedisCommands jedisCommand = null;
    +        try {
    +            jedisCommand = getInstance();
    +
    +            switch (dataType) {
    +                case STRING:
    +                    found = jedisCommand.exists(key);
    +                    break;
    +
    +                case SET:
    +                    if (additionalKey == null) {
    --- End diff --
    
    Addressed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1517: STORM-1919 Introduce FilterBolt on storm-redis

Posted by abhishekagarwal87 <gi...@git.apache.org>.
Github user abhishekagarwal87 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1517#discussion_r68917959
  
    --- Diff: external/storm-redis/src/test/java/org/apache/storm/redis/topology/WhitelistWordCount.java ---
    @@ -0,0 +1,155 @@
    +/**
    + * 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.topology;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.LocalCluster;
    +import org.apache.storm.StormSubmitter;
    +import org.apache.storm.redis.bolt.RedisFilterBolt;
    +import org.apache.storm.redis.common.config.JedisPoolConfig;
    +import org.apache.storm.redis.common.mapper.RedisDataTypeDescription;
    +import org.apache.storm.redis.common.mapper.RedisFilterMapper;
    +import org.apache.storm.task.OutputCollector;
    +import org.apache.storm.task.TopologyContext;
    +import org.apache.storm.topology.OutputFieldsDeclarer;
    +import org.apache.storm.topology.TopologyBuilder;
    +import org.apache.storm.topology.base.BaseRichBolt;
    +import org.apache.storm.tuple.Fields;
    +import org.apache.storm.tuple.ITuple;
    +import org.apache.storm.tuple.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Map;
    +import java.util.Random;
    +
    +public class WhitelistWordCount {
    --- End diff --
    
    Let's have this in examples instead since there is no test in the class. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1517: STORM-1919 Introduce FilterBolt on storm-redis

Posted by abhishekagarwal87 <gi...@git.apache.org>.
Github user abhishekagarwal87 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1517#discussion_r68954287
  
    --- Diff: external/storm-redis/src/test/java/org/apache/storm/redis/topology/WhitelistWordCount.java ---
    @@ -0,0 +1,155 @@
    +/**
    + * 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.topology;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.LocalCluster;
    +import org.apache.storm.StormSubmitter;
    +import org.apache.storm.redis.bolt.RedisFilterBolt;
    +import org.apache.storm.redis.common.config.JedisPoolConfig;
    +import org.apache.storm.redis.common.mapper.RedisDataTypeDescription;
    +import org.apache.storm.redis.common.mapper.RedisFilterMapper;
    +import org.apache.storm.task.OutputCollector;
    +import org.apache.storm.task.TopologyContext;
    +import org.apache.storm.topology.OutputFieldsDeclarer;
    +import org.apache.storm.topology.TopologyBuilder;
    +import org.apache.storm.topology.base.BaseRichBolt;
    +import org.apache.storm.tuple.Fields;
    +import org.apache.storm.tuple.ITuple;
    +import org.apache.storm.tuple.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Map;
    +import java.util.Random;
    +
    +public class WhitelistWordCount {
    --- End diff --
    
    They do. One reason I know of is the dependency collision. But test doesn't seem the right folder, because a user is not going to look in the test folder. We can do the following - 
    1. Module level sub-directories within examples e.g. examples/storm-redis, examples/storm-opentsdb etc.
    2. examples file within the module which are not shipped with jar. The link to such examples can be put up in the documentation so that users can refer to the example code and copy/run it accordingly. 
    what do you say?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1517: STORM-1919 Introduce FilterBolt on storm-redis

Posted by abhishekagarwal87 <gi...@git.apache.org>.
Github user abhishekagarwal87 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1517#discussion_r68916461
  
    --- Diff: external/storm-redis/src/main/java/org/apache/storm/redis/bolt/RedisFilterBolt.java ---
    @@ -0,0 +1,146 @@
    +/**
    + * 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.config.JedisPoolConfig;
    +import org.apache.storm.redis.common.mapper.RedisDataTypeDescription;
    +import org.apache.storm.redis.common.mapper.RedisFilterMapper;
    +import org.apache.storm.topology.OutputFieldsDeclarer;
    +import org.apache.storm.tuple.Tuple;
    +import redis.clients.jedis.GeoCoordinate;
    +import redis.clients.jedis.JedisCommands;
    +
    +import java.util.List;
    +
    +/**
    + * Basic bolt for querying from Redis and filters out if key/field doesn't exist.
    + * If key/field exists on Redis, this bolt just forwards input tuple to default stream.
    + * <p/>
    + * Supported data types: STRING, HASH, SET, SORTED_SET, HYPER_LOG_LOG, GEO.
    + * <p/>
    + * Note: For STRING it checks such key exists on the key space.
    + * For HASH and SORTED_SET and GEO, it checks such field exists on that data structure.
    + * For SET and HYPER_LOG_LOG, it check such value exists on that data structure.
    + * (Note that it still refers key from tuple via RedisFilterMapper#getKeyFromTuple())
    + * In order to apply checking this to SET, you need to input additional key this case.
    + * <p/>
    + * Note2: If you want to just query about existence of key regardless of actual data type,
    + * specify STRING to data type of RedisFilterMapper.
    + */
    +public class RedisFilterBolt extends AbstractRedisBolt {
    +    private final RedisFilterMapper filterMapper;
    +    private final RedisDataTypeDescription.RedisDataType dataType;
    +    private final String additionalKey;
    +
    +    /**
    +     * Constructor for single Redis environment (JedisPool)
    +     * @param config configuration for initializing JedisPool
    +     * @param filterMapper mapper containing which datatype, query key that Bolt uses
    +     */
    +    public RedisFilterBolt(JedisPoolConfig config, RedisFilterMapper filterMapper) {
    +        super(config);
    +
    +        this.filterMapper = filterMapper;
    +
    +        RedisDataTypeDescription dataTypeDescription = filterMapper.getDataTypeDescription();
    +        this.dataType = dataTypeDescription.getDataType();
    +        this.additionalKey = dataTypeDescription.getAdditionalKey();
    +    }
    +
    +    /**
    +     * Constructor for Redis Cluster environment (JedisCluster)
    +     * @param config configuration for initializing JedisCluster
    +     * @param filterMapper mapper containing which datatype, query key that Bolt uses
    +     */
    +    public RedisFilterBolt(JedisClusterConfig config, RedisFilterMapper filterMapper) {
    +        super(config);
    +
    +        this.filterMapper = filterMapper;
    +
    +        RedisDataTypeDescription dataTypeDescription = filterMapper.getDataTypeDescription();
    +        this.dataType = dataTypeDescription.getDataType();
    +        this.additionalKey = dataTypeDescription.getAdditionalKey();
    +    }
    +
    +    /**
    +     * {@inheritDoc}
    +     */
    +    @Override
    +    public void execute(Tuple input) {
    +        String key = filterMapper.getKeyFromTuple(input);
    +
    +        boolean found;
    +        JedisCommands jedisCommand = null;
    +        try {
    +            jedisCommand = getInstance();
    +
    +            switch (dataType) {
    +                case STRING:
    +                    found = jedisCommand.exists(key);
    +                    break;
    +
    +                case SET:
    +                    if (additionalKey == null) {
    --- End diff --
    
    since data type is known at the time of construction, can this check be moved to constructor itself? Failing fast is preferred. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---