[ 
https://issues.apache.org/jira/browse/STORM-1015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14956464#comment-14956464
 ] 

ASF GitHub Bot commented on STORM-1015:
---------------------------------------

Github user choang commented on the pull request:

    https://github.com/apache/storm/pull/705#issuecomment-147967734
  
    `StateStore` implementations could be designed like:
    ```
    public class KafkaStateStore implements StateStore {
        public KafkaStateStore(HostPort kafkaBroker, String consumerId) {  // 
consumerId could be topic?  I'm don't know enough about the compaction feature
            ...
         }
    ...
    }
    
    public class ZkStateStore implements StateStore {
        public ZkStateStore(HostPort zkConnect, String path) {
            ...
        }
    }
    
    // future implementations
    public class JdbcStateStore implements StateStore {
        public JdbcStateStore(String url, String stateTable, String user, 
String password) {
            ...
        }
    }
    
    public class MemoryStateStore implements StateStore {
        public MemoryStateStore() {
            ...
        }
    }
    ```
    
    What I'm trying to illustrate are two factors:
    1) go away from `StormConf` and `SpoutConfig` because `StateStore` does not 
need all properties of either objects.
    2) the concrete implements of `StateStore` do not need to have the same 
constructor because your goal is not to have a factory and make it completely 
config driven.  You want to make it so the topology developer can decide what 
`StateStore` to use.
    
    With the above, it would be pretty easy for someone to implement 
`MemoryStateStore` and `JdbcStateStore`.


> Store Kafka offsets with Kafka's consumer offset management api
> ---------------------------------------------------------------
>
>                 Key: STORM-1015
>                 URL: https://issues.apache.org/jira/browse/STORM-1015
>             Project: Apache Storm
>          Issue Type: Improvement
>          Components: storm-kafka
>    Affects Versions: 0.11.0
>            Reporter: Hang Sun
>            Assignee: Hang Sun
>            Priority: Minor
>              Labels: consumer, kafka, offset
>   Original Estimate: 72h
>  Remaining Estimate: 72h
>
> Current Kafka spout stores the offsets (and some other states) inside ZK with 
> its proprietary format. This does not work well with other Kafka offset 
> monitoring tools such as Burrow, KafkaOffsetMonitor etc. In addition, the 
> performance does not scale well compared with offsets managed by Kafka's 
> built-in offset management api. I have added a new option for Kafka to store 
> the same data using Kafka's built-in offset management capability. The change 
> is completely backward compatible with the current ZK storage option. The 
> feature can be turned on by a single configuration option. Hope this will 
> help people who wants to explore the option of using Kafka's built-in offset 
> management api.
> References:
> https://cwiki.apache.org/confluence/display/KAFKA/Committing+and+fetching+consumer+offsets+in+Kafka
> https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI
> -thanks



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to