Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 0CCD7200BA3 for ; Thu, 20 Oct 2016 16:32:00 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 0B817160AF5; Thu, 20 Oct 2016 14:32:00 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 59967160ADB for ; Thu, 20 Oct 2016 16:31:59 +0200 (CEST) Received: (qmail 5403 invoked by uid 500); 20 Oct 2016 14:31:58 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 5384 invoked by uid 99); 20 Oct 2016 14:31:58 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 20 Oct 2016 14:31:58 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id 64BF32C0D52 for ; Thu, 20 Oct 2016 14:31:58 +0000 (UTC) Date: Thu, 20 Oct 2016 14:31:58 +0000 (UTC) From: "Anton Mushin (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Comment Edited] (FLINK-4832) Count/Sum 0 elements MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Thu, 20 Oct 2016 14:32:00 -0000 [ https://issues.apache.org/jira/browse/FLINK-4832?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15591955#comment-15591955 ] Anton Mushin edited comment on FLINK-4832 at 10/20/16 2:31 PM: --------------------------------------------------------------- Hello I think that it needs to change {{org.apache.flink.api.common.operators.base.MapOperatorBase#executeOnCollections}} also, because {{org.apache.flink.api.table.runtime.aggregate.AggregateMapFunction#map}} will be called if elements are in inputData. {code} TypeSerializer inSerializer = getOperatorInfo().getInputType().createSerializer(executionConfig); TypeSerializer outSerializer = getOperatorInfo().getOutputType().createSerializer(executionConfig); for (IN element : inputData) { IN inCopy = inSerializer.copy(element); OUT out = function.map(inCopy); result.add(outSerializer.copy(out)); } {code} And if {{org.apache.flink.api.table.runtime.aggregate.SumAggregate#initiate}} will be edited for examle as {code} override def initiate(partial: Row): Unit = { partial.setField(sumIndex, 0.asInstanceOf[T]) //cast 0 to type for sum class is extends SumAggregate[T] } {code} then next test will be passed {code} @Test def testSumNullElements(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) val sqlQuery = "SELECT sum(_1),sum(_2),sum(_3),sum(_4),sum(_5),sum(_6) " + "FROM (select * from MyTable where _1 = 4)" val ds = env.fromElements( (1: Byte, 2l,1D,1f,1,1:Short ), (2: Byte, 2l,1D,1f,1,1:Short )) tEnv.registerDataSet("MyTable", ds) val result = tEnv.sql(sqlQuery) val expected = "0,0,0.0,0.0,0,0" val results = result.toDataSet[Row].collect() TestBaseUtils.compareResultAsText(results.asJava, expected) } @Test def testCountNullElements(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) val sqlQuery = "SELECT count(_1),count(_2),count(_3),count(_4),count(_5),count(_6) " + "FROM (select * from MyTable where _1 = 4)" val ds = env.fromElements( (1: Byte, 2l,1D,1f,1,1:Short ), (2: Byte, 2l,1D,1f,1,1:Short )) tEnv.registerDataSet("MyTable", ds) val result = tEnv.sql(sqlQuery) val expected = "0,0,0,0,0,0" val results = result.toDataSet[Row].collect() TestBaseUtils.compareResultAsText(results.asJava, expected) } {code} was (Author: anmu): Hello I think that it needs to change {{org.apache.flink.api.common.operators.base.MapOperatorBase#executeOnCollections}} also, because {{org.apache.flink.api.table.runtime.aggregate.AggregateMapFunction#map}} will be called if elements are in inputData. {code} TypeSerializer inSerializer = getOperatorInfo().getInputType().createSerializer(executionConfig); TypeSerializer outSerializer = getOperatorInfo().getOutputType().createSerializer(executionConfig); for (IN element : inputData) { IN inCopy = inSerializer.copy(element); OUT out = function.map(inCopy); result.add(outSerializer.copy(out)); } {code} And if {{org.apache.flink.api.table.runtime.aggregate.SumAggregate#initiate}} will be edited for examle as {code} override def initiate(partial: Row): Unit = { partial.setField(sumIndex, 0.asInstanceOf[T]) //cast 0 to type for sum class is extends SumAggregate[T] } {code} then next test will be passed {code} @Test def testDataSetAggregation(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) val sqlQuery = "SELECT sum(_1) FROM MyTable" val ds = CollectionDataSets.get3TupleDataSet(env) tEnv.registerDataSet("MyTable", ds) val result = tEnv.sql(sqlQuery) val expected = "231" val results = result.toDataSet[Row].collect() TestBaseUtils.compareResultAsText(results.asJava, expected) } @Test def testSumNullElements(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) val sqlQuery = "SELECT sum(_1),sum(_2),sum(_3),sum(_4),sum(_5),sum(_6) " + "FROM (select * from MyTable where _1 = 4)" val ds = env.fromElements( (1: Byte, 2l,1D,1f,1,1:Short ), (2: Byte, 2l,1D,1f,1,1:Short )) tEnv.registerDataSet("MyTable", ds) val result = tEnv.sql(sqlQuery) val expected = "0,0,0.0,0.0,0,0" val results = result.toDataSet[Row].collect() TestBaseUtils.compareResultAsText(results.asJava, expected) } {code} > Count/Sum 0 elements > -------------------- > > Key: FLINK-4832 > URL: https://issues.apache.org/jira/browse/FLINK-4832 > Project: Flink > Issue Type: Improvement > Components: Table API & SQL > Reporter: Timo Walther > > Currently, the Table API is unable to count or sum up 0 elements. We should improve DataSet aggregations for this. Maybe by union the original DataSet with a dummy record or by using a MapPartition function. Coming up with a good design for this is also part of this issue. -- This message was sent by Atlassian JIRA (v6.3.4#6332)