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

    https://github.com/apache/storm/pull/1635#discussion_r75832422
  
    --- Diff: 
external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/trident/TridentLogicalPlanCompiler.java
 ---
    @@ -0,0 +1,344 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.sql.compiler.backends.trident;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.base.Preconditions;
    +import com.google.common.primitives.Primitives;
    +import org.apache.calcite.adapter.java.JavaTypeFactory;
    +import org.apache.calcite.rel.RelNode;
    +import org.apache.calcite.rel.core.Aggregate;
    +import org.apache.calcite.rel.core.AggregateCall;
    +import org.apache.calcite.rel.core.Filter;
    +import org.apache.calcite.rel.core.Project;
    +import org.apache.calcite.rel.core.TableModify;
    +import org.apache.calcite.rel.core.TableScan;
    +import org.apache.storm.sql.compiler.ExprCompiler;
    +import org.apache.storm.sql.runtime.ISqlTridentDataSource;
    +import org.apache.storm.sql.runtime.trident.functions.EvaluationFilter;
    +import org.apache.storm.sql.runtime.trident.functions.EvaluationFunction;
    +import org.apache.storm.sql.runtime.trident.functions.ForwardFunction;
    +import org.apache.storm.sql.runtime.trident.operations.CountBy;
    +import org.apache.storm.sql.runtime.trident.operations.DivideForAverage;
    +import org.apache.storm.sql.runtime.trident.operations.MaxBy;
    +import org.apache.storm.sql.runtime.trident.operations.MinBy;
    +import org.apache.storm.sql.runtime.trident.operations.SumBy;
    +import org.apache.storm.trident.JoinType;
    +import org.apache.storm.trident.Stream;
    +import org.apache.storm.trident.TridentTopology;
    +import org.apache.storm.trident.fluent.GroupedStream;
    +import org.apache.storm.trident.fluent.IAggregatableStream;
    +import org.apache.storm.trident.operation.builtin.Count;
    +import org.apache.storm.tuple.Fields;
    +
    +import java.io.PrintWriter;
    +import java.io.StringWriter;
    +import java.lang.reflect.Type;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class TridentLogicalPlanCompiler extends 
