Repository: tajo Updated Branches: refs/heads/master f775d7d29 -> 0ec2a890c
TAJO-1948: Change GroupbyNode::setAggFunctions and getAggFunctions to set and get List. Closes #897 Project: http://git-wip-us.apache.org/repos/asf/tajo/repo Commit: http://git-wip-us.apache.org/repos/asf/tajo/commit/0ec2a890 Tree: http://git-wip-us.apache.org/repos/asf/tajo/tree/0ec2a890 Diff: http://git-wip-us.apache.org/repos/asf/tajo/diff/0ec2a890 Branch: refs/heads/master Commit: 0ec2a890c73c79944fb6a0233226c00a0d0b06d1 Parents: f775d7d Author: Dongkyu Hwangbo <[email protected]> Authored: Wed Dec 2 23:54:36 2015 -0800 Committer: Hyunsik Choi <[email protected]> Committed: Wed Dec 2 23:54:36 2015 -0800 ---------------------------------------------------------------------- CHANGES | 3 ++ .../tajo/engine/eval/TestEvalTreeUtil.java | 6 +-- .../engine/planner/global/GlobalPlanner.java | 22 ++++----- .../global/builder/DistinctGroupbyBuilder.java | 52 +++++++++----------- .../planner/physical/AggregationExec.java | 8 +-- .../DistinctGroupbyFirstAggregationExec.java | 16 +++--- .../DistinctGroupbyHashAggregationExec.java | 16 +++--- .../DistinctGroupbySecondAggregationExec.java | 20 ++++---- .../DistinctGroupbySortAggregationExec.java | 2 +- .../DistinctGroupbyThirdAggregationExec.java | 26 +++++----- .../planner/physical/HashAggregateExec.java | 12 ++--- .../planner/physical/SortAggregateExec.java | 16 +++--- .../org/apache/tajo/plan/LogicalPlanner.java | 10 ++-- .../tajo/plan/logical/DistinctGroupbyNode.java | 14 +++--- .../apache/tajo/plan/logical/GroupbyNode.java | 30 +++++------ .../rewrite/rules/ProjectionPushDownRule.java | 15 +++--- .../plan/serder/LogicalNodeDeserializer.java | 10 ++-- .../tajo/plan/serder/LogicalNodeSerializer.java | 9 ++-- .../org/apache/tajo/plan/util/PlannerUtil.java | 2 +- 19 files changed, 145 insertions(+), 144 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/tajo/blob/0ec2a890/CHANGES ---------------------------------------------------------------------- diff --git a/CHANGES b/CHANGES index f3cac0c..66c99a5 100644 --- a/CHANGES +++ b/CHANGES @@ -8,6 +8,9 @@ Release 0.12.0 - unreleased IMPROVEMENT + TAJO-1948: Change GroupbyNode::setAggFunctions and getAggFunctions to set + and get List. (Dongkyu Hwangbo via hyunsik) + TAJO-1979: Usage of some TajoShellCommand is omitted. (Dongkyu Hwangbo via jaehwa) http://git-wip-us.apache.org/repos/asf/tajo/blob/0ec2a890/tajo-core-tests/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java ---------------------------------------------------------------------- diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java b/tajo-core-tests/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java index 3c92842..e87f390 100644 --- a/tajo-core-tests/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java +++ b/tajo-core-tests/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java @@ -345,11 +345,11 @@ public class TestEvalTreeUtil { Expr expr = analyzer.parse(query); LogicalPlan plan = planner.createPlan(defaultContext, expr); GroupbyNode groupByNode = plan.getRootBlock().getNode(NodeType.GROUP_BY); - EvalNode [] aggEvals = groupByNode.getAggFunctions(); + List aggEvals = groupByNode.getAggFunctions(); List<AggregationFunctionCallEval> list = new ArrayList<>(); - for (int i = 0; i < aggEvals.length; i++) { - list.addAll(EvalTreeUtil.findDistinctAggFunction(aggEvals[i])); + for (int i = 0; i < aggEvals.size(); i++) { + list.addAll(EvalTreeUtil.findDistinctAggFunction((EvalNode) aggEvals.get(i))); } assertEquals(2, list.size()); http://git-wip-us.apache.org/repos/asf/tajo/blob/0ec2a890/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java index 0b42a41..bf41d5b 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java @@ -558,7 +558,7 @@ public class GlobalPlanner { // Create the groupby node for the first stage and set all necessary descriptions GroupbyNode firstStageGroupby = new GroupbyNode(context.plan.getLogicalPlan().newPID()); firstStageGroupby.setGroupingColumns(TUtil.toArray(firstStageGroupingColumns, Column.class)); - firstStageGroupby.setAggFunctions(TUtil.toArray(firstStageAggFunctions, AggregationFunctionCallEval.class)); + firstStageGroupby.setAggFunctions(firstStageAggFunctions); firstStageGroupby.setTargets(firstStageTargets); firstStageGroupby.setChild(groupbyNode.getChild()); firstStageGroupby.setInSchema(groupbyNode.getInSchema()); @@ -569,7 +569,7 @@ public class GlobalPlanner { // Create the groupby node for the second stage. GroupbyNode secondPhaseGroupby = new GroupbyNode(context.plan.getLogicalPlan().newPID()); secondPhaseGroupby.setGroupingColumns(originalGroupingColumns); - secondPhaseGroupby.setAggFunctions(TUtil.toArray(secondPhaseEvalNodes, AggregationFunctionCallEval.class)); + secondPhaseGroupby.setAggFunctions(secondPhaseEvalNodes); secondPhaseGroupby.setTargets(groupbyNode.getTargets()); ExecutionBlock secondStage = context.plan.newExecutionBlock(); @@ -750,24 +750,24 @@ public class GlobalPlanner { // Set first phase expressions if (secondPhaseGroupBy.hasAggFunctions()) { - int evalNum = secondPhaseGroupBy.getAggFunctions().length; - AggregationFunctionCallEval [] secondPhaseEvals = secondPhaseGroupBy.getAggFunctions(); - AggregationFunctionCallEval [] firstPhaseEvals = new AggregationFunctionCallEval[evalNum]; + int evalNum = secondPhaseGroupBy.getAggFunctions().size(); + List<AggregationFunctionCallEval> secondPhaseEvals = secondPhaseGroupBy.getAggFunctions(); + List<AggregationFunctionCallEval> firstPhaseEvals = new ArrayList<>(); String [] firstPhaseEvalNames = new String[evalNum]; for (int i = 0; i < evalNum; i++) { try { - firstPhaseEvals[i] = (AggregationFunctionCallEval) secondPhaseEvals[i].clone(); + firstPhaseEvals.add((AggregationFunctionCallEval) secondPhaseEvals.get(i).clone()); } catch (CloneNotSupportedException e) { throw new RuntimeException(e); } - firstPhaseEvals[i].setFirstPhase(); - firstPhaseEvalNames[i] = plan.generateUniqueColumnName(firstPhaseEvals[i]); - FieldEval param = new FieldEval(firstPhaseEvalNames[i], firstPhaseEvals[i].getValueType()); + firstPhaseEvals.get(i).setFirstPhase(); + firstPhaseEvalNames[i] = plan.generateUniqueColumnName(firstPhaseEvals.get(i)); + FieldEval param = new FieldEval(firstPhaseEvalNames[i], firstPhaseEvals.get(i).getValueType()); - secondPhaseEvals[i].setLastPhase(); - secondPhaseEvals[i].setArgs(new EvalNode[]{param}); + secondPhaseEvals.get(i).setLastPhase(); + secondPhaseEvals.get(i).setArgs(new EvalNode[]{param}); } secondPhaseGroupBy.setAggFunctions(secondPhaseEvals); http://git-wip-us.apache.org/repos/asf/tajo/blob/0ec2a890/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/builder/DistinctGroupbyBuilder.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/builder/DistinctGroupbyBuilder.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/builder/DistinctGroupbyBuilder.java index e02e06f..592ea2b 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/builder/DistinctGroupbyBuilder.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/builder/DistinctGroupbyBuilder.java @@ -72,22 +72,21 @@ public class DistinctGroupbyBuilder { baseDistinctNode.setGroupbyPlan(groupbyNode); // Set total Aggregation Functions. - AggregationFunctionCallEval[] aggFunctions = - new AggregationFunctionCallEval[groupbyNode.getAggFunctions().length]; + List<AggregationFunctionCallEval> aggFunctions = new ArrayList<>(); - for (int i = 0; i < aggFunctions.length; i++) { - aggFunctions[i] = (AggregationFunctionCallEval) groupbyNode.getAggFunctions()[i].clone(); - aggFunctions[i].setFirstPhase(); + for (int i = 0; i < groupbyNode.getAggFunctions().size(); i++) { + aggFunctions.add((AggregationFunctionCallEval) groupbyNode.getAggFunctions().get(i).clone()); + aggFunctions.get(i).setFirstPhase(); // If there is not grouping column, we can't find column alias. // Thus we should find the alias at Groupbynode output schema. if (groupbyNode.getGroupingColumns().length == 0 - && aggFunctions.length == groupbyNode.getOutSchema().getRootColumns().size()) { - aggFunctions[i].setAlias(groupbyNode.getOutSchema().getColumn(i).getQualifiedName()); + && aggFunctions.size() == groupbyNode.getOutSchema().getRootColumns().size()) { + aggFunctions.get(i).setAlias(groupbyNode.getOutSchema().getColumn(i).getQualifiedName()); } } if (groupbyNode.getGroupingColumns().length == 0 - && aggFunctions.length == groupbyNode.getOutSchema().getRootColumns().size()) { + && aggFunctions.size() == groupbyNode.getOutSchema().getRootColumns().size()) { groupbyNode.setAggFunctions(aggFunctions); } @@ -204,9 +203,9 @@ public class DistinctGroupbyBuilder { //distinct columns -> GroupbyNode Map<String, DistinctGroupbyNodeBuildInfo> distinctNodeBuildInfos = new HashMap<>(); - AggregationFunctionCallEval[] aggFunctions = groupbyNode.getAggFunctions(); - for (int aggIdx = 0; aggIdx < aggFunctions.length; aggIdx++) { - AggregationFunctionCallEval aggFunction = aggFunctions[aggIdx]; + List<AggregationFunctionCallEval> aggFunctions = groupbyNode.getAggFunctions(); + for (int aggIdx = 0; aggIdx < aggFunctions.size(); aggIdx++) { + AggregationFunctionCallEval aggFunction = aggFunctions.get(aggIdx); aggFunction.setFirstPhase(); Target originAggFunctionTarget = groupbyNode.getTargets().get(originalGroupingColumns.size() + aggIdx); Target aggFunctionTarget = @@ -260,7 +259,7 @@ public class DistinctGroupbyBuilder { targets.addAll(buildInfo.getAggFunctionTargets()); eachGroupbyNode.setTargets(targets); - eachGroupbyNode.setAggFunctions(groupbyAggFunctions.toArray(new AggregationFunctionCallEval[groupbyAggFunctions.size()])); + eachGroupbyNode.setAggFunctions(groupbyAggFunctions); eachGroupbyNode.setDistinct(true); eachGroupbyNode.setInSchema(groupbyNode.getInSchema()); @@ -278,7 +277,7 @@ public class DistinctGroupbyBuilder { otherGroupbyNode.setTargets(targets); otherGroupbyNode.setGroupingColumns(new Column[]{}); - otherGroupbyNode.setAggFunctions(otherAggregationFunctionCallEvals.toArray(new AggregationFunctionCallEval[otherAggregationFunctionCallEvals.size()])); + otherGroupbyNode.setAggFunctions(otherAggregationFunctionCallEvals); otherGroupbyNode.setInSchema(groupbyNode.getInSchema()); childGroupbyNodes.add(otherGroupbyNode); @@ -372,9 +371,9 @@ public class DistinctGroupbyBuilder { //distinct columns -> GroupbyNode Map<String, DistinctGroupbyNodeBuildInfo> distinctNodeBuildInfos = new HashMap<>(); - AggregationFunctionCallEval[] aggFunctions = groupbyNode.getAggFunctions(); - for (int aggIdx = 0; aggIdx < aggFunctions.length; aggIdx++) { - AggregationFunctionCallEval aggFunction = aggFunctions[aggIdx]; + List<AggregationFunctionCallEval> aggFunctions = groupbyNode.getAggFunctions(); + for (int aggIdx = 0; aggIdx < aggFunctions.size(); aggIdx++) { + AggregationFunctionCallEval aggFunction = aggFunctions.get(aggIdx); Target aggFunctionTarget = groupbyNode.getTargets().get(originalGroupingColumns.size() + aggIdx); if (aggFunction.isDistinct()) { @@ -417,7 +416,7 @@ public class DistinctGroupbyBuilder { targets.addAll(buildInfo.getAggFunctionTargets()); eachGroupbyNode.setTargets(targets); - eachGroupbyNode.setAggFunctions(groupbyAggFunctions.toArray(new AggregationFunctionCallEval[groupbyAggFunctions.size()])); + eachGroupbyNode.setAggFunctions(groupbyAggFunctions); eachGroupbyNode.setDistinct(true); eachGroupbyNode.setInSchema(groupbyNode.getInSchema()); @@ -437,7 +436,7 @@ public class DistinctGroupbyBuilder { otherGroupbyNode.setTargets(targets); otherGroupbyNode.setGroupingColumns(originalGroupingColumns.toArray(new Column[originalGroupingColumns.size()])); - otherGroupbyNode.setAggFunctions(otherAggregationFunctionCallEvals.toArray(new AggregationFunctionCallEval[otherAggregationFunctionCallEvals.size()])); + otherGroupbyNode.setAggFunctions(otherAggregationFunctionCallEvals); otherGroupbyNode.setInSchema(groupbyNode.getInSchema()); childGroupbyNodes.add(otherGroupbyNode); @@ -527,7 +526,7 @@ public class DistinctGroupbyBuilder { List<Target> oldTargets = secondStageGroupbyNode.getTargets(); List<Target> secondGroupbyTargets = new ArrayList<>(); - LinkedHashSet<Column> distinctColumns = EvalTreeUtil.findUniqueColumns(secondStageGroupbyNode.getAggFunctions()[0]); + LinkedHashSet<Column> distinctColumns = EvalTreeUtil.findUniqueColumns(secondStageGroupbyNode.getAggFunctions().get(0)); List<Column> uniqueDistinctColumn = new ArrayList<>(); // remove origin group by column from distinctColumns for (Column eachColumn: distinctColumns) { @@ -542,8 +541,8 @@ public class DistinctGroupbyBuilder { } } - for (int aggFuncIdx = 0; aggFuncIdx < secondStageGroupbyNode.getAggFunctions().length; aggFuncIdx++) { - secondStageGroupbyNode.getAggFunctions()[aggFuncIdx].setLastPhase(); + for (int aggFuncIdx = 0; aggFuncIdx < secondStageGroupbyNode.getAggFunctions().size(); aggFuncIdx++) { + secondStageGroupbyNode.getAggFunctions().get(aggFuncIdx).setLastPhase(); int targetIdx = originGroupColumns.size() + uniqueDistinctColumn.size() + aggFuncIdx; Target aggFuncTarget = oldTargets.get(targetIdx); secondGroupbyTargets.add(aggFuncTarget); @@ -571,7 +570,7 @@ public class DistinctGroupbyBuilder { FieldEval firstEval = new FieldEval(firstEvalNames, aggFunction.getValueType()); firstGroupbyTargets.add(new Target(firstEval)); - AggregationFunctionCallEval secondStageAggFunction = secondStageGroupbyNode.getAggFunctions()[aggFuncIdx]; + AggregationFunctionCallEval secondStageAggFunction = secondStageGroupbyNode.getAggFunctions().get(aggFuncIdx); secondStageAggFunction.setArgs(new EvalNode[] {firstEval}); secondStageAggFunction.setLastPhase(); @@ -606,12 +605,9 @@ public class DistinctGroupbyBuilder { lastSecondStageGroupbyNode.setTargets(targets); - AggregationFunctionCallEval[] aggFunctions = - new AggregationFunctionCallEval[lastSecondStageGroupbyNode.getAggFunctions().length + otherGroupbyNode.getAggFunctions().length]; - System.arraycopy(lastSecondStageGroupbyNode.getAggFunctions(), 0, - aggFunctions, 0, lastSecondStageGroupbyNode.getAggFunctions().length); - System.arraycopy(otherGroupbyNode.getAggFunctions(), 0, aggFunctions, - lastSecondStageGroupbyNode.getAggFunctions().length, otherGroupbyNode.getAggFunctions().length); + List<AggregationFunctionCallEval> aggFunctions = new ArrayList<>(); + aggFunctions.addAll(lastSecondStageGroupbyNode.getAggFunctions()); + aggFunctions.addAll(otherGroupbyNode.getAggFunctions()); lastSecondStageGroupbyNode.setAggFunctions(aggFunctions); } http://git-wip-us.apache.org/repos/asf/tajo/blob/0ec2a890/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/AggregationExec.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/AggregationExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/AggregationExec.java index fdb8fdd..1923685 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/AggregationExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/AggregationExec.java @@ -25,12 +25,14 @@ import org.apache.tajo.plan.logical.GroupbyNode; import org.apache.tajo.worker.TaskAttemptContext; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; public abstract class AggregationExec extends UnaryPhysicalExec { protected final int groupingKeyNum; protected final int aggFunctionsNum; - protected final AggregationFunctionCallEval aggFunctions[]; + protected final List<AggregationFunctionCallEval> aggFunctions; public AggregationExec(final TaskAttemptContext context, GroupbyNode plan, PhysicalExec child) throws IOException { @@ -41,9 +43,9 @@ public abstract class AggregationExec extends UnaryPhysicalExec { if (plan.hasAggFunctions()) { aggFunctions = plan.getAggFunctions(); - aggFunctionsNum = aggFunctions.length; + aggFunctionsNum = aggFunctions.size(); } else { - aggFunctions = new AggregationFunctionCallEval[0]; + aggFunctions = new ArrayList<>(); aggFunctionsNum = 0; } } http://git-wip-us.apache.org/repos/asf/tajo/blob/0ec2a890/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyFirstAggregationExec.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyFirstAggregationExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyFirstAggregationExec.java index e1eac05..427e87f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyFirstAggregationExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyFirstAggregationExec.java @@ -226,7 +226,7 @@ public class DistinctGroupbyFirstAggregationExec extends UnaryPhysicalExec { class NonDistinctHashAggregator { private int aggFunctionsNum; - private final AggregationFunctionCallEval aggFunctions[]; + private final List<AggregationFunctionCallEval> aggFunctions; // GroupingKey -> FunctionContext[] private TupleMap<FunctionContext[]> nonDistinctAggrDatas; @@ -240,9 +240,9 @@ public class DistinctGroupbyFirstAggregationExec extends UnaryPhysicalExec { if (groupbyNode.hasAggFunctions()) { aggFunctions = groupbyNode.getAggFunctions(); - aggFunctionsNum = aggFunctions.length; + aggFunctionsNum = aggFunctions.size(); } else { - aggFunctions = new AggregationFunctionCallEval[0]; + aggFunctions = new ArrayList<>(); aggFunctionsNum = 0; } @@ -259,14 +259,14 @@ public class DistinctGroupbyFirstAggregationExec extends UnaryPhysicalExec { public void compute(KeyTuple groupingKeyTuple, Tuple tuple) { FunctionContext[] contexts = nonDistinctAggrDatas.get(groupingKeyTuple); if (contexts != null) { - for (int i = 0; i < aggFunctions.length; i++) { - aggFunctions[i].merge(contexts[i], tuple); + for (int i = 0; i < aggFunctions.size(); i++) { + aggFunctions.get(i).merge(contexts[i], tuple); } } else { // if the key occurs firstly contexts = new FunctionContext[aggFunctionsNum]; for (int i = 0; i < aggFunctionsNum; i++) { - contexts[i] = aggFunctions[i].newContext(); - aggFunctions[i].merge(contexts[i], tuple); + contexts[i] = aggFunctions.get(i).newContext(); + aggFunctions.get(i).merge(contexts[i], tuple); } nonDistinctAggrDatas.put(groupingKeyTuple, contexts); } @@ -279,7 +279,7 @@ public class DistinctGroupbyFirstAggregationExec extends UnaryPhysicalExec { } for (int i = 0; i < aggFunctionsNum; i++) { - outTuple.put(i, aggFunctions[i].terminate(contexts[i])); + outTuple.put(i, aggFunctions.get(i).terminate(contexts[i])); } return outTuple; http://git-wip-us.apache.org/repos/asf/tajo/blob/0ec2a890/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyHashAggregationExec.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyHashAggregationExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyHashAggregationExec.java index aaade21..777dcf5 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyHashAggregationExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyHashAggregationExec.java @@ -352,7 +352,7 @@ public class DistinctGroupbyHashAggregationExec extends UnaryPhysicalExec { private final KeyProjector innerKeyProjector; private final int aggFunctionsNum; - private final AggregationFunctionCallEval aggFunctions[]; + private final List<AggregationFunctionCallEval> aggFunctions; private final Tuple aggregatedTuple; @@ -376,9 +376,9 @@ public class DistinctGroupbyHashAggregationExec extends UnaryPhysicalExec { if (groupbyNode.hasAggFunctions()) { aggFunctions = groupbyNode.getAggFunctions(); - aggFunctionsNum = aggFunctions.length; + aggFunctionsNum = aggFunctions.size(); } else { - aggFunctions = new AggregationFunctionCallEval[0]; + aggFunctions = new ArrayList<>(); aggFunctionsNum = 0; } @@ -407,14 +407,14 @@ public class DistinctGroupbyHashAggregationExec extends UnaryPhysicalExec { KeyTuple innerKeyTuple = innerKeyProjector.project(tuple); FunctionContext[] contexts = distinctEntry.get(innerKeyTuple); if (contexts != null) { - for (int i = 0; i < aggFunctions.length; i++) { - aggFunctions[i].merge(contexts[i], tuple); + for (int i = 0; i < aggFunctions.size(); i++) { + aggFunctions.get(i).merge(contexts[i], tuple); } } else { // if the key occurs firstly contexts = new FunctionContext[aggFunctionsNum]; for (int i = 0; i < aggFunctionsNum; i++) { - contexts[i] = aggFunctions[i].newContext(); - aggFunctions[i].merge(contexts[i], tuple); + contexts[i] = aggFunctions.get(i).newContext(); + aggFunctions.get(i).merge(contexts[i], tuple); } distinctEntry.put(innerKeyTuple, contexts); } @@ -436,7 +436,7 @@ public class DistinctGroupbyHashAggregationExec extends UnaryPhysicalExec { FunctionContext[] contexts = entry.getValue(); for (int i = 0; i < aggFunctionsNum; i++, index++) { - aggregatedTuple.put(index, aggFunctions[i].terminate(contexts[i])); + aggregatedTuple.put(index, aggFunctions.get(i).terminate(contexts[i])); } aggregatedTuples.add(aggregatedTuple); } http://git-wip-us.apache.org/repos/asf/tajo/blob/0ec2a890/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbySecondAggregationExec.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbySecondAggregationExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbySecondAggregationExec.java index be0b513..9826ff7 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbySecondAggregationExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbySecondAggregationExec.java @@ -81,7 +81,7 @@ public class DistinctGroupbySecondAggregationExec extends UnaryPhysicalExec { private int numGroupingColumns; private int[][] distinctKeyIndexes; private FunctionContext[] nonDistinctAggrContexts; - private AggregationFunctionCallEval[] nonDistinctAggrFunctions; + private List<AggregationFunctionCallEval> nonDistinctAggrFunctions; private int nonDistinctAggrTupleStartIndex = -1; // Key tuples may have various lengths. The below two maps are used to cache key tuple instances. @@ -131,7 +131,7 @@ public class DistinctGroupbySecondAggregationExec extends UnaryPhysicalExec { eachFunction.bind(context.getEvalContext(), inSchema); eachFunction.setIntermediatePhase(); } - nonDistinctAggrContexts = new FunctionContext[nonDistinctAggrFunctions.length]; + nonDistinctAggrContexts = new FunctionContext[nonDistinctAggrFunctions.size()]; } } } @@ -163,7 +163,7 @@ public class DistinctGroupbySecondAggregationExec extends UnaryPhysicalExec { } } if (nonDistinctAggrFunctions != null) { - nonDistinctAggrTupleStartIndex = inSchema.size() - nonDistinctAggrFunctions.length; + nonDistinctAggrTupleStartIndex = inSchema.size() - nonDistinctAggrFunctions.size(); } } @@ -235,9 +235,9 @@ public class DistinctGroupbySecondAggregationExec extends UnaryPhysicalExec { private void initNonDistinctAggrContext() { if (nonDistinctAggrFunctions != null) { - nonDistinctAggrContexts = new FunctionContext[nonDistinctAggrFunctions.length]; - for (int i = 0; i < nonDistinctAggrFunctions.length; i++) { - nonDistinctAggrContexts[i] = nonDistinctAggrFunctions[i].newContext(); + nonDistinctAggrContexts = new FunctionContext[nonDistinctAggrFunctions.size()]; + for (int i = 0; i < nonDistinctAggrFunctions.size(); i++) { + nonDistinctAggrContexts[i] = nonDistinctAggrFunctions.get(i).newContext(); } } } @@ -246,8 +246,8 @@ public class DistinctGroupbySecondAggregationExec extends UnaryPhysicalExec { if (nonDistinctAggrFunctions == null) { return; } - for (int i = 0; i < nonDistinctAggrFunctions.length; i++) { - nonDistinctAggrFunctions[i].merge(nonDistinctAggrContexts[i], tuple); + for (int i = 0; i < nonDistinctAggrFunctions.size(); i++) { + nonDistinctAggrFunctions.get(i).merge(nonDistinctAggrContexts[i], tuple); } } @@ -255,8 +255,8 @@ public class DistinctGroupbySecondAggregationExec extends UnaryPhysicalExec { if (nonDistinctAggrFunctions == null) { return; } - for (int i = 0; i < nonDistinctAggrFunctions.length; i++) { - tuple.put(nonDistinctAggrTupleStartIndex + i, nonDistinctAggrFunctions[i].terminate(nonDistinctAggrContexts[i])); + for (int i = 0; i < nonDistinctAggrFunctions.size(); i++) { + tuple.put(nonDistinctAggrTupleStartIndex + i, nonDistinctAggrFunctions.get(i).terminate(nonDistinctAggrContexts[i])); } } http://git-wip-us.apache.org/repos/asf/tajo/blob/0ec2a890/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbySortAggregationExec.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbySortAggregationExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbySortAggregationExec.java index cb7478a..e611d88 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbySortAggregationExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbySortAggregationExec.java @@ -127,7 +127,7 @@ public class DistinctGroupbySortAggregationExec extends PhysicalExec { int tupleIndex = 0; for (SortAggregateExec aggExec: aggregateExecs) { for (int i = 0; i < aggExec.aggFunctionsNum; i++, tupleIndex++) { - String funcName = aggExec.aggFunctions[i].getName(); + String funcName = aggExec.aggFunctions.get(i).getName(); if ("min".equals(funcName) || "max".equals(funcName) || "avg".equals(funcName) || "sum".equals(funcName)) { outTuple.put(resultColumnIdIndexes[tupleIndex], DatumFactory.createNullDatum()); } http://git-wip-us.apache.org/repos/asf/tajo/blob/0ec2a890/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyThirdAggregationExec.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyThirdAggregationExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyThirdAggregationExec.java index eb1921a..a0da674 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyThirdAggregationExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyThirdAggregationExec.java @@ -84,12 +84,12 @@ public class DistinctGroupbyThirdAggregationExec extends UnaryPhysicalExec { Column[] distinctGroupingColumns = eachGroupby.getGroupingColumns(); inTupleIndex += distinctGroupingColumns.length; - outTupleIndex += eachGroupby.getAggFunctions().length; + outTupleIndex += eachGroupby.getAggFunctions().size(); } else { nonDistinctAggr = new DistinctFinalAggregator(-1, inTupleIndex, outTupleIndex, eachGroupby); - outTupleIndex += eachGroupby.getAggFunctions().length; + outTupleIndex += eachGroupby.getAggFunctions().size(); } - resultTupleLength += eachGroupby.getAggFunctions().length; + resultTupleLength += eachGroupby.getAggFunctions().size(); } aggregators = aggregatorList.toArray(new DistinctFinalAggregator[aggregatorList.size()]); outTuple = new VTuple(resultTupleLength); @@ -234,7 +234,7 @@ public class DistinctGroupbyThirdAggregationExec extends UnaryPhysicalExec { class DistinctFinalAggregator { private FunctionContext[] functionContexts; - private AggregationFunctionCallEval[] aggrFunctions; + private List<AggregationFunctionCallEval> aggrFunctions; private int seq; private int inTupleIndex; private int outTupleIndex; @@ -254,15 +254,15 @@ public class DistinctGroupbyThirdAggregationExec extends UnaryPhysicalExec { } private void newFunctionContext() { - functionContexts = new FunctionContext[aggrFunctions.length]; - for (int i = 0; i < aggrFunctions.length; i++) { - functionContexts[i] = aggrFunctions[i].newContext(); + functionContexts = new FunctionContext[aggrFunctions.size()]; + for (int i = 0; i < aggrFunctions.size(); i++) { + functionContexts[i] = aggrFunctions.get(i).newContext(); } } public void merge(Tuple tuple) { - for (int i = 0; i < aggrFunctions.length; i++) { - aggrFunctions[i].merge(functionContexts[i], tuple); + for (int i = 0; i < aggrFunctions.size(); i++) { + aggrFunctions.get(i).merge(functionContexts[i], tuple); } if (seq == 0 && nonDistinctAggr != null) { @@ -273,8 +273,8 @@ public class DistinctGroupbyThirdAggregationExec extends UnaryPhysicalExec { } public void terminate(Tuple resultTuple) { - for (int i = 0; i < aggrFunctions.length; i++) { - resultTuple.put(resultTupleIndexes[outTupleIndex + i], aggrFunctions[i].terminate(functionContexts[i])); + for (int i = 0; i < aggrFunctions.size(); i++) { + resultTuple.put(resultTupleIndexes[outTupleIndex + i], aggrFunctions.get(i).terminate(functionContexts[i])); } newFunctionContext(); @@ -285,8 +285,8 @@ public class DistinctGroupbyThirdAggregationExec extends UnaryPhysicalExec { public void terminateEmpty(Tuple resultTuple) { newFunctionContext(); - for (int i = 0; i < aggrFunctions.length; i++) { - resultTuple.put(resultTupleIndexes[outTupleIndex + i], aggrFunctions[i].terminate(functionContexts[i])); + for (int i = 0; i < aggrFunctions.size(); i++) { + resultTuple.put(resultTupleIndexes[outTupleIndex + i], aggrFunctions.get(i).terminate(functionContexts[i])); } if (seq == 0 && nonDistinctAggr != null) { nonDistinctAggr.terminateEmpty(resultTuple); http://git-wip-us.apache.org/repos/asf/tajo/blob/0ec2a890/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashAggregateExec.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashAggregateExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashAggregateExec.java index 9741982..f86e5bc 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashAggregateExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashAggregateExec.java @@ -55,14 +55,14 @@ public class HashAggregateExec extends AggregationExec { FunctionContext [] contexts = hashTable.get(keyTuple); if(contexts != null) { - for(int i = 0; i < aggFunctions.length; i++) { - aggFunctions[i].merge(contexts[i], tuple); + for(int i = 0; i < aggFunctions.size(); i++) { + aggFunctions.get(i).merge(contexts[i], tuple); } } else { // if the key occurs firstly contexts = new FunctionContext[aggFunctionsNum]; for(int i = 0; i < aggFunctionsNum; i++) { - contexts[i] = aggFunctions[i].newContext(); - aggFunctions[i].merge(contexts[i], tuple); + contexts[i] = aggFunctions.get(i).newContext(); + aggFunctions.get(i).merge(contexts[i], tuple); } hashTable.put(keyTuple, contexts); } @@ -73,7 +73,7 @@ public class HashAggregateExec extends AggregationExec { if (groupingKeyNum == 0 && aggFunctionsNum > 0 && hashTable.entrySet().size() == 0) { FunctionContext[] contexts = new FunctionContext[aggFunctionsNum]; for(int i = 0; i < aggFunctionsNum; i++) { - contexts[i] = aggFunctions[i].newContext(); + contexts[i] = aggFunctions.get(i).newContext(); } hashTable.put(null, contexts); } @@ -99,7 +99,7 @@ public class HashAggregateExec extends AggregationExec { tuple.put(tupleIdx, keyTuple.asDatum(tupleIdx)); } for (int funcIdx = 0; funcIdx < aggFunctionsNum; funcIdx++, tupleIdx++) { - tuple.put(tupleIdx, aggFunctions[funcIdx].terminate(contexts[funcIdx])); + tuple.put(tupleIdx, aggFunctions.get(funcIdx).terminate(contexts[funcIdx])); } return tuple; http://git-wip-us.apache.org/repos/asf/tajo/blob/0ec2a890/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SortAggregateExec.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SortAggregateExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SortAggregateExec.java index 71602b8..8a931e6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SortAggregateExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SortAggregateExec.java @@ -51,7 +51,7 @@ public class SortAggregateExec extends AggregationExec { public SortAggregateExec(TaskAttemptContext context, GroupbyNode plan, PhysicalExec child) throws IOException { super(context, plan, child); - contexts = new FunctionContext[plan.getAggFunctions() == null ? 0 : plan.getAggFunctions().length]; + contexts = new FunctionContext[plan.getAggFunctions() == null ? 0 : plan.getAggFunctions().size()]; final Column [] keyColumns = plan.getGroupingColumns(); groupingKeyIds = new int[groupingKeyNum]; @@ -82,19 +82,19 @@ public class SortAggregateExec extends AggregationExec { if (lastKey == null || lastKey.equals(currentKey)) { if (lastKey == null) { for(int i = 0; i < aggFunctionsNum; i++) { - contexts[i] = aggFunctions[i].newContext(); + contexts[i] = aggFunctions.get(i).newContext(); // Merge when aggregator doesn't receive NullDatum if (!(groupingKeyNum == 0 && aggFunctionsNum == tuple.size() && tuple.isBlankOrNull(i))) { - aggFunctions[i].merge(contexts[i], tuple); + aggFunctions.get(i).merge(contexts[i], tuple); } } lastKey = new VTuple(currentKey.getValues()); } else { // aggregate for (int i = 0; i < aggFunctionsNum; i++) { - aggFunctions[i].merge(contexts[i], tuple); + aggFunctions.get(i).merge(contexts[i], tuple); } } @@ -106,12 +106,12 @@ public class SortAggregateExec extends AggregationExec { outTuple.put(tupleIdx, lastKey.asDatum(tupleIdx)); } for(int aggFuncIdx = 0; aggFuncIdx < aggFunctionsNum; tupleIdx++, aggFuncIdx++) { - outTuple.put(tupleIdx, aggFunctions[aggFuncIdx].terminate(contexts[aggFuncIdx])); + outTuple.put(tupleIdx, aggFunctions.get(aggFuncIdx).terminate(contexts[aggFuncIdx])); } for(int evalIdx = 0; evalIdx < aggFunctionsNum; evalIdx++) { - contexts[evalIdx] = aggFunctions[evalIdx].newContext(); - aggFunctions[evalIdx].merge(contexts[evalIdx], tuple); + contexts[evalIdx] = aggFunctions.get(evalIdx).newContext(); + aggFunctions.get(evalIdx).merge(contexts[evalIdx], tuple); } lastKey.put(currentKey.getValues()); @@ -129,7 +129,7 @@ public class SortAggregateExec extends AggregationExec { outTuple.put(tupleIdx, lastKey.asDatum(tupleIdx)); } for(int aggFuncIdx = 0; aggFuncIdx < aggFunctionsNum; tupleIdx++, aggFuncIdx++) { - outTuple.put(tupleIdx, aggFunctions[aggFuncIdx].terminate(contexts[aggFuncIdx])); + outTuple.put(tupleIdx, aggFunctions.get(aggFuncIdx).terminate(contexts[aggFuncIdx])); } finished = true; return outTuple; http://git-wip-us.apache.org/repos/asf/tajo/blob/0ec2a890/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java ---------------------------------------------------------------------- diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java index 79dad04..3f2b043 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java @@ -534,7 +534,7 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex } if (groupbyNode.hasAggFunctions()) { - verifyIfEvalNodesCanBeEvaluated(projectable, groupbyNode.getAggFunctions()); + verifyIfEvalNodesCanBeEvaluated(projectable, (List<EvalNode>)(List<?>) groupbyNode.getAggFunctions()); } } else if (projectable instanceof WindowAggNode) { @@ -545,7 +545,7 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex } if (windowAggNode.hasAggFunctions()) { - verifyIfEvalNodesCanBeEvaluated(projectable, windowAggNode.getWindowFunctions()); + verifyIfEvalNodesCanBeEvaluated(projectable, Arrays.asList(windowAggNode.getWindowFunctions())); } if (windowAggNode.hasSortSpecs()) { @@ -584,7 +584,7 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex } } - public static void verifyIfEvalNodesCanBeEvaluated(Projectable projectable, EvalNode[] evalNodes) + public static void verifyIfEvalNodesCanBeEvaluated(Projectable projectable, List<EvalNode> evalNodes) throws TajoException { for (EvalNode e : evalNodes) { Set<Column> columns = EvalTreeUtil.findUniqueColumns(e); @@ -804,7 +804,7 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex } groupbyNode.setDistinct(includeDistinctFunction); - groupbyNode.setAggFunctions(aggEvals.toArray(new AggregationFunctionCallEval[aggEvals.size()])); + groupbyNode.setAggFunctions(new ArrayList<>(aggEvals)); List<Target> targets = ProjectionPushDownRule.buildGroupByTarget(groupbyNode, null, aggEvalNames.toArray(new String[aggEvalNames.size()])); groupbyNode.setTargets(targets); @@ -1049,7 +1049,7 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex } // if there is at least one distinct aggregation function groupingNode.setDistinct(includeDistinctFunction); - groupingNode.setAggFunctions(aggEvalNodes.toArray(new AggregationFunctionCallEval[aggEvalNodes.size()])); + groupingNode.setAggFunctions(aggEvalNodes); List<Target> targets = new ArrayList<>(); http://git-wip-us.apache.org/repos/asf/tajo/blob/0ec2a890/tajo-plan/src/main/java/org/apache/tajo/plan/logical/DistinctGroupbyNode.java ---------------------------------------------------------------------- diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/DistinctGroupbyNode.java b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/DistinctGroupbyNode.java index a36f9a4..399c1b8 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/DistinctGroupbyNode.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/DistinctGroupbyNode.java @@ -49,7 +49,7 @@ public class DistinctGroupbyNode extends UnaryNode implements Projectable, Clone private int[] resultColumnIds = new int[]{}; /** Aggregation Functions */ - @Expose private AggregationFunctionCallEval[] aggrFunctions = PlannerUtil.EMPTY_AGG_FUNCS; + @Expose private List<AggregationFunctionCallEval> aggrFunctions = PlannerUtil.EMPTY_AGG_FUNCS; public DistinctGroupbyNode(int pid) { super(pid, NodeType.DISTINCT_GROUP_BY); @@ -99,11 +99,11 @@ public class DistinctGroupbyNode extends UnaryNode implements Projectable, Clone this.resultColumnIds = resultColumnIds; } - public AggregationFunctionCallEval [] getAggFunctions() { + public List<AggregationFunctionCallEval> getAggFunctions() { return this.aggrFunctions; } - public void setAggFunctions(AggregationFunctionCallEval[] evals) { + public void setAggFunctions(List<AggregationFunctionCallEval> evals) { this.aggrFunctions = evals; } @@ -165,7 +165,7 @@ public class DistinctGroupbyNode extends UnaryNode implements Projectable, Clone public int hashCode() { final int prime = 31; int result = 1; - result = prime * result + Arrays.hashCode(aggrFunctions); + result = prime * result + Objects.hashCode(aggrFunctions); result = prime * result + ((groupbyPlan == null) ? 0 : groupbyPlan.hashCode()); result = prime * result + Arrays.hashCode(groupingColumns); result = prime * result + Arrays.hashCode(resultColumnIds); @@ -213,9 +213,9 @@ public class DistinctGroupbyNode extends UnaryNode implements Projectable, Clone String prefix = ""; for (GroupbyNode eachNode: subGroupbyPlan) { if (eachNode.hasAggFunctions()) { - AggregationFunctionCallEval[] aggrFunctions = eachNode.getAggFunctions(); - for (int j = 0; j < aggrFunctions.length; j++) { - sb.append(prefix).append(aggrFunctions[j]); + List<AggregationFunctionCallEval> aggrFunctions = eachNode.getAggFunctions(); + for (int j = 0; j < aggrFunctions.size(); j++) { + sb.append(prefix).append(aggrFunctions.get(j)); prefix = ","; } } http://git-wip-us.apache.org/repos/asf/tajo/blob/0ec2a890/tajo-plan/src/main/java/org/apache/tajo/plan/logical/GroupbyNode.java ---------------------------------------------------------------------- diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/GroupbyNode.java b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/GroupbyNode.java index 4845c1d..3d487b8 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/GroupbyNode.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/GroupbyNode.java @@ -37,7 +37,7 @@ public class GroupbyNode extends UnaryNode implements Projectable, Cloneable { /** Grouping key sets */ @Expose private Column [] groupingKeys = PlannerUtil.EMPTY_COLUMNS; /** Aggregation Functions */ - @Expose private AggregationFunctionCallEval [] aggrFunctions = PlannerUtil.EMPTY_AGG_FUNCS; + @Expose private List<AggregationFunctionCallEval> aggrFunctions = PlannerUtil.EMPTY_AGG_FUNCS; /** * It's a list of targets. The grouping columns should be followed by aggregation functions. * aggrFunctions keep actual aggregation functions, but it only contains field references. @@ -86,18 +86,18 @@ public class GroupbyNode extends UnaryNode implements Projectable, Cloneable { } public boolean hasAggFunctions() { - return aggrFunctions.length > 0; + return aggrFunctions.size() > 0; } public int aggregationFunctionNum() { - return this.aggrFunctions.length; + return this.aggrFunctions.size(); } - public AggregationFunctionCallEval[] getAggFunctions() { + public List<AggregationFunctionCallEval> getAggFunctions() { return this.aggrFunctions; } - public void setAggFunctions(AggregationFunctionCallEval[] evals) { + public void setAggFunctions(List<AggregationFunctionCallEval> evals) { Preconditions.checkNotNull(evals); this.aggrFunctions = evals; } @@ -139,7 +139,7 @@ public class GroupbyNode extends UnaryNode implements Projectable, Cloneable { public int hashCode() { final int prime = 31; int result = 1; - result = prime * result + Arrays.hashCode(aggrFunctions); + result = prime * result + Objects.hashCode(aggrFunctions); result = prime * result + Arrays.hashCode(groupingKeys); result = prime * result + (hasDistinct ? 1231 : 1237); result = prime * result + Objects.hashCode(targets); @@ -172,9 +172,9 @@ public class GroupbyNode extends UnaryNode implements Projectable, Cloneable { } if (aggrFunctions != null) { - grp.aggrFunctions = new AggregationFunctionCallEval[aggrFunctions.length]; - for (int i = 0; i < aggrFunctions.length; i++) { - grp.aggrFunctions[i] = (AggregationFunctionCallEval) aggrFunctions[i].clone(); + grp.aggrFunctions = new ArrayList<>(); + for (int i = 0; i < aggrFunctions.size(); i++) { + grp.aggrFunctions.add((AggregationFunctionCallEval) aggrFunctions.get(i).clone()); } } @@ -205,9 +205,9 @@ public class GroupbyNode extends UnaryNode implements Projectable, Cloneable { if (hasAggFunctions()) { sb.append(", exprs: ("); - for (int j = 0; j < aggrFunctions.length; j++) { - sb.append(aggrFunctions[j]); - if(j < aggrFunctions.length - 1) { + for (int j = 0; j < aggrFunctions.size(); j++) { + sb.append(aggrFunctions.get(j)); + if(j < aggrFunctions.size() - 1) { sb.append(","); } } @@ -253,9 +253,9 @@ public class GroupbyNode extends UnaryNode implements Projectable, Cloneable { sb = new StringBuilder(); sb.append("("); - for (int j = 0; j < aggrFunctions.length; j++) { - sb.append(aggrFunctions[j]); - if(j < aggrFunctions.length - 1) { + for (int j = 0; j < aggrFunctions.size(); j++) { + sb.append(aggrFunctions.get(j)); + if(j < aggrFunctions.size() - 1) { sb.append(","); } } http://git-wip-us.apache.org/repos/asf/tajo/blob/0ec2a890/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/ProjectionPushDownRule.java ---------------------------------------------------------------------- diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/ProjectionPushDownRule.java b/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/ProjectionPushDownRule.java index cadf986..c9ef61f 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/ProjectionPushDownRule.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/ProjectionPushDownRule.java @@ -732,12 +732,12 @@ public class ProjectionPushDownRule extends // Getting eval names if (node.hasAggFunctions()) { - final int evalNum = node.getAggFunctions().length; + final int evalNum = node.getAggFunctions().size(); aggEvalNames = new String[evalNum]; for (int evalIdx = 0, targetIdx = node.getGroupingColumns().length; targetIdx < node.getTargets().size(); evalIdx++, targetIdx++) { Target target = node.getTargets().get(targetIdx); - EvalNode evalNode = node.getAggFunctions()[evalIdx]; + EvalNode evalNode = node.getAggFunctions().get(evalIdx); aggEvalNames[evalIdx] = newContext.addExpr(new Target(evalNode, target.getCanonicalName())); } } @@ -794,19 +794,18 @@ public class ProjectionPushDownRule extends // Getting projected targets if (node.hasAggFunctions() && aggEvalNames != null) { - AggregationFunctionCallEval [] aggEvals = new AggregationFunctionCallEval[aggEvalNames.length]; - int i = 0; + List<AggregationFunctionCallEval> aggEvals = new ArrayList<>(); for (Iterator<String> it = getFilteredReferences(aggEvalNames, TUtil.newList(aggEvalNames)); it.hasNext();) { String referenceName = it.next(); Target target = context.targetListMgr.getTarget(referenceName); if (LogicalPlanner.checkIfBeEvaluatedAtGroupBy(target.getEvalTree(), node)) { - aggEvals[i++] = target.getEvalTree(); + aggEvals.add(target.getEvalTree()); context.targetListMgr.markAsEvaluated(target); } } - if (aggEvals.length > 0) { + if (aggEvals.size() > 0) { node.setAggFunctions(aggEvals); } } @@ -823,7 +822,7 @@ public class ProjectionPushDownRule extends final int groupingKeyNum = groupingKeyTargets == null ? groupbyNode.getGroupingColumns().length : groupingKeyTargets.size(); final int aggrFuncNum = aggEvalNames != null ? aggEvalNames.length : 0; - EvalNode [] aggEvalNodes = groupbyNode.getAggFunctions(); + List<EvalNode> aggEvalNodes = (List<EvalNode>)(List<?>) groupbyNode.getAggFunctions(); List<Target> targets = new ArrayList<>(); if (groupingKeyTargets != null) { @@ -838,7 +837,7 @@ public class ProjectionPushDownRule extends if (aggEvalNames != null) { for (int aggrFuncIdx = 0, targetIdx = groupingKeyNum; aggrFuncIdx < aggrFuncNum; aggrFuncIdx++, targetIdx++) { - targets.add(new Target(new FieldEval(aggEvalNames[aggrFuncIdx], aggEvalNodes[aggrFuncIdx].getValueType()))); + targets.add(new Target(new FieldEval(aggEvalNames[aggrFuncIdx], aggEvalNodes.get(aggrFuncIdx).getValueType()))); } } http://git-wip-us.apache.org/repos/asf/tajo/blob/0ec2a890/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java ---------------------------------------------------------------------- diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java index fdad4c4..bccde90 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java @@ -710,12 +710,12 @@ public class LogicalNodeDeserializer { return dropIndex; } - private static AggregationFunctionCallEval [] convertAggFuncCallEvals(OverridableConf context, EvalContext evalContext, + private static List<AggregationFunctionCallEval> convertAggFuncCallEvals(OverridableConf context, EvalContext evalContext, List<PlanProto.EvalNodeTree> evalTrees) { - AggregationFunctionCallEval [] aggFuncs = new AggregationFunctionCallEval[evalTrees.size()]; - for (int i = 0; i < aggFuncs.length; i++) { - aggFuncs[i] = (AggregationFunctionCallEval) EvalNodeDeserializer.deserialize(context, evalContext, - evalTrees.get(i)); + List<AggregationFunctionCallEval> aggFuncs = new ArrayList<>(); + for (int i = 0; i < evalTrees.size(); i++) { + aggFuncs.add((AggregationFunctionCallEval) EvalNodeDeserializer.deserialize(context, evalContext, + evalTrees.get(i))); } return aggFuncs; } http://git-wip-us.apache.org/repos/asf/tajo/blob/0ec2a890/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java ---------------------------------------------------------------------- diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java index 6b082f7..d7a9598 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java @@ -30,6 +30,7 @@ import org.apache.tajo.exception.NotImplementedException; import org.apache.tajo.exception.TajoRuntimeException; import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.Target; +import org.apache.tajo.plan.expr.EvalNode; import org.apache.tajo.plan.logical.*; import org.apache.tajo.plan.rewrite.rules.IndexScanInfo.SimplePredicate; import org.apache.tajo.plan.serder.PlanProto.AlterTableNode.AddColumn; @@ -295,7 +296,7 @@ public class LogicalNodeSerializer extends BasicLogicalPlanVisitor<LogicalNodeSe } if (node.hasAggFunctions()) { groupbyBuilder.addAllAggFunctions( - ProtoUtil.<PlanProto.EvalNodeTree>toProtoObjects(node.getAggFunctions())); + ProtoUtil.<PlanProto.EvalNodeTree>toProtoObjects(node.getAggFunctions().toArray(new ProtoObject[node.getAggFunctions().size()]))); } if (node.hasTargets()) { groupbyBuilder.addAllTargets(ProtoUtil.<PlanProto.Target>toProtoObjects(node.getTargets().toArray(new ProtoObject[node.getTargets().size()]))); @@ -328,9 +329,9 @@ public class LogicalNodeSerializer extends BasicLogicalPlanVisitor<LogicalNodeSe distGroupbyBuilder.addAllGroupingKeys( ProtoUtil.<CatalogProtos.ColumnProto>toProtoObjects(node.getGroupingColumns())); } - if (node.getAggFunctions().length > 0) { + if (node.getAggFunctions().size() > 0) { distGroupbyBuilder.addAllAggFunctions( - ProtoUtil.<PlanProto.EvalNodeTree>toProtoObjects(node.getAggFunctions())); + ProtoUtil.<PlanProto.EvalNodeTree>toProtoObjects(node.getAggFunctions().toArray(new ProtoObject[node.getAggFunctions().size()]))); } if (node.hasTargets()) { distGroupbyBuilder.addAllTargets(ProtoUtil.<PlanProto.Target>toProtoObjects(node.getTargets().toArray(new ProtoObject[node.getTargets().size()]))); @@ -830,7 +831,7 @@ public class LogicalNodeSerializer extends BasicLogicalPlanVisitor<LogicalNodeSe public static PlanProto.Target convertTarget(Target target) { PlanProto.Target.Builder targetBuilder = PlanProto.Target.newBuilder(); - targetBuilder.setExpr(EvalNodeSerializer.serialize(target.getEvalTree())); + targetBuilder.setExpr(EvalNodeSerializer.serialize((EvalNode) target.getEvalTree())); if (target.hasAlias()) { targetBuilder.setAlias(target.getAlias()); } http://git-wip-us.apache.org/repos/asf/tajo/blob/0ec2a890/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java ---------------------------------------------------------------------- diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java b/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java index 53ac00f..1d286b3 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java @@ -45,7 +45,7 @@ import java.util.*; public class PlannerUtil { public static final Column [] EMPTY_COLUMNS = new Column[] {}; - public static final AggregationFunctionCallEval [] EMPTY_AGG_FUNCS = new AggregationFunctionCallEval[] {}; + public static final List<AggregationFunctionCallEval> EMPTY_AGG_FUNCS = new ArrayList<>(); public static boolean checkIfSetSession(LogicalNode node) { LogicalNode baseNode = node;
