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

[GitHub] storm pull request #1501: [STORM-1914] Storm Kafka Field Topic Selector

GitHub user darionyaphet opened a pull request:

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

    [STORM-1914] Storm Kafka Field Topic Selector

    [STORM-1914](https://issues.apache.org/jira/browse/STORM-1914)
    
    Support field name and field index to select which kafka topic will used as a downstream .

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

    $ git pull https://github.com/darionyaphet/storm STORM-1914

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

    https://github.com/apache/storm/pull/1501.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 #1501
    
----
commit 1ed5fdd37f6039ed1dfa75a0110578eaab2a214a
Author: darionyaphet <da...@gmail.com>
Date:   2016-06-18T06:36:58Z

    STORM-1914 : Storm Kafka Field Topic Selector

----


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501#discussion_r67633346
  
    --- Diff: external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldIndexTopicSelector.java ---
    @@ -0,0 +1,52 @@
    +/**
    + * 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.kafka.bolt.selector;
    +
    +import org.apache.storm.tuple.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Using field index select topic name from tuple .
    + * When the field index out of limit ,message will write to default topic.
    + */
    +public class FieldIndexTopicSelector implements KafkaTopicSelector {
    +    private static final Logger LOG = LoggerFactory.getLogger(FieldNameTopicSelector.class);
    +
    +    private final int fieldIndex;
    +    private final String DEFAULT_TOPIC_NAME;
    +
    +    public FieldIndexTopicSelector(int fieldIndex, String defaultTopicName) {
    +        this.fieldIndex = fieldIndex;
    +        this.DEFAULT_TOPIC_NAME = defaultTopicName;
    +    }
    +
    +    public FieldIndexTopicSelector(int fieldIndex) {
    --- End diff --
    
    Let's remove having default since publishing message to default topic is not making sense for Kafka, and even throws errors if it's not created.


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501
  
    In other word, constructors which receives two parameters are fine with me, but constructors which receives only one parameter are bad for me.


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501#discussion_r67640408
  
    --- Diff: external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldIndexTopicSelector.java ---
    @@ -0,0 +1,52 @@
    +/**
    + * 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.kafka.bolt.selector;
    +
    +import org.apache.storm.tuple.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Using field index select topic name from tuple .
    + * When the field index out of limit ,message will write to default topic.
    + */
    +public class FieldIndexTopicSelector implements KafkaTopicSelector {
    +    private static final Logger LOG = LoggerFactory.getLogger(FieldNameTopicSelector.class);
    +
    +    private final int fieldIndex;
    +    private final String DEFAULT_TOPIC_NAME;
    +
    +    public FieldIndexTopicSelector(int fieldIndex, String defaultTopicName) {
    +        this.fieldIndex = fieldIndex;
    +        this.DEFAULT_TOPIC_NAME = defaultTopicName;
    +    }
    +
    +    public FieldIndexTopicSelector(int fieldIndex) {
    --- End diff --
    
    I mean having default topic is fine but don't define default topic by ourselves which is no effect when users don't create default topic before run this topology. In other words, above constructor is fine but this constructor is bad.


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501
  
    @HeartSaVioR thank you ~~ I have fix issues and add some comment at REAEMD to describe how to use `FieldNameTopicSelector`  and `FieldIndexTopicSelector`


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501
  
    @HeartSaVioR I misunderstood what you meant.
    
    keep constructor with fieldName/fieldIndex and default topic name .
    
    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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501#discussion_r67640174
  
    --- Diff: external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldIndexTopicSelector.java ---
    @@ -0,0 +1,52 @@
    +/**
    + * 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.kafka.bolt.selector;
    +
    +import org.apache.storm.tuple.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Using field index select topic name from tuple .
    + * When the field index out of limit ,message will write to default topic.
    + */
    +public class FieldIndexTopicSelector implements KafkaTopicSelector {
    +    private static final Logger LOG = LoggerFactory.getLogger(FieldNameTopicSelector.class);
    +
    +    private final int fieldIndex;
    +    private final String DEFAULT_TOPIC_NAME;
    +
    +    public FieldIndexTopicSelector(int fieldIndex, String defaultTopicName) {
    +        this.fieldIndex = fieldIndex;
    +        this.DEFAULT_TOPIC_NAME = defaultTopicName;
    +    }
    +
    +    public FieldIndexTopicSelector(int fieldIndex) {
    --- End diff --
    
    I think we should have a topic to collect message which tuple structure is illegal . (such as field name is not found or field index is out of bound .) Otherwise  we will have to discard and lost this message . 


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

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


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501
  
    I don't know why can't support `default topic` to collect the messages which not found topic name from tuple and  keeping remove constructor . 
    
    Now I have update to throw Exception when topic name miss .


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501
  
    @darionyaphet Thanks for following up quickly. +1 on me.
    @abhishekagarwal87 Could you review it again?


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501
  
    I think this is good to have. I commented some places which seems easy to fix.
    In order to let users enjoy the benefit, could you describe how to use to README?
    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 pull request #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501#discussion_r67848450
  
    --- Diff: external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldIndexTopicSelector.java ---
    @@ -0,0 +1,52 @@
    +/**
    + * 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.kafka.bolt.selector;
    +
    +import org.apache.storm.tuple.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Using field index select topic name from tuple .
    + * When the field index out of limit ,message will write to default topic.
    + */
    +public class FieldIndexTopicSelector implements KafkaTopicSelector {
    +    private static final Logger LOG = LoggerFactory.getLogger(FieldNameTopicSelector.class);
    +
    +    private final int fieldIndex;
    +    private final String defaultTopicName;
    +
    +    public FieldIndexTopicSelector(int fieldIndex, String defaultTopicName) {
    +        this.fieldIndex = fieldIndex;
    +        this.defaultTopicName = defaultTopicName;
    +    }
    +
    +    public FieldIndexTopicSelector(int fieldIndex) {
    +        this(fieldIndex, "default");
    +    }
    +
    +    @Override
    +    public String getTopic(Tuple tuple) {
    +        if (tuple.size() - 1 >= fieldIndex) {
    --- End diff --
    
    can be simplified to fieldIndex < tuple.size()


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501#discussion_r67633482
  
    --- Diff: external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldIndexTopicSelector.java ---
    @@ -0,0 +1,52 @@
    +/**
    + * 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.kafka.bolt.selector;
    +
    +import org.apache.storm.tuple.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Using field index select topic name from tuple .
    + * When the field index out of limit ,message will write to default topic.
    + */
    +public class FieldIndexTopicSelector implements KafkaTopicSelector {
    +    private static final Logger LOG = LoggerFactory.getLogger(FieldNameTopicSelector.class);
    +
    +    private final int fieldIndex;
    +    private final String DEFAULT_TOPIC_NAME;
    +
    +    public FieldIndexTopicSelector(int fieldIndex, String defaultTopicName) {
    +        this.fieldIndex = fieldIndex;
    +        this.DEFAULT_TOPIC_NAME = defaultTopicName;
    +    }
    +
    +    public FieldIndexTopicSelector(int fieldIndex) {
    +        this(fieldIndex, "default");
    +    }
    +
    +    @Override
    +    public String getTopic(Tuple tuple) {
    +        if (tuple.size() - 1 >= fieldIndex) {
    +            return tuple.getString(fieldIndex);
    +        } else {
    +            LOG.warn("Field Index " + fieldIndex + " Out of limit . Use default topic " + DEFAULT_TOPIC_NAME);
    --- End diff --
    
    limit -> bound


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501#discussion_r67848494
  
    --- Diff: external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldIndexTopicSelector.java ---
    @@ -0,0 +1,52 @@
    +/**
    + * 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.kafka.bolt.selector;
    +
    +import org.apache.storm.tuple.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Using field index select topic name from tuple .
    + * When the field index out of limit ,message will write to default topic.
    + */
    +public class FieldIndexTopicSelector implements KafkaTopicSelector {
    +    private static final Logger LOG = LoggerFactory.getLogger(FieldNameTopicSelector.class);
    +
    +    private final int fieldIndex;
    +    private final String defaultTopicName;
    +
    +    public FieldIndexTopicSelector(int fieldIndex, String defaultTopicName) {
    +        this.fieldIndex = fieldIndex;
    +        this.defaultTopicName = defaultTopicName;
    +    }
    +
    +    public FieldIndexTopicSelector(int fieldIndex) {
    +        this(fieldIndex, "default");
    +    }
    +
    +    @Override
    +    public String getTopic(Tuple tuple) {
    +        if (tuple.size() - 1 >= fieldIndex) {
    +            return tuple.getString(fieldIndex);
    +        } else {
    +            LOG.warn("Field Index " + fieldIndex + " Out of bound . Use default topic " + defaultTopicName);
    --- End diff --
    
    "Use default topic" --> "Using default topic"


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501#discussion_r67996913
  
    --- Diff: external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldNameTopicSelector.java ---
    @@ -0,0 +1,52 @@
    +/**
    + * 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.kafka.bolt.selector;
    +
    +import org.apache.storm.tuple.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Using field name select topic name from tuple .
    + * When the field not found ,message will write to default topic.
    + */
    +public class FieldNameTopicSelector implements KafkaTopicSelector {
    +    private static final Logger LOG = LoggerFactory.getLogger(FieldNameTopicSelector.class);
    +
    +    private final String fieldName;
    +    private final String defaultTopicName;
    +
    +    public FieldNameTopicSelector(String fieldName, String defaultTopicName) {
    +        this.fieldName = fieldName;
    +        this.defaultTopicName = defaultTopicName;
    +    }
    +
    +    public FieldNameTopicSelector(String fieldName) {
    --- End diff --
    
    We can have only one constructor forcing users to input default topic. We shouldn't define our own default.


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501#discussion_r67849032
  
    --- Diff: external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldNameTopicSelector.java ---
    @@ -0,0 +1,52 @@
    +/**
    + * 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.kafka.bolt.selector;
    +
    +import org.apache.storm.tuple.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Using field name select topic name from tuple .
    --- End diff --
    
    "Uses field name to select the topic name from tuple. 
    If the field is not found, message will be written to default topic". 


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501
  
    @abhishekagarwal87 @HeartSaVioR `Storm-Kafka trident` `mappers` and `selectors` are very similar with `bolt`'s `mappers` and `selectors`  .  Maybe we can move them into one place ?


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501#discussion_r67633371
  
    --- Diff: external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldNameTopicSelector.java ---
    @@ -0,0 +1,52 @@
    +/**
    + * 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.kafka.bolt.selector;
    +
    +import org.apache.storm.tuple.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Using field name select topic name from tuple .
    + * When the field not found ,message will write to default topic.
    + */
    +public class FieldNameTopicSelector implements KafkaTopicSelector {
    +    private static final Logger LOG = LoggerFactory.getLogger(FieldNameTopicSelector.class);
    +
    +    private final String fieldName;
    +    private final String DEFAULT_TOPIC_NAME;
    +
    +    public FieldNameTopicSelector(String fieldName, String defaultTopicName) {
    +        this.fieldName = fieldName;
    +        this.DEFAULT_TOPIC_NAME = defaultTopicName;
    +    }
    +
    +    public FieldNameTopicSelector(String fieldName) {
    --- End diff --
    
    Same here.


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501
  
    I'm not sure why constructor of FieldIndexTopicSelector and FieldNameTopicSelector are resurrected again, but I'm revoking +1 and now -1 on this unless it removes specific meaning of "default". 
    As I said previously, there's no "default" topic in Kafka by default so it should throw errors. We should force users to input default topic name to make them prepared.


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501#discussion_r67897721
  
    --- Diff: external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldNameTopicSelector.java ---
    @@ -0,0 +1,52 @@
    +/**
    + * 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.kafka.bolt.selector;
    +
    +import org.apache.storm.tuple.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Using field name select topic name from tuple .
    + * When the field not found ,message will write to default topic.
    + */
    +public class FieldNameTopicSelector implements KafkaTopicSelector {
    +    private static final Logger LOG = LoggerFactory.getLogger(FieldNameTopicSelector.class);
    +
    +    private final String fieldName;
    +    private final String defaultTopicName;
    +
    +    public FieldNameTopicSelector(String fieldName, String defaultTopicName) {
    +        this.fieldName = fieldName;
    +        this.defaultTopicName = defaultTopicName;
    +    }
    +
    +    public FieldNameTopicSelector(String fieldName) {
    --- End diff --
    
    you should make sure default topic have been created .


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501#discussion_r67633476
  
    --- Diff: external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldIndexTopicSelector.java ---
    @@ -0,0 +1,52 @@
    +/**
    + * 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.kafka.bolt.selector;
    +
    +import org.apache.storm.tuple.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Using field index select topic name from tuple .
    + * When the field index out of limit ,message will write to default topic.
    --- End diff --
    
    the field index out of limit -> the field index is out of bound


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501
  
    +1


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501#discussion_r67848867
  
    --- Diff: external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldNameTopicSelector.java ---
    @@ -0,0 +1,52 @@
    +/**
    + * 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.kafka.bolt.selector;
    +
    +import org.apache.storm.tuple.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Using field name select topic name from tuple .
    + * When the field not found ,message will write to default topic.
    + */
    +public class FieldNameTopicSelector implements KafkaTopicSelector {
    +    private static final Logger LOG = LoggerFactory.getLogger(FieldNameTopicSelector.class);
    +
    +    private final String fieldName;
    +    private final String defaultTopicName;
    +
    +    public FieldNameTopicSelector(String fieldName, String defaultTopicName) {
    +        this.fieldName = fieldName;
    +        this.defaultTopicName = defaultTopicName;
    +    }
    +
    +    public FieldNameTopicSelector(String fieldName) {
    +        this(fieldName, "default");
    +    }
    +
    +    @Override
    +    public String getTopic(Tuple tuple) {
    +        if (tuple.contains(fieldName)) {
    +            return tuple.getStringByField(fieldName);
    +        } else {
    +            LOG.warn("Field " + fieldName + " Not Found . Use default topic " + defaultTopicName);
    --- End diff --
    
    "Use default topic" --> "Using default topic"


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501#discussion_r67633252
  
    --- Diff: external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldNameTopicSelector.java ---
    @@ -0,0 +1,52 @@
    +/**
    + * 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.kafka.bolt.selector;
    +
    +import org.apache.storm.tuple.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Using field name select topic name from tuple .
    + * When the field not found ,message will write to default topic.
    + */
    +public class FieldNameTopicSelector implements KafkaTopicSelector {
    +    private static final Logger LOG = LoggerFactory.getLogger(FieldNameTopicSelector.class);
    +
    +    private final String fieldName;
    +    private final String DEFAULT_TOPIC_NAME;
    --- End diff --
    
    Same here.


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501#discussion_r67633242
  
    --- Diff: external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldIndexTopicSelector.java ---
    @@ -0,0 +1,52 @@
    +/**
    + * 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.kafka.bolt.selector;
    +
    +import org.apache.storm.tuple.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Using field index select topic name from tuple .
    + * When the field index out of limit ,message will write to default topic.
    + */
    +public class FieldIndexTopicSelector implements KafkaTopicSelector {
    +    private static final Logger LOG = LoggerFactory.getLogger(FieldNameTopicSelector.class);
    +
    +    private final int fieldIndex;
    +    private final String DEFAULT_TOPIC_NAME;
    --- End diff --
    
    Could you please change it to defaultTopicName? It's against Java naming convention.


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501#discussion_r67849206
  
    --- Diff: external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldNameTopicSelector.java ---
    @@ -0,0 +1,52 @@
    +/**
    + * 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.kafka.bolt.selector;
    +
    +import org.apache.storm.tuple.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Using field name select topic name from tuple .
    + * When the field not found ,message will write to default topic.
    + */
    +public class FieldNameTopicSelector implements KafkaTopicSelector {
    +    private static final Logger LOG = LoggerFactory.getLogger(FieldNameTopicSelector.class);
    +
    +    private final String fieldName;
    +    private final String defaultTopicName;
    +
    +    public FieldNameTopicSelector(String fieldName, String defaultTopicName) {
    +        this.fieldName = fieldName;
    +        this.defaultTopicName = defaultTopicName;
    +    }
    +
    +    public FieldNameTopicSelector(String fieldName) {
    --- End diff --
    
    I think this constructor is not needed. Similarly for FieldIndexTopicSelector


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501#discussion_r67911405
  
    --- Diff: external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldNameTopicSelector.java ---
    @@ -0,0 +1,52 @@
    +/**
    + * 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.kafka.bolt.selector;
    +
    +import org.apache.storm.tuple.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Using field name select topic name from tuple .
    + * When the field not found ,message will write to default topic.
    + */
    +public class FieldNameTopicSelector implements KafkaTopicSelector {
    +    private static final Logger LOG = LoggerFactory.getLogger(FieldNameTopicSelector.class);
    +
    +    private final String fieldName;
    +    private final String defaultTopicName;
    +
    +    public FieldNameTopicSelector(String fieldName, String defaultTopicName) {
    +        this.fieldName = fieldName;
    +        this.defaultTopicName = defaultTopicName;
    +    }
    +
    +    public FieldNameTopicSelector(String fieldName) {
    --- End diff --
    
    Yes but we need not assign special meaning to topic named "default". It could also result in two clients accidentally publishing to "default" topic. 


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501
  
    You're misunderstanding what I and @abhishekagarwal87 have been saying. We're OK to have default topic, but not OK to have "default" topic. There's no "default" topic in Kafka by default.


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501
  
    Looks good to me too. Thank you @darionyaphet 


---
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 #1501: [STORM-1914] Storm Kafka Field Topic Selector

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

    https://github.com/apache/storm/pull/1501#discussion_r67977419
  
    --- Diff: external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldNameTopicSelector.java ---
    @@ -0,0 +1,52 @@
    +/**
    + * 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.kafka.bolt.selector;
    +
    +import org.apache.storm.tuple.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Using field name select topic name from tuple .
    + * When the field not found ,message will write to default topic.
    + */
    +public class FieldNameTopicSelector implements KafkaTopicSelector {
    +    private static final Logger LOG = LoggerFactory.getLogger(FieldNameTopicSelector.class);
    +
    +    private final String fieldName;
    +    private final String defaultTopicName;
    +
    +    public FieldNameTopicSelector(String fieldName, String defaultTopicName) {
    +        this.fieldName = fieldName;
    +        this.defaultTopicName = defaultTopicName;
    +    }
    +
    +    public FieldNameTopicSelector(String fieldName) {
    --- End diff --
    
    Actually topic name fetch from tuple should be enumeration .  `default topic` will collect other message . 


---
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.
---