You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@calcite.apache.org by "Feng Zhu (Jira)" <ji...@apache.org> on 2020/03/30 02:14:00 UTC

[jira] [Commented] (CALCITE-3866) ReturnTypes.AGG_SUM may cause "numeric field overflow" on PostgreSQL when generate the sql after using the rule AggregateJoinTransposeRule.EXTENDED.

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

Feng Zhu commented on CALCITE-3866:
-----------------------------------

So, the issue is the incorrect type inference of AggregateJoinTransposeRule.EXTENDED rule?

By the way, could you improve the description from an end user's perspective?

>  ReturnTypes.AGG_SUM may cause "numeric field overflow" on PostgreSQL when generate the sql after using the rule AggregateJoinTransposeRule.EXTENDED.
> -----------------------------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: CALCITE-3866
>                 URL: https://issues.apache.org/jira/browse/CALCITE-3866
>             Project: Calcite
>          Issue Type: Bug
>            Reporter: TANG Wen-hui
>            Assignee: TANG Wen-hui
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 20m
>  Remaining Estimate: 0h
>
> When we try to generate a sql after applying AggregateJoinTransposeRule.EXTENDED, the result sql can not run on PostgreSQL, and throws the following exception:
> {code:java}
> PSQLException: ERROR: numeric field overflow Detail: A field with precision 7, scale 2 must round to an absolute value less than 10^5.
> {code}
> I found that the main reason is that :
> the return types of sum may have wrong precision when the type of 
>  its operand is the decimal with precision, for example:
> {code:java}
> @Test public void testSum() {
>   String query =
>       "select sum(e1.\"store_sales\"), sum(e2.\"store_sales\") from \"sales_fact_dec_1998\" as "
>           + "e1 , \"sales_fact_dec_1998\" as e2 where e1.\"product_id\" = e2.\"product_id\"";
>   String expect = "";
>   HepProgramBuilder builder = new HepProgramBuilder();
>   builder.addRuleClass(FilterJoinRule.class);
>   builder.addRuleClass(AggregateProjectMergeRule.class);
>   builder.addRuleClass(AggregateJoinTransposeRule.class);
>   HepPlanner hepPlanner = new HepPlanner(builder.build());
>   RuleSet rules = RuleSets.ofList(FilterJoinRule.FILTER_ON_JOIN, FilterJoinRule.JOIN,
>       AggregateProjectMergeRule.INSTANCE,
>       AggregateJoinTransposeRule.EXTENDED);
>   sql(query).withPostgresql().optimize(rules, hepPlanner).ok(expect);
> }
> {code}
> the result generated sql of the query is :
> {code:java}
> SELECT SUM(CAST(\"t\".\"EXPR$0\" * \"t0\".\"$f1\" AS DECIMAL(10, 4))), SUM(CAST(\"t\".\"$f2\" * \"t0\".\"EXPR$1\" AS DECIMAL(10, 4)))
> FROM (SELECT \"product_id\", SUM(\"store_sales\") AS \"EXPR$0\", COUNT(*) AS \"$f2\"
> FROM \"foodmart\".\"sales_fact_dec_1998\"
> GROUP BY \"product_id\") AS \"t\"
> INNER JOIN (SELECT \"product_id\", COUNT(*) AS \"$f1\", SUM(\"store_sales\") AS \"EXPR$1\"
> FROM \"foodmart\".\"sales_fact_dec_1998\"
> GROUP BY \"product_id\") AS \"t0\" ON \"t\".\"product_id\" = \"t0\".\"product_id\"
> {code}
> AggregateJoinTransposeRule.EXTENDED generates a Aggregate  to sum up the sub-totals:
> {code:java}
> // Aggregate above to sum up the sub-totals
> final List<AggregateCall> newAggCalls = new ArrayList<>();
> final int groupCount = aggregate.getGroupCount();
> final int newLeftWidth = sides.get(0).newInput.getRowType().getFieldCount();
> final List<RexNode> projects =
>     new ArrayList<>(
>         rexBuilder.identityProjects(relBuilder.peek().getRowType()));
> for (Ord<AggregateCall> aggCall : Ord.zip(aggregate.getAggCallList())) {
>   final SqlAggFunction aggregation = aggCall.e.getAggregation();
>   final SqlSplittableAggFunction splitter =
>       Objects.requireNonNull(
>           aggregation.unwrap(SqlSplittableAggFunction.class));
>   final Integer leftSubTotal = sides.get(0).split.get(aggCall.i);
>   final Integer rightSubTotal = sides.get(1).split.get(aggCall.i);
>   newAggCalls.add(
>       splitter.topSplit(rexBuilder, registry(projects),
>           groupCount, relBuilder.peek().getRowType(), aggCall.e,
>           leftSubTotal == null ? -1 : leftSubTotal,
>           rightSubTotal == null ? -1 : rightSubTotal + newLeftWidth));
> }
> public AggregateCall topSplit(RexBuilder rexBuilder,
>     Registry<RexNode> extra, int offset, RelDataType inputRowType,
>     AggregateCall aggregateCall, int leftSubTotal, int rightSubTotal) {
>   final List<RexNode> merges = new ArrayList<>();
>   final List<RelDataTypeField> fieldList = inputRowType.getFieldList();
>   if (leftSubTotal >= 0) {
>     final RelDataType type = fieldList.get(leftSubTotal).getType();
>     merges.add(rexBuilder.makeInputRef(type, leftSubTotal));
>   }
>   if (rightSubTotal >= 0) {
>     final RelDataType type = fieldList.get(rightSubTotal).getType();
>     merges.add(rexBuilder.makeInputRef(type, rightSubTotal));
>   }
>   RexNode node;
>   switch (merges.size()) {
>   case 1:
>     node = merges.get(0);
>     break;
>   case 2:
>     node = rexBuilder.makeCall(SqlStdOperatorTable.MULTIPLY, merges);
>     node = rexBuilder.makeAbstractCast(aggregateCall.type, node);
>     break;
>   default:
>     throw new AssertionError("unexpected count " + merges);
>   }
>   int ordinal = extra.register(node);
>   return AggregateCall.create(getMergeAggFunctionOfTopSplit(), false, false,
>       false, ImmutableList.of(ordinal), -1, aggregateCall.collation,
>       aggregateCall.type, aggregateCall.name);
> }
> {code}
> but it use the type of origin sum to generate a cast like 
> {code:java}
> SUM(CAST(\"t\".\"$f2\" * \"t0\".\"EXPR$1\" AS DECIMAL(10, 4))){code}
> ReturnTypes.AGG_SUM simplely use the operand type as its return type:
> {code:java}
> public static final SqlReturnTypeInference AGG_SUM = opBinding -> {
>   final RelDataTypeFactory typeFactory = opBinding.getTypeFactory();
>   final RelDataType type = typeFactory.getTypeSystem()
>       .deriveSumType(typeFactory, opBinding.getOperandType(0));
>   if (opBinding.getGroupCount() == 0 || opBinding.hasFilter()) {
>     return typeFactory.createTypeWithNullability(type, true);
>   } else {
>     return type;
>   }
> };
> @Override public RelDataType deriveSumType(RelDataTypeFactory typeFactory,
>     RelDataType argumentType) {
>   return argumentType;
> }
> {code}
> So the result of \"t\".\"$f2\" * \"t0\".\"EXPR$1\" may too large to cause overflow 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)