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

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

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

    https://github.com/apache/storm/pull/1324#discussion_r62035668
  
    --- Diff: 
storm-core/src/jvm/org/apache/storm/metric/filter/FilterByMetricName.java ---
    @@ -0,0 +1,110 @@
    +/**
    + * 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.filter;
    +
    +import com.google.common.base.Function;
    +import com.google.common.cache.Cache;
    +import com.google.common.cache.CacheBuilder;
    +import com.google.common.collect.Iterators;
    +import com.google.common.collect.Lists;
    +import org.apache.storm.metric.api.IMetricsConsumer;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.regex.Pattern;
    +
    +public class FilterByMetricName implements MetricsFilter {
    +    private final Cache<String, Boolean> filterCache;
    +    private final List<Pattern> whitelistPattern;
    +    private final List<Pattern> blacklistPattern;
    +    private boolean noneSpecified = false;
    +
    +    public FilterByMetricName(List<String> whitelistPattern, List<String> 
blacklistPattern) {
    +        // guard NPE
    +        if (whitelistPattern == null) {
    +            this.whitelistPattern = Collections.emptyList();
    +        } else {
    +            this.whitelistPattern = 
convertPatternStringsToPatternInstances(whitelistPattern);
    +        }
    +
    +        // guard NPE
    +        if (blacklistPattern == null) {
    +            this.blacklistPattern = Collections.emptyList();
    +        } else {
    +            this.blacklistPattern = 
convertPatternStringsToPatternInstances(blacklistPattern);
    +        }
    +
    +        if (this.whitelistPattern.isEmpty() && 
this.blacklistPattern.isEmpty()) {
    +            noneSpecified = true;
    +        } else if (!this.whitelistPattern.isEmpty() && 
!this.blacklistPattern.isEmpty()) {
    +            throw new IllegalArgumentException("You have to specify either 
includes or excludes, or none.");
    +        }
    +
    +        filterCache = CacheBuilder.newBuilder()
    +                .maximumSize(1000)
    +                .build();
    +    }
    +
    +    private ArrayList<Pattern> 
convertPatternStringsToPatternInstances(List<String> patterns) {
    +        return Lists.newArrayList(Iterators.transform(patterns.iterator(), 
new Function<String, Pattern>() {
    +            @Override
    +            public Pattern apply(String s) {
    +                return Pattern.compile(s);
    +            }
    +        }));
    +    }
    +
    +    @Override
    +    public boolean apply(IMetricsConsumer.DataPoint dataPoint) {
    +        String metricName = dataPoint.name;
    +
    +        Boolean cachedFilteredIn = filterCache.getIfPresent(metricName);
    +        if (cachedFilteredIn != null) {
    +            return cachedFilteredIn;
    +        } else {
    +            boolean filteredIn = isFilteredIn(metricName);
    +            filterCache.put(metricName, filteredIn);
    +            return filteredIn;
    +        }
    +    }
    +
    +    private boolean isFilteredIn(String metricName) {
    +        if (noneSpecified) {
    --- End diff --
    
    It avoids lookup with filterCache, but not more than since filter results 
are cached. Anyway looking up filterCache is also redundant so I'll move it to 
first line of apply().


> Introduce 'whitelist' / 'blacklist' option to MetricsConsumer
> -------------------------------------------------------------
>
>                 Key: STORM-1700
>                 URL: https://issues.apache.org/jira/browse/STORM-1700
>             Project: Apache Storm
>          Issue Type: Sub-task
>          Components: storm-core
>    Affects Versions: 1.0.0, 2.0.0
>            Reporter: Jungtaek Lim
>            Assignee: Jungtaek Lim
>
> Storm provides various metrics by default, and so on some external modules 
> (storm-kafka).
> When we register MetricsConsumer, MetricsConsumer should handle all of 
> metrics. If MetricsConsumer cannot keep up with these metrics, only way to 
> keep up is increasing parallelism, which seems limited. Furthermore, some 
> users don't want to care about some metrics since unintended metrics will 
> fill external storage.
> Though MetricsConsumer itself can filter metrics by name, it would be better 
> to support filter by Storm side. It will reduce the redundant works for Storm 
> community.
> If we provide filter options, it would be great.



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

Reply via email to