Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-5564 User Defined Aggregates
  3. FLINK-5767

New aggregate function interface and built-in aggregate functions

    Details

    • Type: Sub-task
    • Status: Closed
    • Priority: Major
    • Resolution: Implemented
    • Affects Version/s: None
    • Fix Version/s: 1.3.0
    • Component/s: Table API & SQL
    • Labels:
      None

      Description

      Add a new aggregate function interface. This includes implementing the aggregate interface, migrating the existing aggregation functions to this interface, and adding the unit tests for these functions.

        Issue Links

          Activity

          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user shaoxuan-wang opened a pull request:

          https://github.com/apache/flink/pull/3354

          FLINK-5767 [Table] New aggregate function interface and built-in aggregate functions

          Thanks for contributing to Apache Flink. Before you open your pull request, please take the following check list into consideration.
          If your changes take all of the items into account, feel free to open your pull request. For more information and/or questions please refer to the [How To Contribute guide](http://flink.apache.org/how-to-contribute.html).
          In addition to going through the list, please provide a meaningful description of your changes.

          • [X] General
          • The pull request references the related JIRA issue ("[FLINK-XXX] Jira title text")
          • The pull request addresses only one issue
          • Each commit in the PR has a meaningful commit message (including the JIRA id)
          • [ ] Documentation
          • Documentation has been added for new functionality
          • Old documentation affected by the pull request has been updated
          • JavaDoc for public methods has been added
          • [X] Tests & Build
          • Functionality added by the pull request is covered by tests
          • `mvn clean verify` has been executed successfully locally or a Travis build has passed

          You can merge this pull request into a Git repository by running:

          $ git pull https://github.com/shaoxuan-wang/flink F5767-submit

          Alternatively you can review and apply these changes as the patch at:

          https://github.com/apache/flink/pull/3354.patch

          To close this pull request, make a commit to your master/trunk branch
          with (at least) the following in the commit message:

          This closes #3354


          commit 6a2937a94fb2a00345fb5792a8c00c604fbee2c7
          Author: Shaoxuan Wang <wshaoxuan@gmail.com>
          Date: 2017-02-18T09:18:38Z

          FLINK-5767 [Table] New aggregate function interface and built-in aggregate functions


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user shaoxuan-wang opened a pull request: https://github.com/apache/flink/pull/3354 FLINK-5767 [Table] New aggregate function interface and built-in aggregate functions Thanks for contributing to Apache Flink. Before you open your pull request, please take the following check list into consideration. If your changes take all of the items into account, feel free to open your pull request. For more information and/or questions please refer to the [How To Contribute guide] ( http://flink.apache.org/how-to-contribute.html ). In addition to going through the list, please provide a meaningful description of your changes. [X] General The pull request references the related JIRA issue (" [FLINK-XXX] Jira title text") The pull request addresses only one issue Each commit in the PR has a meaningful commit message (including the JIRA id) [ ] Documentation Documentation has been added for new functionality Old documentation affected by the pull request has been updated JavaDoc for public methods has been added [X] Tests & Build Functionality added by the pull request is covered by tests `mvn clean verify` has been executed successfully locally or a Travis build has passed You can merge this pull request into a Git repository by running: $ git pull https://github.com/shaoxuan-wang/flink F5767-submit Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3354.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #3354 commit 6a2937a94fb2a00345fb5792a8c00c604fbee2c7 Author: Shaoxuan Wang <wshaoxuan@gmail.com> Date: 2017-02-18T09:18:38Z FLINK-5767 [Table] New aggregate function interface and built-in aggregate functions
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user shaoxuan-wang commented on the issue:

          https://github.com/apache/flink/pull/3354

          The travis-ci test build failed as it hits the Travis 50 min limit: "The job exceeded the maximum time limit for jobs, and has been terminated". Robert has a fix for this FLINK-5731. Hope this won't block the review process.

          Show
          githubbot ASF GitHub Bot added a comment - Github user shaoxuan-wang commented on the issue: https://github.com/apache/flink/pull/3354 The travis-ci test build failed as it hits the Travis 50 min limit: "The job exceeded the maximum time limit for jobs, and has been terminated". Robert has a fix for this FLINK-5731 . Hope this won't block the review process.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102225539

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/builtInAggFuncs/AvgAggFunction.scala —
          @@ -0,0 +1,268 @@
          +/*
          + * 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.flink.table.functions.builtInAggFuncs
          — End diff –

          Packages are all lowercase by convention.
          Rename to `aggfunctions`?

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102225539 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/builtInAggFuncs/AvgAggFunction.scala — @@ -0,0 +1,268 @@ +/* + * 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.flink.table.functions.builtInAggFuncs — End diff – Packages are all lowercase by convention. Rename to `aggfunctions`?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102431548

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/builtInAggFuncs/CountAggFunction.scala —
          @@ -0,0 +1,49 @@
          +/*
          + * 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.flink.table.functions.builtInAggFuncs
          +
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +/**
          + * built-in count aggregate function
          + */
          +class CountAggFunction extends AggregateFunction[Long] {
          + /** The initial accumulator for count aggregate function */
          + class CountAccumulator extends Accumulator {
          — End diff –

          Convert to `JTuple1`

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102431548 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/builtInAggFuncs/CountAggFunction.scala — @@ -0,0 +1,49 @@ +/* + * 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.flink.table.functions.builtInAggFuncs + +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + +/** + * built-in count aggregate function + */ +class CountAggFunction extends AggregateFunction [Long] { + /** The initial accumulator for count aggregate function */ + class CountAccumulator extends Accumulator { — End diff – Convert to `JTuple1`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102220814

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/AggregateFunction.scala —
          @@ -0,0 +1,73 @@
          +/*
          + * 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.flink.table.functions
          +
          +/**
          + * Base class for User-Defined Aggregates.
          + *
          + * @tparam T the type of the aggregation result
          + */
          +abstract class AggregateFunction[T] extends UserDefinedFunction {
          + /**
          + * Create and init the Accumulator for this [[AggregateFunction]].
          + *
          + * @return the accumulator with the initial value
          + */
          + def createAccumulator(): Accumulator
          +
          + /**
          + * Called every time when an aggregation result should be materialized.
          + * The returned value could be either a speculative result (periodically
          + * emitted as data arrive) or the final result of the aggregation (when
          + * the state of the aggregation is completely removed).
          — End diff –

          How would the function distinguish these cases?
          I think we should not mention that the state is removed because it is not the responsibility of the method.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102220814 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/AggregateFunction.scala — @@ -0,0 +1,73 @@ +/* + * 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.flink.table.functions + +/** + * Base class for User-Defined Aggregates. + * + * @tparam T the type of the aggregation result + */ +abstract class AggregateFunction [T] extends UserDefinedFunction { + /** + * Create and init the Accumulator for this [ [AggregateFunction] ]. + * + * @return the accumulator with the initial value + */ + def createAccumulator(): Accumulator + + /** + * Called every time when an aggregation result should be materialized. + * The returned value could be either a speculative result (periodically + * emitted as data arrive) or the final result of the aggregation (when + * the state of the aggregation is completely removed). — End diff – How would the function distinguish these cases? I think we should not mention that the state is removed because it is not the responsibility of the method.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102433532

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/builtInAggFuncs/SumAggFunction.scala —
          @@ -0,0 +1,146 @@
          +/*
          + * 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.flink.table.functions.builtInAggFuncs
          +
          +import java.math.BigDecimal
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +/**
          + * Base class for built-in Sum aggregate function
          + *
          + * @tparam T the type for the aggregation result
          + */
          +abstract class SumAggFunction[T: Numeric] extends AggregateFunction[T] {
          + /** The initial accumulator for Sum aggregate function */
          + class SumAccumulator[T] extends Accumulator {
          — End diff –

          Same as for the accumulator for `MaxAggFunction`.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102433532 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/builtInAggFuncs/SumAggFunction.scala — @@ -0,0 +1,146 @@ +/* + * 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.flink.table.functions.builtInAggFuncs + +import java.math.BigDecimal +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + +/** + * Base class for built-in Sum aggregate function + * + * @tparam T the type for the aggregation result + */ +abstract class SumAggFunction [T: Numeric] extends AggregateFunction [T] { + /** The initial accumulator for Sum aggregate function */ + class SumAccumulator [T] extends Accumulator { — End diff – Same as for the accumulator for `MaxAggFunction`.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102220328

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/AggregateFunction.scala —
          @@ -0,0 +1,73 @@
          +/*
          + * 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.flink.table.functions
          +
          +/**
          + * Base class for User-Defined Aggregates.
          + *
          + * @tparam T the type of the aggregation result
          + */
          +abstract class AggregateFunction[T] extends UserDefinedFunction {
          + /**
          + * Create and init the Accumulator for this [[AggregateFunction]].
          + *
          + * @return the accumulator with the initial value
          + */
          + def createAccumulator(): Accumulator
          +
          + /**
          + * Called every time when an aggregation result should be materialized.
          + * The returned value could be either a speculative result (periodically
          — End diff –

          `a speculative` -> `an early and incomplete`

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102220328 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/AggregateFunction.scala — @@ -0,0 +1,73 @@ +/* + * 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.flink.table.functions + +/** + * Base class for User-Defined Aggregates. + * + * @tparam T the type of the aggregation result + */ +abstract class AggregateFunction [T] extends UserDefinedFunction { + /** + * Create and init the Accumulator for this [ [AggregateFunction] ]. + * + * @return the accumulator with the initial value + */ + def createAccumulator(): Accumulator + + /** + * Called every time when an aggregation result should be materialized. + * The returned value could be either a speculative result (periodically — End diff – `a speculative` -> `an early and incomplete`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102241426

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/builtInAggFuncs/AvgAggFunction.scala —
          @@ -0,0 +1,268 @@
          +/*
          + * 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.flink.table.functions.builtInAggFuncs
          +
          +import java.math.

          {BigDecimal, BigInteger}

          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +/**
          + * Base class for built-in Integral Avg aggregate function
          + *
          + * @tparam T the type for the aggregation result
          + */
          +abstract class IntegralAvgAggFunction[T] extends AggregateFunction[T] {
          + /** The initial accumulator for Integral Avg aggregate function */
          + class IntegralAvgAccumulator extends Accumulator

          { + var sum: Long = 0 + var count: Long = 0 + }

          +
          + override def createAccumulator(): Accumulator =

          { + new IntegralAvgAccumulator + }

          +
          + override def accumulate(accumulator: Accumulator, value: Any) = {
          + if (value != null)

          { + val v = value.asInstanceOf[Number].longValue() + val accum = accumulator.asInstanceOf[IntegralAvgAccumulator] + accum.sum += v + accum.count += 1 + }

          + }
          +
          + override def getValue(accumulator: Accumulator): T = {
          + val accum = accumulator.asInstanceOf[IntegralAvgAccumulator]
          + val sum = accum.sum
          + if (accum.count == 0)

          { + null.asInstanceOf[T] + } else { + resultTypeConvert(accum.sum / accum.count) + }
          + }
          +
          + override def merge(a: Accumulator, b: Accumulator): Accumulator = { + val aAccum = a.asInstanceOf[IntegralAvgAccumulator] + val bAccum = b.asInstanceOf[IntegralAvgAccumulator] + aAccum.count += bAccum.count + aAccum.sum += bAccum.sum + a + }
          + /**
          + * Convert the intermediate result to the expected aggregation result type
          + *
          + * @param value the intermediate result. We use a Long container to save
          + * the intermediate result to avoid the overflow by sum operation.
          + * @return the result value with the expected aggregation result type
          + */
          + def resultTypeConvert(value: Long): T
          +}
          +
          +/**
          + * Built-in Byte Avg aggregate function
          + */
          +class ByteAvgAggFunction extends IntegralAvgAggFunction[Byte] { + override def resultTypeConvert(value: Long): Byte = value.toByte +}
          +
          +/**
          + * Built-in Short Avg aggregate function
          + */
          +class ShortAvgAggFunction extends IntegralAvgAggFunction[Short] { + override def resultTypeConvert(value: Long): Short = value.toShort +}
          +
          +/**
          + * Built-in Int Avg aggregate function
          + */
          +class IntAvgAggFunction extends IntegralAvgAggFunction[Int] { + override def resultTypeConvert(value: Long): Int = value.toInt +}
          +
          +/**
          + * Base Class for Built-in Big Integral Avg aggregate function
          + *
          + * @tparam T the type for the aggregation result
          + */
          +abstract class BigIntegralAvgAggFunction[T] extends AggregateFunction[T] {
          + /** The initial accumulator for Big Integral Avg aggregate function */
          + class BigIntegralAvgAccumulator extends Accumulator { + var sum: BigInteger = BigInteger.ZERO + var count: Long = 0 + }
          +
          + override def createAccumulator(): Accumulator = { + new BigIntegralAvgAccumulator + }
          +
          + override def accumulate(accumulator: Accumulator, value: Any) = {
          + if (value != null) { + val v = value.asInstanceOf[Long] + val accum = accumulator.asInstanceOf[BigIntegralAvgAccumulator] + accum.sum = accum.sum.add(BigInteger.valueOf(v)) + accum.count += 1 + }
          + }
          +
          + override def getValue(accumulator: Accumulator): T = {
          + val accum = accumulator.asInstanceOf[BigIntegralAvgAccumulator]
          + val sum = accum.sum
          + if (accum.count == 0) { + null.asInstanceOf[T] + }

          else

          { + resultTypeConvert(accum.sum.divide(BigInteger.valueOf(accum.count))) + }

          + }
          +
          + override def merge(a: Accumulator, b: Accumulator): Accumulator =

          { + val aAccum = a.asInstanceOf[BigIntegralAvgAccumulator] + val bAccum = b.asInstanceOf[BigIntegralAvgAccumulator] + aAccum.count += bAccum.count + aAccum.sum = aAccum.sum.add(bAccum.sum) + a + }

          +
          + /**
          + * Convert the intermediate result to the expected aggregation result type
          + *
          + * @param value the intermediate result. We use a BigInteger container to
          + * save the intermediate result to avoid the overflow by sum
          + * operation.
          + * @return the result value with the expected aggregation result type
          + */
          + def resultTypeConvert(value: BigInteger): T
          +}
          +
          +/**
          + * Built-in Long Avg aggregate function
          + */
          +class LongAvgAggFunction extends BigIntegralAvgAggFunction[Long]

          { + override def resultTypeConvert(value: BigInteger): Long = value.longValue() +}

          +
          +/**
          + * Base class for built-in Floating Avg aggregate function
          + *
          + * @tparam T the type for the aggregation result
          + */
          +abstract class FloatingAvgAggFunction[T] extends AggregateFunction[T] {
          + /** The initial accumulator for Floating Avg aggregate function */
          + class FloatingAvgAccumulator extends Accumulator

          { + var sum: Double = 0 + var count: Long = 0 + }

          +
          + override def createAccumulator(): Accumulator =

          { + new FloatingAvgAccumulator + }

          +
          + override def accumulate(accumulator: Accumulator, value: Any) = {
          + if (value != null)

          { + val v = value.asInstanceOf[Number].doubleValue() + val accum = accumulator.asInstanceOf[FloatingAvgAccumulator] + accum.sum += v + accum.count += 1 + }

          + }
          +
          + override def getValue(accumulator: Accumulator): T = {
          + val accum = accumulator.asInstanceOf[FloatingAvgAccumulator]
          + val sum = accum.sum
          + if (accum.count == 0)

          { + null.asInstanceOf[T] + }

          else

          { + resultTypeConvert(accum.sum / accum.count) + }

          + }
          +
          + override def merge(a: Accumulator, b: Accumulator): Accumulator =

          { + val aAccum = a.asInstanceOf[FloatingAvgAccumulator] + val bAccum = b.asInstanceOf[FloatingAvgAccumulator] + aAccum.count += bAccum.count + aAccum.sum += bAccum.sum + a + }

          +
          + /**
          + * Convert the intermediate result to the expected aggregation result type
          + *
          + * @param value the intermediate result. We use a Double container to save
          + * the intermediate result to avoid the overflow by sum operation.
          + * @return the result value with the expected aggregation result type
          + */
          + def resultTypeConvert(value: Double): T
          +}
          +
          +/**
          + * Built-in Float Avg aggregate function
          + */
          +class FloatAvgAggFunction extends FloatingAvgAggFunction[Float]

          { + override def resultTypeConvert(value: Double): Float = value.toFloat +}

          +
          +/**
          + * Built-in Int Double aggregate function
          + */
          +class DoubleAvgAggFunction extends FloatingAvgAggFunction[Double]

          { + override def resultTypeConvert(value: Double): Double = value +}

          +
          +/**
          + * Base class for built-in Big Decimal Avg aggregate function
          + */
          +class DecimalAvgAggFunction extends AggregateFunction[BigDecimal] {
          + /** The initial accumulator for Big Decimal Avg aggregate function */
          + class DecimalAvgAccumulator extends Accumulator

          { + var sum: BigDecimal = null + var count: Long = 0 + }

          +
          + override def createAccumulator(): Accumulator =

          { + new DecimalAvgAccumulator + }

          +
          + override def accumulate(accumulator: Accumulator, value: Any) = {
          + if (value != null) {
          + val v = value.asInstanceOf[BigDecimal]
          + val accum = accumulator.asInstanceOf[DecimalAvgAccumulator]
          + accum.count += 1
          + if (accum.sum == null)

          { + accum.sum = v + }

          else

          { + accum.sum = accum.sum.add(v) + }

          + }
          + }
          +
          + override def getValue(accumulator: Accumulator): BigDecimal = {
          + val sum = accumulator.asInstanceOf[DecimalAvgAccumulator].sum
          + val count = accumulator.asInstanceOf[DecimalAvgAccumulator].count
          + if (sum == null || count == 0)

          { + null.asInstanceOf[BigDecimal] + }

          else

          { + sum.divide(BigDecimal.valueOf(count)) + }

          + }
          +
          + override def merge(a: Accumulator, b: Accumulator): Accumulator = {
          + val aAccum = a.asInstanceOf[DecimalAvgAccumulator]
          + val bAccum = b.asInstanceOf[DecimalAvgAccumulator]
          + aAccum.count += bAccum.count
          + accumulate(a, b.asInstanceOf[DecimalAvgAccumulator].sum)
          — End diff –

          Won't `count` be off by one because `accumulate` will increment `count` as well?

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102241426 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/builtInAggFuncs/AvgAggFunction.scala — @@ -0,0 +1,268 @@ +/* + * 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.flink.table.functions.builtInAggFuncs + +import java.math. {BigDecimal, BigInteger} +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + +/** + * Base class for built-in Integral Avg aggregate function + * + * @tparam T the type for the aggregation result + */ +abstract class IntegralAvgAggFunction [T] extends AggregateFunction [T] { + /** The initial accumulator for Integral Avg aggregate function */ + class IntegralAvgAccumulator extends Accumulator { + var sum: Long = 0 + var count: Long = 0 + } + + override def createAccumulator(): Accumulator = { + new IntegralAvgAccumulator + } + + override def accumulate(accumulator: Accumulator, value: Any) = { + if (value != null) { + val v = value.asInstanceOf[Number].longValue() + val accum = accumulator.asInstanceOf[IntegralAvgAccumulator] + accum.sum += v + accum.count += 1 + } + } + + override def getValue(accumulator: Accumulator): T = { + val accum = accumulator.asInstanceOf [IntegralAvgAccumulator] + val sum = accum.sum + if (accum.count == 0) { + null.asInstanceOf[T] + } else { + resultTypeConvert(accum.sum / accum.count) + } + } + + override def merge(a: Accumulator, b: Accumulator): Accumulator = { + val aAccum = a.asInstanceOf[IntegralAvgAccumulator] + val bAccum = b.asInstanceOf[IntegralAvgAccumulator] + aAccum.count += bAccum.count + aAccum.sum += bAccum.sum + a + } + /** + * Convert the intermediate result to the expected aggregation result type + * + * @param value the intermediate result. We use a Long container to save + * the intermediate result to avoid the overflow by sum operation. + * @return the result value with the expected aggregation result type + */ + def resultTypeConvert(value: Long): T +} + +/** + * Built-in Byte Avg aggregate function + */ +class ByteAvgAggFunction extends IntegralAvgAggFunction [Byte] { + override def resultTypeConvert(value: Long): Byte = value.toByte +} + +/** + * Built-in Short Avg aggregate function + */ +class ShortAvgAggFunction extends IntegralAvgAggFunction [Short] { + override def resultTypeConvert(value: Long): Short = value.toShort +} + +/** + * Built-in Int Avg aggregate function + */ +class IntAvgAggFunction extends IntegralAvgAggFunction [Int] { + override def resultTypeConvert(value: Long): Int = value.toInt +} + +/** + * Base Class for Built-in Big Integral Avg aggregate function + * + * @tparam T the type for the aggregation result + */ +abstract class BigIntegralAvgAggFunction [T] extends AggregateFunction [T] { + /** The initial accumulator for Big Integral Avg aggregate function */ + class BigIntegralAvgAccumulator extends Accumulator { + var sum: BigInteger = BigInteger.ZERO + var count: Long = 0 + } + + override def createAccumulator(): Accumulator = { + new BigIntegralAvgAccumulator + } + + override def accumulate(accumulator: Accumulator, value: Any) = { + if (value != null) { + val v = value.asInstanceOf[Long] + val accum = accumulator.asInstanceOf[BigIntegralAvgAccumulator] + accum.sum = accum.sum.add(BigInteger.valueOf(v)) + accum.count += 1 + } + } + + override def getValue(accumulator: Accumulator): T = { + val accum = accumulator.asInstanceOf [BigIntegralAvgAccumulator] + val sum = accum.sum + if (accum.count == 0) { + null.asInstanceOf[T] + } else { + resultTypeConvert(accum.sum.divide(BigInteger.valueOf(accum.count))) + } + } + + override def merge(a: Accumulator, b: Accumulator): Accumulator = { + val aAccum = a.asInstanceOf[BigIntegralAvgAccumulator] + val bAccum = b.asInstanceOf[BigIntegralAvgAccumulator] + aAccum.count += bAccum.count + aAccum.sum = aAccum.sum.add(bAccum.sum) + a + } + + /** + * Convert the intermediate result to the expected aggregation result type + * + * @param value the intermediate result. We use a BigInteger container to + * save the intermediate result to avoid the overflow by sum + * operation. + * @return the result value with the expected aggregation result type + */ + def resultTypeConvert(value: BigInteger): T +} + +/** + * Built-in Long Avg aggregate function + */ +class LongAvgAggFunction extends BigIntegralAvgAggFunction [Long] { + override def resultTypeConvert(value: BigInteger): Long = value.longValue() +} + +/** + * Base class for built-in Floating Avg aggregate function + * + * @tparam T the type for the aggregation result + */ +abstract class FloatingAvgAggFunction [T] extends AggregateFunction [T] { + /** The initial accumulator for Floating Avg aggregate function */ + class FloatingAvgAccumulator extends Accumulator { + var sum: Double = 0 + var count: Long = 0 + } + + override def createAccumulator(): Accumulator = { + new FloatingAvgAccumulator + } + + override def accumulate(accumulator: Accumulator, value: Any) = { + if (value != null) { + val v = value.asInstanceOf[Number].doubleValue() + val accum = accumulator.asInstanceOf[FloatingAvgAccumulator] + accum.sum += v + accum.count += 1 + } + } + + override def getValue(accumulator: Accumulator): T = { + val accum = accumulator.asInstanceOf [FloatingAvgAccumulator] + val sum = accum.sum + if (accum.count == 0) { + null.asInstanceOf[T] + } else { + resultTypeConvert(accum.sum / accum.count) + } + } + + override def merge(a: Accumulator, b: Accumulator): Accumulator = { + val aAccum = a.asInstanceOf[FloatingAvgAccumulator] + val bAccum = b.asInstanceOf[FloatingAvgAccumulator] + aAccum.count += bAccum.count + aAccum.sum += bAccum.sum + a + } + + /** + * Convert the intermediate result to the expected aggregation result type + * + * @param value the intermediate result. We use a Double container to save + * the intermediate result to avoid the overflow by sum operation. + * @return the result value with the expected aggregation result type + */ + def resultTypeConvert(value: Double): T +} + +/** + * Built-in Float Avg aggregate function + */ +class FloatAvgAggFunction extends FloatingAvgAggFunction [Float] { + override def resultTypeConvert(value: Double): Float = value.toFloat +} + +/** + * Built-in Int Double aggregate function + */ +class DoubleAvgAggFunction extends FloatingAvgAggFunction [Double] { + override def resultTypeConvert(value: Double): Double = value +} + +/** + * Base class for built-in Big Decimal Avg aggregate function + */ +class DecimalAvgAggFunction extends AggregateFunction [BigDecimal] { + /** The initial accumulator for Big Decimal Avg aggregate function */ + class DecimalAvgAccumulator extends Accumulator { + var sum: BigDecimal = null + var count: Long = 0 + } + + override def createAccumulator(): Accumulator = { + new DecimalAvgAccumulator + } + + override def accumulate(accumulator: Accumulator, value: Any) = { + if (value != null) { + val v = value.asInstanceOf [BigDecimal] + val accum = accumulator.asInstanceOf [DecimalAvgAccumulator] + accum.count += 1 + if (accum.sum == null) { + accum.sum = v + } else { + accum.sum = accum.sum.add(v) + } + } + } + + override def getValue(accumulator: Accumulator): BigDecimal = { + val sum = accumulator.asInstanceOf [DecimalAvgAccumulator] .sum + val count = accumulator.asInstanceOf [DecimalAvgAccumulator] .count + if (sum == null || count == 0) { + null.asInstanceOf[BigDecimal] + } else { + sum.divide(BigDecimal.valueOf(count)) + } + } + + override def merge(a: Accumulator, b: Accumulator): Accumulator = { + val aAccum = a.asInstanceOf [DecimalAvgAccumulator] + val bAccum = b.asInstanceOf [DecimalAvgAccumulator] + aAccum.count += bAccum.count + accumulate(a, b.asInstanceOf [DecimalAvgAccumulator] .sum) — End diff – Won't `count` be off by one because `accumulate` will increment `count` as well?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102265663

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/builtInAggFuncs/SumAggFunction.scala —
          @@ -0,0 +1,146 @@
          +/*
          + * 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.flink.table.functions.builtInAggFuncs
          +
          +import java.math.BigDecimal
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +/**
          + * Base class for built-in Sum aggregate function
          + *
          + * @tparam T the type for the aggregation result
          + */
          +abstract class SumAggFunction[T: Numeric] extends AggregateFunction[T] {
          + /** The initial accumulator for Sum aggregate function */
          + class SumAccumulator[T] extends Accumulator {
          + var sum: Option[T] = None
          — End diff –

          why working with `Option` here? The other functions where initialized with `null`. We should keep that consistent, IMO.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102265663 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/builtInAggFuncs/SumAggFunction.scala — @@ -0,0 +1,146 @@ +/* + * 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.flink.table.functions.builtInAggFuncs + +import java.math.BigDecimal +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + +/** + * Base class for built-in Sum aggregate function + * + * @tparam T the type for the aggregation result + */ +abstract class SumAggFunction [T: Numeric] extends AggregateFunction [T] { + /** The initial accumulator for Sum aggregate function */ + class SumAccumulator [T] extends Accumulator { + var sum: Option [T] = None — End diff – why working with `Option` here? The other functions where initialized with `null`. We should keep that consistent, IMO.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102433475

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/builtInAggFuncs/MinAggFunction.scala —
          @@ -0,0 +1,107 @@
          +/*
          + * 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.flink.table.functions.builtInAggFuncs
          +
          +import java.math.BigDecimal
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +/**
          + * Base class for built-in Min aggregate function
          + *
          + * @tparam T the type for the aggregation result
          + */
          +abstract class MinAggFunction[T](implicit ord: Ordering[T]) extends AggregateFunction[T] {
          + /** The initial accumulator for Min aggregate function */
          + class MinAccumulator[T] extends Accumulator {
          — End diff –

          Same as for the accumulator for `MaxAggFunction`.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102433475 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/builtInAggFuncs/MinAggFunction.scala — @@ -0,0 +1,107 @@ +/* + * 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.flink.table.functions.builtInAggFuncs + +import java.math.BigDecimal +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + +/** + * Base class for built-in Min aggregate function + * + * @tparam T the type for the aggregation result + */ +abstract class MinAggFunction [T] (implicit ord: Ordering [T] ) extends AggregateFunction [T] { + /** The initial accumulator for Min aggregate function */ + class MinAccumulator [T] extends Accumulator { — End diff – Same as for the accumulator for `MaxAggFunction`.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102220960

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/AggregateFunction.scala —
          @@ -0,0 +1,73 @@
          +/*
          + * 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.flink.table.functions
          +
          +/**
          + * Base class for User-Defined Aggregates.
          — End diff –

          Mention that all state of the function must be put into the accumulator?

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102220960 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/AggregateFunction.scala — @@ -0,0 +1,73 @@ +/* + * 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.flink.table.functions + +/** + * Base class for User-Defined Aggregates. — End diff – Mention that all state of the function must be put into the accumulator?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102432444

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/builtInAggFuncs/MaxAggFunction.scala —
          @@ -0,0 +1,107 @@
          +/*
          + * 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.flink.table.functions.builtInAggFuncs
          +
          +import java.math.BigDecimal
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +/**
          + * Base class for built-in Max aggregate function
          + *
          + * @tparam T the type for the aggregation result
          + */
          +abstract class MaxAggFunction[T](implicit ord: Ordering[T]) extends AggregateFunction[T] {
          + /** The initial accumulator for Max aggregate function */
          + class MaxAccumulator[T] extends Accumulator {
          — End diff –

          Make this a `class MaxAccumulator[T] extends JTuple2[T, Boolean] with Accumulator`?
          We need the additional boolean field as a flag to indicate null values. Java tuples do not support null values, so the first field must be initialized but should be considered null.
          This does not add space overhead, since a PojoSerializer would also serialize a `byte` to indicate null valued fields.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102432444 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/builtInAggFuncs/MaxAggFunction.scala — @@ -0,0 +1,107 @@ +/* + * 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.flink.table.functions.builtInAggFuncs + +import java.math.BigDecimal +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + +/** + * Base class for built-in Max aggregate function + * + * @tparam T the type for the aggregation result + */ +abstract class MaxAggFunction [T] (implicit ord: Ordering [T] ) extends AggregateFunction [T] { + /** The initial accumulator for Max aggregate function */ + class MaxAccumulator [T] extends Accumulator { — End diff – Make this a `class MaxAccumulator [T] extends JTuple2 [T, Boolean] with Accumulator`? We need the additional boolean field as a flag to indicate null values. Java tuples do not support null values, so the first field must be initialized but should be considered null. This does not add space overhead, since a PojoSerializer would also serialize a `byte` to indicate null valued fields.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102431482

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/builtInAggFuncs/AvgAggFunction.scala —
          @@ -0,0 +1,268 @@
          +/*
          + * 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.flink.table.functions.builtInAggFuncs
          +
          +import java.math.

          {BigDecimal, BigInteger}

          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +/**
          + * Base class for built-in Integral Avg aggregate function
          + *
          + * @tparam T the type for the aggregation result
          + */
          +abstract class IntegralAvgAggFunction[T] extends AggregateFunction[T] {
          + /** The initial accumulator for Integral Avg aggregate function */
          + class IntegralAvgAccumulator extends Accumulator

          { + var sum: Long = 0 + var count: Long = 0 + }

          +
          + override def createAccumulator(): Accumulator =

          { + new IntegralAvgAccumulator + }

          +
          + override def accumulate(accumulator: Accumulator, value: Any) = {
          + if (value != null)

          { + val v = value.asInstanceOf[Number].longValue() + val accum = accumulator.asInstanceOf[IntegralAvgAccumulator] + accum.sum += v + accum.count += 1 + }

          + }
          +
          + override def getValue(accumulator: Accumulator): T = {
          + val accum = accumulator.asInstanceOf[IntegralAvgAccumulator]
          + val sum = accum.sum
          + if (accum.count == 0)

          { + null.asInstanceOf[T] + } else { + resultTypeConvert(accum.sum / accum.count) + }
          + }
          +
          + override def merge(a: Accumulator, b: Accumulator): Accumulator = { + val aAccum = a.asInstanceOf[IntegralAvgAccumulator] + val bAccum = b.asInstanceOf[IntegralAvgAccumulator] + aAccum.count += bAccum.count + aAccum.sum += bAccum.sum + a + }
          + /**
          + * Convert the intermediate result to the expected aggregation result type
          + *
          + * @param value the intermediate result. We use a Long container to save
          + * the intermediate result to avoid the overflow by sum operation.
          + * @return the result value with the expected aggregation result type
          + */
          + def resultTypeConvert(value: Long): T
          +}
          +
          +/**
          + * Built-in Byte Avg aggregate function
          + */
          +class ByteAvgAggFunction extends IntegralAvgAggFunction[Byte] { + override def resultTypeConvert(value: Long): Byte = value.toByte +}
          +
          +/**
          + * Built-in Short Avg aggregate function
          + */
          +class ShortAvgAggFunction extends IntegralAvgAggFunction[Short] { + override def resultTypeConvert(value: Long): Short = value.toShort +}
          +
          +/**
          + * Built-in Int Avg aggregate function
          + */
          +class IntAvgAggFunction extends IntegralAvgAggFunction[Int] { + override def resultTypeConvert(value: Long): Int = value.toInt +}
          +
          +/**
          + * Base Class for Built-in Big Integral Avg aggregate function
          + *
          + * @tparam T the type for the aggregation result
          + */
          +abstract class BigIntegralAvgAggFunction[T] extends AggregateFunction[T] {
          + /** The initial accumulator for Big Integral Avg aggregate function */
          + class BigIntegralAvgAccumulator extends Accumulator { + var sum: BigInteger = BigInteger.ZERO + var count: Long = 0 + }
          +
          + override def createAccumulator(): Accumulator = { + new BigIntegralAvgAccumulator + }
          +
          + override def accumulate(accumulator: Accumulator, value: Any) = {
          + if (value != null) { + val v = value.asInstanceOf[Long] + val accum = accumulator.asInstanceOf[BigIntegralAvgAccumulator] + accum.sum = accum.sum.add(BigInteger.valueOf(v)) + accum.count += 1 + }
          + }
          +
          + override def getValue(accumulator: Accumulator): T = {
          + val accum = accumulator.asInstanceOf[BigIntegralAvgAccumulator]
          + val sum = accum.sum
          + if (accum.count == 0) { + null.asInstanceOf[T] + }

          else

          { + resultTypeConvert(accum.sum.divide(BigInteger.valueOf(accum.count))) + }

          + }
          +
          + override def merge(a: Accumulator, b: Accumulator): Accumulator =

          { + val aAccum = a.asInstanceOf[BigIntegralAvgAccumulator] + val bAccum = b.asInstanceOf[BigIntegralAvgAccumulator] + aAccum.count += bAccum.count + aAccum.sum = aAccum.sum.add(bAccum.sum) + a + }

          +
          + /**
          + * Convert the intermediate result to the expected aggregation result type
          + *
          + * @param value the intermediate result. We use a BigInteger container to
          + * save the intermediate result to avoid the overflow by sum
          + * operation.
          + * @return the result value with the expected aggregation result type
          + */
          + def resultTypeConvert(value: BigInteger): T
          +}
          +
          +/**
          + * Built-in Long Avg aggregate function
          + */
          +class LongAvgAggFunction extends BigIntegralAvgAggFunction[Long]

          { + override def resultTypeConvert(value: BigInteger): Long = value.longValue() +}

          +
          +/**
          + * Base class for built-in Floating Avg aggregate function
          + *
          + * @tparam T the type for the aggregation result
          + */
          +abstract class FloatingAvgAggFunction[T] extends AggregateFunction[T] {
          + /** The initial accumulator for Floating Avg aggregate function */
          + class FloatingAvgAccumulator extends Accumulator

          { + var sum: Double = 0 + var count: Long = 0 + }

          +
          + override def createAccumulator(): Accumulator =

          { + new FloatingAvgAccumulator + }

          +
          + override def accumulate(accumulator: Accumulator, value: Any) = {
          + if (value != null)

          { + val v = value.asInstanceOf[Number].doubleValue() + val accum = accumulator.asInstanceOf[FloatingAvgAccumulator] + accum.sum += v + accum.count += 1 + }

          + }
          +
          + override def getValue(accumulator: Accumulator): T = {
          + val accum = accumulator.asInstanceOf[FloatingAvgAccumulator]
          + val sum = accum.sum
          + if (accum.count == 0)

          { + null.asInstanceOf[T] + }

          else

          { + resultTypeConvert(accum.sum / accum.count) + }

          + }
          +
          + override def merge(a: Accumulator, b: Accumulator): Accumulator =

          { + val aAccum = a.asInstanceOf[FloatingAvgAccumulator] + val bAccum = b.asInstanceOf[FloatingAvgAccumulator] + aAccum.count += bAccum.count + aAccum.sum += bAccum.sum + a + }

          +
          + /**
          + * Convert the intermediate result to the expected aggregation result type
          + *
          + * @param value the intermediate result. We use a Double container to save
          + * the intermediate result to avoid the overflow by sum operation.
          + * @return the result value with the expected aggregation result type
          + */
          + def resultTypeConvert(value: Double): T
          +}
          +
          +/**
          + * Built-in Float Avg aggregate function
          + */
          +class FloatAvgAggFunction extends FloatingAvgAggFunction[Float]

          { + override def resultTypeConvert(value: Double): Float = value.toFloat +}

          +
          +/**
          + * Built-in Int Double aggregate function
          + */
          +class DoubleAvgAggFunction extends FloatingAvgAggFunction[Double]

          { + override def resultTypeConvert(value: Double): Double = value +}

          +
          +/**
          + * Base class for built-in Big Decimal Avg aggregate function
          + */
          +class DecimalAvgAggFunction extends AggregateFunction[BigDecimal] {
          + /** The initial accumulator for Big Decimal Avg aggregate function */
          + class DecimalAvgAccumulator extends Accumulator

          { + var sum: BigDecimal = null + var count: Long = 0 + }

          +
          + override def createAccumulator(): Accumulator =

          { + new DecimalAvgAccumulator + }

          +
          + override def accumulate(accumulator: Accumulator, value: Any) = {
          + if (value != null) {
          + val v = value.asInstanceOf[BigDecimal]
          + val accum = accumulator.asInstanceOf[DecimalAvgAccumulator]
          + accum.count += 1
          + if (accum.sum == null)

          { + accum.sum = v + }

          else

          { + accum.sum = accum.sum.add(v) + }

          + }
          + }
          +
          + override def getValue(accumulator: Accumulator): BigDecimal = {
          + val sum = accumulator.asInstanceOf[DecimalAvgAccumulator].sum
          + val count = accumulator.asInstanceOf[DecimalAvgAccumulator].count
          + if (sum == null || count == 0)

          { + null.asInstanceOf[BigDecimal] + }

          else

          { + sum.divide(BigDecimal.valueOf(count)) + }

          + }
          +
          + override def merge(a: Accumulator, b: Accumulator): Accumulator = {
          + val aAccum = a.asInstanceOf[DecimalAvgAccumulator]
          + val bAccum = b.asInstanceOf[DecimalAvgAccumulator]
          + aAccum.count += bAccum.count
          + accumulate(a, b.asInstanceOf[DecimalAvgAccumulator].sum)
          — End diff –

          This should have been caught by a test. We should revisit the tests and probably add more test data.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102431482 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/builtInAggFuncs/AvgAggFunction.scala — @@ -0,0 +1,268 @@ +/* + * 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.flink.table.functions.builtInAggFuncs + +import java.math. {BigDecimal, BigInteger} +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + +/** + * Base class for built-in Integral Avg aggregate function + * + * @tparam T the type for the aggregation result + */ +abstract class IntegralAvgAggFunction [T] extends AggregateFunction [T] { + /** The initial accumulator for Integral Avg aggregate function */ + class IntegralAvgAccumulator extends Accumulator { + var sum: Long = 0 + var count: Long = 0 + } + + override def createAccumulator(): Accumulator = { + new IntegralAvgAccumulator + } + + override def accumulate(accumulator: Accumulator, value: Any) = { + if (value != null) { + val v = value.asInstanceOf[Number].longValue() + val accum = accumulator.asInstanceOf[IntegralAvgAccumulator] + accum.sum += v + accum.count += 1 + } + } + + override def getValue(accumulator: Accumulator): T = { + val accum = accumulator.asInstanceOf [IntegralAvgAccumulator] + val sum = accum.sum + if (accum.count == 0) { + null.asInstanceOf[T] + } else { + resultTypeConvert(accum.sum / accum.count) + } + } + + override def merge(a: Accumulator, b: Accumulator): Accumulator = { + val aAccum = a.asInstanceOf[IntegralAvgAccumulator] + val bAccum = b.asInstanceOf[IntegralAvgAccumulator] + aAccum.count += bAccum.count + aAccum.sum += bAccum.sum + a + } + /** + * Convert the intermediate result to the expected aggregation result type + * + * @param value the intermediate result. We use a Long container to save + * the intermediate result to avoid the overflow by sum operation. + * @return the result value with the expected aggregation result type + */ + def resultTypeConvert(value: Long): T +} + +/** + * Built-in Byte Avg aggregate function + */ +class ByteAvgAggFunction extends IntegralAvgAggFunction [Byte] { + override def resultTypeConvert(value: Long): Byte = value.toByte +} + +/** + * Built-in Short Avg aggregate function + */ +class ShortAvgAggFunction extends IntegralAvgAggFunction [Short] { + override def resultTypeConvert(value: Long): Short = value.toShort +} + +/** + * Built-in Int Avg aggregate function + */ +class IntAvgAggFunction extends IntegralAvgAggFunction [Int] { + override def resultTypeConvert(value: Long): Int = value.toInt +} + +/** + * Base Class for Built-in Big Integral Avg aggregate function + * + * @tparam T the type for the aggregation result + */ +abstract class BigIntegralAvgAggFunction [T] extends AggregateFunction [T] { + /** The initial accumulator for Big Integral Avg aggregate function */ + class BigIntegralAvgAccumulator extends Accumulator { + var sum: BigInteger = BigInteger.ZERO + var count: Long = 0 + } + + override def createAccumulator(): Accumulator = { + new BigIntegralAvgAccumulator + } + + override def accumulate(accumulator: Accumulator, value: Any) = { + if (value != null) { + val v = value.asInstanceOf[Long] + val accum = accumulator.asInstanceOf[BigIntegralAvgAccumulator] + accum.sum = accum.sum.add(BigInteger.valueOf(v)) + accum.count += 1 + } + } + + override def getValue(accumulator: Accumulator): T = { + val accum = accumulator.asInstanceOf [BigIntegralAvgAccumulator] + val sum = accum.sum + if (accum.count == 0) { + null.asInstanceOf[T] + } else { + resultTypeConvert(accum.sum.divide(BigInteger.valueOf(accum.count))) + } + } + + override def merge(a: Accumulator, b: Accumulator): Accumulator = { + val aAccum = a.asInstanceOf[BigIntegralAvgAccumulator] + val bAccum = b.asInstanceOf[BigIntegralAvgAccumulator] + aAccum.count += bAccum.count + aAccum.sum = aAccum.sum.add(bAccum.sum) + a + } + + /** + * Convert the intermediate result to the expected aggregation result type + * + * @param value the intermediate result. We use a BigInteger container to + * save the intermediate result to avoid the overflow by sum + * operation. + * @return the result value with the expected aggregation result type + */ + def resultTypeConvert(value: BigInteger): T +} + +/** + * Built-in Long Avg aggregate function + */ +class LongAvgAggFunction extends BigIntegralAvgAggFunction [Long] { + override def resultTypeConvert(value: BigInteger): Long = value.longValue() +} + +/** + * Base class for built-in Floating Avg aggregate function + * + * @tparam T the type for the aggregation result + */ +abstract class FloatingAvgAggFunction [T] extends AggregateFunction [T] { + /** The initial accumulator for Floating Avg aggregate function */ + class FloatingAvgAccumulator extends Accumulator { + var sum: Double = 0 + var count: Long = 0 + } + + override def createAccumulator(): Accumulator = { + new FloatingAvgAccumulator + } + + override def accumulate(accumulator: Accumulator, value: Any) = { + if (value != null) { + val v = value.asInstanceOf[Number].doubleValue() + val accum = accumulator.asInstanceOf[FloatingAvgAccumulator] + accum.sum += v + accum.count += 1 + } + } + + override def getValue(accumulator: Accumulator): T = { + val accum = accumulator.asInstanceOf [FloatingAvgAccumulator] + val sum = accum.sum + if (accum.count == 0) { + null.asInstanceOf[T] + } else { + resultTypeConvert(accum.sum / accum.count) + } + } + + override def merge(a: Accumulator, b: Accumulator): Accumulator = { + val aAccum = a.asInstanceOf[FloatingAvgAccumulator] + val bAccum = b.asInstanceOf[FloatingAvgAccumulator] + aAccum.count += bAccum.count + aAccum.sum += bAccum.sum + a + } + + /** + * Convert the intermediate result to the expected aggregation result type + * + * @param value the intermediate result. We use a Double container to save + * the intermediate result to avoid the overflow by sum operation. + * @return the result value with the expected aggregation result type + */ + def resultTypeConvert(value: Double): T +} + +/** + * Built-in Float Avg aggregate function + */ +class FloatAvgAggFunction extends FloatingAvgAggFunction [Float] { + override def resultTypeConvert(value: Double): Float = value.toFloat +} + +/** + * Built-in Int Double aggregate function + */ +class DoubleAvgAggFunction extends FloatingAvgAggFunction [Double] { + override def resultTypeConvert(value: Double): Double = value +} + +/** + * Base class for built-in Big Decimal Avg aggregate function + */ +class DecimalAvgAggFunction extends AggregateFunction [BigDecimal] { + /** The initial accumulator for Big Decimal Avg aggregate function */ + class DecimalAvgAccumulator extends Accumulator { + var sum: BigDecimal = null + var count: Long = 0 + } + + override def createAccumulator(): Accumulator = { + new DecimalAvgAccumulator + } + + override def accumulate(accumulator: Accumulator, value: Any) = { + if (value != null) { + val v = value.asInstanceOf [BigDecimal] + val accum = accumulator.asInstanceOf [DecimalAvgAccumulator] + accum.count += 1 + if (accum.sum == null) { + accum.sum = v + } else { + accum.sum = accum.sum.add(v) + } + } + } + + override def getValue(accumulator: Accumulator): BigDecimal = { + val sum = accumulator.asInstanceOf [DecimalAvgAccumulator] .sum + val count = accumulator.asInstanceOf [DecimalAvgAccumulator] .count + if (sum == null || count == 0) { + null.asInstanceOf[BigDecimal] + } else { + sum.divide(BigDecimal.valueOf(count)) + } + } + + override def merge(a: Accumulator, b: Accumulator): Accumulator = { + val aAccum = a.asInstanceOf [DecimalAvgAccumulator] + val bAccum = b.asInstanceOf [DecimalAvgAccumulator] + aAccum.count += bAccum.count + accumulate(a, b.asInstanceOf [DecimalAvgAccumulator] .sum) — End diff – This should have been caught by a test. We should revisit the tests and probably add more test data.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102455635

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/functions/builtInAggFuncs/AggFunctionTestBase.scala —
          @@ -0,0 +1,104 @@
          +/*
          + * 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.flink.table.functions.builtInAggFuncs
          +
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +import java.math.BigDecimal
          +
          +import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils._
          +import org.apache.flink.types.Row
          +import org.junit.Test
          +import org.junit.Assert.assertEquals
          +
          +/**
          + * Base class for aggregate function test
          + *
          + * @tparam T the type for the aggregation result
          + */
          +abstract class AggFunctionTestBase[T] {
          + private val offset = 2
          — End diff –

          None of the tests uses `offset`, `numOfAggregates`, and `rowArity` or `Row`.

          I think the tests can be simplified by removing these and the methods `getResult()` (directly call `getValue()`) and `mergeRows()` (directly call `merge()`).

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102455635 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/functions/builtInAggFuncs/AggFunctionTestBase.scala — @@ -0,0 +1,104 @@ +/* + * 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.flink.table.functions.builtInAggFuncs + +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} +import java.math.BigDecimal + +import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils._ +import org.apache.flink.types.Row +import org.junit.Test +import org.junit.Assert.assertEquals + +/** + * Base class for aggregate function test + * + * @tparam T the type for the aggregation result + */ +abstract class AggFunctionTestBase [T] { + private val offset = 2 — End diff – None of the tests uses `offset`, `numOfAggregates`, and `rowArity` or `Row`. I think the tests can be simplified by removing these and the methods `getResult()` (directly call `getValue()`) and `mergeRows()` (directly call `merge()`).
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102226285

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/builtInAggFuncs/AvgAggFunction.scala —
          @@ -0,0 +1,268 @@
          +/*
          + * 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.flink.table.functions.builtInAggFuncs
          +
          +import java.math.

          {BigDecimal, BigInteger}

          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +/**
          + * Base class for built-in Integral Avg aggregate function
          + *
          + * @tparam T the type for the aggregation result
          + */
          +abstract class IntegralAvgAggFunction[T] extends AggregateFunction[T] {
          + /** The initial accumulator for Integral Avg aggregate function */
          + class IntegralAvgAccumulator extends Accumulator {
          — End diff –

          This will be probably handled as a Pojo type which is less efficient than a `Tuple2`
          We can define this as

          ```
          import org.apache.flink.api.java.tuple.

          {Tuple2 => JTuple2}

          // use Java tuple because it's mutable.

          class IntegralAvgAccumulator extends JTuple2[Long, Long] with Accumulator
          ```

          which means that Flink can handle it as a Tuple which is more efficient than a POJO which accesses fields (for de/serialization) via reflection.

          I think this implementation will also be closer to the code-gen'd version.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102226285 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/builtInAggFuncs/AvgAggFunction.scala — @@ -0,0 +1,268 @@ +/* + * 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.flink.table.functions.builtInAggFuncs + +import java.math. {BigDecimal, BigInteger} +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + +/** + * Base class for built-in Integral Avg aggregate function + * + * @tparam T the type for the aggregation result + */ +abstract class IntegralAvgAggFunction [T] extends AggregateFunction [T] { + /** The initial accumulator for Integral Avg aggregate function */ + class IntegralAvgAccumulator extends Accumulator { — End diff – This will be probably handled as a Pojo type which is less efficient than a `Tuple2` We can define this as ``` import org.apache.flink.api.java.tuple. {Tuple2 => JTuple2} // use Java tuple because it's mutable. class IntegralAvgAccumulator extends JTuple2 [Long, Long] with Accumulator ``` which means that Flink can handle it as a Tuple which is more efficient than a POJO which accesses fields (for de/serialization) via reflection. I think this implementation will also be closer to the code-gen'd version.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user shaoxuan-wang commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3354#discussion_r102606570

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/builtInAggFuncs/SumAggFunction.scala —
          @@ -0,0 +1,146 @@
          +/*
          + * 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.flink.table.functions.builtInAggFuncs
          +
          +import java.math.BigDecimal
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +/**
          + * Base class for built-in Sum aggregate function
          + *
          + * @tparam T the type for the aggregation result
          + */
          +abstract class SumAggFunction[T: Numeric] extends AggregateFunction[T] {
          + /** The initial accumulator for Sum aggregate function */
          + class SumAccumulator[T] extends Accumulator {
          + var sum: Option[T] = None
          — End diff –

          This was cloning from current existing built-in functions. Did not think it more thoroughly. Yes, you are right. I agree with you that we do not need the Option here.

          Show
          githubbot ASF GitHub Bot added a comment - Github user shaoxuan-wang commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102606570 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/builtInAggFuncs/SumAggFunction.scala — @@ -0,0 +1,146 @@ +/* + * 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.flink.table.functions.builtInAggFuncs + +import java.math.BigDecimal +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + +/** + * Base class for built-in Sum aggregate function + * + * @tparam T the type for the aggregation result + */ +abstract class SumAggFunction [T: Numeric] extends AggregateFunction [T] { + /** The initial accumulator for Sum aggregate function */ + class SumAccumulator [T] extends Accumulator { + var sum: Option [T] = None — End diff – This was cloning from current existing built-in functions. Did not think it more thoroughly. Yes, you are right. I agree with you that we do not need the Option here.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user shaoxuan-wang commented on the issue:

          https://github.com/apache/flink/pull/3354

          @fhueske thanks for the review. I agree with you that we should provide more specific types for the accumulators (backend state). I will update code today to address all your comments.

          Show
          githubbot ASF GitHub Bot added a comment - Github user shaoxuan-wang commented on the issue: https://github.com/apache/flink/pull/3354 @fhueske thanks for the review. I agree with you that we should provide more specific types for the accumulators (backend state). I will update code today to address all your comments.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102693258

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/MaxAggFunction.scala —
          @@ -28,8 +28,9 @@ import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}
          • @tparam T the type for the aggregation result
            */
            abstract class MaxAggFunction[T](implicit ord: Ordering[T]) extends AggregateFunction[T] {
            +
            /** The initial accumulator for Max aggregate function */
          • class MaxAccumulator[T] extends Accumulator {
            + class MaxAccumulator[T] extends JTuple2[T, Boolean] with Accumulator {
            var max: T = null.asInstanceOf[T]
              • End diff –

          Instead of adding a field here, you need to provide default values in the `createAccumulator` method. Note, `f0` and `f1` may not be null. Otherwise, the tuple cannot be serialized.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102693258 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/MaxAggFunction.scala — @@ -28,8 +28,9 @@ import org.apache.flink.table.functions. {Accumulator, AggregateFunction} @tparam T the type for the aggregation result */ abstract class MaxAggFunction [T] (implicit ord: Ordering [T] ) extends AggregateFunction [T] { + /** The initial accumulator for Max aggregate function */ class MaxAccumulator [T] extends Accumulator { + class MaxAccumulator [T] extends JTuple2 [T, Boolean] with Accumulator { var max: T = null.asInstanceOf [T] End diff – Instead of adding a field here, you need to provide default values in the `createAccumulator` method. Note, `f0` and `f1` may not be null. Otherwise, the tuple cannot be serialized.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102692834

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/AvgAggFunction.scala —
          @@ -28,8 +28,9 @@ import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}
          • @tparam T the type for the aggregation result
            */
            abstract class IntegralAvgAggFunction[T] extends AggregateFunction[T] {
            +
            /** The initial accumulator for Integral Avg aggregate function */
          • class IntegralAvgAccumulator extends Accumulator {
            + class IntegralAvgAccumulator extends JTuple2[Long, Long] with Accumulator {
              • End diff –

          If you are using a `JTuple` you may not add additional fields but should use the Tuple2 fields (`f0` and `f1`).
          Since the `TupleSerializer` is not aware of the added fields, `IntegralAvgAccumulator` won't be treated as Tuple2 by Flink.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102692834 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/AvgAggFunction.scala — @@ -28,8 +28,9 @@ import org.apache.flink.table.functions. {Accumulator, AggregateFunction} @tparam T the type for the aggregation result */ abstract class IntegralAvgAggFunction [T] extends AggregateFunction [T] { + /** The initial accumulator for Integral Avg aggregate function */ class IntegralAvgAccumulator extends Accumulator { + class IntegralAvgAccumulator extends JTuple2 [Long, Long] with Accumulator { End diff – If you are using a `JTuple` you may not add additional fields but should use the Tuple2 fields (`f0` and `f1`). Since the `TupleSerializer` is not aware of the added fields, `IntegralAvgAccumulator` won't be treated as Tuple2 by Flink.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102693561

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/functions/aggfunctions/AggFunctionTestBase.scala —
          @@ -56,11 +53,15 @@ abstract class AggFunctionTestBase[T] {

          if (ifMethodExitInFunction("merge", aggregator)) {
          // iterate over input sets

          • for((vals, expected) <- inputValueSets.zip(expectedResults)) {
            + for ((vals, expected) <- inputValueSets.zip(expectedResults)) {
            val (firstVals, secondVals) = vals.splitAt(vals.length / 2)
              • End diff –

          In addition to splitting the `vals` data set, we should also test correctness, if we merge with an empty accumulator, i.e., aggregate all `vals` into one accumulator and merge it with an empty one.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102693561 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/functions/aggfunctions/AggFunctionTestBase.scala — @@ -56,11 +53,15 @@ abstract class AggFunctionTestBase [T] { if (ifMethodExitInFunction("merge", aggregator)) { // iterate over input sets for((vals, expected) <- inputValueSets.zip(expectedResults)) { + for ((vals, expected) <- inputValueSets.zip(expectedResults)) { val (firstVals, secondVals) = vals.splitAt(vals.length / 2) End diff – In addition to splitting the `vals` data set, we should also test correctness, if we merge with an empty accumulator, i.e., aggregate all `vals` into one accumulator and merge it with an empty one.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102835493

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/SumAggFunction.scala —
          @@ -32,7 +31,8 @@ abstract class SumAggFunction[T: Numeric] extends AggregateFunction[T] {

          /** The initial accumulator for Sum aggregate function */
          class SumAccumulator[T] extends JTuple2[T, Boolean] with Accumulator {

          • var sum: T = null.asInstanceOf[T]
            + f0 = 0.asInstanceOf[T] //sum
              • End diff –

          change this to `f0 = numeric.zero`

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102835493 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/SumAggFunction.scala — @@ -32,7 +31,8 @@ abstract class SumAggFunction [T: Numeric] extends AggregateFunction [T] { /** The initial accumulator for Sum aggregate function */ class SumAccumulator [T] extends JTuple2 [T, Boolean] with Accumulator { var sum: T = null.asInstanceOf [T] + f0 = 0.asInstanceOf [T] //sum End diff – change this to `f0 = numeric.zero`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102835657

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/MinAggFunction.scala —
          @@ -31,7 +31,8 @@ abstract class MinAggFunction[T](implicit ord: Ordering[T]) extends AggregateFun

          /** The initial accumulator for Min aggregate function */
          class MinAccumulator[T] extends JTuple2[T, Boolean] with Accumulator {
          — End diff –

          We can remove the type `T`

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102835657 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/MinAggFunction.scala — @@ -31,7 +31,8 @@ abstract class MinAggFunction [T] (implicit ord: Ordering [T] ) extends AggregateFun /** The initial accumulator for Min aggregate function */ class MinAccumulator [T] extends JTuple2 [T, Boolean] with Accumulator { — End diff – We can remove the type `T`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102828658

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/SumAggFunction.scala —
          @@ -67,10 +68,13 @@ abstract class SumAggFunction[T: Numeric] extends AggregateFunction[T] {
          var i: Int = 0
          while (i < accumulators.size()) {
          val a = accumulators.get.asInstanceOf[SumAccumulator[T]]

          • if (ret.sum == null.asInstanceOf[T]) { - ret.sum = a.sum - }

            else if (a.sum != null.asInstanceOf[T]) {

          • ret.sum = numeric.plus(ret.sum, a.sum)
            + if (a.f1) {
            + if (!ret.f1) {
              • End diff –

          Since we start with `sum = 0` we don't need this condition.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102828658 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/SumAggFunction.scala — @@ -67,10 +68,13 @@ abstract class SumAggFunction [T: Numeric] extends AggregateFunction [T] { var i: Int = 0 while (i < accumulators.size()) { val a = accumulators.get .asInstanceOf[SumAccumulator [T] ] if (ret.sum == null.asInstanceOf [T] ) { - ret.sum = a.sum - } else if (a.sum != null.asInstanceOf [T] ) { ret.sum = numeric.plus(ret.sum, a.sum) + if (a.f1) { + if (!ret.f1) { End diff – Since we start with `sum = 0` we don't need this condition.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102835712

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/MaxAggFunction.scala —
          @@ -31,7 +31,8 @@ abstract class MaxAggFunction[T](implicit ord: Ordering[T]) extends AggregateFun

          /** The initial accumulator for Max aggregate function */
          class MaxAccumulator[T] extends JTuple2[T, Boolean] with Accumulator {
          — End diff –

          We can remove the type `T`

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102835712 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/MaxAggFunction.scala — @@ -31,7 +31,8 @@ abstract class MaxAggFunction [T] (implicit ord: Ordering [T] ) extends AggregateFun /** The initial accumulator for Max aggregate function */ class MaxAccumulator [T] extends JTuple2 [T, Boolean] with Accumulator { — End diff – We can remove the type `T`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102826611

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/MaxAggFunction.scala —
          @@ -109,8 +119,11 @@ class DecimalMaxAggFunction extends MaxAggFunction[BigDecimal] {
          if (value != null) {
          val v = value.asInstanceOf[BigDecimal]
          val accum = accumulator.asInstanceOf[MaxAccumulator[BigDecimal]]

          • if (accum.max == null || accum.max.compareTo(v) < 0) {
          • accum.max = v
            + if (!accum.f1 || accum.f0.compareTo(v) < 0) {
            + accum.f0 = v
            + if (!accum.f1) {
              • End diff –

          remove condition.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102826611 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/MaxAggFunction.scala — @@ -109,8 +119,11 @@ class DecimalMaxAggFunction extends MaxAggFunction [BigDecimal] { if (value != null) { val v = value.asInstanceOf [BigDecimal] val accum = accumulator.asInstanceOf[MaxAccumulator [BigDecimal] ] if (accum.max == null || accum.max.compareTo(v) < 0) { accum.max = v + if (!accum.f1 || accum.f0.compareTo(v) < 0) { + accum.f0 = v + if (!accum.f1) { End diff – remove condition.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102826652

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/MinAggFunction.scala —
          @@ -41,24 +42,33 @@ abstract class MinAggFunction[T](implicit ord: Ordering[T]) extends AggregateFun
          override def accumulate(accumulator: Accumulator, value: Any) = {
          if (value != null) {
          val v = value.asInstanceOf[T]

          • val accum = accumulator.asInstanceOf[MinAccumulator[T]]
          • if (accum.max == null || ord.compare(accum.max, v) > 0) {
          • accum.max = v
            + val a = accumulator.asInstanceOf[MinAccumulator[T]]
            + if (!a.f1 || ord.compare(a.f0, v) > 0) {
            + a.f0 = v
            + if (!a.f1) {
              • End diff –

          Remove condition

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102826652 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/MinAggFunction.scala — @@ -41,24 +42,33 @@ abstract class MinAggFunction [T] (implicit ord: Ordering [T] ) extends AggregateFun override def accumulate(accumulator: Accumulator, value: Any) = { if (value != null) { val v = value.asInstanceOf [T] val accum = accumulator.asInstanceOf[MinAccumulator [T] ] if (accum.max == null || ord.compare(accum.max, v) > 0) { accum.max = v + val a = accumulator.asInstanceOf[MinAccumulator [T] ] + if (!a.f1 || ord.compare(a.f0, v) > 0) { + a.f0 = v + if (!a.f1) { End diff – Remove condition
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102828543

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/SumAggFunction.scala —
          @@ -44,21 +44,22 @@ abstract class SumAggFunction[T: Numeric] extends AggregateFunction[T] {
          override def accumulate(accumulator: Accumulator, value: Any) = {
          if (value != null) {
          val v = value.asInstanceOf[T]

          • val accum = accumulator.asInstanceOf[SumAccumulator[T]]
          • if (accum.sum == null.asInstanceOf[T]) {
          • accum.sum = v
            + val a = accumulator.asInstanceOf[SumAccumulator[T]]
            + if (!a.f1) {
              • End diff –

          since we start with `sum = 0` we don't need this condition.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102828543 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/SumAggFunction.scala — @@ -44,21 +44,22 @@ abstract class SumAggFunction [T: Numeric] extends AggregateFunction [T] { override def accumulate(accumulator: Accumulator, value: Any) = { if (value != null) { val v = value.asInstanceOf [T] val accum = accumulator.asInstanceOf[SumAccumulator [T] ] if (accum.sum == null.asInstanceOf [T] ) { accum.sum = v + val a = accumulator.asInstanceOf[SumAccumulator [T] ] + if (!a.f1) { End diff – since we start with `sum = 0` we don't need this condition.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102835585

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/SumAggFunction.scala —
          @@ -32,7 +31,8 @@ abstract class SumAggFunction[T: Numeric] extends AggregateFunction[T] {

          /** The initial accumulator for Sum aggregate function */
          class SumAccumulator[T] extends JTuple2[T, Boolean] with Accumulator {
          — End diff –

          We can remove the type `T`. It will then be passed down from the parent class.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102835585 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/SumAggFunction.scala — @@ -32,7 +31,8 @@ abstract class SumAggFunction [T: Numeric] extends AggregateFunction [T] { /** The initial accumulator for Sum aggregate function */ class SumAccumulator [T] extends JTuple2 [T, Boolean] with Accumulator { — End diff – We can remove the type `T`. It will then be passed down from the parent class.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102828705

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/SumAggFunction.scala —
          @@ -127,30 +132,31 @@ class DecimalSumAggFunction extends AggregateFunction[BigDecimal] {
          if (value != null) {
          val v = value.asInstanceOf[BigDecimal]
          val accum = accumulator.asInstanceOf[DecimalSumAccumulator]

          • if (accum.sum == null) {
          • accum.sum = v
            + if (accum.f1 == false) {
              • End diff –

          remove condition.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102828705 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/SumAggFunction.scala — @@ -127,30 +132,31 @@ class DecimalSumAggFunction extends AggregateFunction [BigDecimal] { if (value != null) { val v = value.asInstanceOf [BigDecimal] val accum = accumulator.asInstanceOf [DecimalSumAccumulator] if (accum.sum == null) { accum.sum = v + if (accum.f1 == false) { End diff – remove condition.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3354#discussion_r102826382

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/MaxAggFunction.scala —
          @@ -41,24 +42,33 @@ abstract class MaxAggFunction[T](implicit ord: Ordering[T]) extends AggregateFun
          override def accumulate(accumulator: Accumulator, value: Any) = {
          if (value != null) {
          val v = value.asInstanceOf[T]

          • val accum = accumulator.asInstanceOf[MaxAccumulator[T]]
          • if (accum.max == null || ord.compare(accum.max, v) < 0) {
          • accum.max = v
            + val a = accumulator.asInstanceOf[MaxAccumulator[T]]
            + if (!a.f1 || ord.compare(a.f0, v) < 0) {
            + a.f0 = v
            + if (!a.f1) {
              • End diff –

          the condition can be removed. We can simply reassign `true`.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3354#discussion_r102826382 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/MaxAggFunction.scala — @@ -41,24 +42,33 @@ abstract class MaxAggFunction [T] (implicit ord: Ordering [T] ) extends AggregateFun override def accumulate(accumulator: Accumulator, value: Any) = { if (value != null) { val v = value.asInstanceOf [T] val accum = accumulator.asInstanceOf[MaxAccumulator [T] ] if (accum.max == null || ord.compare(accum.max, v) < 0) { accum.max = v + val a = accumulator.asInstanceOf[MaxAccumulator [T] ] + if (!a.f1 || ord.compare(a.f0, v) < 0) { + a.f0 = v + if (!a.f1) { End diff – the condition can be removed. We can simply reassign `true`.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

          https://github.com/apache/flink/pull/3354

          Show
          githubbot ASF GitHub Bot added a comment - Github user asfgit closed the pull request at: https://github.com/apache/flink/pull/3354
          Hide
          fhueske Fabian Hueske added a comment -

          Implemented with dd8ef550cf4c590c5a84ba313c57e202d4df94f4

          Show
          fhueske Fabian Hueske added a comment - Implemented with dd8ef550cf4c590c5a84ba313c57e202d4df94f4

            People

            • Assignee:
              ShaoxuanWang Shaoxuan Wang
              Reporter:
              ShaoxuanWang Shaoxuan Wang
            • Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development