TridentPostOrderRelNodeVisitor {
    +    protected final Map<String, ISqlTridentDataSource> sources;
    +    protected final JavaTypeFactory typeFactory;
    +
    +    public TridentLogicalPlanCompiler(Map<String, ISqlTridentDataSource> 
sources, JavaTypeFactory typeFactory, TridentTopology topology) {
    +        super(topology);
    +        this.sources = sources;
    +        this.typeFactory = typeFactory;
    +    }
    +
    +    @Override
    +    public IAggregatableStream defaultValue(RelNode n, 
List<IAggregatableStream> inputStreams) {
    +        throw new UnsupportedOperationException();
    +    }
    +
    +    @Override
    +    public IAggregatableStream visitTableScan(TableScan scan, 
List<IAggregatableStream> inputStreams) throws Exception {
    +        String sourceName = 
Joiner.on('.').join(scan.getTable().getQualifiedName());
    +        if (!sources.containsKey(sourceName)) {
    +            throw new RuntimeException("Cannot find table " + sourceName);
    +        }
    +
    +        String stageName = getStageName(scan);
    +        return topology.newStream(stageName, 
sources.get(sourceName).getProducer());
    +    }
    +
    +    @Override
    +    public IAggregatableStream visitTableModify(TableModify modify, 
List<IAggregatableStream> inputStreams) throws Exception {
    +        if (inputStreams.size() > 1) {
    +            throw new RuntimeException("TableModify is a SingleRel");
    +        }
    +
    +        Preconditions.checkArgument(modify.isInsert(), "Only INSERT 
statement is supported.");
    +        RelNode input = modify.getInput();
    +        String tableName = 
Joiner.on('.').join(modify.getTable().getQualifiedName());
    +        Stream inputStream = inputStreams.get(0).toStream();
    +        String stageName = getStageName(modify);
    +
    +        List<String> inputFields = input.getRowType().getFieldNames();
    +        List<String> outputFields = modify.getRowType().getFieldNames();
    +
    +        return inputStream.each(new Fields(inputFields), 
sources.get(tableName).getConsumer(), new Fields(outputFields))
    +                .name(stageName);
    +    }
    +
    +    @Override
    +    public IAggregatableStream visitProject(Project project, 
List<IAggregatableStream> inputStreams) throws Exception {
    +        if (inputStreams.size() > 1) {
    +            throw new RuntimeException("Project is a SingleRel");
    +        }
    +
    +        Stream inputStream = inputStreams.get(0).toStream();
    +        Fields inputFields = inputStream.getOutputFields();
    +        String stageName = getStageName(project);
    +
    +        // Trident doesn't allow duplicated field name... need to do the 
trick...
    +        List<String> outputFieldNames = 
project.getRowType().getFieldNames();
    +        List<String> temporaryOutputFieldNames = new ArrayList<>();
    +        for (String outputFieldName : outputFieldNames) {
    +            temporaryOutputFieldNames.add("__" + outputFieldName + "__");
    +        }
    +
    +        try (StringWriter sw = new StringWriter(); PrintWriter pw = new 
PrintWriter(sw)) {
    +            pw.write("import org.apache.storm.tuple.Values;\n");
    +
    +            ExprCompiler compiler = new ExprCompiler(pw, typeFactory);
    +
    +            int size = project.getChildExps().size();
    +            String[] res = new String[size];
    +            for (int i = 0; i < size; ++i) {
    +                res[i] = project.getChildExps().get(i).accept(compiler);
    +            }
    +
    +            pw.write(String.format("\nreturn new Values(%s);", 
Joiner.on(',').join(res)));
    +            final String expression = sw.toString();
    +
    +            return inputStream.each(inputFields, new 
EvaluationFunction(expression), new Fields(temporaryOutputFieldNames))
    +                    .project(new Fields(temporaryOutputFieldNames))
    +                    .each(new Fields(temporaryOutputFieldNames), new 
ForwardFunction(), new Fields(outputFieldNames))
    +                    .project(new Fields(outputFieldNames))
    +                    .name(stageName);
    +        }
    +    }
    +
    +    @Override
    +    public IAggregatableStream visitFilter(Filter filter, 
List<IAggregatableStream> inputStreams) throws Exception {
    +        if (inputStreams.size() > 1) {
    +            throw new RuntimeException("Filter is a SingleRel");
    +        }
    +
    +        Stream inputStream = inputStreams.get(0).toStream();
    +        String stageName = getStageName(filter);
    +
    +        try (StringWriter sw = new StringWriter(); PrintWriter pw = new 
PrintWriter(sw)) {
    +            pw.write("import org.apache.storm.tuple.Values;\n");
    +
    +            ExprCompiler compiler = new ExprCompiler(pw, typeFactory);
    +            String ret = filter.getCondition().accept(compiler);
    +            pw.write(String.format("\nreturn %s;", ret));
    +            final String expression = sw.toString();
    +
    +            return inputStream.filter(new 
EvaluationFilter(expression)).name(stageName);
    +        }
    +    }
    +
    +    @Override
    +    public IAggregatableStream visitAggregate(Aggregate aggregate, 
List<IAggregatableStream> inputStreams) throws Exception {
    +        if (inputStreams.size() > 1) {
    +            throw new RuntimeException("Aggregate is a SingleRel");
    +        }
    +
    +        Stream inputStream = inputStreams.get(0).toStream();
    +        String stageName = getStageName(aggregate);
    +
    +        List<String> groupByFieldNames = new ArrayList<>();
    +        for (Integer idx : aggregate.getGroupSet()) {
    +            String fieldName = inputStream.getOutputFields().get(idx);
    +            groupByFieldNames.add(fieldName);
    +        }
    +
    +        Fields groupByFields = new Fields(groupByFieldNames);
    +        GroupedStream groupedStream = inputStream.groupBy(groupByFields);
    +
    +        List<Stream> joiningStreams = new ArrayList<>();
    +        List<Fields> joiningFields = new ArrayList<>();
    +        List<String> newOutputFields = new ArrayList<>();
    +        newOutputFields.addAll(groupByFieldNames);
    +
    +        for (AggregateCall call : aggregate.getAggCallList()) {
    +            Stream aggregateStream = handleAggregateCall(groupedStream, 
groupByFieldNames, inputStream, call);
    +
    +            Fields outputFields = aggregateStream.getOutputFields();
    +            // assuming handleAggregateCall does project so that new 
output field is always placed to end of fields
    +            String newFieldName = outputFields.get(outputFields.size() - 
1);
    +
    +            joiningStreams.add(aggregateStream);
    +            joiningFields.add(groupByFields);
    +            newOutputFields.add(newFieldName);
    +        }
    +
    +        return topology.join(joiningStreams, joiningFields, new 
Fields(newOutputFields), JoinType.INNER).name(stageName);
    --- End diff --
    
    http://storm.apache.org/releases/1.0.1/Trident-API-Overview.html
    
    > The benefits of CombinerAggregators are seen when you use them with the 
aggregate method instead of partitionAggregate. In that case, Trident 
automatically optimizes the computation by doing partial aggregations before 
transferring tuples over the network.
    
    While reading same tuples for multiple time feels redundant, handling all 
aggregations in one base function will drop the chance for optimization like 
this.
    
    I'd like to try out Trident optimizations for all of cases, and address 
optimization for Trident itself when needed.
    
    Btw, I also would like to replace this to chainedAgg(). I'll address this.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to