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

ASF GitHub Bot commented on NIFI-5188:
--------------------------------------

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

    https://github.com/apache/nifi/pull/2696#discussion_r195729409
  
    --- Diff: 
nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/druid/DruidTranquilityController.java
 ---
    @@ -518,56 +515,23 @@ public String getTransitUri() {
         }
     
         private List<AggregatorFactory> getAggregatorList(String 
aggregatorJSON) {
    -        List<AggregatorFactory> aggregatorList = new LinkedList<>();
    -        List<Map<String, String>> aggregatorInfo = 
parseJsonString(aggregatorJSON);
    -        for (Map<String, String> aggregator : aggregatorInfo) {
    -
    -            if (aggregator.get("type").equalsIgnoreCase("count")) {
    -                aggregatorList.add(getCountAggregator(aggregator));
    -            } else if 
(aggregator.get("type").equalsIgnoreCase("doublesum")) {
    -                aggregatorList.add(getDoubleSumAggregator(aggregator));
    -            } else if 
(aggregator.get("type").equalsIgnoreCase("doublemax")) {
    -                aggregatorList.add(getDoubleMaxAggregator(aggregator));
    -            } else if 
(aggregator.get("type").equalsIgnoreCase("doublemin")) {
    -                aggregatorList.add(getDoubleMinAggregator(aggregator));
    -            } else if (aggregator.get("type").equalsIgnoreCase("longsum")) 
{
    -                aggregatorList.add(getLongSumAggregator(aggregator));
    -            } else if (aggregator.get("type").equalsIgnoreCase("longmax")) 
{
    -                aggregatorList.add(getLongMaxAggregator(aggregator));
    -            } else if (aggregator.get("type").equalsIgnoreCase("longmin")) 
{
    -                aggregatorList.add(getLongMinAggregator(aggregator));
    -            }
    -        }
    -
    -        return aggregatorList;
    -    }
    -
    -    private AggregatorFactory getLongMinAggregator(Map<String, String> 
map) {
    -        return new LongMinAggregatorFactory(map.get("name"), 
map.get("fieldName"));
    -    }
    -
    -    private AggregatorFactory getLongMaxAggregator(Map<String, String> 
map) {
    -        return new LongMaxAggregatorFactory(map.get("name"), 
map.get("fieldName"));
    -    }
    -
    -    private AggregatorFactory getLongSumAggregator(Map<String, String> 
map) {
    -        return new LongSumAggregatorFactory(map.get("name"), 
map.get("fieldName"));
    -    }
    -
    -    private AggregatorFactory getDoubleMinAggregator(Map<String, String> 
map) {
    -        return new DoubleMinAggregatorFactory(map.get("name"), 
map.get("fieldName"));
    -    }
    -
    -    private AggregatorFactory getDoubleMaxAggregator(Map<String, String> 
map) {
    -        return new DoubleMaxAggregatorFactory(map.get("name"), 
map.get("fieldName"));
    -    }
    -
    -    private AggregatorFactory getDoubleSumAggregator(Map<String, String> 
map) {
    -        return new DoubleSumAggregatorFactory(map.get("name"), 
map.get("fieldName"));
    -    }
    +        ComponentLog log = getLogger();
    +        ObjectMapper mapper = new ObjectMapper(null);
    +        mapper.registerModule(new AggregatorsModule());
    +        mapper.registerModules(Lists.newArrayList(new 
SketchModule().getJacksonModules()));
    +        mapper.registerModules(Lists.newArrayList(new 
ApproximateHistogramDruidModule().getJacksonModules()));
     
    -    private AggregatorFactory getCountAggregator(Map<String, String> map) {
    -        return new CountAggregatorFactory(map.get("name"));
    +        try {
    +            return mapper.readValue(
    +                aggregatorJSON,
    +                new TypeReference<List<AggregatorFactory>>()
    +                {
    --- End diff --
    
    CheckStyle error here, the curly brace should be on the line above. I'll 
fix this on merge


> DruidTranquilityController does not fully support Druid aggregator
> ------------------------------------------------------------------
>
>                 Key: NIFI-5188
>                 URL: https://issues.apache.org/jira/browse/NIFI-5188
>             Project: Apache NiFi
>          Issue Type: Bug
>    Affects Versions: 1.6.0
>            Reporter: Dongkyu Hwangbo
>            Priority: Major
>
> Currently DruidTranquilityController does not fully support Druid aggregator 
> because of some code line hard-codeded in 
> DruidTranquilityController#getAggregatorList.
> {code:java}
> ...
>             if (aggregator.get("type").equalsIgnoreCase("count")) {
>                 aggregatorList.add(getCountAggregator(aggregator));
>             } else if (aggregator.get("type").equalsIgnoreCase("doublesum")) {
>                 aggregatorList.add(getDoubleSumAggregator(aggregator));
>             } else if (aggregator.get("type").equalsIgnoreCase("doublemax")) {
>                 aggregatorList.add(getDoubleMaxAggregator(aggregator));
>             } else if (aggregator.get("type").equalsIgnoreCase("doublemin")) {
>                 aggregatorList.add(getDoubleMinAggregator(aggregator));
>             } else if (aggregator.get("type").equalsIgnoreCase("longsum")) {
>                 aggregatorList.add(getLongSumAggregator(aggregator));
>             } else if (aggregator.get("type").equalsIgnoreCase("longmax")) {
>                 aggregatorList.add(getLongMaxAggregator(aggregator));
>             } else if (aggregator.get("type").equalsIgnoreCase("longmin")) {
>                 aggregatorList.add(getLongMinAggregator(aggregator));
>             }
> ...
> {code}
> DruidTranquilityController cannot ingest other Druid aggregator not defined 
> in above like cardinality, hyperUnique, thetaSketch and so on. Full 
> aggregator list is in 
> [here|http://druid.io/docs/0.9.2/querying/aggregations.html]



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to