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

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

Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1460#discussion_r66720546
  
    --- Diff: 
storm-core/src/jvm/org/apache/storm/metric/ClusterMetricsConsumerExecutor.java 
---
    @@ -0,0 +1,87 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.storm.metric;
    +
    +import com.google.common.util.concurrent.MoreExecutors;
    +import org.apache.storm.Config;
    +import org.apache.storm.metric.api.DataPoint;
    +import org.apache.storm.metric.api.IClusterMetricsConsumer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.ThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +public class ClusterMetricsConsumerExecutor {
    +    public static final Logger LOG = 
LoggerFactory.getLogger(ClusterMetricsConsumerExecutor.class);
    +
    +    private IClusterMetricsConsumer metricsConsumer;
    +    private String consumerClassName;
    +    private Object registrationArgument;
    +    private ExecutorService consumerExecutorService;
    +
    +    public ClusterMetricsConsumerExecutor(String consumerClassName, Object 
registrationArgument) {
    +        this.consumerClassName = consumerClassName;
    +        this.registrationArgument = registrationArgument;
    +    }
    +
    +    public void prepare() {
    +        try {
    +            metricsConsumer = 
(IClusterMetricsConsumer)Class.forName(consumerClassName).newInstance();
    +        } catch (Exception e) {
    +            throw new RuntimeException("Could not instantiate a class 
listed in config under section " +
    +                    Config.STORM_CLUSTER_METRICS_CONSUMER_REGISTER + " 
with fully qualified name " + consumerClassName, e);
    +        }
    +
    +        metricsConsumer.prepare(registrationArgument);
    +
    +        ThreadPoolExecutor singleThreadExecutor = new 
ThreadPoolExecutor(1, 1, 0L, TimeUnit.MILLISECONDS,
    +                new LinkedBlockingQueue<Runnable>());
    +        consumerExecutorService = 
MoreExecutors.getExitingExecutorService(singleThreadExecutor);
    +    }
    +
    +    public void handleDataPoints(final IClusterMetricsConsumer.ClusterInfo 
clusterInfo, final Collection<DataPoint> dataPoints) {
    +        final IClusterMetricsConsumer metricsConsumerForArg = 
this.metricsConsumer;
    +        consumerExecutorService.execute(new Runnable() {
    --- End diff --
    
    @harshach Addressed. Thanks!


> Introduce ClusterMetricsConsumer
> --------------------------------
>
>                 Key: STORM-1723
>                 URL: https://issues.apache.org/jira/browse/STORM-1723
>             Project: Apache Storm
>          Issue Type: New Feature
>          Components: storm-core
>    Affects Versions: 2.0.0, 1.0.1
>            Reporter: Jungtaek Lim
>            Assignee: Jungtaek Lim
>             Fix For: 2.0.0, 1.1.0
>
>
> NOTE: This issue is already discussed shortly. Please refer 
> [here|http://mail-archives.apache.org/mod_mbox/storm-dev/201604.mbox/%3CCAF5108hDCcMKxLXKUYLReOoKkNNdgW2YudweR+mKr=1hlsl...@mail.gmail.com%3E]
>  for details.
> This issue focuses to introduce ClusterMetricsConsumer and provide interface 
> to let users plugin their consumers.
> ClusterMetricsConsumers will be attached to Nimbus, and leader of Nimbus will 
> push cluster related metrics to ClusterMetricsConsumer.
> Requirements of ClusterMetricsConsumer are here:
> - Only leader of Nimbus should publish cluster metrics to consumer.
> - Nimbus shouldn't be affected by crashing or heavy latency on consumer.
> - Consumer should have resilient when crashing or Nimbus should take care of.



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

Reply via email to